From 3f9e9a7025bec607bb1e646556c415e3336c796d Mon Sep 17 00:00:00 2001 From: Kirill Ershov Date: Fri, 16 Apr 2021 23:18:39 +0300 Subject: [PATCH 001/599] Add INTERSECT and EXCEPT --- src/Common/ErrorCodes.cpp | 6 +- src/Interpreters/InterpreterFactory.cpp | 18 +- .../InterpreterIntersectOrExcept.cpp | 116 +++++++++++ .../InterpreterIntersectOrExcept.h | 35 ++++ src/Parsers/ASTIntersectOrExcept.cpp | 28 +++ src/Parsers/ASTIntersectOrExcept.h | 18 ++ src/Parsers/ParserIntersectOrExcept.cpp | 50 +++++ src/Parsers/ParserIntersectOrExcept.h | 14 ++ src/Parsers/ParserQueryWithOutput.cpp | 36 ++-- .../QueryPlan/IntersectOrExceptStep.cpp | 38 ++++ .../QueryPlan/IntersectOrExceptStep.h | 26 +++ .../Transforms/IntersectOrExceptTransform.cpp | 192 ++++++++++++++++++ .../Transforms/IntersectOrExceptTransform.h | 53 +++++ 13 files changed, 606 insertions(+), 24 deletions(-) create mode 100644 src/Interpreters/InterpreterIntersectOrExcept.cpp create mode 100644 src/Interpreters/InterpreterIntersectOrExcept.h create mode 100644 src/Parsers/ASTIntersectOrExcept.cpp create mode 100644 src/Parsers/ASTIntersectOrExcept.h create mode 100644 src/Parsers/ParserIntersectOrExcept.cpp create mode 100644 src/Parsers/ParserIntersectOrExcept.h create mode 100644 src/Processors/QueryPlan/IntersectOrExceptStep.cpp create mode 100644 src/Processors/QueryPlan/IntersectOrExceptStep.h create mode 100644 src/Processors/Transforms/IntersectOrExceptTransform.cpp create mode 100644 src/Processors/Transforms/IntersectOrExceptTransform.h diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index a2cd65137c0..0d1fd5bd7d8 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -533,7 +533,11 @@ M(564, INTERSERVER_SCHEME_DOESNT_MATCH) \ M(565, TOO_MANY_PARTITIONS) \ M(566, CANNOT_RMDIR) \ - \ + M(567, DUPLICATED_PART_UUIDS) \ + M(568, RAFT_ERROR) \ + M(569, MULTIPLE_COLUMNS_SERIALIZED_TO_SAME_PROTOBUF_FIELD) \ + M(570, DATA_TYPE_INCOMPATIBLE_WITH_PROTOBUF_FIELD) \ + M(571, INTERSECT_OR_EXCEPT_RESULT_STRUCTURES_MISMATCH) \ M(999, KEEPER_EXCEPTION) \ M(1000, POCO_EXCEPTION) \ M(1001, STD_EXCEPTION) \ diff --git a/src/Interpreters/InterpreterFactory.cpp b/src/Interpreters/InterpreterFactory.cpp index 15e4c52f040..e0f6479cc0e 100644 --- a/src/Interpreters/InterpreterFactory.cpp +++ b/src/Interpreters/InterpreterFactory.cpp @@ -1,14 +1,17 @@ #include #include #include -#include -#include #include +#include #include #include +#include #include #include +#include +#include #include +#include #include #include #include @@ -24,11 +27,9 @@ #include #include #include -#include -#include #include -#include #include +#include #include #include @@ -44,9 +45,11 @@ #include #include #include +#include #include #include #include +#include #include #include #include @@ -65,7 +68,6 @@ #include #include #include -#include #include #include @@ -109,6 +111,10 @@ std::unique_ptr InterpreterFactory::get(ASTPtr & query, Context & ProfileEvents::increment(ProfileEvents::SelectQuery); return std::make_unique(query, context, options); } + else if (query->as()) + { + return std::make_unique(query, context); + } else if (query->as()) { ProfileEvents::increment(ProfileEvents::InsertQuery); diff --git a/src/Interpreters/InterpreterIntersectOrExcept.cpp b/src/Interpreters/InterpreterIntersectOrExcept.cpp new file mode 100644 index 00000000000..c85bd29e16f --- /dev/null +++ b/src/Interpreters/InterpreterIntersectOrExcept.cpp @@ -0,0 +1,116 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int INTERSECT_OR_EXCEPT_RESULT_STRUCTURES_MISMATCH; +} + +InterpreterIntersectOrExcept::InterpreterIntersectOrExcept(const ASTPtr & query_ptr_, ContextPtr context_) + : query_ptr(query_ptr_), context(Context::createCopy(context_)) +{ + ASTIntersectOrExcept * ast = query_ptr->as(); + size_t num_children = ast->children.size(); + nested_interpreters.resize(num_children); + for (size_t i = 0; i < num_children; ++i) + { + nested_interpreters[i] = buildCurrentChildInterpreter(ast->children[i]); + } + + Blocks headers(num_children); + for (size_t query_num = 0; query_num < num_children; ++query_num) + headers[query_num] = nested_interpreters[query_num]->getSampleBlock(); + + result_header = getCommonHeader(headers); +} + + +Block InterpreterIntersectOrExcept::getCommonHeader(const Blocks & headers) +{ + size_t num_selects = headers.size(); + Block common_header = headers.front(); + size_t num_columns = common_header.columns(); + + for (size_t query_num = 1; query_num < num_selects; ++query_num) + { + if (headers[query_num].columns() != num_columns) + throw Exception( + "Different number of columns in " + + toString(query_ptr->as()->is_except ? "EXCEPT" : "INTERSECT") + + " elements:\n" + common_header.dumpNames() + "\nand\n" + + headers[query_num].dumpNames() + "\n", + ErrorCodes::INTERSECT_OR_EXCEPT_RESULT_STRUCTURES_MISMATCH); + } + + std::vector columns(num_selects); + + for (size_t column_num = 0; column_num < num_columns; ++column_num) + { + for (size_t i = 0; i < num_selects; ++i) + columns[i] = &headers[i].getByPosition(column_num); + + ColumnWithTypeAndName & result_elem = common_header.getByPosition(column_num); + result_elem = getLeastSuperColumn(columns); + } + + return common_header; +} + + +std::unique_ptr +InterpreterIntersectOrExcept::buildCurrentChildInterpreter(const ASTPtr & ast_ptr_) +{ + if (ast_ptr_->as()) + return std::make_unique(ast_ptr_, context, SelectQueryOptions()); + else + return std::make_unique(ast_ptr_, context, SelectQueryOptions()); +} + +void InterpreterIntersectOrExcept::buildQueryPlan(QueryPlan & query_plan) +{ + size_t num_plans = nested_interpreters.size(); + + std::vector> plans(num_plans); + DataStreams data_streams(num_plans); + + for (size_t i = 0; i < num_plans; ++i) + { + plans[i] = std::make_unique(); + nested_interpreters[i]->buildQueryPlan(*plans[i]); + data_streams[i] = plans[i]->getCurrentDataStream(); + } + + auto max_threads = context->getSettingsRef().max_threads; + auto step = std::make_unique( + query_ptr->as()->is_except, std::move(data_streams), result_header, max_threads); + query_plan.unitePlans(std::move(step), std::move(plans)); +} + +BlockIO InterpreterIntersectOrExcept::execute() +{ + BlockIO res; + + QueryPlan query_plan; + buildQueryPlan(query_plan); + + auto pipeline = query_plan.buildQueryPipeline( + QueryPlanOptimizationSettings::fromContext(context), + BuildQueryPipelineSettings::fromContext(context)); + + res.pipeline = std::move(*pipeline); + res.pipeline.addInterpreterContext(context); + + return res; +} +} diff --git a/src/Interpreters/InterpreterIntersectOrExcept.h b/src/Interpreters/InterpreterIntersectOrExcept.h new file mode 100644 index 00000000000..0069dc02f1d --- /dev/null +++ b/src/Interpreters/InterpreterIntersectOrExcept.h @@ -0,0 +1,35 @@ +#pragma once + +#include +#include +#include + +namespace DB +{ + +class Context; +class InterpreterSelectQuery; +class QueryPlan; + +class InterpreterIntersectOrExcept : public IInterpreter +{ +public: + InterpreterIntersectOrExcept(const ASTPtr & query_ptr_, ContextPtr context_); + + /// Builds QueryPlan for current query. + virtual void buildQueryPlan(QueryPlan & query_plan); + + BlockIO execute() override; + +private: + ASTPtr query_ptr; + ContextPtr context; + Block result_header; + std::vector> nested_interpreters; + Block getCommonHeader(const Blocks & headers); + + std::unique_ptr + buildCurrentChildInterpreter(const ASTPtr & ast_ptr_); +}; + +} diff --git a/src/Parsers/ASTIntersectOrExcept.cpp b/src/Parsers/ASTIntersectOrExcept.cpp new file mode 100644 index 00000000000..073d63963a9 --- /dev/null +++ b/src/Parsers/ASTIntersectOrExcept.cpp @@ -0,0 +1,28 @@ +#include +#include + +namespace DB +{ + +ASTPtr ASTIntersectOrExcept::clone() const +{ + auto res = std::make_shared(*this); + res->children.clear(); + res->children.push_back(children[0]->clone()); + res->children.push_back(children[1]->clone()); + res->is_except = is_except; + cloneOutputOptions(*res); + return res; +} + +void ASTIntersectOrExcept::formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const +{ + children[0]->formatImpl(settings, state, frame); + std::string indent_str = settings.one_line ? "" : std::string(4 * frame.indent, ' '); + settings.ostr << settings.nl_or_ws << indent_str << (settings.hilite ? hilite_keyword : "") + << (is_except ? "EXCEPT" : "INTERSECT ") + << (settings.hilite ? hilite_none : "") << settings.nl_or_ws; + children[1]->formatImpl(settings, state, frame); +} + +} diff --git a/src/Parsers/ASTIntersectOrExcept.h b/src/Parsers/ASTIntersectOrExcept.h new file mode 100644 index 00000000000..a02cb9f7d77 --- /dev/null +++ b/src/Parsers/ASTIntersectOrExcept.h @@ -0,0 +1,18 @@ +#pragma once + +#include + + +namespace DB +{ + +class ASTIntersectOrExcept : public ASTQueryWithOutput +{ +public: + String getID(char) const override { return is_except ? "Except" : "Intersect"; } + ASTPtr clone() const override; + void formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override; + bool is_except; +}; + +} diff --git a/src/Parsers/ParserIntersectOrExcept.cpp b/src/Parsers/ParserIntersectOrExcept.cpp new file mode 100644 index 00000000000..a82b8c2b06b --- /dev/null +++ b/src/Parsers/ParserIntersectOrExcept.cpp @@ -0,0 +1,50 @@ +#include +#include +#include +#include +#include +#include + +namespace DB +{ +bool ParserIntersectOrExcept::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) +{ + ParserKeyword intersect_keyword("INTERSECT"); + ParserKeyword except_keyword("EXCEPT"); + ASTPtr left_node; + ASTPtr right_node; + + auto ast = std::make_shared(); + ast->is_except = false; + + if (!ParserSelectQuery().parse(pos, left_node, expected) && !ParserSubquery().parse(pos, left_node, expected)) + return false; + + if (!intersect_keyword.ignore(pos)) + { + if (!except_keyword.ignore(pos)) + { + return false; + } + else + { + ast->is_except = true; + } + } + + if (!ParserSelectQuery().parse(pos, right_node, expected) && !ParserSubquery().parse(pos, right_node, expected)) + return false; + + if (const auto * ast_subquery = left_node->as()) + left_node = ast_subquery->children.at(0); + if (const auto * ast_subquery = right_node->as()) + right_node = ast_subquery->children.at(0); + + ast->children.push_back(left_node); + ast->children.push_back(right_node); + + node = ast; + return true; +} + +} diff --git a/src/Parsers/ParserIntersectOrExcept.h b/src/Parsers/ParserIntersectOrExcept.h new file mode 100644 index 00000000000..61cc74cf0a9 --- /dev/null +++ b/src/Parsers/ParserIntersectOrExcept.h @@ -0,0 +1,14 @@ +#pragma once +#include + + +namespace DB +{ +class ParserIntersectOrExcept : public IParserBase +{ +protected: + const char * getName() const override { return "INTERSECT or EXCEPT"; } + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; +}; + +} diff --git a/src/Parsers/ParserQueryWithOutput.cpp b/src/Parsers/ParserQueryWithOutput.cpp index d5aa1e47533..35355b29ebf 100644 --- a/src/Parsers/ParserQueryWithOutput.cpp +++ b/src/Parsers/ParserQueryWithOutput.cpp @@ -1,36 +1,37 @@ -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include #include #include #include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include #include #include #include #include #include -#include +#include +#include +#include +#include #include - namespace DB { bool ParserQueryWithOutput::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { ParserShowTablesQuery show_tables_p; + ParserIntersectOrExcept intersect_p; ParserSelectWithUnionQuery select_p; ParserTablePropertiesQuery table_p; ParserDescribeTableQuery describe_table_p; @@ -54,6 +55,7 @@ bool ParserQueryWithOutput::parseImpl(Pos & pos, ASTPtr & node, Expected & expec bool parsed = explain_p.parse(pos, query, expected) + || intersect_p.parse(pos, query, expected) || 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) diff --git a/src/Processors/QueryPlan/IntersectOrExceptStep.cpp b/src/Processors/QueryPlan/IntersectOrExceptStep.cpp new file mode 100644 index 00000000000..d0a820339d7 --- /dev/null +++ b/src/Processors/QueryPlan/IntersectOrExceptStep.cpp @@ -0,0 +1,38 @@ +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +IntersectOrExceptStep::IntersectOrExceptStep(bool is_except_, DataStreams input_streams_, Block result_header, size_t max_threads_) + : is_except(is_except_), header(std::move(result_header)), max_threads(max_threads_) +{ + input_streams = std::move(input_streams_); + output_stream = DataStream{.header = header}; +} + +QueryPipelinePtr IntersectOrExceptStep::updatePipeline(QueryPipelines pipelines, const BuildQueryPipelineSettings & ) +{ + auto pipeline = std::make_unique(); + QueryPipelineProcessorsCollector collector(*pipeline, this); + + pipelines[0]->addTransform(std::make_shared(header, pipelines[0]->getNumStreams(), 1)); + pipelines[1]->addTransform(std::make_shared(header, pipelines[1]->getNumStreams(), 1)); + + *pipeline = QueryPipeline::unitePipelines(std::move(pipelines), max_threads); + pipeline->addTransform(std::make_shared(is_except, header)); + + processors = collector.detachProcessors(); + return pipeline; +} + +void IntersectOrExceptStep::describePipeline(FormatSettings & settings) const +{ + IQueryPlanStep::describePipeline(processors, settings); +} + +} diff --git a/src/Processors/QueryPlan/IntersectOrExceptStep.h b/src/Processors/QueryPlan/IntersectOrExceptStep.h new file mode 100644 index 00000000000..d2b515bb1c4 --- /dev/null +++ b/src/Processors/QueryPlan/IntersectOrExceptStep.h @@ -0,0 +1,26 @@ +#pragma once +#include + +namespace DB +{ + +class IntersectOrExceptStep : public IQueryPlanStep +{ +public: + /// max_threads is used to limit the number of threads for result pipeline. + IntersectOrExceptStep(bool is_except_, DataStreams input_streams_, Block result_header, size_t max_threads_ = 0); + + String getName() const override { return is_except ? "Except" : "Intersect"; } + + QueryPipelinePtr updatePipeline(QueryPipelines pipelines, const BuildQueryPipelineSettings & settings) override; + + void describePipeline(FormatSettings & settings) const override; +private: + bool is_except; + Block header; + size_t max_threads; + Processors processors; +}; + +} + diff --git a/src/Processors/Transforms/IntersectOrExceptTransform.cpp b/src/Processors/Transforms/IntersectOrExceptTransform.cpp new file mode 100644 index 00000000000..199498bf762 --- /dev/null +++ b/src/Processors/Transforms/IntersectOrExceptTransform.cpp @@ -0,0 +1,192 @@ +#include + +namespace DB +{ +IntersectOrExceptTransform::IntersectOrExceptTransform(bool is_except_, const Block & header_) + : IProcessor(InputPorts(2, header_), {header_}), is_except(is_except_), output(outputs.front()) +{ + const Names & columns = header_.getNames(); + size_t num_columns = columns.empty() ? header_.columns() : columns.size(); + + key_columns_pos.reserve(columns.size()); + for (size_t i = 0; i < num_columns; ++i) + { + auto pos = columns.empty() ? i : header_.getPositionByName(columns[i]); + + const auto & col = header_.getByPosition(pos).column; + + if (!(col && isColumnConst(*col))) + key_columns_pos.emplace_back(pos); + } +} + +IntersectOrExceptTransform::Status IntersectOrExceptTransform::prepare() +{ + /// Check can output. + if (output.isFinished()) + { + for (auto & in : inputs) + in.close(); + return Status::Finished; + } + + if (!output.canPush()) + { + if (inputs.front().isFinished()) + { + inputs.back().setNotNeeded(); + } + else + { + inputs.front().setNotNeeded(); + } + return Status::PortFull; + } + + /// Output if has data. + if (current_output_chunk) + { + output.push(std::move(current_output_chunk)); + } + + if (push_empty_chunk) + { + output.push(std::move(empty_chunk)); + push_empty_chunk = false; + } + + if (finished_second_input) + { + if (inputs.front().isFinished()) + { + output.finish(); + return Status::Finished; + } + } + else if (inputs.back().isFinished()) + { + finished_second_input = true; + } + + InputPort & input = finished_second_input ? inputs.front() : inputs.back(); + + /// Check can input. + if (!has_input) + { + input.setNeeded(); + if (!input.hasData()) + { + return Status::NeedData; + } + + current_input_chunk = input.pull(); + has_input = true; + } + + return Status::Ready; +} + +void IntersectOrExceptTransform::work() +{ + if (!finished_second_input) + { + accumulate(std::move(current_input_chunk)); + } + else + { + filter(current_input_chunk); + current_output_chunk = std::move(current_input_chunk); + } + + has_input = false; +} + +template +void IntersectOrExceptTransform::addToSet(Method & method, const ColumnRawPtrs & columns, size_t rows, SetVariants & variants) const +{ + typename Method::State state(columns, key_sizes, nullptr); + + for (size_t i = 0; i < rows; ++i) + { + state.emplaceKey(method.data, i, variants.string_pool); + } +} + +template +size_t IntersectOrExceptTransform::buildFilter( + Method & method, const ColumnRawPtrs & columns, IColumn::Filter & filter, size_t rows, SetVariants & variants) const +{ + typename Method::State state(columns, key_sizes, nullptr); + size_t new_rows_num = 0; + + for (size_t i = 0; i < rows; ++i) + { + auto find_result = state.findKey(method.data, i, variants.string_pool); + filter[i] = is_except ? !find_result.isFound() : find_result.isFound(); + if (filter[i]) + ++new_rows_num; + } + return new_rows_num; +} + +void IntersectOrExceptTransform::accumulate(Chunk chunk) +{ + auto num_rows = chunk.getNumRows(); + auto columns = chunk.detachColumns(); + + ColumnRawPtrs column_ptrs; + column_ptrs.reserve(key_columns_pos.size()); + for (auto pos : key_columns_pos) + column_ptrs.emplace_back(columns[pos].get()); + + if (data.empty()) + data.init(SetVariants::chooseMethod(column_ptrs, key_sizes)); + + switch (data.type) + { + case SetVariants::Type::EMPTY: + break; +#define M(NAME) \ + case SetVariants::Type::NAME: \ + addToSet(*data.NAME, column_ptrs, num_rows, data); \ + break; + APPLY_FOR_SET_VARIANTS(M) +#undef M + } +} + +void IntersectOrExceptTransform::filter(Chunk & chunk) +{ + auto num_rows = chunk.getNumRows(); + auto columns = chunk.detachColumns(); + + ColumnRawPtrs column_ptrs; + column_ptrs.reserve(key_columns_pos.size()); + for (auto pos : key_columns_pos) + column_ptrs.emplace_back(columns[pos].get()); + + if (data.empty()) + data.init(SetVariants::chooseMethod(column_ptrs, key_sizes)); + + IColumn::Filter filter(num_rows); + + size_t new_rows_num = 0; + switch (data.type) + { + case SetVariants::Type::EMPTY: + break; +#define M(NAME) \ + case SetVariants::Type::NAME: \ + new_rows_num = buildFilter(*data.NAME, column_ptrs, filter, num_rows, data); \ + break; + APPLY_FOR_SET_VARIANTS(M) +#undef M + } + + for (auto & column : columns) + column = column->filter(filter, -1); + + chunk.setColumns(std::move(columns), new_rows_num); +} + +} diff --git a/src/Processors/Transforms/IntersectOrExceptTransform.h b/src/Processors/Transforms/IntersectOrExceptTransform.h new file mode 100644 index 00000000000..ebe73fdeb26 --- /dev/null +++ b/src/Processors/Transforms/IntersectOrExceptTransform.h @@ -0,0 +1,53 @@ +#pragma once + +#include +#include +#include + +namespace DB +{ + +class IntersectOrExceptTransform : public IProcessor +{ +public: + IntersectOrExceptTransform(bool is_except_, const Block & header_); + + Status prepare() override; + void work() override; + + String getName() const override { return is_except ? "Except" : "Intersect"; } + +private: + + bool push_empty_chunk = false; + Chunk empty_chunk; + + bool is_except; + ColumnNumbers key_columns_pos; + SetVariants data; + Sizes key_sizes; + Chunk current_input_chunk; + Chunk current_output_chunk; + bool finished_second_input = false; + bool has_input = false; + OutputPort & output; + + void accumulate(Chunk chunk); + void filter(Chunk & chunk); + template + void addToSet( + Method & method, + const ColumnRawPtrs & key_columns, + size_t rows, + SetVariants & variants) const; + + template + size_t buildFilter( + Method & method, + const ColumnRawPtrs & columns, + IColumn::Filter & filter, + size_t rows, + SetVariants & variants) const; +}; + +} From 566b5335ddc505d66ad1fa4db95497751f692cd0 Mon Sep 17 00:00:00 2001 From: Kirill Ershov Date: Sun, 30 May 2021 20:58:08 +0300 Subject: [PATCH 002/599] Add ANY, ALL --- src/Parsers/ExpressionListParsers.cpp | 129 +++++++++++++++++++++++++- src/Parsers/ExpressionListParsers.h | 16 +++- 2 files changed, 141 insertions(+), 4 deletions(-) diff --git a/src/Parsers/ExpressionListParsers.cpp b/src/Parsers/ExpressionListParsers.cpp index eec79edc05e..e9705843b6e 100644 --- a/src/Parsers/ExpressionListParsers.cpp +++ b/src/Parsers/ExpressionListParsers.cpp @@ -9,8 +9,7 @@ namespace DB -{ - +{ const char * ParserMultiplicativeExpression::operators[] = { @@ -56,6 +55,19 @@ const char * ParserComparisonExpression::operators[] = nullptr }; +const char * ParserComparisonWithSubqueryExpression::operators[] = +{ + "==", "equals", + "!=", "notEquals", + "<>", "notEquals", + "<=", "lessOrEquals", + ">=", "greaterOrEquals", + "<", "less", + ">", "greater", + "=", "equals", + nullptr +}; + const char * ParserComparisonExpression::overlapping_operators_to_skip[] = { "IN PARTITION", @@ -359,6 +371,119 @@ bool ParserBetweenExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & exp return true; } +bool ParserComparisonWithSubqueryExpression::modifySubquery(String operator_name, ASTPtr subquery_node, bool is_any) +{ + ASTPtr select_with_union_node = subquery_node->children[0]; + if (select_with_union_node->children[0]->children.size() != 1) + return false; + ASTPtr select_node = select_with_union_node->children[0]->children[0]; + ASTPtr exp_list = select_node->children[0]; + auto function = std::make_shared(); + function->arguments = exp_list; + function->children.push_back(exp_list); + + ASTPtr new_exp_list = std::make_shared(); + new_exp_list->children.push_back(function); + + if (operator_name == "greaterOrEquals" || operator_name == "greater") + { + function->name = is_any ? "min" : "max"; + select_node->children[0] = new_exp_list; + return true; + } + + if (operator_name == "lessOrEquals" || operator_name == "less") + { + function->name = is_any ? "max" : "min"; + select_node->children[0] = new_exp_list; + return true; + } + return false; +} + +bool ParserComparisonWithSubqueryExpression::addFunctionIn(String operator_name, ASTPtr & node, bool is_any) +{ + + auto function_in = std::make_shared(); + auto exp_list_in = std::make_shared(); + exp_list_in->children.push_back(node->children[0]->children[0]); + exp_list_in->children.push_back(node->children[0]->children[1]); + function_in->name = "in"; + function_in->children.push_back(exp_list_in); + function_in->arguments = exp_list_in; + + if (operator_name == "equals" && is_any) + { + node = function_in; + return true; + } + + if (operator_name == "notEquals" && !is_any) + { + auto function_not = std::make_shared(); + auto exp_list_not = std::make_shared(); + exp_list_not->children.push_back(function_in); + function_not->name = "not"; + function_not->children.push_back(exp_list_not); + function_not->arguments = exp_list_not; + node = function_not; + return true; + } + return false; +} + +bool ParserComparisonWithSubqueryExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) +{ + Pos begin = pos; + ASTPtr elem; + if (!elem_parser.parse(pos, elem, expected)) + return next_parser.parse(pos, node, expected); + + /// try to find any of the valid operators + const char ** it; + for (it = operators; *it; it += 2) + if (parseOperator(pos, *it, expected)) + break; + + if (!*it) + { + pos = begin; + return next_parser.parse(pos, node, expected); + } + + bool is_any = true; + if (!ParserKeyword("ANY").ignore(pos, expected)) + { + is_any = false; + if (!ParserKeyword("ALL").ignore(pos, expected)) + { + pos = begin; + return next_parser.parse(pos, node, expected); + } + } + + ASTPtr subquery_node; + if (!ParserSubquery().parse(pos, subquery_node, expected)) + return false; + + /// the first argument of the function is the previous element, the second is the next one + String operator_name = it[1]; + + /// the function corresponding to the operator + auto function = std::make_shared(); + + /// function arguments + auto exp_list = std::make_shared(); + exp_list->children.push_back(elem); + exp_list->children.push_back(subquery_node); + + function->name = operator_name; + function->arguments = exp_list; + function->children.push_back(exp_list); + node = function; + return modifySubquery(operator_name, subquery_node, is_any) || addFunctionIn(operator_name, node, is_any); +} + bool ParserTernaryOperatorExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { ParserToken symbol1(TokenType::QuestionMark); diff --git a/src/Parsers/ExpressionListParsers.h b/src/Parsers/ExpressionListParsers.h index 3a65141d751..acdeac29543 100644 --- a/src/Parsers/ExpressionListParsers.h +++ b/src/Parsers/ExpressionListParsers.h @@ -364,14 +364,26 @@ protected: } }; +class ParserComparisonWithSubqueryExpression : public IParserBase +{ +private: + static const char * operators[]; + ParserComparisonExpression next_parser; + ParserConcatExpression elem_parser; + static bool addFunctionIn(String operator_name, ASTPtr & node, bool is_any); + static bool modifySubquery(String operator_name, ASTPtr subquery_node, bool is_any); +protected: + const char * getName() const override { return "comparison with ANY/ALL expression"; } + + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; +}; /** Parser for nullity checking with IS (NOT) NULL. */ class ParserNullityChecking : public IParserBase { private: - ParserComparisonExpression elem_parser; - + ParserComparisonWithSubqueryExpression elem_parser; protected: const char * getName() const override { return "nullity checking"; } bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; From 1c2e9ba0038e5c18de10530e9f591833b5e0bd62 Mon Sep 17 00:00:00 2001 From: Kirill Ershov Date: Wed, 2 Jun 2021 02:18:15 +0300 Subject: [PATCH 003/599] Fix bug in parser --- src/Parsers/ExpressionListParsers.cpp | 6 +++--- src/Parsers/ExpressionListParsers.h | 2 +- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Parsers/ExpressionListParsers.cpp b/src/Parsers/ExpressionListParsers.cpp index e9705843b6e..a228dc6617b 100644 --- a/src/Parsers/ExpressionListParsers.cpp +++ b/src/Parsers/ExpressionListParsers.cpp @@ -9,7 +9,7 @@ namespace DB -{ +{ const char * ParserMultiplicativeExpression::operators[] = { @@ -447,8 +447,8 @@ bool ParserComparisonWithSubqueryExpression::parseImpl(Pos & pos, ASTPtr & node, if (!*it) { - pos = begin; - return next_parser.parse(pos, node, expected); + node = elem; + return true; } bool is_any = true; diff --git a/src/Parsers/ExpressionListParsers.h b/src/Parsers/ExpressionListParsers.h index acdeac29543..29335e72b2a 100644 --- a/src/Parsers/ExpressionListParsers.h +++ b/src/Parsers/ExpressionListParsers.h @@ -369,7 +369,7 @@ class ParserComparisonWithSubqueryExpression : public IParserBase private: static const char * operators[]; ParserComparisonExpression next_parser; - ParserConcatExpression elem_parser; + ParserBetweenExpression elem_parser; static bool addFunctionIn(String operator_name, ASTPtr & node, bool is_any); static bool modifySubquery(String operator_name, ASTPtr subquery_node, bool is_any); protected: From 289c5d3ad680c6e07ef189eed9ea30417acdac65 Mon Sep 17 00:00:00 2001 From: Kirill Ershov Date: Fri, 4 Jun 2021 05:56:02 +0300 Subject: [PATCH 004/599] Add SingleValueOrNull aggregate function --- .../AggregateFunctionMinMaxAny.h | 58 ++++++++++++++++++- .../AggregateFunctionSingleValueOrNull.cpp | 27 +++++++++ .../registerAggregateFunctions.cpp | 2 + 3 files changed, 86 insertions(+), 1 deletion(-) create mode 100644 src/AggregateFunctions/AggregateFunctionSingleValueOrNull.cpp diff --git a/src/AggregateFunctions/AggregateFunctionMinMaxAny.h b/src/AggregateFunctions/AggregateFunctionMinMaxAny.h index de1f4fad296..6791b98e7e3 100644 --- a/src/AggregateFunctions/AggregateFunctionMinMaxAny.h +++ b/src/AggregateFunctions/AggregateFunctionMinMaxAny.h @@ -5,8 +5,10 @@ #include #include +#include #include #include +#include #include #include @@ -628,6 +630,60 @@ struct AggregateFunctionAnyLastData : Data static const char * name() { return "anyLast"; } }; +template +struct AggregateFunctionSingleValueOrNullData : Data +{ + using Self = AggregateFunctionSingleValueOrNullData; + + bool first_value = true; + bool is_null = false; + + bool changeIfBetter(const IColumn & column, size_t row_num, Arena * arena) + { + if (first_value) + { + first_value = false; + this->change(column, row_num, arena); + return true; + } + else if (!this->isEqualTo(column, row_num)) + { + is_null = true; + } + return false; + } + + bool changeIfBetter(const Self & to, Arena * arena) + { + if (first_value) + { + first_value = false; + this->change(to, arena); + return true; + } + else if (!this->isEqualTo(to)) + { + is_null = true; + } + return false; + } + + void insertResultInto(IColumn & to) const + { + if (is_null || first_value) + { + to.insertDefault(); + } + else + { + ColumnNullable & col = typeid_cast(to); + col.getNullMapColumn().insertDefault(); + this->Data::insertResultInto(col.getNestedColumn()); + } + } + + static const char * name() { return "singleValueOrNull"; } +}; /** Implement 'heavy hitters' algorithm. * Selects most frequent value if its frequency is more than 50% in each thread of execution. @@ -722,7 +778,7 @@ public: DataTypePtr getReturnType() const override { - return type; + return Data::name() == "singleValueOrNull" ? std::make_shared(type) : type; } void add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena * arena) const override diff --git a/src/AggregateFunctions/AggregateFunctionSingleValueOrNull.cpp b/src/AggregateFunctions/AggregateFunctionSingleValueOrNull.cpp new file mode 100644 index 00000000000..cd897dfcf6e --- /dev/null +++ b/src/AggregateFunctions/AggregateFunctionSingleValueOrNull.cpp @@ -0,0 +1,27 @@ +#include +#include +#include +#include "registerAggregateFunctions.h" + + +namespace DB +{ +struct Settings; + +namespace +{ + +AggregateFunctionPtr createAggregateFunctionSingleValueOrNull(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings * settings) +{ + return AggregateFunctionPtr(createAggregateFunctionSingleValue(name, argument_types, parameters, settings)); +} + +} + +void registerAggregateFunctionSingleValueOrNull(AggregateFunctionFactory & factory) +{ + factory.registerFunction("singleValueOrNull", createAggregateFunctionSingleValueOrNull); +} + + +} diff --git a/src/AggregateFunctions/registerAggregateFunctions.cpp b/src/AggregateFunctions/registerAggregateFunctions.cpp index db6d8eb75bc..6fb373a1ce8 100644 --- a/src/AggregateFunctions/registerAggregateFunctions.cpp +++ b/src/AggregateFunctions/registerAggregateFunctions.cpp @@ -48,6 +48,7 @@ void registerAggregateFunctionRankCorrelation(AggregateFunctionFactory &); void registerAggregateFunctionMannWhitney(AggregateFunctionFactory &); void registerAggregateFunctionWelchTTest(AggregateFunctionFactory &); void registerAggregateFunctionStudentTTest(AggregateFunctionFactory &); +void registerAggregateFunctionSingleValueOrNull(AggregateFunctionFactory &); class AggregateFunctionCombinatorFactory; void registerAggregateFunctionCombinatorIf(AggregateFunctionCombinatorFactory &); @@ -111,6 +112,7 @@ void registerAggregateFunctions() registerAggregateFunctionMannWhitney(factory); registerAggregateFunctionWelchTTest(factory); registerAggregateFunctionStudentTTest(factory); + registerAggregateFunctionSingleValueOrNull(factory); registerWindowFunctions(factory); From a524feb73272b172c03a22c1540fd117ea93c4a9 Mon Sep 17 00:00:00 2001 From: Kirill Ershov Date: Thu, 3 Jun 2021 05:19:07 +0300 Subject: [PATCH 005/599] Rewrite ALL/ANY parser --- src/Parsers/ExpressionListParsers.cpp | 282 ++++++++++++++------------ src/Parsers/ExpressionListParsers.h | 21 +- 2 files changed, 158 insertions(+), 145 deletions(-) diff --git a/src/Parsers/ExpressionListParsers.cpp b/src/Parsers/ExpressionListParsers.cpp index a228dc6617b..acb2e35a84e 100644 --- a/src/Parsers/ExpressionListParsers.cpp +++ b/src/Parsers/ExpressionListParsers.cpp @@ -1,8 +1,12 @@ #include +#include #include #include #include +#include +#include +#include #include #include #include @@ -55,19 +59,6 @@ const char * ParserComparisonExpression::operators[] = nullptr }; -const char * ParserComparisonWithSubqueryExpression::operators[] = -{ - "==", "equals", - "!=", "notEquals", - "<>", "notEquals", - "<=", "lessOrEquals", - ">=", "greaterOrEquals", - "<", "less", - ">", "greater", - "=", "equals", - nullptr -}; - const char * ParserComparisonExpression::overlapping_operators_to_skip[] = { "IN PARTITION", @@ -180,6 +171,158 @@ static bool parseOperator(IParser::Pos & pos, const char * op, Expected & expect } } +enum class SubqueryFunctionType +{ + NONE, + ANY, + ALL +}; + +static bool modifyAST(String operator_name, std::shared_ptr & function, SubqueryFunctionType type) +{ + // = ANY --> IN, != ALL --> NOT IN + if ((operator_name == "equals" && type == SubqueryFunctionType::ANY) + || (operator_name == "notEquals" && type == SubqueryFunctionType::ALL)) + { + function->name = "in"; + if (operator_name == "notEquals") + { + auto function_not = std::make_shared(); + auto exp_list_not = std::make_shared(); + exp_list_not->children.push_back(function); + function_not->name = "not"; + function_not->children.push_back(exp_list_not); + function_not->arguments = exp_list_not; + function = function_not; + } + return true; + } + + // subquery --> (SELECT aggregate_function(*) FROM subquery) + auto aggregate_function = std::make_shared(); + auto aggregate_function_exp_list = std::make_shared(); + aggregate_function_exp_list ->children.push_back(std::make_shared()); + aggregate_function->arguments = aggregate_function_exp_list; + aggregate_function->children.push_back(aggregate_function_exp_list); + + ASTPtr subquery_node = function->children[0]->children[1]; + auto select_query = std::make_shared(); + auto tables_in_select = std::make_shared(); + auto tables_in_select_element = std::make_shared(); + auto table_expression = std::make_shared(); + table_expression->subquery = subquery_node; + table_expression->children.push_back(subquery_node); + tables_in_select_element->table_expression = table_expression; + tables_in_select_element->children.push_back(table_expression); + tables_in_select->children.push_back(tables_in_select_element); + auto select_exp_list = std::make_shared(); + select_exp_list->children.push_back(aggregate_function); + select_query->children.push_back(select_exp_list); + select_query->children.push_back(tables_in_select); + select_query->setExpression(ASTSelectQuery::Expression::SELECT, std::move(select_exp_list)); + select_query->setExpression(ASTSelectQuery::Expression::TABLES, std::move(tables_in_select)); + + auto select_with_union_query = std::make_shared(); + auto list_of_selects = std::make_shared(); + list_of_selects->children.push_back(select_query); + select_with_union_query->list_of_selects = list_of_selects; + select_with_union_query->children.push_back(select_with_union_query->list_of_selects); + + auto new_subquery = std::make_shared(); + new_subquery->children.push_back(select_with_union_query); + function->children[0]->children.pop_back(); + function->children[0]->children.push_back(new_subquery); + + if (operator_name == "greaterOrEquals" || operator_name == "greater") + { + aggregate_function->name = type == SubqueryFunctionType::ANY ? "min" : "max"; + return true; + } + if (operator_name == "lessOrEquals" || operator_name == "less") + { + aggregate_function->name = type == SubqueryFunctionType::ANY ? "max" : "min"; + return true; + } + if (operator_name == "equals" || operator_name == "notEquals") + { + aggregate_function->name = "singleValueOrNull"; + return true; + } + return false; +} + +bool ParserComparisonExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) +{ + bool first = true; + + auto current_depth = pos.depth; + while (true) + { + if (first) + { + ASTPtr elem; + if (!elem_parser.parse(pos, elem, expected)) + return false; + + node = elem; + first = false; + } + else + { + /// try to find any of the valid operators + const char ** it; + Expected stub; + for (it = overlapping_operators_to_skip; *it; ++it) + if (ParserKeyword{*it}.checkWithoutMoving(pos, stub)) + break; + + if (*it) + break; + + for (it = operators; *it; it += 2) + if (parseOperator(pos, *it, expected)) + break; + + if (!*it) + break; + + /// the function corresponding to the operator + auto function = std::make_shared(); + + /// function arguments + auto exp_list = std::make_shared(); + + ASTPtr elem; + SubqueryFunctionType subquery_function_type = SubqueryFunctionType::NONE; + if (ParserKeyword("ANY").ignore(pos, expected)) + subquery_function_type = SubqueryFunctionType::ANY; + else if (ParserKeyword("ALL").ignore(pos, expected)) + subquery_function_type = SubqueryFunctionType::ALL; + else if (!elem_parser.parse(pos, elem, expected)) + return false; + + if (subquery_function_type != SubqueryFunctionType::NONE && !ParserSubquery().parse(pos, elem, expected)) + return false; + + /// the first argument of the function is the previous element, the second is the next one + function->name = it[1]; + function->arguments = exp_list; + function->children.push_back(exp_list); + + exp_list->children.push_back(node); + exp_list->children.push_back(elem); + + if (subquery_function_type != SubqueryFunctionType::NONE && !modifyAST(function->name, function, subquery_function_type)) + return false; + + pos.increaseDepth(); + node = function; + } + } + + pos.depth = current_depth; + return true; +} bool ParserLeftAssociativeBinaryOperatorList::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { @@ -371,119 +514,6 @@ bool ParserBetweenExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & exp return true; } -bool ParserComparisonWithSubqueryExpression::modifySubquery(String operator_name, ASTPtr subquery_node, bool is_any) -{ - ASTPtr select_with_union_node = subquery_node->children[0]; - if (select_with_union_node->children[0]->children.size() != 1) - return false; - ASTPtr select_node = select_with_union_node->children[0]->children[0]; - ASTPtr exp_list = select_node->children[0]; - auto function = std::make_shared(); - function->arguments = exp_list; - function->children.push_back(exp_list); - - ASTPtr new_exp_list = std::make_shared(); - new_exp_list->children.push_back(function); - - if (operator_name == "greaterOrEquals" || operator_name == "greater") - { - function->name = is_any ? "min" : "max"; - select_node->children[0] = new_exp_list; - return true; - } - - if (operator_name == "lessOrEquals" || operator_name == "less") - { - function->name = is_any ? "max" : "min"; - select_node->children[0] = new_exp_list; - return true; - } - return false; -} - -bool ParserComparisonWithSubqueryExpression::addFunctionIn(String operator_name, ASTPtr & node, bool is_any) -{ - - auto function_in = std::make_shared(); - auto exp_list_in = std::make_shared(); - exp_list_in->children.push_back(node->children[0]->children[0]); - exp_list_in->children.push_back(node->children[0]->children[1]); - function_in->name = "in"; - function_in->children.push_back(exp_list_in); - function_in->arguments = exp_list_in; - - if (operator_name == "equals" && is_any) - { - node = function_in; - return true; - } - - if (operator_name == "notEquals" && !is_any) - { - auto function_not = std::make_shared(); - auto exp_list_not = std::make_shared(); - exp_list_not->children.push_back(function_in); - function_not->name = "not"; - function_not->children.push_back(exp_list_not); - function_not->arguments = exp_list_not; - node = function_not; - return true; - } - return false; -} - -bool ParserComparisonWithSubqueryExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) -{ - Pos begin = pos; - ASTPtr elem; - if (!elem_parser.parse(pos, elem, expected)) - return next_parser.parse(pos, node, expected); - - /// try to find any of the valid operators - const char ** it; - for (it = operators; *it; it += 2) - if (parseOperator(pos, *it, expected)) - break; - - if (!*it) - { - node = elem; - return true; - } - - bool is_any = true; - if (!ParserKeyword("ANY").ignore(pos, expected)) - { - is_any = false; - if (!ParserKeyword("ALL").ignore(pos, expected)) - { - pos = begin; - return next_parser.parse(pos, node, expected); - } - } - - ASTPtr subquery_node; - if (!ParserSubquery().parse(pos, subquery_node, expected)) - return false; - - /// the first argument of the function is the previous element, the second is the next one - String operator_name = it[1]; - - /// the function corresponding to the operator - auto function = std::make_shared(); - - /// function arguments - auto exp_list = std::make_shared(); - exp_list->children.push_back(elem); - exp_list->children.push_back(subquery_node); - - function->name = operator_name; - function->arguments = exp_list; - function->children.push_back(exp_list); - node = function; - return modifySubquery(operator_name, subquery_node, is_any) || addFunctionIn(operator_name, node, is_any); -} - bool ParserTernaryOperatorExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { ParserToken symbol1(TokenType::QuestionMark); diff --git a/src/Parsers/ExpressionListParsers.h b/src/Parsers/ExpressionListParsers.h index 29335e72b2a..82fd0eefc8e 100644 --- a/src/Parsers/ExpressionListParsers.h +++ b/src/Parsers/ExpressionListParsers.h @@ -353,28 +353,11 @@ class ParserComparisonExpression : public IParserBase private: static const char * operators[]; static const char * overlapping_operators_to_skip[]; - ParserLeftAssociativeBinaryOperatorList operator_parser {operators, overlapping_operators_to_skip, std::make_unique()}; + ParserBetweenExpression elem_parser; protected: const char * getName() const override{ return "comparison expression"; } - bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override - { - return operator_parser.parse(pos, node, expected); - } -}; - -class ParserComparisonWithSubqueryExpression : public IParserBase -{ -private: - static const char * operators[]; - ParserComparisonExpression next_parser; - ParserBetweenExpression elem_parser; - static bool addFunctionIn(String operator_name, ASTPtr & node, bool is_any); - static bool modifySubquery(String operator_name, ASTPtr subquery_node, bool is_any); -protected: - const char * getName() const override { return "comparison with ANY/ALL expression"; } - bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; }; @@ -383,7 +366,7 @@ protected: class ParserNullityChecking : public IParserBase { private: - ParserComparisonWithSubqueryExpression elem_parser; + ParserComparisonExpression elem_parser; protected: const char * getName() const override { return "nullity checking"; } bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; From 15843723ec9b8149e876a7414629b72c68744bb7 Mon Sep 17 00:00:00 2001 From: Kirill Ershov Date: Sat, 5 Jun 2021 02:30:57 +0300 Subject: [PATCH 006/599] Fix ALL/ANY implementation --- src/Parsers/ExpressionListParsers.cpp | 14 ++++++++++++++ 1 file changed, 14 insertions(+) diff --git a/src/Parsers/ExpressionListParsers.cpp b/src/Parsers/ExpressionListParsers.cpp index acb2e35a84e..920a31199d4 100644 --- a/src/Parsers/ExpressionListParsers.cpp +++ b/src/Parsers/ExpressionListParsers.cpp @@ -243,9 +243,23 @@ static bool modifyAST(String operator_name, std::shared_ptr & funct aggregate_function->name = type == SubqueryFunctionType::ANY ? "max" : "min"; return true; } + + // = ALL --> IN (SELECT singleValueOrNull(*) FROM subquery) + // != ANY --> NOT IN (SELECT singleValueOrNull(*) FROM subquery) if (operator_name == "equals" || operator_name == "notEquals") { aggregate_function->name = "singleValueOrNull"; + function->name = "in"; + if (operator_name == "notEquals") + { + auto function_not = std::make_shared(); + auto exp_list_not = std::make_shared(); + exp_list_not->children.push_back(function); + function_not->name = "not"; + function_not->children.push_back(exp_list_not); + function_not->arguments = exp_list_not; + function = function_not; + } return true; } return false; From 1d46e443e29dd77c401b6345f483d750530c1bc2 Mon Sep 17 00:00:00 2001 From: Kirill Ershov Date: Sat, 5 Jun 2021 03:58:22 +0300 Subject: [PATCH 007/599] Fix INTERSECT/EXCEPT parser --- src/Parsers/ASTIntersectOrExcept.cpp | 2 +- src/Parsers/ExpressionElementParsers.cpp | 2 ++ src/Parsers/ParserIntersectOrExcept.cpp | 11 +++-------- src/Processors/QueryPlan/IntersectOrExceptStep.cpp | 2 +- 4 files changed, 7 insertions(+), 10 deletions(-) diff --git a/src/Parsers/ASTIntersectOrExcept.cpp b/src/Parsers/ASTIntersectOrExcept.cpp index 073d63963a9..a05d7ee86c9 100644 --- a/src/Parsers/ASTIntersectOrExcept.cpp +++ b/src/Parsers/ASTIntersectOrExcept.cpp @@ -20,7 +20,7 @@ void ASTIntersectOrExcept::formatQueryImpl(const FormatSettings & settings, Form children[0]->formatImpl(settings, state, frame); std::string indent_str = settings.one_line ? "" : std::string(4 * frame.indent, ' '); settings.ostr << settings.nl_or_ws << indent_str << (settings.hilite ? hilite_keyword : "") - << (is_except ? "EXCEPT" : "INTERSECT ") + << (is_except ? "EXCEPT" : "INTERSECT") << (settings.hilite ? hilite_none : "") << settings.nl_or_ws; children[1]->formatImpl(settings, state, frame); } diff --git a/src/Parsers/ExpressionElementParsers.cpp b/src/Parsers/ExpressionElementParsers.cpp index 75a341a543d..d08ce2a215b 100644 --- a/src/Parsers/ExpressionElementParsers.cpp +++ b/src/Parsers/ExpressionElementParsers.cpp @@ -1697,6 +1697,8 @@ const char * ParserAlias::restricted_keywords[] = "WHERE", "WINDOW", "WITH", + "INTERSECT", + "EXCEPT", nullptr }; diff --git a/src/Parsers/ParserIntersectOrExcept.cpp b/src/Parsers/ParserIntersectOrExcept.cpp index a82b8c2b06b..6d5da54fa38 100644 --- a/src/Parsers/ParserIntersectOrExcept.cpp +++ b/src/Parsers/ParserIntersectOrExcept.cpp @@ -3,7 +3,7 @@ #include #include #include -#include +#include namespace DB { @@ -17,7 +17,7 @@ bool ParserIntersectOrExcept::parseImpl(Pos & pos, ASTPtr & node, Expected & exp auto ast = std::make_shared(); ast->is_except = false; - if (!ParserSelectQuery().parse(pos, left_node, expected) && !ParserSubquery().parse(pos, left_node, expected)) + if (!ParserSelectWithUnionQuery().parse(pos, left_node, expected) && !ParserSubquery().parse(pos, left_node, expected)) return false; if (!intersect_keyword.ignore(pos)) @@ -32,14 +32,9 @@ bool ParserIntersectOrExcept::parseImpl(Pos & pos, ASTPtr & node, Expected & exp } } - if (!ParserSelectQuery().parse(pos, right_node, expected) && !ParserSubquery().parse(pos, right_node, expected)) + if (!ParserSelectWithUnionQuery().parse(pos, right_node, expected) && !ParserSubquery().parse(pos, right_node, expected)) return false; - if (const auto * ast_subquery = left_node->as()) - left_node = ast_subquery->children.at(0); - if (const auto * ast_subquery = right_node->as()) - right_node = ast_subquery->children.at(0); - ast->children.push_back(left_node); ast->children.push_back(right_node); diff --git a/src/Processors/QueryPlan/IntersectOrExceptStep.cpp b/src/Processors/QueryPlan/IntersectOrExceptStep.cpp index d0a820339d7..28f34bda5db 100644 --- a/src/Processors/QueryPlan/IntersectOrExceptStep.cpp +++ b/src/Processors/QueryPlan/IntersectOrExceptStep.cpp @@ -15,7 +15,7 @@ IntersectOrExceptStep::IntersectOrExceptStep(bool is_except_, DataStreams input_ output_stream = DataStream{.header = header}; } -QueryPipelinePtr IntersectOrExceptStep::updatePipeline(QueryPipelines pipelines, const BuildQueryPipelineSettings & ) +QueryPipelinePtr IntersectOrExceptStep::updatePipeline(QueryPipelines pipelines, const BuildQueryPipelineSettings &) { auto pipeline = std::make_unique(); QueryPipelineProcessorsCollector collector(*pipeline, this); From 5badf38d57c13a9641465dc3a3607c7001362d19 Mon Sep 17 00:00:00 2001 From: Vladimir Goncharov Date: Fri, 18 Jun 2021 19:37:37 +0300 Subject: [PATCH 008/599] Update mergetree.md Basic info about projections based on RFC https://github.com/ClickHouse/ClickHouse/issues/14730 --- .../mergetree-family/mergetree.md | 25 ++++++++++++++++++- 1 file changed, 24 insertions(+), 1 deletion(-) diff --git a/docs/en/engines/table-engines/mergetree-family/mergetree.md b/docs/en/engines/table-engines/mergetree-family/mergetree.md index 9d259456ea5..98475bdd7a4 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -39,7 +39,10 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] name2 [type2] [DEFAULT|MATERIALIZED|ALIAS expr2] [TTL expr2], ... INDEX index_name1 expr1 TYPE type1(...) GRANULARITY value1, - INDEX index_name2 expr2 TYPE type2(...) GRANULARITY value2 + INDEX index_name2 expr2 TYPE type2(...) GRANULARITY value2, + ... + PROJECTION projection_name_1 (SELECT [WHERE] [GROUP BY] [ORDER BY]), + PROJECTION projection_name_2 (SELECT [WHERE] [GROUP BY] [ORDER BY]) ) ENGINE = MergeTree() ORDER BY expr [PARTITION BY expr] @@ -385,6 +388,26 @@ Functions with a constant argument that is less than ngram size can’t be used - `s != 1` - `NOT startsWith(s, 'test')` +### Projections {#projections} +Projections are like materialized views, but defined in part-level. It provides consistency guarantees along with automatic usage in queries. + +#### Query {#projection-query} +A projection query is what defines a projection. It has the following grammar: + +`SELECT [WHERE] [GROUP BY] [ORDER BY]` + +It implicitly selects data from the parent table. + +#### Storage {#projection-storage} +Projections are stored inside the part directory. It's similar to an index but contains an subdirectory which stores an anonymous MergeTree table's part. The table is induced by the definition query of the projection. If there is a GROUP BY clause, the underlying storage engine becomes AggregatedMergeTree, and all aggregate functions are converted to either AggregateFunction or SimpleAggregateFunction. If there is an ORDER BY clause, the MergeTree table will use it as its primary key expression. During the merge process, the projection part will be merged via its storage's merge routine. The checksum of the parent table's part will combine the projection's part. Other maintenance jobs are similar to skip indices. + +#### Query Routing {#projection-query-routing} +1. Check if the projection contains all the needed columns and rows. +2. If it's an aggregated projection, also check if it has the right columns inside the GROUP BY clause along with required aggregate functions. +3. If it's an sorted projection, also check how many granules will be selected by the KeyCondition. +4. Select the best feasible match. +5. The query pipeline which uses projections will be different from the one that uses the original parts. if the projection is absent in some parts, we can add the pipeline to "project" it on the fly. + ## Concurrent Data Access {#concurrent-data-access} For concurrent table access, we use multi-versioning. In other words, when a table is simultaneously read and updated, data is read from a set of parts that is current at the time of the query. There are no lengthy locks. Inserts do not get in the way of read operations. From ebf9681888afd51b5a1cc37fcc55d0d11b6798c3 Mon Sep 17 00:00:00 2001 From: Vladimir Goncharov Date: Fri, 18 Jun 2021 19:55:42 +0300 Subject: [PATCH 009/599] Create projection.md info about manipulating projections based on RFC #14730 --- .../statements/alter/projection.md | 23 +++++++++++++++++++ 1 file changed, 23 insertions(+) create mode 100644 docs/en/sql-reference/statements/alter/projection.md diff --git a/docs/en/sql-reference/statements/alter/projection.md b/docs/en/sql-reference/statements/alter/projection.md new file mode 100644 index 00000000000..f7f829c9589 --- /dev/null +++ b/docs/en/sql-reference/statements/alter/projection.md @@ -0,0 +1,23 @@ +--- +toc_priority: 49 +toc_title: PROJECTION +--- + +# Manipulating Projections {#manipulations-with-projections} + +The following operations are available: + +- `ALTER TABLE [db].name ADD PROJECTION name AS SELECT [WHERE] [GROUP BY] [ORDER BY]` - Adds projection description to tables metadata. + +- `ALTER TABLE [db].name DROP PROJECTION name` - Removes projection description from tables metadata and deletes projection files from disk. + +- `ALTER TABLE [db.]table MATERIALIZE PROJECTION name IN PARTITION partition_name` - The query rebuilds the projection `name` in the partition `partition_name`. Implemented as a [mutation](../../../../sql-reference/statements/alter/index.md#mutations). + +- `ALTER TABLE [db.]table CLEAR PROJECTION name IN PARTITION partition_name` + +The first two commands are lightweight in a sense that they only change metadata or remove files. + +Also, they are replicated, syncing indices metadata via ZooKeeper. + +!!! note "Note" + Projection manipulation is supported only for tables with [`*MergeTree`](../../../../engines/table-engines/mergetree-family/mergetree.md) engine (including [replicated](../../../../engines/table-engines/mergetree-family/replication.md) variants). From 043722b7cfda51a263a3ef1bbb44637fc49c913e Mon Sep 17 00:00:00 2001 From: Vladimir Goncharov Date: Sat, 19 Jun 2021 18:10:51 +0300 Subject: [PATCH 010/599] Update mergetree.md Currently WHERE is not supported. --- docs/en/engines/table-engines/mergetree-family/mergetree.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/en/engines/table-engines/mergetree-family/mergetree.md b/docs/en/engines/table-engines/mergetree-family/mergetree.md index 98475bdd7a4..e67b481da2c 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -41,8 +41,8 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] INDEX index_name1 expr1 TYPE type1(...) GRANULARITY value1, INDEX index_name2 expr2 TYPE type2(...) GRANULARITY value2, ... - PROJECTION projection_name_1 (SELECT [WHERE] [GROUP BY] [ORDER BY]), - PROJECTION projection_name_2 (SELECT [WHERE] [GROUP BY] [ORDER BY]) + PROJECTION projection_name_1 (SELECT [GROUP BY] [ORDER BY]), + PROJECTION projection_name_2 (SELECT [GROUP BY] [ORDER BY]) ) ENGINE = MergeTree() ORDER BY expr [PARTITION BY expr] @@ -394,7 +394,7 @@ Projections are like materialized views, but defined in part-level. It provides #### Query {#projection-query} A projection query is what defines a projection. It has the following grammar: -`SELECT [WHERE] [GROUP BY] [ORDER BY]` +`SELECT [GROUP BY] [ORDER BY]` It implicitly selects data from the parent table. From da4dc26975c85441911fed22b9a625a91074bacb Mon Sep 17 00:00:00 2001 From: Vladimir Goncharov Date: Sat, 19 Jun 2021 18:15:25 +0300 Subject: [PATCH 011/599] Update projection.md --- docs/en/sql-reference/statements/alter/projection.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/en/sql-reference/statements/alter/projection.md b/docs/en/sql-reference/statements/alter/projection.md index f7f829c9589..526202bed33 100644 --- a/docs/en/sql-reference/statements/alter/projection.md +++ b/docs/en/sql-reference/statements/alter/projection.md @@ -13,11 +13,11 @@ The following operations are available: - `ALTER TABLE [db.]table MATERIALIZE PROJECTION name IN PARTITION partition_name` - The query rebuilds the projection `name` in the partition `partition_name`. Implemented as a [mutation](../../../../sql-reference/statements/alter/index.md#mutations). -- `ALTER TABLE [db.]table CLEAR PROJECTION name IN PARTITION partition_name` +- `ALTER TABLE [db.]table CLEAR PROJECTION name IN PARTITION partition_name` - Deletes projection files from disk without removing description. -The first two commands are lightweight in a sense that they only change metadata or remove files. +The commands ADD, DROP and CLEAR are lightweight in a sense that they only change metadata or remove files. -Also, they are replicated, syncing indices metadata via ZooKeeper. +Also, they are replicated, syncing projections metadata via ZooKeeper. !!! note "Note" Projection manipulation is supported only for tables with [`*MergeTree`](../../../../engines/table-engines/mergetree-family/mergetree.md) engine (including [replicated](../../../../engines/table-engines/mergetree-family/replication.md) variants). From a1acc56718643ce720dcfea7e77ae44ab5ce42a4 Mon Sep 17 00:00:00 2001 From: Vladimir Goncharov Date: Sat, 19 Jun 2021 18:23:26 +0300 Subject: [PATCH 012/599] Update mergetree.md Query Routing -> Query Analysis. SimpleAggregateFunction is not used for now. --- .../table-engines/mergetree-family/mergetree.md | 12 +++++------- 1 file changed, 5 insertions(+), 7 deletions(-) diff --git a/docs/en/engines/table-engines/mergetree-family/mergetree.md b/docs/en/engines/table-engines/mergetree-family/mergetree.md index e67b481da2c..177b0bfde99 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -399,14 +399,12 @@ A projection query is what defines a projection. It has the following grammar: It implicitly selects data from the parent table. #### Storage {#projection-storage} -Projections are stored inside the part directory. It's similar to an index but contains an subdirectory which stores an anonymous MergeTree table's part. The table is induced by the definition query of the projection. If there is a GROUP BY clause, the underlying storage engine becomes AggregatedMergeTree, and all aggregate functions are converted to either AggregateFunction or SimpleAggregateFunction. If there is an ORDER BY clause, the MergeTree table will use it as its primary key expression. During the merge process, the projection part will be merged via its storage's merge routine. The checksum of the parent table's part will combine the projection's part. Other maintenance jobs are similar to skip indices. +Projections are stored inside the part directory. It's similar to an index but contains an subdirectory which stores an anonymous MergeTree table's part. The table is induced by the definition query of the projection. If there is a GROUP BY clause, the underlying storage engine becomes AggregatedMergeTree, and all aggregate functions are converted to AggregateFunction. If there is an ORDER BY clause, the MergeTree table will use it as its primary key expression. During the merge process, the projection part will be merged via its storage's merge routine. The checksum of the parent table's part will combine the projection's part. Other maintenance jobs are similar to skip indices. -#### Query Routing {#projection-query-routing} -1. Check if the projection contains all the needed columns and rows. -2. If it's an aggregated projection, also check if it has the right columns inside the GROUP BY clause along with required aggregate functions. -3. If it's an sorted projection, also check how many granules will be selected by the KeyCondition. -4. Select the best feasible match. -5. The query pipeline which uses projections will be different from the one that uses the original parts. if the projection is absent in some parts, we can add the pipeline to "project" it on the fly. +#### Query Analysis {#projection-query-analysis} +1. Check if the projection can be used to answer the given query, that is, it generates the same answer as querying the base table. +2. Select the best feasible match, which contains the least granules to read. +3. The query pipeline which uses projections will be different from the one that uses the original parts. if the projection is absent in some parts, we can add the pipeline to "project" it on the fly. ## Concurrent Data Access {#concurrent-data-access} From abf4bf832bcf1717660bebd006e514d9f9436bbc Mon Sep 17 00:00:00 2001 From: Vladimir Goncharov Date: Sat, 19 Jun 2021 18:32:18 +0300 Subject: [PATCH 013/599] Update projection.md Currently WHERE is not supported. --- docs/en/sql-reference/statements/alter/projection.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/statements/alter/projection.md b/docs/en/sql-reference/statements/alter/projection.md index 526202bed33..9d762a07234 100644 --- a/docs/en/sql-reference/statements/alter/projection.md +++ b/docs/en/sql-reference/statements/alter/projection.md @@ -7,7 +7,7 @@ toc_title: PROJECTION The following operations are available: -- `ALTER TABLE [db].name ADD PROJECTION name AS SELECT [WHERE] [GROUP BY] [ORDER BY]` - Adds projection description to tables metadata. +- `ALTER TABLE [db].name ADD PROJECTION name AS SELECT [GROUP BY] [ORDER BY]` - Adds projection description to tables metadata. - `ALTER TABLE [db].name DROP PROJECTION name` - Removes projection description from tables metadata and deletes projection files from disk. From 4bc07771eee09e7f72dd5672fbefc1a2f840e05d Mon Sep 17 00:00:00 2001 From: Peng Jian Date: Wed, 14 Jul 2021 18:18:29 +0800 Subject: [PATCH 014/599] Fix the clearOldTemporaryDirectories is executed too frequently --- src/Core/Settings.h | 2 ++ src/Storages/StorageMergeTree.cpp | 23 +++++++++++++++++------ src/Storages/StorageMergeTree.h | 6 ++++-- 3 files changed, 23 insertions(+), 8 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 8c733415dec..fcfeb5d9543 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -143,6 +143,8 @@ class IColumn; M(UInt64, merge_tree_coarse_index_granularity, 8, "If the index segment can contain the required keys, divide it into as many parts and recursively check them.", 0) \ M(UInt64, merge_tree_max_rows_to_use_cache, (128 * 8192), "The maximum number of rows per request, to use the cache of uncompressed data. If the request is large, the cache is not used. (For large queries not to flush out the cache.)", 0) \ M(UInt64, merge_tree_max_bytes_to_use_cache, (192 * 10 * 1024 * 1024), "The maximum number of bytes per request, to use the cache of uncompressed data. If the request is large, the cache is not used. (For large queries not to flush out the cache.)", 0) \ + M(UInt64, merge_tree_clear_old_temporary_directories_interval_seconds, 60, "The period of executing the clear old temporary directories operation in background.", 0) \ + M(UInt64, merge_tree_clear_old_parts_interval_seconds, 1, "The period of executing the clear old parts operation in background.", 0) \ M(Bool, do_not_merge_across_partitions_select_final, false, "Merge parts only in one partition in select final", 0) \ \ M(UInt64, mysql_max_rows_to_insert, 65536, "The maximum number of rows in MySQL batch insertion of the MySQL storage engine", 0) \ diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 05d18e65068..01d77e4892e 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -109,7 +109,8 @@ void StorageMergeTree::startup() clearOldTemporaryDirectories(0); /// NOTE background task will also do the above cleanups periodically. - time_after_previous_cleanup.restart(); + time_after_previous_cleanup_parts.restart(); + time_after_previous_cleanup_temporary_directories.restart(); try { @@ -1061,22 +1062,32 @@ bool StorageMergeTree::scheduleDataProcessingJob(IBackgroundJobExecutor & execut }, PoolType::MERGE_MUTATE}); return true; } - else if (auto cmp_lock = time_after_previous_cleanup.compareAndRestartDeferred(1)) + bool executed = false; + if (time_after_previous_cleanup_temporary_directories.compareAndRestartDeferred(getContext()->getSettingsRef().merge_tree_clear_old_temporary_directories_interval_seconds)) + { + executor.execute({[this, share_lock] () + { + clearOldTemporaryDirectories(); + return true; + }, PoolType::MERGE_MUTATE}); + executed = true; + } + if (time_after_previous_cleanup_parts.compareAndRestartDeferred(getContext()->getSettingsRef().merge_tree_clear_old_parts_interval_seconds)) { executor.execute({[this, share_lock] () { /// All use relative_data_path which changes during rename /// so execute under share lock. clearOldPartsFromFilesystem(); - clearOldTemporaryDirectories(getSettings()->temporary_directories_lifetime.totalSeconds()); clearOldWriteAheadLogs(); clearOldMutations(); clearEmptyParts(); return true; }, PoolType::MERGE_MUTATE}); - return true; - } - return false; + executed = true; + } + + return executed; } Int64 StorageMergeTree::getCurrentMutationVersion( diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index fa2b2e9f08b..f9552a49018 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -114,8 +114,10 @@ private: /// For block numbers. SimpleIncrement increment; - /// For clearOldParts, clearOldTemporaryDirectories. - AtomicStopwatch time_after_previous_cleanup; + /// For clearOldParts + AtomicStopwatch time_after_previous_cleanup_parts; + /// For clearOldTemporaryDirectories. + AtomicStopwatch time_after_previous_cleanup_temporary_directories; /// Mutex for parts currently processing in background /// merging (also with TTL), mutating or moving. From 300635d4a17e660896dcab82758a29b30e64ea53 Mon Sep 17 00:00:00 2001 From: Peng Jian Date: Wed, 14 Jul 2021 19:17:45 +0800 Subject: [PATCH 015/599] add docs --- docs/en/operations/settings/settings.md | 20 ++++++++++++++++++++ 1 file changed, 20 insertions(+) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 9c1cc6eccfd..a5d1f3b121e 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -708,6 +708,26 @@ Possible value: Default value: 2013265920. +## merge_tree_clear_old_temporary_directories_interval_seconds {#setting-merge-tree-clear-old-temporary-directories-interval-seconds} + +The interval in seconds for ClickHouse to execute the cleanup old temporary directories. + +Possible value: + +- Any positive integer. + +Default value: 60. + +## merge_tree_clear_old_parts_interval_seconds {#setting-merge-tree-clear-old-parts-interval-seconds} + +The interval in seconds for ClickHouse to execute the cleanup old parts, WALs, and mutations. + +Possible value: + +- Any positive integer. + +Default value: 1. + ## min_bytes_to_use_direct_io {#settings-min-bytes-to-use-direct-io} The minimum data volume required for using direct I/O access to the storage disk. From 5be892484dc39f531b7c37c2666a34a4728c4902 Mon Sep 17 00:00:00 2001 From: Peng Jian Date: Wed, 14 Jul 2021 20:18:32 +0800 Subject: [PATCH 016/599] fix build --- src/Storages/StorageMergeTree.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 01d77e4892e..b670c1f909c 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -1067,7 +1067,7 @@ bool StorageMergeTree::scheduleDataProcessingJob(IBackgroundJobExecutor & execut { executor.execute({[this, share_lock] () { - clearOldTemporaryDirectories(); + clearOldTemporaryDirectories(getSettings()->temporary_directories_lifetime.totalSeconds()); return true; }, PoolType::MERGE_MUTATE}); executed = true; From 177611df355bef797b59f571fd02f17662c5ebb4 Mon Sep 17 00:00:00 2001 From: Dmitriy Date: Wed, 21 Jul 2021 20:16:36 +0300 Subject: [PATCH 017/599] Edit and translate to Russian MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Выполнил перевод на русский язык. --- .../materialized-postgresql.md | 24 +++---- .../materialized-postgresql.md | 67 +++++++++++++++++++ .../sql-reference/statements/create/table.md | 2 +- 3 files changed, 78 insertions(+), 15 deletions(-) create mode 100644 docs/ru/engines/database-engines/materialized-postgresql.md diff --git a/docs/en/engines/database-engines/materialized-postgresql.md b/docs/en/engines/database-engines/materialized-postgresql.md index 6b8c6e86952..c53a6c88aba 100644 --- a/docs/en/engines/database-engines/materialized-postgresql.md +++ b/docs/en/engines/database-engines/materialized-postgresql.md @@ -9,35 +9,33 @@ toc_title: MaterializedPostgreSQL ``` sql CREATE DATABASE test_database -ENGINE = MaterializedPostgreSQL('postgres1:5432', 'postgres_database', 'postgres_user', 'postgres_password' +ENGINE = MaterializedPostgreSQL('postgres1:5432', 'postgres_database', 'postgres_user', 'postgres_password'); SELECT * FROM test_database.postgres_table; ``` - ## Settings {#settings} -1. `materialized_postgresql_max_block_size` - Number of rows collected before flushing data into table. Default: `65536`. +1. `materialized_postgresql_max_block_size` — Number of rows collected before flushing data into table. Default: `65536`. -2. `materialized_postgresql_tables_list` - List of tables for MaterializedPostgreSQL database engine. Default: `whole database`. +2. `materialized_postgresql_tables_list` — List of tables for MaterializedPostgreSQL database engine. Default: `whole database`. -3. `materialized_postgresql_allow_automatic_update` - Allow to reload table in the background, when schema changes are detected. Default: `0` (`false`). +3. `materialized_postgresql_allow_automatic_update` — Allow to reload table in the background, when schema changes are detected. Default: `0` (`false`). ``` sql CREATE DATABASE test_database -ENGINE = MaterializedPostgreSQL('postgres1:5432', 'postgres_database', 'postgres_user', 'postgres_password' +ENGINE = MaterializedPostgreSQL('postgres1:5432', 'postgres_database', 'postgres_user', 'postgres_password') SETTINGS materialized_postgresql_max_block_size = 65536, materialized_postgresql_tables_list = 'table1,table2,table3'; SELECT * FROM test_database.table1; ``` - ## Requirements {#requirements} -- Setting `wal_level`to `logical` and `max_replication_slots` to at least `2` in the postgresql config file. +- Setting `wal_level` to `logical` and `max_replication_slots` to at least `2` in the PostgreSQL config file. -- Each replicated table must have one of the following **replica identity**: +- Each replicated table must have one of the following **replica identity**: 1. **default** (primary key) @@ -49,9 +47,8 @@ postgres# CREATE unique INDEX postgres_table_index on postgres_table(a, c, e); postgres# ALTER TABLE postgres_table REPLICA IDENTITY USING INDEX postgres_table_index; ``` - -Primary key is always checked first. If it is absent, then index, defined as replica identity index, is checked. -If index is used as replica identity, there has to be only one such index in a table. +The primary key is always checked first. If it is absent, then the index, defined as replica identity index, is checked. +If the index is used as a replica identity, there has to be only one such index in a table. You can check what type is used for a specific table with the following command: ``` bash @@ -65,7 +62,6 @@ FROM pg_class WHERE oid = 'postgres_table'::regclass; ``` - ## Warning {#warning} -1. **TOAST** values convertion is not supported. Default value for the data type will be used. +1. **TOAST** values conversion is not supported. Default value for the data type will be used. diff --git a/docs/ru/engines/database-engines/materialized-postgresql.md b/docs/ru/engines/database-engines/materialized-postgresql.md new file mode 100644 index 00000000000..79304fca382 --- /dev/null +++ b/docs/ru/engines/database-engines/materialized-postgresql.md @@ -0,0 +1,67 @@ +--- +toc_priority: 30 +toc_title: MaterializedPostgreSQL +--- + +# MaterializedPostgreSQL {#materialize-postgresql} + +## Создание базы данных {#creating-a-database} + +``` sql +CREATE DATABASE test_database +ENGINE = MaterializedPostgreSQL('postgres1:5432', 'postgres_database', 'postgres_user', 'postgres_password'); + +SELECT * FROM test_database.postgres_table; +``` + +## Настройки {#settings} + +1. `materialized_postgresql_max_block_size` — задает максимальное количество строк, собранных перед вставкой данных в таблицу. По умолчанию: `65536`. + +2. `materialized_postgresql_tables_list` — список таблиц для движка баз данных `MaterializedPostgreSQL`. По умолчанию: `whole database`. + +3. `materialized_postgresql_allow_automatic_update` — позволяет автоматически обновить таблицу в фоновом режиме при обнаружении изменений схемы. По умолчанию: `0` (`false`). + +``` sql +CREATE DATABASE test_database +ENGINE = MaterializedPostgreSQL('postgres1:5432', 'postgres_database', 'postgres_user', 'postgres_password') +SETTINGS materialized_postgresql_max_block_size = 65536, + materialized_postgresql_tables_list = 'table1,table2,table3'; + +SELECT * FROM test_database.table1; +``` + +## Требования {#requirements} + +- Настройка `wal_level` должна иметь значение `logical` и настройка `max_replication_slots` должна быть равна, по меньшей мере, `2` в конфигурационном файле в PostgreSQL. + +- Каждая реплицируемая таблица должна иметь один из следующих **идентификаторов реплики**: + +1. **по умолчанию** (первичный ключ) + +2. **индекс** + +``` bash +postgres# CREATE TABLE postgres_table (a Integer NOT NULL, b Integer, c Integer NOT NULL, d Integer, e Integer NOT NULL); +postgres# CREATE unique INDEX postgres_table_index on postgres_table(a, c, e); +postgres# ALTER TABLE postgres_table REPLICA IDENTITY USING INDEX postgres_table_index; +``` + +Первичный ключ всегда проверяется первым. Если он отсутствует, то проверяется индекс, определенный как индекс идентичности реплики. +Если индекс используется в качестве идентификатора реплики, то в таблице должен быть только один такой индекс. +Вы можете проверить, какой тип используется для указанной таблицы, выполнив следующую команду: + +``` bash +postgres# SELECT CASE relreplident + WHEN 'd' THEN 'default' + WHEN 'n' THEN 'nothing' + WHEN 'f' THEN 'full' + WHEN 'i' THEN 'index' + END AS replica_identity +FROM pg_class +WHERE oid = 'postgres_table'::regclass; +``` + +## Предупреждение {#warning} + +1. Конвертация значений **TOAST** не поддерживается. Для типа данных будет использоваться значение по умолчанию. diff --git a/docs/ru/sql-reference/statements/create/table.md b/docs/ru/sql-reference/statements/create/table.md index 1d65d82b24c..edddb1a9bb1 100644 --- a/docs/ru/sql-reference/statements/create/table.md +++ b/docs/ru/sql-reference/statements/create/table.md @@ -183,7 +183,7 @@ CREATE TABLE codec_example dt Date CODEC(ZSTD), ts DateTime CODEC(LZ4HC), float_value Float32 CODEC(NONE), - double_value Float64 CODEC(LZ4HC(9)) + double_value Float64 CODEC(LZ4HC(9)), value Float32 CODEC(Delta, ZSTD) ) ENGINE = From 935f838aebadb00e80c83723ab2f0b3fa3e4e00d Mon Sep 17 00:00:00 2001 From: Dmitriy Date: Wed, 21 Jul 2021 23:08:14 +0300 Subject: [PATCH 018/599] Update materialized-postgresql.md MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Исправил некоторые ошибки перевода. --- docs/ru/engines/database-engines/materialized-postgresql.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/ru/engines/database-engines/materialized-postgresql.md b/docs/ru/engines/database-engines/materialized-postgresql.md index 79304fca382..590d772ebd3 100644 --- a/docs/ru/engines/database-engines/materialized-postgresql.md +++ b/docs/ru/engines/database-engines/materialized-postgresql.md @@ -18,7 +18,7 @@ SELECT * FROM test_database.postgres_table; 1. `materialized_postgresql_max_block_size` — задает максимальное количество строк, собранных перед вставкой данных в таблицу. По умолчанию: `65536`. -2. `materialized_postgresql_tables_list` — список таблиц для движка баз данных `MaterializedPostgreSQL`. По умолчанию: `whole database`. +2. `materialized_postgresql_tables_list` — задает список таблиц для движка баз данных `MaterializedPostgreSQL`. По умолчанию: `whole database`. 3. `materialized_postgresql_allow_automatic_update` — позволяет автоматически обновить таблицу в фоновом режиме при обнаружении изменений схемы. По умолчанию: `0` (`false`). @@ -33,7 +33,7 @@ SELECT * FROM test_database.table1; ## Требования {#requirements} -- Настройка `wal_level` должна иметь значение `logical` и настройка `max_replication_slots` должна быть равна, по меньшей мере, `2` в конфигурационном файле в PostgreSQL. +- Настройка `wal_level` должна иметь значение `logical`, настройка `max_replication_slots` должна быть равна по меньшей мере `2` в конфигурационном файле в PostgreSQL. - Каждая реплицируемая таблица должна иметь один из следующих **идентификаторов реплики**: @@ -64,4 +64,4 @@ WHERE oid = 'postgres_table'::regclass; ## Предупреждение {#warning} -1. Конвертация значений **TOAST** не поддерживается. Для типа данных будет использоваться значение по умолчанию. +1. Преобразование **TOAST**-значений не поддерживается. Для типа данных будет использоваться значение по умолчанию. From 2df3e14559fa096d0b395dd8b29936fc1cc3a38e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 22 Jul 2021 02:56:17 +0300 Subject: [PATCH 019/599] Attempt to add memory tracking to RoaringBitmaps --- base/common/memory.h | 57 -------- contrib/croaring-cmake/CMakeLists.txt | 2 + .../AggregateFunctionGroupBitmapData.h | 9 +- src/CMakeLists.txt | 2 + src/Common/CurrentMemoryTracker.cpp | 9 ++ src/Common/CurrentMemoryTracker.h | 1 + src/Common/memory.h | 131 ++++++++++++++++++ src/Common/new_delete.cpp | 85 +----------- 8 files changed, 152 insertions(+), 144 deletions(-) delete mode 100644 base/common/memory.h create mode 100644 src/Common/memory.h diff --git a/base/common/memory.h b/base/common/memory.h deleted file mode 100644 index e82c019ceab..00000000000 --- a/base/common/memory.h +++ /dev/null @@ -1,57 +0,0 @@ -#pragma once - -#include -#include "defines.h" - -#if USE_JEMALLOC -# include -#endif - -#if !USE_JEMALLOC || JEMALLOC_VERSION_MAJOR < 4 -# include -#endif - - -namespace Memory -{ - -inline ALWAYS_INLINE void * newImpl(std::size_t size) -{ - auto * ptr = malloc(size); - if (likely(ptr != nullptr)) - return ptr; - - /// @note no std::get_new_handler logic implemented - throw std::bad_alloc{}; -} - -inline ALWAYS_INLINE void * newNoExept(std::size_t size) noexcept -{ - return malloc(size); -} - -inline ALWAYS_INLINE void deleteImpl(void * ptr) noexcept -{ - free(ptr); -} - -#if USE_JEMALLOC && JEMALLOC_VERSION_MAJOR >= 4 - -inline ALWAYS_INLINE void deleteSized(void * ptr, std::size_t size) noexcept -{ - if (unlikely(ptr == nullptr)) - return; - - sdallocx(ptr, size, 0); -} - -#else - -inline ALWAYS_INLINE void deleteSized(void * ptr, std::size_t size [[maybe_unused]]) noexcept -{ - free(ptr); -} - -#endif - -} diff --git a/contrib/croaring-cmake/CMakeLists.txt b/contrib/croaring-cmake/CMakeLists.txt index f4a5d8a01dc..522540e5e40 100644 --- a/contrib/croaring-cmake/CMakeLists.txt +++ b/contrib/croaring-cmake/CMakeLists.txt @@ -24,3 +24,5 @@ add_library(roaring ${SRCS}) target_include_directories(roaring PRIVATE "${LIBRARY_DIR}/include/roaring") target_include_directories(roaring SYSTEM BEFORE PUBLIC "${LIBRARY_DIR}/include") target_include_directories(roaring SYSTEM BEFORE PUBLIC "${LIBRARY_DIR}/cpp") + +target_compile_definitions(roaring PRIVATE -Dmalloc=clickhouse_malloc -Dfree=clickhouse_free -Dposix_memalign=clickhouse_posix_memalign) diff --git a/src/AggregateFunctions/AggregateFunctionGroupBitmapData.h b/src/AggregateFunctions/AggregateFunctionGroupBitmapData.h index 067daf6dc3a..d5484c4db79 100644 --- a/src/AggregateFunctions/AggregateFunctionGroupBitmapData.h +++ b/src/AggregateFunctions/AggregateFunctionGroupBitmapData.h @@ -7,6 +7,7 @@ #include #include #include +#include // Include this header last, because it is an auto-generated dump of questionable // garbage that breaks the build (e.g. it changes _POSIX_C_SOURCE). @@ -44,7 +45,7 @@ private: void toLarge() { - rb = std::make_unique(); + rb = std::make_shared(); for (const auto & x : small) rb->add(static_cast(x.getValue())); small.clear(); @@ -67,12 +68,14 @@ public: { toLarge(); rb->add(static_cast(value)); + CurrentMemoryTracker::check(); } } } else { rb->add(static_cast(value)); + CurrentMemoryTracker::check(); } } @@ -114,7 +117,7 @@ public: readVarUInt(size, in); std::unique_ptr buf(new char[size]); in.readStrict(buf.get(), size); - rb = std::make_unique(RoaringBitmap::read(buf.get())); + rb = std::make_shared(RoaringBitmap::read(buf.get())); } } @@ -141,7 +144,7 @@ public: */ std::shared_ptr getNewRoaringBitmapFromSmall() const { - std::shared_ptr ret = std::make_unique(); + std::shared_ptr ret = std::make_shared(); for (const auto & x : small) ret->add(static_cast(x.getValue())); return ret; diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 31286c740d4..9c1f884edc1 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -158,6 +158,8 @@ else() target_link_libraries (clickhouse_new_delete PRIVATE clickhouse_common_io jemalloc) endif() +target_link_libraries (clickhouse_common_io PRIVATE jemalloc) + add_subdirectory(Common/ZooKeeper) add_subdirectory(Common/Config) diff --git a/src/Common/CurrentMemoryTracker.cpp b/src/Common/CurrentMemoryTracker.cpp index d38a5a9c70c..1156c6e56bc 100644 --- a/src/Common/CurrentMemoryTracker.cpp +++ b/src/Common/CurrentMemoryTracker.cpp @@ -2,6 +2,8 @@ #include #include +#include + namespace { @@ -36,6 +38,7 @@ namespace if (current_thread) { current_thread->untracked_memory += size; + if (current_thread->untracked_memory > current_thread->untracked_memory_limit) { /// Zero untracked before track. If tracker throws out-of-limit we would be able to alloc up to untracked_memory_limit bytes @@ -54,6 +57,12 @@ namespace } } +void check() +{ + if (auto * memory_tracker = getMemoryTracker()) + memory_tracker->allocImpl(0, true); +} + void alloc(Int64 size) { bool throw_if_memory_exceeded = true; diff --git a/src/Common/CurrentMemoryTracker.h b/src/Common/CurrentMemoryTracker.h index 5090b7c3687..2f9ace4291f 100644 --- a/src/Common/CurrentMemoryTracker.h +++ b/src/Common/CurrentMemoryTracker.h @@ -9,4 +9,5 @@ namespace CurrentMemoryTracker void allocNoThrow(Int64 size); void realloc(Int64 old_size, Int64 new_size); void free(Int64 size); + void check(); } diff --git a/src/Common/memory.h b/src/Common/memory.h new file mode 100644 index 00000000000..0aa99cdf70d --- /dev/null +++ b/src/Common/memory.h @@ -0,0 +1,131 @@ +#pragma once + +#include +#include + +#include + +#if USE_JEMALLOC +# include +#endif + +#if !USE_JEMALLOC || JEMALLOC_VERSION_MAJOR < 4 +# include +#endif + + +namespace Memory +{ + +inline ALWAYS_INLINE void * newImpl(std::size_t size) +{ + auto * ptr = malloc(size); + if (likely(ptr != nullptr)) + return ptr; + + /// @note no std::get_new_handler logic implemented + throw std::bad_alloc{}; +} + +inline ALWAYS_INLINE void * newNoExept(std::size_t size) noexcept +{ + return malloc(size); +} + +inline ALWAYS_INLINE void deleteImpl(void * ptr) noexcept +{ + free(ptr); +} + +#if USE_JEMALLOC && JEMALLOC_VERSION_MAJOR >= 4 + +inline ALWAYS_INLINE void deleteSized(void * ptr, std::size_t size) noexcept +{ + if (unlikely(ptr == nullptr)) + return; + + sdallocx(ptr, size, 0); +} + +#else + +inline ALWAYS_INLINE void deleteSized(void * ptr, std::size_t size [[maybe_unused]]) noexcept +{ + free(ptr); +} + +#endif + +#if defined(OS_LINUX) +# include +#elif defined(OS_DARWIN) +# include +#endif + +#if defined(OS_DARWIN) && defined(BUNDLED_STATIC_JEMALLOC) +extern "C" +{ +extern void zone_register(); +} + +struct InitializeJemallocZoneAllocatorForOSX +{ + InitializeJemallocZoneAllocatorForOSX() + { + /// In case of OSX jemalloc register itself as a default zone allocator. + /// + /// But when you link statically then zone_register() will not be called, + /// and even will be optimized out: + /// + /// It is ok to call it twice (i.e. in case of shared libraries) + /// Since zone_register() is a no-op if the default zone is already replaced with something. + /// + /// https://github.com/jemalloc/jemalloc/issues/708 + zone_register(); + } +} initializeJemallocZoneAllocatorForOSX; +#endif + +inline ALWAYS_INLINE size_t getActualAllocationSize(size_t size) +{ + size_t actual_size = size; + +#if USE_JEMALLOC && JEMALLOC_VERSION_MAJOR >= 5 + /// The nallocx() function allocates no memory, but it performs the same size computation as the mallocx() function + /// @note je_mallocx() != je_malloc(). It's expected they don't differ much in allocation logic. + if (likely(size != 0)) + actual_size = nallocx(size, 0); +#endif + + return actual_size; +} + +inline ALWAYS_INLINE void trackMemory(std::size_t size) +{ + std::size_t actual_size = getActualAllocationSize(size); + CurrentMemoryTracker::allocNoThrow(actual_size); +} + +inline ALWAYS_INLINE void untrackMemory(void * ptr [[maybe_unused]], std::size_t size [[maybe_unused]] = 0) noexcept +{ + try + { +#if USE_JEMALLOC && JEMALLOC_VERSION_MAJOR >= 5 + /// @note It's also possible to use je_malloc_usable_size() here. + if (likely(ptr != nullptr)) + CurrentMemoryTracker::free(sallocx(ptr, 0)); +#else + if (size) + CurrentMemoryTracker::free(size); +# if defined(_GNU_SOURCE) + /// It's innaccurate resource free for sanitizers. malloc_usable_size() result is greater or equal to allocated size. + else + CurrentMemoryTracker::free(malloc_usable_size(ptr)); +# endif +#endif + } + catch (...) + {} +} + +} diff --git a/src/Common/new_delete.cpp b/src/Common/new_delete.cpp index 56173fb108a..fa32d56b350 100644 --- a/src/Common/new_delete.cpp +++ b/src/Common/new_delete.cpp @@ -1,117 +1,34 @@ -#include -#include - -#include +#include #include -#if defined(OS_LINUX) -# include -#elif defined(OS_DARWIN) -# include -#endif - -#if defined(OS_DARWIN) && defined(BUNDLED_STATIC_JEMALLOC) -extern "C" -{ -extern void zone_register(); -} - -struct InitializeJemallocZoneAllocatorForOSX -{ - InitializeJemallocZoneAllocatorForOSX() - { - /// In case of OSX jemalloc register itself as a default zone allocator. - /// - /// But when you link statically then zone_register() will not be called, - /// and even will be optimized out: - /// - /// It is ok to call it twice (i.e. in case of shared libraries) - /// Since zone_register() is a no-op if the default zone is already replaced with something. - /// - /// https://github.com/jemalloc/jemalloc/issues/708 - zone_register(); - } -} initializeJemallocZoneAllocatorForOSX; -#endif /// Replace default new/delete with memory tracking versions. /// @sa https://en.cppreference.com/w/cpp/memory/new/operator_new /// https://en.cppreference.com/w/cpp/memory/new/operator_delete -namespace Memory -{ - -inline ALWAYS_INLINE size_t getActualAllocationSize(size_t size) -{ - size_t actual_size = size; - -#if USE_JEMALLOC && JEMALLOC_VERSION_MAJOR >= 5 - /// The nallocx() function allocates no memory, but it performs the same size computation as the mallocx() function - /// @note je_mallocx() != je_malloc(). It's expected they don't differ much in allocation logic. - if (likely(size != 0)) - actual_size = nallocx(size, 0); -#endif - - return actual_size; -} - -inline ALWAYS_INLINE void trackMemory(std::size_t size) -{ - std::size_t actual_size = getActualAllocationSize(size); - CurrentMemoryTracker::allocNoThrow(actual_size); -} - -inline ALWAYS_INLINE void untrackMemory(void * ptr [[maybe_unused]], std::size_t size [[maybe_unused]] = 0) noexcept -{ - try - { -#if USE_JEMALLOC && JEMALLOC_VERSION_MAJOR >= 5 - /// @note It's also possible to use je_malloc_usable_size() here. - if (likely(ptr != nullptr)) - CurrentMemoryTracker::free(sallocx(ptr, 0)); -#else - if (size) - CurrentMemoryTracker::free(size); -# if defined(_GNU_SOURCE) - /// It's innaccurate resource free for sanitizers. malloc_usable_size() result is greater or equal to allocated size. - else - CurrentMemoryTracker::free(malloc_usable_size(ptr)); -# endif -#endif - } - catch (...) - {} -} - -} - /// new void * operator new(std::size_t size) { Memory::trackMemory(size); - return Memory::newImpl(size); } void * operator new[](std::size_t size) { Memory::trackMemory(size); - return Memory::newImpl(size); } void * operator new(std::size_t size, const std::nothrow_t &) noexcept { Memory::trackMemory(size); - return Memory::newNoExept(size); } void * operator new[](std::size_t size, const std::nothrow_t &) noexcept { Memory::trackMemory(size); - return Memory::newNoExept(size); } From ca99d8aa0275ec242bbd1deb5d584e6f8849eee8 Mon Sep 17 00:00:00 2001 From: Dmitriy Date: Sun, 25 Jul 2021 23:14:37 +0300 Subject: [PATCH 020/599] Fix documentation MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Поправил документацию. --- .../database-engines/materialized-postgresql.md | 13 ++++++------- .../database-engines/materialized-postgresql.md | 13 ++++++------- 2 files changed, 12 insertions(+), 14 deletions(-) diff --git a/docs/en/engines/database-engines/materialized-postgresql.md b/docs/en/engines/database-engines/materialized-postgresql.md index c53a6c88aba..9112cc7773b 100644 --- a/docs/en/engines/database-engines/materialized-postgresql.md +++ b/docs/en/engines/database-engines/materialized-postgresql.md @@ -33,13 +33,13 @@ SELECT * FROM test_database.table1; ## Requirements {#requirements} -- Setting `wal_level` to `logical` and `max_replication_slots` to at least `2` in the PostgreSQL config file. +1. Setting [wal_level](https://www.postgresql.org/docs/current/runtime-config-wal.html) to `logical` and `max_replication_slots` to at least `2` in the PostgreSQL config file. -- Each replicated table must have one of the following **replica identity**: +2. Each replicated table must have one of the following **replica identity**: -1. **default** (primary key) +- primary key (by default) -2. **index** +- index ``` bash postgres# CREATE TABLE postgres_table (a Integer NOT NULL, b Integer, c Integer NOT NULL, d Integer, e Integer NOT NULL); @@ -62,6 +62,5 @@ FROM pg_class WHERE oid = 'postgres_table'::regclass; ``` -## Warning {#warning} - -1. **TOAST** values conversion is not supported. Default value for the data type will be used. +!!! warning "Warning" + **TOAST** values conversion is not supported. Default value for the data type will be used. diff --git a/docs/ru/engines/database-engines/materialized-postgresql.md b/docs/ru/engines/database-engines/materialized-postgresql.md index 590d772ebd3..71432f0aaeb 100644 --- a/docs/ru/engines/database-engines/materialized-postgresql.md +++ b/docs/ru/engines/database-engines/materialized-postgresql.md @@ -33,13 +33,13 @@ SELECT * FROM test_database.table1; ## Требования {#requirements} -- Настройка `wal_level` должна иметь значение `logical`, настройка `max_replication_slots` должна быть равна по меньшей мере `2` в конфигурационном файле в PostgreSQL. +1. Настройка [wal_level](https://postgrespro.ru/docs/postgrespro/10/runtime-config-wal) должна иметь значение `logical`, параметр `max_replication_slots` должен быть равен по меньшей мере `2` в конфигурационном файле в PostgreSQL. -- Каждая реплицируемая таблица должна иметь один из следующих **идентификаторов реплики**: +2. Каждая реплицируемая таблица должна иметь один из следующих **идентификаторов реплики**: -1. **по умолчанию** (первичный ключ) +- первичный ключ (по умолчанию) -2. **индекс** +- индекс ``` bash postgres# CREATE TABLE postgres_table (a Integer NOT NULL, b Integer, c Integer NOT NULL, d Integer, e Integer NOT NULL); @@ -62,6 +62,5 @@ FROM pg_class WHERE oid = 'postgres_table'::regclass; ``` -## Предупреждение {#warning} - -1. Преобразование **TOAST**-значений не поддерживается. Для типа данных будет использоваться значение по умолчанию. +!!! warning "Предупреждение" + Преобразование **TOAST**-значений не поддерживается. Для типа данных будет использоваться значение по умолчанию. From 2d51f4db44c0de05aa98bb977d633fa48b19af31 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Sun, 25 Jul 2021 23:29:37 +0300 Subject: [PATCH 021/599] Update materialized-postgresql.md --- docs/en/engines/database-engines/materialized-postgresql.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/en/engines/database-engines/materialized-postgresql.md b/docs/en/engines/database-engines/materialized-postgresql.md index 9112cc7773b..4ef2297dd89 100644 --- a/docs/en/engines/database-engines/materialized-postgresql.md +++ b/docs/en/engines/database-engines/materialized-postgresql.md @@ -16,11 +16,11 @@ SELECT * FROM test_database.postgres_table; ## Settings {#settings} -1. `materialized_postgresql_max_block_size` — Number of rows collected before flushing data into table. Default: `65536`. +1. `materialized_postgresql_max_block_size` — Number of rows collected in memory before flushing data into table. Default: `65536`. -2. `materialized_postgresql_tables_list` — List of tables for MaterializedPostgreSQL database engine. Default: `whole database`. +2. `materialized_postgresql_tables_list` — A comma-separated list of PostgreSQL database tables, which will be replicated via MaterializedPostgreSQL database engine. Default: empty list - means whole PostgreSQL database will be replicated. -3. `materialized_postgresql_allow_automatic_update` — Allow to reload table in the background, when schema changes are detected. Default: `0` (`false`). +3. `materialized_postgresql_allow_automatic_update` — Allow to reload table in the background, when schema changes are detected. Default: `0` (`false`). DDL queries on PostgreSQL side are not replicated via ClickHouse `MaterializedPostgreSQL` engine, because it is not allowed with PostgreSQL logical replication protocol, but the fact of DDL changes is detected transactioanlly. In this case the default behaviour is to stop replicating those tables once DDL is detected. However, if this setting is enabled, then, instead of stopping replication of those tables, they will be reloaded in the background via database snapshot without data losses and replication will continue for them. ``` sql CREATE DATABASE test_database From 5b799ef2943e2413962baad59b0abbf3487ef785 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Sun, 25 Jul 2021 23:33:17 +0300 Subject: [PATCH 022/599] Update materialized-postgresql.md --- docs/en/engines/database-engines/materialized-postgresql.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/engines/database-engines/materialized-postgresql.md b/docs/en/engines/database-engines/materialized-postgresql.md index 4ef2297dd89..54343dce59c 100644 --- a/docs/en/engines/database-engines/materialized-postgresql.md +++ b/docs/en/engines/database-engines/materialized-postgresql.md @@ -63,4 +63,4 @@ WHERE oid = 'postgres_table'::regclass; ``` !!! warning "Warning" - **TOAST** values conversion is not supported. Default value for the data type will be used. + Replication of [**TOAST**](https://www.postgresql.org/docs/9.5/storage-toast.html) values is not supported. Default value for the data type will be used. From d801e6a5387fa94b8b9f6545abff0557bd8400f2 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Sun, 25 Jul 2021 23:38:14 +0300 Subject: [PATCH 023/599] Update materialized-postgresql.md --- docs/en/engines/database-engines/materialized-postgresql.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/engines/database-engines/materialized-postgresql.md b/docs/en/engines/database-engines/materialized-postgresql.md index 54343dce59c..778cfff19b8 100644 --- a/docs/en/engines/database-engines/materialized-postgresql.md +++ b/docs/en/engines/database-engines/materialized-postgresql.md @@ -20,7 +20,7 @@ SELECT * FROM test_database.postgres_table; 2. `materialized_postgresql_tables_list` — A comma-separated list of PostgreSQL database tables, which will be replicated via MaterializedPostgreSQL database engine. Default: empty list - means whole PostgreSQL database will be replicated. -3. `materialized_postgresql_allow_automatic_update` — Allow to reload table in the background, when schema changes are detected. Default: `0` (`false`). DDL queries on PostgreSQL side are not replicated via ClickHouse `MaterializedPostgreSQL` engine, because it is not allowed with PostgreSQL logical replication protocol, but the fact of DDL changes is detected transactioanlly. In this case the default behaviour is to stop replicating those tables once DDL is detected. However, if this setting is enabled, then, instead of stopping replication of those tables, they will be reloaded in the background via database snapshot without data losses and replication will continue for them. +3. `materialized_postgresql_allow_automatic_update` — Allow to reload table in the background, when schema changes are detected. Default: `0` (`false`). DDL queries on PostgreSQL side are not replicated via ClickHouse `MaterializedPostgreSQL` engine, because it is not allowed with PostgreSQL logical replication protocol, but the fact of DDL changes is detected transactionally. In this case the default behaviour is to stop replicating those tables once DDL is detected. However, if this setting is enabled, then, instead of stopping replication of those tables, they will be reloaded in the background via database snapshot without data losses and replication will continue for them. ``` sql CREATE DATABASE test_database @@ -35,7 +35,7 @@ SELECT * FROM test_database.table1; 1. Setting [wal_level](https://www.postgresql.org/docs/current/runtime-config-wal.html) to `logical` and `max_replication_slots` to at least `2` in the PostgreSQL config file. -2. Each replicated table must have one of the following **replica identity**: +2. Each replicated table must have one of the following [replica identity](https://www.postgresql.org/docs/10/sql-altertable.html#SQL-CREATETABLE-REPLICA-IDENTITY): - primary key (by default) From f1e98bfa868df7f0054ef7e031b014d965e7913d Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Sun, 25 Jul 2021 23:43:12 +0300 Subject: [PATCH 024/599] Update materialized-postgresql.md --- docs/ru/engines/database-engines/materialized-postgresql.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/engines/database-engines/materialized-postgresql.md b/docs/ru/engines/database-engines/materialized-postgresql.md index 71432f0aaeb..c627041cc8a 100644 --- a/docs/ru/engines/database-engines/materialized-postgresql.md +++ b/docs/ru/engines/database-engines/materialized-postgresql.md @@ -35,7 +35,7 @@ SELECT * FROM test_database.table1; 1. Настройка [wal_level](https://postgrespro.ru/docs/postgrespro/10/runtime-config-wal) должна иметь значение `logical`, параметр `max_replication_slots` должен быть равен по меньшей мере `2` в конфигурационном файле в PostgreSQL. -2. Каждая реплицируемая таблица должна иметь один из следующих **идентификаторов реплики**: +2. Каждая реплицируемая таблица должна иметь один из следующих [репликационных идентификаторов](https://postgrespro.ru/docs/postgresql/10/sql-altertable#SQL-CREATETABLE-REPLICA-IDENTITY): - первичный ключ (по умолчанию) From 933d0b7e3b2e091181c65c70a1bffa7a50178e3e Mon Sep 17 00:00:00 2001 From: Vladimir Goncharov Date: Mon, 26 Jul 2021 17:56:40 +0300 Subject: [PATCH 025/599] Update docs/en/engines/table-engines/mergetree-family/mergetree.md Co-authored-by: Alexey Boykov <33257111+mathalex@users.noreply.github.com> --- docs/en/engines/table-engines/mergetree-family/mergetree.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/engines/table-engines/mergetree-family/mergetree.md b/docs/en/engines/table-engines/mergetree-family/mergetree.md index 177b0bfde99..c49135bbbc5 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -399,7 +399,7 @@ A projection query is what defines a projection. It has the following grammar: It implicitly selects data from the parent table. #### Storage {#projection-storage} -Projections are stored inside the part directory. It's similar to an index but contains an subdirectory which stores an anonymous MergeTree table's part. The table is induced by the definition query of the projection. If there is a GROUP BY clause, the underlying storage engine becomes AggregatedMergeTree, and all aggregate functions are converted to AggregateFunction. If there is an ORDER BY clause, the MergeTree table will use it as its primary key expression. During the merge process, the projection part will be merged via its storage's merge routine. The checksum of the parent table's part will combine the projection's part. Other maintenance jobs are similar to skip indices. +Projections are stored inside the part directory. It's similar to an index but contains a subdirectory that stores an anonymous MergeTree table's part. The table is induced by the definition query of the projection. If there is a GROUP BY clause, the underlying storage engine becomes AggregatedMergeTree, and all aggregate functions are converted to AggregateFunction. If there is an ORDER BY clause, the MergeTree table will use it as its primary key expression. During the merge process, the projection part will be merged via its storage's merge routine. The checksum of the parent table's part will combine the projection's part. Other maintenance jobs are similar to skip indices. #### Query Analysis {#projection-query-analysis} 1. Check if the projection can be used to answer the given query, that is, it generates the same answer as querying the base table. From 0cf20f84824157c0bb1d089e2e106343b540cd25 Mon Sep 17 00:00:00 2001 From: Vladimir Goncharov Date: Mon, 26 Jul 2021 17:56:46 +0300 Subject: [PATCH 026/599] Update docs/en/engines/table-engines/mergetree-family/mergetree.md Co-authored-by: Alexey Boykov <33257111+mathalex@users.noreply.github.com> --- docs/en/engines/table-engines/mergetree-family/mergetree.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/engines/table-engines/mergetree-family/mergetree.md b/docs/en/engines/table-engines/mergetree-family/mergetree.md index c49135bbbc5..d9ff302a866 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -389,7 +389,7 @@ Functions with a constant argument that is less than ngram size can’t be used - `NOT startsWith(s, 'test')` ### Projections {#projections} -Projections are like materialized views, but defined in part-level. It provides consistency guarantees along with automatic usage in queries. +Projections are like materialized views but defined in part-level. It provides consistency guarantees along with automatic usage in queries. #### Query {#projection-query} A projection query is what defines a projection. It has the following grammar: From b1c0509579169e8b2cc4a7121f7b9fd8aa6dd103 Mon Sep 17 00:00:00 2001 From: Dmitriy Date: Mon, 26 Jul 2021 22:32:00 +0300 Subject: [PATCH 027/599] Fix description MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Добавил общее описание движка. --- docs/en/engines/database-engines/materialized-postgresql.md | 6 ++++++ docs/ru/engines/database-engines/materialized-postgresql.md | 6 ++++++ 2 files changed, 12 insertions(+) diff --git a/docs/en/engines/database-engines/materialized-postgresql.md b/docs/en/engines/database-engines/materialized-postgresql.md index 778cfff19b8..4b8525efdcc 100644 --- a/docs/en/engines/database-engines/materialized-postgresql.md +++ b/docs/en/engines/database-engines/materialized-postgresql.md @@ -5,6 +5,12 @@ toc_title: MaterializedPostgreSQL # MaterializedPostgreSQL {#materialize-postgresql} +Creates ClickHouse database with all the tables existing in PostgreSQL, and all the data in those tables. + +ClickHouse server works as PostgreSQL replica. It reads binlog and performs DDL and DML queries. + +This feature is experimental. + ## Creating a Database {#creating-a-database} ``` sql diff --git a/docs/ru/engines/database-engines/materialized-postgresql.md b/docs/ru/engines/database-engines/materialized-postgresql.md index c627041cc8a..8c366ba14b6 100644 --- a/docs/ru/engines/database-engines/materialized-postgresql.md +++ b/docs/ru/engines/database-engines/materialized-postgresql.md @@ -5,6 +5,12 @@ toc_title: MaterializedPostgreSQL # MaterializedPostgreSQL {#materialize-postgresql} +Создает базу данных ClickHouse со всеми таблицами, существующими в PostgreSQL, и всеми данными в этих таблицах. + +Сервер ClickHouse работает как реплика PostgreSQL. Он читает файл binlog и выполняет DDL and DML-запросы. + +`MaterializedPostgreSQL` — экспериментальный движок баз данных. + ## Создание базы данных {#creating-a-database} ``` sql From cf8c35395d32cf776b284ed5115e97685d5e385e Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Tue, 27 Jul 2021 16:42:44 +0300 Subject: [PATCH 028/599] Test new PR. --- docs/en/operations/settings/settings.md | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 9a0aa0af159..1646794914d 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -109,6 +109,10 @@ Enables or disables [fsync](http://pubs.opengroup.org/onlinepubs/9699919799/func It makes sense to disable it if the server has millions of tiny tables that are constantly being created and destroyed. +## function_range_max_elements_in_block {#settings-function_range_max_elements_in_block} + + + ## enable_http_compression {#settings-enable_http_compression} Enables or disables data compression in the response to an HTTP request. From 07c71c226e96ea7510844a35115f6c0e9e33c3ca Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 15 Jun 2021 19:10:41 +0200 Subject: [PATCH 029/599] Add QueryMaterializationLog class --- src/Interpreters/QueryMaterializationLog.cpp | 97 ++++++++++++++++++++ src/Interpreters/QueryMaterializationLog.h | 61 ++++++++++++ src/Interpreters/SystemLog.h | 3 + src/Interpreters/ya.make | 1 + 4 files changed, 162 insertions(+) create mode 100644 src/Interpreters/QueryMaterializationLog.cpp create mode 100644 src/Interpreters/QueryMaterializationLog.h diff --git a/src/Interpreters/QueryMaterializationLog.cpp b/src/Interpreters/QueryMaterializationLog.cpp new file mode 100644 index 00000000000..a900a70ac5f --- /dev/null +++ b/src/Interpreters/QueryMaterializationLog.cpp @@ -0,0 +1,97 @@ +#include "QueryMaterializationLog.h" + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ +Block QueryMaterializationLogElement::createBlock() +{ + auto query_status_datatype = std::make_shared(DataTypeEnum8::Values{ + {"QueryStart", static_cast(QUERY_START)}, + {"QueryFinish", static_cast(QUERY_FINISH)}, + {"ExceptionBeforeStart", static_cast(EXCEPTION_BEFORE_START)}, + {"ExceptionWhileProcessing", static_cast(EXCEPTION_WHILE_PROCESSING)}}); + + return { + {std::make_shared(), "event_date"}, + {std::make_shared(), "event_time"}, + {std::make_shared(6), "event_time_microseconds"}, + {std::make_shared(), "materialization_start_time"}, + {std::make_shared(6), "materialization_start_time_microseconds"}, + {std::make_shared(), "materialization_duration_ms"}, + + {std::make_shared(), "initial_query_id"}, + {std::make_shared(), "materialization_name"}, + {std::make_shared(), "materialization_uuid"}, + {std::make_shared(), "materialization_query"}, + + {std::make_shared(), "read_rows"}, + {std::make_shared(), "read_bytes"}, + {std::make_shared(), "written_rows"}, + {std::make_shared(), "written_bytes"}, + {std::make_shared(), "memory_usage"}, + {std::make_shared(), "peak_memory_usage"}, + {std::make_shared(std::make_shared()), "ProfileEvents.Names"}, + {std::make_shared(std::make_shared()), "ProfileEvents.Values"}, + + {std::move(query_status_datatype), "end_status"}, + {std::make_shared(), "exception_code"}, + {std::make_shared(), "exception"}, + {std::make_shared(), "stack_trace"}}; +} + +void QueryMaterializationLogElement::appendToBlock(MutableColumns & columns) const +{ + size_t i = 0; + + columns[i++]->insert(DateLUT::instance().toDayNum(event_time).toUnderType()); // event_date + columns[i++]->insert(event_time); + columns[i++]->insert(event_time_microseconds); + columns[i++]->insert(materialization_start_time); + columns[i++]->insert(materialization_start_time_microseconds); + columns[i++]->insert(materialization_duration_ms); + + columns[i++]->insertData(initial_query_id.data(), initial_query_id.size()); + columns[i++]->insertData(materialization_name.data(), materialization_name.size()); + columns[i++]->insert(materialization_uuid); + columns[i++]->insertData(materialization_query.data(), materialization_query.size()); + + columns[i++]->insert(read_rows); + columns[i++]->insert(read_bytes); + columns[i++]->insert(written_rows); + columns[i++]->insert(written_bytes); + columns[i++]->insert(memory_usage); + columns[i++]->insert(peak_memory_usage); + + if (profile_counters) + { + auto * column_names = columns[i++].get(); + auto * column_values = columns[i++].get(); + ProfileEvents::dumpToArrayColumns(*profile_counters, column_names, column_values, true); + } + else + { + columns[i++]->insertDefault(); + columns[i++]->insertDefault(); + } + + columns[i++]->insert(end_status); + columns[i++]->insert(exception_code); + columns[i++]->insertData(exception.data(), exception.size()); + columns[i++]->insertData(stack_trace.data(), stack_trace.size()); +} + +} diff --git a/src/Interpreters/QueryMaterializationLog.h b/src/Interpreters/QueryMaterializationLog.h new file mode 100644 index 00000000000..ce642cd1126 --- /dev/null +++ b/src/Interpreters/QueryMaterializationLog.h @@ -0,0 +1,61 @@ +#pragma once + +#include +#include + +#include +#include +#include +#include +#include +#include +#include + +namespace ProfileEvents +{ +class Counters; +} + +namespace DB +{ +struct QueryMaterializationLogElement +{ + using Status = QueryLogElementType; + + time_t event_time{}; + Decimal64 event_time_microseconds{}; + time_t materialization_start_time{}; + Decimal64 materialization_start_time_microseconds{}; + UInt64 materialization_duration_ms{}; + + String initial_query_id; + String materialization_name; + UUID materialization_uuid{UUIDHelpers::Nil}; + String materialization_query; + + UInt64 read_rows{}; + UInt64 read_bytes{}; + UInt64 written_rows{}; + UInt64 written_bytes{}; + Int64 memory_usage{}; + Int64 peak_memory_usage{}; + std::shared_ptr profile_counters; + + Status end_status{EXCEPTION_BEFORE_START}; + Int32 exception_code{}; + String exception; + String stack_trace; + + static std::string name() { return "QueryMutationLog"; } + + static Block createBlock(); + void appendToBlock(MutableColumns & columns) const; +}; + + +class QueryMaterializationLog : public SystemLog +{ + using SystemLog::SystemLog; +}; + +} diff --git a/src/Interpreters/SystemLog.h b/src/Interpreters/SystemLog.h index ee3116362e5..b95cc532236 100644 --- a/src/Interpreters/SystemLog.h +++ b/src/Interpreters/SystemLog.h @@ -74,6 +74,7 @@ class CrashLog; class MetricLog; class AsynchronousMetricLog; class OpenTelemetrySpanLog; +class QueryMaterializationLog; class ISystemLog @@ -110,6 +111,8 @@ struct SystemLogs std::shared_ptr asynchronous_metric_log; /// OpenTelemetry trace spans. std::shared_ptr opentelemetry_span_log; + /// Used to log queries of materialized views + std::shared_ptr query_materialization_log; std::vector logs; }; diff --git a/src/Interpreters/ya.make b/src/Interpreters/ya.make index 17157fe3a8c..5b0c6f0d9b7 100644 --- a/src/Interpreters/ya.make +++ b/src/Interpreters/ya.make @@ -127,6 +127,7 @@ SRCS( ProfileEventsExt.cpp QueryAliasesVisitor.cpp QueryLog.cpp + QueryMaterializationLog.cpp QueryNormalizer.cpp QueryParameterVisitor.cpp QueryThreadLog.cpp From c5b14f50751ce5cd7a4c88eff4e44b8f2394ba69 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 16 Jun 2021 11:45:43 +0200 Subject: [PATCH 030/599] Add materialization log to SystemLog --- src/Interpreters/SystemLog.cpp | 19 ++++++++++++------- 1 file changed, 12 insertions(+), 7 deletions(-) diff --git a/src/Interpreters/SystemLog.cpp b/src/Interpreters/SystemLog.cpp index 31ceca8ec05..71190a84a83 100644 --- a/src/Interpreters/SystemLog.cpp +++ b/src/Interpreters/SystemLog.cpp @@ -1,13 +1,14 @@ -#include -#include -#include -#include -#include -#include +#include #include #include -#include #include +#include +#include +#include +#include +#include +#include +#include #include #include @@ -103,6 +104,8 @@ SystemLogs::SystemLogs(ContextPtr global_context, const Poco::Util::AbstractConf opentelemetry_span_log = createSystemLog( global_context, "system", "opentelemetry_span_log", config, "opentelemetry_span_log"); + query_materialization_log = createSystemLog( + global_context, "system", "query_materialization_log", config, "query_materialization_log"); if (query_log) logs.emplace_back(query_log.get()); @@ -122,6 +125,8 @@ SystemLogs::SystemLogs(ContextPtr global_context, const Poco::Util::AbstractConf logs.emplace_back(asynchronous_metric_log.get()); if (opentelemetry_span_log) logs.emplace_back(opentelemetry_span_log.get()); + if (query_materialization_log) + logs.emplace_back(query_materialization_log.get()); try { From f34cb886ccc25170d0949ba3d6ed5c5f8c585757 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 16 Jun 2021 12:26:18 +0200 Subject: [PATCH 031/599] Add materialization log accessor to Context --- src/Interpreters/Context.cpp | 9 +++++++++ src/Interpreters/Context.h | 2 ++ 2 files changed, 11 insertions(+) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 3e080eb4b4f..5743164f280 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -2052,6 +2052,15 @@ std::shared_ptr Context::getQueryLog() const return shared->system_logs->query_log; } +std::shared_ptr Context::getQueryMaterializationLog() const +{ + auto lock = getLock(); + + if (!shared->system_logs) + return {}; + + return shared->system_logs->query_materialization_log; +} std::shared_ptr Context::getQueryThreadLog() const { diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 2c500c4166b..17d27b45b25 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -69,6 +69,7 @@ class Macros; struct Progress; class Clusters; class QueryLog; +class QueryMaterializationLog; class QueryThreadLog; class PartLog; class TextLog; @@ -708,6 +709,7 @@ public: /// Nullptr if the query log is not ready for this moment. std::shared_ptr getQueryLog() const; + std::shared_ptr getQueryMaterializationLog() const; std::shared_ptr getQueryThreadLog() const; std::shared_ptr getTraceLog() const; std::shared_ptr getTextLog() const; From bc1ccd2d62596b826e090e956675686a8ba8931f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 16 Jun 2021 12:42:10 +0200 Subject: [PATCH 032/599] Flush materialization log on FLUSH_LOGS --- src/Interpreters/InterpreterSystemQuery.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index a682a029258..bcdd2225d68 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -19,6 +19,7 @@ #include #include #include +#include #include #include #include @@ -417,6 +418,7 @@ BlockIO InterpreterSystemQuery::execute() [&] { if (auto metric_log = getContext()->getMetricLog()) metric_log->flush(true); }, [&] { if (auto asynchronous_metric_log = getContext()->getAsynchronousMetricLog()) asynchronous_metric_log->flush(true); }, [&] { if (auto opentelemetry_span_log = getContext()->getOpenTelemetrySpanLog()) opentelemetry_span_log->flush(true); } + [&] () { if (auto query_materialization_log = getContext()->getQueryMaterializationLog()) query_materialization_log->flush(true); } ); break; } From ea5c02a605cef90a57e618bcbde61ad8977880a2 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 18 Jun 2021 15:44:08 +0200 Subject: [PATCH 033/599] WIP --- programs/server/config.xml | 8 + src/Common/ThreadStatus.h | 4 + src/Core/Settings.h | 3 +- .../PushingToViewsBlockOutputStream.cpp | 291 +++++++++++------- .../PushingToViewsBlockOutputStream.h | 21 +- src/Interpreters/Context.cpp | 19 +- src/Interpreters/Context.h | 4 +- src/Interpreters/InterpreterSystemQuery.cpp | 4 +- ...terializationLog.cpp => QueryViewsLog.cpp} | 35 ++- ...ryMaterializationLog.h => QueryViewsLog.h} | 39 ++- src/Interpreters/SystemLog.cpp | 9 +- src/Interpreters/SystemLog.h | 6 +- src/Interpreters/ThreadStatusExt.cpp | 44 ++- src/Interpreters/ya.make | 2 +- 14 files changed, 321 insertions(+), 168 deletions(-) rename src/Interpreters/{QueryMaterializationLog.cpp => QueryViewsLog.cpp} (73%) rename src/Interpreters/{QueryMaterializationLog.h => QueryViewsLog.h} (51%) diff --git a/programs/server/config.xml b/programs/server/config.xml index 6f0b228dda7..3f7f6c25e24 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -955,6 +955,14 @@ 1000 + + + system + query_views_log
+ toYYYYMM(event_date) + 7500 +
+ diff --git a/src/Common/ThreadStatus.h b/src/Common/ThreadStatus.h index 6fc43114621..86aa8cc52a1 100644 --- a/src/Common/ThreadStatus.h +++ b/src/Common/ThreadStatus.h @@ -37,6 +37,8 @@ struct RUsageCounters; struct PerfEventsCounters; class TaskStatsInfoGetter; class InternalTextLogsQueue; +struct ViewInfo; +class QueryViewsLog; using InternalTextLogsQueuePtr = std::shared_ptr; using InternalTextLogsQueueWeakPtr = std::weak_ptr; @@ -213,6 +215,8 @@ public: /// Detaches thread from the thread group and the query, dumps performance counters if they have not been dumped void detachQuery(bool exit_if_already_detached = false, bool thread_exits = false); + void logToQueryViewsLog(QueryViewsLog & views_log, const ViewInfo & vinfo); + protected: void applyQuerySettings(); diff --git a/src/Core/Settings.h b/src/Core/Settings.h index e53db255d20..54e1f00832c 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -351,7 +351,8 @@ class IColumn; \ M(Bool, log_profile_events, true, "Log query performance statistics into the query_log and query_thread_log.", 0) \ M(Bool, log_query_settings, true, "Log query settings into the query_log.", 0) \ - M(Bool, log_query_threads, true, "Log query threads into system.query_thread_log table. This setting have effect only when 'log_queries' is true.", 0) \ + M(Bool, log_query_threads, true, "Log query threads into system.query_thread_log table. This setting have effect only when 'log_queries' is true.", 0) \ + M(Bool, log_query_views, false, "Log query dependent views into system.query_views_log table. This setting have effect only when 'log_queries' is true.", 0) \ M(String, log_comment, "", "Log comment into system.query_log table and server log. It can be set to arbitrary string no longer than max_query_size.", 0) \ M(LogsLevel, send_logs_level, LogsLevel::fatal, "Send server text logs with specified minimum level to client. Valid values: 'trace', 'debug', 'information', 'warning', 'error', 'fatal', 'none'", 0) \ M(Bool, enable_optimize_predicate_expression, 1, "If it is set to true, optimize predicates to subqueries.", 0) \ diff --git a/src/DataStreams/PushingToViewsBlockOutputStream.cpp b/src/DataStreams/PushingToViewsBlockOutputStream.cpp index 7729eb5fb44..2501d4dd42e 100644 --- a/src/DataStreams/PushingToViewsBlockOutputStream.cpp +++ b/src/DataStreams/PushingToViewsBlockOutputStream.cpp @@ -1,25 +1,25 @@ #include +#include +#include #include #include -#include -#include #include #include -#include -#include #include +#include +#include #include -#include -#include -#include -#include -#include +#include +#include #include #include #include #include #include +#include +#include + namespace DB { @@ -79,9 +79,12 @@ PushingToViewsBlockOutputStream::PushingToViewsBlockOutputStream( ASTPtr query; BlockOutputStreamPtr out; + QueryViewsLogElement::ViewType type = QueryViewsLogElement::ViewType::DEFAULT; + String target_name = database_table.getNameForLogs(); if (auto * materialized_view = dynamic_cast(dependent_table.get())) { + type = QueryViewsLogElement::ViewType::MATERIALIZED; addTableLock( materialized_view->lockForShare(getContext()->getInitialQueryId(), getContext()->getSettingsRef().lock_acquire_timeout)); @@ -89,6 +92,7 @@ PushingToViewsBlockOutputStream::PushingToViewsBlockOutputStream( auto inner_table_id = inner_table->getStorageID(); auto inner_metadata_snapshot = inner_table->getInMemoryMetadataPtr(); query = dependent_metadata_snapshot->getSelectQuery().inner_query; + target_name = inner_table_id.getNameForLogs(); std::unique_ptr insert = std::make_unique(); insert->table_id = inner_table_id; @@ -113,15 +117,27 @@ PushingToViewsBlockOutputStream::PushingToViewsBlockOutputStream( InterpreterInsertQuery interpreter(insert_query_ptr, insert_context); BlockIO io = interpreter.execute(); out = io.out; + LOG_WARNING( + log, + "Pushing from {} to {} {}.", + storage->getStorageID().getNameForLogs(), + inner_table_id.getNameForLogs(), + inner_table->getStorageID().getFullTableName()); } - else if (dynamic_cast(dependent_table.get())) + else if (auto * live_view = dynamic_cast(dependent_table.get())) + { + type = QueryViewsLogElement::ViewType::LIVE; + query = live_view->getInnerQuery(); // TODO: Optimize this out = std::make_shared( dependent_table, dependent_metadata_snapshot, insert_context, ASTPtr(), true); + } else out = std::make_shared( dependent_table, dependent_metadata_snapshot, insert_context, ASTPtr()); - views.emplace_back(ViewInfo{std::move(query), database_table, std::move(out), nullptr, 0 /* elapsed_ms */}); + QueryViewsLogElement::ViewRuntimeStats runtime_stats{ + 0, type, std::make_shared(), select_context->getInitialQueryId(), std::chrono::system_clock::now(), target_name}; + views.emplace_back(ViewInfo{std::move(query), database_table, std::move(out), nullptr, std::move(runtime_stats)}); } /// Do not push to destination table if the flag is set @@ -169,41 +185,64 @@ void PushingToViewsBlockOutputStream::write(const Block & block) output->write(block); } + if (!views.size()) + return; + /// Don't process materialized views if this block is duplicate if (!getContext()->getSettingsRef().deduplicate_blocks_in_dependent_materialized_views && replicated_output && replicated_output->lastBlockIsDuplicate()) return; - // Insert data into materialized views only after successful insert into main table + // Push to each view. Only parallel if available const Settings & settings = getContext()->getSettingsRef(); - if (settings.parallel_view_processing && views.size() > 1) + const size_t max_threads = settings.parallel_view_processing ? settings.max_threads : 1; + ThreadPool pool(std::min(max_threads, views.size())); + for (auto & view : views) { - // Push to views concurrently if enabled and more than one view is attached - ThreadPool pool(std::min(size_t(settings.max_threads), views.size())); - for (auto & view : views) - { - auto thread_group = CurrentThread::getGroup(); - pool.scheduleOrThrowOnError([=, &view, this] - { - setThreadName("PushingToViews"); - if (thread_group) - CurrentThread::attachToIfDetached(thread_group); - process(block, view); - }); - } - // Wait for concurrent view processing - pool.wait(); - } - else - { - // Process sequentially - for (auto & view : views) - { - process(block, view); + auto thread_group = CurrentThread::getGroup(); + pool.scheduleOrThrowOnError([=, &view, this] { + LOG_WARNING( + log, + "Pushing from {} to {}. {}. Current thread {} {} {} {}. WRITE START", + storage->getStorageID().getNameForLogs(), + view.table_id.getNameForLogs(), + uint64_t(view.out.get()), + view.runtime_stats.thread_status->progress_in.read_rows, + view.runtime_stats.thread_status->progress_in.written_rows, + view.runtime_stats.thread_status->progress_out.read_rows, + view.runtime_stats.thread_status->progress_out.written_rows); + // current_thread = view.thread_status.get(); - if (view.exception) - std::rethrow_exception(view.exception); - } + setThreadName("PushingToViews"); + if (thread_group) + CurrentThread::attachToIfDetached(thread_group); + + Stopwatch watch; + try + { + process(block, view); + } + catch (...) + { + view.exception = std::current_exception(); + // TODO: Stop processing on exception + } + view.runtime_stats.elapsed_ms += watch.elapsedMilliseconds(); + // TODO: Update other counters + LOG_WARNING( + log, + "Pushing from {} to {}. {}. Current thread {} {} {} {}. WRITE END", + storage->getStorageID().getNameForLogs(), + view.table_id.getNameForLogs(), + uint64_t(view.out.get()), + view.runtime_stats.thread_status->progress_in.read_rows, + view.runtime_stats.thread_status->progress_in.written_rows, + view.runtime_stats.thread_status->progress_out.read_rows, + view.runtime_stats.thread_status->progress_out.written_rows); + }); } + // Wait for concurrent view processing + pool.wait(); + check_exceptions_in_views(); } void PushingToViewsBlockOutputStream::writePrefix() @@ -213,104 +252,97 @@ void PushingToViewsBlockOutputStream::writePrefix() for (auto & view : views) { + Stopwatch watch; try { view.out->writePrefix(); } catch (Exception & ex) { + view.exception = std::current_exception(); ex.addMessage("while write prefix to view " + view.table_id.getNameForLogs()); + log_query_views(); throw; } + view.runtime_stats.elapsed_ms += watch.elapsedMilliseconds(); } } void PushingToViewsBlockOutputStream::writeSuffix() { + LOG_WARNING(log, "STARTING {} WITH {}", uint64_t(this), views.size()); + if (output) output->writeSuffix(); + if (!views.size()) + return; std::exception_ptr first_exception; - const Settings & settings = getContext()->getSettingsRef(); - bool parallel_processing = false; /// Run writeSuffix() for views in separate thread pool. /// In could have been done in PushingToViewsBlockOutputStream::process, however /// it is not good if insert into main table fail but into view succeed. - if (settings.parallel_view_processing && views.size() > 1) - { - parallel_processing = true; - - // Push to views concurrently if enabled and more than one view is attached - ThreadPool pool(std::min(size_t(settings.max_threads), views.size())); - auto thread_group = CurrentThread::getGroup(); - - for (auto & view : views) - { - if (view.exception) - continue; - - pool.scheduleOrThrowOnError([thread_group, &view, this] - { - setThreadName("PushingToViews"); - if (thread_group) - CurrentThread::attachToIfDetached(thread_group); - - Stopwatch watch; - try - { - view.out->writeSuffix(); - } - catch (...) - { - view.exception = std::current_exception(); - } - view.elapsed_ms += watch.elapsedMilliseconds(); - - LOG_TRACE(log, "Pushing from {} to {} took {} ms.", - storage->getStorageID().getNameForLogs(), - view.table_id.getNameForLogs(), - view.elapsed_ms); - }); - } - // Wait for concurrent view processing - pool.wait(); - } + const size_t max_threads = settings.parallel_view_processing ? settings.max_threads : 1; + ThreadPool pool(std::min(max_threads, views.size())); + auto thread_group = CurrentThread::getGroup(); for (auto & view : views) { if (view.exception) - { - if (!first_exception) - first_exception = view.exception; - - continue; - } - - if (parallel_processing) continue; - Stopwatch watch; - try - { - view.out->writeSuffix(); - } - catch (Exception & ex) - { - ex.addMessage("while write prefix to view " + view.table_id.getNameForLogs()); - throw; - } - view.elapsed_ms += watch.elapsedMilliseconds(); + pool.scheduleOrThrowOnError([&] { + LOG_WARNING( + log, + "Pushing from {} to {}. {}. Current thread {} {} {} {}. WRITESUFFIX START", + storage->getStorageID().getNameForLogs(), + view.table_id.getNameForLogs(), + uint64_t(view.out.get()), + view.runtime_stats.thread_status->progress_in.read_rows, + view.runtime_stats.thread_status->progress_in.written_rows, + view.runtime_stats.thread_status->progress_out.read_rows, + view.runtime_stats.thread_status->progress_out.written_rows); + // current_thread = view.thread_status.get(); + setThreadName("PushingToViews"); + if (thread_group) + CurrentThread::attachToIfDetached(thread_group); - LOG_TRACE(log, "Pushing from {} to {} took {} ms.", - storage->getStorageID().getNameForLogs(), - view.table_id.getNameForLogs(), - view.elapsed_ms); + Stopwatch watch; + try + { + LOG_WARNING(log, "BEFORE CALL {} -> {}", uint64_t(this), uint64_t(view.out.get())); + view.out->writeSuffix(); + LOG_WARNING(log, "AFTER CALL {} -> {}", uint64_t(this), uint64_t(view.out.get())); + //Set status here + } + catch (...) + { + view.exception = std::current_exception(); + } + view.runtime_stats.elapsed_ms += watch.elapsedMilliseconds(); + // TODO: Update other counters + LOG_TRACE( + log, + "Pushing from {} to {} took {} ms.", + storage->getStorageID().getNameForLogs(), + view.table_id.getNameForLogs(), + view.runtime_stats.elapsed_ms); + LOG_WARNING( + log, + "Pushing from {} to {}. {}. Current thread {} {} {} {}. WRITESUFFIX END", + storage->getStorageID().getNameForLogs(), + view.table_id.getNameForLogs(), + uint64_t(view.out.get()), + view.runtime_stats.thread_status->progress_in.read_rows, + view.runtime_stats.thread_status->progress_in.written_rows, + view.runtime_stats.thread_status->progress_out.read_rows, + view.runtime_stats.thread_status->progress_out.written_rows); + }); } - - if (first_exception) - std::rethrow_exception(first_exception); + // Wait for concurrent view processing + pool.wait(); + check_exceptions_in_views(); UInt64 milliseconds = main_watch.elapsedMilliseconds(); if (views.size() > 1) @@ -319,6 +351,8 @@ void PushingToViewsBlockOutputStream::writeSuffix() storage->getStorageID().getNameForLogs(), views.size(), milliseconds); } + LOG_WARNING(log, "FINISHING {}", uint64_t(this)); + log_query_views(); } void PushingToViewsBlockOutputStream::flush() @@ -332,8 +366,6 @@ void PushingToViewsBlockOutputStream::flush() void PushingToViewsBlockOutputStream::process(const Block & block, ViewInfo & view) { - Stopwatch watch; - try { BlockInputStreamPtr in; @@ -392,8 +424,49 @@ void PushingToViewsBlockOutputStream::process(const Block & block, ViewInfo & vi { view.exception = std::current_exception(); } - - view.elapsed_ms += watch.elapsedMilliseconds(); } +void PushingToViewsBlockOutputStream::check_exceptions_in_views() +{ + for (auto & view : views) + { + if (view.exception) + { + LOG_WARNING(log, "View exception {}", view.table_id.getNameForLogs()); + log_query_views(); + std::rethrow_exception(view.exception); + } + } +} + +void PushingToViewsBlockOutputStream::log_query_views() +{ + // TODO: Check settings + auto views_log = getContext()->getQueryViewsLog(); + if (!views_log) + { + LOG_WARNING(log, "NO VIEWS LOG"); // NOCHECKIN + return; + } + for (auto const & view : views) + { + LOG_WARNING( + log, + "LOG LOG LOG from {} to {}. {}. Progress {} {} {} {}", + storage->getStorageID().getNameForLogs(), + view.table_id.getNameForLogs(), + uint64_t(view.out.get()), + view.runtime_stats.thread_status->progress_in.read_rows, + view.runtime_stats.thread_status->progress_in.written_rows, + view.runtime_stats.thread_status->progress_out.read_rows, + view.runtime_stats.thread_status->progress_out.written_rows); + try + { + view.runtime_stats.thread_status->logToQueryViewsLog(*views_log, view); + } + catch (...) + { + } + } +} } diff --git a/src/DataStreams/PushingToViewsBlockOutputStream.h b/src/DataStreams/PushingToViewsBlockOutputStream.h index db6b671ce2c..70d5f0982b0 100644 --- a/src/DataStreams/PushingToViewsBlockOutputStream.h +++ b/src/DataStreams/PushingToViewsBlockOutputStream.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include #include #include @@ -15,6 +16,15 @@ namespace DB class ReplicatedMergeTreeSink; +struct ViewInfo +{ + ASTPtr query; + StorageID table_id; + BlockOutputStreamPtr out; + std::exception_ptr exception; + QueryViewsLogElement::ViewRuntimeStats runtime_stats; +}; + /** Writes data to the specified table and to all dependent materialized views. */ class PushingToViewsBlockOutputStream : public IBlockOutputStream, WithContext @@ -44,20 +54,13 @@ private: ASTPtr query_ptr; Stopwatch main_watch; - struct ViewInfo - { - ASTPtr query; - StorageID table_id; - BlockOutputStreamPtr out; - std::exception_ptr exception; - UInt64 elapsed_ms = 0; - }; - std::vector views; ContextMutablePtr select_context; ContextMutablePtr insert_context; void process(const Block & block, ViewInfo & view); + void check_exceptions_in_views(); + void log_query_views(); }; diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 5743164f280..798efeed150 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -2052,16 +2052,6 @@ std::shared_ptr Context::getQueryLog() const return shared->system_logs->query_log; } -std::shared_ptr Context::getQueryMaterializationLog() const -{ - auto lock = getLock(); - - if (!shared->system_logs) - return {}; - - return shared->system_logs->query_materialization_log; -} - std::shared_ptr Context::getQueryThreadLog() const { auto lock = getLock(); @@ -2072,6 +2062,15 @@ std::shared_ptr Context::getQueryThreadLog() const return shared->system_logs->query_thread_log; } +std::shared_ptr Context::getQueryViewsLog() const +{ + auto lock = getLock(); + + if (!shared->system_logs) + return {}; + + return shared->system_logs->query_views_log; +} std::shared_ptr Context::getPartLog(const String & part_database) const { diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 17d27b45b25..b8b4ef37399 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -69,8 +69,8 @@ class Macros; struct Progress; class Clusters; class QueryLog; -class QueryMaterializationLog; class QueryThreadLog; +class QueryViewsLog; class PartLog; class TextLog; class TraceLog; @@ -709,8 +709,8 @@ public: /// Nullptr if the query log is not ready for this moment. std::shared_ptr getQueryLog() const; - std::shared_ptr getQueryMaterializationLog() const; std::shared_ptr getQueryThreadLog() const; + std::shared_ptr getQueryViewsLog() const; std::shared_ptr getTraceLog() const; std::shared_ptr getTextLog() const; std::shared_ptr getMetricLog() const; diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index bcdd2225d68..eb150356fee 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -19,8 +19,8 @@ #include #include #include -#include #include +#include #include #include #include @@ -418,7 +418,7 @@ BlockIO InterpreterSystemQuery::execute() [&] { if (auto metric_log = getContext()->getMetricLog()) metric_log->flush(true); }, [&] { if (auto asynchronous_metric_log = getContext()->getAsynchronousMetricLog()) asynchronous_metric_log->flush(true); }, [&] { if (auto opentelemetry_span_log = getContext()->getOpenTelemetrySpanLog()) opentelemetry_span_log->flush(true); } - [&] () { if (auto query_materialization_log = getContext()->getQueryMaterializationLog()) query_materialization_log->flush(true); } + [&] () { if (auto query_views_log = getContext()->getQueryViewsLog()) query_views_log->flush(true); } ); break; } diff --git a/src/Interpreters/QueryMaterializationLog.cpp b/src/Interpreters/QueryViewsLog.cpp similarity index 73% rename from src/Interpreters/QueryMaterializationLog.cpp rename to src/Interpreters/QueryViewsLog.cpp index a900a70ac5f..09d15cc1769 100644 --- a/src/Interpreters/QueryMaterializationLog.cpp +++ b/src/Interpreters/QueryViewsLog.cpp @@ -1,4 +1,4 @@ -#include "QueryMaterializationLog.h" +#include "QueryViewsLog.h" #include #include @@ -17,7 +17,7 @@ namespace DB { -Block QueryMaterializationLogElement::createBlock() +Block QueryViewsLogElement::createBlock() { auto query_status_datatype = std::make_shared(DataTypeEnum8::Values{ {"QueryStart", static_cast(QUERY_START)}, @@ -25,18 +25,23 @@ Block QueryMaterializationLogElement::createBlock() {"ExceptionBeforeStart", static_cast(EXCEPTION_BEFORE_START)}, {"ExceptionWhileProcessing", static_cast(EXCEPTION_WHILE_PROCESSING)}}); + auto view_type_datatype = std::make_shared(DataTypeEnum8::Values{ + {"Default", static_cast(ViewType::DEFAULT)}, + {"Materialized", static_cast(ViewType::MATERIALIZED)}, + {"Live", static_cast(ViewType::LIVE)}}); + return { {std::make_shared(), "event_date"}, {std::make_shared(), "event_time"}, {std::make_shared(6), "event_time_microseconds"}, - {std::make_shared(), "materialization_start_time"}, - {std::make_shared(6), "materialization_start_time_microseconds"}, - {std::make_shared(), "materialization_duration_ms"}, + {std::make_shared(), "view_duration_ms"}, {std::make_shared(), "initial_query_id"}, - {std::make_shared(), "materialization_name"}, - {std::make_shared(), "materialization_uuid"}, - {std::make_shared(), "materialization_query"}, + {std::make_shared(), "view_name"}, + {std::make_shared(), "view_uuid"}, + {std::move(view_type_datatype), "view_type"}, + {std::make_shared(), "view_query"}, + {std::make_shared(), "view_target"}, {std::make_shared(), "read_rows"}, {std::make_shared(), "read_bytes"}, @@ -53,21 +58,21 @@ Block QueryMaterializationLogElement::createBlock() {std::make_shared(), "stack_trace"}}; } -void QueryMaterializationLogElement::appendToBlock(MutableColumns & columns) const +void QueryViewsLogElement::appendToBlock(MutableColumns & columns) const { size_t i = 0; columns[i++]->insert(DateLUT::instance().toDayNum(event_time).toUnderType()); // event_date columns[i++]->insert(event_time); columns[i++]->insert(event_time_microseconds); - columns[i++]->insert(materialization_start_time); - columns[i++]->insert(materialization_start_time_microseconds); - columns[i++]->insert(materialization_duration_ms); + columns[i++]->insert(view_duration_ms); columns[i++]->insertData(initial_query_id.data(), initial_query_id.size()); - columns[i++]->insertData(materialization_name.data(), materialization_name.size()); - columns[i++]->insert(materialization_uuid); - columns[i++]->insertData(materialization_query.data(), materialization_query.size()); + columns[i++]->insertData(view_name.data(), view_name.size()); + columns[i++]->insert(view_uuid); + columns[i++]->insert(view_type); + columns[i++]->insertData(view_query.data(), view_query.size()); + columns[i++]->insertData(view_target.data(), view_target.size()); columns[i++]->insert(read_rows); columns[i++]->insert(read_bytes); diff --git a/src/Interpreters/QueryMaterializationLog.h b/src/Interpreters/QueryViewsLog.h similarity index 51% rename from src/Interpreters/QueryMaterializationLog.h rename to src/Interpreters/QueryViewsLog.h index ce642cd1126..661d24abfcb 100644 --- a/src/Interpreters/QueryMaterializationLog.h +++ b/src/Interpreters/QueryViewsLog.h @@ -1,5 +1,6 @@ #pragma once +#include #include #include @@ -18,20 +19,38 @@ class Counters; namespace DB { -struct QueryMaterializationLogElement +class ThreadStatus; + +struct QueryViewsLogElement { using Status = QueryLogElementType; + enum class ViewType : int8_t + { + DEFAULT, + MATERIALIZED, + LIVE + }; + + struct ViewRuntimeStats + { + UInt64 elapsed_ms = 0; + ViewType type = ViewType::DEFAULT; + std::shared_ptr thread_status = std::make_shared(); + String initial_query_id; + std::chrono::time_point start; + String target_name; + }; time_t event_time{}; Decimal64 event_time_microseconds{}; - time_t materialization_start_time{}; - Decimal64 materialization_start_time_microseconds{}; - UInt64 materialization_duration_ms{}; + UInt64 view_duration_ms{}; String initial_query_id; - String materialization_name; - UUID materialization_uuid{UUIDHelpers::Nil}; - String materialization_query; + String view_name; + UUID view_uuid{UUIDHelpers::Nil}; + ViewType view_type{ViewType::DEFAULT}; + String view_query; + String view_target; UInt64 read_rows{}; UInt64 read_bytes{}; @@ -46,16 +65,16 @@ struct QueryMaterializationLogElement String exception; String stack_trace; - static std::string name() { return "QueryMutationLog"; } + static std::string name() { return "QueryViewsLog"; } static Block createBlock(); void appendToBlock(MutableColumns & columns) const; }; -class QueryMaterializationLog : public SystemLog +class QueryViewsLog : public SystemLog { - using SystemLog::SystemLog; + using SystemLog::SystemLog; }; } diff --git a/src/Interpreters/SystemLog.cpp b/src/Interpreters/SystemLog.cpp index 71190a84a83..a7400c59e76 100644 --- a/src/Interpreters/SystemLog.cpp +++ b/src/Interpreters/SystemLog.cpp @@ -4,8 +4,8 @@ #include #include #include -#include #include +#include #include #include #include @@ -104,8 +104,7 @@ SystemLogs::SystemLogs(ContextPtr global_context, const Poco::Util::AbstractConf opentelemetry_span_log = createSystemLog( global_context, "system", "opentelemetry_span_log", config, "opentelemetry_span_log"); - query_materialization_log = createSystemLog( - global_context, "system", "query_materialization_log", config, "query_materialization_log"); + query_views_log = createSystemLog(global_context, "system", "query_views_log", config, "query_views_log"); if (query_log) logs.emplace_back(query_log.get()); @@ -125,8 +124,8 @@ SystemLogs::SystemLogs(ContextPtr global_context, const Poco::Util::AbstractConf logs.emplace_back(asynchronous_metric_log.get()); if (opentelemetry_span_log) logs.emplace_back(opentelemetry_span_log.get()); - if (query_materialization_log) - logs.emplace_back(query_materialization_log.get()); + if (query_views_log) + logs.emplace_back(query_views_log.get()); try { diff --git a/src/Interpreters/SystemLog.h b/src/Interpreters/SystemLog.h index b95cc532236..ee839ecf4ff 100644 --- a/src/Interpreters/SystemLog.h +++ b/src/Interpreters/SystemLog.h @@ -74,7 +74,7 @@ class CrashLog; class MetricLog; class AsynchronousMetricLog; class OpenTelemetrySpanLog; -class QueryMaterializationLog; +class QueryViewsLog; class ISystemLog @@ -111,8 +111,8 @@ struct SystemLogs std::shared_ptr asynchronous_metric_log; /// OpenTelemetry trace spans. std::shared_ptr opentelemetry_span_log; - /// Used to log queries of materialized views - std::shared_ptr query_materialization_log; + /// Used to log queries of materialized and live views + std::shared_ptr query_views_log; std::vector logs; }; diff --git a/src/Interpreters/ThreadStatusExt.cpp b/src/Interpreters/ThreadStatusExt.cpp index 8590b3c94f3..e8a2c529dcb 100644 --- a/src/Interpreters/ThreadStatusExt.cpp +++ b/src/Interpreters/ThreadStatusExt.cpp @@ -1,9 +1,12 @@ #include +#include #include -#include #include +#include #include +#include +#include #include #include #include @@ -455,6 +458,45 @@ void ThreadStatus::logToQueryThreadLog(QueryThreadLog & thread_log, const String thread_log.add(elem); } +void ThreadStatus::logToQueryViewsLog(QueryViewsLog & views_log, const ViewInfo & vinfo) +{ + QueryViewsLogElement element; + + element.event_time = time_in_seconds(vinfo.runtime_stats.start); + element.event_time_microseconds = time_in_microseconds(vinfo.runtime_stats.start); + element.view_duration_ms = vinfo.runtime_stats.elapsed_ms; + + element.initial_query_id = vinfo.runtime_stats.initial_query_id; + element.view_name = vinfo.table_id.getNameForLogs(); + element.view_uuid = vinfo.table_id.uuid; + element.view_type = vinfo.runtime_stats.type; + if (vinfo.query) + element.view_query = serializeAST(*vinfo.query, true); // TODO: Anonymize like query_log ? + element.view_target = vinfo.runtime_stats.target_name; + + element.read_rows = progress_in.read_rows.load(std::memory_order_relaxed); + element.read_bytes = progress_in.read_bytes.load(std::memory_order_relaxed); + element.written_rows = progress_out.written_rows.load(std::memory_order_relaxed); + element.written_bytes = progress_out.written_bytes.load(std::memory_order_relaxed); + element.memory_usage = memory_tracker.get(); + element.peak_memory_usage = memory_tracker.getPeak(); + // if (query_context_ptr->getSettingsRef().log_profile_events != 0) // TODO + { + element.profile_counters = std::make_shared(performance_counters.getPartiallyAtomicSnapshot()); + } + + element.end_status = EXCEPTION_BEFORE_START; + element.exception_code = 0; + if (vinfo.exception) + { + element.exception_code = 0; + element.exception = "TODO"; + element.stack_trace = "TODO"; + } + + views_log.add(element); +} + void CurrentThread::initializeQuery() { if (unlikely(!current_thread)) diff --git a/src/Interpreters/ya.make b/src/Interpreters/ya.make index 5b0c6f0d9b7..f1d9e45a3a6 100644 --- a/src/Interpreters/ya.make +++ b/src/Interpreters/ya.make @@ -127,10 +127,10 @@ SRCS( ProfileEventsExt.cpp QueryAliasesVisitor.cpp QueryLog.cpp - QueryMaterializationLog.cpp QueryNormalizer.cpp QueryParameterVisitor.cpp QueryThreadLog.cpp + QueryViewsLog.cpp RemoveInjectiveFunctionsVisitor.cpp RenameColumnVisitor.cpp ReplaceQueryParameterVisitor.cpp From efe19384379dde23bceee1e67b6df85e32d36710 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 18 Jun 2021 18:25:19 +0200 Subject: [PATCH 034/599] WIP: Slow improvements --- src/Common/Exception.cpp | 40 +++++++++ src/Common/Exception.h | 2 + src/Core/Settings.h | 4 +- .../PushingToViewsBlockOutputStream.cpp | 90 ++++--------------- src/Interpreters/InterpreterSystemQuery.cpp | 2 +- src/Interpreters/QueryViewsLog.h | 17 ++-- src/Interpreters/ThreadStatusExt.cpp | 13 +-- 7 files changed, 82 insertions(+), 86 deletions(-) diff --git a/src/Common/Exception.cpp b/src/Common/Exception.cpp index 641f8bbe0f0..104942a7b75 100644 --- a/src/Common/Exception.cpp +++ b/src/Common/Exception.cpp @@ -94,6 +94,22 @@ std::string getExceptionStackTraceString(const std::exception & e) #endif } +std::string getExceptionStackTraceString(std::exception_ptr e) +{ + try + { + std::rethrow_exception(e); + } + catch (const std::exception & exception) + { + return getExceptionStackTraceString(exception); + } + catch (...) + { + return {}; + } +} + std::string Exception::getStackTraceString() const { @@ -380,6 +396,30 @@ int getCurrentExceptionCode() } } +int getExceptionErrorCode(std::exception_ptr e) +{ + try + { + std::rethrow_exception(e); + } + catch (const Exception & e) + { + return e.code(); + } + catch (const Poco::Exception &) + { + return ErrorCodes::POCO_EXCEPTION; + } + catch (const std::exception &) + { + return ErrorCodes::STD_EXCEPTION; + } + catch (...) + { + return ErrorCodes::UNKNOWN_EXCEPTION; + } +} + void rethrowFirstException(const Exceptions & exceptions) { diff --git a/src/Common/Exception.h b/src/Common/Exception.h index 79b4394948a..d04b0f71b9e 100644 --- a/src/Common/Exception.h +++ b/src/Common/Exception.h @@ -82,6 +82,7 @@ private: std::string getExceptionStackTraceString(const std::exception & e); +std::string getExceptionStackTraceString(std::exception_ptr e); /// Contains an additional member `saved_errno`. See the throwFromErrno function. @@ -167,6 +168,7 @@ std::string getCurrentExceptionMessage(bool with_stacktrace, bool check_embedded /// Returns error code from ErrorCodes int getCurrentExceptionCode(); +int getExceptionErrorCode(std::exception_ptr e); /// An execution status of any piece of code, contains return code and optional error diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 54e1f00832c..7019a4091c8 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -351,8 +351,8 @@ class IColumn; \ M(Bool, log_profile_events, true, "Log query performance statistics into the query_log and query_thread_log.", 0) \ M(Bool, log_query_settings, true, "Log query settings into the query_log.", 0) \ - M(Bool, log_query_threads, true, "Log query threads into system.query_thread_log table. This setting have effect only when 'log_queries' is true.", 0) \ - M(Bool, log_query_views, false, "Log query dependent views into system.query_views_log table. This setting have effect only when 'log_queries' is true.", 0) \ + M(Bool, log_query_threads, true, "Log query threads into system.query_thread_log table. This setting have effect only when 'log_queries' is true.", 0) \ + M(Bool, log_query_views, true, "Log query dependent views into system.query_views_log table. This setting have effect only when 'log_queries' is true.", 0) \ M(String, log_comment, "", "Log comment into system.query_log table and server log. It can be set to arbitrary string no longer than max_query_size.", 0) \ M(LogsLevel, send_logs_level, LogsLevel::fatal, "Send server text logs with specified minimum level to client. Valid values: 'trace', 'debug', 'information', 'warning', 'error', 'fatal', 'none'", 0) \ M(Bool, enable_optimize_predicate_expression, 1, "If it is set to true, optimize predicates to subqueries.", 0) \ diff --git a/src/DataStreams/PushingToViewsBlockOutputStream.cpp b/src/DataStreams/PushingToViewsBlockOutputStream.cpp index 2501d4dd42e..63caa3a0d39 100644 --- a/src/DataStreams/PushingToViewsBlockOutputStream.cpp +++ b/src/DataStreams/PushingToViewsBlockOutputStream.cpp @@ -117,12 +117,6 @@ PushingToViewsBlockOutputStream::PushingToViewsBlockOutputStream( InterpreterInsertQuery interpreter(insert_query_ptr, insert_context); BlockIO io = interpreter.execute(); out = io.out; - LOG_WARNING( - log, - "Pushing from {} to {} {}.", - storage->getStorageID().getNameForLogs(), - inner_table_id.getNameForLogs(), - inner_table->getStorageID().getFullTableName()); } else if (auto * live_view = dynamic_cast(dependent_table.get())) { @@ -136,7 +130,13 @@ PushingToViewsBlockOutputStream::PushingToViewsBlockOutputStream( dependent_table, dependent_metadata_snapshot, insert_context, ASTPtr()); QueryViewsLogElement::ViewRuntimeStats runtime_stats{ - 0, type, std::make_shared(), select_context->getInitialQueryId(), std::chrono::system_clock::now(), target_name}; + target_name, + type, + select_context->getInitialQueryId(), + std::make_shared(), + 0, + std::chrono::system_clock::now(), + QueryViewsLogElement::Status::QUERY_START}; views.emplace_back(ViewInfo{std::move(query), database_table, std::move(out), nullptr, std::move(runtime_stats)}); } @@ -200,17 +200,6 @@ void PushingToViewsBlockOutputStream::write(const Block & block) { auto thread_group = CurrentThread::getGroup(); pool.scheduleOrThrowOnError([=, &view, this] { - LOG_WARNING( - log, - "Pushing from {} to {}. {}. Current thread {} {} {} {}. WRITE START", - storage->getStorageID().getNameForLogs(), - view.table_id.getNameForLogs(), - uint64_t(view.out.get()), - view.runtime_stats.thread_status->progress_in.read_rows, - view.runtime_stats.thread_status->progress_in.written_rows, - view.runtime_stats.thread_status->progress_out.read_rows, - view.runtime_stats.thread_status->progress_out.written_rows); - // current_thread = view.thread_status.get(); setThreadName("PushingToViews"); if (thread_group) @@ -224,20 +213,12 @@ void PushingToViewsBlockOutputStream::write(const Block & block) catch (...) { view.exception = std::current_exception(); - // TODO: Stop processing on exception } + /* process might have set view.exception without throwing */ + if (view.exception) + view.runtime_stats.setStatus(QueryViewsLogElement::Status::EXCEPTION_WHILE_PROCESSING); view.runtime_stats.elapsed_ms += watch.elapsedMilliseconds(); // TODO: Update other counters - LOG_WARNING( - log, - "Pushing from {} to {}. {}. Current thread {} {} {} {}. WRITE END", - storage->getStorageID().getNameForLogs(), - view.table_id.getNameForLogs(), - uint64_t(view.out.get()), - view.runtime_stats.thread_status->progress_in.read_rows, - view.runtime_stats.thread_status->progress_in.written_rows, - view.runtime_stats.thread_status->progress_out.read_rows, - view.runtime_stats.thread_status->progress_out.written_rows); }); } // Wait for concurrent view processing @@ -259,8 +240,9 @@ void PushingToViewsBlockOutputStream::writePrefix() } catch (Exception & ex) { - view.exception = std::current_exception(); ex.addMessage("while write prefix to view " + view.table_id.getNameForLogs()); + view.exception = std::current_exception(); + view.runtime_stats.setStatus(QueryViewsLogElement::Status::EXCEPTION_WHILE_PROCESSING); log_query_views(); throw; } @@ -270,8 +252,6 @@ void PushingToViewsBlockOutputStream::writePrefix() void PushingToViewsBlockOutputStream::writeSuffix() { - LOG_WARNING(log, "STARTING {} WITH {}", uint64_t(this), views.size()); - if (output) output->writeSuffix(); @@ -293,17 +273,6 @@ void PushingToViewsBlockOutputStream::writeSuffix() continue; pool.scheduleOrThrowOnError([&] { - LOG_WARNING( - log, - "Pushing from {} to {}. {}. Current thread {} {} {} {}. WRITESUFFIX START", - storage->getStorageID().getNameForLogs(), - view.table_id.getNameForLogs(), - uint64_t(view.out.get()), - view.runtime_stats.thread_status->progress_in.read_rows, - view.runtime_stats.thread_status->progress_in.written_rows, - view.runtime_stats.thread_status->progress_out.read_rows, - view.runtime_stats.thread_status->progress_out.written_rows); - // current_thread = view.thread_status.get(); setThreadName("PushingToViews"); if (thread_group) CurrentThread::attachToIfDetached(thread_group); @@ -311,10 +280,8 @@ void PushingToViewsBlockOutputStream::writeSuffix() Stopwatch watch; try { - LOG_WARNING(log, "BEFORE CALL {} -> {}", uint64_t(this), uint64_t(view.out.get())); view.out->writeSuffix(); - LOG_WARNING(log, "AFTER CALL {} -> {}", uint64_t(this), uint64_t(view.out.get())); - //Set status here + view.runtime_stats.setStatus(QueryViewsLogElement::Status::QUERY_FINISH); } catch (...) { @@ -328,16 +295,6 @@ void PushingToViewsBlockOutputStream::writeSuffix() storage->getStorageID().getNameForLogs(), view.table_id.getNameForLogs(), view.runtime_stats.elapsed_ms); - LOG_WARNING( - log, - "Pushing from {} to {}. {}. Current thread {} {} {} {}. WRITESUFFIX END", - storage->getStorageID().getNameForLogs(), - view.table_id.getNameForLogs(), - uint64_t(view.out.get()), - view.runtime_stats.thread_status->progress_in.read_rows, - view.runtime_stats.thread_status->progress_in.written_rows, - view.runtime_stats.thread_status->progress_out.read_rows, - view.runtime_stats.thread_status->progress_out.written_rows); }); } // Wait for concurrent view processing @@ -351,7 +308,6 @@ void PushingToViewsBlockOutputStream::writeSuffix() storage->getStorageID().getNameForLogs(), views.size(), milliseconds); } - LOG_WARNING(log, "FINISHING {}", uint64_t(this)); log_query_views(); } @@ -432,7 +388,6 @@ void PushingToViewsBlockOutputStream::check_exceptions_in_views() { if (view.exception) { - LOG_WARNING(log, "View exception {}", view.table_id.getNameForLogs()); log_query_views(); std::rethrow_exception(view.exception); } @@ -444,28 +399,19 @@ void PushingToViewsBlockOutputStream::log_query_views() // TODO: Check settings auto views_log = getContext()->getQueryViewsLog(); if (!views_log) - { - LOG_WARNING(log, "NO VIEWS LOG"); // NOCHECKIN return; - } - for (auto const & view : views) + + for (auto & view : views) { - LOG_WARNING( - log, - "LOG LOG LOG from {} to {}. {}. Progress {} {} {} {}", - storage->getStorageID().getNameForLogs(), - view.table_id.getNameForLogs(), - uint64_t(view.out.get()), - view.runtime_stats.thread_status->progress_in.read_rows, - view.runtime_stats.thread_status->progress_in.written_rows, - view.runtime_stats.thread_status->progress_out.read_rows, - view.runtime_stats.thread_status->progress_out.written_rows); + if (view.runtime_stats.event_status == QueryViewsLogElement::Status::QUERY_START) + view.runtime_stats.setStatus(QueryViewsLogElement::Status::EXCEPTION_WHILE_PROCESSING); try { view.runtime_stats.thread_status->logToQueryViewsLog(*views_log, view); } catch (...) { + LOG_WARNING(log, getCurrentExceptionMessage(true)); } } } diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index eb150356fee..e7cc8e3d386 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -417,7 +417,7 @@ BlockIO InterpreterSystemQuery::execute() [&] { if (auto text_log = getContext()->getTextLog()) text_log->flush(true); }, [&] { if (auto metric_log = getContext()->getMetricLog()) metric_log->flush(true); }, [&] { if (auto asynchronous_metric_log = getContext()->getAsynchronousMetricLog()) asynchronous_metric_log->flush(true); }, - [&] { if (auto opentelemetry_span_log = getContext()->getOpenTelemetrySpanLog()) opentelemetry_span_log->flush(true); } + [&] { if (auto opentelemetry_span_log = getContext()->getOpenTelemetrySpanLog()) opentelemetry_span_log->flush(true); }, [&] () { if (auto query_views_log = getContext()->getQueryViewsLog()) query_views_log->flush(true); } ); break; diff --git a/src/Interpreters/QueryViewsLog.h b/src/Interpreters/QueryViewsLog.h index 661d24abfcb..acf0657129d 100644 --- a/src/Interpreters/QueryViewsLog.h +++ b/src/Interpreters/QueryViewsLog.h @@ -33,12 +33,19 @@ struct QueryViewsLogElement struct ViewRuntimeStats { - UInt64 elapsed_ms = 0; - ViewType type = ViewType::DEFAULT; - std::shared_ptr thread_status = std::make_shared(); - String initial_query_id; - std::chrono::time_point start; String target_name; + ViewType type = ViewType::DEFAULT; + String initial_query_id; + std::shared_ptr thread_status = std::make_shared(); + UInt64 elapsed_ms = 0; + std::chrono::time_point event_time; + Status event_status = Status::QUERY_START; + + void setStatus(Status s) + { + event_status = s; + event_time = std::chrono::system_clock::now(); + } }; time_t event_time{}; diff --git a/src/Interpreters/ThreadStatusExt.cpp b/src/Interpreters/ThreadStatusExt.cpp index e8a2c529dcb..b31ca398e76 100644 --- a/src/Interpreters/ThreadStatusExt.cpp +++ b/src/Interpreters/ThreadStatusExt.cpp @@ -462,8 +462,8 @@ void ThreadStatus::logToQueryViewsLog(QueryViewsLog & views_log, const ViewInfo { QueryViewsLogElement element; - element.event_time = time_in_seconds(vinfo.runtime_stats.start); - element.event_time_microseconds = time_in_microseconds(vinfo.runtime_stats.start); + element.event_time = time_in_seconds(vinfo.runtime_stats.event_time); + element.event_time_microseconds = time_in_microseconds(vinfo.runtime_stats.event_time); element.view_duration_ms = vinfo.runtime_stats.elapsed_ms; element.initial_query_id = vinfo.runtime_stats.initial_query_id; @@ -485,13 +485,14 @@ void ThreadStatus::logToQueryViewsLog(QueryViewsLog & views_log, const ViewInfo element.profile_counters = std::make_shared(performance_counters.getPartiallyAtomicSnapshot()); } - element.end_status = EXCEPTION_BEFORE_START; + element.end_status = vinfo.runtime_stats.event_status; element.exception_code = 0; if (vinfo.exception) { - element.exception_code = 0; - element.exception = "TODO"; - element.stack_trace = "TODO"; + element.exception_code = getExceptionErrorCode(vinfo.exception); + element.exception = getExceptionMessage(vinfo.exception, false); + // if (current_settings.calculate_text_stack_trace) // TODO + element.stack_trace = getExceptionStackTraceString(vinfo.exception); } views_log.add(element); From 6b9ec2a62e43e76b41bede499e37eee99a9db640 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 21 Jun 2021 17:58:15 +0200 Subject: [PATCH 035/599] WIP --- programs/server/config.xml | 2 +- src/Common/ThreadStatus.h | 2 +- src/Core/Settings.h | 4 +- .../PushingToViewsBlockOutputStream.cpp | 63 ++++++++++++------- .../PushingToViewsBlockOutputStream.h | 2 +- src/Interpreters/QueryViewsLog.cpp | 2 - src/Interpreters/QueryViewsLog.h | 2 - src/Interpreters/ThreadStatusExt.cpp | 57 ++++++++++++----- 8 files changed, 89 insertions(+), 45 deletions(-) diff --git a/programs/server/config.xml b/programs/server/config.xml index 3f7f6c25e24..298fc176cfa 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -583,7 +583,7 @@ 9019 --> - + diff --git a/src/Common/ThreadStatus.h b/src/Common/ThreadStatus.h index 86aa8cc52a1..2d39ed56e47 100644 --- a/src/Common/ThreadStatus.h +++ b/src/Common/ThreadStatus.h @@ -215,7 +215,7 @@ public: /// Detaches thread from the thread group and the query, dumps performance counters if they have not been dumped void detachQuery(bool exit_if_already_detached = false, bool thread_exits = false); - void logToQueryViewsLog(QueryViewsLog & views_log, const ViewInfo & vinfo); + void logToQueryViewsLog(const ViewInfo & vinfo); protected: void applyQuerySettings(); diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 7019a4091c8..d866bbb5350 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -170,7 +170,7 @@ class IColumn; \ M(Bool, log_queries, 1, "Log requests and write the log to the system table.", 0) \ M(LogQueriesType, log_queries_min_type, QueryLogElementType::QUERY_START, "Minimal type in query_log to log, possible values (from low to high): QUERY_START, QUERY_FINISH, EXCEPTION_BEFORE_START, EXCEPTION_WHILE_PROCESSING.", 0) \ - M(Milliseconds, log_queries_min_query_duration_ms, 0, "Minimal time for the query to run, to get to the query_log/query_thread_log.", 0) \ + M(Milliseconds, log_queries_min_query_duration_ms, 0, "Minimal time for the query to run, to get to the query_log/query_thread_log/query_views_log.", 0) \ M(UInt64, log_queries_cut_to_length, 100000, "If query length is greater than specified threshold (in bytes), then cut query when writing to query log. Also limit length of printed query in ordinary text log.", 0) \ \ M(DistributedProductMode, distributed_product_mode, DistributedProductMode::DENY, "How are distributed subqueries performed inside IN or JOIN sections?", IMPORTANT) \ @@ -349,7 +349,7 @@ class IColumn; M(UInt64, max_network_bandwidth_for_user, 0, "The maximum speed of data exchange over the network in bytes per second for all concurrently running user queries. Zero means unlimited.", 0)\ M(UInt64, max_network_bandwidth_for_all_users, 0, "The maximum speed of data exchange over the network in bytes per second for all concurrently running queries. Zero means unlimited.", 0) \ \ - M(Bool, log_profile_events, true, "Log query performance statistics into the query_log and query_thread_log.", 0) \ + M(Bool, log_profile_events, true, "Log query performance statistics into the query_log, query_thread_log and query_views_log.", 0) \ M(Bool, log_query_settings, true, "Log query settings into the query_log.", 0) \ M(Bool, log_query_threads, true, "Log query threads into system.query_thread_log table. This setting have effect only when 'log_queries' is true.", 0) \ M(Bool, log_query_views, true, "Log query dependent views into system.query_views_log table. This setting have effect only when 'log_queries' is true.", 0) \ diff --git a/src/DataStreams/PushingToViewsBlockOutputStream.cpp b/src/DataStreams/PushingToViewsBlockOutputStream.cpp index 63caa3a0d39..a6015b33225 100644 --- a/src/DataStreams/PushingToViewsBlockOutputStream.cpp +++ b/src/DataStreams/PushingToViewsBlockOutputStream.cpp @@ -12,14 +12,28 @@ #include #include #include -#include #include +#include +#include +#include +#include +#include +#include #include #include #include #include +#include + +namespace ProfileEvents +{ +extern const Event SlowRead; +extern const Event MergedRows; +extern const Event ZooKeeperTransactions; +} + namespace DB { @@ -72,6 +86,8 @@ PushingToViewsBlockOutputStream::PushingToViewsBlockOutputStream( insert_context->setSetting("min_insert_block_size_bytes", insert_settings.min_insert_block_size_bytes_for_materialized_views.value); } + auto thread_group = CurrentThread::getGroup(); + for (const auto & database_table : dependencies) { auto dependent_table = DatabaseCatalog::instance().getTable(database_table, getContext()); @@ -118,7 +134,7 @@ PushingToViewsBlockOutputStream::PushingToViewsBlockOutputStream( BlockIO io = interpreter.execute(); out = io.out; } - else if (auto * live_view = dynamic_cast(dependent_table.get())) + else if (const auto * live_view = dynamic_cast(dependent_table.get())) { type = QueryViewsLogElement::ViewType::LIVE; query = live_view->getInnerQuery(); // TODO: Optimize this @@ -129,14 +145,13 @@ PushingToViewsBlockOutputStream::PushingToViewsBlockOutputStream( out = std::make_shared( dependent_table, dependent_metadata_snapshot, insert_context, ASTPtr()); + auto main_thread = current_thread; + auto thread_status = std::make_shared(); + current_thread = main_thread; + thread_status->attachQueryContext(getContext()); + QueryViewsLogElement::ViewRuntimeStats runtime_stats{ - target_name, - type, - select_context->getInitialQueryId(), - std::make_shared(), - 0, - std::chrono::system_clock::now(), - QueryViewsLogElement::Status::QUERY_START}; + target_name, type, thread_status, 0, std::chrono::system_clock::now(), QueryViewsLogElement::Status::QUERY_START}; views.emplace_back(ViewInfo{std::move(query), database_table, std::move(out), nullptr, std::move(runtime_stats)}); } @@ -150,6 +165,7 @@ PushingToViewsBlockOutputStream::PushingToViewsBlockOutputStream( replicated_output = dynamic_cast(sink.get()); output = std::make_shared(std::move(sink)); } + ProfileEvents::increment(ProfileEvents::ZooKeeperTransactions, 100); } @@ -185,7 +201,7 @@ void PushingToViewsBlockOutputStream::write(const Block & block) output->write(block); } - if (!views.size()) + if (views.empty()) return; /// Don't process materialized views if this block is duplicate @@ -198,12 +214,12 @@ void PushingToViewsBlockOutputStream::write(const Block & block) ThreadPool pool(std::min(max_threads, views.size())); for (auto & view : views) { - auto thread_group = CurrentThread::getGroup(); pool.scheduleOrThrowOnError([=, &view, this] { setThreadName("PushingToViews"); - if (thread_group) - CurrentThread::attachToIfDetached(thread_group); + current_thread = view.runtime_stats.thread_status.get(); + ProfileEvents::increment(ProfileEvents::ZooKeeperTransactions, 1); + LOG_WARNING(log, "WRITE THREAD"); Stopwatch watch; try @@ -218,7 +234,6 @@ void PushingToViewsBlockOutputStream::write(const Block & block) if (view.exception) view.runtime_stats.setStatus(QueryViewsLogElement::Status::EXCEPTION_WHILE_PROCESSING); view.runtime_stats.elapsed_ms += watch.elapsedMilliseconds(); - // TODO: Update other counters }); } // Wait for concurrent view processing @@ -255,7 +270,7 @@ void PushingToViewsBlockOutputStream::writeSuffix() if (output) output->writeSuffix(); - if (!views.size()) + if (views.empty()) return; std::exception_ptr first_exception; const Settings & settings = getContext()->getSettingsRef(); @@ -274,8 +289,9 @@ void PushingToViewsBlockOutputStream::writeSuffix() pool.scheduleOrThrowOnError([&] { setThreadName("PushingToViews"); - if (thread_group) - CurrentThread::attachToIfDetached(thread_group); + current_thread = view.runtime_stats.thread_status.get(); + ProfileEvents::increment(ProfileEvents::ZooKeeperTransactions, 1); + LOG_WARNING(log, "WRITE SUFFIX THREAD"); Stopwatch watch; try @@ -288,7 +304,6 @@ void PushingToViewsBlockOutputStream::writeSuffix() view.exception = std::current_exception(); } view.runtime_stats.elapsed_ms += watch.elapsedMilliseconds(); - // TODO: Update other counters LOG_TRACE( log, "Pushing from {} to {} took {} ms.", @@ -396,18 +411,22 @@ void PushingToViewsBlockOutputStream::check_exceptions_in_views() void PushingToViewsBlockOutputStream::log_query_views() { - // TODO: Check settings - auto views_log = getContext()->getQueryViewsLog(); - if (!views_log) + const auto & settings = getContext()->getSettingsRef(); + const UInt64 min_query_duration = settings.log_queries_min_query_duration_ms.totalMilliseconds(); + if (views.empty() || !settings.log_queries || !settings.log_query_views) return; for (auto & view : views) { if (view.runtime_stats.event_status == QueryViewsLogElement::Status::QUERY_START) view.runtime_stats.setStatus(QueryViewsLogElement::Status::EXCEPTION_WHILE_PROCESSING); + + if (min_query_duration && view.runtime_stats.elapsed_ms <= min_query_duration) + continue; + try { - view.runtime_stats.thread_status->logToQueryViewsLog(*views_log, view); + view.runtime_stats.thread_status->logToQueryViewsLog(view); } catch (...) { diff --git a/src/DataStreams/PushingToViewsBlockOutputStream.h b/src/DataStreams/PushingToViewsBlockOutputStream.h index 70d5f0982b0..062c026ff1c 100644 --- a/src/DataStreams/PushingToViewsBlockOutputStream.h +++ b/src/DataStreams/PushingToViewsBlockOutputStream.h @@ -18,7 +18,7 @@ class ReplicatedMergeTreeSink; struct ViewInfo { - ASTPtr query; + const ASTPtr query; StorageID table_id; BlockOutputStreamPtr out; std::exception_ptr exception; diff --git a/src/Interpreters/QueryViewsLog.cpp b/src/Interpreters/QueryViewsLog.cpp index 09d15cc1769..878ca2984e2 100644 --- a/src/Interpreters/QueryViewsLog.cpp +++ b/src/Interpreters/QueryViewsLog.cpp @@ -47,7 +47,6 @@ Block QueryViewsLogElement::createBlock() {std::make_shared(), "read_bytes"}, {std::make_shared(), "written_rows"}, {std::make_shared(), "written_bytes"}, - {std::make_shared(), "memory_usage"}, {std::make_shared(), "peak_memory_usage"}, {std::make_shared(std::make_shared()), "ProfileEvents.Names"}, {std::make_shared(std::make_shared()), "ProfileEvents.Values"}, @@ -78,7 +77,6 @@ void QueryViewsLogElement::appendToBlock(MutableColumns & columns) const columns[i++]->insert(read_bytes); columns[i++]->insert(written_rows); columns[i++]->insert(written_bytes); - columns[i++]->insert(memory_usage); columns[i++]->insert(peak_memory_usage); if (profile_counters) diff --git a/src/Interpreters/QueryViewsLog.h b/src/Interpreters/QueryViewsLog.h index acf0657129d..fe8a591358e 100644 --- a/src/Interpreters/QueryViewsLog.h +++ b/src/Interpreters/QueryViewsLog.h @@ -35,7 +35,6 @@ struct QueryViewsLogElement { String target_name; ViewType type = ViewType::DEFAULT; - String initial_query_id; std::shared_ptr thread_status = std::make_shared(); UInt64 elapsed_ms = 0; std::chrono::time_point event_time; @@ -63,7 +62,6 @@ struct QueryViewsLogElement UInt64 read_bytes{}; UInt64 written_rows{}; UInt64 written_bytes{}; - Int64 memory_usage{}; Int64 peak_memory_usage{}; std::shared_ptr profile_counters; diff --git a/src/Interpreters/ThreadStatusExt.cpp b/src/Interpreters/ThreadStatusExt.cpp index b31ca398e76..25aac7864b9 100644 --- a/src/Interpreters/ThreadStatusExt.cpp +++ b/src/Interpreters/ThreadStatusExt.cpp @@ -9,7 +9,9 @@ #include #include #include +#include #include +#include #include #include #include @@ -21,6 +23,14 @@ # include #endif +namespace ProfileEvents +{ +extern const Event SelectedRows; +extern const Event SelectedBytes; +extern const Event InsertedRows; +extern const Event InsertedBytes; +} + /// Implement some methods of ThreadStatus and CurrentThread here to avoid extra linking dependencies in clickhouse_common_io /// TODO It doesn't make sense. @@ -458,31 +468,50 @@ void ThreadStatus::logToQueryThreadLog(QueryThreadLog & thread_log, const String thread_log.add(elem); } -void ThreadStatus::logToQueryViewsLog(QueryViewsLog & views_log, const ViewInfo & vinfo) +static String getCleanQueryAst(const ASTPtr q, ContextPtr context) { + String res = serializeAST(*q, true); + if (auto * masker = SensitiveDataMasker::getInstance()) + masker->wipeSensitiveData(res); + + res = res.substr(0, context->getSettingsRef().log_queries_cut_to_length); + + return res; +} + +void ThreadStatus::logToQueryViewsLog(const ViewInfo & vinfo) +{ + updatePerformanceCounters(); + auto query_context_ptr = query_context.lock(); + if (!query_context_ptr) + return; + auto views_log = query_context_ptr->getQueryViewsLog(); + if (!views_log) + return; + QueryViewsLogElement element; element.event_time = time_in_seconds(vinfo.runtime_stats.event_time); element.event_time_microseconds = time_in_microseconds(vinfo.runtime_stats.event_time); element.view_duration_ms = vinfo.runtime_stats.elapsed_ms; - element.initial_query_id = vinfo.runtime_stats.initial_query_id; + element.initial_query_id = query_id; // query_context_ptr->getInitialQueryId(); element.view_name = vinfo.table_id.getNameForLogs(); element.view_uuid = vinfo.table_id.uuid; element.view_type = vinfo.runtime_stats.type; if (vinfo.query) - element.view_query = serializeAST(*vinfo.query, true); // TODO: Anonymize like query_log ? + element.view_query = getCleanQueryAst(vinfo.query, query_context_ptr); element.view_target = vinfo.runtime_stats.target_name; - element.read_rows = progress_in.read_rows.load(std::memory_order_relaxed); - element.read_bytes = progress_in.read_bytes.load(std::memory_order_relaxed); - element.written_rows = progress_out.written_rows.load(std::memory_order_relaxed); - element.written_bytes = progress_out.written_bytes.load(std::memory_order_relaxed); - element.memory_usage = memory_tracker.get(); - element.peak_memory_usage = memory_tracker.getPeak(); - // if (query_context_ptr->getSettingsRef().log_profile_events != 0) // TODO + auto events = std::make_shared(performance_counters.getPartiallyAtomicSnapshot()); + element.read_rows = (*events)[ProfileEvents::SelectedRows]; + element.read_bytes = (*events)[ProfileEvents::SelectedBytes]; + element.written_rows = (*events)[ProfileEvents::InsertedRows]; + element.written_bytes = (*events)[ProfileEvents::InsertedBytes]; + element.peak_memory_usage = memory_tracker.getPeak() > 0 ? memory_tracker.getPeak() : 0; + if (query_context_ptr->getSettingsRef().log_profile_events != 0) { - element.profile_counters = std::make_shared(performance_counters.getPartiallyAtomicSnapshot()); + element.profile_counters = events; } element.end_status = vinfo.runtime_stats.event_status; @@ -491,11 +520,11 @@ void ThreadStatus::logToQueryViewsLog(QueryViewsLog & views_log, const ViewInfo { element.exception_code = getExceptionErrorCode(vinfo.exception); element.exception = getExceptionMessage(vinfo.exception, false); - // if (current_settings.calculate_text_stack_trace) // TODO - element.stack_trace = getExceptionStackTraceString(vinfo.exception); + if (query_context_ptr->getSettingsRef().calculate_text_stack_trace) + element.stack_trace = getExceptionStackTraceString(vinfo.exception); } - views_log.add(element); + views_log->add(element); } void CurrentThread::initializeQuery() From e0ee7839f7ac8cb9cbce32eaaba194114a851dd9 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 22 Jun 2021 13:39:53 +0200 Subject: [PATCH 036/599] Only use threads when necessary and log written status --- .../PushingToViewsBlockOutputStream.cpp | 233 +++++++++++------- .../PushingToViewsBlockOutputStream.h | 2 + src/Interpreters/QueryViewsLog.cpp | 14 +- src/Interpreters/QueryViewsLog.h | 22 +- src/Interpreters/ThreadStatusExt.cpp | 2 +- 5 files changed, 166 insertions(+), 107 deletions(-) diff --git a/src/DataStreams/PushingToViewsBlockOutputStream.cpp b/src/DataStreams/PushingToViewsBlockOutputStream.cpp index a6015b33225..fd3fd1d6a13 100644 --- a/src/DataStreams/PushingToViewsBlockOutputStream.cpp +++ b/src/DataStreams/PushingToViewsBlockOutputStream.cpp @@ -1,7 +1,7 @@ #include #include #include -#include +#include #include #include #include @@ -18,22 +18,13 @@ #include #include #include +#include #include #include -#include #include #include -#include - -namespace ProfileEvents -{ -extern const Event SlowRead; -extern const Event MergedRows; -extern const Event ZooKeeperTransactions; -} - namespace DB { @@ -137,7 +128,7 @@ PushingToViewsBlockOutputStream::PushingToViewsBlockOutputStream( else if (const auto * live_view = dynamic_cast(dependent_table.get())) { type = QueryViewsLogElement::ViewType::LIVE; - query = live_view->getInnerQuery(); // TODO: Optimize this + query = live_view->getInnerQuery(); // Used only to log in system.query_views_log out = std::make_shared( dependent_table, dependent_metadata_snapshot, insert_context, ASTPtr(), true); } @@ -145,13 +136,13 @@ PushingToViewsBlockOutputStream::PushingToViewsBlockOutputStream( out = std::make_shared( dependent_table, dependent_metadata_snapshot, insert_context, ASTPtr()); - auto main_thread = current_thread; + auto * main_thread = current_thread; auto thread_status = std::make_shared(); current_thread = main_thread; thread_status->attachQueryContext(getContext()); QueryViewsLogElement::ViewRuntimeStats runtime_stats{ - target_name, type, thread_status, 0, std::chrono::system_clock::now(), QueryViewsLogElement::Status::QUERY_START}; + target_name, type, thread_status, 0, std::chrono::system_clock::now(), QueryViewsLogElement::ViewStatus::INIT}; views.emplace_back(ViewInfo{std::move(query), database_table, std::move(out), nullptr, std::move(runtime_stats)}); } @@ -165,7 +156,6 @@ PushingToViewsBlockOutputStream::PushingToViewsBlockOutputStream( replicated_output = dynamic_cast(sink.get()); output = std::make_shared(std::move(sink)); } - ProfileEvents::increment(ProfileEvents::ZooKeeperTransactions, 100); } @@ -208,37 +198,43 @@ void PushingToViewsBlockOutputStream::write(const Block & block) if (!getContext()->getSettingsRef().deduplicate_blocks_in_dependent_materialized_views && replicated_output && replicated_output->lastBlockIsDuplicate()) return; - // Push to each view. Only parallel if available const Settings & settings = getContext()->getSettingsRef(); const size_t max_threads = settings.parallel_view_processing ? settings.max_threads : 1; - ThreadPool pool(std::min(max_threads, views.size())); - for (auto & view : views) + bool exception_happened = false; + if (max_threads > 1) { - pool.scheduleOrThrowOnError([=, &view, this] { + ThreadPool pool(std::min(max_threads, views.size())); + auto thread_group = CurrentThread::getGroup(); + std::atomic_uint8_t exception_count = 0; + for (auto & view : views) + { + pool.scheduleOrThrowOnError([&] { + setThreadName("PushingToViews"); + if (exception_count.load(std::memory_order_relaxed)) + return; - setThreadName("PushingToViews"); - current_thread = view.runtime_stats.thread_status.get(); - ProfileEvents::increment(ProfileEvents::ZooKeeperTransactions, 1); - LOG_WARNING(log, "WRITE THREAD"); - - Stopwatch watch; - try - { process(block, view); - } - catch (...) - { - view.exception = std::current_exception(); - } - /* process might have set view.exception without throwing */ - if (view.exception) - view.runtime_stats.setStatus(QueryViewsLogElement::Status::EXCEPTION_WHILE_PROCESSING); - view.runtime_stats.elapsed_ms += watch.elapsedMilliseconds(); - }); + if (view.exception) + exception_count.fetch_add(1, std::memory_order_relaxed); + }); + } + pool.wait(); + exception_happened = exception_count.load(std::memory_order_relaxed) != 0; } - // Wait for concurrent view processing - pool.wait(); - check_exceptions_in_views(); + else + { + for (auto & view : views) + { + process(block, view); + if (view.exception) + { + exception_happened = true; + break; + } + } + } + if (exception_happened) + check_exceptions_in_views(); } void PushingToViewsBlockOutputStream::writePrefix() @@ -248,20 +244,12 @@ void PushingToViewsBlockOutputStream::writePrefix() for (auto & view : views) { - Stopwatch watch; - try + process_prefix(view); + if (view.exception) { - view.out->writePrefix(); - } - catch (Exception & ex) - { - ex.addMessage("while write prefix to view " + view.table_id.getNameForLogs()); - view.exception = std::current_exception(); - view.runtime_stats.setStatus(QueryViewsLogElement::Status::EXCEPTION_WHILE_PROCESSING); log_query_views(); throw; } - view.runtime_stats.elapsed_ms += watch.elapsedMilliseconds(); } } @@ -272,62 +260,60 @@ void PushingToViewsBlockOutputStream::writeSuffix() if (views.empty()) return; - std::exception_ptr first_exception; - const Settings & settings = getContext()->getSettingsRef(); /// Run writeSuffix() for views in separate thread pool. /// In could have been done in PushingToViewsBlockOutputStream::process, however /// it is not good if insert into main table fail but into view succeed. + const Settings & settings = getContext()->getSettingsRef(); const size_t max_threads = settings.parallel_view_processing ? settings.max_threads : 1; - ThreadPool pool(std::min(max_threads, views.size())); - auto thread_group = CurrentThread::getGroup(); - - for (auto & view : views) + bool exception_happened = false; + if (max_threads > 1) { - if (view.exception) - continue; + ThreadPool pool(std::min(max_threads, views.size())); + auto thread_group = CurrentThread::getGroup(); + std::atomic_uint8_t exception_count = 0; + for (auto & view : views) + { + pool.scheduleOrThrowOnError([&] { + setThreadName("PushingToViews"); + if (exception_count.load(std::memory_order_relaxed)) + return; - pool.scheduleOrThrowOnError([&] { - setThreadName("PushingToViews"); - current_thread = view.runtime_stats.thread_status.get(); - ProfileEvents::increment(ProfileEvents::ZooKeeperTransactions, 1); - LOG_WARNING(log, "WRITE SUFFIX THREAD"); - - Stopwatch watch; - try - { - view.out->writeSuffix(); - view.runtime_stats.setStatus(QueryViewsLogElement::Status::QUERY_FINISH); - } - catch (...) - { - view.exception = std::current_exception(); - } - view.runtime_stats.elapsed_ms += watch.elapsedMilliseconds(); - LOG_TRACE( - log, - "Pushing from {} to {} took {} ms.", - storage->getStorageID().getNameForLogs(), - view.table_id.getNameForLogs(), - view.runtime_stats.elapsed_ms); - }); + process_suffix(view); + if (view.exception) + exception_count.fetch_add(1, std::memory_order_relaxed); + }); + } + pool.wait(); + exception_happened = exception_count.load(std::memory_order_relaxed) != 0; } - // Wait for concurrent view processing - pool.wait(); - check_exceptions_in_views(); + else + { + for (auto & view : views) + { + process_suffix(view); + if (view.exception) + { + exception_happened = true; + break; + } + } + } + if (exception_happened) + check_exceptions_in_views(); - UInt64 milliseconds = main_watch.elapsedMilliseconds(); if (views.size() > 1) { - LOG_DEBUG(log, "Pushing from {} to {} views took {} ms.", - storage->getStorageID().getNameForLogs(), views.size(), - milliseconds); + UInt64 milliseconds = main_watch.elapsedMilliseconds(); + LOG_DEBUG(log, "Pushing from {} to {} views took {} ms.", storage->getStorageID().getNameForLogs(), views.size(), milliseconds); } log_query_views(); } void PushingToViewsBlockOutputStream::flush() { + LOG_DEBUG(log, "{} FLUSH CALLED", storage->getStorageID().getNameForLogs()); + if (output) output->flush(); @@ -337,6 +323,11 @@ void PushingToViewsBlockOutputStream::flush() void PushingToViewsBlockOutputStream::process(const Block & block, ViewInfo & view) { + Stopwatch watch; + auto * source_thread = current_thread; // Change thread context to store individual metrics per view + current_thread = view.runtime_stats.thread_status.get(); + SCOPE_EXIT({ current_thread = source_thread; }); + try { BlockInputStreamPtr in; @@ -385,6 +376,7 @@ void PushingToViewsBlockOutputStream::process(const Block & block, ViewInfo & vi } in->readSuffix(); + view.runtime_stats.setStatus(QueryViewsLogElement::ViewStatus::WRITTEN_BLOCK); } catch (Exception & ex) { @@ -395,6 +387,66 @@ void PushingToViewsBlockOutputStream::process(const Block & block, ViewInfo & vi { view.exception = std::current_exception(); } + + view.runtime_stats.elapsed_ms += watch.elapsedMilliseconds(); +} + +void PushingToViewsBlockOutputStream::process_prefix(ViewInfo & view) +{ + Stopwatch watch; + auto * source_thread = current_thread; // Change thread context to store individual metrics per view + current_thread = view.runtime_stats.thread_status.get(); + SCOPE_EXIT({ current_thread = source_thread; }); + + try + { + view.out->writePrefix(); + view.runtime_stats.setStatus(QueryViewsLogElement::ViewStatus::WRITTEN_PREFIX); + } + catch (Exception & ex) + { + ex.addMessage("while writing prefix to view " + view.table_id.getNameForLogs()); + view.exception = std::current_exception(); + } + catch (...) + { + view.exception = std::current_exception(); + } + view.runtime_stats.elapsed_ms += watch.elapsedMilliseconds(); +} + + +void PushingToViewsBlockOutputStream::process_suffix(ViewInfo & view) +{ + Stopwatch watch; + auto * source_thread = current_thread; // Change thread context to store individual metrics per view + current_thread = view.runtime_stats.thread_status.get(); + SCOPE_EXIT({ current_thread = source_thread; }); + + try + { + view.out->writeSuffix(); + view.runtime_stats.setStatus(QueryViewsLogElement::ViewStatus::WRITTEN_SUFFIX); + } + catch (Exception & ex) + { + ex.addMessage("while writing suffix to view " + view.table_id.getNameForLogs()); + view.exception = std::current_exception(); + } + catch (...) + { + view.exception = std::current_exception(); + } + view.runtime_stats.elapsed_ms += watch.elapsedMilliseconds(); + if (!view.exception) + { + LOG_TRACE( + log, + "Pushing from {} to {} took {} ms.", + storage->getStorageID().getNameForLogs(), + view.table_id.getNameForLogs(), + view.runtime_stats.elapsed_ms); + } } void PushingToViewsBlockOutputStream::check_exceptions_in_views() @@ -418,9 +470,6 @@ void PushingToViewsBlockOutputStream::log_query_views() for (auto & view : views) { - if (view.runtime_stats.event_status == QueryViewsLogElement::Status::QUERY_START) - view.runtime_stats.setStatus(QueryViewsLogElement::Status::EXCEPTION_WHILE_PROCESSING); - if (min_query_duration && view.runtime_stats.elapsed_ms <= min_query_duration) continue; diff --git a/src/DataStreams/PushingToViewsBlockOutputStream.h b/src/DataStreams/PushingToViewsBlockOutputStream.h index 062c026ff1c..c836ee54ded 100644 --- a/src/DataStreams/PushingToViewsBlockOutputStream.h +++ b/src/DataStreams/PushingToViewsBlockOutputStream.h @@ -59,6 +59,8 @@ private: ContextMutablePtr insert_context; void process(const Block & block, ViewInfo & view); + void process_prefix(ViewInfo & view); + void process_suffix(ViewInfo & view); void check_exceptions_in_views(); void log_query_views(); }; diff --git a/src/Interpreters/QueryViewsLog.cpp b/src/Interpreters/QueryViewsLog.cpp index 878ca2984e2..f77b5c6caf6 100644 --- a/src/Interpreters/QueryViewsLog.cpp +++ b/src/Interpreters/QueryViewsLog.cpp @@ -19,11 +19,11 @@ namespace DB { Block QueryViewsLogElement::createBlock() { - auto query_status_datatype = std::make_shared(DataTypeEnum8::Values{ - {"QueryStart", static_cast(QUERY_START)}, - {"QueryFinish", static_cast(QUERY_FINISH)}, - {"ExceptionBeforeStart", static_cast(EXCEPTION_BEFORE_START)}, - {"ExceptionWhileProcessing", static_cast(EXCEPTION_WHILE_PROCESSING)}}); + auto view_status_datatype = std::make_shared(DataTypeEnum8::Values{ + {"Init", static_cast(ViewStatus::INIT)}, + {"WrittenPrefix", static_cast(ViewStatus::WRITTEN_PREFIX)}, + {"WrittenBlock", static_cast(ViewStatus::WRITTEN_BLOCK)}, + {"WrittenSuffix", static_cast(ViewStatus::WRITTEN_SUFFIX)}}); auto view_type_datatype = std::make_shared(DataTypeEnum8::Values{ {"Default", static_cast(ViewType::DEFAULT)}, @@ -51,7 +51,7 @@ Block QueryViewsLogElement::createBlock() {std::make_shared(std::make_shared()), "ProfileEvents.Names"}, {std::make_shared(std::make_shared()), "ProfileEvents.Values"}, - {std::move(query_status_datatype), "end_status"}, + {std::move(view_status_datatype), "status"}, {std::make_shared(), "exception_code"}, {std::make_shared(), "exception"}, {std::make_shared(), "stack_trace"}}; @@ -91,7 +91,7 @@ void QueryViewsLogElement::appendToBlock(MutableColumns & columns) const columns[i++]->insertDefault(); } - columns[i++]->insert(end_status); + columns[i++]->insert(status); columns[i++]->insert(exception_code); columns[i++]->insertData(exception.data(), exception.size()); columns[i++]->insertData(stack_trace.data(), stack_trace.size()); diff --git a/src/Interpreters/QueryViewsLog.h b/src/Interpreters/QueryViewsLog.h index fe8a591358e..a2eb3666a17 100644 --- a/src/Interpreters/QueryViewsLog.h +++ b/src/Interpreters/QueryViewsLog.h @@ -23,12 +23,20 @@ class ThreadStatus; struct QueryViewsLogElement { - using Status = QueryLogElementType; + enum class ViewStatus : int8_t + { + INIT = 1, + WRITTEN_PREFIX = 2, + WRITTEN_BLOCK = 3, + WRITTEN_SUFFIX = 4 + }; + + enum class ViewType : int8_t { - DEFAULT, - MATERIALIZED, - LIVE + DEFAULT = 1, + MATERIALIZED = 2, + LIVE = 3 }; struct ViewRuntimeStats @@ -38,9 +46,9 @@ struct QueryViewsLogElement std::shared_ptr thread_status = std::make_shared(); UInt64 elapsed_ms = 0; std::chrono::time_point event_time; - Status event_status = Status::QUERY_START; + ViewStatus event_status = ViewStatus::INIT; - void setStatus(Status s) + void setStatus(ViewStatus s) { event_status = s; event_time = std::chrono::system_clock::now(); @@ -65,7 +73,7 @@ struct QueryViewsLogElement Int64 peak_memory_usage{}; std::shared_ptr profile_counters; - Status end_status{EXCEPTION_BEFORE_START}; + ViewStatus status; Int32 exception_code{}; String exception; String stack_trace; diff --git a/src/Interpreters/ThreadStatusExt.cpp b/src/Interpreters/ThreadStatusExt.cpp index 25aac7864b9..62b12d07d9c 100644 --- a/src/Interpreters/ThreadStatusExt.cpp +++ b/src/Interpreters/ThreadStatusExt.cpp @@ -514,7 +514,7 @@ void ThreadStatus::logToQueryViewsLog(const ViewInfo & vinfo) element.profile_counters = events; } - element.end_status = vinfo.runtime_stats.event_status; + element.status = vinfo.runtime_stats.event_status; element.exception_code = 0; if (vinfo.exception) { From ab05fc12e2d05a56a19c5794b361dd3ab46c3bd6 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 22 Jun 2021 18:25:17 +0200 Subject: [PATCH 037/599] Docs and settings --- .../settings.md | 27 ++++++ docs/en/operations/settings/settings.md | 15 +++- docs/en/operations/system-tables/query_log.md | 1 + .../system-tables/query_thread_log.md | 1 + .../system-tables/query_views_log.md | 83 +++++++++++++++++++ programs/server/config.xml | 20 +++-- programs/server/config.yaml.example | 10 ++- 7 files changed, 146 insertions(+), 11 deletions(-) create mode 100644 docs/en/operations/system-tables/query_views_log.md diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index 41962573546..643b71e6d70 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -869,6 +869,33 @@ If the table does not exist, ClickHouse will create it. If the structure of the ``` +## query_views_log {#server_configuration_parameters-query_views_log} + +Setting for logging views dependant of queries received with the [log_query_views=1](../../operations/settings/settings.md#settings-log-query-views) setting. + +Queries are logged in the [system.query_views_log](../../operations/system-tables/query_thread_log.md#system_tables-query_views_log) table, not in a separate file. You can change the name of the table in the `table` parameter (see below). + +Use the following parameters to configure logging: + +- `database` – Name of the database. +- `table` – Name of the system table the queries will be logged in. +- `partition_by` — [Custom partitioning key](../../engines/table-engines/mergetree-family/custom-partitioning-key.md) for a system table. Can't be used if `engine` defined. +- `engine` - [MergeTree Engine Definition](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) for a system table. Can't be used if `partition_by` defined. +- `flush_interval_milliseconds` – Interval for flushing data from the buffer in memory to the table. + +If the table does not exist, ClickHouse will create it. If the structure of the query views log changed when the ClickHouse server was updated, the table with the old structure is renamed, and a new table is created automatically. + +**Example** + +``` xml + + system + query_views_log
+ toYYYYMM(event_date) + 7500 +
+``` + ## text_log {#server_configuration_parameters-text_log} Settings for the [text_log](../../operations/system-tables/text_log.md#system_tables-text_log) system table for logging text messages. diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 9a0aa0af159..a471ef3293c 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -832,7 +832,7 @@ log_queries_min_type='EXCEPTION_WHILE_PROCESSING' Setting up query threads logging. -Queries’ threads runned by ClickHouse with this setup are logged according to the rules in the [query_thread_log](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-query_thread_log) server configuration parameter. +Queries’ threads run by ClickHouse with this setup are logged according to the rules in the [query_thread_log](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-query_thread_log) server configuration parameter. Example: @@ -840,6 +840,19 @@ Example: log_query_threads=1 ``` +## log_query_views {#settings-log-query-views} + +Setting up query views logging. + +When a query run by ClickHouse with this setup on has associated views (materialized or live views), they are logged in the [query_views_log](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-query_views_log) server configuration parameter. + +Example: + +``` text +log_query_views=1 +``` + + ## log_comment {#settings-log-comment} Specifies the value for the `log_comment` field of the [system.query_log](../system-tables/query_log.md) table and comment text for the server log. diff --git a/docs/en/operations/system-tables/query_log.md b/docs/en/operations/system-tables/query_log.md index d58e549616f..7c76d2f0a4f 100644 --- a/docs/en/operations/system-tables/query_log.md +++ b/docs/en/operations/system-tables/query_log.md @@ -155,5 +155,6 @@ Settings: {'background_pool_size':'32','load_balancing':'random','al **See Also** - [system.query_thread_log](../../operations/system-tables/query_thread_log.md#system_tables-query_thread_log) — This table contains information about each query execution thread. +- [system.query_views_log](../../operations/system-tables/query_views_log.md#system_tables-query_views_log) — This table contains information about each view executed during a query. [Original article](https://clickhouse.tech/docs/en/operations/system-tables/query_log) diff --git a/docs/en/operations/system-tables/query_thread_log.md b/docs/en/operations/system-tables/query_thread_log.md index 7ecea2971b4..152a10504bb 100644 --- a/docs/en/operations/system-tables/query_thread_log.md +++ b/docs/en/operations/system-tables/query_thread_log.md @@ -112,5 +112,6 @@ ProfileEvents: {'Query':1,'SelectQuery':1,'ReadCompressedBytes':36,'Compr **See Also** - [system.query_log](../../operations/system-tables/query_log.md#system_tables-query_log) — Description of the `query_log` system table which contains common information about queries execution. +- [system.query_views_log](../../operations/system-tables/query_views_log.md#system_tables-query_views_log) — This table contains information about each view executed during a query. [Original article](https://clickhouse.tech/docs/en/operations/system-tables/query_thread_log) diff --git a/docs/en/operations/system-tables/query_views_log.md b/docs/en/operations/system-tables/query_views_log.md new file mode 100644 index 00000000000..38d101c1636 --- /dev/null +++ b/docs/en/operations/system-tables/query_views_log.md @@ -0,0 +1,83 @@ +# system.query_views_log {#system_tables-query_views_log} + +Contains information about the dependent views executed when running a query, for example, the view type or the execution time. + +To start logging: + +1. Configure parameters in the [query_views_log](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-query_views_log) section. +2. Set [log_query_views](../../operations/settings/settings.md#settings-log-query-views) to 1. + +The flushing period of data is set in `flush_interval_milliseconds` parameter of the [query_views_log](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-query_views_log) server settings section. To force flushing, use the [SYSTEM FLUSH LOGS](../../sql-reference/statements/system.md#query_language-system-flush_logs) query. + +ClickHouse does not delete data from the table automatically. See [Introduction](../../operations/system-tables/index.md#system-tables-introduction) for more details. + +Columns: + +- `event_date` ([Date](../../sql-reference/data-types/date.md)) — The date when the last event of the view happened. +- `event_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — The date and time when the view finished execution. +- `event_time_microsecinds` ([DateTime](../../sql-reference/data-types/datetime.md)) — The date and time when the view finished execution with microseconds precision. +- `view_duration_ms` ([UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges)) — Duration of view execution (sum of its stages). +- `initial_query_id` ([String](../../sql-reference/data-types/string.md)) — ID of the initial query (for distributed query execution). +- `view_name` ([String](../../sql-reference/data-types/string.md)) — Name of the view. +- `view_uuid` ([UUID](../../sql-reference/data-types/uuid.md)) — UUID of the view. +- `view_type` ([Enum8](../../sql-reference/data-types/enum.md)) — Type of the view. Values: + - `'Default' = 1` — [Default views](../../sql-reference/statements/create/view.md#normal). Should not appear in this log. + - `'Materialized' = 2` — [Materialized views](../../sql-reference/statements/create/view.md#materialized). + - `'Live' = 3` — [Live views](../../sql-reference/statements/create/view.md#live-view). +- `view_query` ([String](../../sql-reference/data-types/string.md)) — The query executed by the view. +- `view_target` ([String](../../sql-reference/data-types/string.md)) — The name of the view target table. +- `read_rows` ([UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges)) — Number of read rows. +- `read_bytes` ([UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges)) — Number of read bytes. +- `written_rows` ([UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges)) — Number of written rows. +- `written_bytes` ([UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges)) — Number of written bytes. +- `peak_memory_usage` ([Int64](../../sql-reference/data-types/int-uint.md)) — The maximum difference between the amount of allocated and freed memory in context of this view. +- `ProfileEvents.Names` ([Array(String)](../../sql-reference/data-types/array.md)) — Counters that measure different metrics for this thread. The description of them could be found in the table [system.events](#system_tables-events). +- `ProfileEvents.Values` ([Array(UInt64)](../../sql-reference/data-types/array.md)) — Values of metrics for this thread that are listed in the `ProfileEvents.Names` column. +- `status` ([Enum8](../../sql-reference/data-types/enum.md)) — Status of the view. Values: + - `'Init' = 1` — The view was cancelled before writing anything to storage. + - `'WrittenPrefix' = 2` — The view was cancelled after writing its prefix to storage. + - `'WrittenBlock' = 3` — The view was cancelled after writing its blocks to storage. It might have materialized the input wholly, partially or none at all. + - `'WrittenSuffix' = 4` — The view wrote its suffix to storage. It completed successfully. +- `exception_code` ([Int32](../../sql-reference/data-types/int-uint.md)) — Code of an exception. +- `exception` ([String](../../sql-reference/data-types/string.md)) — Exception message. +- `stack_trace` ([String](../../sql-reference/data-types/string.md)) — [Stack trace](https://en.wikipedia.org/wiki/Stack_trace). An empty string, if the query was completed successfully. + +**Example** + +``` sql + SELECT * FROM system.query_views_log LIMIT 1 \G +``` + +``` text +Row 1: +────── +event_date: 2021-06-22 +event_time: 2021-06-22 13:23:07 +event_time_microseconds: 2021-06-22 13:23:07.738221 +view_duration_ms: 0 +initial_query_id: c3a1ac02-9cad-479b-af54-9e9c0a7afd70 +view_name: default.matview_inner +view_uuid: 00000000-0000-0000-0000-000000000000 +view_type: Materialized +view_query: SELECT * FROM default.table_b +view_target: default.`.inner.matview_inner` +read_rows: 4 +read_bytes: 64 +written_rows: 2 +written_bytes: 32 +peak_memory_usage: 4196188 +ProfileEvents.Names: ['FileOpen','WriteBufferFromFileDescriptorWrite','WriteBufferFromFileDescriptorWriteBytes','IOBufferAllocs','IOBufferAllocBytes','DiskWriteElapsedMicroseconds','InsertedRows','InsertedBytes','SelectedRows','SelectedBytes','ContextLock','RWLockAcquiredReadLocks','RealTimeMicroseconds','UserTimeMicroseconds','SystemTimeMicroseconds','SoftPageFaults'] +ProfileEvents.Values: [3,3,154,5,5242955,23,2,32,4,64,11,1,12458571345,1955,5860,110] +status: WrittenSuffix +exception_code: 0 +exception: +stack_trace: +``` + +**See Also** + +- [system.query_log](../../operations/system-tables/query_log.md#system_tables-query_log) — Description of the `query_log` system table which contains common information about queries execution. +- [system.query_thread_log](../../operations/system-tables/query_thread_log.md#system_tables-query_thread_log) — This table contains information about each query execution thread. + + +[Original article](https://clickhouse.tech/docs/en/operations/system_tables/query_thread_log) diff --git a/programs/server/config.xml b/programs/server/config.xml index 298fc176cfa..99edce8651f 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -320,7 +320,7 @@ The amount of data in mapped files can be monitored in system.metrics, system.metric_log by the MMappedFiles, MMappedFileBytes metrics and in system.asynchronous_metrics, system.asynchronous_metrics_log by the MMapCacheCells metric, - and also in system.events, system.processes, system.query_log, system.query_thread_log by the + and also in system.events, system.processes, system.query_log, system.query_thread_log, system.query_views_log by the CreatedReadBufferMMap, CreatedReadBufferMMapFailed, MMappedFileCacheHits, MMappedFileCacheMisses events. Note that the amount of data in mapped files does not consume memory directly and is not accounted in query or server memory usage - because this memory can be discarded similar to OS page cache. @@ -583,7 +583,7 @@ 9019 --> - + @@ -878,6 +878,15 @@ 7500 + + + system + query_views_log
+ toYYYYMM(event_date) + 7500 +
+ - - system - query_views_log
- toYYYYMM(event_date) - 7500 -
+ + system + query_views_log
+ toYYYYMM(event_date) + 7500 +
+ +``` sql +INSERT INTO id_val VALUES (1,11)(2,12)(3,13); +``` + +Creating the right-side `Join` table: + +``` sql +CREATE TABLE id_val_join(`id` UInt32, `val` UInt8) ENGINE = Join(ANY, LEFT, id); +``` + +``` sql +INSERT INTO id_val_join VALUES (1,21)(1,22)(3,23); +``` + +Joining the tables: + +``` sql +SELECT * FROM id_val ANY LEFT JOIN id_val_join USING (id); +``` + +``` text +┌─id─┬─val─┬─id_val_join.val─┐ +│ 1 │ 11 │ 21 │ +│ 2 │ 12 │ 0 │ +│ 3 │ 13 │ 23 │ +└────┴─────┴─────────────────┘ +``` + +As an alternative, you can retrieve data from the `Join` table, specifying the join key value: + +``` sql +SELECT joinGet('id_val_join', 'val', toUInt32(1)); +``` + +``` text +┌─joinGet('id_val_join', 'val', toUInt32(1))─┐ +│ 21 │ +└────────────────────────────────────────────┘ +``` + +Deleting a row from the `Join` table: + +```sql +ALTER TABLE id_val_join DELETE WHERE id = 3; +``` + +```text +┌─id─┬─val─┐ +│ 1 │ 21 │ +└────┴─────┘ +``` From 6ac406a019d790eadc2639c7860caa9238d729d5 Mon Sep 17 00:00:00 2001 From: olgarev Date: Sun, 1 Aug 2021 05:31:25 +0000 Subject: [PATCH 082/599] Translated --- docs/ru/engines/table-engines/special/join.md | 117 ++++++++++-------- 1 file changed, 66 insertions(+), 51 deletions(-) diff --git a/docs/ru/engines/table-engines/special/join.md b/docs/ru/engines/table-engines/special/join.md index ef27ac3f10f..e16ed283e77 100644 --- a/docs/ru/engines/table-engines/special/join.md +++ b/docs/ru/engines/table-engines/special/join.md @@ -27,57 +27,15 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] Вводите параметры `join_strictness` и `join_type` без кавычек, например, `Join(ANY, LEFT, col1)`. Они должны быть такими же как и в той операции `JOIN`, в которой таблица будет использоваться. Если параметры не совпадают, ClickHouse не генерирует исключение и может возвращать неверные данные. -## Использование таблицы {#ispolzovanie-tablitsy} +## Особенности и рекомендации {#specifics-and-recommendations} -### Пример {#primer} +### Хранение данных {#data-storage} -Создание левой таблицы: +Данные таблиц `Join` всегда находятся в оперативной памяти. При вставке строк в таблицу ClickHouse записывает блоки данных в каталог на диске, чтобы их можно было восстановить при перезапуске сервера. -``` sql -CREATE TABLE id_val(`id` UInt32, `val` UInt32) ENGINE = TinyLog -``` +При аварийном перезапуске сервера блок данных на диске может быть потерян или повреждён. В последнем случае может потребоваться вручную удалить файл с повреждёнными данными. -``` sql -INSERT INTO id_val VALUES (1,11)(2,12)(3,13) -``` - -Создание правой таблицы с движком `Join`: - -``` sql -CREATE TABLE id_val_join(`id` UInt32, `val` UInt8) ENGINE = Join(ANY, LEFT, id) -``` - -``` sql -INSERT INTO id_val_join VALUES (1,21)(1,22)(3,23) -``` - -Объединение таблиц: - -``` sql -SELECT * FROM id_val ANY LEFT JOIN id_val_join USING (id) SETTINGS join_use_nulls = 1 -``` - -``` text -┌─id─┬─val─┬─id_val_join.val─┐ -│ 1 │ 11 │ 21 │ -│ 2 │ 12 │ ᴺᵁᴸᴸ │ -│ 3 │ 13 │ 23 │ -└────┴─────┴─────────────────┘ -``` - -В качестве альтернативы, можно извлечь данные из таблицы `Join`, указав значение ключа объединения: - -``` sql -SELECT joinGet('id_val_join', 'val', toUInt32(1)) -``` - -``` text -┌─joinGet('id_val_join', 'val', toUInt32(1))─┐ -│ 21 │ -└────────────────────────────────────────────┘ -``` - -### Выборка и вставка данных {#vyborka-i-vstavka-dannykh} +### Выборка и добавление данных {#selecting-and-inserting-data} Для добавления данных в таблицы с движком `Join` используйте запрос `INSERT`. Если таблица создавалась со строгостью `ANY`, то данные с повторяющимися ключами игнорируются. Если задавалась строгость `ALL`, то добавляются все строки. @@ -86,9 +44,13 @@ SELECT joinGet('id_val_join', 'val', toUInt32(1)) - Используйте таблицу как правую в секции `JOIN`. - Используйте функцию [joinGet](../../../engines/table-engines/special/join.md#joinget), которая позволяет извлекать данные из таблицы таким же образом как из словаря. +### Удаление данных {#deleting-data} + +Запросы `ALTER DELETE` для таблиц с движком `Join` выполняются как [мутации](../../../sql-reference/statements/alter/index.md#mutations). При выполнении мутации `DELETE` считываются отфильтрованные данные и перезаписываются данные в оперативную память и на диск. + ### Ограничения и настройки {#join-limitations-and-settings} -При создании таблицы, применяются следующие параметры : +При создании таблицы применяются следующие настройки: - [join_use_nulls](../../../operations/settings/settings.md#join_use_nulls) - [max_rows_in_join](../../../operations/settings/query-complexity.md#settings-max_rows_in_join) @@ -101,9 +63,62 @@ SELECT joinGet('id_val_join', 'val', toUInt32(1)) Движок `Join` позволяет использовать параметр [join_use_nulls](../../../operations/settings/settings.md#join_use_nulls) в запросе `CREATE TABLE`, который также можно использовать в запросе [SELECT](../../../engines/table-engines/special/join.md). Если у вас разные настройки `join_use_nulls`, вы можете получить сообщение об ошибке при объединении таблиц. Это зависит от типа соединения. Когда вы используете функцию [joinGet](../../../engines/table-engines/special/join.md#joinget), вам необходимо использовать один и тот же параметр `join_use_nulls` в запросах `CRATE TABLE` и `SELECT`. -## Хранение данных {#khranenie-dannykh} +## Примеры использования {#example} -Данные таблиц `Join` всегда находятся в RAM. При вставке строк в таблицу ClickHouse записывает блоки данных в каталог на диске, чтобы их можно было восстановить при перезапуске сервера. +Создание левой таблицы: -При аварийном перезапуске сервера блок данных на диске может быть потерян или повреждён. В последнем случае, может потребоваться вручную удалить файл с повреждёнными данными. +``` sql +CREATE TABLE id_val(`id` UInt32, `val` UInt32) ENGINE = TinyLog; +``` +``` sql +INSERT INTO id_val VALUES (1,11)(2,12)(3,13); +``` + +Создание правой таблицы с движком `Join`: + +``` sql +CREATE TABLE id_val_join(`id` UInt32, `val` UInt8) ENGINE = Join(ANY, LEFT, id); +``` + +``` sql +INSERT INTO id_val_join VALUES (1,21)(1,22)(3,23); +``` + +Объединение таблиц: + +``` sql +SELECT * FROM id_val ANY LEFT JOIN id_val_join USING (id); +``` + +``` text +┌─id─┬─val─┬─id_val_join.val─┐ +│ 1 │ 11 │ 21 │ +│ 2 │ 12 │ 0 │ +│ 3 │ 13 │ 23 │ +└────┴─────┴─────────────────┘ +``` + +В качестве альтернативы, можно извлечь данные из таблицы `Join`, указав значение ключа объединения: + +``` sql +SELECT joinGet('id_val_join', 'val', toUInt32(1)); +``` + +``` text +┌─joinGet('id_val_join', 'val', toUInt32(1))─┐ +│ 21 │ +└────────────────────────────────────────────┘ +``` + +Удаление данных из таблицы `Join`: + +```sql +ALTER TABLE id_val_join DELETE WHERE id = 3; +``` + +```text +┌─id─┬─val─┐ +│ 1 │ 21 │ +└────┴─────┘ +``` From b4a8f3c003840f56afc00332c8cced653db8d40d Mon Sep 17 00:00:00 2001 From: olgarev Date: Sun, 1 Aug 2021 05:41:12 +0000 Subject: [PATCH 083/599] Fix --- docs/ru/engines/table-engines/special/join.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/engines/table-engines/special/join.md b/docs/ru/engines/table-engines/special/join.md index e16ed283e77..4735637e8c1 100644 --- a/docs/ru/engines/table-engines/special/join.md +++ b/docs/ru/engines/table-engines/special/join.md @@ -46,7 +46,7 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] ### Удаление данных {#deleting-data} -Запросы `ALTER DELETE` для таблиц с движком `Join` выполняются как [мутации](../../../sql-reference/statements/alter/index.md#mutations). При выполнении мутации `DELETE` считываются отфильтрованные данные и перезаписываются данные в оперативную память и на диск. +Запросы `ALTER DELETE` для таблиц с движком `Join` выполняются как [мутации](../../../sql-reference/statements/alter/index.md#mutations). При выполнении мутации `DELETE` считываются отфильтрованные данные и перезаписываются в оперативную память и на диск. ### Ограничения и настройки {#join-limitations-and-settings} From 788e690ba811bb3ef17ade401d8d84e954348a4e Mon Sep 17 00:00:00 2001 From: Alexey Date: Sun, 1 Aug 2021 13:02:35 +0000 Subject: [PATCH 084/599] First draft --- docs/en/sql-reference/statements/attach.md | 27 +++++++++-- docs/en/sql-reference/statements/detach.md | 15 ++++-- docs/en/sql-reference/statements/exchange.md | 41 ++++++++++++++++ docs/en/sql-reference/statements/rename.md | 50 ++++++++++++++++++-- 4 files changed, 118 insertions(+), 15 deletions(-) create mode 100644 docs/en/sql-reference/statements/exchange.md diff --git a/docs/en/sql-reference/statements/attach.md b/docs/en/sql-reference/statements/attach.md index 84165d30357..03ef2e54a93 100644 --- a/docs/en/sql-reference/statements/attach.md +++ b/docs/en/sql-reference/statements/attach.md @@ -5,14 +5,21 @@ toc_title: ATTACH # ATTACH Statement {#attach} -Attaches the table, for example, when moving a database to another server. +Attaches a table or a dictionary, for example, when moving a database to another server. + +**Syntax** + +``` sql +ATTACH TABLE|DICTIONARY [IF NOT EXISTS] [db.]name [ON CLUSTER cluster] +``` The query does not create data on the disk, but assumes that data is already in the appropriate places, and just adds information about the table to the server. After executing an `ATTACH` query, the server will know about the existence of the table. -If the table was previously detached ([DETACH](../../sql-reference/statements/detach.md)) query, meaning that its structure is known, you can use shorthand without defining the structure. +If a table or a dictionary was previously detached ([DETACH](../../sql-reference/statements/detach.md) query), meaning that its structure is known, you can use shorthand without defining the structure. -## Syntax Forms {#syntax-forms} -### Attach Existing Table {#attach-existing-table} +## Attach Existing Table {#attach-existing-table} + +**Syntax** ``` sql ATTACH TABLE [IF NOT EXISTS] [db.]name [ON CLUSTER cluster] @@ -22,10 +29,12 @@ This query is used when starting the server. The server stores table metadata as If the table was detached permanently, it won't be reattached at the server start, so you need to use `ATTACH` query explicitly. -### Сreate New Table And Attach Data {#create-new-table-and-attach-data} +## Сreate New Table And Attach Data {#create-new-table-and-attach-data} **With specify path to table data** +**Syntax** + ```sql ATTACH TABLE name FROM 'path/to/data/' (col1 Type1, ...) ``` @@ -57,3 +66,11 @@ ATTACH TABLE name UUID '' (col1 Type1, ...) ``` It creates new table with provided structure and attaches data from table with the specified UUID. + +## Attach Existing Dictionary {#attach-existing-dictionary} + +**Syntax** + +``` sql +ATTACH DICTIONARY [IF NOT EXISTS] [db.]name [ON CLUSTER cluster] +``` diff --git a/docs/en/sql-reference/statements/detach.md b/docs/en/sql-reference/statements/detach.md index 049c5a5dad9..a84432745c5 100644 --- a/docs/en/sql-reference/statements/detach.md +++ b/docs/en/sql-reference/statements/detach.md @@ -5,17 +5,18 @@ toc_title: DETACH # DETACH Statement {#detach} -Makes the server "forget" about the existence of the table or materialized view. +Makes the server "forget" about the existence of a table, a materialized view or a dictionary. Syntax: ``` sql -DETACH TABLE|VIEW [IF EXISTS] [db.]name [ON CLUSTER cluster] [PERMANENTLY] +DETACH TABLE|VIEW|DICTIONARY [IF EXISTS] [db.]name [ON CLUSTER cluster] [PERMANENTLY] ``` -Detaching does not delete the data or metadata for the table or materialized view. If the table or view was not detached `PERMANENTLY`, on the next server launch the server will read the metadata and recall the table/view again. If the table or view was detached `PERMANENTLY`, there will be no automatic recall. +Detaching does not delete the data or metadata of the table, the materialized view or the dictionary. If the entity was not detached `PERMANENTLY`, on the next server launch the server will read the metadata and recall the table/view/dictionary again. If the entiry was detached `PERMANENTLY`, there will be no automatic recall. -Whether the table was detached permanently or not, in both cases you can reattach it using the [ATTACH](../../sql-reference/statements/attach.md). System log tables can be also attached back (e.g. `query_log`, `text_log`, etc). Other system tables can't be reattached. On the next server launch the server will recall those tables again. +Whether a table or a dictionary was detached permanently or not, in both cases you can reattach them using the [ATTACH](../../sql-reference/statements/attach.md). +System log tables can be also attached back (e.g. `query_log`, `text_log`, etc). Other system tables can't be reattached. On the next server launch the server will recall those tables again. `ATTACH MATERIALIZED VIEW` does not work with short syntax (without `SELECT`), but you can attach it using the `ATTACH TABLE` query. @@ -67,4 +68,8 @@ Received exception from server (version 21.4.1): Code: 60. DB::Exception: Received from localhost:9000. DB::Exception: Table default.test does not exist. ``` -[Original article](https://clickhouse.tech/docs/en/sql-reference/statements/detach/) +**See Also** + +- [Materialized View](../../sql-reference/statements/create/view.md/#materialized) +- [Dictionaries](../../sql-reference/dictionaries/index.md) + diff --git a/docs/en/sql-reference/statements/exchange.md b/docs/en/sql-reference/statements/exchange.md new file mode 100644 index 00000000000..c56a2224746 --- /dev/null +++ b/docs/en/sql-reference/statements/exchange.md @@ -0,0 +1,41 @@ +--- +toc_priority: 49 +toc_title: EXCHANGE +--- + +# EXCHANGE Statement {#exchange} + +Exchanges names of two tables or dictionaries in an atomic query. + +!!! note "Note" + An `EXCHANGE` query is supported by [Atomic](../../engines/database-engines/atomic.md) database engine only. + +**Syntax** + +```sql +EXCHANGE TABLES|DICTIONARIES [db0.]name_A AND [db1.]name_B +``` + +## EXCHANGE TABLES {#exchange_tables} + +Exchanges names of two tables in an atomic query. + +**Syntax** + +```sql +EXCHANGE TABLES [db0.]table_A AND [db1.]table_B +``` + +## EXCHANGE DICTIONARIES {#exchange_dictionaries} + +Exchanges names of two dictionaries in an atomic query. + +**Syntax** + +```sql +EXCHANGE DICTIONARIES [db0.]dict_A AND [db1.]dict_B +``` + +**See Also** + +- [Dictionaries](../../sql-reference/dictionaries/index.md) diff --git a/docs/en/sql-reference/statements/rename.md b/docs/en/sql-reference/statements/rename.md index 3591c187e93..5d1b56e14ef 100644 --- a/docs/en/sql-reference/statements/rename.md +++ b/docs/en/sql-reference/statements/rename.md @@ -5,18 +5,58 @@ toc_title: RENAME # RENAME Statement {#misc_operations-rename} -## RENAME DATABASE {#misc_operations-rename_database} -Renames database, it is supported only for Atomic database engine. +Renames databases, tables and dictionaries. Several entities can be renamed in a single query. +Note that the `RENAME` query with several entities is non-atomic operation. To swap entity names atomically, use [EXCHANGE](./exchange.md) statement. +!!! note "Note" + A `RENAME` query is supported by [Atomic](../../engines/database-engines/atomic.md) database engine only. + +**Syntax** + +```sql +RENAME DATABASE|TABLE|DICTIONARY name TO new_name [,...] [ON CLUSTER cluster] ``` -RENAME DATABASE atomic_database1 TO atomic_database2 [ON CLUSTER cluster] + +## RENAME DATABASE {#misc_operations-rename_database} + +Renames databases. I + +**Syntax** + +sql +``` +RENAME DATABASE atomic_database1 TO atomic_database2 [,...] [ON CLUSTER cluster] ``` ## RENAME TABLE {#misc_operations-rename_table} + Renames one or more tables. +**Syntax** + ``` sql -RENAME TABLE [db11.]name11 TO [db12.]name12, [db21.]name21 TO [db22.]name22, ... [ON CLUSTER cluster] +RENAME TABLE [db1.]name1 TO [db2.]name2 [,...] [ON CLUSTER cluster] ``` -Renaming tables is a light operation. If you indicated another database after `TO`, the table will be moved to this database. However, the directories with databases must reside in the same file system (otherwise, an error is returned). If you rename multiple tables in one query, this is a non-atomic operation, it may be partially executed, queries in other sessions may receive the error `Table ... does not exist ..`. +Renaming tables is a light operation. If you pass a different database after `TO`, the table will be moved to this database. However, the directories with databases must reside in the same file system. Otherwise, an error is returned. +If you rename multiple tables in one query, this is a non-atomic operation, it may be partially executed, queries in other sessions may receive the error `Table ... does not exist ..`. + +**Example** + +```sql +RENAME TABLE table_A TO table_A_bak, table_B TO table_B_bak; +``` + +## RENAME DICTIONARY {#rename_dictionary} + +Renames one or several dictionaries. This query can be used to move dictionaries between databases. + +**Syntax** + +```sql +RENAME DICTIONARY [db0.]dict_A TO [db1.]dict_B [,...] [ON CLUSTER cluster] +``` + +**See Also** + +- [Dictionaries](../../sql-reference/dictionaries/index.md) \ No newline at end of file From 0a36d8a607a69cce1c05f7296e62abc591ff9083 Mon Sep 17 00:00:00 2001 From: Alexey Date: Sun, 1 Aug 2021 13:03:02 +0000 Subject: [PATCH 085/599] SET moved in the TOC --- docs/en/sql-reference/statements/set.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/statements/set.md b/docs/en/sql-reference/statements/set.md index c6c8d28257d..e5de5c41284 100644 --- a/docs/en/sql-reference/statements/set.md +++ b/docs/en/sql-reference/statements/set.md @@ -1,5 +1,5 @@ --- -toc_priority: 49 +toc_priority: 50 toc_title: SET --- From 722e092059a1192f9af58152f1324030cd2100ef Mon Sep 17 00:00:00 2001 From: Alexey Date: Sun, 1 Aug 2021 16:17:50 +0000 Subject: [PATCH 086/599] Updates --- docs/en/sql-reference/statements/attach.md | 8 ++++---- docs/en/sql-reference/statements/detach.md | 8 ++++---- docs/en/sql-reference/statements/exchange.md | 9 +++++---- docs/en/sql-reference/statements/rename.md | 15 +++++++-------- 4 files changed, 20 insertions(+), 20 deletions(-) diff --git a/docs/en/sql-reference/statements/attach.md b/docs/en/sql-reference/statements/attach.md index 03ef2e54a93..63e4b87b9f8 100644 --- a/docs/en/sql-reference/statements/attach.md +++ b/docs/en/sql-reference/statements/attach.md @@ -10,12 +10,12 @@ Attaches a table or a dictionary, for example, when moving a database to another **Syntax** ``` sql -ATTACH TABLE|DICTIONARY [IF NOT EXISTS] [db.]name [ON CLUSTER cluster] +ATTACH TABLE|DICTIONARY [IF NOT EXISTS] [db.]name [ON CLUSTER cluster] ... ``` -The query does not create data on the disk, but assumes that data is already in the appropriate places, and just adds information about the table to the server. After executing an `ATTACH` query, the server will know about the existence of the table. +The query does not create data on the disk, but assumes that data is already in the appropriate places, and just adds information about the table or the dictionary to the server. After executing the `ATTACH` query, the server will know about the existence of the table or the dictionary. -If a table or a dictionary was previously detached ([DETACH](../../sql-reference/statements/detach.md) query), meaning that its structure is known, you can use shorthand without defining the structure. +If a table was previously detached ([DETACH](../../sql-reference/statements/detach.md) query), meaning that its structure is known, you can use shorthand without defining the structure. ## Attach Existing Table {#attach-existing-table} @@ -29,7 +29,7 @@ This query is used when starting the server. The server stores table metadata as If the table was detached permanently, it won't be reattached at the server start, so you need to use `ATTACH` query explicitly. -## Сreate New Table And Attach Data {#create-new-table-and-attach-data} +## Create New Table And Attach Data {#create-new-table-and-attach-data} **With specify path to table data** diff --git a/docs/en/sql-reference/statements/detach.md b/docs/en/sql-reference/statements/detach.md index a84432745c5..4260c914962 100644 --- a/docs/en/sql-reference/statements/detach.md +++ b/docs/en/sql-reference/statements/detach.md @@ -7,15 +7,15 @@ toc_title: DETACH Makes the server "forget" about the existence of a table, a materialized view or a dictionary. -Syntax: +**Syntax** ``` sql DETACH TABLE|VIEW|DICTIONARY [IF EXISTS] [db.]name [ON CLUSTER cluster] [PERMANENTLY] ``` -Detaching does not delete the data or metadata of the table, the materialized view or the dictionary. If the entity was not detached `PERMANENTLY`, on the next server launch the server will read the metadata and recall the table/view/dictionary again. If the entiry was detached `PERMANENTLY`, there will be no automatic recall. +Detaching does not delete the data or metadata of the table, the materialized view or the dictionary. If the entity was not detached `PERMANENTLY`, on the next server launch the server will read the metadata and recall the table/view/dictionary again. If the entity was detached `PERMANENTLY`, there will be no automatic recall. -Whether a table or a dictionary was detached permanently or not, in both cases you can reattach them using the [ATTACH](../../sql-reference/statements/attach.md). +Whether a table or a dictionary was detached permanently or not, in both cases you can reattach them using the [ATTACH](../../sql-reference/statements/attach.md) query. System log tables can be also attached back (e.g. `query_log`, `text_log`, etc). Other system tables can't be reattached. On the next server launch the server will recall those tables again. `ATTACH MATERIALIZED VIEW` does not work with short syntax (without `SELECT`), but you can attach it using the `ATTACH TABLE` query. @@ -70,6 +70,6 @@ Code: 60. DB::Exception: Received from localhost:9000. DB::Exception: Table defa **See Also** -- [Materialized View](../../sql-reference/statements/create/view.md/#materialized) +- [Materialized View](../../sql-reference/statements/create/view.md#materialized) - [Dictionaries](../../sql-reference/dictionaries/index.md) diff --git a/docs/en/sql-reference/statements/exchange.md b/docs/en/sql-reference/statements/exchange.md index c56a2224746..7abc86330ba 100644 --- a/docs/en/sql-reference/statements/exchange.md +++ b/docs/en/sql-reference/statements/exchange.md @@ -5,10 +5,11 @@ toc_title: EXCHANGE # EXCHANGE Statement {#exchange} -Exchanges names of two tables or dictionaries in an atomic query. +Exchanges the names of two tables or dictionaries atomically. +This task can also be accomplished with a [RENAME](./rename.md) query using a temporary name. But the operation in not atomic in that case. !!! note "Note" - An `EXCHANGE` query is supported by [Atomic](../../engines/database-engines/atomic.md) database engine only. + The `EXCHANGE` query is supported by the [Atomic](../../engines/database-engines/atomic.md) database engine only. **Syntax** @@ -18,7 +19,7 @@ EXCHANGE TABLES|DICTIONARIES [db0.]name_A AND [db1.]name_B ## EXCHANGE TABLES {#exchange_tables} -Exchanges names of two tables in an atomic query. +Exchanges the names of two tables. **Syntax** @@ -28,7 +29,7 @@ EXCHANGE TABLES [db0.]table_A AND [db1.]table_B ## EXCHANGE DICTIONARIES {#exchange_dictionaries} -Exchanges names of two dictionaries in an atomic query. +Exchanges the names of two dictionaries. **Syntax** diff --git a/docs/en/sql-reference/statements/rename.md b/docs/en/sql-reference/statements/rename.md index 5d1b56e14ef..a2c4a305dce 100644 --- a/docs/en/sql-reference/statements/rename.md +++ b/docs/en/sql-reference/statements/rename.md @@ -5,11 +5,11 @@ toc_title: RENAME # RENAME Statement {#misc_operations-rename} -Renames databases, tables and dictionaries. Several entities can be renamed in a single query. +Renames databases, tables or dictionaries. Several entities can be renamed in a single query. Note that the `RENAME` query with several entities is non-atomic operation. To swap entity names atomically, use [EXCHANGE](./exchange.md) statement. !!! note "Note" - A `RENAME` query is supported by [Atomic](../../engines/database-engines/atomic.md) database engine only. + The `RENAME` query is supported by the [Atomic](../../engines/database-engines/atomic.md) database engine only. **Syntax** @@ -19,12 +19,11 @@ RENAME DATABASE|TABLE|DICTIONARY name TO new_name [,...] [ON CLUSTER cluster] ## RENAME DATABASE {#misc_operations-rename_database} -Renames databases. I +Renames databases. **Syntax** -sql -``` +```sql RENAME DATABASE atomic_database1 TO atomic_database2 [,...] [ON CLUSTER cluster] ``` @@ -32,15 +31,15 @@ RENAME DATABASE atomic_database1 TO atomic_database2 [,...] [ON CLUSTER cluster] Renames one or more tables. +Renaming tables is a light operation. If you pass a different database after `TO`, the table will be moved to this database. However, the directories with databases must reside in the same file system. Otherwise, an error is returned. +If you rename multiple tables in one query, the operation is not atomic. It may be partially executed, and queries in other sessions may get `Table ... does not exist ...` error. + **Syntax** ``` sql RENAME TABLE [db1.]name1 TO [db2.]name2 [,...] [ON CLUSTER cluster] ``` -Renaming tables is a light operation. If you pass a different database after `TO`, the table will be moved to this database. However, the directories with databases must reside in the same file system. Otherwise, an error is returned. -If you rename multiple tables in one query, this is a non-atomic operation, it may be partially executed, queries in other sessions may receive the error `Table ... does not exist ..`. - **Example** ```sql From 53c3a4fbc6fb0129b4e225b035e0c79ac8365014 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 2 Aug 2021 14:57:51 +0300 Subject: [PATCH 087/599] Fix excessive logging in NuRaft on server shutdown --- contrib/NuRaft | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/NuRaft b/contrib/NuRaft index 0ce94900930..b721083987c 160000 --- a/contrib/NuRaft +++ b/contrib/NuRaft @@ -1 +1 @@ -Subproject commit 0ce9490093021c63564cca159571a8b27772ad48 +Subproject commit b721083987c3ef306cf311fd59f0722c20c08d7d From f3e0f648d09e77f01b3ee8561f7ed808e722c3fa Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 2 Aug 2021 15:00:05 +0300 Subject: [PATCH 088/599] Fixup --- contrib/NuRaft | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/NuRaft b/contrib/NuRaft index b721083987c..7ecb16844af 160000 --- a/contrib/NuRaft +++ b/contrib/NuRaft @@ -1 +1 @@ -Subproject commit b721083987c3ef306cf311fd59f0722c20c08d7d +Subproject commit 7ecb16844af6a9c283ad432d85ecc2e7d1544676 From 740c16939658e0752dfff6b15e8d43758b77bee4 Mon Sep 17 00:00:00 2001 From: Anton Ivashkin Date: Mon, 2 Aug 2021 16:08:27 +0300 Subject: [PATCH 089/599] Fix test_merge_tree_s3_failover with debug build --- .../test_merge_tree_s3_failover/test.py | 27 +++++++++++++------ 1 file changed, 19 insertions(+), 8 deletions(-) diff --git a/tests/integration/test_merge_tree_s3_failover/test.py b/tests/integration/test_merge_tree_s3_failover/test.py index 4dec1bc713f..d19236f4425 100644 --- a/tests/integration/test_merge_tree_s3_failover/test.py +++ b/tests/integration/test_merge_tree_s3_failover/test.py @@ -68,17 +68,19 @@ def drop_table(cluster): # S3 request will be failed for an appropriate part file write. FILES_PER_PART_BASE = 5 # partition.dat, default_compression_codec.txt, count.txt, columns.txt, checksums.txt FILES_PER_PART_WIDE = FILES_PER_PART_BASE + 1 + 1 + 3 * 2 # Primary index, MinMax, Mark and data file for column(s) +FILES_PER_PART_WIDE_DEBUG = 2 # Additional requests to S3 in debug build FILES_PER_PART_COMPACT = FILES_PER_PART_BASE + 1 + 1 + 2 +FILES_PER_PART_COMPACT_DEBUG = 0 @pytest.mark.parametrize( - "min_bytes_for_wide_part,request_count", + "min_bytes_for_wide_part,request_count,debug_request_count", [ - (0, FILES_PER_PART_WIDE), - (1024 * 1024, FILES_PER_PART_COMPACT) + (0, FILES_PER_PART_WIDE, FILES_PER_PART_WIDE_DEBUG), + (1024 * 1024, FILES_PER_PART_COMPACT, FILES_PER_PART_COMPACT_DEBUG) ] ) -def test_write_failover(cluster, min_bytes_for_wide_part, request_count): +def test_write_failover(cluster, min_bytes_for_wide_part, request_count, debug_request_count): node = cluster.instances["node"] node.query( @@ -95,17 +97,24 @@ def test_write_failover(cluster, min_bytes_for_wide_part, request_count): .format(min_bytes_for_wide_part) ) - for request in range(request_count + 1): + is_debug_mode = False + success_count = 0 + + for request in range(request_count + debug_request_count + 1): # Fail N-th request to S3. fail_request(cluster, request + 1) data = "('2020-03-01',0,'data'),('2020-03-01',1,'data')" - positive = request == request_count + positive = request >= (request_count + debug_request_count if is_debug_mode else request_count) try: node.query("INSERT INTO s3_failover_test VALUES {}".format(data)) - assert positive, "Insert query should be failed, request {}".format(request) + success_count += 1 except QueryRuntimeException as e: + if not is_debug_mode and positive: + is_debug_mode = True + positive = False + assert not positive, "Insert query shouldn't be failed, request {}".format(request) assert str(e).find("Expected Error") != -1, "Unexpected error {}".format(str(e)) @@ -114,7 +123,9 @@ def test_write_failover(cluster, min_bytes_for_wide_part, request_count): fail_request(cluster, 0) assert node.query("CHECK TABLE s3_failover_test") == '1\n' - assert node.query("SELECT * FROM s3_failover_test FORMAT Values") == data + assert success_count > 1 or node.query("SELECT * FROM s3_failover_test FORMAT Values") == data + + assert success_count == (1 if is_debug_mode else debug_request_count + 1), "Insert query should be successful at least once" # Check that second data part move is ended successfully if first attempt was failed. From 04370ccd469ef0ca066f5d663aa9ed0b9f1c09eb Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Mon, 2 Aug 2021 18:35:25 +0300 Subject: [PATCH 090/599] Add setting descr. and update range function. --- docs/en/operations/settings/settings.md | 11 +++++++++++ docs/en/sql-reference/functions/array-functions.md | 4 ++-- 2 files changed, 13 insertions(+), 2 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 1646794914d..73c51f11c91 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -111,7 +111,18 @@ It makes sense to disable it if the server has millions of tiny tables that are ## function_range_max_elements_in_block {#settings-function_range_max_elements_in_block} +Sets the safety threshold for data volume generated by function [range](../../sql-reference/functions/array-functions.md#range). Maximum number of values generated by function per block of data (sum of array sizes for every row in a block). +Possible values: + +- Positive integer. + +Default value: `500000000`. + +**See Also** + +- [max_block_size](#setting-max_block_size). +- [min_insert_block_size_rows](#min-insert-block-size-rows). ## enable_http_compression {#settings-enable_http_compression} diff --git a/docs/en/sql-reference/functions/array-functions.md b/docs/en/sql-reference/functions/array-functions.md index b56d403edf6..e9e466e7ea3 100644 --- a/docs/en/sql-reference/functions/array-functions.md +++ b/docs/en/sql-reference/functions/array-functions.md @@ -68,8 +68,8 @@ range([start, ] end [, step]) **Implementation details** - All arguments must be positive values: `start`, `end`, `step` are `UInt` data types, as well as elements of the returned array. -- An exception is thrown if query results in arrays with a total length of more than 100,000,000 elements. - +- An exception is thrown if query results in arrays with a total length of more than + number of elements specified by the [function_range_max_elements_in_block](../../operations/settings/settings.md#settings-function_range_max_elements_in_block) setting. **Examples** From 73a5437551b8d3e67850b1521d071180ee22f0f6 Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Mon, 2 Aug 2021 19:07:45 +0300 Subject: [PATCH 091/599] Add setting descr. and update range function. --- docs/en/operations/settings/settings.md | 2 +- .../en/sql-reference/functions/array-functions.md | 4 ++-- docs/ru/operations/settings/settings.md | 15 +++++++++++++++ .../ru/sql-reference/functions/array-functions.md | 5 +---- 4 files changed, 19 insertions(+), 7 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 73c51f11c91..eb3fa74ec87 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -111,7 +111,7 @@ It makes sense to disable it if the server has millions of tiny tables that are ## function_range_max_elements_in_block {#settings-function_range_max_elements_in_block} -Sets the safety threshold for data volume generated by function [range](../../sql-reference/functions/array-functions.md#range). Maximum number of values generated by function per block of data (sum of array sizes for every row in a block). +Sets the safety threshold for data volume generated by function [range](../../sql-reference/functions/array-functions.md#range). It defines the maximum number of values generated by function per block of data (sum of array sizes for every row in a block). Possible values: diff --git a/docs/en/sql-reference/functions/array-functions.md b/docs/en/sql-reference/functions/array-functions.md index e9e466e7ea3..422bbe4b4ea 100644 --- a/docs/en/sql-reference/functions/array-functions.md +++ b/docs/en/sql-reference/functions/array-functions.md @@ -68,8 +68,8 @@ range([start, ] end [, step]) **Implementation details** - All arguments must be positive values: `start`, `end`, `step` are `UInt` data types, as well as elements of the returned array. -- An exception is thrown if query results in arrays with a total length of more than - number of elements specified by the [function_range_max_elements_in_block](../../operations/settings/settings.md#settings-function_range_max_elements_in_block) setting. +- An exception is thrown if query results in arrays with a total length of more than number of elements specified by the [function_range_max_elements_in_block](../../operations/settings/settings.md#settings-function_range_max_elements_in_block) setting. + **Examples** diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index 3a023c71cc9..3de6654b116 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -85,6 +85,21 @@ ClickHouse применяет настройку в тех случаях, ко Имеет смысл выключать, если на сервере миллионы мелких таблиц-чанков, которые постоянно создаются и уничтожаются. +## function_range_max_elements_in_block {#settings-function_range_max_elements_in_block} + +Устанавливает порог безопасности для объема данных, создаваемого функцией [range](../../sql-reference/functions/array-functions.md#range). Задаётся максимальное количество значений, генерируемых функцией на блок данных (сумма размеров массива для каждой строки в блоке). + +Возможные значения: + +- Положительное целое. + +Значение по умолчанию: `500000000`. + +**См. также** + +- [max_block_size](#setting-max_block_size). +- [min_insert_block_size_rows](#min-insert-block-size-rows). + ## enable_http_compression {#settings-enable_http_compression} Включает или отключает сжатие данных в ответе на HTTP-запрос. diff --git a/docs/ru/sql-reference/functions/array-functions.md b/docs/ru/sql-reference/functions/array-functions.md index 67c4f6c0136..52fd63864ce 100644 --- a/docs/ru/sql-reference/functions/array-functions.md +++ b/docs/ru/sql-reference/functions/array-functions.md @@ -62,18 +62,15 @@ range([start, ] end [, step]) - `end` — конец диапазона. Обязательный аргумент. Должен быть больше, чем `start`. Тип: [UInt](../data-types/int-uint.md) - `step` — шаг обхода. Необязательный аргумент. По умолчанию равен `1`. Тип: [UInt](../data-types/int-uint.md) - **Возвращаемые значения** - массив `UInt` чисел от `start` до `end - 1` с шагом `step` - **Особенности реализации** - Не поддерживаются отрицательные значения аргументов: `start`, `end`, `step` имеют тип `UInt`. -- Если в результате запроса создаются массивы суммарной длиной больше 100 000 000 элементов, то генерируется исключение. - +- Если в результате запроса создаются массивы суммарной длиной больше, чем количество элементов, указанное настройкой [function_range_max_elements_in_block](../../operations/settings/settings.md#settings-function_range_max_elements_in_block), то генерируется исключение. **Примеры** From 8df8a498b367bc63e5e383074500cb8840e4dac4 Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Mon, 2 Aug 2021 19:43:44 +0300 Subject: [PATCH 092/599] Add setting descr. and update range function. --- 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 eb3fa74ec87..2b478575fdf 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -111,7 +111,7 @@ It makes sense to disable it if the server has millions of tiny tables that are ## function_range_max_elements_in_block {#settings-function_range_max_elements_in_block} -Sets the safety threshold for data volume generated by function [range](../../sql-reference/functions/array-functions.md#range). It defines the maximum number of values generated by function per block of data (sum of array sizes for every row in a block). +Sets the safety threshold for data volume generated by function [range](../../sql-reference/functions/array-functions.md#range). Defines the maximum number of values generated by function per block of data (sum of array sizes for every row in a block). Possible values: From 44dccb2d3f6ddd9c80cefe75a17446617171a6e8 Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Mon, 2 Aug 2021 19:47:15 +0300 Subject: [PATCH 093/599] Add setting descr. and update range function. --- docs/en/operations/settings/settings.md | 4 ++-- docs/ru/operations/settings/settings.md | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 2b478575fdf..ad71ca3bd96 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -121,8 +121,8 @@ Default value: `500000000`. **See Also** -- [max_block_size](#setting-max_block_size). -- [min_insert_block_size_rows](#min-insert-block-size-rows). +- [max_block_size](#setting-max_block_size) +- [min_insert_block_size_rows](#min-insert-block-size-rows) ## enable_http_compression {#settings-enable_http_compression} diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index 3de6654b116..4e60ae88270 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -97,8 +97,8 @@ ClickHouse применяет настройку в тех случаях, ко **См. также** -- [max_block_size](#setting-max_block_size). -- [min_insert_block_size_rows](#min-insert-block-size-rows). +- [max_block_size](#setting-max_block_size) +- [min_insert_block_size_rows](#min-insert-block-size-rows) ## enable_http_compression {#settings-enable_http_compression} From a7d723eb2908f0b2c89d6db5bb04e29b81be65a8 Mon Sep 17 00:00:00 2001 From: olgarev <56617294+olgarev@users.noreply.github.com> Date: Mon, 2 Aug 2021 19:53:59 +0300 Subject: [PATCH 094/599] Apply suggestions from code review Co-authored-by: Anna <42538400+adevyatova@users.noreply.github.com> --- docs/ru/engines/table-engines/integrations/hdfs.md | 3 +-- docs/ru/engines/table-engines/integrations/s3.md | 2 +- 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/docs/ru/engines/table-engines/integrations/hdfs.md b/docs/ru/engines/table-engines/integrations/hdfs.md index 3381f15bddc..902b0beaeb0 100644 --- a/docs/ru/engines/table-engines/integrations/hdfs.md +++ b/docs/ru/engines/table-engines/integrations/hdfs.md @@ -47,7 +47,7 @@ SELECT * FROM hdfs_engine_table LIMIT 2 ## Детали реализации {#implementation-details} - Поддерживается многопоточное чтение и запись. -- Поддерживается репиликация без копирования данных (zero-copy): если данные хранятся на нескольких удаленных машинах, то при синхронизации пересылаются только метаданные (пути к кускам данных), а сами данные не копируются. +- Поддерживается репликация без копирования данных ("zero-copy"): если данные хранятся на нескольких удаленных машинах, то при синхронизации пересылаются только метаданные (пути к кускам данных), а сами данные не копируются. - Не поддерживается: - использование операций `ALTER` и `SELECT...SAMPLE`; - индексы; @@ -203,4 +203,3 @@ CREATE TABLE big_table (name String, value UInt32) ENGINE = HDFS('hdfs://hdfs1:9 **См. также** - [Виртуальные колонки](../../../engines/table-engines/index.md#table_engines-virtual_columns) - diff --git a/docs/ru/engines/table-engines/integrations/s3.md b/docs/ru/engines/table-engines/integrations/s3.md index 39661ccf1bc..9adcfd81aa9 100644 --- a/docs/ru/engines/table-engines/integrations/s3.md +++ b/docs/ru/engines/table-engines/integrations/s3.md @@ -47,7 +47,7 @@ SELECT * FROM s3_engine_table LIMIT 2; ## Детали реализации {#implementation-details} - Чтение и запись могут быть параллельными. -- Поддерживается репиликация без копирования данных (zero-copy): если данные хранятся на нескольких удаленных машинах, то при синхронизации пересылаются только метаданные (пути к кускам данных), а сами данные не копируются. +- Поддерживается репликация без копирования данных ("zero-copy"): если данные хранятся на нескольких удаленных машинах, то при синхронизации пересылаются только метаданные (пути к кускам данных), а сами данные не копируются. - Не поддерживаются: - запросы `ALTER` и `SELECT...SAMPLE`, - индексы, From b1320ead65507e4d8f46fef54a7bfdd1b7bfa5a7 Mon Sep 17 00:00:00 2001 From: olgarev Date: Mon, 2 Aug 2021 17:19:21 +0000 Subject: [PATCH 095/599] Replication is supported now --- docs/en/engines/table-engines/integrations/hdfs.md | 1 - docs/en/engines/table-engines/integrations/s3.md | 1 - docs/ru/engines/table-engines/integrations/hdfs.md | 3 +-- docs/ru/engines/table-engines/integrations/s3.md | 5 ++--- 4 files changed, 3 insertions(+), 7 deletions(-) diff --git a/docs/en/engines/table-engines/integrations/hdfs.md b/docs/en/engines/table-engines/integrations/hdfs.md index a9b9d34a179..87284add50b 100644 --- a/docs/en/engines/table-engines/integrations/hdfs.md +++ b/docs/en/engines/table-engines/integrations/hdfs.md @@ -55,7 +55,6 @@ SELECT * FROM hdfs_engine_table LIMIT 2 - Not supported: - `ALTER` and `SELECT...SAMPLE` operations. - Indexes. - - Replication. **Globs in path** diff --git a/docs/en/engines/table-engines/integrations/s3.md b/docs/en/engines/table-engines/integrations/s3.md index 998e65bfb58..6516265d254 100644 --- a/docs/en/engines/table-engines/integrations/s3.md +++ b/docs/en/engines/table-engines/integrations/s3.md @@ -61,7 +61,6 @@ For more information about virtual columns see [here](../../../engines/table-eng - Not supported: - `ALTER` and `SELECT...SAMPLE` operations. - Indexes. - - Replication. ## Wildcards In Path {#wildcards-in-path} diff --git a/docs/ru/engines/table-engines/integrations/hdfs.md b/docs/ru/engines/table-engines/integrations/hdfs.md index 902b0beaeb0..e6eaec1bf9c 100644 --- a/docs/ru/engines/table-engines/integrations/hdfs.md +++ b/docs/ru/engines/table-engines/integrations/hdfs.md @@ -50,8 +50,7 @@ SELECT * FROM hdfs_engine_table LIMIT 2 - Поддерживается репликация без копирования данных ("zero-copy"): если данные хранятся на нескольких удаленных машинах, то при синхронизации пересылаются только метаданные (пути к кускам данных), а сами данные не копируются. - Не поддерживается: - использование операций `ALTER` и `SELECT...SAMPLE`; - - индексы; - - репликация. + - индексы. **Шаблоны в пути** diff --git a/docs/ru/engines/table-engines/integrations/s3.md b/docs/ru/engines/table-engines/integrations/s3.md index 9adcfd81aa9..1b20f1f947d 100644 --- a/docs/ru/engines/table-engines/integrations/s3.md +++ b/docs/ru/engines/table-engines/integrations/s3.md @@ -47,11 +47,10 @@ SELECT * FROM s3_engine_table LIMIT 2; ## Детали реализации {#implementation-details} - Чтение и запись могут быть параллельными. -- Поддерживается репликация без копирования данных ("zero-copy"): если данные хранятся на нескольких удаленных машинах, то при синхронизации пересылаются только метаданные (пути к кускам данных), а сами данные не копируются. +- Поддерживается репиликация без копирования данных (zero-copy): если данные хранятся на нескольких удаленных машинах, то при синхронизации пересылаются только метаданные (пути к кускам данных), а сами данные не копируются. - Не поддерживаются: - запросы `ALTER` и `SELECT...SAMPLE`, - - индексы, - - репликация. + - индексы. ## Символы подстановки {#wildcards-in-path} From bb6d030fb8beff3dfc29d04ed998a6fb23631e8c Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 17 Jul 2021 08:04:55 +0300 Subject: [PATCH 096/599] Optimize distributed SELECT w/o GROUP BY --- src/Storages/StorageDistributed.cpp | 12 ++---------- 1 file changed, 2 insertions(+), 10 deletions(-) diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 5173abeb9f9..b6a04c5cd34 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -338,16 +338,8 @@ std::optional getOptimizedQueryProcessingStage(const // GROUP BY const ASTPtr group_by = select.groupBy(); - if (!group_by) - { - if (!select.distinct) - return {}; - } - else - { - if (!sharding_block_has(group_by->children)) - return {}; - } + if (!query_info.syntax_analyzer_result->aggregates.empty() && (!group_by || !sharding_block_has(group_by->children))) + return {}; // ORDER BY const ASTPtr order_by = select.orderBy(); From 2fb95d9ee0711e2327f3be884fa130e82b8c04b3 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 17 Jul 2021 16:31:06 +0300 Subject: [PATCH 097/599] Rework SELECT from Distributed query stages optimization Before this patch it wasn't possible to optimize simple SELECT * FROM dist ORDER BY (w/o GROUP BY and DISTINCT) to more optimal stage (QueryProcessingStage::WithMergeableStateAfterAggregationAndLimit), since that code was under allow_nondeterministic_optimize_skip_unused_shards, rework it and make it possible. Also now distributed_push_down_limit is respected for optimize_distributed_group_by_sharding_key. Next step will be to enable distributed_push_down_limit by default. v2: fix detection of aggregates --- src/Storages/StorageDistributed.cpp | 166 +++++++++--------- src/Storages/StorageDistributed.h | 18 ++ ...istributed_group_by_sharding_key.reference | 4 +- ...mize_distributed_group_by_sharding_key.sql | 6 +- ...1814_distributed_push_down_limit.reference | 12 -- .../01814_distributed_push_down_limit.sh | 29 +-- 6 files changed, 109 insertions(+), 126 deletions(-) diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index b6a04c5cd34..85a2efb9963 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -284,78 +284,6 @@ void replaceConstantExpressions( visitor.visit(node); } -/// This is the implementation of optimize_distributed_group_by_sharding_key. -/// It returns up to which stage the query can be processed on a shard, which -/// is one of the following: -/// - QueryProcessingStage::Complete -/// - QueryProcessingStage::WithMergeableStateAfterAggregation -/// - QueryProcessingStage::WithMergeableStateAfterAggregationAndLimit -/// - none (in this case regular WithMergeableState should be used) -std::optional getOptimizedQueryProcessingStage(const SelectQueryInfo & query_info, bool extremes, const Names & sharding_key_columns) -{ - const auto & select = query_info.query->as(); - - auto sharding_block_has = [&](const auto & exprs) -> bool - { - std::unordered_set expr_columns; - for (auto & expr : exprs) - { - auto id = expr->template as(); - if (!id) - continue; - expr_columns.emplace(id->name()); - } - - for (const auto & column : sharding_key_columns) - { - if (!expr_columns.contains(column)) - return false; - } - - return true; - }; - - // GROUP BY qualifiers - // - TODO: WITH TOTALS can be implemented - // - TODO: WITH ROLLUP can be implemented (I guess) - if (select.group_by_with_totals || select.group_by_with_rollup || select.group_by_with_cube) - return {}; - - // Window functions are not supported. - if (query_info.has_window) - return {}; - - // TODO: extremes support can be implemented - if (extremes) - return {}; - - // DISTINCT - if (select.distinct) - { - if (!sharding_block_has(select.select()->children)) - return {}; - } - - // GROUP BY - const ASTPtr group_by = select.groupBy(); - if (!query_info.syntax_analyzer_result->aggregates.empty() && (!group_by || !sharding_block_has(group_by->children))) - return {}; - - // ORDER BY - const ASTPtr order_by = select.orderBy(); - if (order_by) - return QueryProcessingStage::WithMergeableStateAfterAggregationAndLimit; - - // LIMIT BY - // LIMIT - // OFFSET - if (select.limitBy() || select.limitLength() || select.limitOffset()) - return QueryProcessingStage::WithMergeableStateAfterAggregationAndLimit; - - // Only simple SELECT FROM GROUP BY sharding_key can use Complete state. - return QueryProcessingStage::Complete; -} - size_t getClusterQueriedNodes(const Settings & settings, const ClusterPtr & cluster) { size_t num_local_shards = cluster->getLocalShardCount(); @@ -523,9 +451,6 @@ QueryProcessingStage::Enum StorageDistributed::getQueryProcessingStage( } } - if (settings.distributed_push_down_limit) - return QueryProcessingStage::WithMergeableStateAfterAggregationAndLimit; - /// Nested distributed query cannot return Complete stage, /// since the parent query need to aggregate the results after. if (to_stage == QueryProcessingStage::WithMergeableState) @@ -536,22 +461,89 @@ QueryProcessingStage::Enum StorageDistributed::getQueryProcessingStage( if (getClusterQueriedNodes(settings, cluster) == 1) return QueryProcessingStage::Complete; - if (settings.optimize_skip_unused_shards && - settings.optimize_distributed_group_by_sharding_key && - has_sharding_key && - (settings.allow_nondeterministic_optimize_skip_unused_shards || sharding_key_is_deterministic)) - { - auto stage = getOptimizedQueryProcessingStage(query_info, settings.extremes, sharding_key_expr->getRequiredColumns()); - if (stage) - { - LOG_DEBUG(log, "Force processing stage to {}", QueryProcessingStage::toString(*stage)); - return *stage; - } - } + auto optimized_stage = getOptimizedQueryProcessingStage(query_info, settings); + if (optimized_stage) + return *optimized_stage; return QueryProcessingStage::WithMergeableState; } +std::optional StorageDistributed::getOptimizedQueryProcessingStage(const SelectQueryInfo & query_info, const Settings & settings) const +{ + bool optimize_sharding_key_aggregation = + settings.optimize_skip_unused_shards && + settings.optimize_distributed_group_by_sharding_key && + has_sharding_key && + (settings.allow_nondeterministic_optimize_skip_unused_shards || sharding_key_is_deterministic); + + QueryProcessingStage::Enum default_stage = QueryProcessingStage::WithMergeableStateAfterAggregation; + if (settings.distributed_push_down_limit) + default_stage = QueryProcessingStage::WithMergeableStateAfterAggregationAndLimit; + + const auto & select = query_info.query->as(); + + auto expr_contains_sharding_key = [&](const auto & exprs) -> bool + { + std::unordered_set expr_columns; + for (auto & expr : exprs) + { + auto id = expr->template as(); + if (!id) + continue; + expr_columns.emplace(id->name()); + } + + for (const auto & column : sharding_key_expr->getRequiredColumns()) + { + if (!expr_columns.contains(column)) + return false; + } + + return true; + }; + + // GROUP BY qualifiers + // - TODO: WITH TOTALS can be implemented + // - TODO: WITH ROLLUP can be implemented (I guess) + if (select.group_by_with_totals || select.group_by_with_rollup || select.group_by_with_cube) + return {}; + // Window functions are not supported. + if (query_info.has_window) + return {}; + // TODO: extremes support can be implemented + if (settings.extremes) + return {}; + + // DISTINCT + if (select.distinct) + { + if (!optimize_sharding_key_aggregation || !expr_contains_sharding_key(select.select()->children)) + return {}; + } + + // GROUP BY + const ASTPtr group_by = select.groupBy(); + if (!query_info.syntax_analyzer_result->aggregates.empty() || group_by) + { + if (!optimize_sharding_key_aggregation || !group_by || !expr_contains_sharding_key(group_by->children)) + return {}; + } + + // ORDER BY + const ASTPtr order_by = select.orderBy(); + if (order_by) + return default_stage; + + // LIMIT BY + // LIMIT + // OFFSET + if (select.limitBy() || select.limitLength() || select.limitOffset()) + return default_stage; + + // Only simple SELECT FROM GROUP BY sharding_key can use Complete state. + return QueryProcessingStage::Complete; +} + Pipe StorageDistributed::read( const Names & column_names, const StorageMetadataPtr & metadata_snapshot, diff --git a/src/Storages/StorageDistributed.h b/src/Storages/StorageDistributed.h index bf48e814ae2..e09eda00224 100644 --- a/src/Storages/StorageDistributed.h +++ b/src/Storages/StorageDistributed.h @@ -176,6 +176,24 @@ private: ClusterPtr skipUnusedShards(ClusterPtr cluster, const ASTPtr & query_ptr, const StorageMetadataPtr & metadata_snapshot, ContextPtr context) const; + /// This method returns optimal query processing stage. + /// + /// Here is the list of stages (from the less optimal to more optimal): + /// - WithMergeableState + /// - WithMergeableStateAfterAggregation + /// - WithMergeableStateAfterAggregationAndLimit + /// - Complete + /// + /// Some simple queries w/o GROUP BY/DISTINCT can use more optimal stage. + /// + /// Also in case of optimize_distributed_group_by_sharding_key=1 the queries + /// with GROUP BY/DISTINCT sharding_key can also use more optimal stage. + /// (see also optimize_skip_unused_shards/allow_nondeterministic_optimize_skip_unused_shards) + /// + /// @return QueryProcessingStage or empty std::optoinal + /// (in this case regular WithMergeableState should be used) + std::optional getOptimizedQueryProcessingStage(const SelectQueryInfo & query_info, const Settings & settings) const; + size_t getRandomShardIndex(const Cluster::ShardsInfo & shards); const DistributedSettings & getDistributedSettingsRef() const { return distributed_settings; } diff --git a/tests/queries/0_stateless/01244_optimize_distributed_group_by_sharding_key.reference b/tests/queries/0_stateless/01244_optimize_distributed_group_by_sharding_key.reference index 4442b0b6b61..8d356a6966f 100644 --- a/tests/queries/0_stateless/01244_optimize_distributed_group_by_sharding_key.reference +++ b/tests/queries/0_stateless/01244_optimize_distributed_group_by_sharding_key.reference @@ -57,7 +57,9 @@ LIMIT 1 0 LIMIT OFFSET 1 1 -OFFSET +OFFSET distributed_push_down_limit=0 +1 1 +OFFSET distributed_push_down_limit=1 1 1 1 0 1 1 diff --git a/tests/queries/0_stateless/01244_optimize_distributed_group_by_sharding_key.sql b/tests/queries/0_stateless/01244_optimize_distributed_group_by_sharding_key.sql index 4719119165a..10b47f64cc6 100644 --- a/tests/queries/0_stateless/01244_optimize_distributed_group_by_sharding_key.sql +++ b/tests/queries/0_stateless/01244_optimize_distributed_group_by_sharding_key.sql @@ -60,8 +60,10 @@ select 'LIMIT'; select count(), * from dist_01247 group by number limit 1; select 'LIMIT OFFSET'; select count(), * from dist_01247 group by number limit 1 offset 1; -select 'OFFSET'; -select count(), * from dist_01247 group by number offset 1; +select 'OFFSET distributed_push_down_limit=0'; +select count(), * from dist_01247 group by number offset 1 settings distributed_push_down_limit=0; +select 'OFFSET distributed_push_down_limit=1'; +select count(), * from dist_01247 group by number offset 1 settings distributed_push_down_limit=1; -- this will emulate different data on for different shards select 'WHERE LIMIT OFFSET'; select count(), * from dist_01247 where number = _shard_num-1 group by number order by number limit 1 offset 1; diff --git a/tests/queries/0_stateless/01814_distributed_push_down_limit.reference b/tests/queries/0_stateless/01814_distributed_push_down_limit.reference index f879f2cbd21..c542b5b7325 100644 --- a/tests/queries/0_stateless/01814_distributed_push_down_limit.reference +++ b/tests/queries/0_stateless/01814_distributed_push_down_limit.reference @@ -12,18 +12,6 @@ distributed_push_down_limit=1 8 9 40 40 -auto-distributed_push_down_limit -0 -1 -2 -3 -4 -5 -6 -7 -8 -9 -40 40 distributed_push_down_limit=1 with OFFSET 97 96 diff --git a/tests/queries/0_stateless/01814_distributed_push_down_limit.sh b/tests/queries/0_stateless/01814_distributed_push_down_limit.sh index 93321646037..24b27e74ba5 100755 --- a/tests/queries/0_stateless/01814_distributed_push_down_limit.sh +++ b/tests/queries/0_stateless/01814_distributed_push_down_limit.sh @@ -86,9 +86,11 @@ function test_distributed_push_down_limit_0() function test_distributed_push_down_limit_1() { local args=( - "remote('127.{2,3}', $CLICKHOUSE_DATABASE, data_01814)" + "remote('127.{2,3}', $CLICKHOUSE_DATABASE, data_01814, key)" 0 # offset --distributed_push_down_limit 1 + --optimize_skip_unused_shards 1 + --optimize_distributed_group_by_sharding_key 1 ) test_distributed_push_down_limit_with_query_log "${args[@]}" } @@ -97,22 +99,11 @@ function test_distributed_push_down_limit_1_offset() { local settings_and_opts=( --distributed_push_down_limit 1 - ) - - $CLICKHOUSE_CLIENT "${settings_and_opts[@]}" -q "select * from remote('127.{2,3}', $CLICKHOUSE_DATABASE, data_01814) group by key order by key desc limit 5, 10" -} - -function test_auto_distributed_push_down_limit() -{ - local args=( - dist_01814 - 0 # offset --optimize_skip_unused_shards 1 --optimize_distributed_group_by_sharding_key 1 - --prefer_localhost_replica 0 - --distributed_push_down_limit 0 ) - test_distributed_push_down_limit_with_query_log "${args[@]}" + + $CLICKHOUSE_CLIENT "${settings_and_opts[@]}" -q "select * from remote('127.{2,3}', $CLICKHOUSE_DATABASE, data_01814, key) group by key order by key desc limit 5, 10" } function main() @@ -151,16 +142,6 @@ function main() done echo "$out" - echo 'auto-distributed_push_down_limit' - for ((i = 0; i < max_tries; ++i)); do - out=$(test_auto_distributed_push_down_limit) - out_lines=( $out ) - if [[ ${#out_lines[@]} -gt 2 ]] && [[ ${out_lines[-1]} = 40 ]] && [[ ${out_lines[-2]} = 40 ]]; then - break - fi - done - echo "$out" - echo 'distributed_push_down_limit=1 with OFFSET' test_distributed_push_down_limit_1_offset } From c83096424029eadce104aaeb948774aa4cf5ed3e Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 17 Jul 2021 17:16:29 +0300 Subject: [PATCH 098/599] Cover distributed_push_down_limit for queries w/o sharding expr --- ...1951_distributed_push_down_limit.reference | 32 +++++++++++++++++++ .../01951_distributed_push_down_limit.sql | 3 ++ 2 files changed, 35 insertions(+) create mode 100644 tests/queries/0_stateless/01951_distributed_push_down_limit.reference create mode 100644 tests/queries/0_stateless/01951_distributed_push_down_limit.sql diff --git a/tests/queries/0_stateless/01951_distributed_push_down_limit.reference b/tests/queries/0_stateless/01951_distributed_push_down_limit.reference new file mode 100644 index 00000000000..9e803a171c4 --- /dev/null +++ b/tests/queries/0_stateless/01951_distributed_push_down_limit.reference @@ -0,0 +1,32 @@ +-- { echo } +explain select * from remote('127.{1,2}', view(select * from numbers(1e6))) order by number limit 10 settings distributed_push_down_limit=0; +Expression (Projection) + Limit (preliminary LIMIT) + MergingSorted (Merge sorted streams after aggregation stage for ORDER BY) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + Union + MergingSorted (Merge sorted streams for ORDER BY) + MergeSorting (Merge sorted blocks for ORDER BY) + PartialSorting (Sort each block for ORDER BY) + Expression (Before ORDER BY) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + Expression ((Convert VIEW subquery result to VIEW table structure + (Materialize constants after VIEW subquery + (Projection + Before ORDER BY)))) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + ReadFromStorage (SystemNumbers) + ReadFromRemote (Read from remote replica) +explain select * from remote('127.{1,2}', view(select * from numbers(1e6))) order by number limit 10 settings distributed_push_down_limit=1; +Expression (Projection) + Limit (preliminary LIMIT) + MergingSorted (Merge sorted streams after aggregation stage for ORDER BY) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + Union + Limit (preliminary LIMIT) + MergingSorted (Merge sorted streams for ORDER BY) + MergeSorting (Merge sorted blocks for ORDER BY) + PartialSorting (Sort each block for ORDER BY) + Expression (Before ORDER BY) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + Expression ((Convert VIEW subquery result to VIEW table structure + (Materialize constants after VIEW subquery + (Projection + Before ORDER BY)))) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + ReadFromStorage (SystemNumbers) + ReadFromRemote (Read from remote replica) diff --git a/tests/queries/0_stateless/01951_distributed_push_down_limit.sql b/tests/queries/0_stateless/01951_distributed_push_down_limit.sql new file mode 100644 index 00000000000..0d6e2069215 --- /dev/null +++ b/tests/queries/0_stateless/01951_distributed_push_down_limit.sql @@ -0,0 +1,3 @@ +-- { echo } +explain select * from remote('127.{1,2}', view(select * from numbers(1e6))) order by number limit 10 settings distributed_push_down_limit=0; +explain select * from remote('127.{1,2}', view(select * from numbers(1e6))) order by number limit 10 settings distributed_push_down_limit=1; From ff12f5102a0f549f0716954657585ab4cdf1d904 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 17 Jul 2021 20:21:13 +0300 Subject: [PATCH 099/599] Avoid running LIMIT BY/DISTINCT step on the initiator for optimize_distributed_group_by_sharding_key Before the following queries was running LimitBy/Distinct step on the initator: select distinct sharding_key from dist order by k While this can be omitted. --- src/Interpreters/InterpreterSelectQuery.cpp | 4 +- src/Storages/StorageDistributed.cpp | 12 +- ...rd_distributed_group_by_no_merge.reference | 2 + ...istributed_group_by_sharding_key.reference | 2 + ...istributed_group_by_sharding_key.reference | 115 ++++++++++++++++++ ...mize_distributed_group_by_sharding_key.sql | 13 ++ 6 files changed, 143 insertions(+), 5 deletions(-) create mode 100644 tests/queries/0_stateless/01952_optimize_distributed_group_by_sharding_key.reference create mode 100644 tests/queries/0_stateless/01952_optimize_distributed_group_by_sharding_key.sql diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index c126f3bca88..a5b6ff061bc 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -1341,10 +1341,10 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, const BlockInpu /** If there was more than one stream, * then DISTINCT needs to be performed once again after merging all streams. */ - if (query.distinct) + if (!from_aggregation_stage && query.distinct) executeDistinct(query_plan, false, expressions.selected_columns, false); - if (expressions.hasLimitBy()) + if (!from_aggregation_stage && expressions.hasLimitBy()) { executeExpression(query_plan, expressions.before_limit_by, "Before LIMIT BY"); executeLimitBy(query_plan); diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 85a2efb9963..2bbb92cf0b8 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -529,15 +529,21 @@ std::optional StorageDistributed::getOptimizedQueryP return {}; } + // LIMIT BY + if (const ASTPtr limit_by = select.limitBy()) + { + if (!optimize_sharding_key_aggregation || !expr_contains_sharding_key(limit_by->children)) + return {}; + } + // ORDER BY - const ASTPtr order_by = select.orderBy(); - if (order_by) + if (const ASTPtr order_by = select.orderBy()) return default_stage; // LIMIT BY // LIMIT // OFFSET - if (select.limitBy() || select.limitLength() || select.limitOffset()) + if (select.limitLength() || select.limitOffset()) return default_stage; // Only simple SELECT FROM GROUP BY sharding_key can use Complete state. diff --git a/tests/queries/0_stateless/00184_shard_distributed_group_by_no_merge.reference b/tests/queries/0_stateless/00184_shard_distributed_group_by_no_merge.reference index b667c57a14c..b2b0b43e490 100644 --- a/tests/queries/0_stateless/00184_shard_distributed_group_by_no_merge.reference +++ b/tests/queries/0_stateless/00184_shard_distributed_group_by_no_merge.reference @@ -25,6 +25,8 @@ ORDER BY LIMIT LIMIT BY 0 1 +0 +1 LIMIT BY LIMIT 0 GROUP BY ORDER BY diff --git a/tests/queries/0_stateless/01244_optimize_distributed_group_by_sharding_key.reference b/tests/queries/0_stateless/01244_optimize_distributed_group_by_sharding_key.reference index 8d356a6966f..a4a6b87de25 100644 --- a/tests/queries/0_stateless/01244_optimize_distributed_group_by_sharding_key.reference +++ b/tests/queries/0_stateless/01244_optimize_distributed_group_by_sharding_key.reference @@ -67,6 +67,8 @@ WHERE LIMIT OFFSET 1 1 LIMIT BY 1 1 0 +1 0 +1 1 1 1 GROUP BY (Distributed-over-Distributed) 4 0 diff --git a/tests/queries/0_stateless/01952_optimize_distributed_group_by_sharding_key.reference b/tests/queries/0_stateless/01952_optimize_distributed_group_by_sharding_key.reference new file mode 100644 index 00000000000..10787068f43 --- /dev/null +++ b/tests/queries/0_stateless/01952_optimize_distributed_group_by_sharding_key.reference @@ -0,0 +1,115 @@ +-- { echo } +explain select distinct k1 from remote('127.{1,2}', view(select 1 k1, 2 k2, 3 v from numbers(2)), cityHash64(k1, k2)); -- not optimized +Expression (Projection) + Distinct + SettingQuotaAndLimits (Set limits and quota after reading from storage) + Union + Distinct (Preliminary DISTINCT) + Expression (Before ORDER BY) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + Expression ((Convert VIEW subquery result to VIEW table structure + (Materialize constants after VIEW subquery + (Projection + Before ORDER BY)))) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + ReadFromStorage (SystemNumbers) + ReadFromRemote (Read from remote replica) +explain select distinct k1, k2 from remote('127.{1,2}', view(select 1 k1, 2 k2, 3 v from numbers(2)), cityHash64(k1, k2)); -- optimized +SettingQuotaAndLimits (Set limits and quota after reading from storage) + Union + Expression (Projection) + Distinct + Distinct (Preliminary DISTINCT) + Expression (Before ORDER BY) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + Expression ((Convert VIEW subquery result to VIEW table structure + (Materialize constants after VIEW subquery + (Projection + Before ORDER BY)))) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + ReadFromStorage (SystemNumbers) + ReadFromRemote (Read from remote replica) +explain select distinct on (k1) k2 from remote('127.{1,2}', view(select 1 k1, 2 k2, 3 v from numbers(2)), cityHash64(k1, k2)); -- not optimized +Expression (Projection) + LimitBy + Expression (Before LIMIT BY) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + Union + LimitBy + Expression ((Before LIMIT BY + Before ORDER BY)) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + Expression ((Convert VIEW subquery result to VIEW table structure + (Materialize constants after VIEW subquery + (Projection + Before ORDER BY)))) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + ReadFromStorage (SystemNumbers) + ReadFromRemote (Read from remote replica) +explain select distinct on (k1, k2) v from remote('127.{1,2}', view(select 1 k1, 2 k2, 3 v from numbers(2)), cityHash64(k1, k2)); -- optimized +SettingQuotaAndLimits (Set limits and quota after reading from storage) + Union + Expression (Projection) + LimitBy + Expression ((Before LIMIT BY + Before ORDER BY)) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + Expression ((Convert VIEW subquery result to VIEW table structure + (Materialize constants after VIEW subquery + (Projection + Before ORDER BY)))) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + ReadFromStorage (SystemNumbers) + ReadFromRemote (Read from remote replica) +explain select distinct k1 from remote('127.{1,2}', view(select 1 k1, 2 k2, 3 v from numbers(2)), cityHash64(k1, k2)) order by v; -- not optimized +Expression (Projection) + Distinct + MergingSorted (Merge sorted streams for ORDER BY, without aggregation) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + Union + MergingSorted (Merge sorted streams for ORDER BY) + MergeSorting (Merge sorted blocks for ORDER BY) + PartialSorting (Sort each block for ORDER BY) + Distinct (Preliminary DISTINCT) + Expression (Before ORDER BY) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + Expression ((Convert VIEW subquery result to VIEW table structure + (Materialize constants after VIEW subquery + (Projection + Before ORDER BY)))) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + ReadFromStorage (SystemNumbers) + ReadFromRemote (Read from remote replica) +explain select distinct k1, k2 from remote('127.{1,2}', view(select 1 k1, 2 k2, 3 v from numbers(2)), cityHash64(k1, k2)) order by v; -- optimized +Expression (Projection) + MergingSorted (Merge sorted streams after aggregation stage for ORDER BY) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + Union + Distinct + MergingSorted (Merge sorted streams for ORDER BY) + MergeSorting (Merge sorted blocks for ORDER BY) + PartialSorting (Sort each block for ORDER BY) + Distinct (Preliminary DISTINCT) + Expression (Before ORDER BY) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + Expression ((Convert VIEW subquery result to VIEW table structure + (Materialize constants after VIEW subquery + (Projection + Before ORDER BY)))) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + ReadFromStorage (SystemNumbers) + ReadFromRemote (Read from remote replica) +explain select distinct on (k1) k2 from remote('127.{1,2}', view(select 1 k1, 2 k2, 3 v from numbers(2)), cityHash64(k1, k2)) order by v; -- not optimized +Expression (Projection) + LimitBy + Expression (Before LIMIT BY) + MergingSorted (Merge sorted streams for ORDER BY, without aggregation) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + Union + LimitBy + Expression (Before LIMIT BY) + MergingSorted (Merge sorted streams for ORDER BY) + MergeSorting (Merge sorted blocks for ORDER BY) + PartialSorting (Sort each block for ORDER BY) + Expression (Before ORDER BY) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + Expression ((Convert VIEW subquery result to VIEW table structure + (Materialize constants after VIEW subquery + (Projection + Before ORDER BY)))) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + ReadFromStorage (SystemNumbers) + ReadFromRemote (Read from remote replica) +explain select distinct on (k1, k2) v from remote('127.{1,2}', view(select 1 k1, 2 k2, 3 v from numbers(2)), cityHash64(k1, k2)) order by v; -- optimized +Expression (Projection) + MergingSorted (Merge sorted streams after aggregation stage for ORDER BY) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + Union + LimitBy + Expression (Before LIMIT BY) + MergingSorted (Merge sorted streams for ORDER BY) + MergeSorting (Merge sorted blocks for ORDER BY) + PartialSorting (Sort each block for ORDER BY) + Expression (Before ORDER BY) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + Expression ((Convert VIEW subquery result to VIEW table structure + (Materialize constants after VIEW subquery + (Projection + Before ORDER BY)))) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + ReadFromStorage (SystemNumbers) + ReadFromRemote (Read from remote replica) diff --git a/tests/queries/0_stateless/01952_optimize_distributed_group_by_sharding_key.sql b/tests/queries/0_stateless/01952_optimize_distributed_group_by_sharding_key.sql new file mode 100644 index 00000000000..2ae872f72b0 --- /dev/null +++ b/tests/queries/0_stateless/01952_optimize_distributed_group_by_sharding_key.sql @@ -0,0 +1,13 @@ +set optimize_skip_unused_shards=1; +set optimize_distributed_group_by_sharding_key=1; + +-- { echo } +explain select distinct k1 from remote('127.{1,2}', view(select 1 k1, 2 k2, 3 v from numbers(2)), cityHash64(k1, k2)); -- not optimized +explain select distinct k1, k2 from remote('127.{1,2}', view(select 1 k1, 2 k2, 3 v from numbers(2)), cityHash64(k1, k2)); -- optimized +explain select distinct on (k1) k2 from remote('127.{1,2}', view(select 1 k1, 2 k2, 3 v from numbers(2)), cityHash64(k1, k2)); -- not optimized +explain select distinct on (k1, k2) v from remote('127.{1,2}', view(select 1 k1, 2 k2, 3 v from numbers(2)), cityHash64(k1, k2)); -- optimized + +explain select distinct k1 from remote('127.{1,2}', view(select 1 k1, 2 k2, 3 v from numbers(2)), cityHash64(k1, k2)) order by v; -- not optimized +explain select distinct k1, k2 from remote('127.{1,2}', view(select 1 k1, 2 k2, 3 v from numbers(2)), cityHash64(k1, k2)) order by v; -- optimized +explain select distinct on (k1) k2 from remote('127.{1,2}', view(select 1 k1, 2 k2, 3 v from numbers(2)), cityHash64(k1, k2)) order by v; -- not optimized +explain select distinct on (k1, k2) v from remote('127.{1,2}', view(select 1 k1, 2 k2, 3 v from numbers(2)), cityHash64(k1, k2)) order by v; -- optimized From 387136dc846b0368d3b505d65a7724f9025bfffd Mon Sep 17 00:00:00 2001 From: Dmitriy Date: Mon, 2 Aug 2021 22:45:01 +0300 Subject: [PATCH 100/599] Add settings MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Добавил настройки, перевел табличный движок MaterializedPostgreSQL. --- .../materialized-postgresql.md | 36 ++++++++++----- .../integrations/materialized-postgresql.md | 30 ++++++------- docs/en/operations/settings/settings.md | 27 ++++++++++++ .../materialized-postgresql.md | 34 +++++++++----- .../integrations/materialized-postgresql.md | 44 +++++++++++++++++++ docs/ru/operations/settings/settings.md | 27 ++++++++++++ 6 files changed, 161 insertions(+), 37 deletions(-) create mode 100644 docs/ru/engines/table-engines/integrations/materialized-postgresql.md diff --git a/docs/en/engines/database-engines/materialized-postgresql.md b/docs/en/engines/database-engines/materialized-postgresql.md index d1440746b2f..5c1dc105d72 100644 --- a/docs/en/engines/database-engines/materialized-postgresql.md +++ b/docs/en/engines/database-engines/materialized-postgresql.md @@ -14,32 +14,37 @@ This feature is experimental. ## Creating a Database {#creating-a-database} ``` sql -CREATE DATABASE test_database -ENGINE = MaterializedPostgreSQL('postgres1:5432', 'postgres_database', 'postgres_user', 'postgres_password'); - -SELECT * FROM test_database.postgres_table; +CREATE DATABASE [IF NOT EXISTS] db_name [ON CLUSTER cluster] +ENGINE = MaterializedPostgreSQL('host:port', ['database' | database], 'user', 'password') [SETTINGS ...] ``` +**Engine Parameters** + +- `host:port` — PostgreSQL server endpoint. +- `database` — PostgreSQL database name. +- `user` — PostgreSQL user. +- `password` — User password. + ## Settings {#settings} -1. `materialized_postgresql_max_block_size` — Number of rows collected in memory before flushing data into table. Default: `65536`. +- [materialized_postgresql_max_block_size](../../operations/settings/settings.md#materialized-postgresql-max-block-size) -2. `materialized_postgresql_tables_list` — A comma-separated list of PostgreSQL database tables, which will be replicated via MaterializedPostgreSQL database engine. Default: empty list - means whole PostgreSQL database will be replicated. +- [materialized_postgresql_tables_list](../../operations/settings/settings.md#materialized-postgresql-tables-list) -3. `materialized_postgresql_allow_automatic_update` — Allow to reload table in the background, when schema changes are detected. Default: `0` (`false`). DDL queries on PostgreSQL side are not replicated via ClickHouse `MaterializedPostgreSQL` engine, because it is not allowed with PostgreSQL logical replication protocol, but the fact of DDL changes is detected transactionally. In this case the default behaviour is to stop replicating those tables once DDL is detected. However, if this setting is enabled, then, instead of stopping replication of those tables, they will be reloaded in the background via database snapshot without data losses and replication will continue for them. +- [materialized_postgresql_allow_automatic_update](../../operations/settings/settings.md#materialized-postgresql-allow-automatic-update) ``` sql -CREATE DATABASE test_database +CREATE DATABASE database1 ENGINE = MaterializedPostgreSQL('postgres1:5432', 'postgres_database', 'postgres_user', 'postgres_password') SETTINGS materialized_postgresql_max_block_size = 65536, materialized_postgresql_tables_list = 'table1,table2,table3'; -SELECT * FROM test_database.table1; +SELECT * FROM database1.table1; ``` ## Requirements {#requirements} -1. Setting [wal_level](https://www.postgresql.org/docs/current/runtime-config-wal.html) to `logical` and `max_replication_slots` to at least `2` in the PostgreSQL config file. +1. The [wal_level](https://www.postgresql.org/docs/current/runtime-config-wal.html) setting must have a value `logical` and `max_replication_slots` parameter must have a value at least `2` in the PostgreSQL config file. 2. Each replicated table must have one of the following [replica identity](https://www.postgresql.org/docs/10/sql-altertable.html#SQL-CREATETABLE-REPLICA-IDENTITY): @@ -69,4 +74,13 @@ WHERE oid = 'postgres_table'::regclass; ``` !!! warning "Warning" - Replication of [**TOAST**](https://www.postgresql.org/docs/9.5/storage-toast.html) values is not supported. Default value for the data type will be used. + Replication of [**TOAST**](https://www.postgresql.org/docs/9.5/storage-toast.html) values is not supported. The default value for the data type will be used. + +## Example of Use {#example-of-use} + +``` sql +CREATE DATABASE postgresql_db +ENGINE = MaterializedPostgreSQL('postgres1:5432', 'postgres_database', 'postgres_user', 'postgres_password'); + +SELECT * FROM postgresql_db.postgres_table; +``` diff --git a/docs/en/engines/table-engines/integrations/materialized-postgresql.md b/docs/en/engines/table-engines/integrations/materialized-postgresql.md index 142639507d6..1704563570b 100644 --- a/docs/en/engines/table-engines/integrations/materialized-postgresql.md +++ b/docs/en/engines/table-engines/integrations/materialized-postgresql.md @@ -5,42 +5,40 @@ toc_title: MaterializedPostgreSQL # MaterializedPostgreSQL {#materialize-postgresql} +The `MaterializedPostgreSQL` engine allows you to perform `SELECT` and `INSERT` queries on data that is stored on a remote PostgreSQL server. + ## Creating a Table {#creating-a-table} ``` sql -CREATE TABLE test.postgresql_replica (key UInt64, value UInt64) +CREATE TABLE postgresql_db.postgresql_replica (key UInt64, value UInt64) ENGINE = MaterializedPostgreSQL('postgres1:5432', 'postgres_database', 'postgresql_replica', 'postgres_user', 'postgres_password') PRIMARY KEY key; ``` - ## Requirements {#requirements} -- Setting `wal_level`to `logical` and `max_replication_slots` to at least `2` in the postgresql config file. +1. The [wal_level](https://www.postgresql.org/docs/current/runtime-config-wal.html) setting must have a value `logical` and `max_replication_slots` parameter must have a value at least `2` in the PostgreSQL config file. -- A table with engine `MaterializedPostgreSQL` must have a primary key - the same as a replica identity index (default: primary key) of a postgres table (See [details on replica identity index](../../database-engines/materialized-postgresql.md#requirements)). +2. A table with `MaterializedPostgreSQL` engine must have a primary key — the same as a replica identity index (by default: primary key) of a PostgreSQL table (see [details on replica identity index](../../database-engines/materialized-postgresql.md#requirements)). -- Only database `Atomic` is allowed. +3. Only database [Atomic](https://en.wikipedia.org/wiki/Atomicity_(database_systems)) is allowed. +## Virtual columns {#virtual-columns} -## Virtual columns {#creating-a-table} +- `_version` (type: UInt64) -- `_version` (`UInt64`) +- `_sign` (type: Int8) -- `_sign` (`Int8`) - -These columns do not need to be added, when table is created. They are always accessible in `SELECT` query. +These columns do not need to be added when a table is created. They are always accessible in `SELECT` query. `_version` column equals `LSN` position in `WAL`, so it might be used to check how up-to-date replication is. ``` sql -CREATE TABLE test.postgresql_replica (key UInt64, value UInt64) +CREATE TABLE postgresql_db.postgresql_replica (key UInt64, value UInt64) ENGINE = MaterializedPostgreSQL('postgres1:5432', 'postgres_database', 'postgresql_replica', 'postgres_user', 'postgres_password') PRIMARY KEY key; -SELECT key, value, _version FROM test.postgresql_replica; +SELECT key, value, _version FROM postgresql_db.postgresql_replica; ``` - -## Warning {#warning} - -1. **TOAST** values convertion is not supported. Default value for the data type will be used. +!!! warning "Warning" + Replication of [**TOAST**](https://www.postgresql.org/docs/9.5/storage-toast.html) values is not supported. The default value for the data type will be used. diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 5042aeae162..40e1d72170c 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -3285,3 +3285,30 @@ Possible values: - 1 — The `LowCardinality` type is converted to the `DICTIONARY` type. Default value: `0`. + +## materialized_postgresql_max_block_size {#materialized-postgresql-max-block-size} + +Sets the number of rows collected in memory before flushing data into PostgreSQL database table. + +Possible values: + +- Positive integer. + +Default value: `65536`. + +## materialized_postgresql_tables_list {#materialized-postgresql-tables-list} + +Sets a comma-separated list of PostgreSQL database tables, which will be replicated via [MaterializedPostgreSQL](../../engines/database-engines/materialized-postgresql.md) database engine. + +Default value: empty list — means whole PostgreSQL database will be replicated. + +## materialized_postgresql_allow_automatic_update {#materialized-postgresql-allow-automatic-update} + +Allow reloading table in the background, when schema changes are detected. DDL queries on the PostgreSQL side are not replicated via ClickHouse [MaterializedPostgreSQL](../../engines/database-engines/materialized-postgresql.md) engine, because it is not allowed with PostgreSQL logical replication protocol, but the fact of DDL changes is detected transactionally. In this case, the default behaviour is to stop replicating those tables once DDL is detected. However, if this setting is enabled, then, instead of stopping the replication of those tables, they will be reloaded in the background via database snapshot without data losses and replication will continue for them. + +Possible values: + +- 0 — The table is not automatically updated in the background, when schema changes are detected. +- 1 — The table is automatically updated in the background, when schema changes are detected. + +Default value: `0`. diff --git a/docs/ru/engines/database-engines/materialized-postgresql.md b/docs/ru/engines/database-engines/materialized-postgresql.md index 8c366ba14b6..7ec0df20804 100644 --- a/docs/ru/engines/database-engines/materialized-postgresql.md +++ b/docs/ru/engines/database-engines/materialized-postgresql.md @@ -14,27 +14,32 @@ toc_title: MaterializedPostgreSQL ## Создание базы данных {#creating-a-database} ``` sql -CREATE DATABASE test_database -ENGINE = MaterializedPostgreSQL('postgres1:5432', 'postgres_database', 'postgres_user', 'postgres_password'); - -SELECT * FROM test_database.postgres_table; +CREATE DATABASE [IF NOT EXISTS] db_name [ON CLUSTER cluster] +ENGINE = MaterializedPostgreSQL('host:port', ['database' | database], 'user', 'password') [SETTINGS ...] ``` +**Параметры движка** + +- `host:port` — адрес сервера PostgreSQL. +- `database` — имя базы данных на удалённом сервере. +- `user` — пользователь PostgreSQL. +- `password` — пароль пользователя. + ## Настройки {#settings} -1. `materialized_postgresql_max_block_size` — задает максимальное количество строк, собранных перед вставкой данных в таблицу. По умолчанию: `65536`. +- [materialized_postgresql_max_block_size](../../operations/settings/settings.md#materialized-postgresql-max-block-size) -2. `materialized_postgresql_tables_list` — задает список таблиц для движка баз данных `MaterializedPostgreSQL`. По умолчанию: `whole database`. +- [materialized_postgresql_tables_list](../../operations/settings/settings.md#materialized-postgresql-tables-list) -3. `materialized_postgresql_allow_automatic_update` — позволяет автоматически обновить таблицу в фоновом режиме при обнаружении изменений схемы. По умолчанию: `0` (`false`). +- [materialized_postgresql_allow_automatic_update](../../operations/settings/settings.md#materialized-postgresql-allow-automatic-update) ``` sql -CREATE DATABASE test_database +CREATE DATABASE database1 ENGINE = MaterializedPostgreSQL('postgres1:5432', 'postgres_database', 'postgres_user', 'postgres_password') SETTINGS materialized_postgresql_max_block_size = 65536, materialized_postgresql_tables_list = 'table1,table2,table3'; -SELECT * FROM test_database.table1; +SELECT * FROM database1.table1; ``` ## Требования {#requirements} @@ -69,4 +74,13 @@ WHERE oid = 'postgres_table'::regclass; ``` !!! warning "Предупреждение" - Преобразование **TOAST**-значений не поддерживается. Для типа данных будет использоваться значение по умолчанию. + Репликация **TOAST**-значений не поддерживается. Для типа данных будет использоваться значение по умолчанию. + +## Пример использования {#example-of-use} + +``` sql +CREATE DATABASE postgresql_db +ENGINE = MaterializedPostgreSQL('postgres1:5432', 'postgres_database', 'postgres_user', 'postgres_password'); + +SELECT * FROM postgresql_db.postgres_table; +``` diff --git a/docs/ru/engines/table-engines/integrations/materialized-postgresql.md b/docs/ru/engines/table-engines/integrations/materialized-postgresql.md new file mode 100644 index 00000000000..ec56084c338 --- /dev/null +++ b/docs/ru/engines/table-engines/integrations/materialized-postgresql.md @@ -0,0 +1,44 @@ +--- +toc_priority: 12 +toc_title: MaterializedPostgreSQL +--- + +# MaterializedPostgreSQL {#materialize-postgresql} + +Движок `MaterializedPostgreSQL` позволяет выполнять запросы `SELECT` и `INSERT` над данными, хранящимися на удалённом PostgreSQL сервере. + +## Создание таблицы {#creating-a-table} + +``` sql +CREATE TABLE postgresql_db.postgresql_replica (key UInt64, value UInt64) +ENGINE = MaterializedPostgreSQL('postgres1:5432', 'postgres_database', 'postgresql_replica', 'postgres_user', 'postgres_password') +PRIMARY KEY key; +``` + +## Требования {#requirements} + +1. Настройка [wal_level](https://postgrespro.ru/docs/postgrespro/10/runtime-config-wal) должна иметь значение `logical`, параметр `max_replication_slots` должен быть равен по меньшей мере `2` в конфигурационном файле в PostgreSQL. + +2. Таблица, созданная с помощью движка `MaterializedPostgreSQL`, должна иметь первичный ключ — такой же, как индекс идентичности реплики (по умолчанию: первичный ключ) таблицы PostgreSQL (смотрите [индекс идентичности реплики](../../database-engines/materialized-postgresql.md#requirements)). + +3. Допускается только база данных [Atomic](https://en.wikipedia.org/wiki/Atomicity_(database_systems)). + +## Виртуальные столбцы {#virtual-columns} + +- `_version` (тип: UInt64) + +- `_sign` (тип: Int8) + +Эти столбцы не нужно добавлять при создании таблицы. Они всегда доступны в `SELECT` запросе. +Столбец `_version` равен позиции `LSN` в `WAL`, поэтому его можно использовать для проверки актуальности репликации. + +``` sql +CREATE TABLE postgresql_db.postgresql_replica (key UInt64, value UInt64) +ENGINE = MaterializedPostgreSQL('postgres1:5432', 'postgres_database', 'postgresql_replica', 'postgres_user', 'postgres_password') +PRIMARY KEY key; + +SELECT key, value, _version FROM postgresql_db.postgresql_replica; +``` + +!!! warning "Предупреждение" + Репликация **TOAST**-значений не поддерживается. Для типа данных будет использоваться значение по умолчанию. diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index 9e926a63c62..547060ae970 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -3124,3 +3124,30 @@ SETTINGS index_granularity = 8192 │ - 1 — тип `LowCardinality` конвертируется в тип `DICTIONARY`. Значение по умолчанию: `0`. + +## materialized_postgresql_max_block_size {#materialized-postgresql-max-block-size} + +Задает максимальное количество строк, собранных в памяти перед вставкой данных в таблицу базы данных PostgreSQL. + +Возможные значения: + +- Положительное целое число. + +Значение по умолчанию: `65536`. + +## materialized_postgresql_tables_list {#materialized-postgresql-tables-list} + +Задает список таблиц базы данных PostgreSQL, разделенных запятыми, которые будут реплицироваться с помощью движка базы данных [MaterializedPostgreSQL](../../engines/database-engines/materialized-postgresql.md). + +Значение по умолчанию: пустой список — база данных PostgreSQL будет полностью реплицирована. + +## materialized_postgresql_allow_automatic_update {#materialized-postgresql-allow-automatic-update} + +Позволяет автоматически обновить таблицу в фоновом режиме при обнаружении изменений схемы. DDL-запросы на стороне сервера PostgreSQL не реплицируются с помощью движка ClickHouse [MaterializedPostgreSQL](../../engines/database-engines/materialized-postgresql.md), поскольку это запрещено протоколом логической репликации PostgreSQL, но факт DDL-измененений обнаруживается транзакционно. В этом случае по умолчанию прекращается репликация этих таблиц после обнаружения DDL. Однако, если эта настройка включена, то вместо остановки репликации этих таблиц они будут перезагружены в фоновом режиме с помощью снимка базы данных без потери информации, и репликация для них будет продолжена. + +Возможные значения: + +- 0 — таблица не обновляется автоматически в фоновом режиме при обнаружении изменений схемы. +- 1 — таблица обновляется автоматически в фоновом режиме при обнаружении изменений схемы. + +Значение по умолчанию: `0`. From d6b5838282d808e3066ea98fdcc9938f56f2803d Mon Sep 17 00:00:00 2001 From: Dmitriy Date: Mon, 2 Aug 2021 23:16:23 +0300 Subject: [PATCH 101/599] Delete the introductory sentence MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Удалил неверное предложение. --- .../table-engines/integrations/materialized-postgresql.md | 2 -- .../table-engines/integrations/materialized-postgresql.md | 2 -- 2 files changed, 4 deletions(-) diff --git a/docs/en/engines/table-engines/integrations/materialized-postgresql.md b/docs/en/engines/table-engines/integrations/materialized-postgresql.md index 1704563570b..1639be791e9 100644 --- a/docs/en/engines/table-engines/integrations/materialized-postgresql.md +++ b/docs/en/engines/table-engines/integrations/materialized-postgresql.md @@ -5,8 +5,6 @@ toc_title: MaterializedPostgreSQL # MaterializedPostgreSQL {#materialize-postgresql} -The `MaterializedPostgreSQL` engine allows you to perform `SELECT` and `INSERT` queries on data that is stored on a remote PostgreSQL server. - ## Creating a Table {#creating-a-table} ``` sql diff --git a/docs/ru/engines/table-engines/integrations/materialized-postgresql.md b/docs/ru/engines/table-engines/integrations/materialized-postgresql.md index ec56084c338..aaa0a7af1d8 100644 --- a/docs/ru/engines/table-engines/integrations/materialized-postgresql.md +++ b/docs/ru/engines/table-engines/integrations/materialized-postgresql.md @@ -5,8 +5,6 @@ toc_title: MaterializedPostgreSQL # MaterializedPostgreSQL {#materialize-postgresql} -Движок `MaterializedPostgreSQL` позволяет выполнять запросы `SELECT` и `INSERT` над данными, хранящимися на удалённом PostgreSQL сервере. - ## Создание таблицы {#creating-a-table} ``` sql From f9658d220595428addeec35b7d60d301636f573e Mon Sep 17 00:00:00 2001 From: olgarev Date: Mon, 2 Aug 2021 22:52:01 +0000 Subject: [PATCH 102/599] Wrong backticks --- docs/en/engines/table-engines/integrations/hdfs.md | 12 ++++++------ docs/ru/engines/table-engines/integrations/hdfs.md | 12 ++++++------ 2 files changed, 12 insertions(+), 12 deletions(-) diff --git a/docs/en/engines/table-engines/integrations/hdfs.md b/docs/en/engines/table-engines/integrations/hdfs.md index 87284add50b..83ccdc01bb0 100644 --- a/docs/en/engines/table-engines/integrations/hdfs.md +++ b/docs/en/engines/table-engines/integrations/hdfs.md @@ -71,12 +71,12 @@ Constructions with `{}` are similar to the [remote](../../../sql-reference/table 1. Suppose we have several files in TSV format with the following URIs on HDFS: -- ‘hdfs://hdfs1:9000/some_dir/some_file_1’ -- ‘hdfs://hdfs1:9000/some_dir/some_file_2’ -- ‘hdfs://hdfs1:9000/some_dir/some_file_3’ -- ‘hdfs://hdfs1:9000/another_dir/some_file_1’ -- ‘hdfs://hdfs1:9000/another_dir/some_file_2’ -- ‘hdfs://hdfs1:9000/another_dir/some_file_3’ +- 'hdfs://hdfs1:9000/some_dir/some_file_1' +- 'hdfs://hdfs1:9000/some_dir/some_file_2' +- 'hdfs://hdfs1:9000/some_dir/some_file_3' +- 'hdfs://hdfs1:9000/another_dir/some_file_1' +- 'hdfs://hdfs1:9000/another_dir/some_file_2' +- 'hdfs://hdfs1:9000/another_dir/some_file_3' 1. There are several ways to make a table consisting of all six files: diff --git a/docs/ru/engines/table-engines/integrations/hdfs.md b/docs/ru/engines/table-engines/integrations/hdfs.md index e6eaec1bf9c..79a2d22a840 100644 --- a/docs/ru/engines/table-engines/integrations/hdfs.md +++ b/docs/ru/engines/table-engines/integrations/hdfs.md @@ -67,12 +67,12 @@ SELECT * FROM hdfs_engine_table LIMIT 2 1. Предположим, у нас есть несколько файлов со следующими URI в HDFS: -- ‘hdfs://hdfs1:9000/some_dir/some_file_1’ -- ‘hdfs://hdfs1:9000/some_dir/some_file_2’ -- ‘hdfs://hdfs1:9000/some_dir/some_file_3’ -- ‘hdfs://hdfs1:9000/another_dir/some_file_1’ -- ‘hdfs://hdfs1:9000/another_dir/some_file_2’ -- ‘hdfs://hdfs1:9000/another_dir/some_file_3’ +- 'hdfs://hdfs1:9000/some_dir/some_file_1' +- 'hdfs://hdfs1:9000/some_dir/some_file_2' +- 'hdfs://hdfs1:9000/some_dir/some_file_3' +- 'hdfs://hdfs1:9000/another_dir/some_file_1' +- 'hdfs://hdfs1:9000/another_dir/some_file_2' +- 'hdfs://hdfs1:9000/another_dir/some_file_3' 1. Есть несколько возможностей создать таблицу, состояющую из этих шести файлов: From 97851bde088ab3289064b808da7dd63d541d8ea7 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 3 Aug 2021 10:10:08 +0300 Subject: [PATCH 103/599] Fix Distributed over Distributed for WithMergeableStateAfterAggregation* stages In case if one Distributed has multiple shards, and underlying Distributed has only one, there can be the case when the query will be tried to process from Complete to WithMergeableStateAfterAggregation, which is obviously wrong. --- src/Storages/StorageDistributed.cpp | 15 ++++++++++++++- ...t_WithMergeableStateAfterAggregation.reference | 8 ++++++++ ...on_dist_WithMergeableStateAfterAggregation.sql | 6 ++++++ 3 files changed, 28 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02001_dist_on_dist_WithMergeableStateAfterAggregation.reference create mode 100644 tests/queries/0_stateless/02001_dist_on_dist_WithMergeableStateAfterAggregation.sql diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 2bbb92cf0b8..8f9c4bcc655 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -447,6 +447,8 @@ QueryProcessingStage::Enum StorageDistributed::getQueryProcessingStage( { /// NOTE: distributed_group_by_no_merge=1 does not respect distributed_push_down_limit /// (since in this case queries processed separately and the initiator is just a proxy in this case). + if (to_stage != QueryProcessingStage::Complete) + throw Exception("Queries with distributed_group_by_no_merge=1 should be processed to Complete stage", ErrorCodes::LOGICAL_ERROR); return QueryProcessingStage::Complete; } } @@ -459,11 +461,22 @@ QueryProcessingStage::Enum StorageDistributed::getQueryProcessingStage( /// If there is only one node, the query can be fully processed by the /// shard, initiator will work as a proxy only. if (getClusterQueriedNodes(settings, cluster) == 1) - return QueryProcessingStage::Complete; + { + /// In case the query was processed to + /// WithMergeableStateAfterAggregation/WithMergeableStateAfterAggregationAndLimit + /// (which are greater the Complete stage) + /// we cannot return Complete (will break aliases and similar), + /// relevant for Distributed over Distributed + return std::max(to_stage, QueryProcessingStage::Complete); + } auto optimized_stage = getOptimizedQueryProcessingStage(query_info, settings); if (optimized_stage) + { + if (*optimized_stage == QueryProcessingStage::Complete) + return std::min(to_stage, *optimized_stage); return *optimized_stage; + } return QueryProcessingStage::WithMergeableState; } diff --git a/tests/queries/0_stateless/02001_dist_on_dist_WithMergeableStateAfterAggregation.reference b/tests/queries/0_stateless/02001_dist_on_dist_WithMergeableStateAfterAggregation.reference new file mode 100644 index 00000000000..6c680840239 --- /dev/null +++ b/tests/queries/0_stateless/02001_dist_on_dist_WithMergeableStateAfterAggregation.reference @@ -0,0 +1,8 @@ +-- { echo } +select dummy as foo from remote('127.{2,3}', currentDatabase(), dist) limit 1 settings prefer_localhost_replica=0, distributed_push_down_limit=0; +0 +select dummy as foo from remote('127.{2,3}', currentDatabase(), dist) limit 1 settings prefer_localhost_replica=0, distributed_push_down_limit=1; +0 +select dummy as foo from remote('127.{2,3}', currentDatabase(), dist) limit 1 settings prefer_localhost_replica=0, distributed_group_by_no_merge=1; +0 +0 diff --git a/tests/queries/0_stateless/02001_dist_on_dist_WithMergeableStateAfterAggregation.sql b/tests/queries/0_stateless/02001_dist_on_dist_WithMergeableStateAfterAggregation.sql new file mode 100644 index 00000000000..0925df1888d --- /dev/null +++ b/tests/queries/0_stateless/02001_dist_on_dist_WithMergeableStateAfterAggregation.sql @@ -0,0 +1,6 @@ +drop table if exists dist; +create table dist as system.one engine=Distributed('test_shard_localhost', system, one); +-- { echo } +select dummy as foo from remote('127.{2,3}', currentDatabase(), dist) limit 1 settings prefer_localhost_replica=0, distributed_push_down_limit=0; +select dummy as foo from remote('127.{2,3}', currentDatabase(), dist) limit 1 settings prefer_localhost_replica=0, distributed_push_down_limit=1; +select dummy as foo from remote('127.{2,3}', currentDatabase(), dist) limit 1 settings prefer_localhost_replica=0, distributed_group_by_no_merge=1; From aad913291dfcdf47e1f27b3e8ffc050af2bd7f36 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 3 Aug 2021 14:38:59 +0200 Subject: [PATCH 104/599] test_distributed_directory_monitor_split_batch_on_failure: Track everything Reverting the memory reduction and forcing max_untracked_memory to 0 which should be more reliable and closer to the original test --- .../test.py | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/tests/integration/test_distributed_directory_monitor_split_batch_on_failure/test.py b/tests/integration/test_distributed_directory_monitor_split_batch_on_failure/test.py index 19afb1f13c7..b0b89fde41f 100644 --- a/tests/integration/test_distributed_directory_monitor_split_batch_on_failure/test.py +++ b/tests/integration/test_distributed_directory_monitor_split_batch_on_failure/test.py @@ -40,8 +40,9 @@ def test_distributed_directory_monitor_split_batch_on_failure_OFF(started_cluste limit = 100e3 node2.query(f'insert into dist select number/100, number from system.numbers limit {limit} offset {limit*i}', settings={ # max_memory_usage is the limit for the batch on the remote node - # (local query should not be affected since 20MB is enough for 100K rows) - 'max_memory_usage': '20Mi', + # (local query should not be affected since 30MB is enough for 100K rows) + 'max_memory_usage': '30Mi', + 'max_untracked_memory': '0' }) # "Received from" is mandatory, since the exception should be thrown on the remote node. with pytest.raises(QueryRuntimeException, match=r'DB::Exception: Received from.*Memory limit \(for query\) exceeded: .*while pushing to view default\.mv'): @@ -53,8 +54,9 @@ def test_distributed_directory_monitor_split_batch_on_failure_ON(started_cluster limit = 100e3 node1.query(f'insert into dist select number/100, number from system.numbers limit {limit} offset {limit*i}', settings={ # max_memory_usage is the limit for the batch on the remote node - # (local query should not be affected since 20MB is enough for 100K rows) - 'max_memory_usage': '20Mi', + # (local query should not be affected since 30MB is enough for 100K rows) + 'max_memory_usage': '30Mi', + 'max_untracked_memory': '0' }) node1.query('system flush distributed dist') assert int(node1.query('select count() from dist_data')) == 100000 From 46d688deb3fc99fad7132388e04754ad0dc55f5a Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Tue, 3 Aug 2021 16:07:46 +0300 Subject: [PATCH 105/599] Add stings empty function. --- .../functions/string-functions.md | 40 ++++++++++++++++--- .../functions/string-functions.md | 39 ++++++++++++++++-- 2 files changed, 70 insertions(+), 9 deletions(-) diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index 8ec8aa7339d..8723b9d278a 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -10,11 +10,41 @@ toc_title: Strings ## empty {#empty} -Returns 1 for an empty string or 0 for a non-empty string. -The result type is UInt8. -A string is considered non-empty if it contains at least one byte, even if this is a space or a null byte. -The function also works for arrays or UUID. -UUID is empty if it is all zeros (nil UUID). +Checks whether the input string is empty. + +**Syntax** + +``` sql +empty(x) +``` + +A string is considered non-empty if it contains at least one byte, even if this is a space or a null byte. The UUID is empty if it contains all zeros (zero UUID). + +**Arguments** + +- `x` — Input value. [Array](../data-types/array.md), [String](../data-types/string.md), [UUID](../data-types/uuid.md). + +**Returned value** + +- Returns `1` for an empty string or `0` for a non-empty string. + +Type: [UInt8](../data-types/int-uint.md). + +**Example** + +Query: + +```sql +SELECT empty(''); +``` + +Result: + +```text +┌─empty('')─┐ +│ 1 │ +└───────────┘ +``` ## notEmpty {#notempty} diff --git a/docs/ru/sql-reference/functions/string-functions.md b/docs/ru/sql-reference/functions/string-functions.md index b587a991db1..176cd83b0d0 100644 --- a/docs/ru/sql-reference/functions/string-functions.md +++ b/docs/ru/sql-reference/functions/string-functions.md @@ -7,10 +7,41 @@ toc_title: "Функции для работы со строками" ## empty {#empty} -Возвращает 1 для пустой строки, и 0 для непустой строки. -Тип результата — UInt8. -Строка считается непустой, если содержит хотя бы один байт, пусть даже это пробел или нулевой байт. -Функция также работает для массивов. +Проверяет, является ли входное значение пустым. + +**Синтаксис** + +``` sql +empty(x) +``` + +Строка считается непустой, если содержит хотя бы один байт, пусть даже это пробел или нулевой байт. UUID считается пустой, если он содержит только нули (нулевой UUID). + +**Параметры** + +- `x` — Входное значение. [Array](../data-types/array.md), [String](../data-types/string.md), [UUID](../data-types/uuid.md). + +**Возвращаемое значение** + +- Возвращает `1` для пустой строки, и `0` — для непустой строки. + +Тип: [UInt8](../data-types/int-uint.md). + +**Пример** + +Запрос: + +```sql +SELECT empty(''); +``` + +Результат: + +```text +┌─empty('')─┐ +│ 1 │ +└───────────┘ +``` ## notEmpty {#notempty} From 8d061390f3e22e6e464d6e75e4340d69f7faa89e Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Tue, 3 Aug 2021 16:19:56 +0300 Subject: [PATCH 106/599] Updated performance tests --- tests/performance/jit_aggregate_functions.xml | 171 ----------- .../jit_aggregate_functions_no_key.xml | 284 ++++++++++++++++++ 2 files changed, 284 insertions(+), 171 deletions(-) create mode 100644 tests/performance/jit_aggregate_functions_no_key.xml diff --git a/tests/performance/jit_aggregate_functions.xml b/tests/performance/jit_aggregate_functions.xml index b1a937dfae9..3e99f6d9615 100644 --- a/tests/performance/jit_aggregate_functions.xml +++ b/tests/performance/jit_aggregate_functions.xml @@ -150,44 +150,6 @@ FORMAT Null - - SELECT - {function}(value_1), - {function}(value_2), - {function}(value_3) - FROM {table} - FORMAT Null - - - - SELECT - {function}(value_1), - {function}(value_2), - sum(toUInt256(value_3)), - {function}(value_3) - FROM {table} - FORMAT Null - - - - SELECT - {function}If(value_1, predicate), - {function}If(value_2, predicate), - {function}If(value_3, predicate) - FROM {table} - FORMAT Null - - - - SELECT - {function}If(value_1, predicate), - {function}If(value_2, predicate), - sumIf(toUInt256(value_3), predicate), - {function}If(value_3, predicate) - FROM {table} - FORMAT Null - - SELECT {function}(value_1), @@ -238,51 +200,6 @@ FORMAT Null - - SELECT - {function}(value_1), - {function}(value_2), - {function}(value_3), - {function}(value_4), - {function}(value_5) - FROM {table} - FORMAT Null - - - - SELECT - {function}(value_1), - {function}(value_2), - sum(toUInt256(value_3)), - {function}(value_3), - {function}(value_4), - {function}(value_5) - FROM {table} - FORMAT Null - - - - SELECT - {function}If(value_1, predicate), - {function}If(value_2, predicate), - {function}If(value_3, predicate), - {function}If(value_4, predicate), - {function}If(value_5, predicate) - FROM {table} - FORMAT Null - - - - SELECT - {function}If(value_1, predicate), - {function}If(value_2, predicate), - sumIf(toUInt256(value_3), predicate), - {function}If(value_3, predicate), - {function}If(value_4, predicate), - {function}If(value_5, predicate) - FROM {table} - FORMAT Null - SELECT @@ -330,48 +247,6 @@ FORMAT Null - - SELECT - {function}(WatchID), - {function}(CounterID), - {function}(ClientIP) - FROM hits_100m_single - FORMAT Null - - - - SELECT - {function}(WatchID), - {function}(CounterID), - sum(toUInt256(ClientIP)), - {function}(ClientIP) - FROM hits_100m_single - FORMAT Null - - - - SELECT - {function}(WatchID), - {function}(CounterID), - {function}(ClientIP), - {function}(IPNetworkID), - {function}(SearchEngineID) - FROM hits_100m_single - FORMAT Null - - - - SELECT - {function}(WatchID), - {function}(CounterID), - sum(toUInt256(ClientIP)), - {function}(ClientIP), - {function}(IPNetworkID), - {function}(SearchEngineID) - FROM hits_100m_single - FORMAT Null - - WITH (WatchID % 2 == 0) AS predicate SELECT @@ -422,51 +297,5 @@ FORMAT Null - - WITH (WatchID % 2 == 0) AS predicate - SELECT - {function}If(WatchID, predicate), - {function}If(CounterID, predicate), - {function}If(ClientIP, predicate) - FROM hits_100m_single - FORMAT Null - - - - WITH (WatchID % 2 == 0) AS predicate - SELECT - {function}If(WatchID, predicate), - {function}If(CounterID, predicate), - sumIf(toUInt256(ClientIP), predicate), - {function}If(ClientIP, predicate) - FROM hits_100m_single - FORMAT Null - - - - WITH (WatchID % 2 == 0) AS predicate - SELECT - {function}If(WatchID, predicate), - {function}If(CounterID, predicate), - {function}If(ClientIP, predicate), - {function}If(IPNetworkID, predicate), - {function}If(SearchEngineID, predicate) - FROM hits_100m_single - FORMAT Null - - - - WITH (WatchID % 2 == 0) AS predicate - SELECT - {function}If(WatchID, predicate), - {function}If(CounterID, predicate), - sumIf(toUInt256(ClientIP), predicate), - {function}If(ClientIP, predicate), - {function}If(IPNetworkID, predicate), - {function}If(SearchEngineID, predicate) - FROM hits_100m_single - FORMAT Null - - DROP TABLE IF EXISTS {table} diff --git a/tests/performance/jit_aggregate_functions_no_key.xml b/tests/performance/jit_aggregate_functions_no_key.xml new file mode 100644 index 00000000000..0d2577af97c --- /dev/null +++ b/tests/performance/jit_aggregate_functions_no_key.xml @@ -0,0 +1,284 @@ + + + hits_100m_single + + + + 1 + 0 + + + + CREATE TABLE jit_test_memory ( + key UInt64, + value_1 UInt64, + value_2 UInt64, + value_3 UInt64, + value_4 UInt64, + value_5 UInt64, + predicate UInt8 + ) Engine = Memory + + + + CREATE TABLE jit_test_merge_tree ( + key UInt64, + value_1 UInt64, + value_2 UInt64, + value_3 UInt64, + value_4 UInt64, + value_5 UInt64, + predicate UInt8 + ) Engine = MergeTree + ORDER BY key + + + + CREATE TABLE jit_test_merge_tree_nullable ( + key UInt64, + value_1 Nullable(UInt64), + value_2 Nullable(UInt64), + value_3 Nullable(UInt64), + value_4 Nullable(UInt64), + value_5 Nullable(UInt64), + predicate UInt8 + ) Engine = Memory + + + + CREATE TABLE jit_test_memory_nullable ( + key UInt64, + value_1 Nullable(UInt64), + value_2 Nullable(UInt64), + value_3 Nullable(UInt64), + value_4 Nullable(UInt64), + value_5 Nullable(UInt64), + predicate UInt8 + ) Engine = MergeTree + ORDER BY key + + + + + function + + sum + min + max + avg + any + anyLast + count + groupBitOr + groupBitAnd + groupBitXor + + + + + table + + jit_test_memory + jit_test_merge_tree + jit_test_memory_nullable + jit_test_merge_tree_nullable + + + + + group_scale + + 1000000 + + + + + + INSERT INTO {table} + SELECT + number % 1000000, + number, + number, + number, + number, + number, + if (number % 2 == 0, 1, 0) + FROM + system.numbers_mt + LIMIT 100000000 + + + + SELECT + {function}(value_1), + {function}(value_2), + {function}(value_3) + FROM {table} + FORMAT Null + + + + SELECT + {function}(value_1), + {function}(value_2), + sum(toUInt256(value_3)), + {function}(value_3) + FROM {table} + FORMAT Null + + + + SELECT + {function}If(value_1, predicate), + {function}If(value_2, predicate), + {function}If(value_3, predicate) + FROM {table} + FORMAT Null + + + + SELECT + {function}If(value_1, predicate), + {function}If(value_2, predicate), + sumIf(toUInt256(value_3), predicate), + {function}If(value_3, predicate) + FROM {table} + FORMAT Null + + + + SELECT + {function}(value_1), + {function}(value_2), + {function}(value_3), + {function}(value_4), + {function}(value_5) + FROM {table} + FORMAT Null + + + + SELECT + {function}(value_1), + {function}(value_2), + sum(toUInt256(value_3)), + {function}(value_3), + {function}(value_4), + {function}(value_5) + FROM {table} + FORMAT Null + + + + SELECT + {function}If(value_1, predicate), + {function}If(value_2, predicate), + {function}If(value_3, predicate), + {function}If(value_4, predicate), + {function}If(value_5, predicate) + FROM {table} + FORMAT Null + + + + SELECT + {function}If(value_1, predicate), + {function}If(value_2, predicate), + sumIf(toUInt256(value_3), predicate), + {function}If(value_3, predicate), + {function}If(value_4, predicate), + {function}If(value_5, predicate) + FROM {table} + FORMAT Null + + + + SELECT + {function}(WatchID), + {function}(CounterID), + {function}(ClientIP) + FROM hits_100m_single + FORMAT Null + + + + SELECT + {function}(WatchID), + {function}(CounterID), + sum(toUInt256(ClientIP)), + {function}(ClientIP) + FROM hits_100m_single + FORMAT Null + + + + SELECT + {function}(WatchID), + {function}(CounterID), + {function}(ClientIP), + {function}(IPNetworkID), + {function}(SearchEngineID) + FROM hits_100m_single + FORMAT Null + + + + SELECT + {function}(WatchID), + {function}(CounterID), + sum(toUInt256(ClientIP)), + {function}(ClientIP), + {function}(IPNetworkID), + {function}(SearchEngineID) + FROM hits_100m_single + FORMAT Null + + + + WITH (WatchID % 2 == 0) AS predicate + SELECT + {function}If(WatchID, predicate), + {function}If(CounterID, predicate), + {function}If(ClientIP, predicate) + FROM hits_100m_single + FORMAT Null + + + + WITH (WatchID % 2 == 0) AS predicate + SELECT + {function}If(WatchID, predicate), + {function}If(CounterID, predicate), + sumIf(toUInt256(ClientIP), predicate), + {function}If(ClientIP, predicate) + FROM hits_100m_single + FORMAT Null + + + + WITH (WatchID % 2 == 0) AS predicate + SELECT + {function}If(WatchID, predicate), + {function}If(CounterID, predicate), + {function}If(ClientIP, predicate), + {function}If(IPNetworkID, predicate), + {function}If(SearchEngineID, predicate) + FROM hits_100m_single + FORMAT Null + + + + WITH (WatchID % 2 == 0) AS predicate + SELECT + {function}If(WatchID, predicate), + {function}If(CounterID, predicate), + sumIf(toUInt256(ClientIP), predicate), + {function}If(ClientIP, predicate), + {function}If(IPNetworkID, predicate), + {function}If(SearchEngineID, predicate) + FROM hits_100m_single + FORMAT Null + + + DROP TABLE IF EXISTS {table} + From dad7f443a6c458450a9b2b57a90758f284219094 Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Tue, 3 Aug 2021 16:47:16 +0300 Subject: [PATCH 107/599] Update materialize-mysql. --- .../database-engines/materialized-mysql.md | 11 +++++++---- .../database-engines/materialized-mysql.md | 17 +++++++++++++++++ 2 files changed, 24 insertions(+), 4 deletions(-) diff --git a/docs/en/engines/database-engines/materialized-mysql.md b/docs/en/engines/database-engines/materialized-mysql.md index 20e16473115..57bc1fe44a2 100644 --- a/docs/en/engines/database-engines/materialized-mysql.md +++ b/docs/en/engines/database-engines/materialized-mysql.md @@ -42,12 +42,15 @@ CREATE DATABASE mysql ENGINE = MaterializedMySQL('localhost:3306', 'db', 'user', max_wait_time_when_mysql_unavailable=10000; ``` -**Settings on MySQL-server side** +**Settings on Mysql-server Side** -For the correct work of `MaterializeMySQL`, there are few mandatory `MySQL`-side configuration settings that should be set: +For the correct work of `MaterializeMySQL`, there are few mandatory `MySQL`-side configuration settings that must be set: - `default_authentication_plugin = mysql_native_password` since `MaterializeMySQL` can only authorize with this method. -- `gtid_mode = on` since GTID based logging is a mandatory for providing correct `MaterializeMySQL` replication. Pay attention that while turning this mode `On` you should also specify `enforce_gtid_consistency = on`. +- `gtid_mode = on` since GTID based logging is a mandatory for providing correct `MaterializeMySQL` replication. + +!!! attention "Attention" + While turning on `gtid_mode` you should also specify `enforce_gtid_consistency = on`. ## Virtual columns {#virtual-columns} @@ -85,7 +88,7 @@ Other types are not supported. If MySQL table contains a column of such type, Cl ## Specifics and Recommendations {#specifics-and-recommendations} -### Compatibility restrictions +### Compatibility restrictions {#compatibility-restrictions} Apart of the data types limitations there are few restrictions comparing to `MySQL` databases, that should be resolved before replication will be possible: diff --git a/docs/ru/engines/database-engines/materialized-mysql.md b/docs/ru/engines/database-engines/materialized-mysql.md index f5f0166c9dc..c02e81231dd 100644 --- a/docs/ru/engines/database-engines/materialized-mysql.md +++ b/docs/ru/engines/database-engines/materialized-mysql.md @@ -26,6 +26,15 @@ ENGINE = MaterializedMySQL('host:port', ['database' | database], 'user', 'passwo - `user` — пользователь MySQL. - `password` — пароль пользователя. +**Настройки на стороне MySQL-сервера** + +Для правильной работы `MaterializeMySQL` следует обязательно указать на MySQL сервере следующие параметры конфигурации: +- `default_authentication_plugin = mysql_native_password` — `MaterializeMySQL` может авторизоваться только с помощью этого метода. +- `gtid_mode = on` — ведение журнала на основе GTID является обязательным для обеспечения правильной репликации. + +!!! attention "Внимание" + При включении `gtid_mode` вы также должны указать `enforce_gtid_consistency = on`. + ## Виртуальные столбцы {#virtual-columns} При работе с движком баз данных `MaterializedMySQL` используются таблицы семейства [ReplacingMergeTree](../../engines/table-engines/mergetree-family/replacingmergetree.md) с виртуальными столбцами `_sign` и `_version`. @@ -54,6 +63,7 @@ ENGINE = MaterializedMySQL('host:port', ['database' | database], 'user', 'passwo | STRING | [String](../../sql-reference/data-types/string.md) | | VARCHAR, VAR_STRING | [String](../../sql-reference/data-types/string.md) | | BLOB | [String](../../sql-reference/data-types/string.md) | +| BINARY | [FixedString](../../sql-reference/data-types/fixedstring.md) | Другие типы не поддерживаются. Если таблица MySQL содержит столбец другого типа, ClickHouse выдаст исключение "Неподдерживаемый тип данных" ("Unhandled data type") и остановит репликацию. @@ -61,6 +71,13 @@ ENGINE = MaterializedMySQL('host:port', ['database' | database], 'user', 'passwo ## Особенности и рекомендации {#specifics-and-recommendations} +### Ограничения совместимости {#compatibility-restrictions} + +Кроме ограничений на типы данных, существует несколько ограничений по сравнению с базами данных MySQL, которые следует решить до того, как станет возможной репликация: + +- Каждая таблица в `MySQL` должна содержать `PRIMARY KEY`. +- Репликация для таблиц, содержащих строки со значениями полей `ENUM` вне диапазона значений (определяется размерностью `ENUM`) вне диапазона (указанного в подписи "ПЕРЕЧИСЛЕНИЕ"), не будет работать. + ### DDL-запросы {#ddl-queries} DDL-запросы в MySQL конвертируются в соответствующие DDL-запросы в ClickHouse ([ALTER](../../sql-reference/statements/alter/index.md), [CREATE](../../sql-reference/statements/create/index.md), [DROP](../../sql-reference/statements/drop.md), [RENAME](../../sql-reference/statements/rename.md)). Если ClickHouse не может конвертировать какой-либо DDL-запрос, он его игнорирует. From 44bb44cc2c3ddb1ee03e1b762d7bb27841c0f9b3 Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Tue, 3 Aug 2021 16:55:38 +0300 Subject: [PATCH 108/599] Update materialize-mysql. --- docs/ru/engines/database-engines/materialized-mysql.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/engines/database-engines/materialized-mysql.md b/docs/ru/engines/database-engines/materialized-mysql.md index c02e81231dd..761a5ef4aaa 100644 --- a/docs/ru/engines/database-engines/materialized-mysql.md +++ b/docs/ru/engines/database-engines/materialized-mysql.md @@ -76,7 +76,7 @@ ENGINE = MaterializedMySQL('host:port', ['database' | database], 'user', 'passwo Кроме ограничений на типы данных, существует несколько ограничений по сравнению с базами данных MySQL, которые следует решить до того, как станет возможной репликация: - Каждая таблица в `MySQL` должна содержать `PRIMARY KEY`. -- Репликация для таблиц, содержащих строки со значениями полей `ENUM` вне диапазона значений (определяется размерностью `ENUM`) вне диапазона (указанного в подписи "ПЕРЕЧИСЛЕНИЕ"), не будет работать. +- Репликация для таблиц, содержащих строки со значениями полей `ENUM` вне диапазона значений (определяется размерностью `ENUM`), не будет работать. ### DDL-запросы {#ddl-queries} From c310cef87f91b1e87684987620d5d8caba31c2ba Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Tue, 3 Aug 2021 16:56:35 +0300 Subject: [PATCH 109/599] Update materialize-mysql. --- docs/ru/sql-reference/functions/string-functions.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/ru/sql-reference/functions/string-functions.md b/docs/ru/sql-reference/functions/string-functions.md index 176cd83b0d0..16a2c16c466 100644 --- a/docs/ru/sql-reference/functions/string-functions.md +++ b/docs/ru/sql-reference/functions/string-functions.md @@ -7,7 +7,7 @@ toc_title: "Функции для работы со строками" ## empty {#empty} -Проверяет, является ли входное значение пустым. +Проверяет, является ли входная стока пустой. **Синтаксис** @@ -19,7 +19,7 @@ empty(x) **Параметры** -- `x` — Входное значение. [Array](../data-types/array.md), [String](../data-types/string.md), [UUID](../data-types/uuid.md). +- `x` — Входная строка. [Array](../data-types/array.md), [String](../data-types/string.md), [UUID](../data-types/uuid.md). **Возвращаемое значение** From a57f274dd185a907041d5e7a60427b724d878d86 Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Tue, 3 Aug 2021 17:28:05 +0300 Subject: [PATCH 110/599] Update materialize-mysql. --- docs/ru/sql-reference/functions/string-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/string-functions.md b/docs/ru/sql-reference/functions/string-functions.md index 16a2c16c466..e6c8e17bdca 100644 --- a/docs/ru/sql-reference/functions/string-functions.md +++ b/docs/ru/sql-reference/functions/string-functions.md @@ -7,7 +7,7 @@ toc_title: "Функции для работы со строками" ## empty {#empty} -Проверяет, является ли входная стока пустой. +Проверяет, является ли входная строка пустой. **Синтаксис** From ea10317845a35dfe39e087f50e6cd5cd843cac6f Mon Sep 17 00:00:00 2001 From: sevirov <72220289+sevirov@users.noreply.github.com> Date: Tue, 3 Aug 2021 19:41:12 +0300 Subject: [PATCH 111/599] Update docs/ru/engines/database-engines/materialized-postgresql.md Co-authored-by: Anna <42538400+adevyatova@users.noreply.github.com> --- docs/ru/engines/database-engines/materialized-postgresql.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/engines/database-engines/materialized-postgresql.md b/docs/ru/engines/database-engines/materialized-postgresql.md index 7ec0df20804..cc0dc17e37b 100644 --- a/docs/ru/engines/database-engines/materialized-postgresql.md +++ b/docs/ru/engines/database-engines/materialized-postgresql.md @@ -3,7 +3,7 @@ toc_priority: 30 toc_title: MaterializedPostgreSQL --- -# MaterializedPostgreSQL {#materialize-postgresql} +# [экспериментальный] MaterializedPostgreSQL {#materialize-postgresql} Создает базу данных ClickHouse со всеми таблицами, существующими в PostgreSQL, и всеми данными в этих таблицах. From da680d2673438eec84c72965860b4da2007ddb8b Mon Sep 17 00:00:00 2001 From: sevirov <72220289+sevirov@users.noreply.github.com> Date: Tue, 3 Aug 2021 19:41:35 +0300 Subject: [PATCH 112/599] Update docs/en/engines/database-engines/materialized-postgresql.md Co-authored-by: Anna <42538400+adevyatova@users.noreply.github.com> --- docs/en/engines/database-engines/materialized-postgresql.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/engines/database-engines/materialized-postgresql.md b/docs/en/engines/database-engines/materialized-postgresql.md index 5c1dc105d72..204b99ea4ae 100644 --- a/docs/en/engines/database-engines/materialized-postgresql.md +++ b/docs/en/engines/database-engines/materialized-postgresql.md @@ -3,7 +3,7 @@ toc_priority: 30 toc_title: MaterializedPostgreSQL --- -# MaterializedPostgreSQL {#materialize-postgresql} +# [experimental] MaterializedPostgreSQL {#materialize-postgresql} Creates ClickHouse database with all the tables existing in PostgreSQL, and all the data in those tables. From 9859c1171160d4bdc545b868c27dca512e32d77a Mon Sep 17 00:00:00 2001 From: sevirov <72220289+sevirov@users.noreply.github.com> Date: Tue, 3 Aug 2021 19:41:51 +0300 Subject: [PATCH 113/599] Update docs/en/engines/database-engines/materialized-postgresql.md Co-authored-by: Anna <42538400+adevyatova@users.noreply.github.com> --- docs/en/engines/database-engines/materialized-postgresql.md | 1 - 1 file changed, 1 deletion(-) diff --git a/docs/en/engines/database-engines/materialized-postgresql.md b/docs/en/engines/database-engines/materialized-postgresql.md index 204b99ea4ae..ec34d5fbd34 100644 --- a/docs/en/engines/database-engines/materialized-postgresql.md +++ b/docs/en/engines/database-engines/materialized-postgresql.md @@ -9,7 +9,6 @@ Creates ClickHouse database with all the tables existing in PostgreSQL, and all ClickHouse server works as PostgreSQL replica. It reads WAL and performs DML queries. DDL is not replicated, but can be handled (described below). -This feature is experimental. ## Creating a Database {#creating-a-database} From bfb64b224293e07bc9d7de4332aa9db93f159237 Mon Sep 17 00:00:00 2001 From: sevirov <72220289+sevirov@users.noreply.github.com> Date: Tue, 3 Aug 2021 19:42:11 +0300 Subject: [PATCH 114/599] Update docs/ru/engines/database-engines/materialized-postgresql.md Co-authored-by: Anna <42538400+adevyatova@users.noreply.github.com> --- docs/ru/engines/database-engines/materialized-postgresql.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/engines/database-engines/materialized-postgresql.md b/docs/ru/engines/database-engines/materialized-postgresql.md index cc0dc17e37b..153f43e466c 100644 --- a/docs/ru/engines/database-engines/materialized-postgresql.md +++ b/docs/ru/engines/database-engines/materialized-postgresql.md @@ -7,7 +7,7 @@ toc_title: MaterializedPostgreSQL Создает базу данных ClickHouse со всеми таблицами, существующими в PostgreSQL, и всеми данными в этих таблицах. -Сервер ClickHouse работает как реплика PostgreSQL. Он читает файл binlog и выполняет DDL and DML-запросы. +Сервер ClickHouse работает как реплика PostgreSQL. Он читает файл `binlog` и выполняет DDL и DML-запросы. `MaterializedPostgreSQL` — экспериментальный движок баз данных. From c9013d0c8258f889c7725fea66b6e2a9dfc5a32c Mon Sep 17 00:00:00 2001 From: sevirov <72220289+sevirov@users.noreply.github.com> Date: Tue, 3 Aug 2021 19:42:23 +0300 Subject: [PATCH 115/599] Update docs/ru/engines/database-engines/materialized-postgresql.md Co-authored-by: Anna <42538400+adevyatova@users.noreply.github.com> --- docs/ru/engines/database-engines/materialized-postgresql.md | 1 - 1 file changed, 1 deletion(-) diff --git a/docs/ru/engines/database-engines/materialized-postgresql.md b/docs/ru/engines/database-engines/materialized-postgresql.md index 153f43e466c..064b231c2f1 100644 --- a/docs/ru/engines/database-engines/materialized-postgresql.md +++ b/docs/ru/engines/database-engines/materialized-postgresql.md @@ -9,7 +9,6 @@ toc_title: MaterializedPostgreSQL Сервер ClickHouse работает как реплика PostgreSQL. Он читает файл `binlog` и выполняет DDL и DML-запросы. -`MaterializedPostgreSQL` — экспериментальный движок баз данных. ## Создание базы данных {#creating-a-database} From e939dc0b9ca369ee667a01acb25a8814045cd170 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Tue, 3 Aug 2021 20:30:05 +0300 Subject: [PATCH 116/599] fix --- src/Access/ya.make | 1 - src/Access/ya.make.in | 2 +- 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Access/ya.make b/src/Access/ya.make index 5f2f410cabd..38c1b007ff8 100644 --- a/src/Access/ya.make +++ b/src/Access/ya.make @@ -46,7 +46,6 @@ SRCS( SettingsProfilesInfo.cpp User.cpp UsersConfigAccessStorage.cpp - tests/gtest_access_rights_ops.cpp ) diff --git a/src/Access/ya.make.in b/src/Access/ya.make.in index 1f11c7d7d2a..5fa69cec4bb 100644 --- a/src/Access/ya.make.in +++ b/src/Access/ya.make.in @@ -8,7 +8,7 @@ PEERDIR( SRCS( - + ) END() From 699a3d9031c1ba88503b7e275ab6c4f6d2bfce96 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 3 Aug 2021 21:03:24 +0300 Subject: [PATCH 117/599] implement legacy_column_name_of_tuple_literal in less intrusive way --- src/Interpreters/ActionsVisitor.cpp | 26 ++++----- src/Interpreters/ExpressionAnalyzer.cpp | 54 +++++++++---------- src/Interpreters/InterpreterSelectQuery.cpp | 22 ++++---- src/Interpreters/TreeRewriter.cpp | 27 ++++++++++ .../evaluateConstantExpression.cpp | 2 +- src/Parsers/ASTFunction.cpp | 20 +------ src/Parsers/ASTFunction.h | 4 -- src/Parsers/ASTLiteral.cpp | 14 +++-- src/Parsers/ASTLiteral.h | 5 +- src/Parsers/ASTWithAlias.cpp | 8 --- src/Parsers/ASTWithAlias.h | 3 -- src/Parsers/IAST.cpp | 8 --- src/Parsers/IAST.h | 4 -- .../02002_system_table_with_tuple.reference | 1 + .../02002_system_table_with_tuple.sql | 2 + 15 files changed, 92 insertions(+), 108 deletions(-) create mode 100644 tests/queries/0_stateless/02002_system_table_with_tuple.reference create mode 100644 tests/queries/0_stateless/02002_system_table_with_tuple.sql diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index 03fa756276e..61e484ff6f1 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -348,7 +348,7 @@ SetPtr makeExplicitSet( const ASTPtr & left_arg = args.children.at(0); const ASTPtr & right_arg = args.children.at(1); - auto column_name = left_arg->getColumnName(context->getSettingsRef()); + auto column_name = left_arg->getColumnName(); const auto & dag_node = actions.findInIndex(column_name); const DataTypePtr & left_arg_type = dag_node.result_type; @@ -641,7 +641,7 @@ std::optional ActionsMatcher::getNameAndTypeFromAST(const ASTPt { // If the argument is a literal, we generated a unique column name for it. // Use it instead of a generic display name. - auto child_column_name = ast->getColumnName(data.getContext()->getSettingsRef()); + auto child_column_name = ast->getColumnName(); const auto * as_literal = ast->as(); if (as_literal) { @@ -698,7 +698,7 @@ ASTs ActionsMatcher::doUntuple(const ASTFunction * function, ActionsMatcher::Dat auto func = makeASTFunction("tupleElement", tuple_ast, literal); auto function_builder = FunctionFactory::instance().get(func->name, data.getContext()); - data.addFunction(function_builder, {tuple_name_type->name, literal->getColumnName(data.getContext()->getSettingsRef())}, func->getColumnName(data.getContext()->getSettingsRef())); + data.addFunction(function_builder, {tuple_name_type->name, literal->getColumnName()}, func->getColumnName()); columns.push_back(std::move(func)); } @@ -762,7 +762,7 @@ void ActionsMatcher::visit(const ASTIdentifier & identifier, const ASTPtr &, Dat void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data & data) { - auto column_name = ast->getColumnName(data.getContext()->getSettingsRef()); + auto column_name = ast->getColumnName(); if (data.hasColumn(column_name)) return; @@ -778,7 +778,7 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data & ASTPtr arg = node.arguments->children.at(0); visit(arg, data); if (!data.only_consts) - data.addArrayJoin(arg->getColumnName(data.getContext()->getSettingsRef()), column_name); + data.addArrayJoin(arg->getColumnName(), column_name); return; } @@ -800,7 +800,7 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data & /// We are in the part of the tree that we are not going to compute. You just need to define types. /// Do not subquery and create sets. We replace "in*" function to "in*IgnoreSet". - auto argument_name = node.arguments->children.at(0)->getColumnName(data.getContext()->getSettingsRef()); + auto argument_name = node.arguments->children.at(0)->getColumnName(); data.addFunction( FunctionFactory::instance().get(node.name + "IgnoreSet", data.getContext()), @@ -929,7 +929,7 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data & if (!prepared_set->empty()) column.name = data.getUniqueName("__set"); else - column.name = child->getColumnName(data.getContext()->getSettingsRef()); + column.name = child->getColumnName(); if (!data.hasColumn(column.name)) { @@ -1008,7 +1008,7 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data & visit(lambda->arguments->children.at(1), data); auto lambda_dag = data.actions_stack.popLevel(); - String result_name = lambda->arguments->children.at(1)->getColumnName(data.getContext()->getSettingsRef()); + String result_name = lambda->arguments->children.at(1)->getColumnName(); lambda_dag->removeUnusedActions(Names(1, result_name)); auto lambda_actions = std::make_shared( @@ -1023,7 +1023,7 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data & if (findColumn(required_arg, lambda_arguments) == lambda_arguments.end()) captured.push_back(required_arg); - /// We can not name `getColumnName(data.getContext()->getSettingsRef())`, + /// We can not name `getColumnName()`, /// because it does not uniquely define the expression (the types of arguments can be different). String lambda_name = data.getUniqueName("__lambda"); @@ -1053,7 +1053,7 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data & if (arguments_present) { /// Calculate column name here again, because AST may be changed here (in case of untuple). - data.addFunction(function_builder, argument_names, ast->getColumnName(data.getContext()->getSettingsRef())); + data.addFunction(function_builder, argument_names, ast->getColumnName()); } } @@ -1067,7 +1067,7 @@ void ActionsMatcher::visit(const ASTLiteral & literal, const ASTPtr & /* ast */, // AST here? Anyway, do not modify the column name if it is set already. if (literal.unique_column_name.empty()) { - const auto default_name = literal.getColumnName(data.getContext()->getSettingsRef()); + const auto default_name = literal.getColumnName(); const auto & index = data.actions_stack.getLastActionsIndex(); const auto * existing_column = index.tryGetNode(default_name); @@ -1147,7 +1147,7 @@ SetPtr ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool no_su } /// We get the stream of blocks for the subquery. Create Set and put it in place of the subquery. - String set_id = right_in_operand->getColumnName(data.getContext()->getSettingsRef()); + String set_id = right_in_operand->getColumnName(); SubqueryForSet & subquery_for_set = data.subqueries_for_sets[set_id]; @@ -1183,7 +1183,7 @@ SetPtr ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool no_su { const auto & last_actions = data.actions_stack.getLastActions(); const auto & index = data.actions_stack.getLastActionsIndex(); - if (index.contains(left_in_operand->getColumnName(data.getContext()->getSettingsRef()))) + if (index.contains(left_in_operand->getColumnName())) /// An explicit enumeration of values in parentheses. return makeExplicitSet(&node, last_actions, false, data.getContext(), data.set_size_limit, data.prepared_sets); else diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index 66c1cb9ad7b..6b55c728a96 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -243,7 +243,7 @@ void ExpressionAnalyzer::analyzeAggregation() ssize_t size = group_asts.size(); getRootActionsNoMakeSet(group_asts[i], true, temp_actions, false); - const auto & column_name = group_asts[i]->getColumnName(getContext()->getSettingsRef()); + const auto & column_name = group_asts[i]->getColumnName(); const auto * node = temp_actions->tryFindInIndex(column_name); if (!node) throw Exception("Unknown identifier (in GROUP BY): " + column_name, ErrorCodes::UNKNOWN_IDENTIFIER); @@ -408,7 +408,7 @@ void SelectQueryExpressionAnalyzer::makeSetsForIndex(const ASTPtr & node) auto temp_actions = std::make_shared(columns_after_join); getRootActions(left_in_operand, true, temp_actions); - if (temp_actions->tryFindInIndex(left_in_operand->getColumnName(getContext()->getSettingsRef()))) + if (temp_actions->tryFindInIndex(left_in_operand->getColumnName())) makeExplicitSet(func, *temp_actions, true, getContext(), settings.size_limits_for_set, prepared_sets); } } @@ -456,7 +456,7 @@ bool ExpressionAnalyzer::makeAggregateDescriptions(ActionsDAGPtr & actions) if (node->arguments) getRootActionsNoMakeSet(node->arguments, true, actions); - aggregate.column_name = node->getColumnName(getContext()->getSettingsRef()); + aggregate.column_name = node->getColumnName(); const ASTs & arguments = node->arguments ? node->arguments->children : ASTs(); aggregate.argument_names.resize(arguments.size()); @@ -464,7 +464,7 @@ bool ExpressionAnalyzer::makeAggregateDescriptions(ActionsDAGPtr & actions) for (size_t i = 0; i < arguments.size(); ++i) { - const std::string & name = arguments[i]->getColumnName(getContext()->getSettingsRef()); + const std::string & name = arguments[i]->getColumnName(); const auto * dag_node = actions->tryFindInIndex(name); if (!dag_node) { @@ -657,7 +657,7 @@ void ExpressionAnalyzer::makeWindowDescriptions(ActionsDAGPtr actions) WindowFunctionDescription window_function; window_function.function_node = function_node; window_function.column_name - = window_function.function_node->getColumnName(getContext()->getSettingsRef()); + = window_function.function_node->getColumnName(); window_function.function_parameters = window_function.function_node->parameters ? getAggregateFunctionParametersArray( @@ -676,7 +676,7 @@ void ExpressionAnalyzer::makeWindowDescriptions(ActionsDAGPtr actions) window_function.argument_names.resize(arguments.size()); for (size_t i = 0; i < arguments.size(); ++i) { - const std::string & name = arguments[i]->getColumnName(getContext()->getSettingsRef()); + const std::string & name = arguments[i]->getColumnName(); const auto * node = actions->tryFindInIndex(name); if (!node) @@ -973,7 +973,7 @@ ActionsDAGPtr SelectQueryExpressionAnalyzer::appendPrewhere( auto & step = chain.lastStep(sourceColumns()); getRootActions(select_query->prewhere(), only_types, step.actions()); - String prewhere_column_name = select_query->prewhere()->getColumnName(getContext()->getSettingsRef()); + String prewhere_column_name = select_query->prewhere()->getColumnName(); step.addRequiredOutput(prewhere_column_name); const auto & node = step.actions()->findInIndex(prewhere_column_name); @@ -1059,7 +1059,7 @@ bool SelectQueryExpressionAnalyzer::appendWhere(ExpressionActionsChain & chain, getRootActions(select_query->where(), only_types, step.actions()); - auto where_column_name = select_query->where()->getColumnName(getContext()->getSettingsRef()); + auto where_column_name = select_query->where()->getColumnName(); step.addRequiredOutput(where_column_name); const auto & node = step.actions()->findInIndex(where_column_name); @@ -1084,7 +1084,7 @@ bool SelectQueryExpressionAnalyzer::appendGroupBy(ExpressionActionsChain & chain ASTs asts = select_query->groupBy()->children; for (const auto & ast : asts) { - step.addRequiredOutput(ast->getColumnName(getContext()->getSettingsRef())); + step.addRequiredOutput(ast->getColumnName()); getRootActions(ast, only_types, step.actions()); } @@ -1112,7 +1112,7 @@ void SelectQueryExpressionAnalyzer::appendAggregateFunctionsArguments(Expression for (const auto & name : desc.argument_names) step.addRequiredOutput(name); - /// Collect aggregates removing duplicates by node.getColumnName(getContext()->getSettingsRef()) + /// Collect aggregates removing duplicates by node.getColumnName() /// It's not clear why we recollect aggregates (for query parts) while we're able to use previously collected ones (for entire query) /// @note The original recollection logic didn't remove duplicates. GetAggregatesVisitor::Data data; @@ -1167,7 +1167,7 @@ void SelectQueryExpressionAnalyzer::appendWindowFunctionsArguments( // (2b) Required function argument columns. for (const auto & a : f.function_node->arguments->children) { - step.addRequiredOutput(a->getColumnName(getContext()->getSettingsRef())); + step.addRequiredOutput(a->getColumnName()); } } @@ -1189,7 +1189,7 @@ bool SelectQueryExpressionAnalyzer::appendHaving(ExpressionActionsChain & chain, ExpressionActionsChain::Step & step = chain.lastStep(aggregated_columns); getRootActionsForHaving(select_query->having(), only_types, step.actions()); - step.addRequiredOutput(select_query->having()->getColumnName(getContext()->getSettingsRef())); + step.addRequiredOutput(select_query->having()->getColumnName()); return true; } @@ -1213,7 +1213,7 @@ void SelectQueryExpressionAnalyzer::appendSelect(ExpressionActionsChain & chain, continue; } - step.addRequiredOutput(child->getColumnName(getContext()->getSettingsRef())); + step.addRequiredOutput(child->getColumnName()); } } @@ -1241,7 +1241,7 @@ ActionsDAGPtr SelectQueryExpressionAnalyzer::appendOrderBy(ExpressionActionsChai if (!ast || ast->children.empty()) throw Exception("Bad order expression AST", ErrorCodes::UNKNOWN_TYPE_OF_AST_NODE); ASTPtr order_expression = ast->children.at(0); - step.addRequiredOutput(order_expression->getColumnName(getContext()->getSettingsRef())); + step.addRequiredOutput(order_expression->getColumnName()); if (ast->with_fill) with_fill = true; @@ -1291,7 +1291,7 @@ bool SelectQueryExpressionAnalyzer::appendLimitBy(ExpressionActionsChain & chain for (const auto & child : select_query->limitBy()->children) { - auto child_name = child->getColumnName(getContext()->getSettingsRef()); + auto child_name = child->getColumnName(); if (!aggregated_names.count(child_name)) step.addRequiredOutput(std::move(child_name)); } @@ -1307,15 +1307,13 @@ ActionsDAGPtr SelectQueryExpressionAnalyzer::appendProjectResult(ExpressionActio NamesWithAliases result_columns; - const auto & settings = getContext()->getSettingsRef(); - ASTs asts = select_query->select()->children; for (const auto & ast : asts) { - String result_name = ast->getAliasOrColumnName(settings); + String result_name = ast->getAliasOrColumnName(); if (required_result_columns.empty() || required_result_columns.count(result_name)) { - std::string source_name = ast->getColumnName(settings); + std::string source_name = ast->getColumnName(); /* * For temporary columns created by ExpressionAnalyzer for literals, @@ -1357,7 +1355,7 @@ void ExpressionAnalyzer::appendExpression(ExpressionActionsChain & chain, const { ExpressionActionsChain::Step & step = chain.lastStep(sourceColumns()); getRootActions(expr, only_types, step.actions()); - step.addRequiredOutput(expr->getColumnName(getContext()->getSettingsRef())); + step.addRequiredOutput(expr->getColumnName()); } @@ -1374,13 +1372,12 @@ ActionsDAGPtr ExpressionAnalyzer::getActionsDAG(bool add_aliases, bool project_r else asts = ASTs(1, query); - const auto & settings = getContext()->getSettingsRef(); for (const auto & ast : asts) { - std::string name = ast->getColumnName(settings); + std::string name = ast->getColumnName(); std::string alias; if (add_aliases) - alias = ast->getAliasOrColumnName(settings); + alias = ast->getAliasOrColumnName(); else alias = name; result_columns.emplace_back(name, alias); @@ -1509,7 +1506,7 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( if (auto actions = query_analyzer.appendPrewhere(chain, !first_stage, additional_required_columns_after_prewhere)) { - prewhere_info = std::make_shared(actions, query.prewhere()->getColumnName(settings)); + prewhere_info = std::make_shared(actions, query.prewhere()->getColumnName()); if (allowEarlyConstantFolding(*prewhere_info->prewhere_actions, settings)) { @@ -1519,7 +1516,7 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( ExpressionActions( prewhere_info->prewhere_actions, ExpressionActionsSettings::fromSettings(context->getSettingsRef())).execute(before_prewhere_sample); - auto & column_elem = before_prewhere_sample.getByName(query.prewhere()->getColumnName(settings)); + auto & column_elem = before_prewhere_sample.getByName(query.prewhere()->getColumnName()); /// If the filter column is a constant, record it. if (column_elem.column) prewhere_constant_filter_description = ConstantFilterDescription(*column_elem.column); @@ -1554,7 +1551,7 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( ExpressionActions( before_where, ExpressionActionsSettings::fromSettings(context->getSettingsRef())).execute(before_where_sample); - auto & column_elem = before_where_sample.getByName(query.where()->getColumnName(settings)); + auto & column_elem = before_where_sample.getByName(query.where()->getColumnName()); /// If the filter column is a constant, record it. if (column_elem.column) where_constant_filter_description = ConstantFilterDescription(*column_elem.column); @@ -1645,7 +1642,7 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( const auto * select_query = query_analyzer.getSelectQuery(); for (const auto & child : select_query->select()->children) { - step.addRequiredOutput(child->getColumnName(settings)); + step.addRequiredOutput(child->getColumnName()); } } @@ -1701,8 +1698,7 @@ void ExpressionAnalysisResult::finalize(const ExpressionActionsChain & chain, si if (hasWhere()) { - const auto & settings = chain.getContext()->getSettingsRef(); - where_column_name = query.where()->getColumnName(settings); + where_column_name = query.where()->getColumnName(); remove_where_filter = chain.steps.at(where_step_num)->required_output.find(where_column_name)->second; } } diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index c126f3bca88..d24974885b8 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -141,7 +141,7 @@ String InterpreterSelectQuery::generateFilterActions(ActionsDAGPtr & actions, co SelectQueryExpressionAnalyzer analyzer(query_ast, syntax_result, context, metadata_snapshot); actions = analyzer.simpleSelectActions(); - auto column_name = expr_list->children.at(0)->getColumnName(context->getSettingsRef()); + auto column_name = expr_list->children.at(0)->getColumnName(); actions->removeUnusedActions(NameSet{column_name}); actions->projectInput(false); @@ -782,7 +782,7 @@ static SortDescription getSortDescription(const ASTSelectQuery & query, ContextP order_descr.reserve(query.orderBy()->children.size()); for (const auto & elem : query.orderBy()->children) { - String name = elem->children.front()->getColumnName(context->getSettingsRef()); + String name = elem->children.front()->getColumnName(); const auto & order_by_elem = elem->as(); std::shared_ptr collator; @@ -801,14 +801,14 @@ static SortDescription getSortDescription(const ASTSelectQuery & query, ContextP return order_descr; } -static SortDescription getSortDescriptionFromGroupBy(const ASTSelectQuery & query, ContextPtr context) +static SortDescription getSortDescriptionFromGroupBy(const ASTSelectQuery & query) { SortDescription order_descr; order_descr.reserve(query.groupBy()->children.size()); for (const auto & elem : query.groupBy()->children) { - String name = elem->getColumnName(context->getSettingsRef()); + String name = elem->getColumnName(); order_descr.emplace_back(name, 1, 1); } @@ -1918,13 +1918,13 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc { query_info.projection->order_optimizer = std::make_shared( query_info.projection->group_by_elements_actions, - getSortDescriptionFromGroupBy(query, context), + getSortDescriptionFromGroupBy(query), query_info.syntax_analyzer_result); } else { query_info.order_optimizer = std::make_shared( - analysis_result.group_by_elements_actions, getSortDescriptionFromGroupBy(query, context), query_info.syntax_analyzer_result); + analysis_result.group_by_elements_actions, getSortDescriptionFromGroupBy(query), query_info.syntax_analyzer_result); } } @@ -2003,7 +2003,7 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc void InterpreterSelectQuery::executeWhere(QueryPlan & query_plan, const ActionsDAGPtr & expression, bool remove_filter) { auto where_step = std::make_unique( - query_plan.getCurrentDataStream(), expression, getSelectQuery().where()->getColumnName(context->getSettingsRef()), remove_filter); + query_plan.getCurrentDataStream(), expression, getSelectQuery().where()->getColumnName(), remove_filter); where_step->setStepDescription("WHERE"); query_plan.addStep(std::move(where_step)); @@ -2052,7 +2052,7 @@ void InterpreterSelectQuery::executeAggregation(QueryPlan & query_plan, const Ac SortDescription group_by_sort_description; if (group_by_info && settings.optimize_aggregation_in_order) - group_by_sort_description = getSortDescriptionFromGroupBy(getSelectQuery(), context); + group_by_sort_description = getSortDescriptionFromGroupBy(getSelectQuery()); else group_by_info = nullptr; @@ -2100,7 +2100,7 @@ void InterpreterSelectQuery::executeMergeAggregated(QueryPlan & query_plan, bool void InterpreterSelectQuery::executeHaving(QueryPlan & query_plan, const ActionsDAGPtr & expression) { auto having_step - = std::make_unique(query_plan.getCurrentDataStream(), expression, getSelectQuery().having()->getColumnName(context->getSettingsRef()), false); + = std::make_unique(query_plan.getCurrentDataStream(), expression, getSelectQuery().having()->getColumnName(), false); having_step->setStepDescription("HAVING"); query_plan.addStep(std::move(having_step)); @@ -2116,7 +2116,7 @@ void InterpreterSelectQuery::executeTotalsAndHaving( query_plan.getCurrentDataStream(), overflow_row, expression, - has_having ? getSelectQuery().having()->getColumnName(context->getSettingsRef()) : "", + has_having ? getSelectQuery().having()->getColumnName() : "", settings.totals_mode, settings.totals_auto_threshold, final); @@ -2435,7 +2435,7 @@ void InterpreterSelectQuery::executeLimitBy(QueryPlan & query_plan) Names columns; for (const auto & elem : query.limitBy()->children) - columns.emplace_back(elem->getColumnName(context->getSettingsRef())); + columns.emplace_back(elem->getColumnName()); UInt64 length = getLimitUIntValue(query.limitByLength(), context, "LIMIT"); UInt64 offset = (query.limitByOffset() ? getLimitUIntValue(query.limitByOffset(), context, "OFFSET") : 0); diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index cc345004f6f..a2b64d26d42 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -609,6 +609,27 @@ std::vector getWindowFunctions(ASTPtr & query, const ASTSel return data.window_functions; } +class MarkTupleLiteralsAsLegacyData +{ +public: + using TypeToVisit = ASTLiteral; + + void visit(ASTLiteral & literal, ASTPtr &) + { + if (literal.value.getType() == Field::Types::Tuple) + literal.use_legacy_column_name_of_tuple = true; + } +}; + +using MarkTupleLiteralsAsLegacyMatcher = OneTypeMatcher; +using MarkTupleLiteralsAsLegacyVisitor = InDepthNodeVisitor; + +void markTupleLiteralsAsLegacy(ASTPtr & query) +{ + MarkTupleLiteralsAsLegacyVisitor::Data data; + MarkTupleLiteralsAsLegacyVisitor(data).visit(query); +} + } TreeRewriterResult::TreeRewriterResult( @@ -927,6 +948,9 @@ TreeRewriterResultPtr TreeRewriter::analyzeSelect( /// Executing scalar subqueries - replacing them with constant values. executeScalarSubqueries(query, getContext(), subquery_depth, result.scalars, select_options.only_analyze); + if (settings.legacy_column_name_of_tuple_literal) + markTupleLiteralsAsLegacy(query); + TreeOptimizer::apply(query, result, tables_with_columns, getContext()); /// array_join_alias_to_name, array_join_result_to_source. @@ -994,6 +1018,9 @@ TreeRewriterResultPtr TreeRewriter::analyze( /// Executing scalar subqueries. Column defaults could be a scalar subquery. executeScalarSubqueries(query, getContext(), 0, result.scalars, false); + if (settings.legacy_column_name_of_tuple_literal) + markTupleLiteralsAsLegacy(query); + TreeOptimizer::optimizeIf(query, result.aliases, settings.optimize_if_chain_to_multiif); if (allow_aggregations) diff --git a/src/Interpreters/evaluateConstantExpression.cpp b/src/Interpreters/evaluateConstantExpression.cpp index f814e1d8c02..e46f644e836 100644 --- a/src/Interpreters/evaluateConstantExpression.cpp +++ b/src/Interpreters/evaluateConstantExpression.cpp @@ -39,7 +39,7 @@ std::pair> evaluateConstantExpression(co if (context->getSettingsRef().normalize_function_names) FunctionNameNormalizer().visit(ast.get()); - String name = ast->getColumnName(context->getSettingsRef()); + String name = ast->getColumnName(); auto syntax_result = TreeRewriter(context).analyze(ast, source_columns); ExpressionActionsPtr expr_for_constant_folding = ExpressionAnalyzer(ast, syntax_result, context).getConstActions(); diff --git a/src/Parsers/ASTFunction.cpp b/src/Parsers/ASTFunction.cpp index daae3e76aa1..1ff27c61836 100644 --- a/src/Parsers/ASTFunction.cpp +++ b/src/Parsers/ASTFunction.cpp @@ -24,16 +24,6 @@ namespace ErrorCodes } void ASTFunction::appendColumnNameImpl(WriteBuffer & ostr) const -{ - appendColumnNameImpl(ostr, nullptr); -} - -void ASTFunction::appendColumnNameImpl(WriteBuffer & ostr, const Settings & settings) const -{ - appendColumnNameImpl(ostr, &settings); -} - -void ASTFunction::appendColumnNameImpl(WriteBuffer & ostr, const Settings * settings) const { if (name == "view") throw Exception("Table function view cannot be used as an expression", ErrorCodes::UNEXPECTED_EXPRESSION); @@ -48,10 +38,7 @@ void ASTFunction::appendColumnNameImpl(WriteBuffer & ostr, const Settings * sett if (it != parameters->children.begin()) writeCString(", ", ostr); - if (settings) - (*it)->appendColumnName(ostr, *settings); - else - (*it)->appendColumnName(ostr); + (*it)->appendColumnName(ostr); } writeChar(')', ostr); } @@ -64,10 +51,7 @@ void ASTFunction::appendColumnNameImpl(WriteBuffer & ostr, const Settings * sett if (it != arguments->children.begin()) writeCString(", ", ostr); - if (settings) - (*it)->appendColumnName(ostr, *settings); - else - (*it)->appendColumnName(ostr); + (*it)->appendColumnName(ostr); } } diff --git a/src/Parsers/ASTFunction.h b/src/Parsers/ASTFunction.h index 8e657afbf6e..685aaaadd26 100644 --- a/src/Parsers/ASTFunction.h +++ b/src/Parsers/ASTFunction.h @@ -54,10 +54,6 @@ public: protected: void formatImplWithoutAlias(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override; void appendColumnNameImpl(WriteBuffer & ostr) const override; - void appendColumnNameImpl(WriteBuffer & ostr, const Settings & settings) const override; - -private: - void appendColumnNameImpl(WriteBuffer & ostr, const Settings * settings) const; }; diff --git a/src/Parsers/ASTLiteral.cpp b/src/Parsers/ASTLiteral.cpp index c456cb3e933..93d490bc645 100644 --- a/src/Parsers/ASTLiteral.cpp +++ b/src/Parsers/ASTLiteral.cpp @@ -50,16 +50,14 @@ String FieldVisitorToColumnName::operator() (const Tuple & x) const } -void ASTLiteral::appendColumnNameImpl(WriteBuffer & ostr, const Settings & settings) const -{ - if (settings.legacy_column_name_of_tuple_literal) - appendColumnNameImplLegacy(ostr); - else - appendColumnNameImpl(ostr); -} - void ASTLiteral::appendColumnNameImpl(WriteBuffer & ostr) const { + if (use_legacy_column_name_of_tuple) + { + appendColumnNameImplLegacy(ostr); + return; + } + /// 100 - just arbitrary value. constexpr auto min_elements_for_hashing = 100; diff --git a/src/Parsers/ASTLiteral.h b/src/Parsers/ASTLiteral.h index c17310f719b..856bed61979 100644 --- a/src/Parsers/ASTLiteral.h +++ b/src/Parsers/ASTLiteral.h @@ -33,6 +33,10 @@ public: */ String unique_column_name; + /// For compatibility reasons in distributed queries, + /// we may need to use legacy column name for tuple literal. + bool use_legacy_column_name_of_tuple = false; + /** Get the text that identifies this element. */ String getID(char delim) const override { return "Literal" + (delim + applyVisitor(FieldVisitorDump(), value)); } @@ -44,7 +48,6 @@ protected: void formatImplWithoutAlias(const FormatSettings & settings, FormatState &, FormatStateStacked) const override; void appendColumnNameImpl(WriteBuffer & ostr) const override; - void appendColumnNameImpl(WriteBuffer & ostr, const Settings & settings) const override; private: /// Legacy version of 'appendColumnNameImpl'. It differs only with tuple literals. diff --git a/src/Parsers/ASTWithAlias.cpp b/src/Parsers/ASTWithAlias.cpp index 0f5b86763e0..88f6568a719 100644 --- a/src/Parsers/ASTWithAlias.cpp +++ b/src/Parsers/ASTWithAlias.cpp @@ -48,14 +48,6 @@ void ASTWithAlias::appendColumnName(WriteBuffer & ostr) const appendColumnNameImpl(ostr); } -void ASTWithAlias::appendColumnName(WriteBuffer & ostr, const Settings & settings) const -{ - if (prefer_alias_to_column_name && !alias.empty()) - writeString(alias, ostr); - else - appendColumnNameImpl(ostr, settings); -} - void ASTWithAlias::appendColumnNameWithoutAlias(WriteBuffer & ostr) const { appendColumnNameImpl(ostr); diff --git a/src/Parsers/ASTWithAlias.h b/src/Parsers/ASTWithAlias.h index 249be17b74c..ea4419402b0 100644 --- a/src/Parsers/ASTWithAlias.h +++ b/src/Parsers/ASTWithAlias.h @@ -21,10 +21,8 @@ public: using IAST::IAST; void appendColumnName(WriteBuffer & ostr) const final; - void appendColumnName(WriteBuffer & ostr, const Settings & settings) const final; void appendColumnNameWithoutAlias(WriteBuffer & ostr) const final; String getAliasOrColumnName() const override { return alias.empty() ? getColumnName() : alias; } - String getAliasOrColumnName(const Settings & settings) const override { return alias.empty() ? getColumnName(settings) : alias; } String tryGetAlias() const override { return alias; } void setAlias(const String & to) override { alias = to; } @@ -35,7 +33,6 @@ public: protected: virtual void appendColumnNameImpl(WriteBuffer & ostr) const = 0; - virtual void appendColumnNameImpl(WriteBuffer & ostr, const Settings &) const { appendColumnNameImpl(ostr); } }; /// helper for setting aliases and chaining result to other functions diff --git a/src/Parsers/IAST.cpp b/src/Parsers/IAST.cpp index 0f38fcf98dd..3a21d704eb9 100644 --- a/src/Parsers/IAST.cpp +++ b/src/Parsers/IAST.cpp @@ -109,14 +109,6 @@ String IAST::getColumnName() const } -String IAST::getColumnName(const Settings & settings) const -{ - WriteBufferFromOwnString write_buffer; - appendColumnName(write_buffer, settings); - return write_buffer.str(); -} - - String IAST::getColumnNameWithoutAlias() const { WriteBufferFromOwnString write_buffer; diff --git a/src/Parsers/IAST.h b/src/Parsers/IAST.h index 143094e1d7a..f805ae7d8eb 100644 --- a/src/Parsers/IAST.h +++ b/src/Parsers/IAST.h @@ -42,7 +42,6 @@ public: /** Get the canonical name of the column if the element is a column */ String getColumnName() const; - String getColumnName(const Settings & settings) const; /** Same as the above but ensure no alias names are used. This is for index analysis */ String getColumnNameWithoutAlias() const; @@ -52,8 +51,6 @@ public: throw Exception("Trying to get name of not a column: " + getID(), ErrorCodes::LOGICAL_ERROR); } - virtual void appendColumnName(WriteBuffer & ostr, const Settings &) const { appendColumnName(ostr); } - virtual void appendColumnNameWithoutAlias(WriteBuffer &) const { throw Exception("Trying to get name of not a column: " + getID(), ErrorCodes::LOGICAL_ERROR); @@ -61,7 +58,6 @@ public: /** Get the alias, if any, or the canonical name of the column, if it is not. */ virtual String getAliasOrColumnName() const { return getColumnName(); } - virtual String getAliasOrColumnName(const Settings & settings) const { return getColumnName(settings); } /** Get the alias, if any, or an empty string if it does not exist, or if the element does not support aliases. */ virtual String tryGetAlias() const { return String(); } diff --git a/tests/queries/0_stateless/02002_system_table_with_tuple.reference b/tests/queries/0_stateless/02002_system_table_with_tuple.reference new file mode 100644 index 00000000000..d00491fd7e5 --- /dev/null +++ b/tests/queries/0_stateless/02002_system_table_with_tuple.reference @@ -0,0 +1 @@ +1 diff --git a/tests/queries/0_stateless/02002_system_table_with_tuple.sql b/tests/queries/0_stateless/02002_system_table_with_tuple.sql new file mode 100644 index 00000000000..42796e98110 --- /dev/null +++ b/tests/queries/0_stateless/02002_system_table_with_tuple.sql @@ -0,0 +1,2 @@ +SELECT count() > 0 FROM system.tables WHERE database IN ('system', 'system') +SETTINGS legacy_column_name_of_tuple_literal = 1; From 612f42e301f8cf01d16f4a564f65859234d978fb Mon Sep 17 00:00:00 2001 From: Dmitriy Date: Tue, 3 Aug 2021 21:09:58 +0300 Subject: [PATCH 118/599] Add the brief description MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Добавил краткое описание, параметры движка. --- .../materialized-postgresql.md | 1 - .../integrations/materialized-postgresql.md | 20 ++++++++++++++++--- .../materialized-postgresql.md | 1 - .../integrations/materialized-postgresql.md | 19 +++++++++++++++--- 4 files changed, 33 insertions(+), 8 deletions(-) diff --git a/docs/en/engines/database-engines/materialized-postgresql.md b/docs/en/engines/database-engines/materialized-postgresql.md index ec34d5fbd34..a5e83140651 100644 --- a/docs/en/engines/database-engines/materialized-postgresql.md +++ b/docs/en/engines/database-engines/materialized-postgresql.md @@ -9,7 +9,6 @@ Creates ClickHouse database with all the tables existing in PostgreSQL, and all ClickHouse server works as PostgreSQL replica. It reads WAL and performs DML queries. DDL is not replicated, but can be handled (described below). - ## Creating a Database {#creating-a-database} ``` sql diff --git a/docs/en/engines/table-engines/integrations/materialized-postgresql.md b/docs/en/engines/table-engines/integrations/materialized-postgresql.md index 1639be791e9..c5dc807e95b 100644 --- a/docs/en/engines/table-engines/integrations/materialized-postgresql.md +++ b/docs/en/engines/table-engines/integrations/materialized-postgresql.md @@ -5,6 +5,10 @@ toc_title: MaterializedPostgreSQL # MaterializedPostgreSQL {#materialize-postgresql} +Creates ClickHouse table with an initial data dump of PostgreSQL table and starts replication process, i.e. executes background job to apply new changes as they happen on PostgreSQL table in the remote PostgreSQL database. + +If more than one table is required, it is highly recommended to use the [MaterializedPostgreSQL](../../engines/database-engines/materialized-postgresql.md) database engine instead of the table engine and use the [materialized_postgresql_tables_list](../../operations/settings/settings.md#materialized-postgresql-tables-list) setting, which specifies the tables to be replicated. It will be much better in terms of CPU, fewer connections and fewer replication slots inside the remote PostgreSQL database. + ## Creating a Table {#creating-a-table} ``` sql @@ -13,19 +17,29 @@ ENGINE = MaterializedPostgreSQL('postgres1:5432', 'postgres_database', 'postgres PRIMARY KEY key; ``` +**Engine Parameters** + +- `host:port` — PostgreSQL server address. +- `database` — Remote database name. +- `table` — Remote table name. +- `user` — PostgreSQL user. +- `password` — User password. + ## Requirements {#requirements} 1. The [wal_level](https://www.postgresql.org/docs/current/runtime-config-wal.html) setting must have a value `logical` and `max_replication_slots` parameter must have a value at least `2` in the PostgreSQL config file. -2. A table with `MaterializedPostgreSQL` engine must have a primary key — the same as a replica identity index (by default: primary key) of a PostgreSQL table (see [details on replica identity index](../../database-engines/materialized-postgresql.md#requirements)). +2. A table with `MaterializedPostgreSQL` engine must have a primary key — the same as a replica identity index (by default: primary key) of a PostgreSQL table (see [details on replica identity index](../../engines/database-engines/materialized-postgresql.md#requirements)). 3. Only database [Atomic](https://en.wikipedia.org/wiki/Atomicity_(database_systems)) is allowed. ## Virtual columns {#virtual-columns} -- `_version` (type: UInt64) +- `_version` — Transaction counter. Type: [UInt64](../../sql-reference/data-types/int-uint.md). -- `_sign` (type: Int8) +- `_sign` — Deletion mark. Type: [Int8](../../sql-reference/data-types/int-uint.md). Possible values: + - `1` — Row is not deleted, + - `-1` — Row is deleted. These columns do not need to be added when a table is created. They are always accessible in `SELECT` query. `_version` column equals `LSN` position in `WAL`, so it might be used to check how up-to-date replication is. diff --git a/docs/ru/engines/database-engines/materialized-postgresql.md b/docs/ru/engines/database-engines/materialized-postgresql.md index 064b231c2f1..8ac4c9d3e38 100644 --- a/docs/ru/engines/database-engines/materialized-postgresql.md +++ b/docs/ru/engines/database-engines/materialized-postgresql.md @@ -9,7 +9,6 @@ toc_title: MaterializedPostgreSQL Сервер ClickHouse работает как реплика PostgreSQL. Он читает файл `binlog` и выполняет DDL и DML-запросы. - ## Создание базы данных {#creating-a-database} ``` sql diff --git a/docs/ru/engines/table-engines/integrations/materialized-postgresql.md b/docs/ru/engines/table-engines/integrations/materialized-postgresql.md index aaa0a7af1d8..e5fb4dcb213 100644 --- a/docs/ru/engines/table-engines/integrations/materialized-postgresql.md +++ b/docs/ru/engines/table-engines/integrations/materialized-postgresql.md @@ -5,6 +5,10 @@ toc_title: MaterializedPostgreSQL # MaterializedPostgreSQL {#materialize-postgresql} +Создает таблицу ClickHouse с исходным дампом данных таблицы PostgreSQL и запускает процесс репликации, т.е. выполняется применение новых изменений в фоне, как эти изменения происходят в таблице PostgreSQL в удаленной базе данных PostgreSQL. + +Если требуется более одной таблицы, то очень рекомендуется использовать движок баз данных [MaterializedPostgreSQL](../../engines/database-engines/materialized-postgresql.md) вместо движка таблиц и использовать настройку [materialized_postgresql_tables_list](../../operations/settings/settings.md#materialized-postgresql-tables-list), с помощью которой указать таблицы, которые нужно реплицировать. Это будет намного лучше с точки зрения нагрузки на процессор, меньшего количества подключений и меньшего количества слотов репликации внутри удаленной базы данных PostgreSQL. + ## Создание таблицы {#creating-a-table} ``` sql @@ -13,6 +17,14 @@ ENGINE = MaterializedPostgreSQL('postgres1:5432', 'postgres_database', 'postgres PRIMARY KEY key; ``` +**Параметры движка** + +- `host:port` — адрес сервера PostgreSQL. +- `database` — имя базы данных на удалённом сервере. +- `table` — имя таблицы на удалённом сервере. +- `user` — пользователь PostgreSQL. +- `password` — пароль пользователя. + ## Требования {#requirements} 1. Настройка [wal_level](https://postgrespro.ru/docs/postgrespro/10/runtime-config-wal) должна иметь значение `logical`, параметр `max_replication_slots` должен быть равен по меньшей мере `2` в конфигурационном файле в PostgreSQL. @@ -23,9 +35,10 @@ PRIMARY KEY key; ## Виртуальные столбцы {#virtual-columns} -- `_version` (тип: UInt64) - -- `_sign` (тип: Int8) +- `_version` — счетчик транзакций. Тип: [UInt64](../../sql-reference/data-types/int-uint.md). +- `_sign` — метка удаления. Тип: [Int8](../../sql-reference/data-types/int-uint.md). Возможные значения: + - `1` — строка не удалена, + - `-1` — строка удалена. Эти столбцы не нужно добавлять при создании таблицы. Они всегда доступны в `SELECT` запросе. Столбец `_version` равен позиции `LSN` в `WAL`, поэтому его можно использовать для проверки актуальности репликации. From ad4d05d7ca9377bdf9772f5b3fb26afdd3b8af5f Mon Sep 17 00:00:00 2001 From: Dmitriy Date: Tue, 3 Aug 2021 21:44:30 +0300 Subject: [PATCH 119/599] Fix links MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Поправил ссылки. --- .../table-engines/integrations/materialized-postgresql.md | 8 ++++---- .../table-engines/integrations/materialized-postgresql.md | 8 ++++---- 2 files changed, 8 insertions(+), 8 deletions(-) diff --git a/docs/en/engines/table-engines/integrations/materialized-postgresql.md b/docs/en/engines/table-engines/integrations/materialized-postgresql.md index c5dc807e95b..d02a11257c2 100644 --- a/docs/en/engines/table-engines/integrations/materialized-postgresql.md +++ b/docs/en/engines/table-engines/integrations/materialized-postgresql.md @@ -7,7 +7,7 @@ toc_title: MaterializedPostgreSQL Creates ClickHouse table with an initial data dump of PostgreSQL table and starts replication process, i.e. executes background job to apply new changes as they happen on PostgreSQL table in the remote PostgreSQL database. -If more than one table is required, it is highly recommended to use the [MaterializedPostgreSQL](../../engines/database-engines/materialized-postgresql.md) database engine instead of the table engine and use the [materialized_postgresql_tables_list](../../operations/settings/settings.md#materialized-postgresql-tables-list) setting, which specifies the tables to be replicated. It will be much better in terms of CPU, fewer connections and fewer replication slots inside the remote PostgreSQL database. +If more than one table is required, it is highly recommended to use the [MaterializedPostgreSQL](../../../engines/database-engines/materialized-postgresql.md) database engine instead of the table engine and use the [materialized_postgresql_tables_list](../../../operations/settings/settings.md#materialized-postgresql-tables-list) setting, which specifies the tables to be replicated. It will be much better in terms of CPU, fewer connections and fewer replication slots inside the remote PostgreSQL database. ## Creating a Table {#creating-a-table} @@ -29,15 +29,15 @@ PRIMARY KEY key; 1. The [wal_level](https://www.postgresql.org/docs/current/runtime-config-wal.html) setting must have a value `logical` and `max_replication_slots` parameter must have a value at least `2` in the PostgreSQL config file. -2. A table with `MaterializedPostgreSQL` engine must have a primary key — the same as a replica identity index (by default: primary key) of a PostgreSQL table (see [details on replica identity index](../../engines/database-engines/materialized-postgresql.md#requirements)). +2. A table with `MaterializedPostgreSQL` engine must have a primary key — the same as a replica identity index (by default: primary key) of a PostgreSQL table (see [details on replica identity index](../../../engines/database-engines/materialized-postgresql.md#requirements)). 3. Only database [Atomic](https://en.wikipedia.org/wiki/Atomicity_(database_systems)) is allowed. ## Virtual columns {#virtual-columns} -- `_version` — Transaction counter. Type: [UInt64](../../sql-reference/data-types/int-uint.md). +- `_version` — Transaction counter. Type: [UInt64](../../../sql-reference/data-types/int-uint.md). -- `_sign` — Deletion mark. Type: [Int8](../../sql-reference/data-types/int-uint.md). Possible values: +- `_sign` — Deletion mark. Type: [Int8](../../../sql-reference/data-types/int-uint.md). Possible values: - `1` — Row is not deleted, - `-1` — Row is deleted. diff --git a/docs/ru/engines/table-engines/integrations/materialized-postgresql.md b/docs/ru/engines/table-engines/integrations/materialized-postgresql.md index e5fb4dcb213..6d599bd8d86 100644 --- a/docs/ru/engines/table-engines/integrations/materialized-postgresql.md +++ b/docs/ru/engines/table-engines/integrations/materialized-postgresql.md @@ -7,7 +7,7 @@ toc_title: MaterializedPostgreSQL Создает таблицу ClickHouse с исходным дампом данных таблицы PostgreSQL и запускает процесс репликации, т.е. выполняется применение новых изменений в фоне, как эти изменения происходят в таблице PostgreSQL в удаленной базе данных PostgreSQL. -Если требуется более одной таблицы, то очень рекомендуется использовать движок баз данных [MaterializedPostgreSQL](../../engines/database-engines/materialized-postgresql.md) вместо движка таблиц и использовать настройку [materialized_postgresql_tables_list](../../operations/settings/settings.md#materialized-postgresql-tables-list), с помощью которой указать таблицы, которые нужно реплицировать. Это будет намного лучше с точки зрения нагрузки на процессор, меньшего количества подключений и меньшего количества слотов репликации внутри удаленной базы данных PostgreSQL. +Если требуется более одной таблицы, то очень рекомендуется использовать движок баз данных [MaterializedPostgreSQL](../../../engines/database-engines/materialized-postgresql.md) вместо движка таблиц и использовать настройку [materialized_postgresql_tables_list](../../../operations/settings/settings.md#materialized-postgresql-tables-list), с помощью которой указать таблицы, которые нужно реплицировать. Это будет намного лучше с точки зрения нагрузки на процессор, меньшего количества подключений и меньшего количества слотов репликации внутри удаленной базы данных PostgreSQL. ## Создание таблицы {#creating-a-table} @@ -29,14 +29,14 @@ PRIMARY KEY key; 1. Настройка [wal_level](https://postgrespro.ru/docs/postgrespro/10/runtime-config-wal) должна иметь значение `logical`, параметр `max_replication_slots` должен быть равен по меньшей мере `2` в конфигурационном файле в PostgreSQL. -2. Таблица, созданная с помощью движка `MaterializedPostgreSQL`, должна иметь первичный ключ — такой же, как индекс идентичности реплики (по умолчанию: первичный ключ) таблицы PostgreSQL (смотрите [индекс идентичности реплики](../../database-engines/materialized-postgresql.md#requirements)). +2. Таблица, созданная с помощью движка `MaterializedPostgreSQL`, должна иметь первичный ключ — такой же, как индекс идентичности реплики (по умолчанию: первичный ключ) таблицы PostgreSQL (смотрите [индекс идентичности реплики](../../../engines/database-engines/materialized-postgresql.md#requirements)). 3. Допускается только база данных [Atomic](https://en.wikipedia.org/wiki/Atomicity_(database_systems)). ## Виртуальные столбцы {#virtual-columns} -- `_version` — счетчик транзакций. Тип: [UInt64](../../sql-reference/data-types/int-uint.md). -- `_sign` — метка удаления. Тип: [Int8](../../sql-reference/data-types/int-uint.md). Возможные значения: +- `_version` — счетчик транзакций. Тип: [UInt64](../../../sql-reference/data-types/int-uint.md). +- `_sign` — метка удаления. Тип: [Int8](../../../sql-reference/data-types/int-uint.md). Возможные значения: - `1` — строка не удалена, - `-1` — строка удалена. From 7849308b9891c812a7769ca98f0881e88e1c9010 Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Tue, 3 Aug 2021 21:58:32 +0300 Subject: [PATCH 120/599] Add string pad functions. --- .../functions/string-functions.md | 82 ++++++++++++++++++- 1 file changed, 81 insertions(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index 8ec8aa7339d..3c4ab2cd99b 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -43,13 +43,93 @@ The result type is UInt64. Returns the length of a string in Unicode code points (not in characters), assuming that the string contains a set of bytes that make up UTF-8 encoded text. If this assumption is not met, it returns some result (it does not throw an exception). The result type is UInt64. +## leftPad {#leftpad} + +Pads the current string from the left with a specified string (multiple times, if needed) until the resulting string reaches the given length. + +**Syntax** + +``` sql +leftPad('string','length', 'string2`) +``` + +**Arguments** + +- `string` — Input string, that need to be padded. [String](../data-types/string.md). +- `length` — The length of the resulting string once the input string pads. [UInt](../data-types/int-uint.md). If the value is less than input string length, then string is returned as-is. +- `string2` — The string to pad the current input string with. [String](../data-types/string.md). + +[String](../data-types/string.md) + +**Returned value(s)** + +- The resulting string reaches the given length. + +Type: [String](../data-types/string.md). + +**Example** + +Query: + +``` sql +SELECT leftPad('abc', 7, '*'); +``` + +Result: + +``` text +┌─leftPad('abc', 7, '*')─┐ +│ ****abc │ +└────────────────────────┘ +``` + +## rightPad {#rightpad} + +Pads the current string from the right with a specified string (multiple times, if needed) until the resulting string reaches the given length. + +**Syntax** + +``` sql +leftPad('string','length', 'string2`) +``` + +**Arguments** + +- `string` — Input string, that need to be padded. [String](../data-types/string.md). +- `length` — The length of the resulting string once the input string pads. [UInt](../data-types/int-uint.md). If the value is less than input string length, then string is returned as-is. +- `string2` — The string to pad the current input string with. [String](../data-types/string.md). + +[String](../data-types/string.md) + +**Returned value(s)** + +- The resulting string reaches the given length. + +Type: [String](../data-types/string.md). + +**Example** + +Query: + +``` sql +SELECT rightPad('abc', 7, '*'); +``` + +Result: + +``` text +┌─rightPad('abc', 7, '*')─┐ +│ abc**** │ +└─────────────────────────┘ +``` + ## lower, lcase {#lower} Converts ASCII Latin symbols in a string to lowercase. ## upper, ucase {#upper} -Converts ASCII Latin symbols in a string to uppercase. +Converts ASCII Latin symbols in a string to uppercase.pd ## lowerUTF8 {#lowerutf8} From 2b6ffab384f00b42c5bbaf7451437c795dba022b Mon Sep 17 00:00:00 2001 From: lehasm Date: Tue, 3 Aug 2021 23:03:13 +0300 Subject: [PATCH 121/599] Update docs/en/sql-reference/statements/detach.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/en/sql-reference/statements/detach.md | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/docs/en/sql-reference/statements/detach.md b/docs/en/sql-reference/statements/detach.md index 4260c914962..4c8ce355eb4 100644 --- a/docs/en/sql-reference/statements/detach.md +++ b/docs/en/sql-reference/statements/detach.md @@ -5,7 +5,7 @@ toc_title: DETACH # DETACH Statement {#detach} -Makes the server "forget" about the existence of a table, a materialized view or a dictionary. +Makes the server "forget" about the existence of a table, a materialized view, or a dictionary. **Syntax** @@ -72,4 +72,3 @@ Code: 60. DB::Exception: Received from localhost:9000. DB::Exception: Table defa - [Materialized View](../../sql-reference/statements/create/view.md#materialized) - [Dictionaries](../../sql-reference/dictionaries/index.md) - From 45e98e20c1307546b385048752bbae324947e449 Mon Sep 17 00:00:00 2001 From: lehasm Date: Tue, 3 Aug 2021 23:04:57 +0300 Subject: [PATCH 122/599] Update docs/en/sql-reference/statements/detach.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/en/sql-reference/statements/detach.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/statements/detach.md b/docs/en/sql-reference/statements/detach.md index 4c8ce355eb4..b77bcbc00fb 100644 --- a/docs/en/sql-reference/statements/detach.md +++ b/docs/en/sql-reference/statements/detach.md @@ -13,7 +13,7 @@ Makes the server "forget" about the existence of a table, a materialized view, o DETACH TABLE|VIEW|DICTIONARY [IF EXISTS] [db.]name [ON CLUSTER cluster] [PERMANENTLY] ``` -Detaching does not delete the data or metadata of the table, the materialized view or the dictionary. If the entity was not detached `PERMANENTLY`, on the next server launch the server will read the metadata and recall the table/view/dictionary again. If the entity was detached `PERMANENTLY`, there will be no automatic recall. +Detaching does not delete the data or metadata of a table, a materialized view or a dictionary. If an entity was not detached `PERMANENTLY`, on the next server launch the server will read the metadata and recall the table/view/dictionary again. If an entity was detached `PERMANENTLY`, there will be no automatic recall. Whether a table or a dictionary was detached permanently or not, in both cases you can reattach them using the [ATTACH](../../sql-reference/statements/attach.md) query. System log tables can be also attached back (e.g. `query_log`, `text_log`, etc). Other system tables can't be reattached. On the next server launch the server will recall those tables again. From 1ad5606c52bd48863f2d3c0232814be63c063439 Mon Sep 17 00:00:00 2001 From: lehasm Date: Tue, 3 Aug 2021 23:05:32 +0300 Subject: [PATCH 123/599] Update docs/en/sql-reference/statements/exchange.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/en/sql-reference/statements/exchange.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/statements/exchange.md b/docs/en/sql-reference/statements/exchange.md index 7abc86330ba..91b0c48ddcf 100644 --- a/docs/en/sql-reference/statements/exchange.md +++ b/docs/en/sql-reference/statements/exchange.md @@ -6,7 +6,7 @@ toc_title: EXCHANGE # EXCHANGE Statement {#exchange} Exchanges the names of two tables or dictionaries atomically. -This task can also be accomplished with a [RENAME](./rename.md) query using a temporary name. But the operation in not atomic in that case. +This task can also be accomplished with a [RENAME](./rename.md) query using a temporary name, but the operation is not atomic in that case. !!! note "Note" The `EXCHANGE` query is supported by the [Atomic](../../engines/database-engines/atomic.md) database engine only. From 90a682df133e353759833c4ffa93e666800130dd Mon Sep 17 00:00:00 2001 From: lehasm Date: Tue, 3 Aug 2021 23:05:48 +0300 Subject: [PATCH 124/599] Update docs/en/sql-reference/statements/rename.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/en/sql-reference/statements/rename.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/statements/rename.md b/docs/en/sql-reference/statements/rename.md index a2c4a305dce..54df3520b37 100644 --- a/docs/en/sql-reference/statements/rename.md +++ b/docs/en/sql-reference/statements/rename.md @@ -5,7 +5,7 @@ toc_title: RENAME # RENAME Statement {#misc_operations-rename} -Renames databases, tables or dictionaries. Several entities can be renamed in a single query. +Renames databases, tables, or dictionaries. Several entities can be renamed in a single query. Note that the `RENAME` query with several entities is non-atomic operation. To swap entity names atomically, use [EXCHANGE](./exchange.md) statement. !!! note "Note" @@ -58,4 +58,4 @@ RENAME DICTIONARY [db0.]dict_A TO [db1.]dict_B [,...] [ON CLUSTER cluster] **See Also** -- [Dictionaries](../../sql-reference/dictionaries/index.md) \ No newline at end of file +- [Dictionaries](../../sql-reference/dictionaries/index.md) From b073d2b88c2544f555bf3a7b472fe3bd4758c5ed Mon Sep 17 00:00:00 2001 From: lehasm Date: Tue, 3 Aug 2021 23:06:39 +0300 Subject: [PATCH 125/599] Update docs/en/sql-reference/statements/rename.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/en/sql-reference/statements/rename.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/statements/rename.md b/docs/en/sql-reference/statements/rename.md index 54df3520b37..c2192f1a6e1 100644 --- a/docs/en/sql-reference/statements/rename.md +++ b/docs/en/sql-reference/statements/rename.md @@ -6,7 +6,7 @@ toc_title: RENAME # RENAME Statement {#misc_operations-rename} Renames databases, tables, or dictionaries. Several entities can be renamed in a single query. -Note that the `RENAME` query with several entities is non-atomic operation. To swap entity names atomically, use [EXCHANGE](./exchange.md) statement. +Note that the `RENAME` query with several entities is non-atomic operation. To swap entities names atomically, use the [EXCHANGE](./exchange.md) statement. !!! note "Note" The `RENAME` query is supported by the [Atomic](../../engines/database-engines/atomic.md) database engine only. From c7b520aabc77015a2a55d245793e8dc4ffb5dbda Mon Sep 17 00:00:00 2001 From: lehasm Date: Tue, 3 Aug 2021 23:07:12 +0300 Subject: [PATCH 126/599] Update docs/en/sql-reference/statements/attach.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/en/sql-reference/statements/attach.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/statements/attach.md b/docs/en/sql-reference/statements/attach.md index 63e4b87b9f8..766b3ba0e19 100644 --- a/docs/en/sql-reference/statements/attach.md +++ b/docs/en/sql-reference/statements/attach.md @@ -31,7 +31,7 @@ If the table was detached permanently, it won't be reattached at the server star ## Create New Table And Attach Data {#create-new-table-and-attach-data} -**With specify path to table data** +### With Specified Path to Table Data {#attach-with-specified-path} **Syntax** From 42c5cd80d94ffd9aa51abf9a15be184dfb89cc33 Mon Sep 17 00:00:00 2001 From: Alexey Date: Tue, 3 Aug 2021 20:47:09 +0000 Subject: [PATCH 127/599] Yet another header. Other changes. --- docs/en/sql-reference/statements/attach.md | 15 ++++++++++----- 1 file changed, 10 insertions(+), 5 deletions(-) diff --git a/docs/en/sql-reference/statements/attach.md b/docs/en/sql-reference/statements/attach.md index 766b3ba0e19..2949ac6db38 100644 --- a/docs/en/sql-reference/statements/attach.md +++ b/docs/en/sql-reference/statements/attach.md @@ -33,14 +33,14 @@ If the table was detached permanently, it won't be reattached at the server star ### With Specified Path to Table Data {#attach-with-specified-path} +The query creates a new table with provided structure and attaches table data from the provided directory in `user_files`. + **Syntax** ```sql ATTACH TABLE name FROM 'path/to/data/' (col1 Type1, ...) ``` -It creates new table with provided structure and attaches table data from provided directory in `user_files`. - **Example** Query: @@ -59,16 +59,21 @@ Result: └──────┴────┘ ``` -**With specify table UUID** (Only for `Atomic` database) +### With Specified Table UUID {#attach-with-specified-uuid} + +This query creates a new table with provided structure and attaches data from the table with the specified UUID. +It is supported by the [Atomic](../../engines/database-engines/atomic.md) database engine. + +**Syntax** ```sql ATTACH TABLE name UUID '' (col1 Type1, ...) ``` -It creates new table with provided structure and attaches data from table with the specified UUID. - ## Attach Existing Dictionary {#attach-existing-dictionary} +Attaches a previously detached dictionary. + **Syntax** ``` sql From 7673247b4ea22dbc73f6020c578ecd872d6c5d0e Mon Sep 17 00:00:00 2001 From: olgarev Date: Wed, 4 Aug 2021 05:19:31 +0000 Subject: [PATCH 128/599] Quotes and link fixed for s3, section headings for hdfs --- docs/en/engines/table-engines/integrations/hdfs.md | 2 +- docs/en/engines/table-engines/integrations/s3.md | 12 ++++++------ docs/ru/engines/table-engines/integrations/hdfs.md | 4 ++-- docs/ru/engines/table-engines/integrations/s3.md | 2 +- 4 files changed, 10 insertions(+), 10 deletions(-) diff --git a/docs/en/engines/table-engines/integrations/hdfs.md b/docs/en/engines/table-engines/integrations/hdfs.md index 83ccdc01bb0..948dd7b10ef 100644 --- a/docs/en/engines/table-engines/integrations/hdfs.md +++ b/docs/en/engines/table-engines/integrations/hdfs.md @@ -185,7 +185,7 @@ Similar to GraphiteMergeTree, the HDFS engine supports extended configuration us |hadoop\_kerberos\_principal | "" | |hadoop\_kerberos\_kinit\_command | kinit | -#### Limitations {#limitations} +### Limitations {#limitations} * hadoop\_security\_kerberos\_ticket\_cache\_path can be global only, not user specific ## Kerberos support {#kerberos-support} diff --git a/docs/en/engines/table-engines/integrations/s3.md b/docs/en/engines/table-engines/integrations/s3.md index 6516265d254..e2aa7dd6bc7 100644 --- a/docs/en/engines/table-engines/integrations/s3.md +++ b/docs/en/engines/table-engines/integrations/s3.md @@ -77,12 +77,12 @@ Constructions with `{}` are similar to the [remote](../../../sql-reference/table 1. Suppose we have several files in CSV format with the following URIs on S3: -- ‘https://storage.yandexcloud.net/my-test-bucket-768/some_prefix/some_file_1.csv’ -- ‘https://storage.yandexcloud.net/my-test-bucket-768/some_prefix/some_file_2.csv’ -- ‘https://storage.yandexcloud.net/my-test-bucket-768/some_prefix/some_file_3.csv’ -- ‘https://storage.yandexcloud.net/my-test-bucket-768/another_prefix/some_file_1.csv’ -- ‘https://storage.yandexcloud.net/my-test-bucket-768/another_prefix/some_file_2.csv’ -- ‘https://storage.yandexcloud.net/my-test-bucket-768/another_prefix/some_file_3.csv’ +- 'https://storage.yandexcloud.net/my-test-bucket-768/some_prefix/some_file_1.csv' +- 'https://storage.yandexcloud.net/my-test-bucket-768/some_prefix/some_file_2.csv' +- 'https://storage.yandexcloud.net/my-test-bucket-768/some_prefix/some_file_3.csv' +- 'https://storage.yandexcloud.net/my-test-bucket-768/another_prefix/some_file_1.csv' +- 'https://storage.yandexcloud.net/my-test-bucket-768/another_prefix/some_file_2.csv' +- 'https://storage.yandexcloud.net/my-test-bucket-768/another_prefix/some_file_3.csv' There are several ways to make a table consisting of all six files: diff --git a/docs/ru/engines/table-engines/integrations/hdfs.md b/docs/ru/engines/table-engines/integrations/hdfs.md index 79a2d22a840..30cbff987cc 100644 --- a/docs/ru/engines/table-engines/integrations/hdfs.md +++ b/docs/ru/engines/table-engines/integrations/hdfs.md @@ -122,7 +122,7 @@ CREATE TABLE big_table (name String, value UInt32) ENGINE = HDFS('hdfs://hdfs1:9 ``` -### Возможные опции конфигурации {#configuration-options} +### Параметры конфигурации {#configuration-options} #### Поддерживаемые из libhdfs3 {#supported-by-libhdfs3} @@ -181,7 +181,7 @@ CREATE TABLE big_table (name String, value UInt32) ENGINE = HDFS('hdfs://hdfs1:9 |hadoop\_kerberos\_principal | "" | |hadoop\_kerberos\_kinit\_command | kinit | -#### Ограничения {#limitations} +### Ограничения {#limitations} * hadoop\_security\_kerberos\_ticket\_cache\_path могут быть определены только на глобальном уровне ## Поддержка Kerberos {#kerberos-support} diff --git a/docs/ru/engines/table-engines/integrations/s3.md b/docs/ru/engines/table-engines/integrations/s3.md index 1b20f1f947d..53053aa8695 100644 --- a/docs/ru/engines/table-engines/integrations/s3.md +++ b/docs/ru/engines/table-engines/integrations/s3.md @@ -72,7 +72,7 @@ SELECT * FROM s3_engine_table LIMIT 2; - `s3_max_redirects` — максимальное количество разрешенных переадресаций S3. Значение по умолчанию — `10`. - `s3_single_read_retries` — максимальное количество попыток запроса при единичном чтении. Значение по умолчанию — `4`. -Соображение безопасности: если злонамеренный пользователь попробует указать произвольные URL-адреса S3, параметр `s3_max_redirects` должен быть установлен в ноль, чтобы избежать атак [SSRF] (https://en.wikipedia.org/wiki/Server-side_request_forgery). Как альтернатива, в конфигурации сервера должен быть указан `remote_host_filter`. +Соображение безопасности: если злонамеренный пользователь попробует указать произвольные URL-адреса S3, параметр `s3_max_redirects` должен быть установлен в ноль, чтобы избежать атак [SSRF](https://en.wikipedia.org/wiki/Server-side_request_forgery). Как альтернатива, в конфигурации сервера должен быть указан `remote_host_filter`. ## Настройки точки приема запроса {#endpoint-settings} From fbf65fb1c4cce3dc58cb439c01b706daf23df68e Mon Sep 17 00:00:00 2001 From: olgarev <56617294+olgarev@users.noreply.github.com> Date: Wed, 4 Aug 2021 08:21:06 +0300 Subject: [PATCH 129/599] Syntax --- docs/ru/engines/table-engines/integrations/s3.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/engines/table-engines/integrations/s3.md b/docs/ru/engines/table-engines/integrations/s3.md index 53053aa8695..3934e98e5ba 100644 --- a/docs/ru/engines/table-engines/integrations/s3.md +++ b/docs/ru/engines/table-engines/integrations/s3.md @@ -47,7 +47,7 @@ SELECT * FROM s3_engine_table LIMIT 2; ## Детали реализации {#implementation-details} - Чтение и запись могут быть параллельными. -- Поддерживается репиликация без копирования данных (zero-copy): если данные хранятся на нескольких удаленных машинах, то при синхронизации пересылаются только метаданные (пути к кускам данных), а сами данные не копируются. +- Поддерживается репликация без копирования данных (zero-copy): если данные хранятся на нескольких удаленных машинах, то при синхронизации пересылаются только метаданные (пути к кускам данных), а сами данные не копируются. - Не поддерживаются: - запросы `ALTER` и `SELECT...SAMPLE`, - индексы. From 91c71ebb14215cea731edee06cf9e97c986d64b8 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Wed, 4 Aug 2021 09:29:23 +0300 Subject: [PATCH 130/599] Update docs/en/operations/settings/settings.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- 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 ad71ca3bd96..b931e9c6998 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -117,7 +117,7 @@ Possible values: - Positive integer. -Default value: `500000000`. +Default value: `500,000,000`. **See Also** From feb1de449ca450f40192031a2cf618115f5746af Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Wed, 4 Aug 2021 09:29:39 +0300 Subject: [PATCH 131/599] Update docs/ru/operations/settings/settings.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/operations/settings/settings.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index 4e60ae88270..2335eed3fda 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -87,7 +87,7 @@ ClickHouse применяет настройку в тех случаях, ко ## function_range_max_elements_in_block {#settings-function_range_max_elements_in_block} -Устанавливает порог безопасности для объема данных, создаваемого функцией [range](../../sql-reference/functions/array-functions.md#range). Задаётся максимальное количество значений, генерируемых функцией на блок данных (сумма размеров массива для каждой строки в блоке). +Устанавливает порог безопасности для объема данных, создаваемого функцией [range](../../sql-reference/functions/array-functions.md#range). Задаёт максимальное количество значений, генерируемых функцией на блок данных (сумма размеров массивов для каждой строки в блоке). Возможные значения: From 47816275e7d3515c1cc2f2b56553259c2f350557 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Wed, 4 Aug 2021 09:30:37 +0300 Subject: [PATCH 132/599] Update docs/ru/operations/settings/settings.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/operations/settings/settings.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index 2335eed3fda..70807447785 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -93,7 +93,7 @@ ClickHouse применяет настройку в тех случаях, ко - Положительное целое. -Значение по умолчанию: `500000000`. +Значение по умолчанию: `500 000 000`. **См. также** From 46f153a3d2ccb770bb61541f4c03ebb90c08ca60 Mon Sep 17 00:00:00 2001 From: Alexey Date: Wed, 4 Aug 2021 06:46:20 +0000 Subject: [PATCH 133/599] Translation to Russian --- docs/ru/sql-reference/statements/attach.md | 44 ++++++++++++----- docs/ru/sql-reference/statements/detach.md | 12 +++-- docs/ru/sql-reference/statements/exchange.md | 42 ++++++++++++++++ docs/ru/sql-reference/statements/rename.md | 50 +++++++++++++++++--- 4 files changed, 127 insertions(+), 21 deletions(-) create mode 100644 docs/ru/sql-reference/statements/exchange.md diff --git a/docs/ru/sql-reference/statements/attach.md b/docs/ru/sql-reference/statements/attach.md index 2ffd0fe8d5b..d998f41d629 100644 --- a/docs/ru/sql-reference/statements/attach.md +++ b/docs/ru/sql-reference/statements/attach.md @@ -3,16 +3,23 @@ toc_priority: 40 toc_title: ATTACH --- -# ATTACH Statement {#attach} +# Выражение ATTACH {#attach} -Выполняет подключение таблицы, например, при перемещении базы данных на другой сервер. +Выполняет подключение таблицы или словаря, например, при перемещении базы данных на другой сервер. -Запрос не создаёт данные на диске, а предполагает, что данные уже лежат в соответствующих местах, и всего лишь добавляет информацию о таблице на сервер. После выполнения запроса `ATTACH` сервер будет знать о существовании таблицы. +**Синтаксис** + +``` sql +ATTACH TABLE|DICTIONARY [IF NOT EXISTS] [db.]name [ON CLUSTER cluster] ... +``` + +Запрос не создаёт данные на диске, а предполагает, что данные уже лежат в соответствующих местах, и всего лишь добавляет информацию о таблице или словаре на сервер. После выполнения запроса `ATTACH` сервер будет знать о существовании таблицы или словаря. Если таблица перед этим была отключена при помощи ([DETACH](../../sql-reference/statements/detach.md)), т.е. её структура известна, можно использовать сокращенную форму записи без определения структуры. -## Варианты синтаксиса {#syntax-forms} -### Присоединение существующей таблицы {#attach-existing-table} +## Присоединение существующей таблицы {#attach-existing-table} + +**Синтаксис** ``` sql ATTACH TABLE [IF NOT EXISTS] [db.]name [ON CLUSTER cluster] @@ -22,16 +29,18 @@ ATTACH TABLE [IF NOT EXISTS] [db.]name [ON CLUSTER cluster] Если таблица была отключена перманентно, она не будет подключена обратно во время старта сервера, так что нужно явно использовать запрос `ATTACH`, чтобы подключить ее. -### Создание новой таблицы и присоединение данных {#create-new-table-and-attach-data} +## Создание новой таблицы и присоединение данных {#create-new-table-and-attach-data} -**С указанием пути к табличным данным** +### С указанием пути к табличным данным {#attach-with-specified-path} + +Запрос создает новую таблицу с указанной структурой и присоединяет табличные данные из соответствующего каталога в `user_files`. + +**Синтаксис** ```sql ATTACH TABLE name FROM 'path/to/data/' (col1 Type1, ...) ``` -Cоздает новую таблицу с указанной структурой и присоединяет табличные данные из соответствующего каталога в `user_files`. - **Пример** Запрос: @@ -50,10 +59,23 @@ SELECT * FROM test; └──────┴────┘ ``` -**С указанием UUID таблицы** (Только для баз данных `Atomic`) +### С указанием UUID таблицы {#attach-with-specified-uuid} + +Этот запрос создает новую таблицу с указанной структурой и присоединяет данные из таблицы с указанным UUID. +Запрос поддерживается только движком баз данных [Atomic](../../engines/database-engines/atomic.md). + +**Синтаксис** ```sql ATTACH TABLE name UUID '' (col1 Type1, ...) ``` -Cоздает новую таблицу с указанной структурой и присоединяет данные из таблицы с указанным UUID. +## Присоединение существующего словаря {#attach-existing-dictionary} + +Присоединяет ранее отключенный словарь. + +**Синтаксис** + +``` sql +ATTACH DICTIONARY [IF NOT EXISTS] [db.]name [ON CLUSTER cluster] +``` diff --git a/docs/ru/sql-reference/statements/detach.md b/docs/ru/sql-reference/statements/detach.md index af915d38772..cf9df619d3b 100644 --- a/docs/ru/sql-reference/statements/detach.md +++ b/docs/ru/sql-reference/statements/detach.md @@ -5,15 +5,15 @@ toc_title: DETACH # DETACH {#detach-statement} -Заставляет сервер "забыть" о существовании таблицы или материализованного представления. +Заставляет сервер "забыть" о существовании таблицы, материализованного представления или словаря. -Синтаксис: +**Синтаксис** ``` sql -DETACH TABLE|VIEW [IF EXISTS] [db.]name [ON CLUSTER cluster] [PERMANENTLY] +DETACH TABLE|VIEW|DICTIONARY [IF EXISTS] [db.]name [ON CLUSTER cluster] [PERMANENTLY] ``` -Но ни данные, ни метаданные таблицы или материализованного представления не удаляются. При следующем запуске сервера, если не было использовано `PERMANENTLY`, сервер прочитает метаданные и снова узнает о таблице/представлении. Если таблица или представление были отключены перманентно, сервер не подключит их обратно автоматически. +Такой запрос не удаляет ни данные, ни метаданные таблицы, материализованного представления или словаря. Если отключение не было перманентным (запрос без ключевого слова `PERMANENTLY`), то при следующем запуске сервер прочитает метаданные и снова узнает о таблице/представлении/словаре. Если сущность была отключена перманентно, то сервер не подключит их обратно автоматически. Независимо от того, каким способом таблица была отключена, ее можно подключить обратно с помощью запроса [ATTACH](../../sql-reference/statements/attach.md). Системные log таблицы также могут быть подключены обратно (к примеру `query_log`, `text_log` и др.) Другие системные таблицы не могут быть подключены обратно, но на следующем запуске сервер снова "вспомнит" об этих таблицах. @@ -67,3 +67,7 @@ Received exception from server (version 21.4.1): Code: 60. DB::Exception: Received from localhost:9000. DB::Exception: Table default.test doesn't exist. ``` +**Смотрите также** + +- [Материализованные представления](../../sql-reference/statements/create/view.md#materialized) +- [Словари](../../sql-reference/dictionaries/index.md) diff --git a/docs/ru/sql-reference/statements/exchange.md b/docs/ru/sql-reference/statements/exchange.md new file mode 100644 index 00000000000..eb66aee62c8 --- /dev/null +++ b/docs/ru/sql-reference/statements/exchange.md @@ -0,0 +1,42 @@ +--- +toc_priority: 49 +toc_title: EXCHANGE +--- + +# Выражение EXCHANGE {#exchange} + +Атомарно меняет друг с другом имена двух таблиц или словарей. +Это действие также можно выполнить с помощью запроса [RENAME](./rename.md), использую третье временное имя, но в таком случае действие неатомарно. + +!!! note "Примечание" + Запрос `EXCHANGE` поддерживается только движком баз данных [Atomic](../../engines/database-engines/atomic.md). + +**Синтаксис** + +```sql +EXCHANGE TABLES|DICTIONARIES [db0.]name_A AND [db1.]name_B +``` + +## EXCHANGE TABLES {#exchange_tables} + +Обменивает имена двух таблиц. + +**Синтаксис** + +```sql +EXCHANGE TABLES [db0.]table_A AND [db1.]table_B +``` + +## EXCHANGE DICTIONARIES {#exchange_dictionaries} + +Обменивает имена двух словарей. + +**Синтаксис** + +```sql +EXCHANGE DICTIONARIES [db0.]dict_A AND [db1.]dict_B +``` + +**Смотрите также** + +- [Словари](../../sql-reference/dictionaries/index.md) diff --git a/docs/ru/sql-reference/statements/rename.md b/docs/ru/sql-reference/statements/rename.md index b78505ce9c4..b043ec305e0 100644 --- a/docs/ru/sql-reference/statements/rename.md +++ b/docs/ru/sql-reference/statements/rename.md @@ -3,20 +3,58 @@ toc_priority: 48 toc_title: RENAME --- -# RENAME Statement {#misc_operations-rename} +# Выражение RENAME {#misc_operations-rename} + +Переименовывает базы данных, таблицы или словари. Несколько сущностей могут быть переименованы в одном запросе. +Обратите внимание, что запрос `RENAME` с несколькими сущностями это не атомарная операция. Чтобы обменять имена атомарно, используйте выражение [EXCHANGE](./exchange.md). + +!!! note "Примечание" + Запрос `RENAME` поддерживается только движком баз данных [Atomic](../../engines/database-engines/atomic.md). + +**Синтаксис** + +```sql +RENAME DATABASE|TABLE|DICTIONARY name TO new_name [,...] [ON CLUSTER cluster] +``` ## RENAME DATABASE {#misc_operations-rename_database} -Переименовывает базу данных, поддерживается только для движка базы данных Atomic. -``` -RENAME DATABASE atomic_database1 TO atomic_database2 [ON CLUSTER cluster] +Переименовывает базы данных. + +**Синтаксис** + +```sql +RENAME DATABASE atomic_database1 TO atomic_database2 [,...] [ON CLUSTER cluster] ``` ## RENAME TABLE {#misc_operations-rename_table} + Переименовывает одну или несколько таблиц. +Переименовывание таблиц является лёгкой операцией. Если вы указали после `TO` другую базу данных, то таблица будет перенесена в эту базу данных. При этом директории с базами данных должны быть расположены в одной файловой системе. Иначе возвращается ошибка. Если переименовывается несколько таблиц в одном запросе, то такая операция неатомарная. Она может выполнится частично, и запросы в других сессиях могут получить ошибку `Table ... doesn't exist...`. + +**Синтаксис** + ``` sql -RENAME TABLE [db11.]name11 TO [db12.]name12, [db21.]name21 TO [db22.]name22, ... [ON CLUSTER cluster] +RENAME TABLE [db1.]name1 TO [db2.]name2 [,...] [ON CLUSTER cluster] ``` -Переименовывание таблицы является лёгкой операцией. Если вы указали после `TO` другую базу данных, то таблица будет перенесена в эту базу данных. При этом, директории с базами данных должны быть расположены в одной файловой системе (иначе возвращается ошибка). В случае переименования нескольких таблиц в одном запросе — это неатомарная операция, может выполнится частично, запросы в других сессиях могут получить ошибку `Table ... doesn't exist...`. +**Пример** + +```sql +RENAME TABLE table_A TO table_A_bak, table_B TO table_B_bak; +``` + +## RENAME DICTIONARY {#rename_dictionary} + +Переименовывает один или несколько словарей. Этот запрос можно использовать для перемещения словарей между базами данных. + +**Синтаксис** + +```sql +RENAME DICTIONARY [db0.]dict_A TO [db1.]dict_B [,...] [ON CLUSTER cluster] +``` + +**Смотрите также** + +- [Словари](../../sql-reference/dictionaries/index.md) From 6e74728714ebf71c29d93e742651b62c44a48e4f Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Wed, 4 Aug 2021 10:58:39 +0300 Subject: [PATCH 134/599] more sync fixes --- programs/server/Server.cpp | 2 +- src/Compression/CompressionCodecEncrypted.cpp | 4 ++-- src/Compression/CompressionCodecEncrypted.h | 4 ++-- src/Functions/stem.cpp | 2 +- src/Interpreters/Lemmatizers.cpp | 4 ++-- src/Interpreters/SynonymsExtensions.cpp | 2 +- src/Processors/ya.make | 4 ++-- src/Processors/ya.make.in | 6 ++++++ 8 files changed, 17 insertions(+), 11 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 86bb04351b1..a8abb439b62 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -97,7 +97,7 @@ #endif #if USE_SSL -# if USE_INTERNAL_SSL_LIBRARY +# if USE_INTERNAL_SSL_LIBRARY && !defined(ARCADIA_BUILD) # include # endif # include diff --git a/src/Compression/CompressionCodecEncrypted.cpp b/src/Compression/CompressionCodecEncrypted.cpp index d0904b4bf24..5dfdd6f4fcb 100644 --- a/src/Compression/CompressionCodecEncrypted.cpp +++ b/src/Compression/CompressionCodecEncrypted.cpp @@ -5,9 +5,9 @@ #include #include #include -#include +#include // Y_IGNORE #include -#include +#include // Y_IGNORE #include namespace DB diff --git a/src/Compression/CompressionCodecEncrypted.h b/src/Compression/CompressionCodecEncrypted.h index e58fd4ab173..bacd58bcd2f 100644 --- a/src/Compression/CompressionCodecEncrypted.h +++ b/src/Compression/CompressionCodecEncrypted.h @@ -2,11 +2,11 @@ // This depends on BoringSSL-specific API, notably . #include -#if USE_SSL && USE_INTERNAL_SSL_LIBRARY +#if USE_SSL && USE_INTERNAL_SSL_LIBRARY && !defined(ARCADIA_BUILD) #include #include -#include +#include // Y_IGNORE #include namespace DB diff --git a/src/Functions/stem.cpp b/src/Functions/stem.cpp index 98dcbccd005..7092bac06ec 100644 --- a/src/Functions/stem.cpp +++ b/src/Functions/stem.cpp @@ -11,7 +11,7 @@ #include #include -#include +#include // Y_IGNORE namespace DB diff --git a/src/Interpreters/Lemmatizers.cpp b/src/Interpreters/Lemmatizers.cpp index 38cd4c33678..78af43285ef 100644 --- a/src/Interpreters/Lemmatizers.cpp +++ b/src/Interpreters/Lemmatizers.cpp @@ -6,8 +6,8 @@ #if USE_NLP #include -#include -#include +#include // Y_IGNORE +#include // Y_IGNORE #include #include diff --git a/src/Interpreters/SynonymsExtensions.cpp b/src/Interpreters/SynonymsExtensions.cpp index 22fa91a4349..6147fa14674 100644 --- a/src/Interpreters/SynonymsExtensions.cpp +++ b/src/Interpreters/SynonymsExtensions.cpp @@ -11,7 +11,7 @@ #include #include -#include +#include // Y_IGNORE namespace DB { diff --git a/src/Processors/ya.make b/src/Processors/ya.make index 4b95484a828..97b730dd8a4 100644 --- a/src/Processors/ya.make +++ b/src/Processors/ya.make @@ -7,11 +7,11 @@ PEERDIR( clickhouse/src/Common contrib/libs/msgpack contrib/libs/protobuf - contrib/libs/arrow + contrib/libs/apache/arrow ) ADDINCL( - contrib/libs/arrow/src + contrib/libs/apache/arrow/src ) CFLAGS(-DUSE_ARROW=1) diff --git a/src/Processors/ya.make.in b/src/Processors/ya.make.in index 06230b96be8..d4b98acf01c 100644 --- a/src/Processors/ya.make.in +++ b/src/Processors/ya.make.in @@ -6,8 +6,14 @@ PEERDIR( clickhouse/src/Common contrib/libs/msgpack contrib/libs/protobuf + contrib/libs/apache/arrow ) +ADDINCL( + contrib/libs/apache/arrow/src +) + +CFLAGS(-DUSE_ARROW=1) SRCS( From 9539ef18d2768a6843ece53895c25685536fbbfd Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Wed, 4 Aug 2021 11:55:51 +0300 Subject: [PATCH 135/599] fix --- src/Compression/CompressionCodecEncrypted.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Compression/CompressionCodecEncrypted.cpp b/src/Compression/CompressionCodecEncrypted.cpp index 5dfdd6f4fcb..6b921fb9c0a 100644 --- a/src/Compression/CompressionCodecEncrypted.cpp +++ b/src/Compression/CompressionCodecEncrypted.cpp @@ -1,4 +1,6 @@ -#include +#if !defined(ARCADIA_BUILD) +# include +#endif #include #if USE_SSL && USE_INTERNAL_SSL_LIBRARY From b613331a2593b3e0992817d910dd38014b2c8d05 Mon Sep 17 00:00:00 2001 From: adevyatova Date: Wed, 4 Aug 2021 12:31:24 +0300 Subject: [PATCH 136/599] Add desc for currentRoles, enabledRoles, defaultRoles func --- .../functions/other-functions.md | 144 +++--------------- .../functions/other-functions.md | 47 ++++++ 2 files changed, 65 insertions(+), 126 deletions(-) diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index 63892bc712c..a8129d3f794 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -2138,158 +2138,50 @@ Result: - [tcp_port](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-tcp_port) -## currentProfiles {#current-profiles} +## currentRoles {#current-roles} -Short description. +Returns the names of the roles which are current for the current user. The command [SET ROLE](../../sql-reference/statements/set-role.md#set-role-statement) could be used to change the current roles, by default the current roles are returned by the function [defaultRoles](#default-roles) called when the current user logins. **Syntax** ``` sql - +currentRoles() ``` -Alias: ``. (Optional) +**Returned value** -More text (Optional). +- List of the current roles for the current user. -**Arguments** (Optional) +Type: [Array](../../sql-reference/data-types/array.md)([String](../../sql-reference/data-types/string.md)). -- `x` — Description. Optional (only for optional arguments). Possible values: . Default value: . [Type name](relative/path/to/type/dscr.md#type). -- `y` — Description. Optional (only for optional arguments). Possible values: .Default value: . [Type name](relative/path/to/type/dscr.md#type). +## enabledRoles {#enabled-roles} -**Parameters** (Optional, only for parametric aggregate functions) - -- `z` — Description. Optional (only for optional parameters). Possible values: . Default value: . [Type name](relative/path/to/type/dscr.md#type). - -**Returned value(s)** - -- Returned values list. - -Type: [Type name](relative/path/to/type/dscr.md#type). - -**Example** - -The example must show usage and/or a use cases. The following text contains recommended parts of an example. - -Input table (Optional): - -``` text -``` - -Query: - -``` sql -``` - -Result: - -``` text -``` - -**See Also** (Optional) - -- [link](#) - -## enabledProfiles {#enabled-profiles} - -Short description. +Returns the names of the current roles merged with those roles which are granted to some of the current roles. **Syntax** ``` sql - +enabledRoles() ``` -Alias: ``. (Optional) +**Returned value** -More text (Optional). +- List of the enabled roles for the current user. -**Arguments** (Optional) +Type: [Array](../../sql-reference/data-types/array.md)([String](../../sql-reference/data-types/string.md)). -- `x` — Description. Optional (only for optional arguments). Possible values: . Default value: . [Type name](relative/path/to/type/dscr.md#type). -- `y` — Description. Optional (only for optional arguments). Possible values: .Default value: . [Type name](relative/path/to/type/dscr.md#type). +## defaultRoles {#default-roles} -**Parameters** (Optional, only for parametric aggregate functions) - -- `z` — Description. Optional (only for optional parameters). Possible values: . Default value: . [Type name](relative/path/to/type/dscr.md#type). - -**Returned value(s)** - -- Returned values list. - -Type: [Type name](relative/path/to/type/dscr.md#type). - -**Example** - -The example must show usage and/or a use cases. The following text contains recommended parts of an example. - -Input table (Optional): - -``` text -``` - -Query: - -``` sql -``` - -Result: - -``` text -``` - -**See Also** (Optional) - -- [link](#) - -## defaultProfiles {#default-profiles} - -Short description. +Returns the names of the roles which are set as current when the user logins. The command [SET DEFAULT ROLE](../../sql-reference/statements/set-role.md#set-default-role-statement) could be used to change the default roles. By default are all the roles which are granted to a user. **Syntax** ``` sql - +defaultRoles() ``` -Alias: ``. (Optional) +**Returned value** -More text (Optional). +- List of the default roles for the current user. -**Arguments** (Optional) - -- `x` — Description. Optional (only for optional arguments). Possible values: . Default value: . [Type name](relative/path/to/type/dscr.md#type). -- `y` — Description. Optional (only for optional arguments). Possible values: .Default value: . [Type name](relative/path/to/type/dscr.md#type). - -**Parameters** (Optional, only for parametric aggregate functions) - -- `z` — Description. Optional (only for optional parameters). Possible values: . Default value: . [Type name](relative/path/to/type/dscr.md#type). - -**Returned value(s)** - -- Returned values list. - -Type: [Type name](relative/path/to/type/dscr.md#type). - -**Example** - -The example must show usage and/or a use cases. The following text contains recommended parts of an example. - -Input table (Optional): - -``` text -``` - -Query: - -``` sql -``` - -Result: - -``` text -``` - -**See Also** (Optional) - -- [link](#) \ No newline at end of file +Type: [Array](../../sql-reference/data-types/array.md)([String](../../sql-reference/data-types/string.md)). diff --git a/docs/ru/sql-reference/functions/other-functions.md b/docs/ru/sql-reference/functions/other-functions.md index a07bd19faa1..9e08b357b0f 100644 --- a/docs/ru/sql-reference/functions/other-functions.md +++ b/docs/ru/sql-reference/functions/other-functions.md @@ -2088,3 +2088,50 @@ SELECT tcpPort(); - [tcp_port](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-tcp_port) +## currentRoles {#current-roles} + +Возвращает имена текущих ролей для текущего пользователя. Для изменения текущих ролей может использоваться команда [SET ROLE](../../sql-reference/statements/set-role.md#set-role-statement). Текущие роли по умолчанию можно получить при вызове функции [defaultRoles](#default-roles), когда пользователь входит в систему. + +**Синтаксис** + +``` sql +currentRoles() +``` + +**Возвращаемое значение** + +- Список текущих ролей для текущего пользователя. + +Тип: [Array](../../sql-reference/data-types/array.md)([String](../../sql-reference/data-types/string.md)). + +## enabledRoles {#enabled-roles} + +Возвращает имена текущих ролей вместе с теми ролями, которые предоставлены путем назначения привилегий. + +**Синтаксис** + +``` sql +enabledRoles() +``` + +**Возвращаемое значение** + +- Список доступных ролей для текущего пользователя. + +Тип: [Array](../../sql-reference/data-types/array.md)([String](../../sql-reference/data-types/string.md)). + +## defaultRoles {#default-roles} + +Возвращает имена ролей, которые задаются как текущие для данного пользователя, когда он входит в систему. Они могут быть изменены при помощи команды [SET DEFAULT ROLE](../../sql-reference/statements/set-role.md#set-default-role-statement), но по умолчанию это все роли, которые предоставляются пользователю. + +**Синтаксис** + +``` sql +defaultRoles() +``` + +**Возвращаемое значение** + +- Список ролей по умолчанию. + +Тип: [Array](../../sql-reference/data-types/array.md)([String](../../sql-reference/data-types/string.md)). From 82721d593e2eb0f8ce5d09fad37bff8770603808 Mon Sep 17 00:00:00 2001 From: adevyatova Date: Wed, 4 Aug 2021 12:38:23 +0300 Subject: [PATCH 137/599] Small fixes --- docs/en/engines/database-engines/materialized-mysql.md | 7 +++---- docs/ru/engines/database-engines/materialized-mysql.md | 5 +---- 2 files changed, 4 insertions(+), 8 deletions(-) diff --git a/docs/en/engines/database-engines/materialized-mysql.md b/docs/en/engines/database-engines/materialized-mysql.md index 20e16473115..62e58c7876b 100644 --- a/docs/en/engines/database-engines/materialized-mysql.md +++ b/docs/en/engines/database-engines/materialized-mysql.md @@ -3,16 +3,15 @@ toc_priority: 29 toc_title: MaterializedMySQL --- -# MaterializedMySQL {#materialized-mysql} +# [experimental] MaterializedMySQL {#materialized-mysql} -**This is experimental feature that should not be used in production.** +!!! warning "Warning" + This is an experimental feature that should not be used in production. Creates ClickHouse database with all the tables existing in MySQL, and all the data in those tables. ClickHouse server works as MySQL replica. It reads binlog and performs DDL and DML queries. -This feature is experimental. - ## Creating a Database {#creating-a-database} ``` sql diff --git a/docs/ru/engines/database-engines/materialized-mysql.md b/docs/ru/engines/database-engines/materialized-mysql.md index f5f0166c9dc..0175e794cd5 100644 --- a/docs/ru/engines/database-engines/materialized-mysql.md +++ b/docs/ru/engines/database-engines/materialized-mysql.md @@ -1,17 +1,14 @@ - --- toc_priority: 29 toc_title: MaterializedMySQL --- -# MaterializedMySQL {#materialized-mysql} +# [экспериментальный] MaterializedMySQL {#materialized-mysql} Создает базу данных ClickHouse со всеми таблицами, существующими в MySQL, и всеми данными в этих таблицах. Сервер ClickHouse работает как реплика MySQL. Он читает файл binlog и выполняет DDL and DML-запросы. -`MaterializedMySQL` — экспериментальный движок баз данных. - ## Создание базы данных {#creating-a-database} ``` sql From 55e06ea16b5b4f4b7366b6d5e4ec53aba7714275 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Wed, 4 Aug 2021 13:06:01 +0300 Subject: [PATCH 138/599] fix --- src/Storages/MergeTree/IMergeTreeReader.cpp | 2 +- src/Storages/StorageInMemoryMetadata.cpp | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/IMergeTreeReader.cpp b/src/Storages/MergeTree/IMergeTreeReader.cpp index 4efd3d669eb..5378b84a5d0 100644 --- a/src/Storages/MergeTree/IMergeTreeReader.cpp +++ b/src/Storages/MergeTree/IMergeTreeReader.cpp @@ -50,7 +50,7 @@ IMergeTreeReader::IMergeTreeReader( columns_from_part.set_empty_key(StringRef()); for (const auto & column_from_part : part_columns) - columns_from_part.emplace(column_from_part.name, &column_from_part.type); + columns_from_part[column_from_part.name] = &column_from_part.type; } IMergeTreeReader::~IMergeTreeReader() = default; diff --git a/src/Storages/StorageInMemoryMetadata.cpp b/src/Storages/StorageInMemoryMetadata.cpp index dad83f64c70..7b1a126b1ab 100644 --- a/src/Storages/StorageInMemoryMetadata.cpp +++ b/src/Storages/StorageInMemoryMetadata.cpp @@ -331,7 +331,7 @@ Block StorageInMemoryMetadata::getSampleBlockForColumns( /// Virtual columns must be appended after ordinary, because user can /// override them. for (const auto & column : virtuals) - virtuals_map.emplace(column.name, &column.type); + virtuals_map[column.name] = &column.type; for (const auto & name : column_names) { From 48c14fae6fcd917246a860cc3db01309c431421f Mon Sep 17 00:00:00 2001 From: adevyatova Date: Wed, 4 Aug 2021 15:11:42 +0300 Subject: [PATCH 139/599] Applied suggestions from code review --- docs/en/sql-reference/functions/other-functions.md | 4 ++-- docs/ru/sql-reference/functions/other-functions.md | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index a8129d3f794..446c7e6714c 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -2140,7 +2140,7 @@ Result: ## currentRoles {#current-roles} -Returns the names of the roles which are current for the current user. The command [SET ROLE](../../sql-reference/statements/set-role.md#set-role-statement) could be used to change the current roles, by default the current roles are returned by the function [defaultRoles](#default-roles) called when the current user logins. +Returns the names of the roles which are current for the current user. The command [SET ROLE](../../sql-reference/statements/set-role.md#set-role-statement) could be used to change the current roles. If the command `SET ROLE` hasn't been used the function `currentRoles` returns the same as `defaultRoles`. **Syntax** @@ -2172,7 +2172,7 @@ Type: [Array](../../sql-reference/data-types/array.md)([String](../../sql-refere ## defaultRoles {#default-roles} -Returns the names of the roles which are set as current when the user logins. The command [SET DEFAULT ROLE](../../sql-reference/statements/set-role.md#set-default-role-statement) could be used to change the default roles. By default are all the roles which are granted to a user. +Returns the names of the roles which are set as current when the current user logins. The command [SET DEFAULT ROLE](../../sql-reference/statements/set-role.md#set-default-role-statement) could be used to change the default roles. By default the default roles are all the roles granted to the current user. **Syntax** diff --git a/docs/ru/sql-reference/functions/other-functions.md b/docs/ru/sql-reference/functions/other-functions.md index 9e08b357b0f..fdd9ff44fd9 100644 --- a/docs/ru/sql-reference/functions/other-functions.md +++ b/docs/ru/sql-reference/functions/other-functions.md @@ -2090,7 +2090,7 @@ SELECT tcpPort(); ## currentRoles {#current-roles} -Возвращает имена текущих ролей для текущего пользователя. Для изменения текущих ролей может использоваться команда [SET ROLE](../../sql-reference/statements/set-role.md#set-role-statement). Текущие роли по умолчанию можно получить при вызове функции [defaultRoles](#default-roles), когда пользователь входит в систему. +Возвращает имена текущих ролей для текущего пользователя. Для изменения текущих ролей может использоваться команда [SET ROLE](../../sql-reference/statements/set-role.md#set-role-statement). Если команда `SET ROLE` не использовалась, функция возвращает тот же результат, что и функция [defaultRoles](#default-roles). **Синтаксис** @@ -2122,7 +2122,7 @@ enabledRoles() ## defaultRoles {#default-roles} -Возвращает имена ролей, которые задаются как текущие для данного пользователя, когда он входит в систему. Они могут быть изменены при помощи команды [SET DEFAULT ROLE](../../sql-reference/statements/set-role.md#set-default-role-statement), но по умолчанию это все роли, которые предоставляются пользователю. +Возвращает имена ролей, которые задаются как текущие для данного пользователя при входе в систему. Они могут быть изменены при помощи команды [SET DEFAULT ROLE](../../sql-reference/statements/set-role.md#set-default-role-statement). По умолчанию это все роли, которые разрешено использовать текущему пользователю (см. команду [GRANT](../../sql-reference/statements/grant/#grant-select)). **Синтаксис** From c2a29e906de1e4c57363054bd1b1d560bcc4abce Mon Sep 17 00:00:00 2001 From: adevyatova Date: Wed, 4 Aug 2021 15:15:58 +0300 Subject: [PATCH 140/599] Small fix --- docs/ru/sql-reference/functions/other-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/other-functions.md b/docs/ru/sql-reference/functions/other-functions.md index fdd9ff44fd9..f5e583e456c 100644 --- a/docs/ru/sql-reference/functions/other-functions.md +++ b/docs/ru/sql-reference/functions/other-functions.md @@ -2106,7 +2106,7 @@ currentRoles() ## enabledRoles {#enabled-roles} -Возвращает имена текущих ролей вместе с теми ролями, которые предоставлены путем назначения привилегий. +Возвращает имена текущих ролей вместе с теми ролями, которые разрешено использовать текущему пользователю путем назначения привилегий. **Синтаксис** From 61016da2b11665b8dda2cf7af86918a676e2dbb0 Mon Sep 17 00:00:00 2001 From: Anton Ivashkin Date: Thu, 29 Jul 2021 11:25:22 +0300 Subject: [PATCH 141/599] Set allow_remote_fs_zero_copy_replication to true by default --- src/Storages/MergeTree/MergeTreeSettings.h | 2 +- .../ReplicatedMergeTreeMergeStrategyPicker.cpp | 11 ++++++++--- .../configs/config.d/storage_conf.xml | 1 - .../configs/config.d/storage_conf.xml | 1 - .../configs/config.d/s3.xml | 1 - .../01700_system_zookeeper_path_in.reference | 2 ++ 6 files changed, 11 insertions(+), 7 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index d018059c248..531091bb7f9 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -124,7 +124,7 @@ struct Settings; M(UInt64, concurrent_part_removal_threshold, 100, "Activate concurrent part removal (see 'max_part_removal_threads') only if the number of inactive data parts is at least this.", 0) \ M(String, storage_policy, "default", "Name of storage disk policy", 0) \ M(Bool, allow_nullable_key, false, "Allow Nullable types as primary keys.", 0) \ - M(Bool, allow_remote_fs_zero_copy_replication, false, "Allow Zero-copy replication over remote fs", 0) \ + M(Bool, allow_remote_fs_zero_copy_replication, true, "Allow Zero-copy replication over remote fs", 0) \ M(Bool, remove_empty_parts, true, "Remove empty parts after they were pruned by TTL, mutation, or collapsing merge algorithm", 0) \ M(Bool, assign_part_uuids, false, "Generate UUIDs for parts. Before enabling check that all replicas support new format.", 0) \ M(Int64, max_partitions_to_read, -1, "Limit the max number of partitions that can be accessed in one query. <= 0 means unlimited. This setting is the default that can be overridden by the query-level setting with the same name.", 0) \ diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeMergeStrategyPicker.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeMergeStrategyPicker.cpp index 13e05681fd9..2d611dc0c97 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeMergeStrategyPicker.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeMergeStrategyPicker.cpp @@ -144,9 +144,14 @@ void ReplicatedMergeTreeMergeStrategyPicker::refreshState() if (current_replica_index_tmp < 0 || active_replicas_tmp.size() < 2) { - LOG_WARNING(storage.log, "Can't find current replica in the active replicas list, or too few active replicas to use execute_merges_on_single_replica_time_threshold!"); - /// we can reset the settings w/o lock (it's atomic) - execute_merges_on_single_replica_time_threshold = 0; + if (execute_merges_on_single_replica_time_threshold > 0) + { + LOG_WARNING(storage.log, "Can't find current replica in the active replicas list, or too few active replicas to use execute_merges_on_single_replica_time_threshold!"); + /// we can reset the settings w/o lock (it's atomic) + execute_merges_on_single_replica_time_threshold = 0; + } + /// default value of remote_fs_execute_merges_on_single_replica_time_threshold is not 0 + /// so we write no warning in log here remote_fs_execute_merges_on_single_replica_time_threshold = 0; return; } diff --git a/tests/integration/test_replicated_merge_tree_hdfs_zero_copy/configs/config.d/storage_conf.xml b/tests/integration/test_replicated_merge_tree_hdfs_zero_copy/configs/config.d/storage_conf.xml index 46a11a8fe16..1405c7274ca 100644 --- a/tests/integration/test_replicated_merge_tree_hdfs_zero_copy/configs/config.d/storage_conf.xml +++ b/tests/integration/test_replicated_merge_tree_hdfs_zero_copy/configs/config.d/storage_conf.xml @@ -59,7 +59,6 @@ 1024000 1 - 1 diff --git a/tests/integration/test_replicated_merge_tree_s3_zero_copy/configs/config.d/storage_conf.xml b/tests/integration/test_replicated_merge_tree_s3_zero_copy/configs/config.d/storage_conf.xml index 0cf9191c4af..20b750ffff3 100644 --- a/tests/integration/test_replicated_merge_tree_s3_zero_copy/configs/config.d/storage_conf.xml +++ b/tests/integration/test_replicated_merge_tree_s3_zero_copy/configs/config.d/storage_conf.xml @@ -21,7 +21,6 @@ 0 - 1 diff --git a/tests/integration/test_s3_zero_copy_replication/configs/config.d/s3.xml b/tests/integration/test_s3_zero_copy_replication/configs/config.d/s3.xml index 89c97aa3360..e3d3d3543fa 100644 --- a/tests/integration/test_s3_zero_copy_replication/configs/config.d/s3.xml +++ b/tests/integration/test_s3_zero_copy_replication/configs/config.d/s3.xml @@ -66,7 +66,6 @@ 1024 1 - 1 diff --git a/tests/queries/0_stateless/01700_system_zookeeper_path_in.reference b/tests/queries/0_stateless/01700_system_zookeeper_path_in.reference index e491dd9e091..dcee18b33e0 100644 --- a/tests/queries/0_stateless/01700_system_zookeeper_path_in.reference +++ b/tests/queries/0_stateless/01700_system_zookeeper_path_in.reference @@ -12,3 +12,5 @@ blocks failed_parts last_part parallel +shared +shared From a59d9e4620676e7536ec2e9b8dce7404c098a919 Mon Sep 17 00:00:00 2001 From: Artur <613623@mail.ru> Date: Wed, 4 Aug 2021 16:56:15 +0000 Subject: [PATCH 142/599] Client with flag for memory tracker --- programs/client/Client.cpp | 31 +++++++++++++++++++++++++++++-- 1 file changed, 29 insertions(+), 2 deletions(-) diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index c5e43449557..f7bbdc15327 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -1,3 +1,6 @@ +#include +#include "Common/MemoryTracker.h" +#include "Columns/ColumnsNumber.h" #include "ConnectionParameters.h" #include "QueryFuzzer.h" #include "Suggest.h" @@ -100,6 +103,14 @@ #pragma GCC optimize("-fno-var-tracking-assignments") #endif +namespace CurrentMetrics +{ + extern const Metric Revision; + extern const Metric VersionInteger; + extern const Metric MemoryTracking; + extern const Metric MaxDDLEntryID; +} + namespace fs = std::filesystem; namespace DB @@ -305,7 +316,7 @@ private: } catch (const Exception & e) { - bool print_stack_trace = config().getBool("stacktrace", false) && e.code() != ErrorCodes::NETWORK_ERROR; + bool print_stack_trace = config().getBool("stacktrace", false) && e.code() != ErrorCodes::NETWORK_ERROR; std::cerr << getExceptionMessage(e, print_stack_trace, true) << std::endl << std::endl; @@ -450,7 +461,7 @@ private: {TokenType::ErrorSingleQuoteIsNotClosed, Replxx::Color::RED}, {TokenType::ErrorDoubleQuoteIsNotClosed, Replxx::Color::RED}, {TokenType::ErrorSinglePipeMark, Replxx::Color::RED}, - {TokenType::ErrorWrongNumber, Replxx::Color::RED}, + {TokenType::ErrorWrongNumber, Replxx::Color::RED}, {TokenType::ErrorMaxQuerySizeExceeded, Replxx::Color::RED }}; const Replxx::Color unknown_token_color = Replxx::Color::RED; @@ -524,6 +535,21 @@ private: { UseSSL use_ssl; + MainThreadStatus::getInstance(); + + if (config().has("max_memory_usage_in_client")) + { + /// Limit on total memory usage + size_t max_client_memory_usage = config().getInt64("max_memory_usage_in_client"); + + if (max_client_memory_usage != 0) + { + total_memory_tracker.setHardLimit(max_client_memory_usage); + total_memory_tracker.setDescription("(total)"); + total_memory_tracker.setMetric(CurrentMetrics::MemoryTracking); + } + } + registerFormats(); registerFunctions(); registerAggregateFunctions(); @@ -2573,6 +2599,7 @@ public: ("opentelemetry-tracestate", po::value(), "OpenTelemetry tracestate header as described by W3C Trace Context recommendation") ("history_file", po::value(), "path to history file") ("no-warnings", "disable warnings when client connects to server") + ("max_memory_usage_in_client", po::value(), "sets memory limit in client") ; Settings cmd_settings; From 9a9aebc6443f5fd37c672386cd6537273f6c23e1 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 4 Aug 2021 20:38:10 +0300 Subject: [PATCH 143/599] fix tests --- src/Interpreters/TreeRewriter.cpp | 2 +- .../queries/0_stateless/02002_system_table_with_tuple.sh | 9 +++++++++ .../0_stateless/02002_system_table_with_tuple.sql | 2 -- 3 files changed, 10 insertions(+), 3 deletions(-) create mode 100755 tests/queries/0_stateless/02002_system_table_with_tuple.sh delete mode 100644 tests/queries/0_stateless/02002_system_table_with_tuple.sql diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index a2b64d26d42..9294cca7bb4 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -614,7 +614,7 @@ class MarkTupleLiteralsAsLegacyData public: using TypeToVisit = ASTLiteral; - void visit(ASTLiteral & literal, ASTPtr &) + static void visit(ASTLiteral & literal, ASTPtr &) { if (literal.value.getType() == Field::Types::Tuple) literal.use_legacy_column_name_of_tuple = true; diff --git a/tests/queries/0_stateless/02002_system_table_with_tuple.sh b/tests/queries/0_stateless/02002_system_table_with_tuple.sh new file mode 100755 index 00000000000..2abf5b8b525 --- /dev/null +++ b/tests/queries/0_stateless/02002_system_table_with_tuple.sh @@ -0,0 +1,9 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +$CLICKHOUSE_CLIENT -q "SELECT count() > 0 FROM system.tables \ + WHERE database IN ('system', '$CLICKHOUSE_DATABASE') \ + SETTINGS legacy_column_name_of_tuple_literal = 1" diff --git a/tests/queries/0_stateless/02002_system_table_with_tuple.sql b/tests/queries/0_stateless/02002_system_table_with_tuple.sql deleted file mode 100644 index 42796e98110..00000000000 --- a/tests/queries/0_stateless/02002_system_table_with_tuple.sql +++ /dev/null @@ -1,2 +0,0 @@ -SELECT count() > 0 FROM system.tables WHERE database IN ('system', 'system') -SETTINGS legacy_column_name_of_tuple_literal = 1; From 8d14f2ef8fd99d4f751f983f8500845902b9dfec Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 4 Aug 2021 20:58:18 +0300 Subject: [PATCH 144/599] Streams -> Processors for dicts, part 1. --- src/Dictionaries/CacheDictionary.cpp | 33 ++-- src/Dictionaries/CacheDictionary.h | 2 +- .../CassandraBlockInputStream.cpp | 26 ++-- src/Dictionaries/CassandraBlockInputStream.h | 10 +- .../CassandraDictionarySource.cpp | 22 ++- src/Dictionaries/CassandraDictionarySource.h | 8 +- .../ClickHouseDictionarySource.cpp | 49 +++--- src/Dictionaries/ClickHouseDictionarySource.h | 12 +- .../DictionaryBlockInputStream.cpp | 24 ++- src/Dictionaries/DictionaryBlockInputStream.h | 32 ++-- .../DictionaryBlockInputStreamBase.cpp | 21 +-- .../DictionaryBlockInputStreamBase.h | 10 +- src/Dictionaries/DictionaryHelpers.h | 19 ++- src/Dictionaries/DictionarySourceHelpers.cpp | 60 +++----- src/Dictionaries/DictionarySourceHelpers.h | 17 +-- src/Dictionaries/DirectDictionary.cpp | 37 ++--- src/Dictionaries/DirectDictionary.h | 6 +- .../ExecutableDictionarySource.cpp | 142 ++++++++++-------- src/Dictionaries/ExecutableDictionarySource.h | 12 +- .../ExecutablePoolDictionarySource.cpp | 95 ++++++------ .../ExecutablePoolDictionarySource.h | 12 +- src/Dictionaries/FileDictionarySource.cpp | 11 +- src/Dictionaries/FileDictionarySource.h | 10 +- src/Dictionaries/FlatDictionary.cpp | 35 +++-- src/Dictionaries/FlatDictionary.h | 4 +- src/Dictionaries/HashedDictionary.cpp | 8 +- src/Dictionaries/HashedDictionary.h | 4 +- src/Dictionaries/IDictionary.h | 2 +- src/Dictionaries/IDictionarySource.h | 29 ++-- src/Dictionaries/readInvalidateQuery.cpp | 19 ++- src/Dictionaries/readInvalidateQuery.h | 6 +- 31 files changed, 407 insertions(+), 370 deletions(-) diff --git a/src/Dictionaries/CacheDictionary.cpp b/src/Dictionaries/CacheDictionary.cpp index 8a8a64fab36..a8754691425 100644 --- a/src/Dictionaries/CacheDictionary.cpp +++ b/src/Dictionaries/CacheDictionary.cpp @@ -13,6 +13,9 @@ #include #include +#include +#include + namespace ProfileEvents { extern const Event DictCacheKeysRequested; @@ -481,24 +484,28 @@ MutableColumns CacheDictionary::aggregateColumns( } template -BlockInputStreamPtr CacheDictionary::getBlockInputStream(const Names & column_names, size_t max_block_size) const +Pipe CacheDictionary::read(const Names & column_names, size_t max_block_size) const { - std::shared_ptr stream; + Pipe pipe; { /// Write lock on storage const ProfilingScopedWriteRWLock write_lock{rw_lock, ProfileEvents::DictCacheLockWriteNs}; if constexpr (dictionary_key_type == DictionaryKeyType::simple) - stream = std::make_shared(shared_from_this(), max_block_size, cache_storage_ptr->getCachedSimpleKeys(), column_names); + pipe = Pipe(std::make_shared( + DictionarySourceData(shared_from_this(), cache_storage_ptr->getCachedSimpleKeys(), column_names), + max_block_size)); else { auto keys = cache_storage_ptr->getCachedComplexKeys(); - stream = std::make_shared(shared_from_this(), max_block_size, keys, column_names); + pipe = Pipe(std::make_shared( + DictionarySourceData(shared_from_this(), keys, column_names), + max_block_size)); } } - return stream; + return pipe; } template @@ -567,21 +574,21 @@ void CacheDictionary::update(CacheDictionaryUpdateUnitPtrloadIds(requested_keys_vector); + pipeline.init(current_source_ptr->loadIds(requested_keys_vector)); else - stream = current_source_ptr->loadKeys(update_unit_ptr->key_columns, requested_complex_key_rows); - - stream->readPrefix(); + pipeline.init(current_source_ptr->loadKeys(update_unit_ptr->key_columns, requested_complex_key_rows)); size_t skip_keys_size_offset = dict_struct.getKeysSize(); PaddedPODArray found_keys_in_source; Columns fetched_columns_during_update = fetch_request.makeAttributesResultColumnsNonMutable(); - while (Block block = stream->read()) + PullingPipelineExecutor executor(pipeline); + Block block; + while (executor.pull(block)) { Columns key_columns; key_columns.reserve(skip_keys_size_offset); @@ -625,8 +632,6 @@ void CacheDictionary::update(CacheDictionaryUpdateUnitPtrassumeMutable()); - stream->readSuffix(); - { /// Lock for cache modification ProfilingScopedWriteRWLock write_lock{rw_lock, ProfileEvents::DictCacheLockWriteNs}; @@ -686,4 +691,4 @@ void CacheDictionary::update(CacheDictionaryUpdateUnitPtr; template class CacheDictionary; -} +} \ No newline at end of file diff --git a/src/Dictionaries/CacheDictionary.h b/src/Dictionaries/CacheDictionary.h index baaf99d290b..613d73b0f83 100644 --- a/src/Dictionaries/CacheDictionary.h +++ b/src/Dictionaries/CacheDictionary.h @@ -137,7 +137,7 @@ public: ColumnUInt8::Ptr hasKeys(const Columns & key_columns, const DataTypes & key_types) const override; - BlockInputStreamPtr getBlockInputStream(const Names & column_names, size_t max_block_size) const override; + Pipe read(const Names & column_names, size_t max_block_size) const override; std::exception_ptr getLastException() const override; diff --git a/src/Dictionaries/CassandraBlockInputStream.cpp b/src/Dictionaries/CassandraBlockInputStream.cpp index 57a4555ea87..384717e2ba2 100644 --- a/src/Dictionaries/CassandraBlockInputStream.cpp +++ b/src/Dictionaries/CassandraBlockInputStream.cpp @@ -22,12 +22,13 @@ namespace ErrorCodes extern const int UNKNOWN_TYPE; } -CassandraBlockInputStream::CassandraBlockInputStream( +CassandraSource::CassandraSource( const CassSessionShared & session_, const String & query_str, const Block & sample_block, size_t max_block_size_) - : session(session_) + : SourceWithProgress(sample_block) + , session(session_) , statement(query_str.c_str(), /*parameters count*/ 0) , max_block_size(max_block_size_) , has_more_pages(cass_true) @@ -36,7 +37,7 @@ CassandraBlockInputStream::CassandraBlockInputStream( cassandraCheck(cass_statement_set_paging_size(statement, max_block_size)); } -void CassandraBlockInputStream::insertValue(IColumn & column, ValueType type, const CassValue * cass_value) +void CassandraSource::insertValue(IColumn & column, ValueType type, const CassValue * cass_value) { switch (type) { @@ -148,13 +149,15 @@ void CassandraBlockInputStream::insertValue(IColumn & column, ValueType type, co } } -void CassandraBlockInputStream::readPrefix() -{ - result_future = cass_session_execute(*session, statement); -} -Block CassandraBlockInputStream::readImpl() +Chunk CassandraSource::generate() { + if (!is_initialized) + { + result_future = cass_session_execute(*session, statement); + is_initialized = true; + } + if (!has_more_pages) return {}; @@ -194,12 +197,13 @@ Block CassandraBlockInputStream::readImpl() } } - assert(cass_result_row_count(result) == columns.front()->size()); + size_t num_rows = columns.front()->size(); + assert(cass_result_row_count(result) == num_rows); - return description.sample_block.cloneWithColumns(std::move(columns)); + return Chunk(std::move(columns), num_rows); } -void CassandraBlockInputStream::assertTypes(const CassResultPtr & result) +void CassandraSource::assertTypes(const CassResultPtr & result) { if (!assert_types) return; diff --git a/src/Dictionaries/CassandraBlockInputStream.h b/src/Dictionaries/CassandraBlockInputStream.h index 3b0e583e3ad..98adb19fee6 100644 --- a/src/Dictionaries/CassandraBlockInputStream.h +++ b/src/Dictionaries/CassandraBlockInputStream.h @@ -4,17 +4,17 @@ #if USE_CASSANDRA #include -#include +#include #include namespace DB { -class CassandraBlockInputStream final : public IBlockInputStream +class CassandraSource final : public SourceWithProgress { public: - CassandraBlockInputStream( + CassandraSource( const CassSessionShared & session_, const String & query_str, const Block & sample_block, @@ -24,12 +24,11 @@ public: Block getHeader() const override { return description.sample_block.cloneEmpty(); } - void readPrefix() override; private: using ValueType = ExternalResultDescription::ValueType; - Block readImpl() override; + Chunk generate() override; static void insertValue(IColumn & column, ValueType type, const CassValue * cass_value); void assertTypes(const CassResultPtr & result); @@ -40,6 +39,7 @@ private: ExternalResultDescription description; cass_bool_t has_more_pages; bool assert_types = true; + bool is_initialized = false; }; } diff --git a/src/Dictionaries/CassandraDictionarySource.cpp b/src/Dictionaries/CassandraDictionarySource.cpp index 7605b86ef90..8b31b4d6fa2 100644 --- a/src/Dictionaries/CassandraDictionarySource.cpp +++ b/src/Dictionaries/CassandraDictionarySource.cpp @@ -40,7 +40,6 @@ void registerDictionarySourceCassandra(DictionarySourceFactory & factory) #include #include "CassandraBlockInputStream.h" #include -#include namespace DB { @@ -132,12 +131,12 @@ void CassandraDictionarySource::maybeAllowFiltering(String & query) const query += " ALLOW FILTERING;"; } -BlockInputStreamPtr CassandraDictionarySource::loadAll() +Pipe CassandraDictionarySource::loadAll() { String query = query_builder.composeLoadAllQuery(); maybeAllowFiltering(query); LOG_INFO(log, "Loading all using query: {}", query); - return std::make_shared(getSession(), query, sample_block, max_block_size); + return Pipe(std::make_shared(getSession(), query, sample_block, max_block_size)); } std::string CassandraDictionarySource::toString() const @@ -145,15 +144,15 @@ std::string CassandraDictionarySource::toString() const return "Cassandra: " + settings.db + '.' + settings.table; } -BlockInputStreamPtr CassandraDictionarySource::loadIds(const std::vector & ids) +Pipe CassandraDictionarySource::loadIds(const std::vector & ids) { String query = query_builder.composeLoadIdsQuery(ids); maybeAllowFiltering(query); LOG_INFO(log, "Loading ids using query: {}", query); - return std::make_shared(getSession(), query, sample_block, max_block_size); + return Pipe(std::make_shared(getSession(), query, sample_block, max_block_size)); } -BlockInputStreamPtr CassandraDictionarySource::loadKeys(const Columns & key_columns, const std::vector & requested_rows) +Pipe CassandraDictionarySource::loadKeys(const Columns & key_columns, const std::vector & requested_rows) { if (requested_rows.empty()) throw Exception(ErrorCodes::LOGICAL_ERROR, "No rows requested"); @@ -168,22 +167,19 @@ BlockInputStreamPtr CassandraDictionarySource::loadKeys(const Columns & key_colu partitions[partition_key.get64()].push_back(row); } - BlockInputStreams streams; + Pipes pipes; for (const auto & partition : partitions) { String query = query_builder.composeLoadKeysQuery(key_columns, partition.second, ExternalQueryBuilder::CASSANDRA_SEPARATE_PARTITION_KEY, settings.partition_key_prefix); maybeAllowFiltering(query); LOG_INFO(log, "Loading keys for partition hash {} using query: {}", partition.first, query); - streams.push_back(std::make_shared(getSession(), query, sample_block, max_block_size)); + pipes.push_back(Pipe(std::make_shared(getSession(), query, sample_block, max_block_size))); } - if (streams.size() == 1) - return streams.front(); - - return std::make_shared(streams, nullptr, settings.max_threads); + return Pipe::unitePipes(std::move(pipes)); } -BlockInputStreamPtr CassandraDictionarySource::loadUpdatedAll() +Pipe CassandraDictionarySource::loadUpdatedAll() { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method loadUpdatedAll is unsupported for CassandraDictionarySource"); } diff --git a/src/Dictionaries/CassandraDictionarySource.h b/src/Dictionaries/CassandraDictionarySource.h index c0a4e774d23..871e3dc4857 100644 --- a/src/Dictionaries/CassandraDictionarySource.h +++ b/src/Dictionaries/CassandraDictionarySource.h @@ -49,7 +49,7 @@ public: const String & config_prefix, Block & sample_block); - BlockInputStreamPtr loadAll() override; + Pipe loadAll() override; bool supportsSelectiveLoad() const override { return true; } @@ -62,11 +62,11 @@ public: return std::make_unique(dict_struct, settings, sample_block); } - BlockInputStreamPtr loadIds(const std::vector & ids) override; + Pipe loadIds(const std::vector & ids) override; - BlockInputStreamPtr loadKeys(const Columns & key_columns, const std::vector & requested_rows) override; + Pipe loadKeys(const Columns & key_columns, const std::vector & requested_rows) override; - BlockInputStreamPtr loadUpdatedAll() override; + Pipe loadUpdatedAll() override; String toString() const override; diff --git a/src/Dictionaries/ClickHouseDictionarySource.cpp b/src/Dictionaries/ClickHouseDictionarySource.cpp index 42ec73ee520..8b2373302c8 100644 --- a/src/Dictionaries/ClickHouseDictionarySource.cpp +++ b/src/Dictionaries/ClickHouseDictionarySource.cpp @@ -1,8 +1,11 @@ #include "ClickHouseDictionarySource.h" #include #include -#include -#include +#include +#include +#include +#include +#include #include #include #include @@ -105,29 +108,29 @@ std::string ClickHouseDictionarySource::getUpdateFieldAndDate() } } -BlockInputStreamPtr ClickHouseDictionarySource::loadAllWithSizeHint(std::atomic * result_size_hint) +Pipe ClickHouseDictionarySource::loadAllWithSizeHint(std::atomic * result_size_hint) { return createStreamForQuery(load_all_query, result_size_hint); } -BlockInputStreamPtr ClickHouseDictionarySource::loadAll() +Pipe ClickHouseDictionarySource::loadAll() { return createStreamForQuery(load_all_query); } -BlockInputStreamPtr ClickHouseDictionarySource::loadUpdatedAll() +Pipe ClickHouseDictionarySource::loadUpdatedAll() { String load_update_query = getUpdateFieldAndDate(); return createStreamForQuery(load_update_query); } -BlockInputStreamPtr ClickHouseDictionarySource::loadIds(const std::vector & ids) +Pipe ClickHouseDictionarySource::loadIds(const std::vector & ids) { return createStreamForQuery(query_builder.composeLoadIdsQuery(ids)); } -BlockInputStreamPtr ClickHouseDictionarySource::loadKeys(const Columns & key_columns, const std::vector & requested_rows) +Pipe ClickHouseDictionarySource::loadKeys(const Columns & key_columns, const std::vector & requested_rows) { String query = query_builder.composeLoadKeysQuery(key_columns, requested_rows, ExternalQueryBuilder::IN_WITH_TUPLES); return createStreamForQuery(query); @@ -157,32 +160,41 @@ std::string ClickHouseDictionarySource::toString() const return "ClickHouse: " + configuration.db + '.' + configuration.table + (where.empty() ? "" : ", where: " + where); } -BlockInputStreamPtr ClickHouseDictionarySource::createStreamForQuery(const String & query, std::atomic * result_size_hint) +Pipe ClickHouseDictionarySource::createStreamForQuery(const String & query, std::atomic * result_size_hint) { - BlockInputStreamPtr stream; + QueryPipeline pipeline; /// Sample block should not contain first row default values auto empty_sample_block = sample_block.cloneEmpty(); if (configuration.is_local) { - stream = executeQuery(query, context, true).getInputStream(); - stream = std::make_shared(stream, empty_sample_block, ConvertingBlockInputStream::MatchColumnsMode::Position); + pipeline = executeQuery(query, context, true).pipeline; + auto converting = ActionsDAG::makeConvertingActions( + pipeline.getHeader().getColumnsWithTypeAndName(), + empty_sample_block.getColumnsWithTypeAndName(), + ActionsDAG::MatchColumnsMode::Position); + + pipeline.addSimpleTransform([&](const Block & header) + { + return std::make_shared(header, std::make_shared(converting)); + }); } else { - stream = std::make_shared(pool, query, empty_sample_block, context); + pipeline.init(Pipe(std::make_shared( + std::make_shared(pool, query, empty_sample_block, context), false, false))); } if (result_size_hint) { - stream->setProgressCallback([result_size_hint](const Progress & progress) + pipeline.setProgressCallback([result_size_hint](const Progress & progress) { *result_size_hint += progress.total_rows_to_read; }); } - return stream; + return QueryPipeline::getPipe(std::move(pipeline)); } std::string ClickHouseDictionarySource::doInvalidateQuery(const std::string & request) const @@ -191,15 +203,16 @@ std::string ClickHouseDictionarySource::doInvalidateQuery(const std::string & re if (configuration.is_local) { auto query_context = Context::createCopy(context); - auto input_block = executeQuery(request, query_context, true).getInputStream(); - return readInvalidateQuery(*input_block); + auto pipe = QueryPipeline::getPipe(executeQuery(request, query_context, true).pipeline); + return readInvalidateQuery(std::move(pipe)); } else { /// We pass empty block to RemoteBlockInputStream, because we don't know the structure of the result. Block invalidate_sample_block; - RemoteBlockInputStream invalidate_stream(pool, request, invalidate_sample_block, context); - return readInvalidateQuery(invalidate_stream); + Pipe pipe(std::make_shared( + std::make_shared(pool, request, invalidate_sample_block, context), false, false)); + return readInvalidateQuery(std::move(pipe)); } } diff --git a/src/Dictionaries/ClickHouseDictionarySource.h b/src/Dictionaries/ClickHouseDictionarySource.h index fe37610b9c4..f293c010ec3 100644 --- a/src/Dictionaries/ClickHouseDictionarySource.h +++ b/src/Dictionaries/ClickHouseDictionarySource.h @@ -44,15 +44,15 @@ public: ClickHouseDictionarySource(const ClickHouseDictionarySource & other); ClickHouseDictionarySource & operator=(const ClickHouseDictionarySource &) = delete; - BlockInputStreamPtr loadAllWithSizeHint(std::atomic * result_size_hint) override; + Pipe loadAllWithSizeHint(std::atomic * result_size_hint) override; - BlockInputStreamPtr loadAll() override; + Pipe loadAll() override; - BlockInputStreamPtr loadUpdatedAll() override; + Pipe loadUpdatedAll() override; - BlockInputStreamPtr loadIds(const std::vector & ids) override; + Pipe loadIds(const std::vector & ids) override; - BlockInputStreamPtr loadKeys(const Columns & key_columns, const std::vector & requested_rows) override; + Pipe loadKeys(const Columns & key_columns, const std::vector & requested_rows) override; bool isModified() const override; bool supportsSelectiveLoad() const override { return true; } @@ -70,7 +70,7 @@ public: private: std::string getUpdateFieldAndDate(); - BlockInputStreamPtr createStreamForQuery(const String & query, std::atomic * result_size_hint = nullptr); + Pipe createStreamForQuery(const String & query, std::atomic * result_size_hint = nullptr); std::string doInvalidateQuery(const std::string & request) const; diff --git a/src/Dictionaries/DictionaryBlockInputStream.cpp b/src/Dictionaries/DictionaryBlockInputStream.cpp index c902a87cfe3..fedde8bd886 100644 --- a/src/Dictionaries/DictionaryBlockInputStream.cpp +++ b/src/Dictionaries/DictionaryBlockInputStream.cpp @@ -8,9 +8,9 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -DictionaryBlockInputStream::DictionaryBlockInputStream( - std::shared_ptr dictionary_, UInt64 max_block_size_, PaddedPODArray && ids_, const Names & column_names_) - : DictionaryBlockInputStreamBase(ids_.size(), max_block_size_) +DictionarySourceData::DictionarySourceData( + std::shared_ptr dictionary_, PaddedPODArray && ids_, const Names & column_names_) + : num_rows(ids_.size()) , dictionary(dictionary_) , column_names(column_names_) , ids(std::move(ids_)) @@ -18,12 +18,11 @@ DictionaryBlockInputStream::DictionaryBlockInputStream( { } -DictionaryBlockInputStream::DictionaryBlockInputStream( +DictionarySourceData::DictionarySourceData( std::shared_ptr dictionary_, - UInt64 max_block_size_, const PaddedPODArray & keys, const Names & column_names_) - : DictionaryBlockInputStreamBase(keys.size(), max_block_size_) + : num_rows(keys.size()) , dictionary(dictionary_) , column_names(column_names_) , key_type(DictionaryInputStreamKeyType::ComplexKey) @@ -32,14 +31,13 @@ DictionaryBlockInputStream::DictionaryBlockInputStream( fillKeyColumns(keys, 0, keys.size(), dictionary_structure, key_columns); } -DictionaryBlockInputStream::DictionaryBlockInputStream( +DictionarySourceData::DictionarySourceData( std::shared_ptr dictionary_, - UInt64 max_block_size_, const Columns & data_columns_, const Names & column_names_, GetColumnsFunction && get_key_columns_function_, GetColumnsFunction && get_view_columns_function_) - : DictionaryBlockInputStreamBase(data_columns_.front()->size(), max_block_size_) + : num_rows(data_columns_.front()->size()) , dictionary(dictionary_) , column_names(column_names_) , data_columns(data_columns_) @@ -49,7 +47,7 @@ DictionaryBlockInputStream::DictionaryBlockInputStream( { } -Block DictionaryBlockInputStream::getBlock(size_t start, size_t length) const +Block DictionarySourceData::getBlock(size_t start, size_t length) const { /// TODO: Rewrite switch (key_type) @@ -98,7 +96,7 @@ Block DictionaryBlockInputStream::getBlock(size_t start, size_t length) const throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected DictionaryInputStreamKeyType."); } -Block DictionaryBlockInputStream::fillBlock( +Block DictionarySourceData::fillBlock( const PaddedPODArray & ids_to_fill, const Columns & keys, const DataTypes & types, @@ -161,14 +159,14 @@ Block DictionaryBlockInputStream::fillBlock( return Block(block_columns); } -ColumnPtr DictionaryBlockInputStream::getColumnFromIds(const PaddedPODArray & ids_to_fill) +ColumnPtr DictionarySourceData::getColumnFromIds(const PaddedPODArray & ids_to_fill) { auto column_vector = ColumnVector::create(); column_vector->getData().assign(ids_to_fill); return column_vector; } -void DictionaryBlockInputStream::fillKeyColumns( +void DictionarySourceData::fillKeyColumns( const PaddedPODArray & keys, size_t start, size_t size, diff --git a/src/Dictionaries/DictionaryBlockInputStream.h b/src/Dictionaries/DictionaryBlockInputStream.h index 7692c910b94..c15406487e2 100644 --- a/src/Dictionaries/DictionaryBlockInputStream.h +++ b/src/Dictionaries/DictionaryBlockInputStream.h @@ -20,18 +20,16 @@ namespace DB /* BlockInputStream implementation for external dictionaries * read() returns blocks consisting of the in-memory contents of the dictionaries */ -class DictionaryBlockInputStream : public DictionaryBlockInputStreamBase +class DictionarySourceData { public: - DictionaryBlockInputStream( + DictionarySourceData( std::shared_ptr dictionary, - UInt64 max_block_size, PaddedPODArray && ids, const Names & column_names); - DictionaryBlockInputStream( + DictionarySourceData( std::shared_ptr dictionary, - UInt64 max_block_size, const PaddedPODArray & keys, const Names & column_names); @@ -41,18 +39,15 @@ public: // Calls get_key_columns_function to get key column for dictionary get function call // and get_view_columns_function to get key representation. // Now used in trie dictionary, where columns are stored as ip and mask, and are showed as string - DictionaryBlockInputStream( + DictionarySourceData( std::shared_ptr dictionary, - UInt64 max_block_size, const Columns & data_columns, const Names & column_names, GetColumnsFunction && get_key_columns_function, GetColumnsFunction && get_view_columns_function); - String getName() const override { return "Dictionary"; } - -protected: - Block getBlock(size_t start, size_t length) const override; + Block getBlock(size_t start, size_t length) const; + size_t getNumRows() const { return num_rows; } private: Block fillBlock( @@ -70,6 +65,7 @@ private: const DictionaryStructure & dictionary_structure, ColumnsWithTypeAndName & result); + const size_t num_rows; std::shared_ptr dictionary; Names column_names; PaddedPODArray ids; @@ -89,4 +85,18 @@ private: DictionaryInputStreamKeyType key_type; }; +class DictionarySource final : public DictionarySourceBase +{ +public: + DictionarySource(DictionarySourceData data_, UInt64 max_block_size) + : DictionarySourceBase(data_.getBlock(0, 0), data_.getNumRows(), max_block_size) + , data(std::move(data_)) + {} + + String getName() const override { return "DictionarySource"; } + Block getBlock(size_t start, size_t length) const override { return data.getBlock(start, length); } + + DictionarySourceData data; +}; + } diff --git a/src/Dictionaries/DictionaryBlockInputStreamBase.cpp b/src/Dictionaries/DictionaryBlockInputStreamBase.cpp index 3a3fd09220f..d5e6e8a1cda 100644 --- a/src/Dictionaries/DictionaryBlockInputStreamBase.cpp +++ b/src/Dictionaries/DictionaryBlockInputStreamBase.cpp @@ -2,25 +2,20 @@ namespace DB { -DictionaryBlockInputStreamBase::DictionaryBlockInputStreamBase(size_t rows_count_, size_t max_block_size_) - : rows_count(rows_count_), max_block_size(max_block_size_) +DictionarySourceBase::DictionarySourceBase(const Block & header, size_t rows_count_, size_t max_block_size_) + : SourceWithProgress(header), rows_count(rows_count_), max_block_size(max_block_size_) { } -Block DictionaryBlockInputStreamBase::readImpl() +Chunk DictionarySourceBase::generate() { if (next_row == rows_count) - return Block(); + return {}; - size_t block_size = std::min(max_block_size, rows_count - next_row); - Block block = getBlock(next_row, block_size); - next_row += block_size; - return block; -} - -Block DictionaryBlockInputStreamBase::getHeader() const -{ - return getBlock(0, 0); + size_t size = std::min(max_block_size, rows_count - next_row); + auto chunk = getChunk(next_row, size); + next_row += size; + return chunk; } } diff --git a/src/Dictionaries/DictionaryBlockInputStreamBase.h b/src/Dictionaries/DictionaryBlockInputStreamBase.h index fb99918aed8..1533cba82f6 100644 --- a/src/Dictionaries/DictionaryBlockInputStreamBase.h +++ b/src/Dictionaries/DictionaryBlockInputStreamBase.h @@ -1,24 +1,22 @@ #pragma once -#include +#include namespace DB { -class DictionaryBlockInputStreamBase : public IBlockInputStream +class DictionarySourceBase : public SourceWithProgress { protected: - DictionaryBlockInputStreamBase(size_t rows_count_, size_t max_block_size_); + DictionarySourceBase(const Block & header, size_t rows_count_, size_t max_block_size_); virtual Block getBlock(size_t start, size_t length) const = 0; - Block getHeader() const override; - private: const size_t rows_count; const size_t max_block_size; size_t next_row = 0; - Block readImpl() override; + Chunk generate() override; }; } diff --git a/src/Dictionaries/DictionaryHelpers.h b/src/Dictionaries/DictionaryHelpers.h index ed124ce1e0a..79459057822 100644 --- a/src/Dictionaries/DictionaryHelpers.h +++ b/src/Dictionaries/DictionaryHelpers.h @@ -15,7 +15,8 @@ #include #include #include -#include +#include +#include namespace DB @@ -501,10 +502,10 @@ private: * Note: readPrefix readImpl readSuffix will be called on stream object during function execution. */ template -void mergeBlockWithStream( +void mergeBlockWithPipe( size_t key_columns_size, Block & block_to_update, - BlockInputStreamPtr & stream) + Pipe pipe) { using KeyType = std::conditional_t; static_assert(dictionary_key_type != DictionaryKeyType::range, "Range key type is not supported by updatePreviousyLoadedBlockWithStream"); @@ -555,9 +556,13 @@ void mergeBlockWithStream( auto result_fetched_columns = block_to_update.cloneEmptyColumns(); - stream->readPrefix(); + QueryPipeline pipeline; + pipeline.init(std::move(pipe)); - while (Block block = stream->read()) + PullingPipelineExecutor executor(pipeline); + Block block; + + while (executor.pull(block)) { Columns block_key_columns; block_key_columns.reserve(key_columns_size); @@ -591,8 +596,6 @@ void mergeBlockWithStream( } } - stream->readSuffix(); - size_t result_fetched_rows = result_fetched_columns.front()->size(); size_t filter_hint = filter.size() - indexes_to_remove_count; @@ -645,4 +648,4 @@ static const PaddedPODArray & getColumnVectorData( } } -} +} \ No newline at end of file diff --git a/src/Dictionaries/DictionarySourceHelpers.cpp b/src/Dictionaries/DictionarySourceHelpers.cpp index 54ed07092d3..79d4d2e5376 100644 --- a/src/Dictionaries/DictionarySourceHelpers.cpp +++ b/src/Dictionaries/DictionarySourceHelpers.cpp @@ -85,62 +85,44 @@ ContextMutablePtr copyContextAndApplySettings( return local_context; } - -BlockInputStreamWithAdditionalColumns::BlockInputStreamWithAdditionalColumns( - Block block_to_add_, std::unique_ptr && stream_) - : block_to_add(std::move(block_to_add_)) - , stream(std::move(stream_)) +static Block transformHeader(Block header, Block block_to_add) { -} - -Block BlockInputStreamWithAdditionalColumns::getHeader() const -{ - auto header = stream->getHeader(); - - if (header) - { - for (Int64 i = static_cast(block_to_add.columns() - 1); i >= 0; --i) - header.insert(0, block_to_add.getByPosition(i).cloneEmpty()); - } + for (Int64 i = static_cast(block_to_add.columns() - 1); i >= 0; --i) + header.insert(0, block_to_add.getByPosition(i).cloneEmpty()); return header; } -Block BlockInputStreamWithAdditionalColumns::readImpl() +TransformWithAdditionalColumns::TransformWithAdditionalColumns( + Block block_to_add_, const Block & header) + : ISimpleTransform(header, transformHeader(header, block_to_add_), true) + , block_to_add(std::move(block_to_add_)) { - auto block = stream->read(); +} - if (block) +void TransformWithAdditionalColumns::transform(Chunk & chunk) +{ + if (chunk) { - auto block_rows = block.rows(); + auto num_rows = chunk.getNumRows(); + auto columns = chunk.detachColumns(); - auto cut_block = block_to_add.cloneWithCutColumns(current_range_index, block_rows); + auto cut_block = block_to_add.cloneWithCutColumns(current_range_index, num_rows); - if (cut_block.rows() != block_rows) + if (cut_block.rows() != num_rows) throw Exception(ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH, "Number of rows in block to add after cut must equal to number of rows in block from inner stream"); for (Int64 i = static_cast(cut_block.columns() - 1); i >= 0; --i) - block.insert(0, cut_block.getByPosition(i)); + columns.insert(columns.begin(), cut_block.getByPosition(i).column); - current_range_index += block_rows; + current_range_index += num_rows; + chunk.setColumns(std::move(columns), num_rows); } - - return block; } -void BlockInputStreamWithAdditionalColumns::readPrefix() +String TransformWithAdditionalColumns::getName() const { - stream->readPrefix(); -} - -void BlockInputStreamWithAdditionalColumns::readSuffix() -{ - stream->readSuffix(); -} - -String BlockInputStreamWithAdditionalColumns::getName() const -{ - return "BlockInputStreamWithAdditionalColumns"; -} + return "TransformWithAdditionalColumns"; } +} \ No newline at end of file diff --git a/src/Dictionaries/DictionarySourceHelpers.h b/src/Dictionaries/DictionarySourceHelpers.h index 6fed4c7181c..ba05cf9ebc9 100644 --- a/src/Dictionaries/DictionarySourceHelpers.h +++ b/src/Dictionaries/DictionarySourceHelpers.h @@ -4,7 +4,7 @@ #include #include -#include +#include #include #include #include @@ -38,25 +38,18 @@ ContextMutablePtr copyContextAndApplySettings( * * block_to_add rows size must be equal to final sum rows size of all inner stream blocks. */ -class BlockInputStreamWithAdditionalColumns final : public IBlockInputStream +class TransformWithAdditionalColumns final : public ISimpleTransform { public: - BlockInputStreamWithAdditionalColumns(Block block_to_add_, std::unique_ptr && stream_); + TransformWithAdditionalColumns(Block block_to_add_, const Block & header); - Block getHeader() const override; - - Block readImpl() override; - - void readPrefix() override; - - void readSuffix() override; + void transform(Chunk & chunk) override; String getName() const override; private: Block block_to_add; - std::unique_ptr stream; size_t current_range_index = 0; }; -} +} \ No newline at end of file diff --git a/src/Dictionaries/DirectDictionary.cpp b/src/Dictionaries/DirectDictionary.cpp index c9b38acfbb5..afdd9bee92c 100644 --- a/src/Dictionaries/DirectDictionary.cpp +++ b/src/Dictionaries/DirectDictionary.cpp @@ -8,6 +8,8 @@ #include #include +#include +#include namespace DB { @@ -66,11 +68,13 @@ Columns DirectDictionary::getColumns( size_t dictionary_keys_size = dict_struct.getKeysNames().size(); block_key_columns.reserve(dictionary_keys_size); - BlockInputStreamPtr stream = getSourceBlockInputStream(key_columns, requested_keys); + QueryPipeline pipeline; + pipeline.init(getSourceBlockInputStream(key_columns, requested_keys)); - stream->readPrefix(); + PullingPipelineExecutor executor(pipeline); - while (const auto block = stream->read()) + Block block; + while (executor.pull(block)) { /// Split into keys columns and attribute columns for (size_t i = 0; i < dictionary_keys_size; ++i) @@ -98,8 +102,6 @@ Columns DirectDictionary::getColumns( block_key_columns.clear(); } - stream->readSuffix(); - Field value_to_insert; size_t requested_keys_size = requested_keys.size(); @@ -183,13 +185,14 @@ ColumnUInt8::Ptr DirectDictionary::hasKeys( size_t dictionary_keys_size = dict_struct.getKeysNames().size(); block_key_columns.reserve(dictionary_keys_size); - BlockInputStreamPtr stream = getSourceBlockInputStream(key_columns, requested_keys); + QueryPipeline pipeline; + pipeline.init(getSourceBlockInputStream(key_columns, requested_keys)); - stream->readPrefix(); + PullingPipelineExecutor executor(pipeline); size_t keys_found = 0; - - while (const auto block = stream->read()) + Block block; + while (executor.pull(block)) { /// Split into keys columns and attribute columns for (size_t i = 0; i < dictionary_keys_size; ++i) @@ -216,8 +219,6 @@ ColumnUInt8::Ptr DirectDictionary::hasKeys( block_key_columns.clear(); } - stream->readSuffix(); - query_count.fetch_add(requested_keys_size, std::memory_order_relaxed); found_count.fetch_add(keys_found, std::memory_order_relaxed); @@ -260,13 +261,13 @@ ColumnUInt8::Ptr DirectDictionary::isInHierarchy( } template -BlockInputStreamPtr DirectDictionary::getSourceBlockInputStream( +Pipe DirectDictionary::getSourceBlockInputStream( const Columns & key_columns [[maybe_unused]], const PaddedPODArray & requested_keys [[maybe_unused]]) const { size_t requested_keys_size = requested_keys.size(); - BlockInputStreamPtr stream; + Pipe pipe; if constexpr (dictionary_key_type == DictionaryKeyType::simple) { @@ -276,7 +277,7 @@ BlockInputStreamPtr DirectDictionary::getSourceBlockInputSt for (auto key : requested_keys) ids.emplace_back(key); - stream = source_ptr->loadIds(ids); + pipe = source_ptr->loadIds(ids); } else { @@ -285,14 +286,14 @@ BlockInputStreamPtr DirectDictionary::getSourceBlockInputSt for (size_t i = 0; i < requested_keys_size; ++i) requested_rows.emplace_back(i); - stream = source_ptr->loadKeys(key_columns, requested_rows); + pipe = source_ptr->loadKeys(key_columns, requested_rows); } - return stream; + return pipe; } template -BlockInputStreamPtr DirectDictionary::getBlockInputStream(const Names & /* column_names */, size_t /* max_block_size */) const +Pipe DirectDictionary::read(const Names & /* column_names */, size_t /* max_block_size */) const { return source_ptr->loadAll(); } @@ -353,4 +354,4 @@ void registerDictionaryDirect(DictionaryFactory & factory) } -} +} \ No newline at end of file diff --git a/src/Dictionaries/DirectDictionary.h b/src/Dictionaries/DirectDictionary.h index a4e8d5f82e6..841590164f6 100644 --- a/src/Dictionaries/DirectDictionary.h +++ b/src/Dictionaries/DirectDictionary.h @@ -97,10 +97,10 @@ public: ColumnPtr in_key_column, const DataTypePtr & key_type) const override; - BlockInputStreamPtr getBlockInputStream(const Names & column_names, size_t max_block_size) const override; + Pipe read(const Names & column_names, size_t max_block_size) const override; private: - BlockInputStreamPtr getSourceBlockInputStream(const Columns & key_columns, const PaddedPODArray & requested_keys) const; + Pipe getSourceBlockInputStream(const Columns & key_columns, const PaddedPODArray & requested_keys) const; const DictionaryStructure dict_struct; const DictionarySourcePtr source_ptr; @@ -113,4 +113,4 @@ private: extern template class DirectDictionary; extern template class DirectDictionary; -} +} \ No newline at end of file diff --git a/src/Dictionaries/ExecutableDictionarySource.cpp b/src/Dictionaries/ExecutableDictionarySource.cpp index daf79965428..82b74d8196e 100644 --- a/src/Dictionaries/ExecutableDictionarySource.cpp +++ b/src/Dictionaries/ExecutableDictionarySource.cpp @@ -2,13 +2,17 @@ #include #include -#include #include #include +#include +#include +#include +#include +#include #include +#include #include #include -#include #include #include #include @@ -34,26 +38,34 @@ namespace ErrorCodes namespace { /// Owns ShellCommand and calls wait for it. - class ShellCommandOwningBlockInputStream : public OwningBlockInputStream + class ShellCommandOwningTransform final : public ISimpleTransform { private: Poco::Logger * log; + std::unique_ptr command; public: - ShellCommandOwningBlockInputStream(Poco::Logger * log_, const BlockInputStreamPtr & impl, std::unique_ptr command_) - : OwningBlockInputStream(std::move(impl), std::move(command_)), log(log_) + ShellCommandOwningTransform(const Block & header, Poco::Logger * log_, std::unique_ptr command_) + : ISimpleTransform(header, header, true), log(log_), command(std::move(command_)) { } - void readSuffix() override + String getName() const override { return "ShellCommandOwningTransform"; } + void transform(Chunk &) override {} + + Status prepare() override { - OwningBlockInputStream::readSuffix(); + auto status = ISimpleTransform::prepare(); + if (status == Status::Finished) + { + std::string err; + readStringUntilEOF(err, command->err); + if (!err.empty()) + LOG_ERROR(log, "Having stderr: {}", err); - std::string err; - readStringUntilEOF(err, own->err); - if (!err.empty()) - LOG_ERROR(log, "Having stderr: {}", err); + command->wait(); + } - own->wait(); + return status; } }; @@ -94,18 +106,19 @@ ExecutableDictionarySource::ExecutableDictionarySource(const ExecutableDictionar { } -BlockInputStreamPtr ExecutableDictionarySource::loadAll() +Pipe ExecutableDictionarySource::loadAll() { if (configuration.implicit_key) throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "ExecutableDictionarySource with implicit_key does not support loadAll method"); LOG_TRACE(log, "loadAll {}", toString()); auto process = ShellCommand::execute(configuration.command); - auto input_stream = context->getInputFormat(configuration.format, process->out, sample_block, max_block_size); - return std::make_shared(log, input_stream, std::move(process)); + Pipe pipe(FormatFactory::instance().getInput(configuration.format, process->out, sample_block, context, max_block_size)); + pipe.addTransform(std::make_shared(pipe.getHeader(), log, std::move(process))); + return pipe; } -BlockInputStreamPtr ExecutableDictionarySource::loadUpdatedAll() +Pipe ExecutableDictionarySource::loadUpdatedAll() { if (configuration.implicit_key) throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "ExecutableDictionarySource with implicit_key does not support loadUpdatedAll method"); @@ -119,81 +132,86 @@ BlockInputStreamPtr ExecutableDictionarySource::loadUpdatedAll() LOG_TRACE(log, "loadUpdatedAll {}", command_with_update_field); auto process = ShellCommand::execute(command_with_update_field); - auto input_stream = context->getInputFormat(configuration.format, process->out, sample_block, max_block_size); - return std::make_shared(log, input_stream, std::move(process)); + + Pipe pipe(FormatFactory::instance().getInput(configuration.format, process->out, sample_block, context, max_block_size)); + pipe.addTransform(std::make_shared(pipe.getHeader(), log, std::move(process))); + return pipe; } namespace { /** A stream, that runs child process and sends data to its stdin in background thread, * and receives data from its stdout. + * + * TODO: implement without background thread. */ - class BlockInputStreamWithBackgroundThread final : public IBlockInputStream + class SourceWithBackgroundThread final : public SourceWithProgress { public: - BlockInputStreamWithBackgroundThread( + SourceWithBackgroundThread( ContextPtr context, const std::string & format, const Block & sample_block, const std::string & command_str, Poco::Logger * log_, std::function && send_data_) - : log(log_), - command(ShellCommand::execute(command_str)), - send_data(std::move(send_data_)), - thread([this] { send_data(command->in); }) + : SourceWithProgress(sample_block) + , log(log_) + , command(ShellCommand::execute(command_str)) + , send_data(std::move(send_data_)) + , thread([this] { send_data(command->in); }) { - stream = context->getInputFormat(format, command->out, sample_block, max_block_size); + pipeline.init(Pipe(FormatFactory::instance().getInput(format, command->out, sample_block, context, max_block_size))); + executor = std::make_unique(pipeline); } - ~BlockInputStreamWithBackgroundThread() override + ~SourceWithBackgroundThread() override { if (thread.joinable()) thread.join(); } - Block getHeader() const override + protected: + Chunk generate() override { - return stream->getHeader(); + Chunk chunk; + executor->pull(chunk); + return chunk; } - private: - Block readImpl() override + public: + Status prepare() override { - return stream->read(); + auto status = SourceWithProgress::prepare(); + + if (status == Status::Finished) + { + std::string err; + readStringUntilEOF(err, command->err); + if (!err.empty()) + LOG_ERROR(log, "Having stderr: {}", err); + + if (thread.joinable()) + thread.join(); + + command->wait(); + } + + return status; } - void readPrefix() override - { - stream->readPrefix(); - } - - void readSuffix() override - { - stream->readSuffix(); - - std::string err; - readStringUntilEOF(err, command->err); - if (!err.empty()) - LOG_ERROR(log, "Having stderr: {}", err); - - if (thread.joinable()) - thread.join(); - - command->wait(); - } - - String getName() const override { return "WithBackgroundThread"; } + String getName() const override { return "SourceWithBackgroundThread"; } Poco::Logger * log; - BlockInputStreamPtr stream; + QueryPipeline pipeline; + std::unique_ptr executor; std::unique_ptr command; std::function send_data; ThreadFromGlobalPool thread; }; } -BlockInputStreamPtr ExecutableDictionarySource::loadIds(const std::vector & ids) +Pipe ExecutableDictionarySource::loadIds(const std::vector & ids) { LOG_TRACE(log, "loadIds {} size = {}", toString(), ids.size()); @@ -201,7 +219,7 @@ BlockInputStreamPtr ExecutableDictionarySource::loadIds(const std::vector & requested_rows) +Pipe ExecutableDictionarySource::loadKeys(const Columns & key_columns, const std::vector & requested_rows) { LOG_TRACE(log, "loadKeys {} size = {}", toString(), requested_rows.size()); @@ -209,21 +227,21 @@ BlockInputStreamPtr ExecutableDictionarySource::loadKeys(const Columns & key_col return getStreamForBlock(block); } -BlockInputStreamPtr ExecutableDictionarySource::getStreamForBlock(const Block & block) +Pipe ExecutableDictionarySource::getStreamForBlock(const Block & block) { - auto stream = std::make_unique( + Pipe pipe(std::make_unique( context, configuration.format, sample_block, configuration.command, log, [block, this](WriteBufferFromFile & out) mutable { auto output_stream = context->getOutputStream(configuration.format, out, block.cloneEmpty()); formatBlock(output_stream, block); out.close(); - }); + })); if (configuration.implicit_key) - return std::make_shared(block, std::move(stream)); - else - return std::shared_ptr(stream.release()); + pipe.addTransform(std::make_shared(block, pipe.getHeader())); + + return pipe; } bool ExecutableDictionarySource::isModified() const @@ -289,4 +307,4 @@ void registerDictionarySourceExecutable(DictionarySourceFactory & factory) factory.registerSource("executable", create_table_source); } -} +} \ No newline at end of file diff --git a/src/Dictionaries/ExecutableDictionarySource.h b/src/Dictionaries/ExecutableDictionarySource.h index 0b92023df36..084b8c13c25 100644 --- a/src/Dictionaries/ExecutableDictionarySource.h +++ b/src/Dictionaries/ExecutableDictionarySource.h @@ -36,17 +36,17 @@ public: ExecutableDictionarySource(const ExecutableDictionarySource & other); ExecutableDictionarySource & operator=(const ExecutableDictionarySource &) = delete; - BlockInputStreamPtr loadAll() override; + Pipe loadAll() override; /** The logic of this method is flawed, absolutely incorrect and ignorant. * It may lead to skipping some values due to clock sync or timezone changes. * The intended usage of "update_field" is totally different. */ - BlockInputStreamPtr loadUpdatedAll() override; + Pipe loadUpdatedAll() override; - BlockInputStreamPtr loadIds(const std::vector & ids) override; + Pipe loadIds(const std::vector & ids) override; - BlockInputStreamPtr loadKeys(const Columns & key_columns, const std::vector & requested_rows) override; + Pipe loadKeys(const Columns & key_columns, const std::vector & requested_rows) override; bool isModified() const override; @@ -58,7 +58,7 @@ public: std::string toString() const override; - BlockInputStreamPtr getStreamForBlock(const Block & block); + Pipe getStreamForBlock(const Block & block); private: Poco::Logger * log; @@ -69,4 +69,4 @@ private: ContextPtr context; }; -} +} \ No newline at end of file diff --git a/src/Dictionaries/ExecutablePoolDictionarySource.cpp b/src/Dictionaries/ExecutablePoolDictionarySource.cpp index 9eacda343cf..2cd567bbc02 100644 --- a/src/Dictionaries/ExecutablePoolDictionarySource.cpp +++ b/src/Dictionaries/ExecutablePoolDictionarySource.cpp @@ -2,12 +2,15 @@ #include #include -#include +#include +#include +#include #include #include #include #include -#include +#include +#include #include #include #include @@ -69,12 +72,12 @@ ExecutablePoolDictionarySource::ExecutablePoolDictionarySource(const ExecutableP { } -BlockInputStreamPtr ExecutablePoolDictionarySource::loadAll() +Pipe ExecutablePoolDictionarySource::loadAll() { throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "ExecutablePoolDictionarySource does not support loadAll method"); } -BlockInputStreamPtr ExecutablePoolDictionarySource::loadUpdatedAll() +Pipe ExecutablePoolDictionarySource::loadUpdatedAll() { throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "ExecutablePoolDictionarySource does not support loadUpdatedAll method"); } @@ -84,19 +87,19 @@ namespace /** A stream, that runs child process and sends data to its stdin in background thread, * and receives data from its stdout. */ - class PoolBlockInputStreamWithBackgroundThread final : public IBlockInputStream + class PoolSourceWithBackgroundThread final : public SourceWithProgress { public: - PoolBlockInputStreamWithBackgroundThread( + PoolSourceWithBackgroundThread( std::shared_ptr process_pool_, std::unique_ptr && command_, - BlockInputStreamPtr && stream_, + Pipe pipe, size_t read_rows_, Poco::Logger * log_, std::function && send_data_) - : process_pool(process_pool_) + : SourceWithProgress(pipe.getHeader()) + , process_pool(process_pool_) , command(std::move(command_)) - , stream(std::move(stream_)) , rows_to_read(read_rows_) , log(log_) , send_data(std::move(send_data_)) @@ -112,9 +115,12 @@ namespace exception_during_read = std::current_exception(); } }) - {} + { + pipeline.init(std::move(pipe)); + executor = std::make_unique(pipeline); + } - ~PoolBlockInputStreamWithBackgroundThread() override + ~PoolSourceWithBackgroundThread() override { if (thread.joinable()) thread.join(); @@ -123,25 +129,22 @@ namespace process_pool->returnObject(std::move(command)); } - Block getHeader() const override - { - return stream->getHeader(); - } - - private: - Block readImpl() override + protected: + Chunk generate() override { rethrowExceptionDuringReadIfNeeded(); if (current_read_rows == rows_to_read) - return Block(); + return {}; - Block block; + Chunk chunk; try { - block = stream->read(); - current_read_rows += block.rows(); + if (!executor->pull(chunk)) + return {}; + + current_read_rows += chunk.getNumRows(); } catch (...) { @@ -150,22 +153,23 @@ namespace throw; } - return block; + return chunk; } - void readPrefix() override + public: + Status prepare() override { - rethrowExceptionDuringReadIfNeeded(); - stream->readPrefix(); - } + auto status = SourceWithProgress::prepare(); - void readSuffix() override - { - if (thread.joinable()) - thread.join(); + if (status == Status::Finished) + { + if (thread.joinable()) + thread.join(); - rethrowExceptionDuringReadIfNeeded(); - stream->readSuffix(); + rethrowExceptionDuringReadIfNeeded(); + } + + return status; } void rethrowExceptionDuringReadIfNeeded() @@ -182,7 +186,8 @@ namespace std::shared_ptr process_pool; std::unique_ptr command; - BlockInputStreamPtr stream; + QueryPipeline pipeline; + std::unique_ptr executor; size_t rows_to_read; Poco::Logger * log; std::function send_data; @@ -194,7 +199,7 @@ namespace } -BlockInputStreamPtr ExecutablePoolDictionarySource::loadIds(const std::vector & ids) +Pipe ExecutablePoolDictionarySource::loadIds(const std::vector & ids) { LOG_TRACE(log, "loadIds {} size = {}", toString(), ids.size()); @@ -202,7 +207,7 @@ BlockInputStreamPtr ExecutablePoolDictionarySource::loadIds(const std::vector & requested_rows) +Pipe ExecutablePoolDictionarySource::loadKeys(const Columns & key_columns, const std::vector & requested_rows) { LOG_TRACE(log, "loadKeys {} size = {}", toString(), requested_rows.size()); @@ -210,7 +215,7 @@ BlockInputStreamPtr ExecutablePoolDictionarySource::loadKeys(const Columns & key return getStreamForBlock(block); } -BlockInputStreamPtr ExecutablePoolDictionarySource::getStreamForBlock(const Block & block) +Pipe ExecutablePoolDictionarySource::getStreamForBlock(const Block & block) { std::unique_ptr process; bool result = process_pool->tryBorrowObject(process, [this]() @@ -227,20 +232,20 @@ BlockInputStreamPtr ExecutablePoolDictionarySource::getStreamForBlock(const Bloc configuration.max_command_execution_time); size_t rows_to_read = block.rows(); - auto read_stream = context->getInputFormat(configuration.format, process->out, sample_block, rows_to_read); + auto format = FormatFactory::instance().getInput(configuration.format, process->out, sample_block, context, rows_to_read); - auto stream = std::make_unique( - process_pool, std::move(process), std::move(read_stream), rows_to_read, log, + Pipe pipe(std::make_unique( + process_pool, std::move(process), Pipe(std::move(format)), rows_to_read, log, [block, this](WriteBufferFromFile & out) mutable { auto output_stream = context->getOutputStream(configuration.format, out, block.cloneEmpty()); formatBlock(output_stream, block); - }); + })); if (configuration.implicit_key) - return std::make_shared(block, std::move(stream)); - else - return std::shared_ptr(stream.release()); + pipe.addTransform(std::make_shared(block, pipe.getHeader())); + + return pipe; } bool ExecutablePoolDictionarySource::isModified() const @@ -320,4 +325,4 @@ void registerDictionarySourceExecutablePool(DictionarySourceFactory & factory) factory.registerSource("executable_pool", create_table_source); } -} +} \ No newline at end of file diff --git a/src/Dictionaries/ExecutablePoolDictionarySource.h b/src/Dictionaries/ExecutablePoolDictionarySource.h index 02b0288a52e..9c8730632f7 100644 --- a/src/Dictionaries/ExecutablePoolDictionarySource.h +++ b/src/Dictionaries/ExecutablePoolDictionarySource.h @@ -48,17 +48,17 @@ public: ExecutablePoolDictionarySource(const ExecutablePoolDictionarySource & other); ExecutablePoolDictionarySource & operator=(const ExecutablePoolDictionarySource &) = delete; - BlockInputStreamPtr loadAll() override; + Pipe loadAll() override; /** The logic of this method is flawed, absolutely incorrect and ignorant. * It may lead to skipping some values due to clock sync or timezone changes. * The intended usage of "update_field" is totally different. */ - BlockInputStreamPtr loadUpdatedAll() override; + Pipe loadUpdatedAll() override; - BlockInputStreamPtr loadIds(const std::vector & ids) override; + Pipe loadIds(const std::vector & ids) override; - BlockInputStreamPtr loadKeys(const Columns & key_columns, const std::vector & requested_rows) override; + Pipe loadKeys(const Columns & key_columns, const std::vector & requested_rows) override; bool isModified() const override; @@ -70,7 +70,7 @@ public: std::string toString() const override; - BlockInputStreamPtr getStreamForBlock(const Block & block); + Pipe getStreamForBlock(const Block & block); private: Poco::Logger * log; @@ -83,4 +83,4 @@ private: std::shared_ptr process_pool; }; -} +} \ No newline at end of file diff --git a/src/Dictionaries/FileDictionarySource.cpp b/src/Dictionaries/FileDictionarySource.cpp index 239c13e71c2..3766da0a28d 100644 --- a/src/Dictionaries/FileDictionarySource.cpp +++ b/src/Dictionaries/FileDictionarySource.cpp @@ -5,6 +5,8 @@ #include #include #include +#include +#include #include "DictionarySourceFactory.h" #include "DictionaryStructure.h" #include "registerDictionaries.h" @@ -45,14 +47,15 @@ FileDictionarySource::FileDictionarySource(const FileDictionarySource & other) } -BlockInputStreamPtr FileDictionarySource::loadAll() +Pipe FileDictionarySource::loadAll() { LOG_TRACE(&Poco::Logger::get("FileDictionary"), "loadAll {}", toString()); auto in_ptr = std::make_unique(filepath); - auto stream = context->getInputFormat(format, *in_ptr, sample_block, max_block_size); + auto source = FormatFactory::instance().getInput(format, *in_ptr, sample_block, context, max_block_size); + source->addBuffer(std::move(in_ptr)); last_modification = getLastModification(); - return std::make_shared>(stream, std::move(in_ptr)); + return Pipe(std::move(source)); } @@ -92,4 +95,4 @@ void registerDictionarySourceFile(DictionarySourceFactory & factory) factory.registerSource("file", create_table_source); } -} +} \ No newline at end of file diff --git a/src/Dictionaries/FileDictionarySource.h b/src/Dictionaries/FileDictionarySource.h index ffc29374f4f..efc86be8ba9 100644 --- a/src/Dictionaries/FileDictionarySource.h +++ b/src/Dictionaries/FileDictionarySource.h @@ -21,19 +21,19 @@ public: FileDictionarySource(const FileDictionarySource & other); - BlockInputStreamPtr loadAll() override; + Pipe loadAll() override; - BlockInputStreamPtr loadUpdatedAll() override + Pipe loadUpdatedAll() override { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method loadUpdatedAll is unsupported for FileDictionarySource"); } - BlockInputStreamPtr loadIds(const std::vector & /*ids*/) override + Pipe loadIds(const std::vector & /*ids*/) override { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method loadIds is unsupported for FileDictionarySource"); } - BlockInputStreamPtr loadKeys(const Columns & /*key_columns*/, const std::vector & /*requested_rows*/) override + Pipe loadKeys(const Columns & /*key_columns*/, const std::vector & /*requested_rows*/) override { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method loadKeys is unsupported for FileDictionarySource"); } @@ -65,4 +65,4 @@ private: Poco::Timestamp last_modification; }; -} +} \ No newline at end of file diff --git a/src/Dictionaries/FlatDictionary.cpp b/src/Dictionaries/FlatDictionary.cpp index abef2335ffd..40afe70f504 100644 --- a/src/Dictionaries/FlatDictionary.cpp +++ b/src/Dictionaries/FlatDictionary.cpp @@ -10,6 +10,9 @@ #include #include +#include +#include + #include #include #include @@ -319,10 +322,12 @@ void FlatDictionary::updateData() { if (!update_field_loaded_block || update_field_loaded_block->rows() == 0) { - auto stream = source_ptr->loadUpdatedAll(); - stream->readPrefix(); + QueryPipeline pipeline; + pipeline.init(source_ptr->loadUpdatedAll()); - while (const auto block = stream->read()) + PullingPipelineExecutor executor(pipeline); + Block block; + while (executor.pull(block)) { /// We are using this to keep saved data if input stream consists of multiple blocks if (!update_field_loaded_block) @@ -335,15 +340,14 @@ void FlatDictionary::updateData() saved_column->insertRangeFrom(update_column, 0, update_column.size()); } } - stream->readSuffix(); } else { - auto stream = source_ptr->loadUpdatedAll(); - mergeBlockWithStream( + Pipe pipe(source_ptr->loadUpdatedAll()); + mergeBlockWithPipe( dict_struct.getKeysSize(), *update_field_loaded_block, - stream); + std::move(pipe)); } if (update_field_loaded_block) @@ -354,13 +358,13 @@ void FlatDictionary::loadData() { if (!source_ptr->hasUpdateField()) { - auto stream = source_ptr->loadAll(); - stream->readPrefix(); + QueryPipeline pipeline; + pipeline.init(source_ptr->loadAll()); + PullingPipelineExecutor executor(pipeline); - while (const auto block = stream->read()) + Block block; + while (executor.pull(block)) blockToAttributes(block); - - stream->readSuffix(); } else updateData(); @@ -531,7 +535,7 @@ void FlatDictionary::setAttributeValue(Attribute & attribute, const UInt64 key, callOnDictionaryAttributeType(attribute.type, type_call); } -BlockInputStreamPtr FlatDictionary::getBlockInputStream(const Names & column_names, size_t max_block_size) const +Pipe FlatDictionary::read(const Names & column_names, size_t max_block_size) const { const auto keys_count = loaded_keys.size(); @@ -542,7 +546,8 @@ BlockInputStreamPtr FlatDictionary::getBlockInputStream(const Names & column_nam if (loaded_keys[key_index]) keys.push_back(key_index); - return std::make_shared(shared_from_this(), max_block_size, std::move(keys), column_names); + return Pipe(std::make_shared( + DictionarySourceData(shared_from_this(), std::move(keys), column_names), max_block_size)); } void registerDictionaryFlat(DictionaryFactory & factory) @@ -586,4 +591,4 @@ void registerDictionaryFlat(DictionaryFactory & factory) } -} +} \ No newline at end of file diff --git a/src/Dictionaries/FlatDictionary.h b/src/Dictionaries/FlatDictionary.h index ccd3bf9d9eb..c16547bd3b4 100644 --- a/src/Dictionaries/FlatDictionary.h +++ b/src/Dictionaries/FlatDictionary.h @@ -97,7 +97,7 @@ public: const DataTypePtr & key_type, size_t level) const override; - BlockInputStreamPtr getBlockInputStream(const Names & column_names, size_t max_block_size) const override; + Pipe read(const Names & column_names, size_t max_block_size) const override; private: template @@ -178,4 +178,4 @@ private: BlockPtr update_field_loaded_block; }; -} +} \ No newline at end of file diff --git a/src/Dictionaries/HashedDictionary.cpp b/src/Dictionaries/HashedDictionary.cpp index d65338b9a4b..33e06de23bf 100644 --- a/src/Dictionaries/HashedDictionary.cpp +++ b/src/Dictionaries/HashedDictionary.cpp @@ -637,7 +637,7 @@ void HashedDictionary::calculateBytesAllocated() } template -BlockInputStreamPtr HashedDictionary::getBlockInputStream(const Names & column_names, size_t max_block_size) const +Pipe HashedDictionary::read(const Names & column_names, size_t max_block_size) const { PaddedPODArray keys; @@ -667,9 +667,9 @@ BlockInputStreamPtr HashedDictionary::getBlockInput } if constexpr (dictionary_key_type == DictionaryKeyType::simple) - return std::make_shared(shared_from_this(), max_block_size, std::move(keys), column_names); + return Pipe(std::make_shared(DictionarySourceData(shared_from_this(), std::move(keys), column_names), max_block_size)); else - return std::make_shared(shared_from_this(), max_block_size, keys, column_names); + return Pipe(std::make_shared(DictionarySourceData(shared_from_this(), keys, column_names), max_block_size)); } template @@ -767,4 +767,4 @@ void registerDictionaryHashed(DictionaryFactory & factory) } -} +} \ No newline at end of file diff --git a/src/Dictionaries/HashedDictionary.h b/src/Dictionaries/HashedDictionary.h index 842e49aa8f0..82e8a91b603 100644 --- a/src/Dictionaries/HashedDictionary.h +++ b/src/Dictionaries/HashedDictionary.h @@ -116,7 +116,7 @@ public: const DataTypePtr & key_type, size_t level) const override; - BlockInputStreamPtr getBlockInputStream(const Names & column_names, size_t max_block_size) const override; + Pipe read(const Names & column_names, size_t max_block_size) const override; private: template @@ -225,4 +225,4 @@ extern template class HashedDictionary; extern template class HashedDictionary; extern template class HashedDictionary; -} +} \ No newline at end of file diff --git a/src/Dictionaries/IDictionary.h b/src/Dictionaries/IDictionary.h index 5467a673503..f9e0223a698 100644 --- a/src/Dictionaries/IDictionary.h +++ b/src/Dictionaries/IDictionary.h @@ -191,7 +191,7 @@ struct IDictionary : public IExternalLoadable getDictionaryID().getNameForLogs()); } - virtual BlockInputStreamPtr getBlockInputStream(const Names & column_names, size_t max_block_size) const = 0; + virtual Pipe read(const Names & column_names, size_t max_block_size) const = 0; bool supportUpdates() const override { return true; } diff --git a/src/Dictionaries/IDictionarySource.h b/src/Dictionaries/IDictionarySource.h index 42b35c95062..661f5b8eeb8 100644 --- a/src/Dictionaries/IDictionarySource.h +++ b/src/Dictionaries/IDictionarySource.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include #include #include @@ -21,28 +21,30 @@ class IDictionarySource { public: - /// Returns an input stream with all the data available from this source. - virtual BlockInputStreamPtr loadAll() = 0; + /// Returns a pipe with all the data available from this source. + virtual Pipe loadAll() = 0; - /// Returns an input stream with updated data available from this source. - virtual BlockInputStreamPtr loadUpdatedAll() = 0; + /// Returns a pipe with updated data available from this source. + virtual Pipe loadUpdatedAll() = 0; /** * result_size_hint - approx number of rows in the stream. - * Returns an input stream with all the data available from this source. + * Returns a pipe with all the data available from this source. * * NOTE: result_size_hint may be changed during you are reading (usually it * will be non zero for the first block and zero for others, since it uses - * Progress::total_rows_approx,) from the input stream, and may be called + * Progress::total_rows_approx,) from the pipe, and may be called * in parallel, so you should use something like this: * * ... * std::atomic new_size = 0; * - * auto stream = source->loadAll(&new_size); - * stream->readPrefix(); + * QueryPipeline pipeline; + * pipeline.init(source->loadAll(&new_size)); + * PullingPipelineExecutor executor; * - * while (const auto block = stream->read()) + * Block block; + * while (executor.pull(block)) * { * if (new_size) * { @@ -56,10 +58,9 @@ public: * } * } * - * stream->readSuffix(); * ... */ - virtual BlockInputStreamPtr loadAllWithSizeHint(std::atomic * /* result_size_hint */) + virtual Pipe loadAllWithSizeHint(std::atomic * /* result_size_hint */) { return loadAll(); } @@ -72,13 +73,13 @@ public: /** Returns an input stream with the data for a collection of identifiers. * It must be guaranteed, that 'ids' array will live at least until all data will be read from returned stream. */ - virtual BlockInputStreamPtr loadIds(const std::vector & ids) = 0; + virtual Pipe loadIds(const std::vector & ids) = 0; /** Returns an input stream with the data for a collection of composite keys. * `requested_rows` contains indices of all rows containing unique keys. * It must be guaranteed, that 'requested_rows' array will live at least until all data will be read from returned stream. */ - virtual BlockInputStreamPtr loadKeys(const Columns & key_columns, const std::vector & requested_rows) = 0; + virtual Pipe loadKeys(const Columns & key_columns, const std::vector & requested_rows) = 0; /// indicates whether the source has been modified since last load* operation virtual bool isModified() const = 0; diff --git a/src/Dictionaries/readInvalidateQuery.cpp b/src/Dictionaries/readInvalidateQuery.cpp index 4664b61bfc4..bd1ec0e0983 100644 --- a/src/Dictionaries/readInvalidateQuery.cpp +++ b/src/Dictionaries/readInvalidateQuery.cpp @@ -1,5 +1,6 @@ #include "readInvalidateQuery.h" -#include +#include +#include #include #include @@ -14,11 +15,18 @@ namespace ErrorCodes extern const int RECEIVED_EMPTY_DATA; } -std::string readInvalidateQuery(IBlockInputStream & block_input_stream) +std::string readInvalidateQuery(Pipe pipe) { - block_input_stream.readPrefix(); + QueryPipeline pipeline; + pipeline.init(std::move(pipe)); + + PullingPipelineExecutor executor(pipeline); + + Block block; + while (executor.pull(block)) + if (block) + break; - Block block = block_input_stream.read(); if (!block) throw Exception(ErrorCodes::RECEIVED_EMPTY_DATA, "Empty response"); @@ -36,11 +44,10 @@ std::string readInvalidateQuery(IBlockInputStream & block_input_stream) auto & column_type = block.getByPosition(0); column_type.type->getDefaultSerialization()->serializeTextQuoted(*column_type.column->convertToFullColumnIfConst(), 0, out, FormatSettings()); - while ((block = block_input_stream.read())) + while (executor.pull(block)) if (block.rows() > 0) throw Exception(ErrorCodes::TOO_MANY_ROWS, "Expected single row in resultset, got at least {}", std::to_string(rows + 1)); - block_input_stream.readSuffix(); return out.str(); } diff --git a/src/Dictionaries/readInvalidateQuery.h b/src/Dictionaries/readInvalidateQuery.h index 0a259e1ec24..61d5b29dc89 100644 --- a/src/Dictionaries/readInvalidateQuery.h +++ b/src/Dictionaries/readInvalidateQuery.h @@ -1,13 +1,13 @@ #pragma once -#include - #include namespace DB { +class Pipe; + /// Using in MySQLDictionarySource and XDBCDictionarySource after processing invalidate_query. -std::string readInvalidateQuery(IBlockInputStream & block_input_stream); +std::string readInvalidateQuery(Pipe pipe); } From 0cac0fd5e5b890e85757463f6ef55b92845553c8 Mon Sep 17 00:00:00 2001 From: sevirov <72220289+sevirov@users.noreply.github.com> Date: Wed, 4 Aug 2021 22:41:12 +0300 Subject: [PATCH 145/599] Update docs/ru/engines/table-engines/integrations/materialized-postgresql.md Co-authored-by: Anna <42538400+adevyatova@users.noreply.github.com> --- .../table-engines/integrations/materialized-postgresql.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/engines/table-engines/integrations/materialized-postgresql.md b/docs/ru/engines/table-engines/integrations/materialized-postgresql.md index 6d599bd8d86..34dc0a893fe 100644 --- a/docs/ru/engines/table-engines/integrations/materialized-postgresql.md +++ b/docs/ru/engines/table-engines/integrations/materialized-postgresql.md @@ -7,7 +7,7 @@ toc_title: MaterializedPostgreSQL Создает таблицу ClickHouse с исходным дампом данных таблицы PostgreSQL и запускает процесс репликации, т.е. выполняется применение новых изменений в фоне, как эти изменения происходят в таблице PostgreSQL в удаленной базе данных PostgreSQL. -Если требуется более одной таблицы, то очень рекомендуется использовать движок баз данных [MaterializedPostgreSQL](../../../engines/database-engines/materialized-postgresql.md) вместо движка таблиц и использовать настройку [materialized_postgresql_tables_list](../../../operations/settings/settings.md#materialized-postgresql-tables-list), с помощью которой указать таблицы, которые нужно реплицировать. Это будет намного лучше с точки зрения нагрузки на процессор, меньшего количества подключений и меньшего количества слотов репликации внутри удаленной базы данных PostgreSQL. +Если требуется более одной таблицы, вместо движка таблиц рекомендуется использовать движок баз данных [MaterializedPostgreSQL](../../../engines/database-engines/materialized-postgresql.md) и с помощью настройки [materialized_postgresql_tables_list](../../../operations/settings/settings.md#materialized-postgresql-tables-list) указывать таблицы, которые нужно реплицировать. Это будет намного лучше с точки зрения нагрузки на процессор, уменьшит количество подключений и количество слотов репликации внутри удаленной базы данных PostgreSQL. ## Создание таблицы {#creating-a-table} From b23291df5637e0b570fe5009dc9745e750aa7ffa Mon Sep 17 00:00:00 2001 From: sevirov <72220289+sevirov@users.noreply.github.com> Date: Wed, 4 Aug 2021 22:41:42 +0300 Subject: [PATCH 146/599] Update docs/ru/operations/settings/settings.md Co-authored-by: Anna <42538400+adevyatova@users.noreply.github.com> --- docs/ru/operations/settings/settings.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index 547060ae970..377a9596e7a 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -3143,7 +3143,7 @@ SETTINGS index_granularity = 8192 │ ## materialized_postgresql_allow_automatic_update {#materialized-postgresql-allow-automatic-update} -Позволяет автоматически обновить таблицу в фоновом режиме при обнаружении изменений схемы. DDL-запросы на стороне сервера PostgreSQL не реплицируются с помощью движка ClickHouse [MaterializedPostgreSQL](../../engines/database-engines/materialized-postgresql.md), поскольку это запрещено протоколом логической репликации PostgreSQL, но факт DDL-измененений обнаруживается транзакционно. В этом случае по умолчанию прекращается репликация этих таблиц после обнаружения DDL. Однако, если эта настройка включена, то вместо остановки репликации этих таблиц они будут перезагружены в фоновом режиме с помощью снимка базы данных без потери информации, и репликация для них будет продолжена. +Позволяет автоматически обновить таблицу в фоновом режиме при обнаружении изменений схемы. DDL-запросы на стороне сервера PostgreSQL не реплицируются с помощью движка ClickHouse [MaterializedPostgreSQL](../../engines/database-engines/materialized-postgresql.md), поскольку это запрещено протоколом логической репликации PostgreSQL, но факт DDL-измененений обнаруживается транзакционно. После обнаружения DDL по умолчанию прекращается репликация этих таблиц. Однако, если эта настройка включена, то вместо остановки репликации, таблицы будут перезагружены в фоновом режиме с помощью снимка базы данных без потери информации, и репликация для них будет продолжена. Возможные значения: From 54e260ab121dfbf93f92b14bc7add3f33ae8457f Mon Sep 17 00:00:00 2001 From: Alexey Date: Wed, 4 Aug 2021 20:52:55 +0000 Subject: [PATCH 147/599] Atomic engine updates Links added Minor fixes. Added ru settings default_replica_path default_replica_name --- docs/en/engines/database-engines/atomic.md | 12 ++++++------ docs/ru/engines/database-engines/atomic.md | 12 ++++++------ .../settings.md | 19 +++++++++++++++++++ 3 files changed, 31 insertions(+), 12 deletions(-) diff --git a/docs/en/engines/database-engines/atomic.md b/docs/en/engines/database-engines/atomic.md index bdab87aa4b1..dd9689d3e10 100644 --- a/docs/en/engines/database-engines/atomic.md +++ b/docs/en/engines/database-engines/atomic.md @@ -5,12 +5,12 @@ toc_title: Atomic # Atomic {#atomic} -It supports non-blocking [DROP TABLE](#drop-detach-table) and [RENAME TABLE](#rename-table) queries and atomic [EXCHANGE TABLES t1 AND t2](#exchange-tables) queries. `Atomic` database engine is used by default. +It supports non-blocking [DROP TABLE](#drop-detach-table) and [RENAME TABLE](#rename-table) queries and atomic [EXCHANGE TABLES](#exchange-tables) queries. `Atomic` database engine is used by default. ## Creating a Database {#creating-a-database} ``` sql - CREATE DATABASE test[ ENGINE = Atomic]; +CREATE DATABASE test [ENGINE = Atomic]; ``` ## Specifics and recommendations {#specifics-and-recommendations} @@ -25,16 +25,16 @@ CREATE TABLE name UUID '28f1c61c-2970-457a-bffe-454156ddcfef' (n UInt64) ENGINE ``` ### RENAME TABLE {#rename-table} -`RENAME` queries are performed without changing UUID and moving table data. These queries do not wait for the completion of queries using the table and will be executed instantly. +[RENAME](../../sql-reference/statements/rename.md) queries are performed without changing UUID and moving table data. These queries do not wait for the completion of queries using the table and will be executed instantly. ### DROP/DETACH TABLE {#drop-detach-table} On `DROP TABLE` no data is removed, database `Atomic` just marks table as dropped by moving metadata to `/clickhouse_path/metadata_dropped/` and notifies background thread. Delay before final table data deletion is specify by [database_atomic_delay_before_drop_table_sec](../../operations/server-configuration-parameters/settings.md#database_atomic_delay_before_drop_table_sec) setting. You can specify synchronous mode using `SYNC` modifier. Use the [database_atomic_wait_for_drop_and_detach_synchronously](../../operations/settings/settings.md#database_atomic_wait_for_drop_and_detach_synchronously) setting to do this. In this case `DROP` waits for running `SELECT`, `INSERT` and other queries which are using the table to finish. Table will be actually removed when it's not in use. -### EXCHANGE TABLES {#exchange-tables} +### EXCHANGE TABLES/DICTIONARIES {#exchange-tables} -`EXCHANGE` query swaps tables atomically. So instead of this non-atomic operation: +[EXCHANGE](../../sql-reference/statements/exchange.md) query swaps tables or dictionaries atomically. For instance, instead of this non-atomic operation: ```sql RENAME TABLE new_table TO tmp, old_table TO new_table, tmp TO old_table; @@ -47,7 +47,7 @@ EXCHANGE TABLES new_table AND old_table; ### ReplicatedMergeTree in Atomic Database {#replicatedmergetree-in-atomic-database} -For [ReplicatedMergeTree](../table-engines/mergetree-family/replication.md#table_engines-replication) tables, it is recommended to not specify engine parameters - path in ZooKeeper and replica name. In this case, configuration parameters will be used [default_replica_path](../../operations/server-configuration-parameters/settings.md#default_replica_path) and [default_replica_name](../../operations/server-configuration-parameters/settings.md#default_replica_name). If you want to specify engine parameters explicitly, it is recommended to use `{uuid}` macros. This is useful so that unique paths are automatically generated for each table in ZooKeeper. +For [ReplicatedMergeTree](../table-engines/mergetree-family/replication.md#table_engines-replication) tables, it is recommended not to specify engine parameters - path in ZooKeeper and replica name. In this case, configuration parameters [default_replica_path](../../operations/server-configuration-parameters/settings.md#default_replica_path) and [default_replica_name](../../operations/server-configuration-parameters/settings.md#default_replica_name) will be used. If you want to specify engine parameters explicitly, it is recommended to use `{uuid}` macros. This is useful so that unique paths are automatically generated for each table in ZooKeeper. ## See Also diff --git a/docs/ru/engines/database-engines/atomic.md b/docs/ru/engines/database-engines/atomic.md index ecdd809b6ec..5578b6d7422 100644 --- a/docs/ru/engines/database-engines/atomic.md +++ b/docs/ru/engines/database-engines/atomic.md @@ -5,12 +5,12 @@ toc_title: Atomic # Atomic {#atomic} -Поддерживает неблокирующие запросы [DROP TABLE](#drop-detach-table) и [RENAME TABLE](#rename-table) и атомарные запросы [EXCHANGE TABLES t1 AND t](#exchange-tables). Движок `Atomic` используется по умолчанию. +Поддерживает неблокирующие запросы [DROP TABLE](#drop-detach-table) и [RENAME TABLE](#rename-table) и атомарные запросы [EXCHANGE TABLES](#exchange-tables). Движок `Atomic` используется по умолчанию. ## Создание БД {#creating-a-database} ``` sql - CREATE DATABASE test[ ENGINE = Atomic]; +CREATE DATABASE test [ENGINE = Atomic]; ``` ## Особенности и рекомендации {#specifics-and-recommendations} @@ -25,16 +25,16 @@ CREATE TABLE name UUID '28f1c61c-2970-457a-bffe-454156ddcfef' (n UInt64) ENGINE ``` ### RENAME TABLE {#rename-table} -Запросы `RENAME` выполняются без изменения UUID и перемещения табличных данных. Эти запросы не ожидают завершения использующих таблицу запросов и будут выполнены мгновенно. +Запросы [RENAME](../../sql-reference/statements/rename.md) выполняются без изменения UUID и перемещения табличных данных. Эти запросы не ожидают завершения использующих таблицу запросов и будут выполнены мгновенно. ### DROP/DETACH TABLE {#drop-detach-table} При выполнении запроса `DROP TABLE` никакие данные не удаляются. Таблица помечается как удаленная, метаданные перемещаются в папку `/clickhouse_path/metadata_dropped/` и база данных уведомляет фоновый поток. Задержка перед окончательным удалением данных задается настройкой [database_atomic_delay_before_drop_table_sec](../../operations/server-configuration-parameters/settings.md#database_atomic_delay_before_drop_table_sec). Вы можете задать синхронный режим, определяя модификатор `SYNC`. Используйте для этого настройку [database_atomic_wait_for_drop_and_detach_synchronously](../../operations/settings/settings.md#database_atomic_wait_for_drop_and_detach_synchronously). В этом случае запрос `DROP` ждет завершения `SELECT`, `INSERT` и других запросов, которые используют таблицу. Таблица будет фактически удалена, когда она не будет использоваться. -### EXCHANGE TABLES {#exchange-tables} +### EXCHANGE TABLES/DICTIONARIES {#exchange-tables} -Запрос `EXCHANGE` меняет местами две таблицы атомарно. Вместо неатомарной операции: +Запрос [EXCHANGE](../../sql-reference/statements/exchange.md) атомарно меняет местами две таблицы или два словаря. Например, вместо неатомарной операции: ```sql RENAME TABLE new_table TO tmp, old_table TO new_table, tmp TO old_table; @@ -47,7 +47,7 @@ EXCHANGE TABLES new_table AND old_table; ### ReplicatedMergeTree in Atomic Database {#replicatedmergetree-in-atomic-database} -Для таблиц [ReplicatedMergeTree](../table-engines/mergetree-family/replication.md#table_engines-replication) рекомендуется не указывать параметры движка - путь в ZooKeeper и имя реплики. В этом случае будут использоваться параметры конфигурации: [default_replica_path](../../operations/server-configuration-parameters/settings.md#default_replica_path) и [default_replica_name](../../operations/server-configuration-parameters/settings.md#default_replica_name). Если вы хотите определить параметры движка явно, рекомендуется использовать макрос {uuid}. Это удобно, так как автоматически генерируются уникальные пути для каждой таблицы в ZooKeeper. +Для таблиц [ReplicatedMergeTree](../table-engines/mergetree-family/replication.md#table_engines-replication) рекомендуется не указывать параметры движка - путь в ZooKeeper и имя реплики. В этом случае будут использоваться параметры конфигурации: [default_replica_path](../../operations/server-configuration-parameters/settings.md#default_replica_path) и [default_replica_name](../../operations/server-configuration-parameters/settings.md#default_replica_name). Если вы хотите определить параметры движка явно, рекомендуется использовать макрос `{uuid}`. Это удобно, так как автоматически генерируются уникальные пути для каждой таблицы в ZooKeeper. ## Смотрите также diff --git a/docs/ru/operations/server-configuration-parameters/settings.md b/docs/ru/operations/server-configuration-parameters/settings.md index 6b4e25eb692..a8ae3f7eb3e 100644 --- a/docs/ru/operations/server-configuration-parameters/settings.md +++ b/docs/ru/operations/server-configuration-parameters/settings.md @@ -134,6 +134,25 @@ ClickHouse проверяет условия для `min_part_size` и `min_part default ``` +## default_replica_path {#default_replica_path} + +Путь к таблице в ZooKeeper. + +**Пример** + +``` xml +/clickhouse/tables/{uuid}/{shard} +``` +## default_replica_name {#default_replica_name} + +Имя реплики в ZooKeeper. + +**Пример** + +``` xml +{replica} +``` + ## dictionaries_config {#server_configuration_parameters-dictionaries_config} Путь к конфигурации внешних словарей. From 4f9f055eb99085585d6b7ecdd7d98b29b6c7ed12 Mon Sep 17 00:00:00 2001 From: Alexey Date: Wed, 4 Aug 2021 20:53:43 +0000 Subject: [PATCH 148/599] ru updated --- docs/ru/sql-reference/statements/detach.md | 2 +- docs/ru/sql-reference/statements/exchange.md | 4 ++-- docs/ru/sql-reference/statements/rename.md | 4 ++-- 3 files changed, 5 insertions(+), 5 deletions(-) diff --git a/docs/ru/sql-reference/statements/detach.md b/docs/ru/sql-reference/statements/detach.md index cf9df619d3b..71b4012104b 100644 --- a/docs/ru/sql-reference/statements/detach.md +++ b/docs/ru/sql-reference/statements/detach.md @@ -15,7 +15,7 @@ DETACH TABLE|VIEW|DICTIONARY [IF EXISTS] [db.]name [ON CLUSTER cluster] [PERMANE Такой запрос не удаляет ни данные, ни метаданные таблицы, материализованного представления или словаря. Если отключение не было перманентным (запрос без ключевого слова `PERMANENTLY`), то при следующем запуске сервер прочитает метаданные и снова узнает о таблице/представлении/словаре. Если сущность была отключена перманентно, то сервер не подключит их обратно автоматически. -Независимо от того, каким способом таблица была отключена, ее можно подключить обратно с помощью запроса [ATTACH](../../sql-reference/statements/attach.md). Системные log таблицы также могут быть подключены обратно (к примеру `query_log`, `text_log` и др.) Другие системные таблицы не могут быть подключены обратно, но на следующем запуске сервер снова "вспомнит" об этих таблицах. +Независимо от того, каким способом таблица была отключена, ее можно подключить обратно с помощью запроса [ATTACH](../../sql-reference/statements/attach.md). Системные log таблицы также могут быть подключены обратно (к примеру, `query_log`, `text_log` и др.). Другие системные таблицы не могут быть подключены обратно, но на следующем запуске сервер снова "вспомнит" об этих таблицах. `ATTACH MATERIALIZED VIEW` не может быть использован с кратким синтаксисом (без `SELECT`), но можно подключить представление с помощью запроса `ATTACH TABLE`. diff --git a/docs/ru/sql-reference/statements/exchange.md b/docs/ru/sql-reference/statements/exchange.md index eb66aee62c8..e809fc5863c 100644 --- a/docs/ru/sql-reference/statements/exchange.md +++ b/docs/ru/sql-reference/statements/exchange.md @@ -5,8 +5,8 @@ toc_title: EXCHANGE # Выражение EXCHANGE {#exchange} -Атомарно меняет друг с другом имена двух таблиц или словарей. -Это действие также можно выполнить с помощью запроса [RENAME](./rename.md), использую третье временное имя, но в таком случае действие неатомарно. +Атомарно обменивает имена двух таблиц или словарей. +Это действие также можно выполнить с помощью запроса [RENAME](./rename.md), используя третье временное имя, но в таком случае действие неатомарно. !!! note "Примечание" Запрос `EXCHANGE` поддерживается только движком баз данных [Atomic](../../engines/database-engines/atomic.md). diff --git a/docs/ru/sql-reference/statements/rename.md b/docs/ru/sql-reference/statements/rename.md index b043ec305e0..727c9d96615 100644 --- a/docs/ru/sql-reference/statements/rename.md +++ b/docs/ru/sql-reference/statements/rename.md @@ -6,7 +6,7 @@ toc_title: RENAME # Выражение RENAME {#misc_operations-rename} Переименовывает базы данных, таблицы или словари. Несколько сущностей могут быть переименованы в одном запросе. -Обратите внимание, что запрос `RENAME` с несколькими сущностями это не атомарная операция. Чтобы обменять имена атомарно, используйте выражение [EXCHANGE](./exchange.md). +Обратите внимание, что запрос `RENAME` с несколькими сущностями это неатомарная операция. Чтобы обменять имена атомарно, используйте выражение [EXCHANGE](./exchange.md). !!! note "Примечание" Запрос `RENAME` поддерживается только движком баз данных [Atomic](../../engines/database-engines/atomic.md). @@ -31,7 +31,7 @@ RENAME DATABASE atomic_database1 TO atomic_database2 [,...] [ON CLUSTER cluster] Переименовывает одну или несколько таблиц. -Переименовывание таблиц является лёгкой операцией. Если вы указали после `TO` другую базу данных, то таблица будет перенесена в эту базу данных. При этом директории с базами данных должны быть расположены в одной файловой системе. Иначе возвращается ошибка. Если переименовывается несколько таблиц в одном запросе, то такая операция неатомарная. Она может выполнится частично, и запросы в других сессиях могут получить ошибку `Table ... doesn't exist...`. +Переименовывание таблиц является лёгкой операцией. Если вы указали после `TO` другую базу данных, то таблица будет перенесена в эту базу данных. При этом директории с базами данных должны быть расположены в одной файловой системе, иначе возвращается ошибка. Если переименовывается несколько таблиц в одном запросе, то такая операция неатомарная. Она может выполнится частично, и запросы в других сессиях могут получить ошибку `Table ... doesn't exist...`. **Синтаксис** From 617cdd5d4d6082b15e15b8e6cb63f8a3c2cd2db6 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 4 Aug 2021 10:11:25 +0300 Subject: [PATCH 149/599] check-style: add a check for ReplicatedMergeTree in *.sql --- utils/check-style/check-style | 10 ++-------- 1 file changed, 2 insertions(+), 8 deletions(-) diff --git a/utils/check-style/check-style b/utils/check-style/check-style index 551368e5fd8..e541d2cbfb1 100755 --- a/utils/check-style/check-style +++ b/utils/check-style/check-style @@ -132,14 +132,8 @@ for test_case in "${tests_with_replicated_merge_tree[@]}"; do grep -q -e "ReplicatedMergeTree.*$test_case_zk_prefix" "$test_case" || echo "ReplicatedMergeTree should contain '$test_case_zk_prefix' in zookeeper path to avoid overlaps ($test_case)" ;; *.sql) - # NOTE: *.sql is not supported because it is not possible right now, because: - # - ReplicatedMergeTree supports only ASTLiteral for zookeeper path - # (and adding support of other nodes, with evaluating them are not that easy, due to zk_prefix is "optional") - # - Hence concat(currentDatabase(), 'foo') - # - Also params cannot be used, because the are wrapped with CAST() - # - # But hopefully they will not be a problem - # (since they do not do any "stressing" and overlap probability should be lower). + test_case_zk_prefix="{[ ]*database[ ]*}" + grep -q -e "ReplicatedMergeTree.*$test_case_zk_prefix" "$test_case" || echo "ReplicatedMergeTree should contain '$test_case_zk_prefix' in zookeeper path to avoid overlaps ($test_case)" ;; *.py) # Right now there is not such tests anyway From 6c61d3731e90e91b75858a19c7d285c3d75da3c1 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 4 Aug 2021 10:24:23 +0300 Subject: [PATCH 150/599] check-style: cover *.j2 too --- utils/check-style/check-style | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/utils/check-style/check-style b/utils/check-style/check-style index e541d2cbfb1..a096076a3f0 100755 --- a/utils/check-style/check-style +++ b/utils/check-style/check-style @@ -89,7 +89,7 @@ find $ROOT_PATH/tests/queries -iname '*fail*' | # Queries to system.query_log/system.query_thread_log should have current_database = currentDatabase() condition # NOTE: it is not that accuate, but at least something. tests_with_query_log=( $( - find $ROOT_PATH/tests/queries -iname '*.sql' -or -iname '*.sh' -or -iname '*.py' | + find $ROOT_PATH/tests/queries -iname '*.sql' -or -iname '*.sh' -or -iname '*.py' -or -iname '*.j2' | grep -vP $EXCLUDE_DIRS | xargs grep --with-filename -e system.query_log -e system.query_thread_log | cut -d: -f1 | sort -u ) ) @@ -102,7 +102,7 @@ done # Queries to system.tables/system.parts/system.detached_parts/system.parts_columns/system.columns should have database = currentDatabase() condition # NOTE: it is not that accuate, but at least something. tests_with_database_column=( $( - find $ROOT_PATH/tests/queries -iname '*.sql' -or -iname '*.sh' -or -iname '*.py' | + find $ROOT_PATH/tests/queries -iname '*.sql' -or -iname '*.sh' -or -iname '*.py' -or -iname '*.j2' | grep -vP $EXCLUDE_DIRS | grep -v -x -e $ROOT_PATH/tests/queries/query_test.py | xargs grep --with-filename -e system.tables -e system.parts -e system.detached_parts -e system.parts_columns -e system.columns | cut -d: -f1 | sort -u @@ -121,7 +121,7 @@ done # Queries with ReplicatedMergeTree # NOTE: it is not that accuate, but at least something. tests_with_replicated_merge_tree=( $( - find $ROOT_PATH/tests/queries -iname '*.sql' -or -iname '*.sh' -or -iname '*.py' | + find $ROOT_PATH/tests/queries -iname '*.sql' -or -iname '*.sh' -or -iname '*.py' -or -iname '*.j2' | grep -vP $EXCLUDE_DIRS | xargs grep --with-filename -e ReplicatedMergeTree | cut -d: -f1 | sort -u ) ) @@ -131,7 +131,7 @@ for test_case in "${tests_with_replicated_merge_tree[@]}"; do test_case_zk_prefix="\$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX" grep -q -e "ReplicatedMergeTree.*$test_case_zk_prefix" "$test_case" || echo "ReplicatedMergeTree should contain '$test_case_zk_prefix' in zookeeper path to avoid overlaps ($test_case)" ;; - *.sql) + *.sql|*.sql.j2) test_case_zk_prefix="{[ ]*database[ ]*}" grep -q -e "ReplicatedMergeTree.*$test_case_zk_prefix" "$test_case" || echo "ReplicatedMergeTree should contain '$test_case_zk_prefix' in zookeeper path to avoid overlaps ($test_case)" ;; @@ -180,7 +180,7 @@ find $ROOT_PATH/{src,programs,utils} -name '*.h' | while read file; do [[ $(head -n1 $file) != '#pragma once' ]] && echo "File $file must have '#pragma once' in first line"; done # Check for executable bit on non-executable files -find $ROOT_PATH/{src,base,programs,utils,tests,docs,website,cmake} '(' -name '*.cpp' -or -name '*.h' -or -name '*.sql' -or -name '*.xml' -or -name '*.reference' -or -name '*.txt' -or -name '*.md' ')' -and -executable | grep -P '.' && echo "These files should not be executable." +find $ROOT_PATH/{src,base,programs,utils,tests,docs,website,cmake} '(' -name '*.cpp' -or -name '*.h' -or -name '*.sql' -or -name '*.j2' -or -name '*.xml' -or -name '*.reference' -or -name '*.txt' -or -name '*.md' ')' -and -executable | grep -P '.' && echo "These files should not be executable." # Check for BOM find $ROOT_PATH/{src,base,programs,utils,tests,docs,website,cmake} -name '*.md' -or -name '*.cpp' -or -name '*.h' | xargs grep -l -F $'\xEF\xBB\xBF' | grep -P '.' && echo "Files should not have UTF-8 BOM" From a783b05755b64f51422a8aed0618739356e4011d Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 4 Aug 2021 10:28:37 +0300 Subject: [PATCH 151/599] check-style: improve check for ReplicatedMergeTree --- utils/check-style/check-style | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/utils/check-style/check-style b/utils/check-style/check-style index a096076a3f0..6ab8feab61c 100755 --- a/utils/check-style/check-style +++ b/utils/check-style/check-style @@ -129,11 +129,11 @@ for test_case in "${tests_with_replicated_merge_tree[@]}"; do case "$test_case" in *.sh) test_case_zk_prefix="\$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX" - grep -q -e "ReplicatedMergeTree.*$test_case_zk_prefix" "$test_case" || echo "ReplicatedMergeTree should contain '$test_case_zk_prefix' in zookeeper path to avoid overlaps ($test_case)" + grep -q -e "ReplicatedMergeTree[ ]*(.*$test_case_zk_prefix" "$test_case" || echo "ReplicatedMergeTree should contain '$test_case_zk_prefix' in zookeeper path to avoid overlaps ($test_case)" ;; *.sql|*.sql.j2) test_case_zk_prefix="{[ ]*database[ ]*}" - grep -q -e "ReplicatedMergeTree.*$test_case_zk_prefix" "$test_case" || echo "ReplicatedMergeTree should contain '$test_case_zk_prefix' in zookeeper path to avoid overlaps ($test_case)" + grep -q -e "ReplicatedMergeTree[ ]*(.*$test_case_zk_prefix" "$test_case" || echo "ReplicatedMergeTree should contain '$test_case_zk_prefix' in zookeeper path to avoid overlaps ($test_case)" ;; *.py) # Right now there is not such tests anyway From 507eae9aed045395cde49569cfd324cce50c9df9 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 4 Aug 2021 10:32:51 +0300 Subject: [PATCH 152/599] check-style: skip *.gen for ReplicatedMergeTree check --- utils/check-style/check-style | 2 ++ 1 file changed, 2 insertions(+) diff --git a/utils/check-style/check-style b/utils/check-style/check-style index 6ab8feab61c..d338dd4a2f3 100755 --- a/utils/check-style/check-style +++ b/utils/check-style/check-style @@ -127,6 +127,8 @@ tests_with_replicated_merge_tree=( $( ) ) for test_case in "${tests_with_replicated_merge_tree[@]}"; do case "$test_case" in + *.gen.*) + ;; *.sh) test_case_zk_prefix="\$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX" grep -q -e "ReplicatedMergeTree[ ]*(.*$test_case_zk_prefix" "$test_case" || echo "ReplicatedMergeTree should contain '$test_case_zk_prefix' in zookeeper path to avoid overlaps ($test_case)" From 09fa82c24446bdc330a731d7417f5c49cadee7db Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 5 Aug 2021 09:58:04 +0300 Subject: [PATCH 153/599] check-style: fix {database} macro pattern --- utils/check-style/check-style | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/utils/check-style/check-style b/utils/check-style/check-style index d338dd4a2f3..1f2354efe5a 100755 --- a/utils/check-style/check-style +++ b/utils/check-style/check-style @@ -134,7 +134,7 @@ for test_case in "${tests_with_replicated_merge_tree[@]}"; do grep -q -e "ReplicatedMergeTree[ ]*(.*$test_case_zk_prefix" "$test_case" || echo "ReplicatedMergeTree should contain '$test_case_zk_prefix' in zookeeper path to avoid overlaps ($test_case)" ;; *.sql|*.sql.j2) - test_case_zk_prefix="{[ ]*database[ ]*}" + test_case_zk_prefix="{database}" grep -q -e "ReplicatedMergeTree[ ]*(.*$test_case_zk_prefix" "$test_case" || echo "ReplicatedMergeTree should contain '$test_case_zk_prefix' in zookeeper path to avoid overlaps ($test_case)" ;; *.py) From 3cda5960ec14751fbb4e12b9131f0b75f9b18c7a Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 5 Aug 2021 10:07:32 +0300 Subject: [PATCH 154/599] check-style: allow currentDatabase() for ReplicatedMergeTree definition too --- utils/check-style/check-style | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/utils/check-style/check-style b/utils/check-style/check-style index 1f2354efe5a..05b8a1485e4 100755 --- a/utils/check-style/check-style +++ b/utils/check-style/check-style @@ -134,7 +134,7 @@ for test_case in "${tests_with_replicated_merge_tree[@]}"; do grep -q -e "ReplicatedMergeTree[ ]*(.*$test_case_zk_prefix" "$test_case" || echo "ReplicatedMergeTree should contain '$test_case_zk_prefix' in zookeeper path to avoid overlaps ($test_case)" ;; *.sql|*.sql.j2) - test_case_zk_prefix="{database}" + test_case_zk_prefix="\({database}\|currentDatabase()\)" grep -q -e "ReplicatedMergeTree[ ]*(.*$test_case_zk_prefix" "$test_case" || echo "ReplicatedMergeTree should contain '$test_case_zk_prefix' in zookeeper path to avoid overlaps ($test_case)" ;; *.py) From bc0af176feeecd504d1ff86a624dbc8358850c0e Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 5 Aug 2021 09:55:07 +0300 Subject: [PATCH 155/599] tests/stateless: add {database} macro for ReplicatedMergeTree in *.sql --- ...cated_merge_tree_alter_zookeeper.reference | 48 +++++++++---------- ..._replicated_merge_tree_alter_zookeeper.sql | 4 +- .../00083_create_merge_tree_zookeeper.sql | 4 +- .../00121_drop_column_zookeeper.sql | 4 +- .../00215_primary_key_order_zookeeper.sql | 2 +- ...per_deduplication_and_unexpected_parts.sql | 2 +- ...eplicated_drop_on_non_leader_zookeeper.sql | 4 +- ...46_clear_column_in_partition_zookeeper.sql | 4 +- ...stom_partitioning_replicated_zookeeper.sql | 20 ++++---- ...ed_storage_definition_syntax_zookeeper.sql | 4 +- ...lized_view_and_deduplication_zookeeper.sql | 4 +- ...ication_after_drop_partition_zookeeper.sql | 2 +- ...00563_insert_into_remote_and_zookeeper.sql | 2 +- ...23_replicated_truncate_table_zookeeper.sql | 4 +- .../00643_cast_zookeeper.reference | 2 +- .../0_stateless/00643_cast_zookeeper.sql | 4 +- ...identical_result_after_merge_zookeeper.sql | 4 +- .../0_stateless/00725_comment_columns.sql | 2 +- ...sert_have_data_before_quorum_zookeeper.sql | 4 +- ...ert_lost_part_and_alive_part_zookeeper.sql | 4 +- ...0732_quorum_insert_lost_part_zookeeper.sql | 4 +- ..._old_data_and_without_quorum_zookeeper.sql | 4 +- ...m_insert_simple_test_1_parts_zookeeper.sql | 4 +- ...m_insert_simple_test_2_parts_zookeeper.sql | 4 +- .../00753_comment_columns_zookeeper.reference | 4 +- .../00753_comment_columns_zookeeper.sql | 2 +- ...r_modify_order_by_replicated_zookeeper.sql | 2 +- ...dices_alter_replicated_zookeeper.reference | 24 +++++----- ...836_indices_alter_replicated_zookeeper.sql | 8 ++-- ...0837_minmax_index_replicated_zookeeper.sql | 4 +- ...om_compression_codecs_replicated.reference | 2 +- ...r_custom_compression_codecs_replicated.sql | 12 ++--- ...per_test_alter_compression_codecs_long.sql | 4 +- ...y_replicated_merge_tree_optimize_final.sql | 4 +- ...ndex_granularity_replicated_merge_tree.sql | 12 ++--- .../00933_ttl_replicated_zookeeper.reference | 2 +- .../00933_ttl_replicated_zookeeper.sql | 4 +- ...0988_constraints_replication_zookeeper.sql | 4 +- .../01037_zookeeper_check_table_empty_pk.sql | 2 +- .../01049_zookeeper_synchronous_mutations.sql | 4 +- .../01062_alter_on_mutataion_zookeeper.sql | 4 +- .../01079_alter_default_zookeeper.reference | 16 +++---- .../01079_alter_default_zookeeper.sql | 2 +- ..._zookeeper_mutations_and_insert_quorum.sql | 4 +- ...1135_default_and_alter_zookeeper.reference | 2 +- .../01135_default_and_alter_zookeeper.sql | 2 +- ...lumn_compact_part_replicated_zookeeper.sql | 2 +- ...213_alter_rename_primary_key_zookeeper.sql | 4 +- ...er_rename_with_default_zookeeper.reference | 6 +-- ...13_alter_rename_with_default_zookeeper.sql | 2 +- ...67_alter_default_key_columns_zookeeper.sql | 4 +- ...name_column_constraint_zookeeper.reference | 4 +- ...ter_rename_column_constraint_zookeeper.sql | 2 +- .../01319_manual_write_to_replicas.sql | 4 +- ...num_partition_key_replicated_zookeeper.sql | 4 +- .../01378_alter_rename_with_ttl_zookeeper.sql | 2 +- .../01430_modify_sample_by_zookeeper.sql | 2 +- ...1451_replicated_detach_drop_and_quorum.sql | 4 +- ...01451_replicated_detach_drop_part_long.sql | 4 +- ...ter_remove_no_property_zookeeper.reference | 4 +- ...493_alter_remove_no_property_zookeeper.sql | 2 +- ...lter_remove_properties_zookeeper.reference | 24 +++++----- ...1493_alter_remove_properties_zookeeper.sql | 4 +- ...509_parallel_quorum_insert_no_replicas.sql | 4 +- ...12_create_replicate_merge_tree_one_arg.sql | 2 +- ..._select_sequence_consistency_zookeeper.sql | 6 +-- .../01530_drop_database_atomic_sync.sql | 12 ++--- ...01532_execute_merges_on_single_replica.sql | 4 +- ...1581_deduplicate_by_columns_replicated.sql | 4 +- .../01646_system_restart_replicas_smoke.sql | 2 +- ..._drop_part_and_deduplication_zookeeper.sql | 10 ++-- ...0_fetch_patition_with_macro_in_zk_path.sql | 8 ++-- tests/queries/0_stateless/01666_blns.sql | 4 +- .../01669_columns_declaration_serde.sql | 4 +- .../01700_system_zookeeper_path_in.sql | 18 +++---- .../01710_projection_fetch.reference | 8 ++-- .../0_stateless/01710_projection_fetch.sql | 4 +- .../01713_table_ttl_old_syntax_zookeeper.sql | 6 +-- ...5_background_checker_blather_zookeeper.sql | 4 +- ...747_alter_partition_key_enum_zookeeper.sql | 2 +- .../01761_alter_decimal_zookeeper.reference | 4 +- .../01761_alter_decimal_zookeeper.sql | 2 +- .../01901_test_attach_partition_from.sql | 2 +- 83 files changed, 228 insertions(+), 228 deletions(-) diff --git a/tests/queries/0_stateless/00062_replicated_merge_tree_alter_zookeeper.reference b/tests/queries/0_stateless/00062_replicated_merge_tree_alter_zookeeper.reference index cb61ab3e9b9..59e3e1e4a3f 100644 --- a/tests/queries/0_stateless/00062_replicated_merge_tree_alter_zookeeper.reference +++ b/tests/queries/0_stateless/00062_replicated_merge_tree_alter_zookeeper.reference @@ -1,22 +1,22 @@ d Date k UInt64 i32 Int32 -CREATE TABLE default.replicated_alter1\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00062/alter\', \'r1\', d, k, 8192) +CREATE TABLE default.replicated_alter1\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_00062/alter\', \'r1\', d, k, 8192) d Date k UInt64 i32 Int32 -CREATE TABLE default.replicated_alter2\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00062/alter\', \'r2\', d, k, 8192) +CREATE TABLE default.replicated_alter2\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_00062/alter\', \'r2\', d, k, 8192) 2015-01-01 10 42 d Date k UInt64 i32 Int32 dt DateTime(\'UTC\') -CREATE TABLE default.replicated_alter1\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\')\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00062/alter\', \'r1\', d, k, 8192) +CREATE TABLE default.replicated_alter1\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\')\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_00062/alter\', \'r1\', d, k, 8192) d Date k UInt64 i32 Int32 dt DateTime(\'UTC\') -CREATE TABLE default.replicated_alter2\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\')\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00062/alter\', \'r2\', d, k, 8192) +CREATE TABLE default.replicated_alter2\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\')\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_00062/alter\', \'r2\', d, k, 8192) 2015-01-01 9 41 1992-01-01 08:00:00 2015-01-01 10 42 1970-01-01 00:00:00 d Date @@ -25,14 +25,14 @@ i32 Int32 dt DateTime(\'UTC\') n.ui8 Array(UInt8) n.s Array(String) -CREATE TABLE default.replicated_alter1\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `n.ui8` Array(UInt8),\n `n.s` Array(String)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00062/alter\', \'r1\', d, k, 8192) +CREATE TABLE default.replicated_alter1\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `n.ui8` Array(UInt8),\n `n.s` Array(String)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_00062/alter\', \'r1\', d, k, 8192) d Date k UInt64 i32 Int32 dt DateTime(\'UTC\') n.ui8 Array(UInt8) n.s Array(String) -CREATE TABLE default.replicated_alter2\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `n.ui8` Array(UInt8),\n `n.s` Array(String)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00062/alter\', \'r2\', d, k, 8192) +CREATE TABLE default.replicated_alter2\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `n.ui8` Array(UInt8),\n `n.s` Array(String)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_00062/alter\', \'r2\', d, k, 8192) 2015-01-01 8 40 2012-12-12 12:12:12 [1,2,3] ['12','13','14'] 2015-01-01 9 41 1992-01-01 08:00:00 [] [] 2015-01-01 10 42 1970-01-01 00:00:00 [] [] @@ -43,7 +43,7 @@ dt DateTime(\'UTC\') n.ui8 Array(UInt8) n.s Array(String) n.d Array(Date) -CREATE TABLE default.replicated_alter1\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `n.ui8` Array(UInt8),\n `n.s` Array(String),\n `n.d` Array(Date)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00062/alter\', \'r1\', d, k, 8192) +CREATE TABLE default.replicated_alter1\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `n.ui8` Array(UInt8),\n `n.s` Array(String),\n `n.d` Array(Date)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_00062/alter\', \'r1\', d, k, 8192) d Date k UInt64 i32 Int32 @@ -51,7 +51,7 @@ dt DateTime(\'UTC\') n.ui8 Array(UInt8) n.s Array(String) n.d Array(Date) -CREATE TABLE default.replicated_alter2\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `n.ui8` Array(UInt8),\n `n.s` Array(String),\n `n.d` Array(Date)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00062/alter\', \'r2\', d, k, 8192) +CREATE TABLE default.replicated_alter2\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `n.ui8` Array(UInt8),\n `n.s` Array(String),\n `n.d` Array(Date)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_00062/alter\', \'r2\', d, k, 8192) 2015-01-01 7 39 2014-07-14 13:26:50 [10,20,30] ['120','130','140'] ['2000-01-01','2000-01-01','2000-01-03'] 2015-01-01 8 40 2012-12-12 12:12:12 [1,2,3] ['12','13','14'] ['1970-01-01','1970-01-01','1970-01-01'] 2015-01-01 9 41 1992-01-01 08:00:00 [] [] [] @@ -64,7 +64,7 @@ n.ui8 Array(UInt8) n.s Array(String) n.d Array(Date) s String DEFAULT \'0\' -CREATE TABLE default.replicated_alter1\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `n.ui8` Array(UInt8),\n `n.s` Array(String),\n `n.d` Array(Date),\n `s` String DEFAULT \'0\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00062/alter\', \'r1\', d, k, 8192) +CREATE TABLE default.replicated_alter1\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `n.ui8` Array(UInt8),\n `n.s` Array(String),\n `n.d` Array(Date),\n `s` String DEFAULT \'0\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_00062/alter\', \'r1\', d, k, 8192) d Date k UInt64 i32 Int32 @@ -73,7 +73,7 @@ n.ui8 Array(UInt8) n.s Array(String) n.d Array(Date) s String DEFAULT \'0\' -CREATE TABLE default.replicated_alter2\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `n.ui8` Array(UInt8),\n `n.s` Array(String),\n `n.d` Array(Date),\n `s` String DEFAULT \'0\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00062/alter\', \'r2\', d, k, 8192) +CREATE TABLE default.replicated_alter2\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `n.ui8` Array(UInt8),\n `n.s` Array(String),\n `n.d` Array(Date),\n `s` String DEFAULT \'0\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_00062/alter\', \'r2\', d, k, 8192) 2015-01-01 6 38 2014-07-15 13:26:50 [10,20,30] ['asd','qwe','qwe'] ['2000-01-01','2000-01-01','2000-01-03'] 100500 2015-01-01 7 39 2014-07-14 13:26:50 [10,20,30] ['120','130','140'] ['2000-01-01','2000-01-01','2000-01-03'] 0 2015-01-01 8 40 2012-12-12 12:12:12 [1,2,3] ['12','13','14'] ['1970-01-01','1970-01-01','1970-01-01'] 0 @@ -86,7 +86,7 @@ dt DateTime(\'UTC\') n.ui8 Array(UInt8) n.s Array(String) s Int64 DEFAULT \'0\' -CREATE TABLE default.replicated_alter1\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `n.ui8` Array(UInt8),\n `n.s` Array(String),\n `s` Int64 DEFAULT \'0\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00062/alter\', \'r1\', d, k, 8192) +CREATE TABLE default.replicated_alter1\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `n.ui8` Array(UInt8),\n `n.s` Array(String),\n `s` Int64 DEFAULT \'0\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_00062/alter\', \'r1\', d, k, 8192) d Date k UInt64 i32 Int32 @@ -94,7 +94,7 @@ dt DateTime(\'UTC\') n.ui8 Array(UInt8) n.s Array(String) s Int64 DEFAULT \'0\' -CREATE TABLE default.replicated_alter2\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `n.ui8` Array(UInt8),\n `n.s` Array(String),\n `s` Int64 DEFAULT \'0\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00062/alter\', \'r2\', d, k, 8192) +CREATE TABLE default.replicated_alter2\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `n.ui8` Array(UInt8),\n `n.s` Array(String),\n `s` Int64 DEFAULT \'0\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_00062/alter\', \'r2\', d, k, 8192) 2015-01-01 6 38 2014-07-15 13:26:50 [10,20,30] ['asd','qwe','qwe'] 100500 2015-01-01 7 39 2014-07-14 13:26:50 [10,20,30] ['120','130','140'] 0 2015-01-01 8 40 2012-12-12 12:12:12 [1,2,3] ['12','13','14'] 0 @@ -108,7 +108,7 @@ n.ui8 Array(UInt8) n.s Array(String) s UInt32 DEFAULT \'0\' n.d Array(Date) -CREATE TABLE default.replicated_alter1\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `n.ui8` Array(UInt8),\n `n.s` Array(String),\n `s` UInt32 DEFAULT \'0\',\n `n.d` Array(Date)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00062/alter\', \'r1\', d, k, 8192) +CREATE TABLE default.replicated_alter1\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `n.ui8` Array(UInt8),\n `n.s` Array(String),\n `s` UInt32 DEFAULT \'0\',\n `n.d` Array(Date)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_00062/alter\', \'r1\', d, k, 8192) d Date k UInt64 i32 Int32 @@ -117,7 +117,7 @@ n.ui8 Array(UInt8) n.s Array(String) s UInt32 DEFAULT \'0\' n.d Array(Date) -CREATE TABLE default.replicated_alter2\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `n.ui8` Array(UInt8),\n `n.s` Array(String),\n `s` UInt32 DEFAULT \'0\',\n `n.d` Array(Date)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00062/alter\', \'r2\', d, k, 8192) +CREATE TABLE default.replicated_alter2\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `n.ui8` Array(UInt8),\n `n.s` Array(String),\n `s` UInt32 DEFAULT \'0\',\n `n.d` Array(Date)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_00062/alter\', \'r2\', d, k, 8192) 2015-01-01 6 38 2014-07-15 13:26:50 [10,20,30] ['asd','qwe','qwe'] 100500 ['1970-01-01','1970-01-01','1970-01-01'] 2015-01-01 7 39 2014-07-14 13:26:50 [10,20,30] ['120','130','140'] 0 ['1970-01-01','1970-01-01','1970-01-01'] 2015-01-01 8 40 2012-12-12 12:12:12 [1,2,3] ['12','13','14'] 0 ['1970-01-01','1970-01-01','1970-01-01'] @@ -129,14 +129,14 @@ i32 Int32 dt DateTime(\'UTC\') n.s Array(String) s UInt32 DEFAULT \'0\' -CREATE TABLE default.replicated_alter1\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `n.s` Array(String),\n `s` UInt32 DEFAULT \'0\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00062/alter\', \'r1\', d, k, 8192) +CREATE TABLE default.replicated_alter1\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `n.s` Array(String),\n `s` UInt32 DEFAULT \'0\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_00062/alter\', \'r1\', d, k, 8192) d Date k UInt64 i32 Int32 dt DateTime(\'UTC\') n.s Array(String) s UInt32 DEFAULT \'0\' -CREATE TABLE default.replicated_alter2\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `n.s` Array(String),\n `s` UInt32 DEFAULT \'0\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00062/alter\', \'r2\', d, k, 8192) +CREATE TABLE default.replicated_alter2\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `n.s` Array(String),\n `s` UInt32 DEFAULT \'0\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_00062/alter\', \'r2\', d, k, 8192) 2015-01-01 6 38 2014-07-15 13:26:50 ['asd','qwe','qwe'] 100500 2015-01-01 7 39 2014-07-14 13:26:50 ['120','130','140'] 0 2015-01-01 8 40 2012-12-12 12:12:12 ['12','13','14'] 0 @@ -147,13 +147,13 @@ k UInt64 i32 Int32 dt DateTime(\'UTC\') s UInt32 DEFAULT \'0\' -CREATE TABLE default.replicated_alter1\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `s` UInt32 DEFAULT \'0\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00062/alter\', \'r1\', d, k, 8192) +CREATE TABLE default.replicated_alter1\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `s` UInt32 DEFAULT \'0\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_00062/alter\', \'r1\', d, k, 8192) d Date k UInt64 i32 Int32 dt DateTime(\'UTC\') s UInt32 DEFAULT \'0\' -CREATE TABLE default.replicated_alter2\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `s` UInt32 DEFAULT \'0\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00062/alter\', \'r2\', d, k, 8192) +CREATE TABLE default.replicated_alter2\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `s` UInt32 DEFAULT \'0\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_00062/alter\', \'r2\', d, k, 8192) 2015-01-01 6 38 2014-07-15 13:26:50 100500 2015-01-01 7 39 2014-07-14 13:26:50 0 2015-01-01 8 40 2012-12-12 12:12:12 0 @@ -166,7 +166,7 @@ dt DateTime(\'UTC\') s UInt32 DEFAULT \'0\' n.s Array(String) n.d Array(Date) -CREATE TABLE default.replicated_alter1\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `s` UInt32 DEFAULT \'0\',\n `n.s` Array(String),\n `n.d` Array(Date)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00062/alter\', \'r1\', d, k, 8192) +CREATE TABLE default.replicated_alter1\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `s` UInt32 DEFAULT \'0\',\n `n.s` Array(String),\n `n.d` Array(Date)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_00062/alter\', \'r1\', d, k, 8192) d Date k UInt64 i32 Int32 @@ -174,7 +174,7 @@ dt DateTime(\'UTC\') s UInt32 DEFAULT \'0\' n.s Array(String) n.d Array(Date) -CREATE TABLE default.replicated_alter2\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `s` UInt32 DEFAULT \'0\',\n `n.s` Array(String),\n `n.d` Array(Date)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00062/alter\', \'r2\', d, k, 8192) +CREATE TABLE default.replicated_alter2\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `s` UInt32 DEFAULT \'0\',\n `n.s` Array(String),\n `n.d` Array(Date)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_00062/alter\', \'r2\', d, k, 8192) 2015-01-01 6 38 2014-07-15 13:26:50 100500 [] [] 2015-01-01 7 39 2014-07-14 13:26:50 0 [] [] 2015-01-01 8 40 2012-12-12 12:12:12 0 [] [] @@ -185,13 +185,13 @@ k UInt64 i32 Int32 dt DateTime(\'UTC\') s UInt32 DEFAULT \'0\' -CREATE TABLE default.replicated_alter1\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `s` UInt32 DEFAULT \'0\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00062/alter\', \'r1\', d, k, 8192) +CREATE TABLE default.replicated_alter1\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `s` UInt32 DEFAULT \'0\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_00062/alter\', \'r1\', d, k, 8192) d Date k UInt64 i32 Int32 dt DateTime(\'UTC\') s UInt32 DEFAULT \'0\' -CREATE TABLE default.replicated_alter2\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `s` UInt32 DEFAULT \'0\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00062/alter\', \'r2\', d, k, 8192) +CREATE TABLE default.replicated_alter2\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `s` UInt32 DEFAULT \'0\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_00062/alter\', \'r2\', d, k, 8192) 2015-01-01 6 38 2014-07-15 13:26:50 100500 2015-01-01 7 39 2014-07-14 13:26:50 0 2015-01-01 8 40 2012-12-12 12:12:12 0 @@ -202,13 +202,13 @@ k UInt64 i32 Int32 dt Date s DateTime(\'UTC\') DEFAULT \'1970-01-01 00:00:00\' -CREATE TABLE default.replicated_alter1\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` Date,\n `s` DateTime(\'UTC\') DEFAULT \'1970-01-01 00:00:00\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00062/alter\', \'r1\', d, k, 8192) +CREATE TABLE default.replicated_alter1\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` Date,\n `s` DateTime(\'UTC\') DEFAULT \'1970-01-01 00:00:00\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_00062/alter\', \'r1\', d, k, 8192) d Date k UInt64 i32 Int32 dt Date s DateTime(\'UTC\') DEFAULT \'1970-01-01 00:00:00\' -CREATE TABLE default.replicated_alter2\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` Date,\n `s` DateTime(\'UTC\') DEFAULT \'1970-01-01 00:00:00\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00062/alter\', \'r2\', d, k, 8192) +CREATE TABLE default.replicated_alter2\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` Date,\n `s` DateTime(\'UTC\') DEFAULT \'1970-01-01 00:00:00\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_00062/alter\', \'r2\', d, k, 8192) 2015-01-01 6 38 2014-07-15 1970-01-02 03:55:00 2015-01-01 7 39 2014-07-14 1970-01-01 00:00:00 2015-01-01 8 40 2012-12-12 1970-01-01 00:00:00 diff --git a/tests/queries/0_stateless/00062_replicated_merge_tree_alter_zookeeper.sql b/tests/queries/0_stateless/00062_replicated_merge_tree_alter_zookeeper.sql index ac56b3416cd..9901dfad9c8 100644 --- a/tests/queries/0_stateless/00062_replicated_merge_tree_alter_zookeeper.sql +++ b/tests/queries/0_stateless/00062_replicated_merge_tree_alter_zookeeper.sql @@ -3,8 +3,8 @@ DROP TABLE IF EXISTS replicated_alter2; SET replication_alter_partitions_sync = 2; -CREATE TABLE replicated_alter1 (d Date, k UInt64, i32 Int32) ENGINE=ReplicatedMergeTree('/clickhouse/tables/test_00062/alter', 'r1', d, k, 8192); -CREATE TABLE replicated_alter2 (d Date, k UInt64, i32 Int32) ENGINE=ReplicatedMergeTree('/clickhouse/tables/test_00062/alter', 'r2', d, k, 8192); +CREATE TABLE replicated_alter1 (d Date, k UInt64, i32 Int32) ENGINE=ReplicatedMergeTree('/clickhouse/tables/{database}/test_00062/alter', 'r1', d, k, 8192); +CREATE TABLE replicated_alter2 (d Date, k UInt64, i32 Int32) ENGINE=ReplicatedMergeTree('/clickhouse/tables/{database}/test_00062/alter', 'r2', d, k, 8192); INSERT INTO replicated_alter1 VALUES ('2015-01-01', 10, 42); diff --git a/tests/queries/0_stateless/00083_create_merge_tree_zookeeper.sql b/tests/queries/0_stateless/00083_create_merge_tree_zookeeper.sql index 998a4517163..65247d7b12c 100644 --- a/tests/queries/0_stateless/00083_create_merge_tree_zookeeper.sql +++ b/tests/queries/0_stateless/00083_create_merge_tree_zookeeper.sql @@ -56,7 +56,7 @@ CREATE TABLE aggregating_merge_tree_with_sampling (d Date, a String, b UInt8, x String, y Int8, z UInt32) ENGINE = AggregatingMergeTree(d, sipHash64(a) + b, (a, sipHash64(a) + b), 111); CREATE TABLE replicated_merge_tree - (d Date, a String, b UInt8, x String, y Int8, z UInt32) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_00083/01/replicated_merge_tree/', 'r1', d, (a, b), 111); + (d Date, a String, b UInt8, x String, y Int8, z UInt32) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_00083/01/replicated_merge_tree/', 'r1', d, (a, b), 111); CREATE TABLE replicated_collapsing_merge_tree (d Date, a String, b UInt8, x String, y Int8, z UInt32) ENGINE = ReplicatedCollapsingMergeTree('/clickhouse/tables/test_00083/01/replicated_collapsing_merge_tree/', 'r1', d, (a, b), 111, y); CREATE TABLE replicated_versioned_collapsing_merge_tree @@ -69,7 +69,7 @@ CREATE TABLE replicated_aggregating_merge_tree (d Date, a String, b UInt8, x String, y Int8, z UInt32) ENGINE = ReplicatedAggregatingMergeTree('/clickhouse/tables/test_00083/01/replicated_aggregating_merge_tree/', 'r1', d, (a, b), 111); CREATE TABLE replicated_merge_tree_with_sampling - (d Date, a String, b UInt8, x String, y Int8, z UInt32) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_00083/01/replicated_merge_tree_with_sampling/', 'r1', d, sipHash64(a) + b, (a, sipHash64(a) + b), 111); + (d Date, a String, b UInt8, x String, y Int8, z UInt32) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_00083/01/replicated_merge_tree_with_sampling/', 'r1', d, sipHash64(a) + b, (a, sipHash64(a) + b), 111); CREATE TABLE replicated_collapsing_merge_tree_with_sampling (d Date, a String, b UInt8, x String, y Int8, z UInt32) ENGINE = ReplicatedCollapsingMergeTree('/clickhouse/tables/test_00083/01/replicated_collapsing_merge_tree_with_sampling/', 'r1', d, sipHash64(a) + b, (a, sipHash64(a) + b), 111, y); CREATE TABLE replicated_versioned_collapsing_merge_tree_with_sampling diff --git a/tests/queries/0_stateless/00121_drop_column_zookeeper.sql b/tests/queries/0_stateless/00121_drop_column_zookeeper.sql index 7ccf69e46b2..2aee56135f2 100644 --- a/tests/queries/0_stateless/00121_drop_column_zookeeper.sql +++ b/tests/queries/0_stateless/00121_drop_column_zookeeper.sql @@ -1,12 +1,12 @@ DROP TABLE IF EXISTS alter_00121; -CREATE TABLE alter_00121 (d Date, x UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/alter_00121/t1', 'r1', d, (d), 8192); +CREATE TABLE alter_00121 (d Date, x UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test/alter_00121/t1', 'r1', d, (d), 8192); INSERT INTO alter_00121 VALUES ('2014-01-01', 1); ALTER TABLE alter_00121 DROP COLUMN x; DROP TABLE alter_00121; -CREATE TABLE alter_00121 (d Date) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/alter_00121/t2', 'r1', d, (d), 8192); +CREATE TABLE alter_00121 (d Date) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test/alter_00121/t2', 'r1', d, (d), 8192); INSERT INTO alter_00121 VALUES ('2014-01-01'); SELECT * FROM alter_00121 ORDER BY d; diff --git a/tests/queries/0_stateless/00215_primary_key_order_zookeeper.sql b/tests/queries/0_stateless/00215_primary_key_order_zookeeper.sql index 8e36cbc85e0..86b84f3f63c 100644 --- a/tests/queries/0_stateless/00215_primary_key_order_zookeeper.sql +++ b/tests/queries/0_stateless/00215_primary_key_order_zookeeper.sql @@ -1,5 +1,5 @@ DROP TABLE IF EXISTS primary_key; -CREATE TABLE primary_key (d Date DEFAULT today(), x Int8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_00215/primary_key', 'r1', d, -x, 1); +CREATE TABLE primary_key (d Date DEFAULT today(), x Int8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_00215/primary_key', 'r1', d, -x, 1); INSERT INTO primary_key (x) VALUES (1), (2), (3); INSERT INTO primary_key (x) VALUES (1), (3), (2); diff --git a/tests/queries/0_stateless/00226_zookeeper_deduplication_and_unexpected_parts.sql b/tests/queries/0_stateless/00226_zookeeper_deduplication_and_unexpected_parts.sql index c14ce53d4a3..d70f337213e 100644 --- a/tests/queries/0_stateless/00226_zookeeper_deduplication_and_unexpected_parts.sql +++ b/tests/queries/0_stateless/00226_zookeeper_deduplication_and_unexpected_parts.sql @@ -1,5 +1,5 @@ DROP TABLE IF EXISTS deduplication; -CREATE TABLE deduplication (d Date DEFAULT '2015-01-01', x Int8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_00226/deduplication', 'r1', d, x, 1); +CREATE TABLE deduplication (d Date DEFAULT '2015-01-01', x Int8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_00226/deduplication', 'r1', d, x, 1); INSERT INTO deduplication (x) VALUES (1); INSERT INTO deduplication (x) VALUES (1); diff --git a/tests/queries/0_stateless/00236_replicated_drop_on_non_leader_zookeeper.sql b/tests/queries/0_stateless/00236_replicated_drop_on_non_leader_zookeeper.sql index 95a5c7c97f1..96a131514f8 100644 --- a/tests/queries/0_stateless/00236_replicated_drop_on_non_leader_zookeeper.sql +++ b/tests/queries/0_stateless/00236_replicated_drop_on_non_leader_zookeeper.sql @@ -3,8 +3,8 @@ SET replication_alter_partitions_sync = 2; DROP TABLE IF EXISTS attach_r1; DROP TABLE IF EXISTS attach_r2; -CREATE TABLE attach_r1 (d Date) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_00236/01/attach', 'r1', d, d, 8192); -CREATE TABLE attach_r2 (d Date) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_00236/01/attach', 'r2', d, d, 8192); +CREATE TABLE attach_r1 (d Date) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_00236/01/attach', 'r1', d, d, 8192); +CREATE TABLE attach_r2 (d Date) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_00236/01/attach', 'r2', d, d, 8192); INSERT INTO attach_r1 VALUES ('2014-01-01'), ('2014-02-01'), ('2014-03-01'); diff --git a/tests/queries/0_stateless/00446_clear_column_in_partition_zookeeper.sql b/tests/queries/0_stateless/00446_clear_column_in_partition_zookeeper.sql index 5d8c4de1c06..e4d60f1b960 100644 --- a/tests/queries/0_stateless/00446_clear_column_in_partition_zookeeper.sql +++ b/tests/queries/0_stateless/00446_clear_column_in_partition_zookeeper.sql @@ -24,8 +24,8 @@ SELECT '===Replicated case==='; DROP TABLE IF EXISTS clear_column1; DROP TABLE IF EXISTS clear_column2; SELECT sleep(1) FORMAT Null; -CREATE TABLE clear_column1 (d Date, i Int64) ENGINE = ReplicatedMergeTree('/clickhouse/test_00446/tables/clear_column', '1') ORDER BY d PARTITION by toYYYYMM(d) SETTINGS min_bytes_for_wide_part = 0; -CREATE TABLE clear_column2 (d Date, i Int64) ENGINE = ReplicatedMergeTree('/clickhouse/test_00446/tables/clear_column', '2') ORDER BY d PARTITION by toYYYYMM(d) SETTINGS min_bytes_for_wide_part = 0; +CREATE TABLE clear_column1 (d Date, i Int64) ENGINE = ReplicatedMergeTree('/clickhouse/{database}/test_00446/tables/clear_column', '1') ORDER BY d PARTITION by toYYYYMM(d) SETTINGS min_bytes_for_wide_part = 0; +CREATE TABLE clear_column2 (d Date, i Int64) ENGINE = ReplicatedMergeTree('/clickhouse/{database}/test_00446/tables/clear_column', '2') ORDER BY d PARTITION by toYYYYMM(d) SETTINGS min_bytes_for_wide_part = 0; INSERT INTO clear_column1 (d) VALUES ('2000-01-01'), ('2000-02-01'); SYSTEM SYNC REPLICA clear_column2; diff --git a/tests/queries/0_stateless/00502_custom_partitioning_replicated_zookeeper.sql b/tests/queries/0_stateless/00502_custom_partitioning_replicated_zookeeper.sql index a2a771e6cc0..bd92fcefcff 100644 --- a/tests/queries/0_stateless/00502_custom_partitioning_replicated_zookeeper.sql +++ b/tests/queries/0_stateless/00502_custom_partitioning_replicated_zookeeper.sql @@ -4,8 +4,8 @@ SELECT '*** Not partitioned ***'; DROP TABLE IF EXISTS not_partitioned_replica1_00502; DROP TABLE IF EXISTS not_partitioned_replica2_00502; -CREATE TABLE not_partitioned_replica1_00502(x UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/not_partitioned_00502', '1') ORDER BY x; -CREATE TABLE not_partitioned_replica2_00502(x UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/not_partitioned_00502', '2') ORDER BY x; +CREATE TABLE not_partitioned_replica1_00502(x UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test/not_partitioned_00502', '1') ORDER BY x; +CREATE TABLE not_partitioned_replica2_00502(x UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test/not_partitioned_00502', '2') ORDER BY x; INSERT INTO not_partitioned_replica1_00502 VALUES (1), (2), (3); INSERT INTO not_partitioned_replica1_00502 VALUES (4), (5); @@ -30,8 +30,8 @@ SELECT '*** Partitioned by week ***'; DROP TABLE IF EXISTS partitioned_by_week_replica1; DROP TABLE IF EXISTS partitioned_by_week_replica2; -CREATE TABLE partitioned_by_week_replica1(d Date, x UInt8) ENGINE ReplicatedMergeTree('/clickhouse/tables/test/partitioned_by_week_00502', '1') PARTITION BY toMonday(d) ORDER BY x; -CREATE TABLE partitioned_by_week_replica2(d Date, x UInt8) ENGINE ReplicatedMergeTree('/clickhouse/tables/test/partitioned_by_week_00502', '2') PARTITION BY toMonday(d) ORDER BY x; +CREATE TABLE partitioned_by_week_replica1(d Date, x UInt8) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test/partitioned_by_week_00502', '1') PARTITION BY toMonday(d) ORDER BY x; +CREATE TABLE partitioned_by_week_replica2(d Date, x UInt8) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test/partitioned_by_week_00502', '2') PARTITION BY toMonday(d) ORDER BY x; -- 2000-01-03 belongs to a different week than 2000-01-01 and 2000-01-02 INSERT INTO partitioned_by_week_replica1 VALUES ('2000-01-01', 1), ('2000-01-02', 2), ('2000-01-03', 3); @@ -57,8 +57,8 @@ SELECT '*** Partitioned by a (Date, UInt8) tuple ***'; DROP TABLE IF EXISTS partitioned_by_tuple_replica1_00502; DROP TABLE IF EXISTS partitioned_by_tuple_replica2_00502; -CREATE TABLE partitioned_by_tuple_replica1_00502(d Date, x UInt8, y UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/partitioned_by_tuple_00502', '1') ORDER BY x PARTITION BY (d, x); -CREATE TABLE partitioned_by_tuple_replica2_00502(d Date, x UInt8, y UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/partitioned_by_tuple_00502', '2') ORDER BY x PARTITION BY (d, x); +CREATE TABLE partitioned_by_tuple_replica1_00502(d Date, x UInt8, y UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test/partitioned_by_tuple_00502', '1') ORDER BY x PARTITION BY (d, x); +CREATE TABLE partitioned_by_tuple_replica2_00502(d Date, x UInt8, y UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test/partitioned_by_tuple_00502', '2') ORDER BY x PARTITION BY (d, x); INSERT INTO partitioned_by_tuple_replica1_00502 VALUES ('2000-01-01', 1, 1), ('2000-01-01', 2, 2), ('2000-01-02', 1, 3); INSERT INTO partitioned_by_tuple_replica1_00502 VALUES ('2000-01-02', 1, 4), ('2000-01-01', 1, 5); @@ -84,8 +84,8 @@ SELECT '*** Partitioned by String ***'; DROP TABLE IF EXISTS partitioned_by_string_replica1; DROP TABLE IF EXISTS partitioned_by_string_replica2; -CREATE TABLE partitioned_by_string_replica1(s String, x UInt8) ENGINE ReplicatedMergeTree('/clickhouse/tables/test/partitioned_by_string_00502', '1') PARTITION BY s ORDER BY x; -CREATE TABLE partitioned_by_string_replica2(s String, x UInt8) ENGINE ReplicatedMergeTree('/clickhouse/tables/test/partitioned_by_string_00502', '2') PARTITION BY s ORDER BY x; +CREATE TABLE partitioned_by_string_replica1(s String, x UInt8) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test/partitioned_by_string_00502', '1') PARTITION BY s ORDER BY x; +CREATE TABLE partitioned_by_string_replica2(s String, x UInt8) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test/partitioned_by_string_00502', '2') PARTITION BY s ORDER BY x; INSERT INTO partitioned_by_string_replica1 VALUES ('aaa', 1), ('aaa', 2), ('bbb', 3); INSERT INTO partitioned_by_string_replica1 VALUES ('bbb', 4), ('aaa', 5); @@ -110,8 +110,8 @@ SELECT '*** Table without columns with fixed size ***'; DROP TABLE IF EXISTS without_fixed_size_columns_replica1; DROP TABLE IF EXISTS without_fixed_size_columns_replica2; -CREATE TABLE without_fixed_size_columns_replica1(s String) ENGINE ReplicatedMergeTree('/clickhouse/tables/test/without_fixed_size_columns_00502', '1') PARTITION BY length(s) ORDER BY s; -CREATE TABLE without_fixed_size_columns_replica2(s String) ENGINE ReplicatedMergeTree('/clickhouse/tables/test/without_fixed_size_columns_00502', '2') PARTITION BY length(s) ORDER BY s; +CREATE TABLE without_fixed_size_columns_replica1(s String) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test/without_fixed_size_columns_00502', '1') PARTITION BY length(s) ORDER BY s; +CREATE TABLE without_fixed_size_columns_replica2(s String) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test/without_fixed_size_columns_00502', '2') PARTITION BY length(s) ORDER BY s; INSERT INTO without_fixed_size_columns_replica1 VALUES ('a'), ('aa'), ('b'), ('cc'); diff --git a/tests/queries/0_stateless/00509_extended_storage_definition_syntax_zookeeper.sql b/tests/queries/0_stateless/00509_extended_storage_definition_syntax_zookeeper.sql index ef8655a1861..48678329404 100644 --- a/tests/queries/0_stateless/00509_extended_storage_definition_syntax_zookeeper.sql +++ b/tests/queries/0_stateless/00509_extended_storage_definition_syntax_zookeeper.sql @@ -5,7 +5,7 @@ SELECT '*** Replicated with sampling ***'; DROP TABLE IF EXISTS replicated_with_sampling; CREATE TABLE replicated_with_sampling(x UInt8) - ENGINE ReplicatedMergeTree('/clickhouse/tables/test_00509/replicated_with_sampling', 'r1') + ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_00509/replicated_with_sampling', 'r1') ORDER BY x SAMPLE BY x; @@ -72,7 +72,7 @@ SELECT '*** Table definition with SETTINGS ***'; DROP TABLE IF EXISTS with_settings; CREATE TABLE with_settings(x UInt32) - ENGINE ReplicatedMergeTree('/clickhouse/tables/test_00509/with_settings', 'r1') + ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_00509/with_settings', 'r1') ORDER BY x SETTINGS replicated_can_become_leader = 0; diff --git a/tests/queries/0_stateless/00510_materizlized_view_and_deduplication_zookeeper.sql b/tests/queries/0_stateless/00510_materizlized_view_and_deduplication_zookeeper.sql index 8df012a8588..24f64a577f1 100644 --- a/tests/queries/0_stateless/00510_materizlized_view_and_deduplication_zookeeper.sql +++ b/tests/queries/0_stateless/00510_materizlized_view_and_deduplication_zookeeper.sql @@ -4,9 +4,9 @@ DROP TABLE IF EXISTS with_deduplication_mv; DROP TABLE IF EXISTS without_deduplication_mv; CREATE TABLE with_deduplication(x UInt32) - ENGINE ReplicatedMergeTree('/clickhouse/tables/test_00510/with_deduplication', 'r1') ORDER BY x; + ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_00510/with_deduplication', 'r1') ORDER BY x; CREATE TABLE without_deduplication(x UInt32) - ENGINE ReplicatedMergeTree('/clickhouse/tables/test_00510/without_deduplication', 'r1') ORDER BY x SETTINGS replicated_deduplication_window = 0; + ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_00510/without_deduplication', 'r1') ORDER BY x SETTINGS replicated_deduplication_window = 0; CREATE MATERIALIZED VIEW with_deduplication_mv UUID '00000510-1000-4000-8000-000000000001' ENGINE = ReplicatedAggregatingMergeTree('/clickhouse/tables/test_00510/with_deduplication_mv', 'r1') ORDER BY dummy diff --git a/tests/queries/0_stateless/00516_deduplication_after_drop_partition_zookeeper.sql b/tests/queries/0_stateless/00516_deduplication_after_drop_partition_zookeeper.sql index 2a9ebb992a8..71ed99f78fa 100644 --- a/tests/queries/0_stateless/00516_deduplication_after_drop_partition_zookeeper.sql +++ b/tests/queries/0_stateless/00516_deduplication_after_drop_partition_zookeeper.sql @@ -1,6 +1,6 @@ DROP TABLE IF EXISTS deduplication_by_partition; CREATE TABLE deduplication_by_partition(d Date, x UInt32) ENGINE = - ReplicatedMergeTree('/clickhouse/tables/test_00516/deduplication_by_partition', 'r1', d, x, 8192); + ReplicatedMergeTree('/clickhouse/tables/{database}/test_00516/deduplication_by_partition', 'r1', d, x, 8192); INSERT INTO deduplication_by_partition VALUES ('2000-01-01', 1); INSERT INTO deduplication_by_partition VALUES ('2000-01-01', 2), ('2000-01-01', 3); diff --git a/tests/queries/0_stateless/00563_insert_into_remote_and_zookeeper.sql b/tests/queries/0_stateless/00563_insert_into_remote_and_zookeeper.sql index 9c00ee68ba9..4cfbc1d0fb0 100644 --- a/tests/queries/0_stateless/00563_insert_into_remote_and_zookeeper.sql +++ b/tests/queries/0_stateless/00563_insert_into_remote_and_zookeeper.sql @@ -1,6 +1,6 @@ -- Check that settings are correctly passed through Distributed table DROP TABLE IF EXISTS simple; -CREATE TABLE simple (d Int8) ENGINE = ReplicatedMergeTree('/clickhouse/test_00563/tables/simple', '1') ORDER BY d; +CREATE TABLE simple (d Int8) ENGINE = ReplicatedMergeTree('/clickhouse/{database}/test_00563/tables/simple', '1') ORDER BY d; -- TODO: replace '127.0.0.2' -> '127.0.0.1' after a fix INSERT INTO TABLE FUNCTION remote('127.0.0.2', currentDatabase(), 'simple') VALUES (1); diff --git a/tests/queries/0_stateless/00623_replicated_truncate_table_zookeeper.sql b/tests/queries/0_stateless/00623_replicated_truncate_table_zookeeper.sql index 0f6d51b8716..06e88754888 100644 --- a/tests/queries/0_stateless/00623_replicated_truncate_table_zookeeper.sql +++ b/tests/queries/0_stateless/00623_replicated_truncate_table_zookeeper.sql @@ -1,8 +1,8 @@ DROP TABLE IF EXISTS replicated_truncate1; DROP TABLE IF EXISTS replicated_truncate2; -CREATE TABLE replicated_truncate1 (d Date, k UInt64, i32 Int32) ENGINE=ReplicatedMergeTree('/clickhouse/tables/test_00623/truncate', 'r1', d, k, 8192); -CREATE TABLE replicated_truncate2 (d Date, k UInt64, i32 Int32) ENGINE=ReplicatedMergeTree('/clickhouse/tables/test_00623/truncate', 'r2', d, k, 8192); +CREATE TABLE replicated_truncate1 (d Date, k UInt64, i32 Int32) ENGINE=ReplicatedMergeTree('/clickhouse/tables/{database}/test_00623/truncate', 'r1', d, k, 8192); +CREATE TABLE replicated_truncate2 (d Date, k UInt64, i32 Int32) ENGINE=ReplicatedMergeTree('/clickhouse/tables/{database}/test_00623/truncate', 'r2', d, k, 8192); SELECT '======Before Truncate======'; INSERT INTO replicated_truncate1 VALUES ('2015-01-01', 10, 42); diff --git a/tests/queries/0_stateless/00643_cast_zookeeper.reference b/tests/queries/0_stateless/00643_cast_zookeeper.reference index 9123463de1a..4eb4a4e0365 100644 --- a/tests/queries/0_stateless/00643_cast_zookeeper.reference +++ b/tests/queries/0_stateless/00643_cast_zookeeper.reference @@ -3,7 +3,7 @@ CREATE TABLE default.cast1 `x` UInt8, `e` Enum8('hello' = 1, 'world' = 2) DEFAULT CAST(x, 'Enum8(\'hello\' = 1, \'world\' = 2)') ) -ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_00643/cast', 'r1') +ENGINE = ReplicatedMergeTree('/clickhouse/tables/default/test_00643/cast', 'r1') ORDER BY e SETTINGS index_granularity = 8192 x UInt8 diff --git a/tests/queries/0_stateless/00643_cast_zookeeper.sql b/tests/queries/0_stateless/00643_cast_zookeeper.sql index c9760f00ca7..21e7a89c603 100644 --- a/tests/queries/0_stateless/00643_cast_zookeeper.sql +++ b/tests/queries/0_stateless/00643_cast_zookeeper.sql @@ -22,7 +22,7 @@ CREATE TABLE cast1 'world' = 2 ) ) -) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_00643/cast', 'r1') ORDER BY e; +) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_00643/cast', 'r1') ORDER BY e; SHOW CREATE TABLE cast1 FORMAT TSVRaw; DESC TABLE cast1; @@ -30,7 +30,7 @@ DESC TABLE cast1; INSERT INTO cast1 (x) VALUES (1); SELECT * FROM cast1; -CREATE TABLE cast2 AS cast1 ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_00643/cast', 'r2') ORDER BY e; +CREATE TABLE cast2 AS cast1 ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_00643/cast', 'r2') ORDER BY e; SYSTEM SYNC REPLICA cast2; diff --git a/tests/queries/0_stateless/00721_force_by_identical_result_after_merge_zookeeper.sql b/tests/queries/0_stateless/00721_force_by_identical_result_after_merge_zookeeper.sql index 498896d31a7..50f51510d61 100644 --- a/tests/queries/0_stateless/00721_force_by_identical_result_after_merge_zookeeper.sql +++ b/tests/queries/0_stateless/00721_force_by_identical_result_after_merge_zookeeper.sql @@ -1,8 +1,8 @@ DROP TABLE IF EXISTS byte_identical_r1; DROP TABLE IF EXISTS byte_identical_r2; -CREATE TABLE byte_identical_r1(x UInt32) ENGINE ReplicatedMergeTree('/clickhouse/tables/test_00721/byte_identical', 'r1') ORDER BY x; -CREATE TABLE byte_identical_r2(x UInt32) ENGINE ReplicatedMergeTree('/clickhouse/tables/test_00721/byte_identical', 'r2') ORDER BY x; +CREATE TABLE byte_identical_r1(x UInt32) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_00721/byte_identical', 'r1') ORDER BY x; +CREATE TABLE byte_identical_r2(x UInt32) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_00721/byte_identical', 'r2') ORDER BY x; INSERT INTO byte_identical_r1(x) VALUES (1), (2), (3); SYSTEM SYNC REPLICA byte_identical_r2; diff --git a/tests/queries/0_stateless/00725_comment_columns.sql b/tests/queries/0_stateless/00725_comment_columns.sql index 6fe095d9935..3eec0ba31b7 100644 --- a/tests/queries/0_stateless/00725_comment_columns.sql +++ b/tests/queries/0_stateless/00725_comment_columns.sql @@ -87,4 +87,4 @@ FORMAT PrettyCompactNoEscapes; DROP TABLE IF EXISTS check_query_comment_column; --- TODO: add here tests with ReplicatedMergeTree +-- TODO: add here tests with ReplicatedMergeTree({database}) diff --git a/tests/queries/0_stateless/00732_quorum_insert_have_data_before_quorum_zookeeper.sql b/tests/queries/0_stateless/00732_quorum_insert_have_data_before_quorum_zookeeper.sql index a1fc25fbf0b..a92646a41fc 100644 --- a/tests/queries/0_stateless/00732_quorum_insert_have_data_before_quorum_zookeeper.sql +++ b/tests/queries/0_stateless/00732_quorum_insert_have_data_before_quorum_zookeeper.sql @@ -3,8 +3,8 @@ SET send_logs_level = 'fatal'; DROP TABLE IF EXISTS quorum1; DROP TABLE IF EXISTS quorum2; -CREATE TABLE quorum1(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/test_00732/quorum_have_data', '1') ORDER BY x PARTITION BY y; -CREATE TABLE quorum2(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/test_00732/quorum_have_data', '2') ORDER BY x PARTITION BY y; +CREATE TABLE quorum1(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_00732/quorum_have_data', '1') ORDER BY x PARTITION BY y; +CREATE TABLE quorum2(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_00732/quorum_have_data', '2') ORDER BY x PARTITION BY y; INSERT INTO quorum1 VALUES (1, '1990-11-15'); INSERT INTO quorum1 VALUES (2, '1990-11-15'); diff --git a/tests/queries/0_stateless/00732_quorum_insert_lost_part_and_alive_part_zookeeper.sql b/tests/queries/0_stateless/00732_quorum_insert_lost_part_and_alive_part_zookeeper.sql index 914d98ff3f5..4f9afb93f3a 100644 --- a/tests/queries/0_stateless/00732_quorum_insert_lost_part_and_alive_part_zookeeper.sql +++ b/tests/queries/0_stateless/00732_quorum_insert_lost_part_and_alive_part_zookeeper.sql @@ -3,8 +3,8 @@ SET send_logs_level = 'fatal'; DROP TABLE IF EXISTS quorum1; DROP TABLE IF EXISTS quorum2; -CREATE TABLE quorum1(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/test_00732/quorum_lost_alive', '1') ORDER BY x PARTITION BY y; -CREATE TABLE quorum2(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/test_00732/quorum_lost_alive', '2') ORDER BY x PARTITION BY y; +CREATE TABLE quorum1(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_00732/quorum_lost_alive', '1') ORDER BY x PARTITION BY y; +CREATE TABLE quorum2(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_00732/quorum_lost_alive', '2') ORDER BY x PARTITION BY y; SET insert_quorum=2, insert_quorum_parallel=0; SET select_sequential_consistency=1; diff --git a/tests/queries/0_stateless/00732_quorum_insert_lost_part_zookeeper.sql b/tests/queries/0_stateless/00732_quorum_insert_lost_part_zookeeper.sql index 18cad0dda41..1374fa776ca 100644 --- a/tests/queries/0_stateless/00732_quorum_insert_lost_part_zookeeper.sql +++ b/tests/queries/0_stateless/00732_quorum_insert_lost_part_zookeeper.sql @@ -3,8 +3,8 @@ SET send_logs_level = 'fatal'; DROP TABLE IF EXISTS quorum1; DROP TABLE IF EXISTS quorum2; -CREATE TABLE quorum1(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/test_00732/quorum_lost', '1') ORDER BY x PARTITION BY y; -CREATE TABLE quorum2(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/test_00732/quorum_lost', '2') ORDER BY x PARTITION BY y; +CREATE TABLE quorum1(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_00732/quorum_lost', '1') ORDER BY x PARTITION BY y; +CREATE TABLE quorum2(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_00732/quorum_lost', '2') ORDER BY x PARTITION BY y; SET insert_quorum=2, insert_quorum_parallel=0; SET select_sequential_consistency=1; diff --git a/tests/queries/0_stateless/00732_quorum_insert_select_with_old_data_and_without_quorum_zookeeper.sql b/tests/queries/0_stateless/00732_quorum_insert_select_with_old_data_and_without_quorum_zookeeper.sql index 11c8324ac25..0cc162b6010 100644 --- a/tests/queries/0_stateless/00732_quorum_insert_select_with_old_data_and_without_quorum_zookeeper.sql +++ b/tests/queries/0_stateless/00732_quorum_insert_select_with_old_data_and_without_quorum_zookeeper.sql @@ -3,8 +3,8 @@ SET send_logs_level = 'fatal'; DROP TABLE IF EXISTS quorum1; DROP TABLE IF EXISTS quorum2; -CREATE TABLE quorum1(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/test_00732/quorum_old_data', '1') ORDER BY x PARTITION BY y; -CREATE TABLE quorum2(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/test_00732/quorum_old_data', '2') ORDER BY x PARTITION BY y; +CREATE TABLE quorum1(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_00732/quorum_old_data', '1') ORDER BY x PARTITION BY y; +CREATE TABLE quorum2(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_00732/quorum_old_data', '2') ORDER BY x PARTITION BY y; INSERT INTO quorum1 VALUES (1, '1990-11-15'); INSERT INTO quorum1 VALUES (2, '1990-11-15'); diff --git a/tests/queries/0_stateless/00732_quorum_insert_simple_test_1_parts_zookeeper.sql b/tests/queries/0_stateless/00732_quorum_insert_simple_test_1_parts_zookeeper.sql index a4e262ce3c4..0ceca3743b2 100644 --- a/tests/queries/0_stateless/00732_quorum_insert_simple_test_1_parts_zookeeper.sql +++ b/tests/queries/0_stateless/00732_quorum_insert_simple_test_1_parts_zookeeper.sql @@ -3,8 +3,8 @@ SET send_logs_level = 'fatal'; DROP TABLE IF EXISTS quorum1; DROP TABLE IF EXISTS quorum2; -CREATE TABLE quorum1(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/test_00732/quorum1', '1') ORDER BY x PARTITION BY y; -CREATE TABLE quorum2(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/test_00732/quorum1', '2') ORDER BY x PARTITION BY y; +CREATE TABLE quorum1(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_00732/quorum1', '1') ORDER BY x PARTITION BY y; +CREATE TABLE quorum2(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_00732/quorum1', '2') ORDER BY x PARTITION BY y; SET insert_quorum=2, insert_quorum_parallel=0; SET select_sequential_consistency=1; diff --git a/tests/queries/0_stateless/00732_quorum_insert_simple_test_2_parts_zookeeper.sql b/tests/queries/0_stateless/00732_quorum_insert_simple_test_2_parts_zookeeper.sql index 40c08fd926f..a95d992705e 100644 --- a/tests/queries/0_stateless/00732_quorum_insert_simple_test_2_parts_zookeeper.sql +++ b/tests/queries/0_stateless/00732_quorum_insert_simple_test_2_parts_zookeeper.sql @@ -3,8 +3,8 @@ SET send_logs_level = 'fatal'; DROP TABLE IF EXISTS quorum1; DROP TABLE IF EXISTS quorum2; -CREATE TABLE quorum1(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/test_00732/quorum2', '1') ORDER BY x PARTITION BY y; -CREATE TABLE quorum2(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/test_00732/quorum2', '2') ORDER BY x PARTITION BY y; +CREATE TABLE quorum1(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_00732/quorum2', '1') ORDER BY x PARTITION BY y; +CREATE TABLE quorum2(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_00732/quorum2', '2') ORDER BY x PARTITION BY y; SET insert_quorum=2, insert_quorum_parallel=0; SET select_sequential_consistency=1; diff --git a/tests/queries/0_stateless/00753_comment_columns_zookeeper.reference b/tests/queries/0_stateless/00753_comment_columns_zookeeper.reference index 74aa60a41f2..9bf0304d7d7 100644 --- a/tests/queries/0_stateless/00753_comment_columns_zookeeper.reference +++ b/tests/queries/0_stateless/00753_comment_columns_zookeeper.reference @@ -1,6 +1,6 @@ -CREATE TABLE default.check_comments\n(\n `column_name1` UInt8 DEFAULT 1 COMMENT \'comment\',\n `column_name2` UInt8 COMMENT \'non default comment\'\n)\nENGINE = ReplicatedMergeTree(\'clickhouse/tables/test_00753/comments\', \'r1\')\nORDER BY column_name1\nSETTINGS index_granularity = 8192 +CREATE TABLE default.check_comments\n(\n `column_name1` UInt8 DEFAULT 1 COMMENT \'comment\',\n `column_name2` UInt8 COMMENT \'non default comment\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_00753/comments\', \'r1\')\nORDER BY column_name1\nSETTINGS index_granularity = 8192 column_name1 UInt8 DEFAULT 1 comment column_name2 UInt8 non default comment -CREATE TABLE default.check_comments\n(\n `column_name1` UInt8 DEFAULT 1 COMMENT \'another comment\',\n `column_name2` UInt8 COMMENT \'non default comment\'\n)\nENGINE = ReplicatedMergeTree(\'clickhouse/tables/test_00753/comments\', \'r1\')\nORDER BY column_name1\nSETTINGS index_granularity = 8192 +CREATE TABLE default.check_comments\n(\n `column_name1` UInt8 DEFAULT 1 COMMENT \'another comment\',\n `column_name2` UInt8 COMMENT \'non default comment\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_00753/comments\', \'r1\')\nORDER BY column_name1\nSETTINGS index_granularity = 8192 column_name1 UInt8 DEFAULT 1 another comment column_name2 UInt8 non default comment diff --git a/tests/queries/0_stateless/00753_comment_columns_zookeeper.sql b/tests/queries/0_stateless/00753_comment_columns_zookeeper.sql index e98630bb00d..93e2c9fb9d5 100644 --- a/tests/queries/0_stateless/00753_comment_columns_zookeeper.sql +++ b/tests/queries/0_stateless/00753_comment_columns_zookeeper.sql @@ -4,7 +4,7 @@ CREATE TABLE check_comments ( column_name1 UInt8 DEFAULT 1 COMMENT 'comment', column_name2 UInt8 COMMENT 'non default comment' - ) ENGINE = ReplicatedMergeTree('clickhouse/tables/test_00753/comments', 'r1') + ) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_00753/comments', 'r1') ORDER BY column_name1; SHOW CREATE check_comments; diff --git a/tests/queries/0_stateless/00754_alter_modify_order_by_replicated_zookeeper.sql b/tests/queries/0_stateless/00754_alter_modify_order_by_replicated_zookeeper.sql index 809adfaa498..9e08406202e 100644 --- a/tests/queries/0_stateless/00754_alter_modify_order_by_replicated_zookeeper.sql +++ b/tests/queries/0_stateless/00754_alter_modify_order_by_replicated_zookeeper.sql @@ -3,7 +3,7 @@ SET optimize_on_insert = 0; SET send_logs_level = 'fatal'; DROP TABLE IF EXISTS old_style; -CREATE TABLE old_style(d Date, x UInt32) ENGINE ReplicatedMergeTree('/clickhouse/tables/test_00754/old_style', 'r1', d, x, 8192); +CREATE TABLE old_style(d Date, x UInt32) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_00754/old_style', 'r1', d, x, 8192); ALTER TABLE old_style ADD COLUMN y UInt32, MODIFY ORDER BY (x, y); -- { serverError 36 } DROP TABLE old_style; diff --git a/tests/queries/0_stateless/00836_indices_alter_replicated_zookeeper.reference b/tests/queries/0_stateless/00836_indices_alter_replicated_zookeeper.reference index 838bd93ebaf..fbe0e7f564f 100644 --- a/tests/queries/0_stateless/00836_indices_alter_replicated_zookeeper.reference +++ b/tests/queries/0_stateless/00836_indices_alter_replicated_zookeeper.reference @@ -1,5 +1,5 @@ -CREATE TABLE default.minmax_idx\n(\n `u64` UInt64,\n `i32` Int32,\n INDEX idx1 u64 * i32 TYPE minmax GRANULARITY 10,\n INDEX idx3 u64 - i32 TYPE minmax GRANULARITY 10,\n INDEX idx2 u64 + i32 TYPE minmax GRANULARITY 10\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00836/indices_alter1\', \'r1\')\nORDER BY u64\nSETTINGS index_granularity = 8192 -CREATE TABLE default.minmax_idx_r\n(\n `u64` UInt64,\n `i32` Int32,\n INDEX idx1 u64 * i32 TYPE minmax GRANULARITY 10,\n INDEX idx3 u64 - i32 TYPE minmax GRANULARITY 10,\n INDEX idx2 u64 + i32 TYPE minmax GRANULARITY 10\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00836/indices_alter1\', \'r2\')\nORDER BY u64\nSETTINGS index_granularity = 8192 +CREATE TABLE default.minmax_idx\n(\n `u64` UInt64,\n `i32` Int32,\n INDEX idx1 u64 * i32 TYPE minmax GRANULARITY 10,\n INDEX idx3 u64 - i32 TYPE minmax GRANULARITY 10,\n INDEX idx2 u64 + i32 TYPE minmax GRANULARITY 10\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_00836/indices_alter1\', \'r1\')\nORDER BY u64\nSETTINGS index_granularity = 8192 +CREATE TABLE default.minmax_idx_r\n(\n `u64` UInt64,\n `i32` Int32,\n INDEX idx1 u64 * i32 TYPE minmax GRANULARITY 10,\n INDEX idx3 u64 - i32 TYPE minmax GRANULARITY 10,\n INDEX idx2 u64 + i32 TYPE minmax GRANULARITY 10\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_00836/indices_alter1\', \'r2\')\nORDER BY u64\nSETTINGS index_granularity = 8192 1 2 1 2 1 2 @@ -14,8 +14,8 @@ CREATE TABLE default.minmax_idx_r\n(\n `u64` UInt64,\n `i32` Int32,\n I 3 2 19 9 65 75 -CREATE TABLE default.minmax_idx\n(\n `u64` UInt64,\n `i32` Int32,\n INDEX idx3 u64 - i32 TYPE minmax GRANULARITY 10,\n INDEX idx2 u64 + i32 TYPE minmax GRANULARITY 10\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00836/indices_alter1\', \'r1\')\nORDER BY u64\nSETTINGS index_granularity = 8192 -CREATE TABLE default.minmax_idx_r\n(\n `u64` UInt64,\n `i32` Int32,\n INDEX idx3 u64 - i32 TYPE minmax GRANULARITY 10,\n INDEX idx2 u64 + i32 TYPE minmax GRANULARITY 10\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00836/indices_alter1\', \'r2\')\nORDER BY u64\nSETTINGS index_granularity = 8192 +CREATE TABLE default.minmax_idx\n(\n `u64` UInt64,\n `i32` Int32,\n INDEX idx3 u64 - i32 TYPE minmax GRANULARITY 10,\n INDEX idx2 u64 + i32 TYPE minmax GRANULARITY 10\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_00836/indices_alter1\', \'r1\')\nORDER BY u64\nSETTINGS index_granularity = 8192 +CREATE TABLE default.minmax_idx_r\n(\n `u64` UInt64,\n `i32` Int32,\n INDEX idx3 u64 - i32 TYPE minmax GRANULARITY 10,\n INDEX idx2 u64 + i32 TYPE minmax GRANULARITY 10\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_00836/indices_alter1\', \'r2\')\nORDER BY u64\nSETTINGS index_granularity = 8192 1 2 1 4 1 5 @@ -28,10 +28,10 @@ CREATE TABLE default.minmax_idx_r\n(\n `u64` UInt64,\n `i32` Int32,\n I 3 2 19 9 65 75 -CREATE TABLE default.minmax_idx\n(\n `u64` UInt64,\n `i32` Int32\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00836/indices_alter1\', \'r1\')\nORDER BY u64\nSETTINGS index_granularity = 8192 -CREATE TABLE default.minmax_idx_r\n(\n `u64` UInt64,\n `i32` Int32\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00836/indices_alter1\', \'r2\')\nORDER BY u64\nSETTINGS index_granularity = 8192 -CREATE TABLE default.minmax_idx\n(\n `u64` UInt64,\n `i32` Int32,\n INDEX idx1 u64 * i32 TYPE minmax GRANULARITY 10\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00836/indices_alter1\', \'r1\')\nORDER BY u64\nSETTINGS index_granularity = 8192 -CREATE TABLE default.minmax_idx_r\n(\n `u64` UInt64,\n `i32` Int32,\n INDEX idx1 u64 * i32 TYPE minmax GRANULARITY 10\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00836/indices_alter1\', \'r2\')\nORDER BY u64\nSETTINGS index_granularity = 8192 +CREATE TABLE default.minmax_idx\n(\n `u64` UInt64,\n `i32` Int32\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_00836/indices_alter1\', \'r1\')\nORDER BY u64\nSETTINGS index_granularity = 8192 +CREATE TABLE default.minmax_idx_r\n(\n `u64` UInt64,\n `i32` Int32\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_00836/indices_alter1\', \'r2\')\nORDER BY u64\nSETTINGS index_granularity = 8192 +CREATE TABLE default.minmax_idx\n(\n `u64` UInt64,\n `i32` Int32,\n INDEX idx1 u64 * i32 TYPE minmax GRANULARITY 10\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_00836/indices_alter1\', \'r1\')\nORDER BY u64\nSETTINGS index_granularity = 8192 +CREATE TABLE default.minmax_idx_r\n(\n `u64` UInt64,\n `i32` Int32,\n INDEX idx1 u64 * i32 TYPE minmax GRANULARITY 10\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_00836/indices_alter1\', \'r2\')\nORDER BY u64\nSETTINGS index_granularity = 8192 1 2 1 4 1 5 @@ -44,14 +44,14 @@ CREATE TABLE default.minmax_idx_r\n(\n `u64` UInt64,\n `i32` Int32,\n I 3 2 19 9 65 75 -CREATE TABLE default.minmax_idx2\n(\n `u64` UInt64,\n `i32` Int32,\n INDEX idx1 u64 + i32 TYPE minmax GRANULARITY 10,\n INDEX idx2 u64 * i32 TYPE minmax GRANULARITY 10\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00836/indices_alter2\', \'r1\')\nORDER BY u64\nSETTINGS index_granularity = 8192 -CREATE TABLE default.minmax_idx2_r\n(\n `u64` UInt64,\n `i32` Int32,\n INDEX idx1 u64 + i32 TYPE minmax GRANULARITY 10,\n INDEX idx2 u64 * i32 TYPE minmax GRANULARITY 10\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00836/indices_alter2\', \'r2\')\nORDER BY u64\nSETTINGS index_granularity = 8192 +CREATE TABLE default.minmax_idx2\n(\n `u64` UInt64,\n `i32` Int32,\n INDEX idx1 u64 + i32 TYPE minmax GRANULARITY 10,\n INDEX idx2 u64 * i32 TYPE minmax GRANULARITY 10\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_00836/indices_alter2\', \'r1\')\nORDER BY u64\nSETTINGS index_granularity = 8192 +CREATE TABLE default.minmax_idx2_r\n(\n `u64` UInt64,\n `i32` Int32,\n INDEX idx1 u64 + i32 TYPE minmax GRANULARITY 10,\n INDEX idx2 u64 * i32 TYPE minmax GRANULARITY 10\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_00836/indices_alter2\', \'r2\')\nORDER BY u64\nSETTINGS index_granularity = 8192 1 2 1 3 1 2 1 3 -CREATE TABLE default.minmax_idx2\n(\n `u64` UInt64,\n `i32` Int32\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00836/indices_alter2\', \'r1\')\nORDER BY u64\nSETTINGS index_granularity = 8192 -CREATE TABLE default.minmax_idx2_r\n(\n `u64` UInt64,\n `i32` Int32\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00836/indices_alter2\', \'r2\')\nORDER BY u64\nSETTINGS index_granularity = 8192 +CREATE TABLE default.minmax_idx2\n(\n `u64` UInt64,\n `i32` Int32\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_00836/indices_alter2\', \'r1\')\nORDER BY u64\nSETTINGS index_granularity = 8192 +CREATE TABLE default.minmax_idx2_r\n(\n `u64` UInt64,\n `i32` Int32\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_00836/indices_alter2\', \'r2\')\nORDER BY u64\nSETTINGS index_granularity = 8192 1 2 1 3 1 2 diff --git a/tests/queries/0_stateless/00836_indices_alter_replicated_zookeeper.sql b/tests/queries/0_stateless/00836_indices_alter_replicated_zookeeper.sql index e038d2d425e..6274a70a381 100644 --- a/tests/queries/0_stateless/00836_indices_alter_replicated_zookeeper.sql +++ b/tests/queries/0_stateless/00836_indices_alter_replicated_zookeeper.sql @@ -9,14 +9,14 @@ CREATE TABLE minmax_idx ( u64 UInt64, i32 Int32 -) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_00836/indices_alter1', 'r1') +) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_00836/indices_alter1', 'r1') ORDER BY u64; CREATE TABLE minmax_idx_r ( u64 UInt64, i32 Int32 -) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_00836/indices_alter1', 'r2') +) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_00836/indices_alter1', 'r2') ORDER BY u64; INSERT INTO minmax_idx VALUES (1, 2); @@ -74,7 +74,7 @@ CREATE TABLE minmax_idx2 i32 Int32, INDEX idx1 u64 + i32 TYPE minmax GRANULARITY 10, INDEX idx2 u64 * i32 TYPE minmax GRANULARITY 10 -) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_00836/indices_alter2', 'r1') +) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_00836/indices_alter2', 'r1') ORDER BY u64; CREATE TABLE minmax_idx2_r @@ -83,7 +83,7 @@ CREATE TABLE minmax_idx2_r i32 Int32, INDEX idx1 u64 + i32 TYPE minmax GRANULARITY 10, INDEX idx2 u64 * i32 TYPE minmax GRANULARITY 10 -) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_00836/indices_alter2', 'r2') +) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_00836/indices_alter2', 'r2') ORDER BY u64; diff --git a/tests/queries/0_stateless/00837_minmax_index_replicated_zookeeper.sql b/tests/queries/0_stateless/00837_minmax_index_replicated_zookeeper.sql index 2213ccecd4e..0dd8dd41f97 100644 --- a/tests/queries/0_stateless/00837_minmax_index_replicated_zookeeper.sql +++ b/tests/queries/0_stateless/00837_minmax_index_replicated_zookeeper.sql @@ -15,7 +15,7 @@ CREATE TABLE minmax_idx1 idx_all (i32, i32 + f64, d, s, e, dt) TYPE minmax GRANULARITY 1, INDEX idx_2 (u64 + toYear(dt), substring(s, 2, 4)) TYPE minmax GRANULARITY 3 -) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_00837/minmax', 'r1') +) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_00837/minmax', 'r1') ORDER BY u64 SETTINGS index_granularity = 2; @@ -32,7 +32,7 @@ CREATE TABLE minmax_idx2 idx_all (i32, i32 + f64, d, s, e, dt) TYPE minmax GRANULARITY 1, INDEX idx_2 (u64 + toYear(dt), substring(s, 2, 4)) TYPE minmax GRANULARITY 3 -) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_00837/minmax', 'r2') +) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_00837/minmax', 'r2') ORDER BY u64 SETTINGS index_granularity = 2; diff --git a/tests/queries/0_stateless/00910_zookeeper_custom_compression_codecs_replicated.reference b/tests/queries/0_stateless/00910_zookeeper_custom_compression_codecs_replicated.reference index 29e7b23d3dd..3b7faecbba4 100644 --- a/tests/queries/0_stateless/00910_zookeeper_custom_compression_codecs_replicated.reference +++ b/tests/queries/0_stateless/00910_zookeeper_custom_compression_codecs_replicated.reference @@ -20,7 +20,7 @@ 274972506.6 9175437371954010821 9175437371954010821 -CREATE TABLE default.compression_codec_multiple_more_types_replicated\n(\n `id` Decimal(38, 13) CODEC(ZSTD(1), LZ4, ZSTD(1), ZSTD(1), Delta(2), Delta(4), Delta(1), LZ4HC(0)),\n `data` FixedString(12) CODEC(ZSTD(1), ZSTD(1), Delta(1), Delta(1), Delta(1), NONE, NONE, NONE, LZ4HC(0)),\n `ddd.age` Array(UInt8) CODEC(LZ4, LZ4HC(0), NONE, NONE, NONE, ZSTD(1), Delta(8)),\n `ddd.Name` Array(String) CODEC(LZ4, LZ4HC(0), NONE, NONE, NONE, ZSTD(1), Delta(8))\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00910/compression_codec_multiple_more_types_replicated\', \'1\')\nORDER BY tuple()\nSETTINGS index_granularity = 8192 +CREATE TABLE default.compression_codec_multiple_more_types_replicated\n(\n `id` Decimal(38, 13) CODEC(ZSTD(1), LZ4, ZSTD(1), ZSTD(1), Delta(2), Delta(4), Delta(1), LZ4HC(0)),\n `data` FixedString(12) CODEC(ZSTD(1), ZSTD(1), Delta(1), Delta(1), Delta(1), NONE, NONE, NONE, LZ4HC(0)),\n `ddd.age` Array(UInt8) CODEC(LZ4, LZ4HC(0), NONE, NONE, NONE, ZSTD(1), Delta(8)),\n `ddd.Name` Array(String) CODEC(LZ4, LZ4HC(0), NONE, NONE, NONE, ZSTD(1), Delta(8))\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_00910/compression_codec_multiple_more_types_replicated\', \'1\')\nORDER BY tuple()\nSETTINGS index_granularity = 8192 1.5555555555555 hello world! [77] ['John'] 7.1000000000000 xxxxxxxxxxxx [127] ['Henry'] ! diff --git a/tests/queries/0_stateless/00910_zookeeper_custom_compression_codecs_replicated.sql b/tests/queries/0_stateless/00910_zookeeper_custom_compression_codecs_replicated.sql index 52eb1d4e411..3fe121edc94 100644 --- a/tests/queries/0_stateless/00910_zookeeper_custom_compression_codecs_replicated.sql +++ b/tests/queries/0_stateless/00910_zookeeper_custom_compression_codecs_replicated.sql @@ -11,7 +11,7 @@ CREATE TABLE compression_codec_replicated1( somenum Float64 CODEC(ZSTD(2)), somestr FixedString(3) CODEC(LZ4HC(7)), othernum Int64 CODEC(Delta) -) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_00910/compression_codec_replicated', '1') ORDER BY tuple(); +) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_00910/compression_codec_replicated', '1') ORDER BY tuple(); CREATE TABLE compression_codec_replicated2( id UInt64 CODEC(LZ4), @@ -20,7 +20,7 @@ CREATE TABLE compression_codec_replicated2( somenum Float64 CODEC(ZSTD(2)), somestr FixedString(3) CODEC(LZ4HC(7)), othernum Int64 CODEC(Delta) -) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_00910/compression_codec_replicated', '2') ORDER BY tuple(); +) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_00910/compression_codec_replicated', '2') ORDER BY tuple(); INSERT INTO compression_codec_replicated1 VALUES(1, 'hello', toDate('2018-12-14'), 1.1, 'aaa', 5); @@ -57,14 +57,14 @@ CREATE TABLE compression_codec_multiple_replicated1 ( data String CODEC(ZSTD(2), NONE, Delta(2), LZ4HC, LZ4, LZ4, Delta(8)), ddd Date CODEC(NONE, NONE, NONE, Delta(1), LZ4, ZSTD, LZ4HC, LZ4HC), somenum Float64 CODEC(Delta(4), LZ4, LZ4, ZSTD(2), LZ4HC(5), ZSTD(3), ZSTD) -) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_00910/compression_codec_multiple', '1') ORDER BY tuple(); +) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_00910/compression_codec_multiple', '1') ORDER BY tuple(); CREATE TABLE compression_codec_multiple_replicated2 ( id UInt64 CODEC(LZ4, ZSTD, NONE, LZ4HC, Delta(4)), data String CODEC(ZSTD(2), NONE, Delta(2), LZ4HC, LZ4, LZ4, Delta(8)), ddd Date CODEC(NONE, NONE, NONE, Delta(1), LZ4, ZSTD, LZ4HC, LZ4HC), somenum Float64 CODEC(Delta(4), LZ4, LZ4, ZSTD(2), LZ4HC(5), ZSTD(3), ZSTD) -) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_00910/compression_codec_multiple', '2') ORDER BY tuple(); +) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_00910/compression_codec_multiple', '2') ORDER BY tuple(); INSERT INTO compression_codec_multiple_replicated2 VALUES (1, 'world', toDate('2018-10-05'), 1.1), (2, 'hello', toDate('2018-10-01'), 2.2), (3, 'buy', toDate('2018-10-11'), 3.3); @@ -106,7 +106,7 @@ CREATE TABLE compression_codec_multiple_more_types_replicated ( id Decimal128(13) CODEC(ZSTD, LZ4, ZSTD, ZSTD, Delta(2), Delta(4), Delta(1), LZ4HC), data FixedString(12) CODEC(ZSTD, ZSTD, Delta(1), Delta(1), Delta(1), NONE, NONE, NONE, LZ4HC), ddd Nested (age UInt8, Name String) CODEC(LZ4, LZ4HC, NONE, NONE, NONE, ZSTD, Delta(8)) -) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_00910/compression_codec_multiple_more_types_replicated', '1') ORDER BY tuple(); +) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_00910/compression_codec_multiple_more_types_replicated', '1') ORDER BY tuple(); SHOW CREATE TABLE compression_codec_multiple_more_types_replicated; @@ -124,7 +124,7 @@ CREATE TABLE compression_codec_multiple_with_key_replicated ( somedate Date CODEC(ZSTD, ZSTD, ZSTD(12), LZ4HC(12), Delta, Delta), id UInt64 CODEC(LZ4, ZSTD, Delta, NONE, LZ4HC, Delta), data String CODEC(ZSTD(2), Delta(1), LZ4HC, NONE, LZ4, LZ4) -) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_00910/compression_codec_multiple_with_key_replicated', '1') PARTITION BY somedate ORDER BY id SETTINGS index_granularity = 2; +) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_00910/compression_codec_multiple_with_key_replicated', '1') PARTITION BY somedate ORDER BY id SETTINGS index_granularity = 2; INSERT INTO compression_codec_multiple_with_key_replicated VALUES(toDate('2018-10-12'), 100000, 'hello'), (toDate('2018-10-12'), 100002, 'world'), (toDate('2018-10-12'), 1111, '!'); diff --git a/tests/queries/0_stateless/00910_zookeeper_test_alter_compression_codecs_long.sql b/tests/queries/0_stateless/00910_zookeeper_test_alter_compression_codecs_long.sql index 548f26eadd0..e4994ff8d75 100644 --- a/tests/queries/0_stateless/00910_zookeeper_test_alter_compression_codecs_long.sql +++ b/tests/queries/0_stateless/00910_zookeeper_test_alter_compression_codecs_long.sql @@ -7,12 +7,12 @@ DROP TABLE IF EXISTS alter_compression_codec2; CREATE TABLE alter_compression_codec1 ( somedate Date CODEC(LZ4), id UInt64 CODEC(NONE) -) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_00910/'||currentDatabase()||'alter_compression_codecs/{shard}', '1_{replica}') PARTITION BY somedate ORDER BY id; +) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_00910/'||currentDatabase()||'alter_compression_codecs/{shard}', '1_{replica}') PARTITION BY somedate ORDER BY id; CREATE TABLE alter_compression_codec2 ( somedate Date CODEC(LZ4), id UInt64 CODEC(NONE) -) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_00910/'||currentDatabase()||'alter_compression_codecs/{shard}', '2_{replica}') PARTITION BY somedate ORDER BY id; +) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_00910/'||currentDatabase()||'alter_compression_codecs/{shard}', '2_{replica}') PARTITION BY somedate ORDER BY id; INSERT INTO alter_compression_codec1 VALUES('2018-01-01', 1); INSERT INTO alter_compression_codec1 VALUES('2018-01-01', 2); diff --git a/tests/queries/0_stateless/00925_zookeeper_empty_replicated_merge_tree_optimize_final.sql b/tests/queries/0_stateless/00925_zookeeper_empty_replicated_merge_tree_optimize_final.sql index e227222a939..2b119836ebf 100644 --- a/tests/queries/0_stateless/00925_zookeeper_empty_replicated_merge_tree_optimize_final.sql +++ b/tests/queries/0_stateless/00925_zookeeper_empty_replicated_merge_tree_optimize_final.sql @@ -1,7 +1,7 @@ DROP TABLE IF EXISTS replicated_optimize1; DROP TABLE IF EXISTS replicated_optimize2; -CREATE TABLE replicated_optimize1 (d Date, k UInt64, i32 Int32) ENGINE=ReplicatedMergeTree('/clickhouse/tables/test_00925/optimize', 'r1', d, k, 8192); -CREATE TABLE replicated_optimize2 (d Date, k UInt64, i32 Int32) ENGINE=ReplicatedMergeTree('/clickhouse/tables/test_00925/optimize', 'r2', d, k, 8192); +CREATE TABLE replicated_optimize1 (d Date, k UInt64, i32 Int32) ENGINE=ReplicatedMergeTree('/clickhouse/tables/{database}/test_00925/optimize', 'r1', d, k, 8192); +CREATE TABLE replicated_optimize2 (d Date, k UInt64, i32 Int32) ENGINE=ReplicatedMergeTree('/clickhouse/tables/{database}/test_00925/optimize', 'r2', d, k, 8192); OPTIMIZE TABLE replicated_optimize1 FINAL; diff --git a/tests/queries/0_stateless/00926_zookeeper_adaptive_index_granularity_replicated_merge_tree.sql b/tests/queries/0_stateless/00926_zookeeper_adaptive_index_granularity_replicated_merge_tree.sql index 92b34a243a4..1e04679e1f3 100644 --- a/tests/queries/0_stateless/00926_zookeeper_adaptive_index_granularity_replicated_merge_tree.sql +++ b/tests/queries/0_stateless/00926_zookeeper_adaptive_index_granularity_replicated_merge_tree.sql @@ -8,14 +8,14 @@ CREATE TABLE zero_rows_per_granule1 ( k UInt64, v1 UInt64, v2 Int64 -) ENGINE ReplicatedMergeTree('/clickhouse/tables/test_00926/zero_rows_in_granule', '1') PARTITION BY toYYYYMM(p) ORDER BY k SETTINGS index_granularity_bytes = 20, min_index_granularity_bytes = 10, write_final_mark = 0; +) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_00926/zero_rows_in_granule', '1') PARTITION BY toYYYYMM(p) ORDER BY k SETTINGS index_granularity_bytes = 20, min_index_granularity_bytes = 10, write_final_mark = 0; CREATE TABLE zero_rows_per_granule2 ( p Date, k UInt64, v1 UInt64, v2 Int64 -) ENGINE ReplicatedMergeTree('/clickhouse/tables/test_00926/zero_rows_in_granule', '2') PARTITION BY toYYYYMM(p) ORDER BY k SETTINGS index_granularity_bytes = 20, min_index_granularity_bytes = 10, write_final_mark = 0; +) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_00926/zero_rows_in_granule', '2') PARTITION BY toYYYYMM(p) ORDER BY k SETTINGS index_granularity_bytes = 20, min_index_granularity_bytes = 10, write_final_mark = 0; INSERT INTO zero_rows_per_granule1 (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); @@ -68,14 +68,14 @@ CREATE TABLE four_rows_per_granule1 ( k UInt64, v1 UInt64, v2 Int64 -) ENGINE ReplicatedMergeTree('/clickhouse/tables/test_00926/four_rows_in_granule', '1') PARTITION BY toYYYYMM(p) ORDER BY k SETTINGS index_granularity_bytes = 110, min_index_granularity_bytes = 100, write_final_mark = 0; +) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_00926/four_rows_in_granule', '1') PARTITION BY toYYYYMM(p) ORDER BY k SETTINGS index_granularity_bytes = 110, min_index_granularity_bytes = 100, write_final_mark = 0; CREATE TABLE four_rows_per_granule2 ( p Date, k UInt64, v1 UInt64, v2 Int64 -) ENGINE ReplicatedMergeTree('/clickhouse/tables/test_00926/four_rows_in_granule', '2') PARTITION BY toYYYYMM(p) ORDER BY k SETTINGS index_granularity_bytes = 110, min_index_granularity_bytes = 100 ,write_final_mark = 0; +) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_00926/four_rows_in_granule', '2') PARTITION BY toYYYYMM(p) ORDER BY k SETTINGS index_granularity_bytes = 110, min_index_granularity_bytes = 100 ,write_final_mark = 0; INSERT INTO four_rows_per_granule1 (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); @@ -135,14 +135,14 @@ CREATE TABLE adaptive_granularity_alter1 ( k UInt64, v1 UInt64, v2 Int64 -) ENGINE ReplicatedMergeTree('/clickhouse/tables/test_00926/adaptive_granularity_alter', '1') PARTITION BY toYYYYMM(p) ORDER BY k SETTINGS index_granularity_bytes = 110, min_index_granularity_bytes = 100, write_final_mark = 0; +) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_00926/adaptive_granularity_alter', '1') PARTITION BY toYYYYMM(p) ORDER BY k SETTINGS index_granularity_bytes = 110, min_index_granularity_bytes = 100, write_final_mark = 0; CREATE TABLE adaptive_granularity_alter2 ( p Date, k UInt64, v1 UInt64, v2 Int64 -) ENGINE ReplicatedMergeTree('/clickhouse/tables/test_00926/adaptive_granularity_alter', '2') PARTITION BY toYYYYMM(p) ORDER BY k SETTINGS index_granularity_bytes = 110, min_index_granularity_bytes = 100, write_final_mark = 0; +) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_00926/adaptive_granularity_alter', '2') PARTITION BY toYYYYMM(p) ORDER BY k SETTINGS index_granularity_bytes = 110, min_index_granularity_bytes = 100, write_final_mark = 0; INSERT INTO adaptive_granularity_alter1 (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); diff --git a/tests/queries/0_stateless/00933_ttl_replicated_zookeeper.reference b/tests/queries/0_stateless/00933_ttl_replicated_zookeeper.reference index e84c3beabee..3dc07236d2b 100644 --- a/tests/queries/0_stateless/00933_ttl_replicated_zookeeper.reference +++ b/tests/queries/0_stateless/00933_ttl_replicated_zookeeper.reference @@ -1,3 +1,3 @@ 200 400 -CREATE TABLE default.ttl_repl2\n(\n `d` Date,\n `x` UInt32\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00933/ttl_repl\', \'2\')\nPARTITION BY toDayOfMonth(d)\nORDER BY x\nTTL d + toIntervalDay(1)\nSETTINGS index_granularity = 8192 +CREATE TABLE default.ttl_repl2\n(\n `d` Date,\n `x` UInt32\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_00933/ttl_repl\', \'2\')\nPARTITION BY toDayOfMonth(d)\nORDER BY x\nTTL d + toIntervalDay(1)\nSETTINGS index_granularity = 8192 diff --git a/tests/queries/0_stateless/00933_ttl_replicated_zookeeper.sql b/tests/queries/0_stateless/00933_ttl_replicated_zookeeper.sql index dbbbe887e9f..6085545dc02 100644 --- a/tests/queries/0_stateless/00933_ttl_replicated_zookeeper.sql +++ b/tests/queries/0_stateless/00933_ttl_replicated_zookeeper.sql @@ -1,9 +1,9 @@ DROP TABLE IF EXISTS ttl_repl1; DROP TABLE IF EXISTS ttl_repl2; -CREATE TABLE ttl_repl1(d Date, x UInt32) ENGINE ReplicatedMergeTree('/clickhouse/tables/test_00933/ttl_repl', '1') +CREATE TABLE ttl_repl1(d Date, x UInt32) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_00933/ttl_repl', '1') PARTITION BY toDayOfMonth(d) ORDER BY x TTL d + INTERVAL 1 DAY; -CREATE TABLE ttl_repl2(d Date, x UInt32) ENGINE ReplicatedMergeTree('/clickhouse/tables/test_00933/ttl_repl', '2') +CREATE TABLE ttl_repl2(d Date, x UInt32) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_00933/ttl_repl', '2') PARTITION BY toDayOfMonth(d) ORDER BY x TTL d + INTERVAL 1 DAY; INSERT INTO TABLE ttl_repl1 VALUES (toDate('2000-10-10 00:00:00'), 100); diff --git a/tests/queries/0_stateless/00988_constraints_replication_zookeeper.sql b/tests/queries/0_stateless/00988_constraints_replication_zookeeper.sql index f8895843dd3..c36bee7fe4f 100644 --- a/tests/queries/0_stateless/00988_constraints_replication_zookeeper.sql +++ b/tests/queries/0_stateless/00988_constraints_replication_zookeeper.sql @@ -6,14 +6,14 @@ CREATE TABLE replicated_constraints1 a UInt32, b UInt32, CONSTRAINT a_constraint CHECK a < 10 -) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_00988/alter_constraints', 'r1') ORDER BY (a); +) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_00988/alter_constraints', 'r1') ORDER BY (a); CREATE TABLE replicated_constraints2 ( a UInt32, b UInt32, CONSTRAINT a_constraint CHECK a < 10 -) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_00988/alter_constraints', 'r2') ORDER BY (a); +) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_00988/alter_constraints', 'r2') ORDER BY (a); INSERT INTO replicated_constraints1 VALUES (1, 2); INSERT INTO replicated_constraints2 VALUES (3, 4); diff --git a/tests/queries/0_stateless/01037_zookeeper_check_table_empty_pk.sql b/tests/queries/0_stateless/01037_zookeeper_check_table_empty_pk.sql index 60c6d55c4f5..262ccecc7d4 100644 --- a/tests/queries/0_stateless/01037_zookeeper_check_table_empty_pk.sql +++ b/tests/queries/0_stateless/01037_zookeeper_check_table_empty_pk.sql @@ -13,7 +13,7 @@ DROP TABLE IF EXISTS mt_without_pk; DROP TABLE IF EXISTS replicated_mt_without_pk; -CREATE TABLE replicated_mt_without_pk (SomeField1 Int64, SomeField2 Double) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_01037/replicated_mt_without_pk', '1') ORDER BY tuple(); +CREATE TABLE replicated_mt_without_pk (SomeField1 Int64, SomeField2 Double) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_01037/replicated_mt_without_pk', '1') ORDER BY tuple(); INSERT INTO replicated_mt_without_pk VALUES (1, 2); diff --git a/tests/queries/0_stateless/01049_zookeeper_synchronous_mutations.sql b/tests/queries/0_stateless/01049_zookeeper_synchronous_mutations.sql index c84fb22043f..e369b500c35 100644 --- a/tests/queries/0_stateless/01049_zookeeper_synchronous_mutations.sql +++ b/tests/queries/0_stateless/01049_zookeeper_synchronous_mutations.sql @@ -3,9 +3,9 @@ DROP TABLE IF EXISTS table_for_synchronous_mutations2; SELECT 'Replicated'; -CREATE TABLE table_for_synchronous_mutations1(k UInt32, v1 UInt64) ENGINE ReplicatedMergeTree('/clickhouse/tables/test_01049/table_for_synchronous_mutations', '1') ORDER BY k; +CREATE TABLE table_for_synchronous_mutations1(k UInt32, v1 UInt64) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_01049/table_for_synchronous_mutations', '1') ORDER BY k; -CREATE TABLE table_for_synchronous_mutations2(k UInt32, v1 UInt64) ENGINE ReplicatedMergeTree('/clickhouse/tables/test_01049/table_for_synchronous_mutations', '2') ORDER BY k; +CREATE TABLE table_for_synchronous_mutations2(k UInt32, v1 UInt64) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_01049/table_for_synchronous_mutations', '2') ORDER BY k; INSERT INTO table_for_synchronous_mutations1 select number, number from numbers(100000); diff --git a/tests/queries/0_stateless/01062_alter_on_mutataion_zookeeper.sql b/tests/queries/0_stateless/01062_alter_on_mutataion_zookeeper.sql index b830c549dc3..53d7100a38e 100644 --- a/tests/queries/0_stateless/01062_alter_on_mutataion_zookeeper.sql +++ b/tests/queries/0_stateless/01062_alter_on_mutataion_zookeeper.sql @@ -6,7 +6,7 @@ CREATE TABLE test_alter_on_mutation key UInt64, value String ) -ENGINE ReplicatedMergeTree('/clickhouse/tables/test_01062/alter_on_mutation', '1') +ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_01062/alter_on_mutation', '1') ORDER BY key PARTITION BY date; INSERT INTO test_alter_on_mutation select toDate('2020-01-05'), number, toString(number) from system.numbers limit 100; @@ -58,7 +58,7 @@ DROP TABLE IF EXISTS test_alter_on_mutation; DROP TABLE IF EXISTS nested_alter; -CREATE TABLE nested_alter (`d` Date, `k` UInt64, `i32` Int32, `dt` DateTime, `n.ui8` Array(UInt8), `n.s` Array(String), `n.d` Array(Date), `s` String DEFAULT '0') ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_01062/nested_alter', 'r2', d, k, 8192); +CREATE TABLE nested_alter (`d` Date, `k` UInt64, `i32` Int32, `dt` DateTime, `n.ui8` Array(UInt8), `n.s` Array(String), `n.d` Array(Date), `s` String DEFAULT '0') ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_01062/nested_alter', 'r2', d, k, 8192); INSERT INTO nested_alter VALUES ('2015-01-01', 6,38,'2014-07-15 13:26:50',[10,20,30],['asd','qwe','qwe'],['2000-01-01','2000-01-01','2000-01-03'],'100500'); diff --git a/tests/queries/0_stateless/01079_alter_default_zookeeper.reference b/tests/queries/0_stateless/01079_alter_default_zookeeper.reference index 7dd539b2683..32d31af0058 100644 --- a/tests/queries/0_stateless/01079_alter_default_zookeeper.reference +++ b/tests/queries/0_stateless/01079_alter_default_zookeeper.reference @@ -1,11 +1,11 @@ -CREATE TABLE default.alter_default\n(\n `date` Date,\n `key` UInt64,\n `value` String DEFAULT \'10\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_01079/alter_default\', \'1\')\nORDER BY key\nSETTINGS index_granularity = 8192 +CREATE TABLE default.alter_default\n(\n `date` Date,\n `key` UInt64,\n `value` String DEFAULT \'10\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_01079/alter_default\', \'1\')\nORDER BY key\nSETTINGS index_granularity = 8192 1000 -CREATE TABLE default.alter_default\n(\n `date` Date,\n `key` UInt64,\n `value` UInt64 DEFAULT \'10\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_01079/alter_default\', \'1\')\nORDER BY key\nSETTINGS index_granularity = 8192 -CREATE TABLE default.alter_default\n(\n `date` Date,\n `key` UInt64,\n `value` UInt64 DEFAULT 10\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_01079/alter_default\', \'1\')\nORDER BY key\nSETTINGS index_granularity = 8192 +CREATE TABLE default.alter_default\n(\n `date` Date,\n `key` UInt64,\n `value` UInt64 DEFAULT \'10\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_01079/alter_default\', \'1\')\nORDER BY key\nSETTINGS index_granularity = 8192 +CREATE TABLE default.alter_default\n(\n `date` Date,\n `key` UInt64,\n `value` UInt64 DEFAULT 10\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_01079/alter_default\', \'1\')\nORDER BY key\nSETTINGS index_granularity = 8192 1000 -CREATE TABLE default.alter_default\n(\n `date` Date,\n `key` UInt64,\n `value` UInt64 DEFAULT 100\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_01079/alter_default\', \'1\')\nORDER BY key\nSETTINGS index_granularity = 8192 -CREATE TABLE default.alter_default\n(\n `date` Date,\n `key` UInt64,\n `value` UInt16 DEFAULT 100\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_01079/alter_default\', \'1\')\nORDER BY key\nSETTINGS index_granularity = 8192 +CREATE TABLE default.alter_default\n(\n `date` Date,\n `key` UInt64,\n `value` UInt64 DEFAULT 100\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_01079/alter_default\', \'1\')\nORDER BY key\nSETTINGS index_granularity = 8192 +CREATE TABLE default.alter_default\n(\n `date` Date,\n `key` UInt64,\n `value` UInt16 DEFAULT 100\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_01079/alter_default\', \'1\')\nORDER BY key\nSETTINGS index_granularity = 8192 10000 -CREATE TABLE default.alter_default\n(\n `date` Date,\n `key` UInt64,\n `value` UInt8 DEFAULT 10\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_01079/alter_default\', \'1\')\nORDER BY key\nSETTINGS index_granularity = 8192 -CREATE TABLE default.alter_default\n(\n `date` Date,\n `key` UInt64,\n `value` UInt8 DEFAULT 10,\n `better_column` UInt8 DEFAULT \'1\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_01079/alter_default\', \'1\')\nORDER BY key\nSETTINGS index_granularity = 8192 -CREATE TABLE default.alter_default\n(\n `date` Date,\n `key` UInt64,\n `value` UInt8 DEFAULT 10,\n `better_column` UInt8 DEFAULT \'1\',\n `other_date` String DEFAULT 1\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_01079/alter_default\', \'1\')\nORDER BY key\nSETTINGS index_granularity = 8192 +CREATE TABLE default.alter_default\n(\n `date` Date,\n `key` UInt64,\n `value` UInt8 DEFAULT 10\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_01079/alter_default\', \'1\')\nORDER BY key\nSETTINGS index_granularity = 8192 +CREATE TABLE default.alter_default\n(\n `date` Date,\n `key` UInt64,\n `value` UInt8 DEFAULT 10,\n `better_column` UInt8 DEFAULT \'1\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_01079/alter_default\', \'1\')\nORDER BY key\nSETTINGS index_granularity = 8192 +CREATE TABLE default.alter_default\n(\n `date` Date,\n `key` UInt64,\n `value` UInt8 DEFAULT 10,\n `better_column` UInt8 DEFAULT \'1\',\n `other_date` String DEFAULT 1\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_01079/alter_default\', \'1\')\nORDER BY key\nSETTINGS index_granularity = 8192 diff --git a/tests/queries/0_stateless/01079_alter_default_zookeeper.sql b/tests/queries/0_stateless/01079_alter_default_zookeeper.sql index 6fa9d2bf4e0..7257f86c4e8 100644 --- a/tests/queries/0_stateless/01079_alter_default_zookeeper.sql +++ b/tests/queries/0_stateless/01079_alter_default_zookeeper.sql @@ -5,7 +5,7 @@ CREATE TABLE alter_default date Date, key UInt64 ) -ENGINE ReplicatedMergeTree('/clickhouse/tables/test_01079/alter_default', '1') +ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_01079/alter_default', '1') ORDER BY key; INSERT INTO alter_default select toDate('2020-01-05'), number from system.numbers limit 100; diff --git a/tests/queries/0_stateless/01090_zookeeper_mutations_and_insert_quorum.sql b/tests/queries/0_stateless/01090_zookeeper_mutations_and_insert_quorum.sql index 5597bf523e4..d38c639458d 100644 --- a/tests/queries/0_stateless/01090_zookeeper_mutations_and_insert_quorum.sql +++ b/tests/queries/0_stateless/01090_zookeeper_mutations_and_insert_quorum.sql @@ -1,8 +1,8 @@ DROP TABLE IF EXISTS mutations_and_quorum1; DROP TABLE IF EXISTS mutations_and_quorum2; -CREATE TABLE mutations_and_quorum1 (`server_date` Date, `something` String) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_01090/mutations_and_quorum', '1') PARTITION BY toYYYYMM(server_date) ORDER BY (server_date, something); -CREATE TABLE mutations_and_quorum2 (`server_date` Date, `something` String) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_01090/mutations_and_quorum', '2') PARTITION BY toYYYYMM(server_date) ORDER BY (server_date, something); +CREATE TABLE mutations_and_quorum1 (`server_date` Date, `something` String) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_01090/mutations_and_quorum', '1') PARTITION BY toYYYYMM(server_date) ORDER BY (server_date, something); +CREATE TABLE mutations_and_quorum2 (`server_date` Date, `something` String) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_01090/mutations_and_quorum', '2') PARTITION BY toYYYYMM(server_date) ORDER BY (server_date, something); SET insert_quorum=2, insert_quorum_parallel=0; diff --git a/tests/queries/0_stateless/01135_default_and_alter_zookeeper.reference b/tests/queries/0_stateless/01135_default_and_alter_zookeeper.reference index cf69f1c5896..30cdc08e87d 100644 --- a/tests/queries/0_stateless/01135_default_and_alter_zookeeper.reference +++ b/tests/queries/0_stateless/01135_default_and_alter_zookeeper.reference @@ -1,2 +1,2 @@ 4 -CREATE TABLE default.default_table\n(\n `id` UInt64,\n `enum_column` Enum8(\'undefined\' = 0, \'fox\' = 1, \'index\' = 2) DEFAULT \'fox\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/test_01135/default_table\', \'1\')\nORDER BY tuple()\nSETTINGS index_granularity = 8192 +CREATE TABLE default.default_table\n(\n `id` UInt64,\n `enum_column` Enum8(\'undefined\' = 0, \'fox\' = 1, \'index\' = 2) DEFAULT \'fox\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/default/test_01135/default_table\', \'1\')\nORDER BY tuple()\nSETTINGS index_granularity = 8192 diff --git a/tests/queries/0_stateless/01135_default_and_alter_zookeeper.sql b/tests/queries/0_stateless/01135_default_and_alter_zookeeper.sql index 982c72eb333..ea8c943b9a4 100644 --- a/tests/queries/0_stateless/01135_default_and_alter_zookeeper.sql +++ b/tests/queries/0_stateless/01135_default_and_alter_zookeeper.sql @@ -5,7 +5,7 @@ CREATE TABLE default_table id UInt64, enum_column Enum8('undefined' = 0, 'fox' = 1, 'index' = 2) ) -ENGINE ReplicatedMergeTree('/clickhouse/test_01135/default_table', '1') +ENGINE ReplicatedMergeTree('/clickhouse/{database}/test_01135/default_table', '1') ORDER BY tuple(); INSERT INTO default_table VALUES(1, 'index'), (2, 'fox'); diff --git a/tests/queries/0_stateless/01201_drop_column_compact_part_replicated_zookeeper.sql b/tests/queries/0_stateless/01201_drop_column_compact_part_replicated_zookeeper.sql index 2aa903d99f0..9464ac24dff 100644 --- a/tests/queries/0_stateless/01201_drop_column_compact_part_replicated_zookeeper.sql +++ b/tests/queries/0_stateless/01201_drop_column_compact_part_replicated_zookeeper.sql @@ -3,7 +3,7 @@ set replication_alter_partitions_sync = 2; drop table if exists mt_compact; create table mt_compact(a UInt64, b UInt64 DEFAULT a * a, s String, n Nested(x UInt32, y String), lc LowCardinality(String)) -engine = ReplicatedMergeTree('/clickhouse/test_01201/mt_compact_replicated', '1') +engine = ReplicatedMergeTree('/clickhouse/{database}/test_01201/mt_compact_replicated', '1') order by a partition by a % 10 settings index_granularity = 8, min_rows_for_wide_part = 10; diff --git a/tests/queries/0_stateless/01213_alter_rename_primary_key_zookeeper.sql b/tests/queries/0_stateless/01213_alter_rename_primary_key_zookeeper.sql index 616a213e46c..91dd6e67512 100644 --- a/tests/queries/0_stateless/01213_alter_rename_primary_key_zookeeper.sql +++ b/tests/queries/0_stateless/01213_alter_rename_primary_key_zookeeper.sql @@ -9,7 +9,7 @@ CREATE TABLE table_for_rename_pk value1 String, value2 String ) -ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_01213/table_for_rename_pk1', '1') +ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_01213/table_for_rename_pk1', '1') PARTITION BY date ORDER BY (key1, pow(key2, 2), key3); @@ -37,7 +37,7 @@ CREATE TABLE table_for_rename_with_primary_key value2 String, INDEX idx (value1) TYPE set(1) GRANULARITY 1 ) -ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_01213/table_for_rename_pk2', '1') +ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_01213/table_for_rename_pk2', '1') PARTITION BY date ORDER BY (key1, key2, key3) PRIMARY KEY (key1, key2); diff --git a/tests/queries/0_stateless/01213_alter_rename_with_default_zookeeper.reference b/tests/queries/0_stateless/01213_alter_rename_with_default_zookeeper.reference index da3dad5cb16..2a6b00cdddb 100644 --- a/tests/queries/0_stateless/01213_alter_rename_with_default_zookeeper.reference +++ b/tests/queries/0_stateless/01213_alter_rename_with_default_zookeeper.reference @@ -8,10 +8,10 @@ Hello 1 Word 1 date1 date2 value1 value2 2019-10-02 2018-10-02 1 1 -CREATE TABLE default.table_rename_with_ttl\n(\n `date1` Date,\n `date2` Date,\n `value1` String,\n `value2` String TTL date1 + toIntervalMonth(10000)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/test_01213/table_rename_with_ttl\', \'1\')\nORDER BY tuple()\nTTL date2 + toIntervalMonth(10000)\nSETTINGS index_granularity = 8192 +CREATE TABLE default.table_rename_with_ttl\n(\n `date1` Date,\n `date2` Date,\n `value1` String,\n `value2` String TTL date1 + toIntervalMonth(10000)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/default/test_01213/table_rename_with_ttl\', \'1\')\nORDER BY tuple()\nTTL date2 + toIntervalMonth(10000)\nSETTINGS index_granularity = 8192 renamed_date1 date2 value1 value2 2019-10-02 2018-10-02 1 1 -CREATE TABLE default.table_rename_with_ttl\n(\n `renamed_date1` Date,\n `date2` Date,\n `value1` String,\n `value2` String TTL renamed_date1 + toIntervalMonth(10000)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/test_01213/table_rename_with_ttl\', \'1\')\nORDER BY tuple()\nTTL date2 + toIntervalMonth(10000)\nSETTINGS index_granularity = 8192 +CREATE TABLE default.table_rename_with_ttl\n(\n `renamed_date1` Date,\n `date2` Date,\n `value1` String,\n `value2` String TTL renamed_date1 + toIntervalMonth(10000)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/default/test_01213/table_rename_with_ttl\', \'1\')\nORDER BY tuple()\nTTL date2 + toIntervalMonth(10000)\nSETTINGS index_granularity = 8192 renamed_date1 renamed_date2 value1 value2 2019-10-02 2018-10-02 1 1 -CREATE TABLE default.table_rename_with_ttl\n(\n `renamed_date1` Date,\n `renamed_date2` Date,\n `value1` String,\n `value2` String TTL renamed_date1 + toIntervalMonth(10000)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/test_01213/table_rename_with_ttl\', \'1\')\nORDER BY tuple()\nTTL renamed_date2 + toIntervalMonth(10000)\nSETTINGS index_granularity = 8192 +CREATE TABLE default.table_rename_with_ttl\n(\n `renamed_date1` Date,\n `renamed_date2` Date,\n `value1` String,\n `value2` String TTL renamed_date1 + toIntervalMonth(10000)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/default/test_01213/table_rename_with_ttl\', \'1\')\nORDER BY tuple()\nTTL renamed_date2 + toIntervalMonth(10000)\nSETTINGS index_granularity = 8192 diff --git a/tests/queries/0_stateless/01213_alter_rename_with_default_zookeeper.sql b/tests/queries/0_stateless/01213_alter_rename_with_default_zookeeper.sql index e5701077770..cb9e918e335 100644 --- a/tests/queries/0_stateless/01213_alter_rename_with_default_zookeeper.sql +++ b/tests/queries/0_stateless/01213_alter_rename_with_default_zookeeper.sql @@ -38,7 +38,7 @@ CREATE TABLE table_rename_with_ttl value1 String, value2 String TTL date1 + INTERVAL 10000 MONTH ) -ENGINE = ReplicatedMergeTree('/clickhouse/test_01213/table_rename_with_ttl', '1') +ENGINE = ReplicatedMergeTree('/clickhouse/{database}/test_01213/table_rename_with_ttl', '1') ORDER BY tuple() TTL date2 + INTERVAL 10000 MONTH; diff --git a/tests/queries/0_stateless/01267_alter_default_key_columns_zookeeper.sql b/tests/queries/0_stateless/01267_alter_default_key_columns_zookeeper.sql index d96085bc086..6e7f4d55a53 100644 --- a/tests/queries/0_stateless/01267_alter_default_key_columns_zookeeper.sql +++ b/tests/queries/0_stateless/01267_alter_default_key_columns_zookeeper.sql @@ -8,8 +8,8 @@ DROP TABLE test_alter; DROP TABLE IF EXISTS test_alter_r1; DROP TABLE IF EXISTS test_alter_r2; -CREATE TABLE test_alter_r1 (x Date, s String) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_01267/alter', 'r1') ORDER BY s PARTITION BY x; -CREATE TABLE test_alter_r2 (x Date, s String) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_01267/alter', 'r2') ORDER BY s PARTITION BY x; +CREATE TABLE test_alter_r1 (x Date, s String) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_01267/alter', 'r1') ORDER BY s PARTITION BY x; +CREATE TABLE test_alter_r2 (x Date, s String) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_01267/alter', 'r2') ORDER BY s PARTITION BY x; ALTER TABLE test_alter_r1 MODIFY COLUMN s DEFAULT 'Hello' SETTINGS replication_alter_partitions_sync = 2; ALTER TABLE test_alter_r2 MODIFY COLUMN x DEFAULT '2000-01-01' SETTINGS replication_alter_partitions_sync = 2; diff --git a/tests/queries/0_stateless/01277_alter_rename_column_constraint_zookeeper.reference b/tests/queries/0_stateless/01277_alter_rename_column_constraint_zookeeper.reference index 84ca8273128..382ccb592af 100644 --- a/tests/queries/0_stateless/01277_alter_rename_column_constraint_zookeeper.reference +++ b/tests/queries/0_stateless/01277_alter_rename_column_constraint_zookeeper.reference @@ -7,7 +7,7 @@ 2019-10-01 6 6 7 8 2019-10-02 7 7 8 9 2019-10-03 8 8 9 10 -CREATE TABLE default.table_for_rename1\n(\n `date` Date,\n `key` UInt64,\n `value4` String,\n `value5` String,\n `value3` String,\n CONSTRAINT cs_value1 CHECK toInt64(value4) < toInt64(value5),\n CONSTRAINT cs_value2 CHECK toInt64(value5) < toInt64(value3)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_01277/test_for_rename\', \'1\')\nPARTITION BY date\nORDER BY key\nSETTINGS index_granularity = 8192 +CREATE TABLE default.table_for_rename1\n(\n `date` Date,\n `key` UInt64,\n `value4` String,\n `value5` String,\n `value3` String,\n CONSTRAINT cs_value1 CHECK toInt64(value4) < toInt64(value5),\n CONSTRAINT cs_value2 CHECK toInt64(value5) < toInt64(value3)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_01277/test_for_rename\', \'1\')\nPARTITION BY date\nORDER BY key\nSETTINGS index_granularity = 8192 2019-10-01 0 0 1 2 2019-10-02 1 1 2 3 2019-10-03 2 2 3 4 @@ -38,7 +38,7 @@ CREATE TABLE default.table_for_rename1\n(\n `date` Date,\n `key` UInt64,\n 2019-10-01 18 18 19 20 2019-10-02 19 19 20 21 -- rename columns back -- -CREATE TABLE default.table_for_rename1\n(\n `date` Date,\n `key` UInt64,\n `value1` String,\n `value2` String,\n `value3` String,\n CONSTRAINT cs_value1 CHECK toInt64(value1) < toInt64(value2),\n CONSTRAINT cs_value2 CHECK toInt64(value2) < toInt64(value3)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_01277/test_for_rename\', \'1\')\nPARTITION BY date\nORDER BY key\nSETTINGS index_granularity = 8192 +CREATE TABLE default.table_for_rename1\n(\n `date` Date,\n `key` UInt64,\n `value1` String,\n `value2` String,\n `value3` String,\n CONSTRAINT cs_value1 CHECK toInt64(value1) < toInt64(value2),\n CONSTRAINT cs_value2 CHECK toInt64(value2) < toInt64(value3)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_01277/test_for_rename\', \'1\')\nPARTITION BY date\nORDER BY key\nSETTINGS index_granularity = 8192 2019-10-01 0 0 1 2 2019-10-02 1 1 2 3 2019-10-03 2 2 3 4 diff --git a/tests/queries/0_stateless/01277_alter_rename_column_constraint_zookeeper.sql b/tests/queries/0_stateless/01277_alter_rename_column_constraint_zookeeper.sql index 28f17dced97..10ab75e42bd 100644 --- a/tests/queries/0_stateless/01277_alter_rename_column_constraint_zookeeper.sql +++ b/tests/queries/0_stateless/01277_alter_rename_column_constraint_zookeeper.sql @@ -10,7 +10,7 @@ CREATE TABLE table_for_rename1 CONSTRAINT cs_value1 CHECK toInt64(value1) < toInt64(value2), CONSTRAINT cs_value2 CHECK toInt64(value2) < toInt64(value3) ) -ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_01277/test_for_rename', '1') +ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_01277/test_for_rename', '1') PARTITION BY date ORDER BY key; diff --git a/tests/queries/0_stateless/01319_manual_write_to_replicas.sql b/tests/queries/0_stateless/01319_manual_write_to_replicas.sql index 5388f0017c0..7fb4d0b7d61 100644 --- a/tests/queries/0_stateless/01319_manual_write_to_replicas.sql +++ b/tests/queries/0_stateless/01319_manual_write_to_replicas.sql @@ -1,8 +1,8 @@ DROP TABLE IF EXISTS r1; DROP TABLE IF EXISTS r2; -CREATE TABLE r1 (x String) ENGINE = ReplicatedMergeTree('/clickhouse/tables/r', 'r1') ORDER BY x; -CREATE TABLE r2 (x String) ENGINE = ReplicatedMergeTree('/clickhouse/tables/r', 'r2') ORDER BY x; +CREATE TABLE r1 (x String) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/r', 'r1') ORDER BY x; +CREATE TABLE r2 (x String) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/r', 'r2') ORDER BY x; SYSTEM STOP REPLICATED SENDS; diff --git a/tests/queries/0_stateless/01346_alter_enum_partition_key_replicated_zookeeper.sql b/tests/queries/0_stateless/01346_alter_enum_partition_key_replicated_zookeeper.sql index d185973f564..d0b6fbe3dde 100644 --- a/tests/queries/0_stateless/01346_alter_enum_partition_key_replicated_zookeeper.sql +++ b/tests/queries/0_stateless/01346_alter_enum_partition_key_replicated_zookeeper.sql @@ -1,8 +1,8 @@ DROP TABLE IF EXISTS test; DROP TABLE IF EXISTS test2; -CREATE TABLE test (x Enum('hello' = 1, 'world' = 2), y String) ENGINE = ReplicatedMergeTree('/clickhouse/test_01346/table', 'r1') PARTITION BY x ORDER BY y; -CREATE TABLE test2 (x Enum('hello' = 1, 'world' = 2), y String) ENGINE = ReplicatedMergeTree('/clickhouse/test_01346/table', 'r2') PARTITION BY x ORDER BY y; +CREATE TABLE test (x Enum('hello' = 1, 'world' = 2), y String) ENGINE = ReplicatedMergeTree('/clickhouse/{database}/test_01346/table', 'r1') PARTITION BY x ORDER BY y; +CREATE TABLE test2 (x Enum('hello' = 1, 'world' = 2), y String) ENGINE = ReplicatedMergeTree('/clickhouse/{database}/test_01346/table', 'r2') PARTITION BY x ORDER BY y; INSERT INTO test VALUES ('hello', 'test'); SELECT * FROM test; diff --git a/tests/queries/0_stateless/01378_alter_rename_with_ttl_zookeeper.sql b/tests/queries/0_stateless/01378_alter_rename_with_ttl_zookeeper.sql index 0cd6feb9da1..79d1276ae61 100644 --- a/tests/queries/0_stateless/01378_alter_rename_with_ttl_zookeeper.sql +++ b/tests/queries/0_stateless/01378_alter_rename_with_ttl_zookeeper.sql @@ -5,7 +5,7 @@ CREATE TABLE table_rename_with_ttl date1 Date, value1 String ) -ENGINE = ReplicatedMergeTree('/clickhouse/test/table_rename_with_ttl_01378', '1') +ENGINE = ReplicatedMergeTree('/clickhouse/{database}/test/table_rename_with_ttl_01378', '1') ORDER BY tuple(); INSERT INTO table_rename_with_ttl SELECT toDate('2018-10-01') + number % 3, toString(number) from numbers(9); diff --git a/tests/queries/0_stateless/01430_modify_sample_by_zookeeper.sql b/tests/queries/0_stateless/01430_modify_sample_by_zookeeper.sql index dc7f5017bfd..288fa97d218 100644 --- a/tests/queries/0_stateless/01430_modify_sample_by_zookeeper.sql +++ b/tests/queries/0_stateless/01430_modify_sample_by_zookeeper.sql @@ -11,7 +11,7 @@ SELECT count(), min(x), max(x), sum(x), uniqExact(x) FROM modify_sample SAMPLE 0 ALTER TABLE modify_sample MODIFY SAMPLE BY x; SELECT count(), min(x), max(x), sum(x), uniqExact(x) FROM modify_sample SAMPLE 0.1; -CREATE TABLE modify_sample_replicated (d Date DEFAULT '2000-01-01', x UInt8, y UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_01430', 'modify_sample') PARTITION BY d ORDER BY (x, y); +CREATE TABLE modify_sample_replicated (d Date DEFAULT '2000-01-01', x UInt8, y UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_01430', 'modify_sample') PARTITION BY d ORDER BY (x, y); INSERT INTO modify_sample_replicated (x, y) SELECT toUInt8(number) AS x, toUInt64(number) as y FROM system.numbers LIMIT 256; diff --git a/tests/queries/0_stateless/01451_replicated_detach_drop_and_quorum.sql b/tests/queries/0_stateless/01451_replicated_detach_drop_and_quorum.sql index 72e588c5c02..2c6bafe4936 100644 --- a/tests/queries/0_stateless/01451_replicated_detach_drop_and_quorum.sql +++ b/tests/queries/0_stateless/01451_replicated_detach_drop_and_quorum.sql @@ -4,8 +4,8 @@ SET replication_alter_partitions_sync = 2; DROP TABLE IF EXISTS replica1; DROP TABLE IF EXISTS replica2; -CREATE TABLE replica1 (v UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/01451/quorum', 'r1') order by tuple() settings max_replicated_merges_in_queue = 0; -CREATE TABLE replica2 (v UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/01451/quorum', 'r2') order by tuple() settings max_replicated_merges_in_queue = 0; +CREATE TABLE replica1 (v UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test/01451/quorum', 'r1') order by tuple() settings max_replicated_merges_in_queue = 0; +CREATE TABLE replica2 (v UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test/01451/quorum', 'r2') order by tuple() settings max_replicated_merges_in_queue = 0; INSERT INTO replica1 VALUES (0); diff --git a/tests/queries/0_stateless/01451_replicated_detach_drop_part_long.sql b/tests/queries/0_stateless/01451_replicated_detach_drop_part_long.sql index cd8267ce59a..5da86f45fde 100644 --- a/tests/queries/0_stateless/01451_replicated_detach_drop_part_long.sql +++ b/tests/queries/0_stateless/01451_replicated_detach_drop_part_long.sql @@ -3,8 +3,8 @@ SET replication_alter_partitions_sync = 2; DROP TABLE IF EXISTS replica1; DROP TABLE IF EXISTS replica2; -CREATE TABLE replica1 (v UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/'||currentDatabase()||'test/01451/attach', 'r1') order by tuple() settings max_replicated_merges_in_queue = 0; -CREATE TABLE replica2 (v UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/'||currentDatabase()||'test/01451/attach', 'r2') order by tuple() settings max_replicated_merges_in_queue = 0; +CREATE TABLE replica1 (v UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/'||currentDatabase()||'test/01451/attach', 'r1') order by tuple() settings max_replicated_merges_in_queue = 0; +CREATE TABLE replica2 (v UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/'||currentDatabase()||'test/01451/attach', 'r2') order by tuple() settings max_replicated_merges_in_queue = 0; INSERT INTO replica1 VALUES (0); INSERT INTO replica1 VALUES (1); diff --git a/tests/queries/0_stateless/01493_alter_remove_no_property_zookeeper.reference b/tests/queries/0_stateless/01493_alter_remove_no_property_zookeeper.reference index 82f6fbd6615..a770460c8b8 100644 --- a/tests/queries/0_stateless/01493_alter_remove_no_property_zookeeper.reference +++ b/tests/queries/0_stateless/01493_alter_remove_no_property_zookeeper.reference @@ -1,4 +1,4 @@ CREATE TABLE default.no_prop_table\n(\n `some_column` UInt64\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS index_granularity = 8192 CREATE TABLE default.no_prop_table\n(\n `some_column` UInt64\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS index_granularity = 8192 -CREATE TABLE default.r_no_prop_table\n(\n `some_column` UInt64\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/test/01493_r_no_prop_table\', \'1\')\nORDER BY tuple()\nSETTINGS index_granularity = 8192 -CREATE TABLE default.r_no_prop_table\n(\n `some_column` UInt64\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/test/01493_r_no_prop_table\', \'1\')\nORDER BY tuple()\nSETTINGS index_granularity = 8192 +CREATE TABLE default.r_no_prop_table\n(\n `some_column` UInt64\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/default/test/01493_r_no_prop_table\', \'1\')\nORDER BY tuple()\nSETTINGS index_granularity = 8192 +CREATE TABLE default.r_no_prop_table\n(\n `some_column` UInt64\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/default/test/01493_r_no_prop_table\', \'1\')\nORDER BY tuple()\nSETTINGS index_granularity = 8192 diff --git a/tests/queries/0_stateless/01493_alter_remove_no_property_zookeeper.sql b/tests/queries/0_stateless/01493_alter_remove_no_property_zookeeper.sql index aceb4e8140d..8d37900aaf8 100644 --- a/tests/queries/0_stateless/01493_alter_remove_no_property_zookeeper.sql +++ b/tests/queries/0_stateless/01493_alter_remove_no_property_zookeeper.sql @@ -29,7 +29,7 @@ CREATE TABLE r_no_prop_table ( some_column UInt64 ) -ENGINE ReplicatedMergeTree('/clickhouse/test/01493_r_no_prop_table', '1') +ENGINE ReplicatedMergeTree('/clickhouse/{database}/test/01493_r_no_prop_table', '1') ORDER BY tuple(); SHOW CREATE TABLE r_no_prop_table; diff --git a/tests/queries/0_stateless/01493_alter_remove_properties_zookeeper.reference b/tests/queries/0_stateless/01493_alter_remove_properties_zookeeper.reference index 7cd2c81b460..42f9b24715f 100644 --- a/tests/queries/0_stateless/01493_alter_remove_properties_zookeeper.reference +++ b/tests/queries/0_stateless/01493_alter_remove_properties_zookeeper.reference @@ -1,21 +1,21 @@ -CREATE TABLE default.r_prop_table1\n(\n `column_default` UInt64 DEFAULT 42,\n `column_codec` String CODEC(ZSTD(10)),\n `column_comment` Date COMMENT \'Some comment\',\n `column_ttl` UInt64 TTL column_comment + toIntervalMonth(1)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/test_01493/r_prop_table\', \'1\')\nORDER BY tuple()\nTTL column_comment + toIntervalMonth(2)\nSETTINGS index_granularity = 8192 -CREATE TABLE default.r_prop_table2\n(\n `column_default` UInt64 DEFAULT 42,\n `column_codec` String CODEC(ZSTD(10)),\n `column_comment` Date COMMENT \'Some comment\',\n `column_ttl` UInt64 TTL column_comment + toIntervalMonth(1)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/test_01493/r_prop_table\', \'2\')\nORDER BY tuple()\nTTL column_comment + toIntervalMonth(2)\nSETTINGS index_granularity = 8192 +CREATE TABLE default.r_prop_table1\n(\n `column_default` UInt64 DEFAULT 42,\n `column_codec` String CODEC(ZSTD(10)),\n `column_comment` Date COMMENT \'Some comment\',\n `column_ttl` UInt64 TTL column_comment + toIntervalMonth(1)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/default/test_01493/r_prop_table\', \'1\')\nORDER BY tuple()\nTTL column_comment + toIntervalMonth(2)\nSETTINGS index_granularity = 8192 +CREATE TABLE default.r_prop_table2\n(\n `column_default` UInt64 DEFAULT 42,\n `column_codec` String CODEC(ZSTD(10)),\n `column_comment` Date COMMENT \'Some comment\',\n `column_ttl` UInt64 TTL column_comment + toIntervalMonth(1)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/default/test_01493/r_prop_table\', \'2\')\nORDER BY tuple()\nTTL column_comment + toIntervalMonth(2)\nSETTINGS index_granularity = 8192 ====== remove column comment ====== -CREATE TABLE default.r_prop_table1\n(\n `column_default` UInt64 DEFAULT 42,\n `column_codec` String CODEC(ZSTD(10)),\n `column_comment` Date,\n `column_ttl` UInt64 TTL column_comment + toIntervalMonth(1)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/test_01493/r_prop_table\', \'1\')\nORDER BY tuple()\nTTL column_comment + toIntervalMonth(2)\nSETTINGS index_granularity = 8192 -CREATE TABLE default.r_prop_table2\n(\n `column_default` UInt64 DEFAULT 42,\n `column_codec` String CODEC(ZSTD(10)),\n `column_comment` Date,\n `column_ttl` UInt64 TTL column_comment + toIntervalMonth(1)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/test_01493/r_prop_table\', \'2\')\nORDER BY tuple()\nTTL column_comment + toIntervalMonth(2)\nSETTINGS index_granularity = 8192 +CREATE TABLE default.r_prop_table1\n(\n `column_default` UInt64 DEFAULT 42,\n `column_codec` String CODEC(ZSTD(10)),\n `column_comment` Date,\n `column_ttl` UInt64 TTL column_comment + toIntervalMonth(1)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/default/test_01493/r_prop_table\', \'1\')\nORDER BY tuple()\nTTL column_comment + toIntervalMonth(2)\nSETTINGS index_granularity = 8192 +CREATE TABLE default.r_prop_table2\n(\n `column_default` UInt64 DEFAULT 42,\n `column_codec` String CODEC(ZSTD(10)),\n `column_comment` Date,\n `column_ttl` UInt64 TTL column_comment + toIntervalMonth(1)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/default/test_01493/r_prop_table\', \'2\')\nORDER BY tuple()\nTTL column_comment + toIntervalMonth(2)\nSETTINGS index_granularity = 8192 ====== remove column codec ====== -CREATE TABLE default.r_prop_table1\n(\n `column_default` UInt64 DEFAULT 42,\n `column_codec` String,\n `column_comment` Date,\n `column_ttl` UInt64 TTL column_comment + toIntervalMonth(1)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/test_01493/r_prop_table\', \'1\')\nORDER BY tuple()\nTTL column_comment + toIntervalMonth(2)\nSETTINGS index_granularity = 8192 -CREATE TABLE default.r_prop_table2\n(\n `column_default` UInt64 DEFAULT 42,\n `column_codec` String,\n `column_comment` Date,\n `column_ttl` UInt64 TTL column_comment + toIntervalMonth(1)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/test_01493/r_prop_table\', \'2\')\nORDER BY tuple()\nTTL column_comment + toIntervalMonth(2)\nSETTINGS index_granularity = 8192 +CREATE TABLE default.r_prop_table1\n(\n `column_default` UInt64 DEFAULT 42,\n `column_codec` String,\n `column_comment` Date,\n `column_ttl` UInt64 TTL column_comment + toIntervalMonth(1)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/default/test_01493/r_prop_table\', \'1\')\nORDER BY tuple()\nTTL column_comment + toIntervalMonth(2)\nSETTINGS index_granularity = 8192 +CREATE TABLE default.r_prop_table2\n(\n `column_default` UInt64 DEFAULT 42,\n `column_codec` String,\n `column_comment` Date,\n `column_ttl` UInt64 TTL column_comment + toIntervalMonth(1)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/default/test_01493/r_prop_table\', \'2\')\nORDER BY tuple()\nTTL column_comment + toIntervalMonth(2)\nSETTINGS index_granularity = 8192 ====== remove column default ====== 42 str 1 0 tsr 2 -CREATE TABLE default.r_prop_table1\n(\n `column_default` UInt64,\n `column_codec` String,\n `column_comment` Date,\n `column_ttl` UInt64 TTL column_comment + toIntervalMonth(1)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/test_01493/r_prop_table\', \'1\')\nORDER BY tuple()\nTTL column_comment + toIntervalMonth(2)\nSETTINGS index_granularity = 8192 -CREATE TABLE default.r_prop_table2\n(\n `column_default` UInt64,\n `column_codec` String,\n `column_comment` Date,\n `column_ttl` UInt64 TTL column_comment + toIntervalMonth(1)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/test_01493/r_prop_table\', \'2\')\nORDER BY tuple()\nTTL column_comment + toIntervalMonth(2)\nSETTINGS index_granularity = 8192 +CREATE TABLE default.r_prop_table1\n(\n `column_default` UInt64,\n `column_codec` String,\n `column_comment` Date,\n `column_ttl` UInt64 TTL column_comment + toIntervalMonth(1)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/default/test_01493/r_prop_table\', \'1\')\nORDER BY tuple()\nTTL column_comment + toIntervalMonth(2)\nSETTINGS index_granularity = 8192 +CREATE TABLE default.r_prop_table2\n(\n `column_default` UInt64,\n `column_codec` String,\n `column_comment` Date,\n `column_ttl` UInt64 TTL column_comment + toIntervalMonth(1)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/default/test_01493/r_prop_table\', \'2\')\nORDER BY tuple()\nTTL column_comment + toIntervalMonth(2)\nSETTINGS index_granularity = 8192 ====== remove column TTL ====== -CREATE TABLE default.r_prop_table1\n(\n `column_default` UInt64,\n `column_codec` String,\n `column_comment` Date,\n `column_ttl` UInt64\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/test_01493/r_prop_table\', \'1\')\nORDER BY tuple()\nTTL column_comment + toIntervalMonth(2)\nSETTINGS index_granularity = 8192 -CREATE TABLE default.r_prop_table2\n(\n `column_default` UInt64,\n `column_codec` String,\n `column_comment` Date,\n `column_ttl` UInt64\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/test_01493/r_prop_table\', \'2\')\nORDER BY tuple()\nTTL column_comment + toIntervalMonth(2)\nSETTINGS index_granularity = 8192 +CREATE TABLE default.r_prop_table1\n(\n `column_default` UInt64,\n `column_codec` String,\n `column_comment` Date,\n `column_ttl` UInt64\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/default/test_01493/r_prop_table\', \'1\')\nORDER BY tuple()\nTTL column_comment + toIntervalMonth(2)\nSETTINGS index_granularity = 8192 +CREATE TABLE default.r_prop_table2\n(\n `column_default` UInt64,\n `column_codec` String,\n `column_comment` Date,\n `column_ttl` UInt64\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/default/test_01493/r_prop_table\', \'2\')\nORDER BY tuple()\nTTL column_comment + toIntervalMonth(2)\nSETTINGS index_granularity = 8192 ====== remove table TTL ====== -CREATE TABLE default.r_prop_table1\n(\n `column_default` UInt64,\n `column_codec` String,\n `column_comment` Date,\n `column_ttl` UInt64\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/test_01493/r_prop_table\', \'1\')\nORDER BY tuple()\nSETTINGS index_granularity = 8192 -CREATE TABLE default.r_prop_table2\n(\n `column_default` UInt64,\n `column_codec` String,\n `column_comment` Date,\n `column_ttl` UInt64\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/test_01493/r_prop_table\', \'2\')\nORDER BY tuple()\nSETTINGS index_granularity = 8192 +CREATE TABLE default.r_prop_table1\n(\n `column_default` UInt64,\n `column_codec` String,\n `column_comment` Date,\n `column_ttl` UInt64\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/default/test_01493/r_prop_table\', \'1\')\nORDER BY tuple()\nSETTINGS index_granularity = 8192 +CREATE TABLE default.r_prop_table2\n(\n `column_default` UInt64,\n `column_codec` String,\n `column_comment` Date,\n `column_ttl` UInt64\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/default/test_01493/r_prop_table\', \'2\')\nORDER BY tuple()\nSETTINGS index_granularity = 8192 3 3 diff --git a/tests/queries/0_stateless/01493_alter_remove_properties_zookeeper.sql b/tests/queries/0_stateless/01493_alter_remove_properties_zookeeper.sql index b810e2a8737..55f610eb7d0 100644 --- a/tests/queries/0_stateless/01493_alter_remove_properties_zookeeper.sql +++ b/tests/queries/0_stateless/01493_alter_remove_properties_zookeeper.sql @@ -10,7 +10,7 @@ CREATE TABLE r_prop_table1 column_comment Date COMMENT 'Some comment', column_ttl UInt64 TTL column_comment + INTERVAL 1 MONTH ) -ENGINE ReplicatedMergeTree('/clickhouse/test_01493/r_prop_table', '1') +ENGINE ReplicatedMergeTree('/clickhouse/{database}/test_01493/r_prop_table', '1') ORDER BY tuple() TTL column_comment + INTERVAL 2 MONTH; @@ -21,7 +21,7 @@ CREATE TABLE r_prop_table2 column_comment Date COMMENT 'Some comment', column_ttl UInt64 TTL column_comment + INTERVAL 1 MONTH ) -ENGINE ReplicatedMergeTree('/clickhouse/test_01493/r_prop_table', '2') +ENGINE ReplicatedMergeTree('/clickhouse/{database}/test_01493/r_prop_table', '2') ORDER BY tuple() TTL column_comment + INTERVAL 2 MONTH; diff --git a/tests/queries/0_stateless/01509_parallel_quorum_insert_no_replicas.sql b/tests/queries/0_stateless/01509_parallel_quorum_insert_no_replicas.sql index 16c4a4df936..ba2b774b718 100644 --- a/tests/queries/0_stateless/01509_parallel_quorum_insert_no_replicas.sql +++ b/tests/queries/0_stateless/01509_parallel_quorum_insert_no_replicas.sql @@ -4,13 +4,13 @@ DROP TABLE IF EXISTS r2 SYNC; CREATE TABLE r1 ( key UInt64, value String ) -ENGINE = ReplicatedMergeTree('/clickhouse/01509_parallel_quorum_insert_no_replicas', '1') +ENGINE = ReplicatedMergeTree('/clickhouse/{database}/01509_parallel_quorum_insert_no_replicas', '1') ORDER BY tuple(); CREATE TABLE r2 ( key UInt64, value String ) -ENGINE = ReplicatedMergeTree('/clickhouse/01509_parallel_quorum_insert_no_replicas', '2') +ENGINE = ReplicatedMergeTree('/clickhouse/{database}/01509_parallel_quorum_insert_no_replicas', '2') ORDER BY tuple(); SET insert_quorum_parallel=1; diff --git a/tests/queries/0_stateless/01512_create_replicate_merge_tree_one_arg.sql b/tests/queries/0_stateless/01512_create_replicate_merge_tree_one_arg.sql index f33c1534a44..2ee73865122 100644 --- a/tests/queries/0_stateless/01512_create_replicate_merge_tree_one_arg.sql +++ b/tests/queries/0_stateless/01512_create_replicate_merge_tree_one_arg.sql @@ -1,3 +1,3 @@ -CREATE TABLE mt (v UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_01497/mt') +CREATE TABLE mt (v UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_01497/mt') ORDER BY tuple() -- { serverError 36 } diff --git a/tests/queries/0_stateless/01513_count_without_select_sequence_consistency_zookeeper.sql b/tests/queries/0_stateless/01513_count_without_select_sequence_consistency_zookeeper.sql index 0eb9f20ad91..925dd042435 100644 --- a/tests/queries/0_stateless/01513_count_without_select_sequence_consistency_zookeeper.sql +++ b/tests/queries/0_stateless/01513_count_without_select_sequence_consistency_zookeeper.sql @@ -4,9 +4,9 @@ DROP TABLE IF EXISTS quorum1 SYNC; DROP TABLE IF EXISTS quorum2 SYNC; DROP TABLE IF EXISTS quorum3 SYNC; -CREATE TABLE quorum1(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/test_01513/sequence_consistency', '1') ORDER BY x PARTITION BY y; -CREATE TABLE quorum2(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/test_01513/sequence_consistency', '2') ORDER BY x PARTITION BY y; -CREATE TABLE quorum3(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/test_01513/sequence_consistency', '3') ORDER BY x PARTITION BY y; +CREATE TABLE quorum1(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_01513/sequence_consistency', '1') ORDER BY x PARTITION BY y; +CREATE TABLE quorum2(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_01513/sequence_consistency', '2') ORDER BY x PARTITION BY y; +CREATE TABLE quorum3(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_01513/sequence_consistency', '3') ORDER BY x PARTITION BY y; INSERT INTO quorum1 VALUES (1, '1990-11-15'); INSERT INTO quorum1 VALUES (2, '1990-11-15'); diff --git a/tests/queries/0_stateless/01530_drop_database_atomic_sync.sql b/tests/queries/0_stateless/01530_drop_database_atomic_sync.sql index d5fb25a9241..07fe00693d1 100644 --- a/tests/queries/0_stateless/01530_drop_database_atomic_sync.sql +++ b/tests/queries/0_stateless/01530_drop_database_atomic_sync.sql @@ -1,33 +1,33 @@ drop database if exists db_01530_atomic sync; create database db_01530_atomic Engine=Atomic; -create table db_01530_atomic.data (key Int) Engine=ReplicatedMergeTree('/clickhouse/tables/db_01530_atomic/data', 'test') order by key; +create table db_01530_atomic.data (key Int) Engine=ReplicatedMergeTree('/clickhouse/tables/{database}/db_01530_atomic/data', 'test') order by key; drop database db_01530_atomic sync; create database db_01530_atomic Engine=Atomic; -create table db_01530_atomic.data (key Int) Engine=ReplicatedMergeTree('/clickhouse/tables/db_01530_atomic/data', 'test') order by key; +create table db_01530_atomic.data (key Int) Engine=ReplicatedMergeTree('/clickhouse/tables/{database}/db_01530_atomic/data', 'test') order by key; drop database db_01530_atomic sync; set database_atomic_wait_for_drop_and_detach_synchronously=1; create database db_01530_atomic Engine=Atomic; -create table db_01530_atomic.data (key Int) Engine=ReplicatedMergeTree('/clickhouse/tables/db_01530_atomic/data', 'test') order by key; +create table db_01530_atomic.data (key Int) Engine=ReplicatedMergeTree('/clickhouse/tables/{database}/db_01530_atomic/data', 'test') order by key; drop database db_01530_atomic; create database db_01530_atomic Engine=Atomic; -create table db_01530_atomic.data (key Int) Engine=ReplicatedMergeTree('/clickhouse/tables/db_01530_atomic/data', 'test') order by key; +create table db_01530_atomic.data (key Int) Engine=ReplicatedMergeTree('/clickhouse/tables/{database}/db_01530_atomic/data', 'test') order by key; drop database db_01530_atomic; set database_atomic_wait_for_drop_and_detach_synchronously=0; create database db_01530_atomic Engine=Atomic; -create table db_01530_atomic.data (key Int) Engine=ReplicatedMergeTree('/clickhouse/tables/db_01530_atomic/data', 'test') order by key; +create table db_01530_atomic.data (key Int) Engine=ReplicatedMergeTree('/clickhouse/tables/{database}/db_01530_atomic/data', 'test') order by key; drop database db_01530_atomic; create database db_01530_atomic Engine=Atomic; -create table db_01530_atomic.data (key Int) Engine=ReplicatedMergeTree('/clickhouse/tables/db_01530_atomic/data', 'test') order by key; -- { serverError 253; } +create table db_01530_atomic.data (key Int) Engine=ReplicatedMergeTree('/clickhouse/tables/{database}/db_01530_atomic/data', 'test') order by key; -- { serverError 253; } set database_atomic_wait_for_drop_and_detach_synchronously=1; diff --git a/tests/queries/0_stateless/01532_execute_merges_on_single_replica.sql b/tests/queries/0_stateless/01532_execute_merges_on_single_replica.sql index 69369321d26..1acae560c93 100644 --- a/tests/queries/0_stateless/01532_execute_merges_on_single_replica.sql +++ b/tests/queries/0_stateless/01532_execute_merges_on_single_replica.sql @@ -1,7 +1,7 @@ DROP TABLE IF EXISTS execute_on_single_replica_r1 NO DELAY; DROP TABLE IF EXISTS execute_on_single_replica_r2 NO DELAY; -/* that test requires fixed zookeeper path */ +/* that test requires fixed zookeeper path, so we cannot use ReplicatedMergeTree({database}) */ CREATE TABLE execute_on_single_replica_r1 (x UInt64) ENGINE=ReplicatedMergeTree('/clickhouse/tables/test_01532/execute_on_single_replica', 'r1') ORDER BY tuple() SETTINGS execute_merges_on_single_replica_time_threshold=10; CREATE TABLE execute_on_single_replica_r2 (x UInt64) ENGINE=ReplicatedMergeTree('/clickhouse/tables/test_01532/execute_on_single_replica', 'r2') ORDER BY tuple() SETTINGS execute_merges_on_single_replica_time_threshold=10; @@ -124,4 +124,4 @@ ORDER BY part_name FORMAT Vertical; DROP TABLE execute_on_single_replica_r1 NO DELAY; -DROP TABLE execute_on_single_replica_r2 NO DELAY; \ No newline at end of file +DROP TABLE execute_on_single_replica_r2 NO DELAY; diff --git a/tests/queries/0_stateless/01581_deduplicate_by_columns_replicated.sql b/tests/queries/0_stateless/01581_deduplicate_by_columns_replicated.sql index 9779dca90a2..cc8c721b6a8 100644 --- a/tests/queries/0_stateless/01581_deduplicate_by_columns_replicated.sql +++ b/tests/queries/0_stateless/01581_deduplicate_by_columns_replicated.sql @@ -11,11 +11,11 @@ SET replication_alter_partitions_sync = 2; -- IRL insert_replica_id were filled from hostname CREATE TABLE IF NOT EXISTS replicated_deduplicate_by_columns_r1 ( id Int32, val UInt32, unique_value UInt64 MATERIALIZED rowNumberInBlock() -) ENGINE=ReplicatedMergeTree('/clickhouse/tables/test_01581/replicated_deduplicate', 'r1') ORDER BY id; +) ENGINE=ReplicatedMergeTree('/clickhouse/tables/{database}/test_01581/replicated_deduplicate', 'r1') ORDER BY id; CREATE TABLE IF NOT EXISTS replicated_deduplicate_by_columns_r2 ( id Int32, val UInt32, unique_value UInt64 MATERIALIZED rowNumberInBlock() -) ENGINE=ReplicatedMergeTree('/clickhouse/tables/test_01581/replicated_deduplicate', 'r2') ORDER BY id; +) ENGINE=ReplicatedMergeTree('/clickhouse/tables/{database}/test_01581/replicated_deduplicate', 'r2') ORDER BY id; -- insert some data, 2 records: (3, 1003), (4, 1004) are duplicated and have difference in unique_value / insert_replica_id diff --git a/tests/queries/0_stateless/01646_system_restart_replicas_smoke.sql b/tests/queries/0_stateless/01646_system_restart_replicas_smoke.sql index cfd70df8dd4..0b7fba5b02e 100644 --- a/tests/queries/0_stateless/01646_system_restart_replicas_smoke.sql +++ b/tests/queries/0_stateless/01646_system_restart_replicas_smoke.sql @@ -1,5 +1,5 @@ DROP TABLE IF EXISTS data_01646; -CREATE TABLE data_01646 (x Date, s String) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_01646/data_01646', 'r') ORDER BY s PARTITION BY x; +CREATE TABLE data_01646 (x Date, s String) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_01646/data_01646', 'r') ORDER BY s PARTITION BY x; SYSTEM RESTART REPLICAS; DESCRIBE TABLE data_01646; DROP TABLE data_01646; diff --git a/tests/queries/0_stateless/01650_drop_part_and_deduplication_zookeeper.sql b/tests/queries/0_stateless/01650_drop_part_and_deduplication_zookeeper.sql index c3e459dfc49..a839883e683 100644 --- a/tests/queries/0_stateless/01650_drop_part_and_deduplication_zookeeper.sql +++ b/tests/queries/0_stateless/01650_drop_part_and_deduplication_zookeeper.sql @@ -5,7 +5,7 @@ CREATE TABLE partitioned_table ( partitioner UInt8, value String ) -ENGINE ReplicatedMergeTree('/clickhouse/01650_drop_part_and_deduplication_partitioned_table', '1') +ENGINE ReplicatedMergeTree('/clickhouse/{database}/01650_drop_part_and_deduplication_partitioned_table', '1') ORDER BY key PARTITION BY partitioner; @@ -16,24 +16,24 @@ INSERT INTO partitioned_table VALUES (11, 1, 'AA'), (22, 2, 'BB'), (33, 3, 'CC') SELECT partition_id, name FROM system.parts WHERE table = 'partitioned_table' AND database = currentDatabase() ORDER BY name; -SELECT substring(name, 1, 2), value FROM system.zookeeper WHERE path='/clickhouse/01650_drop_part_and_deduplication_partitioned_table/blocks/' ORDER BY value; +SELECT substring(name, 1, 2), value FROM system.zookeeper WHERE path='/clickhouse/' || currentDatabase() || '/01650_drop_part_and_deduplication_partitioned_table/blocks/' ORDER BY value; INSERT INTO partitioned_table VALUES (33, 3, 'CC'); -- must be deduplicated SELECT partition_id, name FROM system.parts WHERE table = 'partitioned_table' AND database = currentDatabase() ORDER BY name; -SELECT substring(name, 1, 2), value FROM system.zookeeper WHERE path='/clickhouse/01650_drop_part_and_deduplication_partitioned_table/blocks/' ORDER BY value; +SELECT substring(name, 1, 2), value FROM system.zookeeper WHERE path='/clickhouse/' || currentDatabase() || '/01650_drop_part_and_deduplication_partitioned_table/blocks/' ORDER BY value; ALTER TABLE partitioned_table DROP PART '3_1_1_0'; SELECT partition_id, name FROM system.parts WHERE table = 'partitioned_table' AND database = currentDatabase() ORDER BY name; -SELECT substring(name, 1, 2), value FROM system.zookeeper WHERE path='/clickhouse/01650_drop_part_and_deduplication_partitioned_table/blocks/' ORDER BY value; +SELECT substring(name, 1, 2), value FROM system.zookeeper WHERE path='/clickhouse/' || currentDatabase() || '/01650_drop_part_and_deduplication_partitioned_table/blocks/' ORDER BY value; INSERT INTO partitioned_table VALUES (33, 3, 'CC'); -- mustn't be deduplicated SELECT partition_id, name FROM system.parts WHERE table = 'partitioned_table' AND database = currentDatabase() ORDER BY name; -SELECT substring(name, 1, 2), value FROM system.zookeeper WHERE path='/clickhouse/01650_drop_part_and_deduplication_partitioned_table/blocks/' ORDER BY value; +SELECT substring(name, 1, 2), value FROM system.zookeeper WHERE path='/clickhouse/' || currentDatabase() || '/01650_drop_part_and_deduplication_partitioned_table/blocks/' ORDER BY value; DROP TABLE IF EXISTS partitioned_table; diff --git a/tests/queries/0_stateless/01650_fetch_patition_with_macro_in_zk_path.sql b/tests/queries/0_stateless/01650_fetch_patition_with_macro_in_zk_path.sql index 2c50a7a8516..85b37cee15a 100644 --- a/tests/queries/0_stateless/01650_fetch_patition_with_macro_in_zk_path.sql +++ b/tests/queries/0_stateless/01650_fetch_patition_with_macro_in_zk_path.sql @@ -2,17 +2,17 @@ DROP TABLE IF EXISTS test_01640; DROP TABLE IF EXISTS restore_01640; CREATE TABLE test_01640(i Int64, d Date, s String) -ENGINE = ReplicatedMergeTree('/clickhouse/{shard}/tables/test_01640','{replica}') +ENGINE = ReplicatedMergeTree('/clickhouse/{database}/{shard}/tables/test_01640','{replica}') PARTITION BY toYYYYMM(d) ORDER BY i; insert into test_01640 values (1, '2021-01-01','some'); CREATE TABLE restore_01640(i Int64, d Date, s String) -ENGINE = ReplicatedMergeTree('/clickhouse/{shard}/tables/restore_01640','{replica}') +ENGINE = ReplicatedMergeTree('/clickhouse/{database}/{shard}/tables/restore_01640','{replica}') PARTITION BY toYYYYMM(d) ORDER BY i; -ALTER TABLE restore_01640 FETCH PARTITION tuple(toYYYYMM(toDate('2021-01-01'))) - FROM '/clickhouse/{shard}/tables/test_01640'; +ALTER TABLE restore_01640 FETCH PARTITION tuple(toYYYYMM(toDate('2021-01-01'))) + FROM '/clickhouse/{database}/{shard}/tables/test_01640'; SELECT partition_id FROM system.detached_parts diff --git a/tests/queries/0_stateless/01666_blns.sql b/tests/queries/0_stateless/01666_blns.sql index be9632092bc..19caf45832f 100644 --- a/tests/queries/0_stateless/01666_blns.sql +++ b/tests/queries/0_stateless/01666_blns.sql @@ -554,9 +554,9 @@ SELECT count() FROM test; DROP TABLE IF EXISTS test_r1; DROP TABLE IF EXISTS test_r2; -CREATE TABLE test_r1 AS test ENGINE = ReplicatedMergeTree('/clickhouse/test_01666', 'r1') ORDER BY "\\" SETTINGS min_bytes_for_wide_part = '100G'; +CREATE TABLE test_r1 AS test ENGINE = ReplicatedMergeTree('/clickhouse/{database}/test_01666', 'r1') ORDER BY "\\" SETTINGS min_bytes_for_wide_part = '100G'; INSERT INTO test_r1 SELECT * FROM test; -CREATE TABLE test_r2 AS test ENGINE = ReplicatedMergeTree('/clickhouse/test_01666', 'r2') ORDER BY "\\" SETTINGS min_bytes_for_wide_part = '100G'; +CREATE TABLE test_r2 AS test ENGINE = ReplicatedMergeTree('/clickhouse/{database}/test_01666', 'r2') ORDER BY "\\" SETTINGS min_bytes_for_wide_part = '100G'; SYSTEM SYNC REPLICA test_r2; diff --git a/tests/queries/0_stateless/01669_columns_declaration_serde.sql b/tests/queries/0_stateless/01669_columns_declaration_serde.sql index a6bf1184e9f..b1a85754fce 100644 --- a/tests/queries/0_stateless/01669_columns_declaration_serde.sql +++ b/tests/queries/0_stateless/01669_columns_declaration_serde.sql @@ -22,12 +22,12 @@ DROP TABLE IF EXISTS test_r1; DROP TABLE IF EXISTS test_r2; CREATE TABLE test_r1 (x UInt64, "\\" String DEFAULT '\r\n\t\\' || ' -') ENGINE = ReplicatedMergeTree('/clickhouse/test_01669', 'r1') ORDER BY "\\"; +') ENGINE = ReplicatedMergeTree('/clickhouse/{database}/test_01669', 'r1') ORDER BY "\\"; INSERT INTO test_r1 ("\\") VALUES ('\\'); CREATE TABLE test_r2 (x UInt64, "\\" String DEFAULT '\r\n\t\\' || ' -') ENGINE = ReplicatedMergeTree('/clickhouse/test_01669', 'r2') ORDER BY "\\"; +') ENGINE = ReplicatedMergeTree('/clickhouse/{database}/test_01669', 'r2') ORDER BY "\\"; SYSTEM SYNC REPLICA test_r2; diff --git a/tests/queries/0_stateless/01700_system_zookeeper_path_in.sql b/tests/queries/0_stateless/01700_system_zookeeper_path_in.sql index 02457a956a1..a147952dfe1 100644 --- a/tests/queries/0_stateless/01700_system_zookeeper_path_in.sql +++ b/tests/queries/0_stateless/01700_system_zookeeper_path_in.sql @@ -3,20 +3,20 @@ DROP TABLE IF EXISTS sample_table; CREATE TABLE sample_table ( key UInt64 ) -ENGINE ReplicatedMergeTree('/clickhouse/01700_system_zookeeper_path_in/{shard}', '{replica}') +ENGINE ReplicatedMergeTree('/clickhouse/{database}/01700_system_zookeeper_path_in/{shard}', '{replica}') ORDER BY tuple(); -SELECT name FROM system.zookeeper WHERE path = '/clickhouse/01700_system_zookeeper_path_in/s1' AND name like 'block%' ORDER BY name; -SELECT name FROM system.zookeeper WHERE path = '/clickhouse/01700_system_zookeeper_path_in/s1/replicas' AND name LIKE '%r1%' ORDER BY name; +SELECT name FROM system.zookeeper WHERE path = '/clickhouse/' || currentDatabase() || '/01700_system_zookeeper_path_in/s1' AND name like 'block%' ORDER BY name; +SELECT name FROM system.zookeeper WHERE path = '/clickhouse/' || currentDatabase() || '/01700_system_zookeeper_path_in/s1/replicas' AND name LIKE '%r1%' ORDER BY name; SELECT '========'; -SELECT name FROM system.zookeeper WHERE path IN ('/clickhouse/01700_system_zookeeper_path_in/s1') AND name LIKE 'block%' ORDER BY name; -SELECT name FROM system.zookeeper WHERE path IN ('/clickhouse/01700_system_zookeeper_path_in/s1/replicas') AND name LIKE '%r1%' ORDER BY name; +SELECT name FROM system.zookeeper WHERE path IN ('/clickhouse/' || currentDatabase() || '/01700_system_zookeeper_path_in/s1') AND name LIKE 'block%' ORDER BY name; +SELECT name FROM system.zookeeper WHERE path IN ('/clickhouse/' || currentDatabase() || '/01700_system_zookeeper_path_in/s1/replicas') AND name LIKE '%r1%' ORDER BY name; SELECT '========'; -SELECT name FROM system.zookeeper WHERE path IN ('/clickhouse/01700_system_zookeeper_path_in/s1', - '/clickhouse/01700_system_zookeeper_path_in/s1/replicas') AND name LIKE 'block%' ORDER BY name; +SELECT name FROM system.zookeeper WHERE path IN ('/clickhouse/' || currentDatabase() || '/01700_system_zookeeper_path_in/s1', + '/clickhouse/' || currentDatabase() || '/01700_system_zookeeper_path_in/s1/replicas') AND name LIKE 'block%' ORDER BY name; SELECT '========'; -SELECT name FROM system.zookeeper WHERE path IN (SELECT concat('/clickhouse/01700_system_zookeeper_path_in/s1/', name) - FROM system.zookeeper WHERE (name != 'replicas' AND name NOT LIKE 'leader_election%' AND path = '/clickhouse/01700_system_zookeeper_path_in/s1')) ORDER BY name; +SELECT name FROM system.zookeeper WHERE path IN (SELECT concat('/clickhouse/' || currentDatabase() || '/01700_system_zookeeper_path_in/s1/', name) + FROM system.zookeeper WHERE (name != 'replicas' AND name NOT LIKE 'leader_election%' AND path = '/clickhouse/' || currentDatabase() || '/01700_system_zookeeper_path_in/s1')) ORDER BY name; DROP TABLE IF EXISTS sample_table; diff --git a/tests/queries/0_stateless/01710_projection_fetch.reference b/tests/queries/0_stateless/01710_projection_fetch.reference index abce5410b26..6fc199d6ba2 100644 --- a/tests/queries/0_stateless/01710_projection_fetch.reference +++ b/tests/queries/0_stateless/01710_projection_fetch.reference @@ -10,8 +10,8 @@ 3 3 4 4 0 -CREATE TABLE default.tp_2\n(\n `x` Int32,\n `y` Int32,\n PROJECTION p\n (\n SELECT \n x,\n y\n ORDER BY x\n ),\n PROJECTION pp\n (\n SELECT \n x,\n count()\n GROUP BY x\n )\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/{shard}/01710_projection_fetch_default\', \'2_{replica}\')\nORDER BY y\nSETTINGS min_rows_for_compact_part = 2, min_rows_for_wide_part = 4, min_bytes_for_compact_part = 16, min_bytes_for_wide_part = 32, index_granularity = 8192 +CREATE TABLE default.tp_2\n(\n `x` Int32,\n `y` Int32,\n PROJECTION p\n (\n SELECT \n x,\n y\n ORDER BY x\n ),\n PROJECTION pp\n (\n SELECT \n x,\n count()\n GROUP BY x\n )\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/{shard}/01710_projection_fetch_default\', \'2_{replica}\')\nORDER BY y\nSETTINGS min_rows_for_compact_part = 2, min_rows_for_wide_part = 4, min_bytes_for_compact_part = 16, min_bytes_for_wide_part = 32, index_granularity = 8192 2 -CREATE TABLE default.tp_2\n(\n `x` Int32,\n `y` Int32,\n PROJECTION p\n (\n SELECT \n x,\n y\n ORDER BY x\n ),\n PROJECTION pp\n (\n SELECT \n x,\n count()\n GROUP BY x\n )\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/{shard}/01710_projection_fetch_default\', \'2_{replica}\')\nORDER BY y\nSETTINGS min_rows_for_compact_part = 2, min_rows_for_wide_part = 4, min_bytes_for_compact_part = 16, min_bytes_for_wide_part = 32, index_granularity = 8192 -CREATE TABLE default.tp_2\n(\n `x` Int32,\n `y` Int32,\n PROJECTION p\n (\n SELECT \n x,\n y\n ORDER BY x\n ),\n PROJECTION pp\n (\n SELECT \n x,\n count()\n GROUP BY x\n )\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/{shard}/01710_projection_fetch_default\', \'2_{replica}\')\nORDER BY y\nSETTINGS min_rows_for_compact_part = 2, min_rows_for_wide_part = 4, min_bytes_for_compact_part = 16, min_bytes_for_wide_part = 32, index_granularity = 8192 -CREATE TABLE default.tp_2\n(\n `x` Int32,\n `y` Int32,\n PROJECTION p\n (\n SELECT \n x,\n y\n ORDER BY x\n )\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/{shard}/01710_projection_fetch_default\', \'2_{replica}\')\nORDER BY y\nSETTINGS min_rows_for_compact_part = 2, min_rows_for_wide_part = 4, min_bytes_for_compact_part = 16, min_bytes_for_wide_part = 32, index_granularity = 8192 +CREATE TABLE default.tp_2\n(\n `x` Int32,\n `y` Int32,\n PROJECTION p\n (\n SELECT \n x,\n y\n ORDER BY x\n ),\n PROJECTION pp\n (\n SELECT \n x,\n count()\n GROUP BY x\n )\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/{shard}/01710_projection_fetch_default\', \'2_{replica}\')\nORDER BY y\nSETTINGS min_rows_for_compact_part = 2, min_rows_for_wide_part = 4, min_bytes_for_compact_part = 16, min_bytes_for_wide_part = 32, index_granularity = 8192 +CREATE TABLE default.tp_2\n(\n `x` Int32,\n `y` Int32,\n PROJECTION p\n (\n SELECT \n x,\n y\n ORDER BY x\n ),\n PROJECTION pp\n (\n SELECT \n x,\n count()\n GROUP BY x\n )\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/{shard}/01710_projection_fetch_default\', \'2_{replica}\')\nORDER BY y\nSETTINGS min_rows_for_compact_part = 2, min_rows_for_wide_part = 4, min_bytes_for_compact_part = 16, min_bytes_for_wide_part = 32, index_granularity = 8192 +CREATE TABLE default.tp_2\n(\n `x` Int32,\n `y` Int32,\n PROJECTION p\n (\n SELECT \n x,\n y\n ORDER BY x\n )\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/{shard}/01710_projection_fetch_default\', \'2_{replica}\')\nORDER BY y\nSETTINGS min_rows_for_compact_part = 2, min_rows_for_wide_part = 4, min_bytes_for_compact_part = 16, min_bytes_for_wide_part = 32, index_granularity = 8192 diff --git a/tests/queries/0_stateless/01710_projection_fetch.sql b/tests/queries/0_stateless/01710_projection_fetch.sql index 7e4f6cc1d9a..5781d63788f 100644 --- a/tests/queries/0_stateless/01710_projection_fetch.sql +++ b/tests/queries/0_stateless/01710_projection_fetch.sql @@ -1,9 +1,9 @@ drop table if exists tp_1; drop table if exists tp_2; -create table tp_1 (x Int32, y Int32, projection p (select x, y order by x)) engine = ReplicatedMergeTree('/clickhouse/tables/{shard}/01710_projection_fetch_' || currentDatabase(), '1_{replica}') order by y settings min_rows_for_compact_part = 2, min_rows_for_wide_part = 4, min_bytes_for_compact_part = 16, min_bytes_for_wide_part = 32; +create table tp_1 (x Int32, y Int32, projection p (select x, y order by x)) engine = ReplicatedMergeTree('/clickhouse/tables/{database}/{shard}/01710_projection_fetch_' || currentDatabase(), '1_{replica}') order by y settings min_rows_for_compact_part = 2, min_rows_for_wide_part = 4, min_bytes_for_compact_part = 16, min_bytes_for_wide_part = 32; -create table tp_2 (x Int32, y Int32, projection p (select x, y order by x)) engine = ReplicatedMergeTree('/clickhouse/tables/{shard}/01710_projection_fetch_' || currentDatabase(), '2_{replica}') order by y settings min_rows_for_compact_part = 2, min_rows_for_wide_part = 4, min_bytes_for_compact_part = 16, min_bytes_for_wide_part = 32; +create table tp_2 (x Int32, y Int32, projection p (select x, y order by x)) engine = ReplicatedMergeTree('/clickhouse/tables/{database}/{shard}/01710_projection_fetch_' || currentDatabase(), '2_{replica}') order by y settings min_rows_for_compact_part = 2, min_rows_for_wide_part = 4, min_bytes_for_compact_part = 16, min_bytes_for_wide_part = 32; insert into tp_1 select number, number from numbers(3); diff --git a/tests/queries/0_stateless/01713_table_ttl_old_syntax_zookeeper.sql b/tests/queries/0_stateless/01713_table_ttl_old_syntax_zookeeper.sql index 7d4c83c9d3a..71898bdedcb 100644 --- a/tests/queries/0_stateless/01713_table_ttl_old_syntax_zookeeper.sql +++ b/tests/queries/0_stateless/01713_table_ttl_old_syntax_zookeeper.sql @@ -5,7 +5,7 @@ CREATE TABLE ttl_table date Date, value UInt64 ) -ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_01713_table_ttl', '1', date, date, 8192) +ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_01713_table_ttl', '1', date, date, 8192) TTL date + INTERVAL 2 MONTH; --{ serverError 36 } CREATE TABLE ttl_table @@ -13,7 +13,7 @@ CREATE TABLE ttl_table date Date, value UInt64 ) -ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_01713_table_ttl', '1', date, date, 8192) +ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_01713_table_ttl', '1', date, date, 8192) PARTITION BY date; --{ serverError 42 } CREATE TABLE ttl_table @@ -21,7 +21,7 @@ CREATE TABLE ttl_table date Date, value UInt64 ) -ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_01713_table_ttl', '1', date, date, 8192) +ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_01713_table_ttl', '1', date, date, 8192) ORDER BY value; --{ serverError 42 } SELECT 1; diff --git a/tests/queries/0_stateless/01715_background_checker_blather_zookeeper.sql b/tests/queries/0_stateless/01715_background_checker_blather_zookeeper.sql index 66b53369517..c0335ae3753 100644 --- a/tests/queries/0_stateless/01715_background_checker_blather_zookeeper.sql +++ b/tests/queries/0_stateless/01715_background_checker_blather_zookeeper.sql @@ -2,11 +2,11 @@ DROP TABLE IF EXISTS i20203_1; DROP TABLE IF EXISTS i20203_2; CREATE TABLE i20203_1 (a Int8) -ENGINE = ReplicatedMergeTree('/clickhouse/01715_background_checker_i20203', 'r1') +ENGINE = ReplicatedMergeTree('/clickhouse/{database}/01715_background_checker_i20203', 'r1') ORDER BY tuple(); CREATE TABLE i20203_2 (a Int8) -ENGINE = ReplicatedMergeTree('/clickhouse/01715_background_checker_i20203', 'r2') +ENGINE = ReplicatedMergeTree('/clickhouse/{database}/01715_background_checker_i20203', 'r2') ORDER BY tuple(); DETACH TABLE i20203_2; diff --git a/tests/queries/0_stateless/01747_alter_partition_key_enum_zookeeper.sql b/tests/queries/0_stateless/01747_alter_partition_key_enum_zookeeper.sql index 759c8ba3a0b..033a66a477f 100644 --- a/tests/queries/0_stateless/01747_alter_partition_key_enum_zookeeper.sql +++ b/tests/queries/0_stateless/01747_alter_partition_key_enum_zookeeper.sql @@ -39,7 +39,7 @@ CREATE TABLE replicated_report `branch` String, `generated_time` DateTime ) -ENGINE = ReplicatedMergeTree('/clickhouse/01747_alter_partition_key/t', '1') +ENGINE = ReplicatedMergeTree('/clickhouse/{database}/01747_alter_partition_key/t', '1') PARTITION BY (product, toYYYYMM(generated_time)) ORDER BY (product, machine, branch, generated_time); diff --git a/tests/queries/0_stateless/01761_alter_decimal_zookeeper.reference b/tests/queries/0_stateless/01761_alter_decimal_zookeeper.reference index 5dcc95fd7b7..ea3f608b6c7 100644 --- a/tests/queries/0_stateless/01761_alter_decimal_zookeeper.reference +++ b/tests/queries/0_stateless/01761_alter_decimal_zookeeper.reference @@ -1,9 +1,9 @@ 1 5.00000000 2 6.00000000 -CREATE TABLE default.test_alter_decimal\n(\n `n` UInt64,\n `d` Decimal(18, 8)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/01761_alter_decimal_zookeeper\', \'r1\')\nORDER BY tuple()\nSETTINGS index_granularity = 8192 +CREATE TABLE default.test_alter_decimal\n(\n `n` UInt64,\n `d` Decimal(18, 8)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/default/01761_alter_decimal_zookeeper\', \'r1\')\nORDER BY tuple()\nSETTINGS index_granularity = 8192 1 5.00000000 2 6.00000000 -CREATE TABLE default.test_alter_decimal\n(\n `n` UInt64,\n `d` Decimal(18, 8)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/01761_alter_decimal_zookeeper\', \'r1\')\nORDER BY tuple()\nSETTINGS index_granularity = 8192 +CREATE TABLE default.test_alter_decimal\n(\n `n` UInt64,\n `d` Decimal(18, 8)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/default/01761_alter_decimal_zookeeper\', \'r1\')\nORDER BY tuple()\nSETTINGS index_granularity = 8192 1 5.00000000 2 6.00000000 3 7.00000000 diff --git a/tests/queries/0_stateless/01761_alter_decimal_zookeeper.sql b/tests/queries/0_stateless/01761_alter_decimal_zookeeper.sql index 01766f0d6c2..d1bd3b65182 100644 --- a/tests/queries/0_stateless/01761_alter_decimal_zookeeper.sql +++ b/tests/queries/0_stateless/01761_alter_decimal_zookeeper.sql @@ -2,7 +2,7 @@ DROP TABLE IF EXISTS test_alter_decimal; CREATE TABLE test_alter_decimal (n UInt64, d Decimal(15, 8)) -ENGINE = ReplicatedMergeTree('/clickhouse/01761_alter_decimal_zookeeper', 'r1') +ENGINE = ReplicatedMergeTree('/clickhouse/{database}/01761_alter_decimal_zookeeper', 'r1') ORDER BY tuple(); INSERT INTO test_alter_decimal VALUES (1, toDecimal32(5, 5)); diff --git a/tests/queries/0_stateless/01901_test_attach_partition_from.sql b/tests/queries/0_stateless/01901_test_attach_partition_from.sql index 0ffa4cdecf2..c15edacc6c8 100644 --- a/tests/queries/0_stateless/01901_test_attach_partition_from.sql +++ b/tests/queries/0_stateless/01901_test_attach_partition_from.sql @@ -5,7 +5,7 @@ CREATE TABLE test_alter_attach_01901S (A Int64, D date) ENGINE = MergeTree PARTI INSERT INTO test_alter_attach_01901S VALUES (1, '2020-01-01'); CREATE TABLE test_alter_attach_01901D (A Int64, D date) -Engine=ReplicatedMergeTree('/clickhouse/tables/test_alter_attach_01901D', 'r1') +Engine=ReplicatedMergeTree('/clickhouse/tables/{database}/test_alter_attach_01901D', 'r1') PARTITION BY D ORDER BY A; ALTER TABLE test_alter_attach_01901D ATTACH PARTITION '2020-01-01' FROM test_alter_attach_01901S; From c806534eeed384cd47106ee7b4b5b1b75dda8aad Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 5 Aug 2021 10:28:43 +0300 Subject: [PATCH 156/599] Update parallel skip_list --- tests/queries/skip_list.json | 29 ----------------------------- 1 file changed, 29 deletions(-) diff --git a/tests/queries/skip_list.json b/tests/queries/skip_list.json index d40d9a940d0..5a4c982e13d 100644 --- a/tests/queries/skip_list.json +++ b/tests/queries/skip_list.json @@ -191,29 +191,23 @@ [ /// Pessimistic list of tests which work badly in parallel. /// Probably they need better investigation. - "00062_replicated_merge_tree_alter_zookeeper", "00080_show_tables_and_system_tables", "00101_materialized_views_and_insert_without_explicit_database", "00109_shard_totals_after_having", "00110_external_sort", "00116_storage_set", - "00121_drop_column_zookeeper", "00133_long_shard_memory_tracker_and_exception_safety", "00158_buffer_and_nonexistent_table", "00180_attach_materialized_view", - "00226_zookeeper_deduplication_and_unexpected_parts", - "00236_replicated_drop_on_non_leader_zookeeper", "00305_http_and_readonly", "00311_array_primary_key", "00417_kill_query", "00423_storage_log_single_thread", "00429_long_http_bufferization", "00446_clear_column_in_partition_concurrent_zookeeper", - "00446_clear_column_in_partition_zookeeper", "00463_long_sessions_in_http_interface", "00505_shard_secure", "00508_materialized_view_to", - "00516_deduplication_after_drop_partition_zookeeper", "00534_functions_bad_arguments10", "00552_or_nullable", "00564_versioned_collapsing_merge_tree", @@ -231,7 +225,6 @@ "00626_replace_partition_from_table", "00626_replace_partition_from_table_zookeeper", "00633_materialized_view_and_too_many_parts_zookeeper", - "00643_cast_zookeeper", "00652_mergetree_mutations", "00652_replicated_mutations_zookeeper", "00682_empty_parts_merge", @@ -247,7 +240,6 @@ "00751_default_databasename_for_view", "00753_alter_attach", "00754_alter_modify_column_partitions", - "00754_alter_modify_order_by_replicated_zookeeper", "00763_long_lock_buffer_alter_destination_table", "00800_versatile_storage_join", "00804_test_alter_compression_codecs", @@ -262,22 +254,18 @@ "00857_global_joinsavel_table_alias", "00899_long_attach_memory_limit", "00910_buffer_prewhere", - "00910_zookeeper_custom_compression_codecs_replicated", "00926_adaptive_index_granularity_merge_tree", "00926_adaptive_index_granularity_pk", "00926_adaptive_index_granularity_replacing_merge_tree", - "00926_zookeeper_adaptive_index_granularity_replicated_merge_tree", "00933_alter_ttl", "00933_reserved_word", "00933_test_fix_extra_seek_on_compressed_cache", - "00933_ttl_replicated_zookeeper", "00933_ttl_with_default", "00950_dict_get", "00955_test_final_mark", "00976_ttl_with_old_parts", "00980_merge_alter_settings", "00980_zookeeper_merge_tree_alter_settings", - "00988_constraints_replication_zookeeper", "00989_parallel_parts_loading", "00992_system_parts_race_condition_zookeeper_long", "00993_system_parts_race_condition_drop_zookeeper", @@ -349,7 +337,6 @@ "01125_dict_ddl_cannot_add_column", "01127_month_partitioning_consistency_select", "01130_in_memory_parts_partitons", - "01135_default_and_alter_zookeeper", "01148_zookeeper_path_macros_unfolding", "01150_ddl_guard_rwr", "01153_attach_mv_uuid", @@ -374,11 +361,9 @@ "01254_dict_load_after_detach_attach", "01257_dictionary_mismatch_types", "01259_dictionary_custom_settings_ddl", - "01267_alter_default_key_columns_zookeeper", "01268_dictionary_direct_layout", "01269_alias_type_differs", "01272_suspicious_codecs", - "01277_alter_rename_column_constraint_zookeeper", "01280_ssd_complex_key_dictionary", "01280_ttl_where_group_by", "01281_group_by_limit_memory_tracking", @@ -392,7 +377,6 @@ "01305_replica_create_drop_zookeeper", "01307_multiple_leaders_zookeeper", "01318_long_unsuccessful_mutation_zookeeper", - "01319_manual_write_to_replicas", "01320_create_sync_race_condition_zookeeper", "01338_long_select_and_alter", "01338_long_select_and_alter_zookeeper", @@ -401,7 +385,6 @@ "01357_version_collapsing_attach_detach_zookeeper", "01375_compact_parts_codecs", "01376_GROUP_BY_injective_elimination_dictGet", - "01378_alter_rename_with_ttl_zookeeper", "01383_remote_ambiguous_column_shard", "01388_clear_all_columns", "01391_join_on_dict_crash", @@ -413,7 +396,6 @@ "01415_sticking_mutations", "01417_freeze_partition_verbose", "01417_freeze_partition_verbose_zookeeper", - "01430_modify_sample_by_zookeeper", "01444_create_table_drop_database_race", "01454_storagememory_data_race_challenge", "01455_rank_correlation_spearman", @@ -425,7 +407,6 @@ "01470_show_databases_like", "01471_calculate_ttl_during_merge", "01487_distributed_in_not_default_db", - "01493_alter_remove_properties_zookeeper", "01493_storage_set_persistency", "01494_storage_join_persistency", "01501_cache_dictionary_all_fields", @@ -437,7 +418,6 @@ "01526_complex_key_dict_direct_layout", "01527_clickhouse_local_optimize", "01527_dist_sharding_key_dictGet_reload", - "01530_drop_database_atomic_sync", "01541_max_memory_usage_for_user_long", "01542_dictionary_load_exception_race", "01545_system_errors", // looks at the difference of values in system.errors @@ -453,30 +433,23 @@ "01601_detach_permanently", "01602_show_create_view", "01603_rename_overwrite_bug", - "01666_blns", "01646_system_restart_replicas_smoke", // system restart replicas is a global query "01656_test_query_log_factories_info", "01658_read_file_to_stringcolumn", - "01669_columns_declaration_serde", "01676_dictget_in_default_expression", "01681_cache_dictionary_simple_key", "01682_cache_dictionary_complex_key", "01683_flat_dictionary", "01684_ssd_cache_dictionary_simple_key", "01685_ssd_cache_dictionary_complex_key", - "01700_system_zookeeper_path_in", "01702_system_query_log", // It's ok to execute in parallel with oter tests but not several instances of the same test. "01702_system_query_log", // Runs many global system queries - "01715_background_checker_blather_zookeeper", "01721_engine_file_truncate_on_insert", // It's ok to execute in parallel but not several instances of the same test. "01722_long_brotli_http_compression_json_format", // It is broken in some unimaginable way with the genius error 'cannot write to ofstream'. Not sure how to debug this - "01747_alter_partition_key_enum_zookeeper", "01748_dictionary_table_dot", // creates database "01760_polygon_dictionaries", "01760_system_dictionaries", - "01761_alter_decimal_zookeeper", "01360_materialized_view_with_join_on_query_log", // creates and drops MVs on query_log, which may interrupt flushes. - "01509_parallel_quorum_insert_no_replicas", // It's ok to execute in parallel with oter tests but not several instances of the same test. "attach", "ddl_dictionaries", "dictionary", @@ -508,7 +481,6 @@ "01804_dictionary_decimal256_type", "01850_dist_INSERT_preserve_error", // uses cluster with different static databases shard_0/shard_1 "01821_table_comment", - "01710_projection_fetch", "01824_prefer_global_in_and_join", "01870_modulo_partition_key", "01870_buffer_flush", // creates database @@ -526,7 +498,6 @@ "01902_table_function_merge_db_repr", "01946_test_zstd_decompression_with_escape_sequence_at_the_end_of_buffer", "01946_test_wrong_host_name_access", - "01213_alter_rename_with_default_zookeeper", /// Warning: Removing leftovers from table. "02001_add_default_database_to_system_users" ///create user ] } From 7264c7bf4b17e4875a564d7b25ec8f38f7653bdc Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Thu, 5 Aug 2021 12:37:15 +0300 Subject: [PATCH 157/599] Fixed performance tests --- tests/performance/jit_aggregate_functions_no_key.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/performance/jit_aggregate_functions_no_key.xml b/tests/performance/jit_aggregate_functions_no_key.xml index 0d2577af97c..2d8f390059a 100644 --- a/tests/performance/jit_aggregate_functions_no_key.xml +++ b/tests/performance/jit_aggregate_functions_no_key.xml @@ -105,7 +105,7 @@ if (number % 2 == 0, 1, 0) FROM system.numbers_mt - LIMIT 100000000 + LIMIT 10000000 From e4c5d7e3b12eceeca90419202e4e9b7e31733c19 Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Thu, 1 Jul 2021 20:59:28 +0300 Subject: [PATCH 158/599] Support inserting nested as Array of structs, add some refactoring --- src/Columns/ColumnLowCardinality.h | 1 + src/Core/Settings.h | 3 + src/DataTypes/NestedUtils.cpp | 12 + src/DataTypes/NestedUtils.h | 3 + src/Formats/FormatFactory.cpp | 3 + src/Formats/FormatSettings.h | 7 + .../Formats/Impl/ArrowBlockInputFormat.cpp | 14 +- .../Formats/Impl/ArrowBlockInputFormat.h | 5 +- .../Formats/Impl/ArrowColumnToCHColumn.cpp | 1009 +++++++---------- .../Formats/Impl/ArrowColumnToCHColumn.h | 12 +- .../Formats/Impl/CHColumnToArrowColumn.cpp | 1 + .../Formats/Impl/ORCBlockInputFormat.cpp | 17 +- .../Formats/Impl/ORCBlockInputFormat.h | 5 +- .../Formats/Impl/ParquetBlockInputFormat.cpp | 18 +- .../Formats/Impl/ParquetBlockInputFormat.h | 4 +- .../00900_orc_arrow_parquet_nested.reference | 21 + .../00900_orc_arrow_parquet_nested.sh | 36 + .../01273_arrow_dictionaries_load.reference | 3 + .../01273_arrow_dictionaries_load.sh | 2 + .../nested_nested_arrow.cpp | 178 +++ .../nested_nested_table.arrow | Bin 0 -> 2410 bytes .../nested_nested_table.orc | Bin 0 -> 1050 bytes .../nested_nested_table.parquet | Bin 0 -> 2287 bytes .../nested_table.arrow | Bin 0 -> 1218 bytes .../nested_table.orc | Bin 0 -> 620 bytes .../nested_table.parquet | Bin 0 -> 1105 bytes tests/queries/0_stateless/dicts.arrow | Bin 0 -> 20030554 bytes tests/queries/0_stateless/maps | 0 28 files changed, 759 insertions(+), 595 deletions(-) create mode 100644 tests/queries/0_stateless/00900_orc_arrow_parquet_nested.reference create mode 100755 tests/queries/0_stateless/00900_orc_arrow_parquet_nested.sh create mode 100644 tests/queries/0_stateless/data_orc_arrow_parquet_nested/nested_nested_arrow.cpp create mode 100644 tests/queries/0_stateless/data_orc_arrow_parquet_nested/nested_nested_table.arrow create mode 100644 tests/queries/0_stateless/data_orc_arrow_parquet_nested/nested_nested_table.orc create mode 100644 tests/queries/0_stateless/data_orc_arrow_parquet_nested/nested_nested_table.parquet create mode 100644 tests/queries/0_stateless/data_orc_arrow_parquet_nested/nested_table.arrow create mode 100644 tests/queries/0_stateless/data_orc_arrow_parquet_nested/nested_table.orc create mode 100644 tests/queries/0_stateless/data_orc_arrow_parquet_nested/nested_table.parquet create mode 100644 tests/queries/0_stateless/dicts.arrow create mode 100644 tests/queries/0_stateless/maps diff --git a/src/Columns/ColumnLowCardinality.h b/src/Columns/ColumnLowCardinality.h index 698f65b1281..03ef388b257 100644 --- a/src/Columns/ColumnLowCardinality.h +++ b/src/Columns/ColumnLowCardinality.h @@ -193,6 +193,7 @@ public: const IColumnUnique & getDictionary() const { return dictionary.getColumnUnique(); } IColumnUnique & getDictionary() { return dictionary.getColumnUnique(); } const ColumnPtr & getDictionaryPtr() const { return dictionary.getColumnUniquePtr(); } + ColumnPtr & getDictionaryPtr() { return dictionary.getColumnUniquePtr(); } /// IColumnUnique & getUnique() { return static_cast(*column_unique); } /// ColumnPtr getUniquePtr() const { return column_unique; } diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 64c4c5621b2..3000dc4f195 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -522,6 +522,9 @@ class IColumn; M(Bool, input_format_tsv_empty_as_default, false, "Treat empty fields in TSV input as default values.", 0) \ M(Bool, input_format_tsv_enum_as_number, false, "Treat inserted enum values in TSV formats as enum indices \\N", 0) \ M(Bool, input_format_null_as_default, true, "For text input formats initialize null fields with default values if data type of this field is not nullable", 0) \ + M(Bool, input_format_arrow_import_nested, false, "Allow to insert array of structs into Nested table in Arrow input format.", 0) \ + M(Bool, input_format_orc_import_nested, false, "Allow to insert array of structs into Nested table in ORC input format.", 0) \ + M(Bool, input_format_parquet_import_nested, false, "Allow to insert array of structs into Nested table in Parquet input format.", 0) \ \ M(DateTimeInputFormat, date_time_input_format, FormatSettings::DateTimeInputFormat::Basic, "Method to read DateTime from text input formats. Possible values: 'basic' and 'best_effort'.", 0) \ M(DateTimeOutputFormat, date_time_output_format, FormatSettings::DateTimeOutputFormat::Simple, "Method to write DateTime to text output. Possible values: 'simple', 'iso', 'unix_timestamp'.", 0) \ diff --git a/src/DataTypes/NestedUtils.cpp b/src/DataTypes/NestedUtils.cpp index ed9ea3e1b5c..94b3b2f3cf7 100644 --- a/src/DataTypes/NestedUtils.cpp +++ b/src/DataTypes/NestedUtils.cpp @@ -208,6 +208,18 @@ void validateArraySizes(const Block & block) } } +std::unordered_set getAllTableNames(const Block & block) +{ + std::unordered_set nested_table_names; + for (auto & name : block.getNames()) + { + auto nested_table_name = Nested::extractTableName(name); + if (!nested_table_name.empty()) + nested_table_names.insert(nested_table_name); + } + return nested_table_names; +} + } } diff --git a/src/DataTypes/NestedUtils.h b/src/DataTypes/NestedUtils.h index b8428b96d3e..d16e309fc81 100644 --- a/src/DataTypes/NestedUtils.h +++ b/src/DataTypes/NestedUtils.h @@ -28,6 +28,9 @@ namespace Nested /// Check that sizes of arrays - elements of nested data structures - are equal. void validateArraySizes(const Block & block); + + /// Get all nested tables names from a block. + std::unordered_set getAllTableNames(const Block & block); } } diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index d2d6d92dea3..c58a6795832 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -88,6 +88,7 @@ FormatSettings getFormatSettings(ContextPtr context, const Settings & settings) format_settings.json.quote_denormals = settings.output_format_json_quote_denormals; format_settings.null_as_default = settings.input_format_null_as_default; format_settings.parquet.row_group_size = settings.output_format_parquet_row_group_size; + format_settings.parquet.import_nested = settings.input_format_parquet_import_nested; format_settings.pretty.charset = settings.output_format_pretty_grid_charset.toString() == "ASCII" ? FormatSettings::Pretty::Charset::ASCII : FormatSettings::Pretty::Charset::UTF8; format_settings.pretty.color = settings.output_format_pretty_color; format_settings.pretty.max_column_pad_width = settings.output_format_pretty_max_column_pad_width; @@ -114,6 +115,8 @@ FormatSettings getFormatSettings(ContextPtr context, const Settings & settings) format_settings.with_names_use_header = settings.input_format_with_names_use_header; format_settings.write_statistics = settings.output_format_write_statistics; format_settings.arrow.low_cardinality_as_dictionary = settings.output_format_arrow_low_cardinality_as_dictionary; + format_settings.arrow.import_nested = settings.input_format_arrow_import_nested; + format_settings.orc.import_nested = settings.input_format_orc_import_nested; /// Validate avro_schema_registry_url with RemoteHostFilter when non-empty and in Server context if (format_settings.schema.is_server) diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index 69df095bca8..d77a7c95d69 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -53,6 +53,7 @@ struct FormatSettings { UInt64 row_group_size = 1000000; bool low_cardinality_as_dictionary = false; + bool import_nested = false; } arrow; struct @@ -100,6 +101,7 @@ struct FormatSettings struct { UInt64 row_group_size = 1000000; + bool import_nested = false; } parquet; struct Pretty @@ -174,6 +176,11 @@ struct FormatSettings bool deduce_templates_of_expressions = true; bool accurate_types_of_literals = true; } values; + + struct + { + bool import_nested = false; + } orc; }; } diff --git a/src/Processors/Formats/Impl/ArrowBlockInputFormat.cpp b/src/Processors/Formats/Impl/ArrowBlockInputFormat.cpp index 269faac5258..84ca789261f 100644 --- a/src/Processors/Formats/Impl/ArrowBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ArrowBlockInputFormat.cpp @@ -22,8 +22,8 @@ namespace ErrorCodes extern const int CANNOT_READ_ALL_DATA; } -ArrowBlockInputFormat::ArrowBlockInputFormat(ReadBuffer & in_, const Block & header_, bool stream_) - : IInputFormat(header_, in_), stream{stream_} +ArrowBlockInputFormat::ArrowBlockInputFormat(ReadBuffer & in_, const Block & header_, bool stream_, const FormatSettings & format_settings_) + : IInputFormat(header_, in_), stream{stream_}, format_settings(format_settings_) { } @@ -102,7 +102,7 @@ void ArrowBlockInputFormat::prepareReader() schema = file_reader->schema(); } - arrow_column_to_ch_column = std::make_unique(getPort().getHeader(), std::move(schema), "Arrow"); + arrow_column_to_ch_column = std::make_unique(getPort().getHeader(), "Arrow", format_settings.arrow.import_nested); if (stream) record_batch_total = -1; @@ -119,9 +119,9 @@ void registerInputFormatProcessorArrow(FormatFactory & factory) [](ReadBuffer & buf, const Block & sample, const RowInputFormatParams & /* params */, - const FormatSettings & /* format_settings */) + const FormatSettings & format_settings) { - return std::make_shared(buf, sample, false); + return std::make_shared(buf, sample, false, format_settings); }); factory.markFormatAsColumnOriented("Arrow"); factory.registerInputFormatProcessor( @@ -129,9 +129,9 @@ void registerInputFormatProcessorArrow(FormatFactory & factory) [](ReadBuffer & buf, const Block & sample, const RowInputFormatParams & /* params */, - const FormatSettings & /* format_settings */) + const FormatSettings & format_settings) { - return std::make_shared(buf, sample, true); + return std::make_shared(buf, sample, true, format_settings); }); } diff --git a/src/Processors/Formats/Impl/ArrowBlockInputFormat.h b/src/Processors/Formats/Impl/ArrowBlockInputFormat.h index 9f458dece7f..705c47c9b17 100644 --- a/src/Processors/Formats/Impl/ArrowBlockInputFormat.h +++ b/src/Processors/Formats/Impl/ArrowBlockInputFormat.h @@ -6,6 +6,7 @@ #if USE_ARROW #include +#include namespace arrow { class RecordBatchReader; } namespace arrow::ipc { class RecordBatchFileReader; } @@ -19,7 +20,7 @@ class ArrowColumnToCHColumn; class ArrowBlockInputFormat : public IInputFormat { public: - ArrowBlockInputFormat(ReadBuffer & in_, const Block & header_, bool stream_); + ArrowBlockInputFormat(ReadBuffer & in_, const Block & header_, bool stream_, const FormatSettings & format_settings_); void resetParser() override; @@ -41,6 +42,8 @@ private: int record_batch_total = 0; int record_batch_current = 0; + const FormatSettings format_settings; + void prepareReader(); }; diff --git a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp index 84c56f0f2b7..93b49f332fc 100644 --- a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp +++ b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp @@ -1,7 +1,7 @@ +#include "config_formats.h" #include "ArrowColumnToCHColumn.h" #if USE_ARROW || USE_ORC || USE_PARQUET - #include #include #include @@ -11,10 +11,10 @@ #include #include #include +#include #include #include #include -#include #include #include #include @@ -22,664 +22,533 @@ #include #include #include +#include #include #include +#include +#include #include -#define FOR_ARROW_NUMERIC_TYPES(M) \ - M(arrow::Type::UINT8, DB::UInt8) \ - M(arrow::Type::INT8, DB::Int8) \ - M(arrow::Type::UINT16, DB::UInt16) \ - M(arrow::Type::INT16, DB::Int16) \ - M(arrow::Type::UINT32, DB::UInt32) \ - M(arrow::Type::INT32, DB::Int32) \ - M(arrow::Type::UINT64, DB::UInt64) \ - M(arrow::Type::INT64, DB::Int64) \ - M(arrow::Type::HALF_FLOAT, DB::Float32) \ - M(arrow::Type::FLOAT, DB::Float32) \ - M(arrow::Type::DOUBLE, DB::Float64) - -#define FOR_ARROW_INDEXES_TYPES(M) \ - M(arrow::Type::UINT8, DB::UInt8) \ - M(arrow::Type::INT8, DB::UInt8) \ - M(arrow::Type::UINT16, DB::UInt16) \ - M(arrow::Type::INT16, DB::UInt16) \ - M(arrow::Type::UINT32, DB::UInt32) \ - M(arrow::Type::INT32, DB::UInt32) \ - M(arrow::Type::UINT64, DB::UInt64) \ - M(arrow::Type::INT64, DB::UInt64) - - namespace DB { - -namespace ErrorCodes -{ - extern const int UNKNOWN_TYPE; - extern const int VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE; - extern const int CANNOT_CONVERT_TYPE; - extern const int CANNOT_INSERT_NULL_IN_ORDINARY_COLUMN; - extern const int THERE_IS_NO_COLUMN; - extern const int BAD_ARGUMENTS; -} - -static const std::initializer_list> arrow_type_to_internal_type = -{ - {arrow::Type::UINT8, "UInt8"}, - {arrow::Type::INT8, "Int8"}, - {arrow::Type::UINT16, "UInt16"}, - {arrow::Type::INT16, "Int16"}, - {arrow::Type::UINT32, "UInt32"}, - {arrow::Type::INT32, "Int32"}, - {arrow::Type::UINT64, "UInt64"}, - {arrow::Type::INT64, "Int64"}, - {arrow::Type::HALF_FLOAT, "Float32"}, - {arrow::Type::FLOAT, "Float32"}, - {arrow::Type::DOUBLE, "Float64"}, - - {arrow::Type::BOOL, "UInt8"}, - {arrow::Type::DATE32, "Date"}, - {arrow::Type::DATE32, "Date32"}, - {arrow::Type::DATE64, "DateTime"}, - {arrow::Type::TIMESTAMP, "DateTime"}, - - {arrow::Type::STRING, "String"}, - {arrow::Type::BINARY, "String"}, - - // TODO: add other types that are convertible to internal ones: - // 0. ENUM? - // 1. UUID -> String - // 2. JSON -> String - // Full list of types: contrib/arrow/cpp/src/arrow/type.h -}; - -/// Inserts numeric data right into internal column data to reduce an overhead -template > -static void fillColumnWithNumericData(std::shared_ptr & arrow_column, IColumn & internal_column) -{ - auto & column_data = static_cast(internal_column).getData(); - column_data.reserve(arrow_column->length()); - - for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i) + namespace ErrorCodes { - std::shared_ptr chunk = arrow_column->chunk(chunk_i); - /// buffers[0] is a null bitmap and buffers[1] are actual values - std::shared_ptr buffer = chunk->data()->buffers[1]; - - const auto * raw_data = reinterpret_cast(buffer->data()); - column_data.insert_assume_reserved(raw_data, raw_data + chunk->length()); + extern const int UNKNOWN_TYPE; + extern const int VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE; + extern const int CANNOT_CONVERT_TYPE; + extern const int CANNOT_INSERT_NULL_IN_ORDINARY_COLUMN; + extern const int THERE_IS_NO_COLUMN; + extern const int BAD_ARGUMENTS; + extern const int UNKNOWN_EXCEPTION; } -} -/// Inserts chars and offsets right into internal column data to reduce an overhead. -/// Internal offsets are shifted by one to the right in comparison with Arrow ones. So the last offset should map to the end of all chars. -/// Also internal strings are null terminated. -static void fillColumnWithStringData(std::shared_ptr & arrow_column, IColumn & internal_column) -{ - PaddedPODArray & column_chars_t = assert_cast(internal_column).getChars(); - PaddedPODArray & column_offsets = assert_cast(internal_column).getOffsets(); - - size_t chars_t_size = 0; - for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i) + static void checkStatus(const arrow::Status & status, const String & column_name, const String & format_name) { - arrow::BinaryArray & chunk = dynamic_cast(*(arrow_column->chunk(chunk_i))); - const size_t chunk_length = chunk.length(); + if (!status.ok()) + throw Exception{fmt::format("Error with a {} column \"{}\": {}.", format_name, column_name, status.ToString()), ErrorCodes::UNKNOWN_EXCEPTION}; + } - if (chunk_length > 0) + /// Inserts numeric data right into internal column data to reduce an overhead + template > + static ColumnWithTypeAndName readColumnWithNumericData(std::shared_ptr & arrow_column, const String & column_name) + { + auto internal_type = std::make_shared>(); + auto internal_column = internal_type->createColumn(); + auto & column_data = static_cast(*internal_column).getData(); + column_data.reserve(arrow_column->length()); + + for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i) { - chars_t_size += chunk.value_offset(chunk_length - 1) + chunk.value_length(chunk_length - 1); - chars_t_size += chunk_length; /// additional space for null bytes + std::shared_ptr chunk = arrow_column->chunk(chunk_i); + /// buffers[0] is a null bitmap and buffers[1] are actual values + std::shared_ptr buffer = chunk->data()->buffers[1]; + + const auto * raw_data = reinterpret_cast(buffer->data()); + column_data.insert_assume_reserved(raw_data, raw_data + chunk->length()); } + return {std::move(internal_column), std::move(internal_type), column_name}; } - column_chars_t.reserve(chars_t_size); - column_offsets.reserve(arrow_column->length()); - - for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i) + /// Inserts chars and offsets right into internal column data to reduce an overhead. + /// Internal offsets are shifted by one to the right in comparison with Arrow ones. So the last offset should map to the end of all chars. + /// Also internal strings are null terminated. + static ColumnWithTypeAndName readColumnWithStringData(std::shared_ptr & arrow_column, const String & column_name) { - arrow::BinaryArray & chunk = dynamic_cast(*(arrow_column->chunk(chunk_i))); - std::shared_ptr buffer = chunk.value_data(); - const size_t chunk_length = chunk.length(); + auto internal_type = std::make_shared(); + auto internal_column = internal_type->createColumn(); + PaddedPODArray & column_chars_t = assert_cast(*internal_column).getChars(); + PaddedPODArray & column_offsets = assert_cast(*internal_column).getOffsets(); - for (size_t offset_i = 0; offset_i != chunk_length; ++offset_i) + size_t chars_t_size = 0; + for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i) { - if (!chunk.IsNull(offset_i) && buffer) + arrow::BinaryArray & chunk = static_cast(*(arrow_column->chunk(chunk_i))); + const size_t chunk_length = chunk.length(); + + if (chunk_length > 0) { - const auto * raw_data = buffer->data() + chunk.value_offset(offset_i); - column_chars_t.insert_assume_reserved(raw_data, raw_data + chunk.value_length(offset_i)); + chars_t_size += chunk.value_offset(chunk_length - 1) + chunk.value_length(chunk_length - 1); + chars_t_size += chunk_length; /// additional space for null bytes } - column_chars_t.emplace_back('\0'); - - column_offsets.emplace_back(column_chars_t.size()); } - } -} -static void fillColumnWithBooleanData(std::shared_ptr & arrow_column, IColumn & internal_column) -{ - auto & column_data = assert_cast &>(internal_column).getData(); - column_data.reserve(arrow_column->length()); + column_chars_t.reserve(chars_t_size); + column_offsets.reserve(arrow_column->length()); - for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i) - { - arrow::BooleanArray & chunk = dynamic_cast(*(arrow_column->chunk(chunk_i))); - /// buffers[0] is a null bitmap and buffers[1] are actual values - std::shared_ptr buffer = chunk.data()->buffers[1]; - - for (size_t bool_i = 0; bool_i != static_cast(chunk.length()); ++bool_i) - column_data.emplace_back(chunk.Value(bool_i)); - } -} - -/// Arrow stores Parquet::DATE in Int32, while ClickHouse stores Date in UInt16. Therefore, it should be checked before saving -static void fillColumnWithDate32Data(std::shared_ptr & arrow_column, IColumn & internal_column) -{ - PaddedPODArray & column_data = assert_cast &>(internal_column).getData(); - column_data.reserve(arrow_column->length()); - - for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i) - { - arrow::Date32Array & chunk = dynamic_cast(*(arrow_column->chunk(chunk_i))); - - for (size_t value_i = 0, length = static_cast(chunk.length()); value_i < length; ++value_i) + for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i) { - UInt32 days_num = static_cast(chunk.Value(value_i)); + arrow::BinaryArray & chunk = static_cast(*(arrow_column->chunk(chunk_i))); + std::shared_ptr buffer = chunk.value_data(); + const size_t chunk_length = chunk.length(); - if (days_num > DATE_LUT_MAX_DAY_NUM) - throw Exception(ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE, - "Input value {} of a column '{}' is greater than max allowed Date value, which is {}", - days_num, internal_column.getName(), DATE_LUT_MAX_DAY_NUM); + for (size_t offset_i = 0; offset_i != chunk_length; ++offset_i) + { + if (!chunk.IsNull(offset_i) && buffer) + { + const auto * raw_data = buffer->data() + chunk.value_offset(offset_i); + column_chars_t.insert_assume_reserved(raw_data, raw_data + chunk.value_length(offset_i)); + } + column_chars_t.emplace_back('\0'); - column_data.emplace_back(days_num); + column_offsets.emplace_back(column_chars_t.size()); + } } + return {std::move(internal_column), std::move(internal_type), column_name}; } -} -static void fillDate32ColumnWithDate32Data(std::shared_ptr & arrow_column, IColumn & internal_column) -{ - PaddedPODArray & column_data = assert_cast &>(internal_column).getData(); - column_data.reserve(arrow_column->length()); - - for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i) + static ColumnWithTypeAndName readColumnWithBooleanData(std::shared_ptr & arrow_column, const String & column_name) { - arrow::Date32Array & chunk = dynamic_cast(*(arrow_column->chunk(chunk_i))); + auto internal_type = std::make_shared(); + auto internal_column = internal_type->createColumn(); + auto & column_data = assert_cast &>(*internal_column).getData(); + column_data.reserve(arrow_column->length()); - for (size_t value_i = 0, length = static_cast(chunk.length()); value_i < length; ++value_i) + for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i) { - Int32 days_num = static_cast(chunk.Value(value_i)); - if (days_num > DATE_LUT_MAX_EXTEND_DAY_NUM) - throw Exception(ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE, - "Input value {} of a column '{}' is greater than max allowed Date value, which is {}", days_num, internal_column.getName(), DATE_LUT_MAX_DAY_NUM); + arrow::BooleanArray & chunk = static_cast(*(arrow_column->chunk(chunk_i))); + /// buffers[0] is a null bitmap and buffers[1] are actual values + std::shared_ptr buffer = chunk.data()->buffers[1]; - column_data.emplace_back(days_num); + for (size_t bool_i = 0; bool_i != static_cast(chunk.length()); ++bool_i) + column_data.emplace_back(chunk.Value(bool_i)); } + return {std::move(internal_column), std::move(internal_type), column_name}; } -} -/// Arrow stores Parquet::DATETIME in Int64, while ClickHouse stores DateTime in UInt32. Therefore, it should be checked before saving -static void fillColumnWithDate64Data(std::shared_ptr & arrow_column, IColumn & internal_column) -{ - auto & column_data = assert_cast &>(internal_column).getData(); - column_data.reserve(arrow_column->length()); - - for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i) + static ColumnWithTypeAndName readColumnWithDate32Data(std::shared_ptr & arrow_column, const String & column_name) { - auto & chunk = dynamic_cast(*(arrow_column->chunk(chunk_i))); - for (size_t value_i = 0, length = static_cast(chunk.length()); value_i < length; ++value_i) + auto internal_type = std::make_shared(); + auto internal_column = internal_type->createColumn(); + PaddedPODArray & column_data = assert_cast &>(*internal_column).getData(); + column_data.reserve(arrow_column->length()); + + for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i) { - auto timestamp = static_cast(chunk.Value(value_i) / 1000); // Always? in ms - column_data.emplace_back(timestamp); + arrow::Date32Array & chunk = static_cast(*(arrow_column->chunk(chunk_i))); + + for (size_t value_i = 0, length = static_cast(chunk.length()); value_i < length; ++value_i) + { + Int32 days_num = static_cast(chunk.Value(value_i)); + if (days_num > DATE_LUT_MAX_EXTEND_DAY_NUM) + { + // TODO: will it rollback correctly? + throw Exception + { + fmt::format("Input value {} of a column \"{}\" is greater than max allowed Date value, which is {}", days_num, column_name, DATE_LUT_MAX_DAY_NUM), + ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE + }; + } + + column_data.emplace_back(days_num); + } } + return {std::move(internal_column), std::move(internal_type), column_name}; } -} -static void fillColumnWithTimestampData(std::shared_ptr & arrow_column, IColumn & internal_column) -{ - auto & column_data = assert_cast &>(internal_column).getData(); - column_data.reserve(arrow_column->length()); - - for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i) + /// Arrow stores Parquet::DATETIME in Int64, while ClickHouse stores DateTime in UInt32. Therefore, it should be checked before saving + static ColumnWithTypeAndName readColumnWithDate64Data(std::shared_ptr & arrow_column, const String & column_name) { - auto & chunk = dynamic_cast(*(arrow_column->chunk(chunk_i))); - const auto & type = static_cast(*chunk.type()); + auto internal_type = std::make_shared(); + auto internal_column = internal_type->createColumn(); + auto & column_data = assert_cast &>(*internal_column).getData(); + column_data.reserve(arrow_column->length()); - UInt32 divide = 1; - const auto unit = type.unit(); - switch (unit) + for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i) { - case arrow::TimeUnit::SECOND: - divide = 1; - break; - case arrow::TimeUnit::MILLI: - divide = 1000; - break; - case arrow::TimeUnit::MICRO: - divide = 1000000; - break; - case arrow::TimeUnit::NANO: - divide = 1000000000; - break; + auto & chunk = static_cast(*(arrow_column->chunk(chunk_i))); + for (size_t value_i = 0, length = static_cast(chunk.length()); value_i < length; ++value_i) + { + auto timestamp = static_cast(chunk.Value(value_i) / 1000); // Always? in ms + column_data.emplace_back(timestamp); + } } + return {std::move(internal_column), std::move(internal_type), column_name}; + } - for (size_t value_i = 0, length = static_cast(chunk.length()); value_i < length; ++value_i) + static ColumnWithTypeAndName readColumnWithTimestampData(std::shared_ptr & arrow_column, const String & column_name) + { + auto internal_type = std::make_shared(); + auto internal_column = internal_type->createColumn(); + auto & column_data = assert_cast &>(*internal_column).getData(); + column_data.reserve(arrow_column->length()); + + for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i) { - auto timestamp = static_cast(chunk.Value(value_i) / divide); // ms! TODO: check other 's' 'ns' ... - column_data.emplace_back(timestamp); + auto & chunk = static_cast(*(arrow_column->chunk(chunk_i))); + const auto & type = static_cast(*chunk.type()); + + UInt32 divide = 1; + const auto unit = type.unit(); + switch (unit) + { + case arrow::TimeUnit::SECOND: + divide = 1; + break; + case arrow::TimeUnit::MILLI: + divide = 1000; + break; + case arrow::TimeUnit::MICRO: + divide = 1000000; + break; + case arrow::TimeUnit::NANO: + divide = 1000000000; + break; + } + + for (size_t value_i = 0, length = static_cast(chunk.length()); value_i < length; ++value_i) + { + auto timestamp = static_cast(chunk.Value(value_i) / divide); // ms! TODO: check other 's' 'ns' ... + column_data.emplace_back(timestamp); + } } + return {std::move(internal_column), std::move(internal_type), column_name}; } -} -template -static void fillColumnWithDecimalData(std::shared_ptr & arrow_column, IColumn & internal_column) -{ - auto & column = assert_cast &>(internal_column); - auto & column_data = column.getData(); - column_data.reserve(arrow_column->length()); - - for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i) + template + static ColumnWithTypeAndName readColumnWithDecimalData(std::shared_ptr & arrow_column, const String & column_name) { - auto & chunk = static_cast(*(arrow_column->chunk(chunk_i))); - for (size_t value_i = 0, length = static_cast(chunk.length()); value_i < length; ++value_i) + const auto * arrow_decimal_type = static_cast(arrow_column->type().get()); + auto internal_type = std::make_shared>(arrow_decimal_type->precision(), arrow_decimal_type->scale()); + auto internal_column = internal_type->createColumn(); + auto & column = assert_cast &>(*internal_column); + auto & column_data = column.getData(); + column_data.reserve(arrow_column->length()); + + for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i) { - column_data.emplace_back(chunk.IsNull(value_i) ? DecimalType(0) : *reinterpret_cast(chunk.Value(value_i))); // TODO: copy column + auto & chunk = static_cast(*(arrow_column->chunk(chunk_i))); + for (size_t value_i = 0, length = static_cast(chunk.length()); value_i < length; ++value_i) + { + column_data.emplace_back(chunk.IsNull(value_i) ? DecimalType(0) : *reinterpret_cast(chunk.Value(value_i))); // TODO: copy column + } } + return {std::move(internal_column), std::move(internal_type), column_name}; } -} -/// Creates a null bytemap from arrow's null bitmap -static void fillByteMapFromArrowColumn(std::shared_ptr & arrow_column, IColumn & bytemap) -{ - PaddedPODArray & bytemap_data = assert_cast &>(bytemap).getData(); - bytemap_data.reserve(arrow_column->length()); - - for (size_t chunk_i = 0; chunk_i != static_cast(arrow_column->num_chunks()); ++chunk_i) + /// Creates a null bytemap from arrow's null bitmap + static ColumnPtr readByteMapFromArrowColumn(std::shared_ptr & arrow_column) { - std::shared_ptr chunk = arrow_column->chunk(chunk_i); + auto nullmap_column = ColumnUInt8::create(); + PaddedPODArray & bytemap_data = assert_cast &>(*nullmap_column).getData(); + bytemap_data.reserve(arrow_column->length()); - for (size_t value_i = 0; value_i != static_cast(chunk->length()); ++value_i) - bytemap_data.emplace_back(chunk->IsNull(value_i)); + for (size_t chunk_i = 0; chunk_i != static_cast(arrow_column->num_chunks()); ++chunk_i) + { + std::shared_ptr chunk = arrow_column->chunk(chunk_i); + + for (size_t value_i = 0; value_i != static_cast(chunk->length()); ++value_i) + bytemap_data.emplace_back(chunk->IsNull(value_i)); + } + return nullmap_column; } -} -static void fillOffsetsFromArrowListColumn(std::shared_ptr & arrow_column, IColumn & offsets) -{ - ColumnArray::Offsets & offsets_data = assert_cast &>(offsets).getData(); - offsets_data.reserve(arrow_column->length()); - - for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i) + static ColumnPtr readOffsetsFromArrowListColumn(std::shared_ptr & arrow_column) { - arrow::ListArray & list_chunk = dynamic_cast(*(arrow_column->chunk(chunk_i))); - auto arrow_offsets_array = list_chunk.offsets(); - auto & arrow_offsets = dynamic_cast(*arrow_offsets_array); - auto start = offsets_data.back(); - for (int64_t i = 1; i < arrow_offsets.length(); ++i) - offsets_data.emplace_back(start + arrow_offsets.Value(i)); + auto offsets_column = ColumnUInt64::create(); + ColumnArray::Offsets & offsets_data = assert_cast &>(*offsets_column).getData(); + offsets_data.reserve(arrow_column->length()); + + for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i) + { + arrow::ListArray & list_chunk = static_cast(*(arrow_column->chunk(chunk_i))); + auto arrow_offsets_array = list_chunk.offsets(); + auto & arrow_offsets = static_cast(*arrow_offsets_array); + auto start = offsets_data.back(); + for (int64_t i = 1; i < arrow_offsets.length(); ++i) + offsets_data.emplace_back(start + arrow_offsets.Value(i)); + } + return offsets_column; } -} -static ColumnPtr createAndFillColumnWithIndexesData(std::shared_ptr & arrow_column) -{ - switch (arrow_column->type()->id()) + + static ColumnPtr readColumnWithIndexesData(std::shared_ptr & arrow_column) { + switch (arrow_column->type()->id()) + { # define DISPATCH(ARROW_NUMERIC_TYPE, CPP_NUMERIC_TYPE) \ - case ARROW_NUMERIC_TYPE: \ - { \ - auto column = DataTypeNumber().createColumn(); \ - fillColumnWithNumericData(arrow_column, *column); \ - return column; \ - } - FOR_ARROW_INDEXES_TYPES(DISPATCH) + case ARROW_NUMERIC_TYPE: \ + { \ + return readColumnWithNumericData(arrow_column, "").column; \ + } + FOR_ARROW_INDEXES_TYPES(DISPATCH) # undef DISPATCH - default: - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unsupported type for indexes in LowCardinality: {}.", arrow_column->type()->name()); - } -} - -static void readColumnFromArrowColumn( - std::shared_ptr & arrow_column, - IColumn & internal_column, - const std::string & column_name, - const std::string & format_name, - bool is_nullable, - std::unordered_map dictionary_values) -{ - if (internal_column.isNullable()) - { - ColumnNullable & column_nullable = assert_cast(internal_column); - readColumnFromArrowColumn( - arrow_column, column_nullable.getNestedColumn(), column_name, format_name, true, dictionary_values); - fillByteMapFromArrowColumn(arrow_column, column_nullable.getNullMapColumn()); - return; + default: + throw Exception(fmt::format("Unsupported type for indexes in LowCardinality: {}.", arrow_column->type()->name()), ErrorCodes::BAD_ARGUMENTS); + } } - /// TODO: check if a column is const? - if (!is_nullable && arrow_column->null_count() && arrow_column->type()->id() != arrow::Type::LIST - && arrow_column->type()->id() != arrow::Type::MAP && arrow_column->type()->id() != arrow::Type::STRUCT) + static std::shared_ptr getNestedArrowColumn(std::shared_ptr & arrow_column) { - throw Exception - { - ErrorCodes::CANNOT_INSERT_NULL_IN_ORDINARY_COLUMN, - "Can not insert NULL data into non-nullable column \"{}\".", column_name - }; + arrow::ArrayVector array_vector; + array_vector.reserve(arrow_column->num_chunks()); + for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i) + { + arrow::ListArray & list_chunk = static_cast(*(arrow_column->chunk(chunk_i))); + std::shared_ptr chunk = list_chunk.values(); + array_vector.emplace_back(std::move(chunk)); + } + return std::make_shared(array_vector); } - switch (arrow_column->type()->id()) + static ColumnWithTypeAndName readColumnFromArrowColumn( + std::shared_ptr & arrow_column, + const std::string & column_name, + const std::string & format_name, + bool is_nullable, + std::unordered_map> & dictionary_values) { - case arrow::Type::STRING: - case arrow::Type::BINARY: - //case arrow::Type::FIXED_SIZE_BINARY: - fillColumnWithStringData(arrow_column, internal_column); - break; - case arrow::Type::BOOL: - fillColumnWithBooleanData(arrow_column, internal_column); - break; - case arrow::Type::DATE32: - if (WhichDataType(internal_column.getDataType()).isUInt16()) - { - fillColumnWithDate32Data(arrow_column, internal_column); - } - else - { - fillDate32ColumnWithDate32Data(arrow_column, internal_column); - } - break; - case arrow::Type::DATE64: - fillColumnWithDate64Data(arrow_column, internal_column); - break; - case arrow::Type::TIMESTAMP: - fillColumnWithTimestampData(arrow_column, internal_column); - break; + if (!is_nullable && arrow_column->null_count() && arrow_column->type()->id() != arrow::Type::LIST + && arrow_column->type()->id() != arrow::Type::MAP && arrow_column->type()->id() != arrow::Type::STRUCT) + { + auto nested_column = readColumnFromArrowColumn(arrow_column, column_name, format_name, true, dictionary_values); + auto nullmap_column = readByteMapFromArrowColumn(arrow_column); + auto nullable_type = std::make_shared(std::move(nested_column.type)); + auto nullable_column = ColumnNullable::create(std::move(nested_column.column), std::move(nullmap_column)); + return {std::move(nullable_column), std::move(nullable_type), column_name}; + } + + switch (arrow_column->type()->id()) + { + case arrow::Type::STRING: + case arrow::Type::BINARY: + //case arrow::Type::FIXED_SIZE_BINARY: + return readColumnWithStringData(arrow_column, column_name); + case arrow::Type::BOOL: + return readColumnWithBooleanData(arrow_column, column_name); + case arrow::Type::DATE32: + return readColumnWithDate32Data(arrow_column, column_name); + case arrow::Type::DATE64: + return readColumnWithDate64Data(arrow_column, column_name); + case arrow::Type::TIMESTAMP: + return readColumnWithTimestampData(arrow_column, column_name); #if defined(ARCADIA_BUILD) - case arrow::Type::DECIMAL: - fillColumnWithDecimalData(arrow_column, internal_column /*, internal_nested_type*/); - break; + case arrow::Type::DECIMAL: + return readColumnWithDecimalData(arrow_column, column_name); #else - case arrow::Type::DECIMAL128: - fillColumnWithDecimalData(arrow_column, internal_column /*, internal_nested_type*/); - break; - case arrow::Type::DECIMAL256: - fillColumnWithDecimalData(arrow_column, internal_column /*, internal_nested_type*/); - break; -#endif - case arrow::Type::MAP: [[fallthrough]]; - case arrow::Type::LIST: - { - arrow::ArrayVector array_vector; - array_vector.reserve(arrow_column->num_chunks()); - for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i) + case arrow::Type::DECIMAL128: + return readColumnWithDecimalData(arrow_column, column_name); + case arrow::Type::DECIMAL256: + return readColumnWithDecimalData(arrow_column, column_name); +#endif + case arrow::Type::MAP: { - arrow::ListArray & list_chunk = dynamic_cast(*(arrow_column->chunk(chunk_i))); - std::shared_ptr chunk = list_chunk.values(); - array_vector.emplace_back(std::move(chunk)); + auto arrow_nested_column = getNestedArrowColumn(arrow_column); + auto nested_column = readColumnFromArrowColumn(arrow_nested_column, column_name, format_name, false, dictionary_values); + auto offsets_column = readOffsetsFromArrowListColumn(arrow_column); + + const auto * tuple_column = assert_cast(nested_column.column.get()); + const auto * tuple_type = assert_cast(nested_column.type.get()); + auto map_column = ColumnMap::create(std::move(tuple_column->getColumnPtr(0)), std::move(tuple_column->getColumnPtr(1)), std::move(offsets_column)); + auto map_type = std::make_shared(tuple_type->getElements()[0], tuple_type->getElements()[1]); + return {std::move(map_column), std::move(map_type), column_name}; } - auto arrow_nested_column = std::make_shared(array_vector); - - ColumnArray & column_array = arrow_column->type()->id() == arrow::Type::MAP - ? assert_cast(internal_column).getNestedColumn() - : assert_cast(internal_column); - - readColumnFromArrowColumn( - arrow_nested_column, column_array.getData(), column_name, format_name, false, dictionary_values); - - fillOffsetsFromArrowListColumn(arrow_column, column_array.getOffsetsColumn()); - break; - } - case arrow::Type::STRUCT: - { - ColumnTuple & column_tuple = assert_cast(internal_column); - int fields_count = column_tuple.tupleSize(); - std::vector nested_arrow_columns(fields_count); - for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i) + case arrow::Type::LIST: { - arrow::StructArray & struct_chunk = dynamic_cast(*(arrow_column->chunk(chunk_i))); - for (int i = 0; i < fields_count; ++i) - nested_arrow_columns[i].emplace_back(struct_chunk.field(i)); + auto arrow_nested_column = getNestedArrowColumn(arrow_column); + auto nested_column = readColumnFromArrowColumn(arrow_nested_column, column_name, format_name, false, dictionary_values); + auto offsets_column = readOffsetsFromArrowListColumn(arrow_column); + auto array_column = ColumnArray::create(std::move(nested_column.column), std::move(offsets_column)); + auto array_type = std::make_shared(nested_column.type); + return {std::move(array_column), std::move(array_type), column_name}; } - - for (int i = 0; i != fields_count; ++i) + case arrow::Type::STRUCT: { - auto nested_arrow_column = std::make_shared(nested_arrow_columns[i]); - readColumnFromArrowColumn( - nested_arrow_column, column_tuple.getColumn(i), column_name, format_name, false, dictionary_values); - } - break; - } - case arrow::Type::DICTIONARY: - { - ColumnLowCardinality & column_lc = assert_cast(internal_column); - auto & dict_values = dictionary_values[column_name]; - - /// Load dictionary values only once and reuse it. - if (!dict_values) - { - arrow::ArrayVector dict_array; + auto arrow_type = arrow_column->type(); + auto arrow_struct_type = assert_cast(arrow_type.get()); + std::vector nested_arrow_columns(arrow_struct_type->num_fields()); for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i) { - arrow::DictionaryArray & dict_chunk = dynamic_cast(*(arrow_column->chunk(chunk_i))); - dict_array.emplace_back(dict_chunk.dictionary()); + arrow::StructArray & struct_chunk = static_cast(*(arrow_column->chunk(chunk_i))); + for (int i = 0; i < arrow_struct_type->num_fields(); ++i) + nested_arrow_columns[i].emplace_back(struct_chunk.field(i)); } - auto arrow_dict_column = std::make_shared(dict_array); - auto dict_column = IColumn::mutate(column_lc.getDictionaryPtr()); - auto * uniq_column = static_cast(dict_column.get()); - auto values_column = uniq_column->getNestedColumn()->cloneEmpty(); - readColumnFromArrowColumn( - arrow_dict_column, *values_column, column_name, format_name, false, dictionary_values); - uniq_column->uniqueInsertRangeFrom(*values_column, 0, values_column->size()); - dict_values = std::move(dict_column); + Columns tuple_elements; + DataTypes tuple_types; + std::vector tuple_names; + + for (int i = 0; i != arrow_struct_type->num_fields(); ++i) + { + auto nested_arrow_column = std::make_shared(nested_arrow_columns[i]); + auto element = readColumnFromArrowColumn(nested_arrow_column, arrow_struct_type->field(i)->name(), format_name, false, dictionary_values); + tuple_elements.emplace_back(std::move(element.column)); + tuple_types.emplace_back(std::move(element.type)); + tuple_names.emplace_back(std::move(element.name)); + } + + auto tuple_column = ColumnTuple::create(std::move(tuple_elements)); + auto tuple_type = std::make_shared(std::move(tuple_types), std::move(tuple_names)); + return {std::move(tuple_column), std::move(tuple_type), column_name}; } - - arrow::ArrayVector indexes_array; - for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i) + case arrow::Type::DICTIONARY: { - arrow::DictionaryArray & dict_chunk = dynamic_cast(*(arrow_column->chunk(chunk_i))); - indexes_array.emplace_back(dict_chunk.indices()); + auto & dict_values = dictionary_values[column_name]; + /// Load dictionary values only once and reuse it. + if (!dict_values) + { + arrow::ArrayVector dict_array; + for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i) + { + arrow::DictionaryArray & dict_chunk = static_cast(*(arrow_column->chunk(chunk_i))); + dict_array.emplace_back(dict_chunk.dictionary()); + } + auto arrow_dict_column = std::make_shared(dict_array); + auto dict_column = readColumnFromArrowColumn(arrow_dict_column, column_name, format_name, false, dictionary_values); + + /// We should convert read column to ColumnUnique. + auto tmp_lc_column = DataTypeLowCardinality(dict_column.type).createColumn(); + auto tmp_dict_column = IColumn::mutate(assert_cast(tmp_lc_column.get())->getDictionaryPtr()); + static_cast(tmp_dict_column.get())->uniqueInsertRangeFrom(*dict_column.column, 0, dict_column.column->size()); + dict_column.column = std::move(tmp_dict_column); + dict_values = std::make_shared(std::move(dict_column)); + } + + arrow::ArrayVector indexes_array; + for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i) + { + arrow::DictionaryArray & dict_chunk = static_cast(*(arrow_column->chunk(chunk_i))); + indexes_array.emplace_back(dict_chunk.indices()); + } + + auto arrow_indexes_column = std::make_shared(indexes_array); + auto indexes_column = readColumnWithIndexesData(arrow_indexes_column); + auto lc_column = ColumnLowCardinality::create(dict_values->column, std::move(indexes_column)); + auto lc_type = std::make_shared(dict_values->type); + return {std::move(lc_column), std::move(lc_type), column_name}; + } +# define DISPATCH(ARROW_NUMERIC_TYPE, CPP_NUMERIC_TYPE) \ + case ARROW_NUMERIC_TYPE: \ + return readColumnWithNumericData(arrow_column, column_name); + FOR_ARROW_NUMERIC_TYPES(DISPATCH) +# undef DISPATCH + // TODO: support TIMESTAMP_MICROS and TIMESTAMP_MILLIS with truncated micro- and milliseconds? + // TODO: read JSON as a string? + // TODO: read UUID as a string? + default: + throw Exception + { + fmt::format(R"(Unsupported {} type "{}" of an input column "{}".)", format_name, arrow_column->type()->name(), column_name), + ErrorCodes::UNKNOWN_TYPE + }; + } + } + + static Block arrowSchemaToCHHeader(const arrow::Schema & schema, const std::string & format_name) + { + ColumnsWithTypeAndName sample_columns; + for (const auto & field : schema.fields()) + { + /// Create empty arrow column by it's type and convert it to ClickHouse column. + arrow::MemoryPool* pool = arrow::default_memory_pool(); + std::unique_ptr array_builder; + arrow::Status status = MakeBuilder(pool, field->type(), &array_builder); + checkStatus(status, field->name(), format_name); + std::shared_ptr arrow_array; + status = array_builder->Finish(&arrow_array); + checkStatus(status, field->name(), format_name); + arrow::ArrayVector array_vector = {arrow_array}; + auto arrow_column = std::make_shared(array_vector); + std::unordered_map> dict_values; + ColumnWithTypeAndName sample_column = readColumnFromArrowColumn(arrow_column, field->name(), format_name, false, dict_values); + sample_columns.emplace_back(std::move(sample_column)); + } + return Block(std::move(sample_columns)); + } + + ArrowColumnToCHColumn::ArrowColumnToCHColumn( + const Block & header_, const std::string & format_name_, bool import_nested_) + : header(header_), format_name(format_name_), import_nested(import_nested_) + { + } + + ArrowColumnToCHColumn::ArrowColumnToCHColumn( + const arrow::Schema & schema, const std::string & format_name_, bool import_nested_) + : header(arrowSchemaToCHHeader(schema, format_name_)), format_name(format_name_), import_nested(import_nested_) + { + } + + void ArrowColumnToCHColumn::arrowTableToCHChunk(Chunk & res, std::shared_ptr & table) + { + Columns columns_list; + UInt64 num_rows = 0; + + columns_list.reserve(header.rows()); + + using NameToColumnPtr = std::unordered_map>; + + NameToColumnPtr name_to_column_ptr; + for (const auto& column_name : table->ColumnNames()) + { + std::shared_ptr arrow_column = table->GetColumnByName(column_name); + name_to_column_ptr[column_name] = arrow_column; + } + + std::unordered_map nested_tables; + for (size_t column_i = 0, columns = header.columns(); column_i < columns; ++column_i) + { + const ColumnWithTypeAndName & header_column = header.getByPosition(column_i); + + bool read_from_nested = false; + String nested_table_name = Nested::extractTableName(header_column.name); + if (!name_to_column_ptr.contains(header_column.name)) + { + /// Check if it's a column from nested table. + if (import_nested && name_to_column_ptr.contains(nested_table_name)) + { + if (!nested_tables.contains(nested_table_name)) + { + std::shared_ptr arrow_column = name_to_column_ptr[nested_table_name]; + ColumnsWithTypeAndName cols = {readColumnFromArrowColumn(arrow_column, nested_table_name, format_name, false, dictionary_values)}; + Block block(cols); + nested_tables[nested_table_name] = std::make_shared(Nested::flatten(block)); + } + + read_from_nested = nested_tables[nested_table_name]->has(header_column.name); + } + + + // TODO: What if some columns were not presented? Insert NULLs? What if a column is not nullable? + if (!read_from_nested) + throw Exception{ + fmt::format("Column \"{}\" is not presented in input data.", header_column.name), ErrorCodes::THERE_IS_NO_COLUMN}; } - auto arrow_indexes_column = std::make_shared(indexes_array); - auto indexes_column = createAndFillColumnWithIndexesData(arrow_indexes_column); + std::shared_ptr arrow_column = name_to_column_ptr[header_column.name]; - auto new_column_lc = ColumnLowCardinality::create(dict_values, std::move(indexes_column)); - column_lc = std::move(*new_column_lc); - break; + ColumnWithTypeAndName column; + if (read_from_nested) + column = nested_tables[nested_table_name]->getByName(header_column.name); + else + column = readColumnFromArrowColumn(arrow_column, header_column.name, format_name, false, dictionary_values); + + column.column = castColumn(column, header_column.type); + column.type = header_column.type; + num_rows = column.column->size(); + columns_list.push_back(std::move(column.column)); } -# define DISPATCH(ARROW_NUMERIC_TYPE, CPP_NUMERIC_TYPE) \ - case ARROW_NUMERIC_TYPE: \ - fillColumnWithNumericData(arrow_column, internal_column); \ - break; - FOR_ARROW_NUMERIC_TYPES(DISPATCH) -# undef DISPATCH - // TODO: support TIMESTAMP_MICROS and TIMESTAMP_MILLIS with truncated micro- and milliseconds? - // TODO: read JSON as a string? - // TODO: read UUID as a string? - default: - throw Exception(ErrorCodes::UNKNOWN_TYPE, - "Unsupported {} type '{}' of an input column '{}'.", format_name, arrow_column->type()->name(), column_name); + res.setColumns(columns_list, num_rows); } } - -static DataTypePtr getInternalType( - std::shared_ptr arrow_type, - const DataTypePtr & column_type, - const std::string & column_name, - const std::string & format_name) -{ - if (column_type->isNullable()) - { - DataTypePtr nested_type = assert_cast(column_type.get())->getNestedType(); - return makeNullable(getInternalType(arrow_type, nested_type, column_name, format_name)); - } - -#if defined(ARCADIA_BUILD) - if (arrow_type->id() == arrow::Type::DECIMAL) - { - const auto & decimal_type = dynamic_cast(*arrow_type); - return std::make_shared>(decimal_type.precision(), decimal_type.scale()); - } -#else - if (arrow_type->id() == arrow::Type::DECIMAL128) - { - const auto & decimal_type = dynamic_cast(*arrow_type); - return std::make_shared>(decimal_type.precision(), decimal_type.scale()); - } - - if (arrow_type->id() == arrow::Type::DECIMAL256) - { - const auto & decimal_type = dynamic_cast(*arrow_type); - return std::make_shared>(decimal_type.precision(), decimal_type.scale()); - } #endif - if (arrow_type->id() == arrow::Type::LIST) - { - const auto & list_type = dynamic_cast(*arrow_type); - auto list_nested_type = list_type.value_type(); - - const DataTypeArray * array_type = typeid_cast(column_type.get()); - if (!array_type) - throw Exception{ErrorCodes::CANNOT_CONVERT_TYPE, - "Cannot convert arrow LIST type to a not Array ClickHouse type {}.", column_type->getName()}; - - return std::make_shared(getInternalType(list_nested_type, array_type->getNestedType(), column_name, format_name)); - } - - if (arrow_type->id() == arrow::Type::STRUCT) - { - const auto & struct_type = dynamic_cast(*arrow_type); - const DataTypeTuple * tuple_type = typeid_cast(column_type.get()); - if (!tuple_type) - throw Exception{ErrorCodes::CANNOT_CONVERT_TYPE, - "Cannot convert arrow STRUCT type to a not Tuple ClickHouse type {}.", column_type->getName()}; - - const DataTypes & tuple_nested_types = tuple_type->getElements(); - int internal_fields_num = tuple_nested_types.size(); - /// If internal column has less elements then arrow struct, we will select only first internal_fields_num columns. - if (internal_fields_num > struct_type.num_fields()) - throw Exception( - ErrorCodes::CANNOT_CONVERT_TYPE, - "Cannot convert arrow STRUCT with {} fields to a ClickHouse Tuple with {} elements: {}.", - struct_type.num_fields(), - internal_fields_num, - column_type->getName()); - - DataTypes nested_types; - for (int i = 0; i < internal_fields_num; ++i) - nested_types.push_back(getInternalType(struct_type.field(i)->type(), tuple_nested_types[i], column_name, format_name)); - - return std::make_shared(std::move(nested_types)); - } - - if (arrow_type->id() == arrow::Type::DICTIONARY) - { - const auto & arrow_dict_type = dynamic_cast(*arrow_type); - const auto * lc_type = typeid_cast(column_type.get()); - /// We allow to insert arrow dictionary into a non-LowCardinality column. - const auto & dict_type = lc_type ? lc_type->getDictionaryType() : column_type; - return std::make_shared(getInternalType(arrow_dict_type.value_type(), dict_type, column_name, format_name)); - } - - if (arrow_type->id() == arrow::Type::MAP) - { - const auto & arrow_map_type = typeid_cast(*arrow_type); - const auto * map_type = typeid_cast(column_type.get()); - if (!map_type) - throw Exception{ErrorCodes::CANNOT_CONVERT_TYPE, "Cannot convert arrow MAP type to a not Map ClickHouse type {}.", column_type->getName()}; - - return std::make_shared( - getInternalType(arrow_map_type.key_type(), map_type->getKeyType(), column_name, format_name), - getInternalType(arrow_map_type.item_type(), map_type->getValueType(), column_name, format_name)); - } - - if (arrow_type->id() == arrow::Type::UINT16 - && (isDate(column_type) || isDateTime(column_type) || isDate32(column_type) || isDateTime64(column_type))) - { - /// Read UInt16 as Date. It will allow correct conversion to DateTime further. - return std::make_shared(); - } - - auto filter = [=](auto && elem) - { - auto which = WhichDataType(column_type); - if (arrow_type->id() == arrow::Type::DATE32 && which.isDateOrDate32()) - { - return (strcmp(elem.second, "Date") == 0 && which.isDate()) - || (strcmp(elem.second, "Date32") == 0 && which.isDate32()); - } - else - { - return elem.first == arrow_type->id(); - } - }; - if (const auto * internal_type_it = std::find_if(arrow_type_to_internal_type.begin(), arrow_type_to_internal_type.end(), filter); - internal_type_it != arrow_type_to_internal_type.end()) - { - return DataTypeFactory::instance().get(internal_type_it->second); - } - - throw Exception(ErrorCodes::CANNOT_CONVERT_TYPE, - "The type '{}' of an input column '{}' is not supported for conversion from {} data format.", - arrow_type->name(), column_name, format_name); -} - -ArrowColumnToCHColumn::ArrowColumnToCHColumn(const Block & header_, std::shared_ptr schema_, const std::string & format_name_) - : header(header_), format_name(format_name_) -{ - for (const auto & field : schema_->fields()) - { - if (header.has(field->name())) - { - const auto column_type = recursiveRemoveLowCardinality(header.getByName(field->name()).type); - name_to_internal_type[field->name()] = getInternalType(field->type(), column_type, field->name(), format_name); - } - } -} - -void ArrowColumnToCHColumn::arrowTableToCHChunk(Chunk & res, std::shared_ptr & table) -{ - Columns columns_list; - UInt64 num_rows = 0; - - columns_list.reserve(header.rows()); - - using NameToColumnPtr = std::unordered_map>; - - NameToColumnPtr name_to_column_ptr; - for (const auto & column_name : table->ColumnNames()) - { - std::shared_ptr arrow_column = table->GetColumnByName(column_name); - name_to_column_ptr[column_name] = arrow_column; - } - - for (size_t column_i = 0, columns = header.columns(); column_i < columns; ++column_i) - { - const ColumnWithTypeAndName & header_column = header.getByPosition(column_i); - - if (name_to_column_ptr.find(header_column.name) == name_to_column_ptr.end()) - // TODO: What if some columns were not presented? Insert NULLs? What if a column is not nullable? - throw Exception(ErrorCodes::THERE_IS_NO_COLUMN, - "Column '{}' is not presented in input data.", header_column.name); - - std::shared_ptr arrow_column = name_to_column_ptr[header_column.name]; - - DataTypePtr & internal_type = name_to_internal_type[header_column.name]; - MutableColumnPtr read_column = internal_type->createColumn(); - readColumnFromArrowColumn(arrow_column, *read_column, header_column.name, format_name, false, dictionary_values); - - ColumnWithTypeAndName column; - column.name = header_column.name; - column.type = internal_type; - column.column = std::move(read_column); - - column.column = castColumn(column, header_column.type); - column.type = header_column.type; - num_rows = column.column->size(); - columns_list.push_back(std::move(column.column)); - } - - res.setColumns(columns_list, num_rows); -} - -} - -#endif diff --git a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.h b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.h index 7f38dc7a31c..5917d9a81a7 100644 --- a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.h +++ b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.h @@ -7,6 +7,7 @@ #if USE_ARROW || USE_ORC || USE_PARQUET #include +#include #include @@ -19,19 +20,22 @@ class Chunk; class ArrowColumnToCHColumn { public: - ArrowColumnToCHColumn(const Block & header_, std::shared_ptr schema_, const std::string & format_name_); + ArrowColumnToCHColumn(const Block & header_, const std::string & format_name_, bool import_nested_); + + /// Create header by arrow schema. It will be useful for inserting + /// data from file without knowing table structure. + ArrowColumnToCHColumn(const arrow::Schema & schema, const std::string & format_name, bool import_nested_); void arrowTableToCHChunk(Chunk & res, std::shared_ptr & table); private: const Block & header; - std::unordered_map name_to_internal_type; const std::string format_name; - + bool import_nested; /// Map {column name : dictionary column}. /// To avoid converting dictionary from Arrow Dictionary /// to LowCardinality every chunk we save it and reuse. - std::unordered_map dictionary_values; + std::unordered_map> dictionary_values; }; } diff --git a/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp b/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp index 42aa9e6ddc7..7f85e26785f 100644 --- a/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp +++ b/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp @@ -295,6 +295,7 @@ namespace DB FOR_ARROW_TYPES(DISPATCH) #undef DISPATCH + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot fill arrow array with {} data.", column_type->getName()); } template diff --git a/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp b/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp index 6ee247413e9..9d56d2c8fa8 100644 --- a/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp @@ -9,6 +9,7 @@ #include #include "ArrowBufferedStreams.h" #include "ArrowColumnToCHColumn.h" +#include namespace DB { @@ -26,7 +27,8 @@ namespace ErrorCodes throw Exception(_s.ToString(), ErrorCodes::BAD_ARGUMENTS); \ } while (false) -ORCBlockInputFormat::ORCBlockInputFormat(ReadBuffer & in_, Block header_) : IInputFormat(std::move(header_), in_) +ORCBlockInputFormat::ORCBlockInputFormat(ReadBuffer & in_, Block header_, const FormatSettings & format_settings_) + : IInputFormat(std::move(header_), in_), format_settings(format_settings_) { } @@ -98,7 +100,11 @@ void ORCBlockInputFormat::prepareReader() std::shared_ptr schema; THROW_ARROW_NOT_OK(file_reader->ReadSchema(&schema)); - arrow_column_to_ch_column = std::make_unique(getPort().getHeader(), schema, "ORC"); + arrow_column_to_ch_column = std::make_unique(getPort().getHeader(), "ORC", format_settings.orc.import_nested); + + std::unordered_set nested_table_names; + if (format_settings.orc.import_nested) + nested_table_names = Nested::getAllTableNames(getPort().getHeader()); /// In ReadStripe column indices should be started from 1, /// because 0 indicates to select all columns. @@ -108,7 +114,8 @@ void ORCBlockInputFormat::prepareReader() /// LIST type require 2 indices, STRUCT - the number of elements + 1, /// so we should recursively count the number of indices we need for this type. int indexes_count = countIndicesForType(schema->field(i)->type()); - if (getPort().getHeader().has(schema->field(i)->name())) + const auto & name = schema->field(i)->name(); + if (getPort().getHeader().has(name) || nested_table_names.contains(name)) { for (int j = 0; j != indexes_count; ++j) include_indices.push_back(index + j); @@ -124,9 +131,9 @@ void registerInputFormatProcessorORC(FormatFactory &factory) [](ReadBuffer &buf, const Block &sample, const RowInputFormatParams &, - const FormatSettings & /* settings */) + const FormatSettings & settings) { - return std::make_shared(buf, sample); + return std::make_shared(buf, sample, settings); }); factory.markFormatAsColumnOriented("ORC"); } diff --git a/src/Processors/Formats/Impl/ORCBlockInputFormat.h b/src/Processors/Formats/Impl/ORCBlockInputFormat.h index f27685a9884..254d0554cb2 100644 --- a/src/Processors/Formats/Impl/ORCBlockInputFormat.h +++ b/src/Processors/Formats/Impl/ORCBlockInputFormat.h @@ -3,6 +3,7 @@ #if USE_ORC #include +#include namespace arrow::adapters::orc { class ORCFileReader; } @@ -14,7 +15,7 @@ class ArrowColumnToCHColumn; class ORCBlockInputFormat : public IInputFormat { public: - ORCBlockInputFormat(ReadBuffer & in_, Block header_); + ORCBlockInputFormat(ReadBuffer & in_, Block header_, const FormatSettings & format_settings_); String getName() const override { return "ORCBlockInputFormat"; } @@ -38,6 +39,8 @@ private: // indices of columns to read from ORC file std::vector include_indices; + const FormatSettings format_settings; + void prepareReader(); }; diff --git a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp index 07a0e15cb6b..a0b92f98ca9 100644 --- a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp @@ -11,6 +11,7 @@ #include #include "ArrowBufferedStreams.h" #include "ArrowColumnToCHColumn.h" +#include #include @@ -30,8 +31,8 @@ namespace ErrorCodes throw Exception(_s.ToString(), ErrorCodes::BAD_ARGUMENTS); \ } while (false) -ParquetBlockInputFormat::ParquetBlockInputFormat(ReadBuffer & in_, Block header_) - : IInputFormat(std::move(header_), in_) +ParquetBlockInputFormat::ParquetBlockInputFormat(ReadBuffer & in_, Block header_, const FormatSettings & format_settings_) + : IInputFormat(std::move(header_), in_), format_settings(format_settings_) { } @@ -98,7 +99,11 @@ void ParquetBlockInputFormat::prepareReader() std::shared_ptr schema; THROW_ARROW_NOT_OK(file_reader->GetSchema(&schema)); - arrow_column_to_ch_column = std::make_unique(getPort().getHeader(), schema, "Parquet"); + arrow_column_to_ch_column = std::make_unique(getPort().getHeader(), "Parquet", format_settings.parquet.import_nested); + + std::unordered_set nested_table_names; + if (format_settings.parquet.import_nested) + nested_table_names = Nested::getAllTableNames(getPort().getHeader()); int index = 0; for (int i = 0; i < schema->num_fields(); ++i) @@ -107,7 +112,8 @@ void ParquetBlockInputFormat::prepareReader() /// nested elements, so we should recursively /// count the number of indices we need for this type. int indexes_count = countIndicesForType(schema->field(i)->type()); - if (getPort().getHeader().has(schema->field(i)->name())) + const auto & name = schema->field(i)->name(); + if (getPort().getHeader().has(name) || nested_table_names.contains(name)) { for (int j = 0; j != indexes_count; ++j) column_indices.push_back(index + j); @@ -123,9 +129,9 @@ void registerInputFormatProcessorParquet(FormatFactory &factory) [](ReadBuffer &buf, const Block &sample, const RowInputFormatParams &, - const FormatSettings & /* settings */) + const FormatSettings & settings) { - return std::make_shared(buf, sample); + return std::make_shared(buf, sample, settings); }); factory.markFormatAsColumnOriented("Parquet"); } diff --git a/src/Processors/Formats/Impl/ParquetBlockInputFormat.h b/src/Processors/Formats/Impl/ParquetBlockInputFormat.h index b68f97c005a..c2ed1552423 100644 --- a/src/Processors/Formats/Impl/ParquetBlockInputFormat.h +++ b/src/Processors/Formats/Impl/ParquetBlockInputFormat.h @@ -4,6 +4,7 @@ #if USE_PARQUET #include +#include namespace parquet::arrow { class FileReader; } @@ -17,7 +18,7 @@ class ArrowColumnToCHColumn; class ParquetBlockInputFormat : public IInputFormat { public: - ParquetBlockInputFormat(ReadBuffer & in_, Block header_); + ParquetBlockInputFormat(ReadBuffer & in_, Block header_, const FormatSettings & format_settings_); void resetParser() override; @@ -36,6 +37,7 @@ private: std::vector column_indices; std::unique_ptr arrow_column_to_ch_column; int row_group_current = 0; + const FormatSettings format_settings; }; } diff --git a/tests/queries/0_stateless/00900_orc_arrow_parquet_nested.reference b/tests/queries/0_stateless/00900_orc_arrow_parquet_nested.reference new file mode 100644 index 00000000000..22eb936a0e0 --- /dev/null +++ b/tests/queries/0_stateless/00900_orc_arrow_parquet_nested.reference @@ -0,0 +1,21 @@ +Arrow +[1,2,3] ['123','456','789'] [9.8,10.12,11.14] +[4,5,6] ['101112','131415','161718'] [123.8,10.2,11.414] +[7,8,9] ['101','415','118'] [13.08,1.12,0.414] +[1,2,3] ['123','456','789'] [9.8,10.12,11.14] [[(1,'123',9.8),(2,'456',10.12),(3,'789',11.14)],[(4,'101112',123.8),(5,'131415',10.2),(6,'161718',11.414)],[(7,'101',13.08),(8,'415',1.12),(9,'118',0.414)]] +[4,5,6] ['101112','131415','161718'] [123.8,10.2,11.414] [[(1,'123',9.8),(2,'456',10.12),(3,'789',11.14)],[(4,'101112',123.8),(5,'131415',10.2),(6,'161718',11.414)],[(7,'101',13.08),(8,'415',1.12),(9,'118',0.414)]] +[7,8,9] ['101','415','118'] [13.08,1.12,0.414] [[(1,'123',9.8),(2,'456',10.12),(3,'789',11.14)],[(4,'101112',123.8),(5,'131415',10.2),(6,'161718',11.414)],[(7,'101',13.08),(8,'415',1.12),(9,'118',0.414)]] +Parquet +[1,2,3] ['123','456','789'] [9.8,10.12,11.14] +[4,5,6] ['101112','131415','161718'] [123.8,10.2,11.414] +[7,8,9] ['101','415','118'] [13.08,1.12,0.414] +[1,2,3] ['123','456','789'] [9.8,10.12,11.14] [[(1,'123',9.8),(2,'456',10.12),(3,'789',11.14)],[(4,'101112',123.8),(5,'131415',10.2),(6,'161718',11.414)],[(7,'101',13.08),(8,'415',1.12),(9,'118',0.414)]] +[4,5,6] ['101112','131415','161718'] [123.8,10.2,11.414] [[(1,'123',9.8),(2,'456',10.12),(3,'789',11.14)],[(4,'101112',123.8),(5,'131415',10.2),(6,'161718',11.414)],[(7,'101',13.08),(8,'415',1.12),(9,'118',0.414)]] +[7,8,9] ['101','415','118'] [13.08,1.12,0.414] [[(1,'123',9.8),(2,'456',10.12),(3,'789',11.14)],[(4,'101112',123.8),(5,'131415',10.2),(6,'161718',11.414)],[(7,'101',13.08),(8,'415',1.12),(9,'118',0.414)]] +ORC +[1,2,3] ['123','456','789'] [9.8,10.12,11.14] +[4,5,6] ['101112','131415','161718'] [123.8,10.2,11.414] +[7,8,9] ['101','415','118'] [13.08,1.12,0.414] +[1,2,3] ['123','456','789'] [9.8,10.12,11.14] [[(1,'123',9.8),(2,'456',10.12),(3,'789',11.14)],[(4,'101112',123.8),(5,'131415',10.2),(6,'161718',11.414)],[(7,'101',13.08),(8,'415',1.12),(9,'118',0.414)]] +[4,5,6] ['101112','131415','161718'] [123.8,10.2,11.414] [[(1,'123',9.8),(2,'456',10.12),(3,'789',11.14)],[(4,'101112',123.8),(5,'131415',10.2),(6,'161718',11.414)],[(7,'101',13.08),(8,'415',1.12),(9,'118',0.414)]] +[7,8,9] ['101','415','118'] [13.08,1.12,0.414] [[(1,'123',9.8),(2,'456',10.12),(3,'789',11.14)],[(4,'101112',123.8),(5,'131415',10.2),(6,'161718',11.414)],[(7,'101',13.08),(8,'415',1.12),(9,'118',0.414)]] diff --git a/tests/queries/0_stateless/00900_orc_arrow_parquet_nested.sh b/tests/queries/0_stateless/00900_orc_arrow_parquet_nested.sh new file mode 100755 index 00000000000..47d7b8519be --- /dev/null +++ b/tests/queries/0_stateless/00900_orc_arrow_parquet_nested.sh @@ -0,0 +1,36 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS nested_table" +${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS nested_nested_table" + +${CLICKHOUSE_CLIENT} --query="CREATE TABLE nested_table (table Nested(elem1 Int32, elem2 String, elem3 Float32)) engine=Memory" + +${CLICKHOUSE_CLIENT} --query="CREATE TABLE nested_nested_table (table Nested(elem1 Int32, elem2 String, elem3 Float32, nested Nested(elem1 Int32, elem2 String, elem3 Float32))) engine=Memory" + + +formats=('Arrow' 'Parquet' 'ORC') +format_files=('arrow' 'parquet' 'orc') + +for ((i = 0; i < 3; i++)) do + echo ${formats[i]} + + ${CLICKHOUSE_CLIENT} --query="TRUNCATE TABLE nested_table" + cat $CUR_DIR/data_orc_arrow_parquet_nested/nested_table.${format_files[i]} | ${CLICKHOUSE_CLIENT} -q "INSERT INTO nested_table FORMAT ${formats[i]} SETTINGS input_format_${format_files[i]}_import_nested = 1" + + ${CLICKHOUSE_CLIENT} --query="SELECT * FROM nested_table" + + + ${CLICKHOUSE_CLIENT} --query="TRUNCATE TABLE nested_nested_table" + cat $CUR_DIR/data_orc_arrow_parquet_nested/nested_nested_table.${format_files[i]} | ${CLICKHOUSE_CLIENT} -q "INSERT INTO nested_nested_table FORMAT ${formats[i]} SETTINGS input_format_${format_files[i]}_import_nested = 1" + + ${CLICKHOUSE_CLIENT} --query="SELECT * FROM nested_nested_table" + + +done + +${CLICKHOUSE_CLIENT} --query="DROP TABLE nested_table" +${CLICKHOUSE_CLIENT} --query="DROP TABLE nested_nested_table" diff --git a/tests/queries/0_stateless/01273_arrow_dictionaries_load.reference b/tests/queries/0_stateless/01273_arrow_dictionaries_load.reference index 7321c396a59..99f5e505d82 100644 --- a/tests/queries/0_stateless/01273_arrow_dictionaries_load.reference +++ b/tests/queries/0_stateless/01273_arrow_dictionaries_load.reference @@ -1,2 +1,5 @@ 1 ['a','b','c'] ('z','6') 2 ['d','e'] ('x','9') +1 ['a','b','c'] ('z','6') +2 ['d','e'] ('x','9') +20000000 diff --git a/tests/queries/0_stateless/01273_arrow_dictionaries_load.sh b/tests/queries/0_stateless/01273_arrow_dictionaries_load.sh index 38e6c2c1b01..3f6fcff3dcc 100755 --- a/tests/queries/0_stateless/01273_arrow_dictionaries_load.sh +++ b/tests/queries/0_stateless/01273_arrow_dictionaries_load.sh @@ -20,5 +20,7 @@ ${CLICKHOUSE_CLIENT} --query="SELECT * FROM arrow_dicts FORMAT Arrow SETTINGS ou cat "${CLICKHOUSE_TMP}"/dicts.arrow | ${CLICKHOUSE_CLIENT} -q "INSERT INTO arrow_dicts FORMAT Arrow" +${CLICKHOUSE_CLIENT} --query="SELECT count() FROM arrow_dicts" + ${CLICKHOUSE_CLIENT} --query="DROP TABLE arrow_dicts" diff --git a/tests/queries/0_stateless/data_orc_arrow_parquet_nested/nested_nested_arrow.cpp b/tests/queries/0_stateless/data_orc_arrow_parquet_nested/nested_nested_arrow.cpp new file mode 100644 index 00000000000..2eb1739453c --- /dev/null +++ b/tests/queries/0_stateless/data_orc_arrow_parquet_nested/nested_nested_arrow.cpp @@ -0,0 +1,178 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +void write_arrow(const arrow::Table & table) +{ + auto file = arrow::io::FileOutputStream::Open("nested_nested_table.arrow"); + + auto writer = arrow::ipc::MakeFileWriter(file->get(), table.schema()).ValueOrDie(); + + auto status = writer->WriteTable(table, 100000); + + if (!status.ok()) + throw std::runtime_error(status.ToString()); + + status = writer->Close(); + + if (!status.ok()) + throw std::runtime_error(status.ToString()); +} + +void write_parquet(const arrow::Table & table) +{ + auto file = arrow::io::FileOutputStream::Open("nested_nested_table.parquet"); + + std::unique_ptr writer; + parquet::WriterProperties::Builder prop_builder; + auto props = prop_builder.build(); + auto status = parquet::arrow::FileWriter::Open( + *table.schema(), + arrow::default_memory_pool(), + *file, + props, + &writer); + + + status = writer->WriteTable(table, 100000); + + if (!status.ok()) + throw std::runtime_error(status.ToString()); + + status = writer->Close(); + + if (!status.ok()) + throw std::runtime_error(status.ToString()); +} + +void write_orc(const arrow::Table & table) +{ + auto file = arrow::io::FileOutputStream::Open("nested_nested_table.orc"); + + auto writer = arrow::adapters::orc::ORCFileWriter::Open(file->get()).ValueOrDie(); + + auto status = writer->Write(table); + + if (!status.ok()) + throw std::runtime_error(status.ToString()); + + status = writer->Close(); + + if (!status.ok()) + throw std::runtime_error(status.ToString()); +} + + +void fillNested(arrow::ArrayBuilder * builder, bool nested) +{ + arrow::ListBuilder * list_builder = static_cast(builder); + arrow::StructBuilder * struct_builder = static_cast(list_builder->value_builder()); + arrow::Int32Builder * elem1_builder = static_cast(struct_builder->field_builder(0)); + arrow::BinaryBuilder * elem2_builder = static_cast(struct_builder->field_builder(1)); + arrow::FloatBuilder * elem3_builder = static_cast(struct_builder->field_builder(2)); + + arrow::ListBuilder * nested_list_builder = nullptr; + if (nested) + nested_list_builder = static_cast(struct_builder->field_builder(3)); + + arrow::Status status; + status = list_builder->Append(); + + std::vector elem1 = {1, 2, 3}; + std::vector elem2 = {"123", "456", "789"}; + std::vector elem3 = {9.8, 10.12, 11.14}; + status = elem1_builder->AppendValues(elem1); + status = elem2_builder->AppendValues(elem2); + status = elem3_builder->AppendValues(elem3); + if (nested) + fillNested(nested_list_builder, false); + + for (size_t i = 0; i != elem1.size(); ++i) + status = struct_builder->Append(); + + status = list_builder->Append(); + + elem1 = {4, 5, 6}; + elem2 = {"101112", "131415", "161718"}; + elem3 = {123.8, 10.2, 11.414}; + status = elem1_builder->AppendValues(elem1); + status = elem2_builder->AppendValues(elem2); + status = elem3_builder->AppendValues(elem3); + if (nested) + fillNested(nested_list_builder, false); + + for (size_t i = 0; i != elem1.size(); ++i) + status = struct_builder->Append(); + + status = list_builder->Append(); + + elem1 = {7, 8, 9}; + elem2 = {"101", "415", "118"}; + elem3 = {13.08, 1.12, 0.414}; + status = elem1_builder->AppendValues(elem1); + status = elem2_builder->AppendValues(elem2); + status = elem3_builder->AppendValues(elem3); + if (nested) + fillNested(nested_list_builder, false); + + for (size_t i = 0; i != elem1.size(); ++i) + status = struct_builder->Append(); +} + +int main() +{ + std::vector> nested_struct_fields; + nested_struct_fields.push_back(std::make_shared("elem1", arrow::int32())); + nested_struct_fields.push_back(std::make_shared("elem2", arrow::binary())); + nested_struct_fields.push_back(std::make_shared("elem3", arrow::float32())); + auto nested_struct_type = arrow::struct_(nested_struct_fields); + auto nested_field = std::make_shared("nested", nested_struct_type); + auto nested_list_type = arrow::list(nested_field); + auto nested_list_field = std::make_shared("nested", nested_list_type); + + std::vector> struct_fields; + struct_fields.push_back(std::make_shared("elem1", arrow::int32())); + struct_fields.push_back(std::make_shared("elem2", arrow::binary())); + struct_fields.push_back(std::make_shared("elem3", arrow::float32())); + struct_fields.push_back(std::make_shared("nested", nested_list_type)); + + + auto struct_type = arrow::struct_(struct_fields); + auto field = std::make_shared("table", struct_type); + auto list_type = arrow::list(field); + + arrow::MemoryPool* pool = arrow::default_memory_pool(); + std::unique_ptr tmp; + auto status = MakeBuilder(pool, list_type, &tmp); + + if (!status.ok()) + throw std::runtime_error(status.ToString()); + + fillNested(tmp.get(), true); + + std::shared_ptr array; + status = tmp->Finish(&array); + + if (!status.ok()) + throw std::runtime_error(status.ToString()); + + std::vector> fields_for_schema = {std::make_shared("table", list_type)}; + auto schema = std::make_shared(std::move(fields_for_schema)); + auto table = arrow::Table::Make(schema, {array}); + + if (!table) + throw std::runtime_error("WTF"); + + write_orc(*table); + write_arrow(*table); + write_parquet(*table); + + return 0; +} diff --git a/tests/queries/0_stateless/data_orc_arrow_parquet_nested/nested_nested_table.arrow b/tests/queries/0_stateless/data_orc_arrow_parquet_nested/nested_nested_table.arrow new file mode 100644 index 0000000000000000000000000000000000000000..2f3709d079ea37753ed6a9db467653ddea4a4931 GIT binary patch literal 2410 zcmeHJJ!@1!6up~$*?op528j|d$O{IHg0QojY&M|A7Zd}Enlx!vh?*w^HnQRtR(3)P z!B%Z-ti{qwEG!iZE&LB!*rgHIbMDNo^o%3YKkn<8_ATp9j=Fs=Xl$wKP{90p8Cwi(EM-8gkiucgZUg6pIiw|W z(4CY-N0wrw`FjM7wGr?HQP{K8#2zyr+z)SlEYCwvEzNt&dUNhKtzq3x={e6FZB0vi zKKQxixOblW@m}}_=RNo1ecILDi?^I^i5Da=&)cr79I9f#<%RwY;lgURk$? zMPPr+4+UfK^G`YZFh<@##=>QCX{_U&gCA|6yPf;9i+(ZDzJU z*O)gIV@%c5%#4{ebu(ugsNuKKqwwzSM0Wr4WOn;hHGA@Sd$CqKlD&Fb&z_%O&W<%7 zF1+9Va2ms@h4Ut;;oSQH&OZ(?hJMyDC=I4l3*+jQkE2i?2TlNd!{&hnU=cV6ECCmQ zOTaR)3S0$l05^dRfM=|qybel(>7<|i8I*?e@m-_mJCopb<4c=>ald&oPom^q=NaWb gR@{9&OcK^c?IVq)Bmx40A>3p^#A|> literal 0 HcmV?d00001 diff --git a/tests/queries/0_stateless/data_orc_arrow_parquet_nested/nested_nested_table.orc b/tests/queries/0_stateless/data_orc_arrow_parquet_nested/nested_nested_table.orc new file mode 100644 index 0000000000000000000000000000000000000000..4112df41a52f72001347427fc00b7c55f00064d8 GIT binary patch literal 1050 zcmeYda+YOa;Nsz8VE_Ul77peB1{ns1$B&*Tr8h7rit#;TU}Hd4!5P4y15_adQ6b0Y zr)b8f#;Dl20;p02B=Pi7N&}-JAD^GP8ecz9Fa#)g=~+@T1GAzlpPy!?pTg^d=XXVa zYjS97{`lO2-II;2`MS8=Qfn3i22{(>VOTC5z@P^-Zqfs=^Wyk270bYu+bshsRRh`n z7;L*AU#5CmpL7o+4?_^L^_keLZ=1qvv|dVHo*P zY(LyS$zs~SMvf(bN8O|A-lx45RdP_RVGIn`)o#Z&e%-n#dEwfLkJrq+KewlMs_gf( z=EX73RL6P$=P7}okY=&84F5jH-~>$bu1 zk_C^~O@+^^1TzE}wlFe0P7u3sLhI=1gE|)us~EO>E}R_L9O>E7?A|dkuxqP|p{9Ch zqhs^I2FK3X&E6doIZs_=(@IE8OH7@dlkR(G;{8nu-=j0|FVyq{R$SKrwYHp6XQ0)kvzNg%3m^`AtN` zOj0V3Sywki9QHF}f@C<9C`Li96&C}a2Bm7P;mUsCuW%q*Qu`6X!UCrDdySFCrBIN> z(nN)8r0_U`rKL+OgIl(!a*ivinfw_cjAuFLR_NJ0$DPn~`3%p8Q?fjq!kw_i^S#<# zl8Ry^J&JKKnizGE+Ap#RiN0#BQ0=CqZAxs1ve!|wEA)X1JI?LXN7K9eebQS$XL%F|Z;7wh7bm8;vxAVPg*Qpq(Q_&~RZNJq=$&PA*+EqdJ$Nw{lJ3T6Jfw2+l7Fv>CwrA>(|P+hI+azw zTJvv&yf*zdu&#Dr!onUNWm`x78)Un-r;Dq`jUK!qU&q=255FOTX;*tIVPPMRv-+`- zFx@4t4{F0o-5PHbKD<-`ee3M+oM6wAe+j*0Y&%3(s!PlMN)a=dBk2w9Kmuj%EJEv0 zJtVHHQzams#-U-pB#>zN76G;8+ag$#(?3LO^(sS!(-gxrf>CF1HiDeVMxMP6C3CQU zb355#+zjnqLQ{}6tj7sSoh%nC>o@({^isJzU942rZcS|Xm1=2ibt0F^X0k?L64F!1 Lb^I*P2!7vw8#+B@l~ z{$$=1#-@QeU>>LftX&6Tt=?$1R`rV3I?$o6>AG&{rf$K*Z*5!i_F6VqUsZFl9+1OqcW1H-@n4G@x(f#CoL6BC5P3v@0s z&~Ro41_sd+I|UgGI9N7L+q1|0e{Z94uR;HsYabX6b52@x@vM0iM29B>!`U-3j;*g1 z9Sd!Z9B0k^?qqDN?6_~fnd3V5Ku67(e*5#^FWYTmVDLR-$k(92)0(KEt0g>ZhDbQO z`uT@Xp8Y+aRGHc1ZU1Ze!Y78;7ZksAn|@~r@7&VXMaw1M#3i&mY4e(OQMl(>!`zLN zEf;Qn_u^6cqT`4D#?Nl~Szn*(ef{sAE?`J}dYbewfiIz9@*D>?+9Vc+2z~G7A6BX8#HPc;}t z4U9K$@Y=ZS*^*~cP8!M_sXU>F4<6)6FFX*rMqnD_W>IeTVg}|pPgpipT3EE+u$aNp zqOz#L1`?0_3f>c&gS}==ZftyrEAlLSK73AKVc?j{D8MMu(7>X^#K5EBVZhe(g_+qu H$XNmagnI2P literal 0 HcmV?d00001 diff --git a/tests/queries/0_stateless/data_orc_arrow_parquet_nested/nested_table.parquet b/tests/queries/0_stateless/data_orc_arrow_parquet_nested/nested_table.parquet new file mode 100644 index 0000000000000000000000000000000000000000..0bdcc288107451070f2eb65b183c7d98da40d69b GIT binary patch literal 1105 zcmcIkze^)g5T1Q``&_}|2+pz#3wj*4J1qsTyqOKb7(%+h`1%SA)8`U zVl!8sXJNXawGL7*)I}YoUP;gCavD<5X^4)}nJ)KhFF`ItOiqS)h>cGTQu}CnI5AXh zkgB~D_NDNjr2i}9eV`twu%~YKcV2w*{5jrV2;z(Lx1})r65n2x;_Kxf@wcC+-yh!o zEcWN_NIvOE-s8Hx&xR;@vOKhgDhYe)_IJ$x#|`pM-FMLdUBW$%z)`Kutz@T0#f*g0 z=?WC6Rw|TRB%QQbK5QyL@s*CItphmG^#eew>xW<~*K;eQ?mmA6_4Y^p^7gA~D|#BT xlK{D?Myc{kwa&E7k*GqH1Wa&p{p%J|HN;O>RuR!S)@HC#$tN_LLm#jS!%jp9(6 zwh+8VaoW^yO5MA^-%2(~zAOFyZNGE4*LD3eYi5?s>{;^We%_gz-u13`zUQE7EyHuG zYG+clT)o;@OSL&;8QNkk=H`8wrn_y)?qp}}%@kFeGWE_pa&+EW?4G+xa(2>9-woY7 zKa^}fGRNYPe|tyn_cFfe$o+1zZh76>zB^yHU2n)^-7>qOm$9Q;*2{Q?4U0#fyUn)l zPPhDSi;phik%_a*&g zn2eKYGEc6b>`QKx+$6bKa*O0v$!(I`CEaVkQ}X@EU6Z>d_ek!Q+$XtT@_^)l$%B#y zCl5&;mOLVPRPvbQN0T24yC2vgLlKe{Y>&b5>?@HdAd?5Ml$QJNet>ACiAg{w?{>r2DWgBpZ`0NiW%voRXZD>`l%}&P^^z zE>5nVTr0V5(oaUoG+9aZB{xoPmfSMAO>+C>PRU)8yCwHb?vvaBS*AIVy> zVMEnYvN_qBY)^J2yOYzCGm~?Y^OK8`t0mV=u9I9Z877ltp4=e0QF7Dd7RjxX+a-5Q zzCZba)?k>p{?Ba_D@KbHJt@`U8c$;x{o;)Xce)99l zOOlr-uS$L)`Q_yG$(xe5Ccm2eM)Hp2-O2ls-%1W9zn2_JK9+nk`Ge$-lg}idPrj7= zS@IXjUngHnzM1?@@^{HUCjXNBd-7jN%fV|g*_13N+mfBhsmY$?jO6U(yyU{ojfzypIn+;kvunfLGq&HrO7Lj zS0}%iyf%45^5*1i$*(1EPu`ilCwYJJ!Q^+64<#Q-KAwCk`NQPX$!C)tt>in&-zWc+{A=S4*y$Tqn6+ zGE64PJh?$~qvWQ^Es|R&w@dDre1CG6UI2w({qseGGnvLe8m2rPO z7!Svz@pwELPsg+Ie7rL0PX?3WWHcF1CX?x8HknUWrv2$)I-HKCB_7> z8_b5Y(QG`M%%-#1Y(86=_veH8a6X!k=acz#KAX?yD=Yn#!Ae%Yl0~m%wv{|I==TTx z!5~|ijmkD;m03d;nE5k#_gseIU@#gC#)H9RFqjSov%z3K7_1D2{qC7zUg2mk91n(* z!Eibl&IZHzV7M|E^}Cry!@(#ocRUzP2BYa z7G8?SS2P-SXe!qLg{o$}b8urJ-{$$vn4*RoVe?IJI zp1g^9Kk|0uHRpw8>$8d3j;w4Y^K|c1H;jh)Fyzyak3&8W`9KU;hNFJ>7Ur2zJ{aTS zXfhm4hojkWG#`#uhU0$s?q;TOK1cbKOorp>a6B81=fm;JaMJJI_$+TS8cy;t%SUE9 zoXm!k`Ear_oc6oFNmex-4X5MbG@rtJerChzd^lYh&idV-D_b!e4QJ!wY%-kXLz)lM zd^lSf&imb8G8;G_4d>(Gd@`I*hx2@f^U2Bwsh>Y`wmqMy;mUZpG8wK+hbyz;N{mH059rb6U{(RKWo10HtJ{@_{`EcYDkUw+&midF_uad>({mq-0 z_akpdUUObpwmzGf?O4g;x)0Qm9jM!dvJYh^%3hS+XtXjK_q)$p-oiXH&i*tRji;mW zY&4#a#w(*qzxx>G-OWss?0(s~rlZMhG?|YkE2C+@`-J9=&+?|@(KLH!_O#h(Iv-6} zMzenR;mzM9tD22Rv&m?doj1GPd^B4b&HLTwI)AQg#e6)PPe$|UXr6sL`(Sptem?T~ zOJ)PJ`;As6qm}7sWj0#LuAhCs-|Y?gBWK(D<8gm7?oY@4*|?+wG@}bWsJ0IV?@A)9+)0R(1UUWVj`2^(8oWEuMU|Cb%`nyA6!ktc_q9D8!`8Lx~d{cfksXDx4Go|)t@G#yW7wHSs73J-Ts=7 zVcy-$G|f>f$E4YKIv-D0#Hli?y zthDIPNsI2hI=?PRi|(Sd=q^c%?&@jLT{A7ZYo|qb-L&YgmloY1ExJ)!bd$8`W@*u_ zq(ygwwCHY_7Tt~OCUw)a=x&}C-7V9iyLDQ0w@r)g_G!`GF)g|~r$u*{wCH{yExNm> zMR(7%=>$T6Fu_0)1rHET68Z>i|*xV(Y-P)x>wh0>KE!4)2937dTm;Dudg?xUH7JX zb6R$9t+%CZ_p9}5Y2E!sy*=%_cho!6!h3hUCvCj<)%(-R`>pz5+Ia`-chb`Pz4}nv zdWY&GY3+TiKA!g8C+kyb@%=&lVcL9uT%S&>?=$t;wEI3^Ur5XEOZDZn{r;>Dr}g(2 z^_OY?{dIjMEx@nU*V6|4W_>HIz`v>Qq#gKo_4jEB{$u@9+JgU5|C-j|zt?}HJ@{Xh zk5FrELtAKzZK-W+o7(2Kr7gFut=G1-?QKWf*><&4+No`KJFV?$r? zY3H`{+WGB*c451yUED5dS8G>q*J#&l*J{^p*J;;n-`B3!`fbpLZPdnX(xz?J=53{2 zzulniYd35+YBz2-X*X>*Yd3GVXt!*)YPW8;X}4{+YqxKAXm@OPYIknm-|o`x+J2zj zt=+xdqusOJtKGZZr`@;Ruid{rp#5NbVEdu=p!UP zr`w;j&$Q3B&$Z9DFSIYVFSReXKW%^34!1vVf6@N3{Z;$x_LcV4_OFH5)xD~FRrjjC zU46UycJ=N0vFOL5AB%o0`m5=$roWp0YWk(>m#SZ?eyRGO>wm8Qx&G(AXTtYP_?`*h zC*%8Me4mW(lkvS#zBkJEM)}?--w)^e;e0=w?}zg}kiG}f_dxm{NZQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^ ze$LOg6+h?a{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^ ze$LPN`Lm0k^K*XA&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lpy}QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC zoS*Y^e$LPNIX^$A_&Go4=lqQdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHaxUse2^pYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQdKj-KCoS*aa*ED|4&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lq3WGSg(6*VX^k+=FnPOpJF-pd$pJ; zmh;HIWM^_XCyVb68@@a8-`pNNww$VNi6_q2P2OrlEoA9S*@{hB$Cj*RYu4vD$1AMt|h<1M>h;v?sn=7lr(RFJo_am? zdg}Gm>#5gMucuy5y`FkK^?K^{)a$9&Q?I99PraTx8Gg>s`8hx5=lqQdKj-KCoS*Y^e*UYC zpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^ ze*S^t=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hwoapULwoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKmTCk=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hwoWASr-&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQd zKj-KCoS*Y^e$LPD-1s>^=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqQdKj-KC zoS*Y^e$LPNIX~y;{G6ZPzxX*n=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=lqQdKj-H^*!Vd==jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6XZy7)Oi=jZ&KpYwBm&d>QdKj-KCoS*aaE2E#+ zkzp;vYQ}aSy<5H7QyXikHfOBYJ+`n|dvkMWt*uY7ocq06%oNLcWM8r~Ih>2a?+zQj zJM!P$9z3?3s&0uV&eu)eYC|n#=}Xy)Ou zpYN~LI#8F^Wp#O7QO@Vi=g#NO=g#NO=g#Nq@73R{zgK^+{$BmP`g`^F>hIOxtG`!& zul`>Bz509g_v-JzUFz@E->bh@f1gsT*4l=)&=%WL+t@a>&239tZd+TgZEM>bSL142 zjjM4ruEy248du|LT#c)7HLk|hxEfdEYFv%0aW$^S)wtSY8&~6MT#c*wUMt@-pQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hxTPUGkNoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=lqQdKYv>BbAHax`8hx5 z=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZvSMhUx&d>Qd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hxD6+h?a z{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN`4z>_ z`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAG

QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC zoS&av{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN zIX~y;uPlDf&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqSzs~bP(=lqQdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`T2EQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqUC@zImU4jx-hRky?w=j$eJnR_8iU&>Z&$~v}WEnBm`ZMD62WKFy3 zlsYwAds?>h^lZr)b!MGajzf+^jzd?@~Tyx?FX+>T=cPs>@X; z!_WCSKj-KCoS*Y^e$LOoukmw!&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hw|SNxox^K*XA&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=kG6m&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj^WSRxoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQdKj-InD1Oe*`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj^G_5%=jZ&KpYwBm&d>QdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN`KKB`=jZ&KpYwBm&d>QdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC{7c2p`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAJAS#?SdVKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=luLp#n1UUKj-KC zoS*Y^e$LPNIX~y;{G6X(IsLqj3~L!yGq(Ha-Rjkz+E`1qIb*%;kEUrbF<<2=t*M-k1eOFTjGiHbrZMDy^y6ZWh*vi9b2-Nty$l; z+Fm=drd@SPotmvZE!%l|w&aXDv(75VA;%%dp)2Qc$bP=RR_j1rT9?)3rCp(2p~Tyx?FX+>T=cPs>@ZEt1kCjr7l-puDV=x zx$1J&<*Jk6=lqQdKj-KCoS*Y^e$LPNIX~y;k7@jz zpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax zpIZE!pYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN zIX~y;PjCF3pYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKi}5)IX~y;{G6ZjbAHax`8hx5=lqQd zKj-KCoS*Y^e$LOI-S{~_=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN`8kcB^K*XA&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=luLtji2*#e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC{MyCO`8hx5=lqQd zKj-KCoS*Y^e$LPNIX~y;{G6X(S^d0@3~L!yGq(Ha-Rjkz+E`1qIb*%;kEUrbF<<2=t*M-k1eOFTjGiHbrZMDy^y6ZWh*vi9b2*% z*Bh=kTyMDEaJ}Jr!}Z3M^?Jkke1EOhfx5IVE9Z0PbLVsCbLVsCbLVsQ_irin_v-J} z->bh@f3N;t{k{5o_4n%U)!(bXSAVbmUj4oLd-eC~@73R{zgK^+{$4#CKj-KCoS*Y^ ze$LPNIX}N{QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC zoS*Y^e$LPNIX}Naehu=5b)&j*-K1_>H>;c1E$WtatGadFrfyretJ@b><7!-ut8q21 z#?`nQSL142jjM4ruEy248du|LT#c)7HLk|hxEfdEYFv%0y|463)h|`QRDRCS`8hx5 z=lqQdKj-KCoS*Y^e$LPNIX~y;H*EZzpYwBm&d>Qd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;e*gUA#n1UU zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAJBG z#?SdVKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN zIY0km@pFF8&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQd zKj-KCoS%QW@pFF8&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqbh@f3N;t{k{5o_4n%U)!(bXSAVbmUj4oLd-eB^ZtCyV->bh@ zf3N;tJsdyh=lqQdKj-KCoS*Y^e$LPNIX~y;-){Vz zpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*aa z|0sUW&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC zoS*aa<>Kf3oS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKfkQ_IX~y;{G6ZjbAHax`8hx5=lqQdKj-KC zoS*Y^e$LN(ji2*#e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC{EEiU`8hx5=lqQdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAEni@pFF8&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=lqQd zKj-KCoS*Y^e$LPNIX~y;uWbCBpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqQdKj-KCoS*aaHy1zW=lqQdKj-KC zoS*Y^e$LPNIX}PZ`gt80)-tSSZ1>T-)vGbz; zv7ATtB|DSDYwJ(vX2bE(lg17nTTWHC#1rT1CT^K~AxmG%R&2^Twqz|^v%YP$y>?_x zyXurWHCuaHw)6CC$r;7>`99z0`+T48U#IbXzR&mhKHum2e4p?0eZJ54`99z0`+T48 z^L@V0_xV2G=W2YP?_YI&pP%oq)jCj@mi^p*Za-HiqfSPhj5-;0GU{a1$-K3xlTjz5 zPDY)KIvI5`>SWZ(sFP7AqfSPhj5-;0GU{a1$*7Z2C!QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQd zKj-KCoS*Y^e!j2qbAHax`8hx5=lqQdKj-KCoS*Y^ ze$LPNIX~y;{G6Y^zwvW^&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LN7R{Wfw^K*XA&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=XYrQoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-J4X#AX?^K*XA&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=l3ao&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqQdKj-KCoS*Y^e$LPN`TZI{=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN`6G&-^K*XA&-pn&=jZ&KpYwBm&d>Qd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=luK|#n1UUKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax|Nr&#Ix?(fSk2h(qj#%Udun4X)#i-#y2ln4Yj182t+n+jmUF*XiDiJq%Jqip4c8m4H(YPH-f+F~|L^sN^ZEW-tpjyw`F}$HPw4*%UAMb# zcirx~-F3U`cGvB$+x4^2&qhBR{cQBJ(a%Od8~tqbv(e8+KO6mQ^s~{==21;Q8~tqb zv(e8+KO6OM{G6ZjbAHax`8hx5=lqQdKj-KCoS%QI z@pFF8&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqk7_&Go4=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHaxpW662Kj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqSzrNz(rIX~y;{G6ZjbAHax`8hx5=lqQd zKj-KCoS*Y^e!jKwbAHax`8hx5=lqQdKj-KCoS*Y^ ze$LPNIX~y;{G6X(-uO8`=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN`B{yh^K*XA&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=luK?ji2*#e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC{F=qj`8hx5=lqQd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAJA&;^+LFpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=T~JvuOq`+hSiMiK6#5gMuXpXHUQfNAdOh`e>h;v?sn=7lr(RFJo_am? zdg}Gm>#5gMucuy5y`FkK^?K^{)XDI3e$LPNIX~y;{G6ZjbAJAo#?SdVKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=lqQd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*aa8#I2-&-pn& z=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*aa+ZI3P z=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`T0kS zpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y; z{QTpMpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC zoS*Y^e*XF5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hwoZ{z3uoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKmTIm=lqQdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hv-c=2<7&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lq9jtpxVRx`Hy=-uko zp4wPTwK-$G?y-f%+MAn0Yi)gs<=pSpVy0NmBm0t_$>Fv2r*pI6_~=Pv2aheMs$1fT z^K}!q%)OAMFJ&t>WgV_JTyMDEaJ}Jr!}W&ijVtr@hV%LUTCD@+eC~YieC~YieC~Yi ze6Idp{k{5o_4n%U)!(bXSAVbmUj4oLd-eC~@73R{zgK^+{$Bn4Bb)kr_4n%U)!(bX zR}aU}`8hx5=lqQdKj-KCoS*Y^e$LPN`8OLs=jZ&K zpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC{NIY7 z^K*XA&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=luLB zji2*#e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC zd~@-0e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC zoSz>ke$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC zoS*aa<;KtXIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC zoS*Y^etucw=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hw|Tl}1#^K*XA&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=PxaO&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQd zKj-KCoS*Y^e$LPN`O6zW=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqQdKj-KC{EfxW`8hx5=lqQdKj-KC zoS*Y^e$LOY+J0U~hP4c<8QXpIZuM$UZLFo*oUvZ_*urA%&CQ{;wm!vj?)PdjQ!M9^ zeaX(`@Y?#*x!G`h^rW$a$CgvoE%C(px`|unUdYmyvK5=MjxAZs)~s(^ZLb|!)2=$D zPR-VymhC(}TXII7S!dPRbxxgI=hgXjL0woE)x~v5U9GNO+>iTRt8qW>$Njh;_v3!t zkNa^y?#KPOANS*a+>iTlKkmop6eSWZ(sFP7AqfSPhj5-;0GU{a1$*7Z2C!QdKj-KCoS)B&pYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqQdKj-KCoS*Y^et!MN&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^etw(c=lqQd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hxTNbz%i&d>QdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6YEtnqVx&d>QdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LN7SNxox^K*XA&-pn&=jZ&K zpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=l5y+oS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqQdKj-IPX#AX?^K*XA&-pn& z=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=MO7>&d>QdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqf*Yj zu2xqs*Bh=kTyMDEaJ}Jr!}Z4h?)8TA`Tklh9>4>5fa`W1zyo-I_PO@C_PO@C_PO@C z_PO@C{&f1&=})IWo&I$C)9FvAKb`(``qMq4=})IWo&I$C)9FvA9*&>$bAHax`8hx5 z=lqQdKj-KCoS*aaZ!~_+&-pn&=jZ&KpYwBm&d>Qd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqSzzZO5|=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`T3I@Kj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=lqSzrsC)PoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=lqQdKVL0=&d>QdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC{L;qH`8hx5 z=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAEn$@pFF8 z&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y; zFKhgqpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC zoS*aaHxxhT=lqQdKj-KCoS*Y^e$LPNIX}Ou`*|H1 z)-tSSZ1>T-)vGbz;v7ATtB|DSDYwJ(vX2bE( zlg17nTTWHC#1rT1CT^K~AxmG%RyYnh4ml28)sI8=^ZjK%x1Zb3)$6I(Q?I99PraUc zy=yl0dg}Gm>#5gMucuy5y`FkK^?K^{)a$9&Q?I99PraUcJ@tC(_0;RB*Hf>jPKKZJ zbAHax`8hx5=lqQdKj-KCoS*aaS@Cmz&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQdKj-KCoS*Y^e$LNV8b9af{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN`Fk2a=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqQdKj-KC{7~_8e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS)ye@pFF8&-pn&=jZ&KpYwBm&d>QdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS)yb_&Go4=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHaxKU@5qpYwBm&d>QdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;pKtt}pYwBm&d>QdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*aauNFV&=lq`@LGs z6w7&JU$Qefyte*yZZ;erJ!$OVvE@{COFVJDZsL}?7qaxFY=!F$*Bh=kTyMDEaJ}Jr zBz509g_v-J}->bh@f3N;t{k{5o z_4n%U)!(bXe|S@Wul`>Bz509g_v+#JIX~y;{G6ZjbAHax`8hx5=lqQdKmU5;=lqQdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hv-Lh*Bc&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQd zKj-KCoS*Y^e$LPNIX~y;{G6XZsqu4u&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqQdKj-KCoSz?P{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;dy1d)bAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{QSkm&-pn&=jZ&KpYwBm&d>QdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6X(t@t@V=jZ&KpYwBm&d>Qd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lq%^B-; zk1Z_L-rO8oYwJ@i=YFpiGsSWq*_Z504zI00otq8EM^73%cx*XU-4ainuba4K?u9IU zDO<5A>)4XDY|Z+%)%MzvHSMZX>eOuQY1z)xvn6NLnRQm3UFX!fbzYrc7u1DyQC(b@ z6!+tP*J#|2`*A<+$Njh;_v3!tkNa^y?#KPOANS*a+>iTlKkmo<{C_+5<9=6t_p_hd z&+X^-bAGOWy#Dd}$E%Z3C!SWZ(sFP7AqfSPhj5-;0GU{a1$*7Z2 zC!QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=kF|j&d>QdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX}Nu@pFF8&-pn&=jZ&K zpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHaxKhpR)Kj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;pDBLM z&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*aa z&ozF|&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqSzSBjtWbAHax`8hx5=lqSz|GuBskzp;vYQ}aS zy<5H7QyXikHfOBYJ+`n|dvkMWt*uY7ocq06%oNLcWM8r~IlQ+1bZ#~rA3bU8;IZXY zbxS;PzHZ`{xfinZrEJBftYb^ovNh}5R@-Yw*0ifmsZ+DHr)4`&&z77~XVzJDcAZn_ z)_HY)T~HU+MRjpqQm!{#Z@AuYz2SPp^@i(>|NZL?=W`yw19*Vzb{@b3c!2h~_PO@C z_PO@C_PO@C_PPFa`qSx8r$3$kbo$fjPp3bf{&f1&J*??Zr$3$kbo$fjPsh*sIX~y; z{G6ZjbAHax`8hx5=lqQdKmS_e=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hv-eDQOB&d>QdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6XZvGH?$&d>Qd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCd{g7+ z{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX_=* z{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y; zrxicv=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`T2{ApYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC zoS*Y^e*Tii&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqSz z_ky3-kzp;vYQ}aSy<5H7QyXikHfOBYJ+`n|dvkMWt*uY7ocq06%oNLcWM8r~IlQ+1 zbZ#~rA3bU8;IZXYbxS;PzHZ`{xfil@$05fd$D!|qjz=BO)tfpVbv){L)bXg}QOBc>M;(tk9(6qGc+~Nz<59<> zjz=AjIv#aA>Uh-gsN>=1{G6ZjbAHax`8hx5=lqQdKj-KCoS#pMpYwBm&d>QdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^em-mboS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqQdKj-J~YW$p^^K*XA&-pn& z=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=eI0=&d>QdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN`K=p2=jZ&K zpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIY0lC z;^+LFpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN zIY0kwQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqS7{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^ ze$LPNIY0mF;^+LFpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;R|Y??Bg0yT)r{>vdbfJD zr#9A7ZO&M)du(B`_U7i$T3erDIrn?Dm?@U?$i8G}a(He1>D+8MK6=vF!DGv*>Xvxo zeBHz?b1!7++7;Rr+7;Rr+7;Rr+7;Rr+7;Rr+7(xZcE!fpl(xZ^v<|kWov^L8*N(Ih zcGW3$YFZi2=g#NO=g#NO=g#NO=g#N)+307ZpN)Pt`q}7bqo0j_Hu~AfzMG@pFF8&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^ ze$LPNIX~y;{QRqppYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e*RCz&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=lqQdKj-KCoS*Y^e!f`zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-H^SNxox^K*XA&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=NlV8=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax_cVUa&-pn&=jZ&KpYwBm&d>QdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*aai;JK0bAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{QR}W&-pn&=jZ&KpYwBm&d>Qd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHaxzc>86jtpxVRx`Hy=-ukop4wPTwK-$G?y-f% z+MAn0Yi)gs<=pSpVy0NmBm0t_$>Fv2r*pI6_~=Pv2aheMs$1fT^K}!q%)OAMFJ&t> zWgT0xmaSRew%T4hvZh^iN}ZamJuTaLdbZ?@Iw>zlE-LQF{jS!y zANS*a+>iTlKkmoQdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`MG|n{G6ZjbAHax`8huy7eD9c{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN`EM3K=jZ&KpYwBm&d>QdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS)yK_&Go4=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHaxf3NsCKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHaxKiv2^Kj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;pDupR z&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*aa z&oq9{&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqSzUll*+=lqQdKj-KCoS*Y^e$LPNIY0lO@$)(| ztYuiu*zTiut5cYCHv@5hLv@5hLv@5hLv@5hLv@5hLv@8B++7+(b_t$D2s7vdz zy1cHaXV-J;x%Iqye!ZYxSlo~MaX;;I?Q`vO?Q`vO?Q`vO?Q`vO?Q`vO?Q{KR^_$gi zR=-*OX7!tWNYigtzghie^_$giRy~}4v--{IH>>WIpYwBm&d>QdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hxTO5^AJoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKmW<%=lqQdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hxT$KvPwoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKYx7V=lqQdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hvdD1Oe*`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj^Jf-6=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX^$O_&Go4=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHaxUs(K{pYwBm&d>QdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*aa7d3v)&-pn&=jZ&KpYwBm z&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lq}pYwBm&d>QdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS%QM_<0=})-tSSZ1>T-)vGbz;v7ATtB|DSDYwJ(vX2bE(lg17nTTWHC#1rT1CT^MA zamaDVap-&HIJB`gjz=AjIv#aAmo#-e>Uh-g zsN+$`qmD-%k2)T8JnDGV@u=fb$D@u%9gjL5bv){L)bXg}QOCp2`8hx5=lqQdKj-KCoS*Y^ ze$LPNIX@p2Kj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqSz+l!y`bAHax`8hx5=lqQdKj-KCoS*Y^ ze$LPNIX~y;{Cv{*IX~y;{G6ZjbAHax`8hx5=lqQd zKj-KCoS*Y^e$LO|(fBz(=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lq!ak!)!h zfZyZpc4(J9o*9qFbK~v;ONJ$gXmZZG1d%g{oI&KcAh$i?-d#YjOHP7?T{0pk6J+9& zMFx{Z&OgY-S9Y&$$^1qWo{~y+s`^y;al7jrKkqk}pYwBm&d>QdKj-KCoS*Y^e$LPN zIX~y;KOKI~&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQdKj-KCoS*Y^ ze$LPNIX~y;Kil{@Kj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC zoS*Y^e$LPNIX~y;zY%`U&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=lqQd zKj-KCoS*Y^e$LPNIX~y;zt#9TKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;e-?hu&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^etxg` zd5)L9mRYqnPj=H~D~p+Axmw+DY&6bxZ4Uc3?=h|YF5~Jkts^_NZq$0z&p)os!dde| z>)`Y{$#BK<=M9}(oo!xWUSVEgUSVEgUSVEgUSVEgUSVEwugohJvs80~m6{!_);wV? z>$y}jgv+@S?{n{S?{n{S?{n{S?{n{SbvEj3)Y+)BQD>vhMxBj18+A77Y}DDPvr%WG z&gMg#IvaI1>TJ~6sI!rWlZTUslZWHy{G6ZjbAHax`8hx5=lqSzUo?Ks&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqSzzlNXlbAHax`8hx5=lqQdKj-KC zoS*Y^e$LPNIX~y;{QOfIKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqSzLijm9=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=lqQd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAEm${G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;UmSkU&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{QPO*=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hxThVXNK&d>QdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=lqTUpE`%hl?JW214lYjfDQ zd5>xBcNtfYX&u?Ab)(j!e*ST77S5U%S_h}sNro$)KX2&V>g-WneNs=cRQFh^yR6pz z*0P>Ub*IbW`+T48^L@V0_g}j4eZJ54`99z0`+T48^L@V0_xV2G=lgu0@AG}W&-eL0 z-{)$4pYK07zR%BhGs|A~b2GPcJ9qL#p3HOd-0*XL&d>QdKj-J))c83+=jZ&KpYwBm z&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax)t~coe$LPNIX~y;{QT9!&-pn&=jZ&K zpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqlp4oS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKYz`}&-pn& z=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lq< zbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{QOPB z&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQd zKY#Pa&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN zIX~y;{QRB6&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lq*448oS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKY!Q8&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqQdKj-KC zoS*Y^e$LPNIX~y;{QQH$&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=lqM^Y&JGE}qdeqN9uFb+( z^Fr(3^g79K#q;M4om-tfs;f`xDVFLUD|MIEy5CyXbE)ohIp!7S73LM@73LM@73LM@ z73LM@73LLp&%DC8eK)i0Wj{A_E4OnePvprwC(n&>yK%d5yK%d5yK%d5yK%d5yK%d5 zyLvkHbn5BU)2XLZPp6(vJ)L?w^>iQ7)YGY_Q%|R!PCcDGoO(L-bn5BkUimpc=jZ&K zpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6Zj^FMF=oS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqQdKj-J47=F&r`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj^M46H=jZ&KpYwBm&d>QdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj^A>*2&-pn&=jZ&KpYwBm z&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqSzsPS`t&d>QdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hwo9DdHv`8hx5 z=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj^XG=2^K*XA z&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=luL?QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAJBR z@N<67&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lq@)VfjYQ9u8mQdKj-KCoS*Y^e$LPNIX~y; z{G6Zj^H&Q$=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC{52Xs=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQdKj-KCoS*Y^ ze$LPNIX~y;{G6Zj^EU}U=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqQdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax->mU-e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*-6QdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=lq{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;@6z}=Kj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqQdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqiO{G6ZjbAHax`8hx5=lq2;Ffis#Q8I-6IRSD06rSD06rSD06rSD06rSD07aOY@4wEY;j# zrDg}KHBVT}dM?!r!Ta3%-22@7-22@7-22@7T%CTJ~6sI&Rtrp`v4jXE23HtKBT;pE}u;pE}?IX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=luK=!q53RKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIY0lT#?SdVKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=luNh!_WCSKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqtNKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHaxH^R^PIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e*T>BbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6X(Y5bg@^K*XA&-pn&=jZ&KpYwBm z&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=U?3TIX~y;{G6ZjbAHax`8hx5 z=lqQdKj-KCoS*Y^e$LOI5`NCl`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj^REj(=jZ&KpYwBm&d>Qd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hv-u>3s7OJB>ZTAL@k>9Uo@OtM_9Za6j? zXS+6seVg}~)_#|9^_bR?omw|)J?iHl*Jk0Yd7*W1dYxpr;`#H2&aKWK)zv5U6iaoF zmAcDn-ES@Hxm0($oGZCnPkXJN^LjnWjXacx^GF`eV|hH>kNZ8XaX;?I{kR|Z<9^(a z`*A<+$Njh;_v3!tkNa^y?#KPOANS*a+>iS`SnkKwb~DRf_H#40ayxhOM4pVgS#`7O zX8Ack=jY$h_&Go4=lqQdKj-KCoS*Y^e$LO;rSfxr z&d>QdKj-KC{2Afr{G6ZjbAHax`8hx5=lqQdKj-KC zoS*Y^e$LPNIY0ld@N<67&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=lqQd zKj-KCoS*Y^e$LPNIY0lN#?SdVKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqI_w{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIY0l&@N<67&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIY0lY#?SdVKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lq<=me$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*+{_&Go4=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax-?{N~e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQdKj-KCoS*-CQdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoSz?spYwBm&d>QdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=lqM54$9xHX1 z)w7er|qler|qler|qler|qler|qler|rQ zZdTo_x>Soo=eo#|4t8P}^th!lsv+8Em&8nMKH!Jtb&-pn&=jZ&KpYwBm&d>Qd zKj-KCoS*Y^e$LPNIX~y;{G6ZvY2)YooS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKmYjfbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZvQ}{VQ=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZvOXKJKoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqQdKmWY&bAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Y+#?SdVKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=luLq_&Go4=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHaxpA0|e=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`T6C>&-pn&=jZ&KpYwBm z&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e*Sp)IX~y;{G6Zj zbAHax`8hx5=lqQdKj-KCoS*Y^e$LOoHvF8Q^K*XA z&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{QN=l^BgaIEwgHEp6sT}Ru(hK za<#hQ*l3*X+8p+6-eX$(UB=a8T1R$j-Kh1bpMPANg|p^`*1_p@lHrQy&l@_oI(t-C zpVU(<)jd|~E~|CFwXBEl^L@V0_xV2Ge`@3Ve4p?0eZJ54`99z0`+T48^L@V0_xV2G z=lgu0@AG}W&-eL0-+$12pP%n$mc8ufW^Uzn?&OK^bAHax`8hx5=U?CWIX~y;{G6Zj zbAHax`8hx5=lqQdKj-KCoS*Y^e*T@|=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hv-wZ_l+ zIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e*WE! zpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^ ze*P2T=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hv-lg7{aIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC zoS*Y^etyvSIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC zoS*Y^e$LO|Dg2zD^K*XA&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=f4tu&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^ ze$LPNIX~y;{G6Zj^IvQHoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-J~AAZiy`8hx5=lqQdKj-KC zoS*Y^e$LPNIX~y;{G6Zj^FIzh=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6Zj z^SjN@bG-Dm%&N6{vYRejSK9oeaMqt>H-{&8&< z&YBlm2dCFbhAW;wZ|L0W>``5PQctl|_gJaBtk(V3vL5pa^9u6{^9u6{^9u6{^9u6{ z^9u8dyKP=!+`gMx_OhRwxs}_wlP6-_ZrpC%ZrpC%ZrpC%ZrpC%ZrpC%uAWXkoq9U; zbn5BU)2XLZPp6(vJ>3U3^>pg#)YGY_Q%|R!PCcD^I`wpNul$^!^K*XA&-pn&=jZ&K zpYwBm&d>QdKj-KCoS*Y^e$LPN`JXg?&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqQdKj-KCoS*Y^e$LPN`9FrA^K*XA&-pn&=jZ&KpYwBm&d>QdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=luK=8b9af{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN`9C*)&d>QdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNc^W_G=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`T6HJe$LPNIX~y;{G6ZjbAHax z`8hx5=lqQdKj-KCoS*aa_3(3k&d>QdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAJ9<_&Go4=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHaxzb5>gpYwBm z&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=luM^^Ya`peJ!(UZJz9=%T^XM z$#S*2;n--L?b;mnZQf&A`(4J>V_HXcYTc;ysGoman}xIHh1S98b&}zV=g%8Dw>o=N zSD(~VEY&?$>MpBwzqRmvzR&mhKHum2PicIg@AG}W&-eL0-{<>$pYQX1zR&mhKHum2 ze4p?0eZJ54`99z0`wyP)^Yh)zvX}kb%&pwco$zyh&d>QdKj-IP*Z4U<=jZ&KpYwBm z&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax)t~coe$LPNIX~y;{QOnI&-pn&=jZ&K zpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKYvE!=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hxTuEx*# zIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e*T8x z=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hxT z@$hqg&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hxT$;Qw5IX~y;{G6ZjbAHax`8hx5=lqQdKj-KC zoS*Y^e*TW(=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hv-ariku=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQd zKj-KCoS*Y^e$LN-wefR)&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPTDEyqC^K*XA&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPD zK0nX#($_Mp*5=7>x@=`JlPp)O8;*^}*{;oD-{w80wcll2J*IVJr`C;HkNWw?wOKf8 zUT7VhUMCr@c>cVhbE~sQb@fR-#Zuj4rS7s?_gjm3g?WW}g?WW}g?WW}g?WW}g?WW} z#oafrFmB(?EPL6{&D_fE+=+3!al3K5al3K5al3K5al3K5al3K5dOG!V>gm+esi#v< zr=Ctdoq9U;bRW>v)2XLZPp6(vJ)L?w^>pg#)YHkm@^gO9&-pn&=jZ&KpYwBm&d>Qd zKj-KCoS*Y^e$LPNIX~y;M~$EJbAHax`8hx5=lqQd zKj-KCoS*Y^e$LPNIX~y;{QP6X&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqzhoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKmYi~&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{QPsm&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC{PP+==jZ&KpYwBm&d>QdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax?}VT8bAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{QOem=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hv-vhj0%&d>QdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKYtMYJjY94%dA?PC%fsgmBmc5T&-?6 zHX3KUHivzi_n6jxmvQx&){&iBH)=hq|A=0)EWD)ge{2sOc`|*9g&-eL0-{<>}H@?sJ`99z0`+T48^L@V0_xV2G=lgu0@AG}W&-eL0 z-{<>$pYQYi2hsQW`EF*}%YJU=R&Ix%^K*XA&-pn&|JugS`8hx5=lqQdKj-KCoS*Y^ey;wUpYwBm&d>QdKj-JK9DdHv`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj^KTD7=jZ&KpYwBm&d>Qd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj^Y3i@oS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=lqQdKj-If5Pr_j`8hx5 z=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj^B)U8=jZ&K zpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj^Pgz^ zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-J~ z5Pr_j`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj z^Ir}>=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y; z{G6Zj^IvKFoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-J~8-C8u`8hx5=lqQdKj-KCoS*Y^e$LPN zIX~y;{G6Zj^FItf=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6Zj^SjZ{bG-Dm z%&N6{vYRejSK9oeaMqt>IE{JmsZcuC{`*dCl- zCmF7I{=A`k%R)wV^+`R&Qr%;v?y_3pg#)YGY_Q%|R!PCcD^I`wqw z>E6Gor&CX-o=!cTdOG!V>gg&{*;-p@qc(1nw%C^1a$9NBw%WS3*47(W<7!-ut8q21 z#?`nQSL142jjM4ruEy248du|LT#c)7wLflLjjM4ruEy248dvl0RQmU1)TQ!se$LPN zIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*aazYjm>=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`T55+e$LPNIX~y;{G6ZjbAHax z`8hx5=lqQdKj-KCoS*aae{B4mpYwBm&d>QdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*aa{|P_m=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`T6HIe$LPNIX~y; z{G6ZjbAHax`8hx5=lqQdKj-KCoS*aaF8rLI^K*XA z&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=eNVp`8hx5 z=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAGQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHaxpJ@D? zpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC{QmUw z94~z>vubUg?54|B7Bk6mwYuThXq@fZ9QJMAV_N%N#?@n5M|Nu6sP(9ye_We|v*v}? z!Rd99;fm+a8#=c-dsJ7S)Ke_gJyz;2t98G%tmjhQ>2j{*YCY|>dd}$pYQX1zR&mhKHum2e4p?0eZJ54`99z0`+T48^L@V0)%ZT&zdwDS zpYLXtz3k^^ZpHW9_uThfPDV~fPUdTxoQ#}|oQ#}|oQ#}|oQ#}|oQ#}|oQ#}|oQ#}| zoQ#}|oQ#}|oQ#}|oQ#}|oQ#}|oD4tb=lqQdKj-H!3P0!P{G6ZjbAHax`8hx5 z=lqQdKj-KCoS*Y^e$LPN`KvU3&d>QdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=R4u&{G6ZjbAHax z`8hx5=lqQdKj-KCoS*Y^e$LPNIY0l=@N<67&-pn& z=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIY0mL#?SdV zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIY0lU z@N<67&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN zIX{1KQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQd zKj-KCoS*+e_&Go4=lqQdKj-KCoS)yFexBo{uVq%P z&6C}9*~(%jS*}($92XUkkrMkyT-DS1zx0dx>sykiIm0Yc-y;je8y`JPo9?HWp-Z0)U-Z0)U z-Z0)U-ncu*8{X%;nPo5gxtUw>cS3(B^mjtzcH?&AcH?&AcH?&AcH?$+HtKBD*{HKo zXQR$WosBvhbvEj3-mj^%QD>vhMxBj18+A77Y}DDPvyq46=lqQdKj-KCoS*Y^e$LPTsPS`t&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqQdKj-KCoS*Y^e$LPTF8rLI^K*XA&-pn&=jZ&KpYwBm&d>QdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=O5enIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPTq49Hm&d>QdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPTJN%rV^K*XA&-pn&=jZ&KpYwBm z&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=bzK~IX~y;{G6ZjbAHax`8hx5 z=lqQdKj-KCoS*Y^e$LNV!_WCSKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKYuvQdKj-IP8Gg>s`8hx5 z=lqQdKj-KCoS*Y^e$LPNIX~y;_p6`hcQdKj-KCoS*Y^ ze$LPNIX~y;FAP8D=lqQdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`S~k1e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*aaZ*TmZpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQdKj-KCoS*aa9|=F_=lqQdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`S}|(e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*aaA8Y)apYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=lqQdKj-KCoS*aaUkpFz=lqQd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`T09Ee$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*aaUvB)IpYwBm&d>QdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*aa-w!|M=lqQdKfhP~JjY94%dA?PC%fsgmBmc5T&-?6HX3KUHivzi_n6jxmvQx&){&iB zH)=iV=O5Q*;jDR~b#Qu}WVqt_^M=l?&K}j(C-oFdb&r+0i}8l>hVh2+hVh2+hVjO| zI^OU;-_0z0+0V^*pL?HspL?HspL?HspUdCN-^<_2-^<_2-^<_2-^<_2-^<_2-^<_2 z-@k8@zn8z4zn8z4zn8z4zn8z4zn6#O=lqQdKj-KC zoS*Y^e$LPTu<>(#&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQd zKj-KCoS*Y^e$LPTHvF8Q^K*XA&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=O5GfIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPTzVUN@&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQdKj-KCoS*Y^e$LPTEBu_F^K*XA&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=bzp9IX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LOQ;phCEpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZPZv32|^K*XA&-pn&=jZ&KpYwBm&d>Qd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=MROS^K*XA&-pn&=jZ&KpYwBm&d>Qd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=luLD!q53RKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lq2;Ffis#Q8I=4D|R9BzWQ!Ldz zR_ZRRb-%T&=ThD2a<1fRJ?*u6&g=CgH^TS%{v(a=^L@V0_xV2G=lgu0@AG}W&-eL0 z-{<>$pYQX1zR&mhKHum2e4p=gHNMaH?_b~N=ewC@FZ=O5_dWMLmy?l`k(2qVCMP2& zBPSy#BPSy#BPSy#BPSy#BPSy#BPSy#BPSy#BPSy#BPSy#BPSy#BPSy#BPYYp`8hx5 z=lqSzw}qecbAHax`8hx5=lqQdKj-KCoS*Y^ ze$LPNIX~y;{QQ*~Kj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqSziyA-Y=lqQdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`T6UGpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQdKj-KCoS*Y^e!kQAIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LN-wDEI(&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqQdKj-KCoS*Y^e$LN-A^e=5^K*XA&-pn&=jZ&KpYwBm&d>QdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=WpNmIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LN-squ4u&d>QdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LN-FZ`UJ^K*XA&-pn&=jZ&KpYwBm z&d>QdKj-KCoS*Y^e$LPDUO&(A($_Mp*5=7>x@=`JlPp)O8;*^}*{;oD-{w80wcll2 zJ*IVJr`C;HkNWw?wOKf8UT7VhUMCr@c>cVhbE~sQb@fR-#Zuj4rS7s?_gl+)F4dhb z=Sr^D(_X9Ryk1XoBgPxX8^#;P8^#;P8^#-V?|8%ed^fY~Wk3E-=osBvhbvEj3)Y+)BQD>vhMxD+3G<7!WY}DDPvr%WG&PJV$IvaI1 z@^Ji|pYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`S~9-e$LPNIX~y;{G6Zj zbAHax`8hx5=lqQdKj-KCoS*aaj|@NO=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`T5_3pYwBm&d>Qd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqcx=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`T5^9e$LPN zIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*aa&j>%~ z=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`T2i_ zpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lq6==lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`T0utIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^ ze$LPL!_WCSKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQd zKj-KCoS*aa8{y~toS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKYxDsIX~y;{G6ZjbAHax`8hx5=lqQd zKj-KCoS)y%exBo{uVq%P&6C}9*~(%jS*}($92XUkkrMidTA-_X@hwkU!A>Z@e%(55XbKi5{ zb9p^^y@#8;p1hvCp1hvCp1hvCp1hvCp1hvCp1hvCp1hvCp1hvCp1hvCp1hvCp1hvC zp1hvCo}3In=jZ&KpYwBm{*{fN^K*XA&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=luNT!_WCSKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQdKj-KCoS*Y^e$LPNIX{2J#?SdVKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAJB1;phCEpYwBm&d>QdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS(mbQdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAJ9d;phCEpYwBm&d>QdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS(m4Qd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAJAw;phCEpYwBm z&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqTUpE`%hl?JW214lYjfDQ zd5>xBcNtfYX&u?Ab)(j!e*ST77S5U%S_h}sNro$)KX2&V>g-WneNs=cRQE96Fy1iU zFy1iUFy1iUxR=Kp-siiSWiQ_6-sj%u-sj%u-sj%u^7r!h^7r!h^7r!h^7r!h^7r!h z^7r!h^7r!h@7?6@B}_Kj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=lqQd zKj-KCoS*Y^e$LPNIX~y;9}#}e&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqSzM>T%V&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;pB{eB&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=lqSzXEuJ$&-pn&=jZ&KpYwBm&d>QdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zfg`e|te$LPNIX~y;{G6ZjbAHax z`8hx5=lqQdKj-KCeA@UqKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqQdKj-KCoS*aa?eKGc&d>QdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=lqf*D|Zt=E-ilY-KT% zELW==j*Z6IuFYZJ<~^ph-(_4qrgdbe){R<^`uWGTSvYH6XdRqhCmF7I{=A`ctFuRS z^+`R&Qr%;v?y_3Qd zKj-KCoS*aamkU4V=lqQdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`S}aN&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqQdKj-KC zoS*Y^e$LPNIX~y;{QTP*Kj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqSz>x7^4bAHax`8hx5=lqQd zKj-KCoS*Y^e$LPNIX~y;{QL*Q&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{QQR-Kj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqSzTZfQdKj-KCoS*Y^e$LPNIX~y;{QT#_&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{QMUhKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=lqSzdxW3!bAHax`8hx5=lqQdKfk;EJjY94%dA?PC%fsgmBmc5T&-?6HX3KUHivzi_n6jx zmvQx&){&iBH)=iV=O5Q*;jDR~b#Qu}WVqt_^M=l?&K}j(C-oFdb&r+0%WBQdKj-KCoS*Y^e$LPNIX~y;{G6Y^ zSL5gWoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQd zKmWbP&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKmV)nbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN zIX~y;{G6YEWaH=joS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKmVJ?&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqQdKmYgebAHax`8hx5=lqQdKj-KC zoS*Y^e$LPNIX~y;{G6YEM&sxFoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKVJ$z=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;D~+G?bAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{CvOhbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6X(4?pMU{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN`Im>E^YeS#&vU%=waluud9s@>TUpE`%hl?J zW214UWs<|b&3jC1zstCKOzX%_tsAu-_4AKwvvAhD&^kE1PBL8a{CPv?R%egu>XUj3 z;|=2t;|=2t;|=4D8_js*-X3pwpYO)|-22@7-22@7-243b&HG&bUjAPGUjAPGUjAPG zUjAPGUjAPGUjAPGUjAPGUjAPGUjAPGUjAPGUjAPGULKC0^K*XA&-pn&=jSgQe$LPN zIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*aaZwWu= z=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`T5H? ze$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*aa zZ*BaXpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC zoS*aa9|%9^=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`T6TMe$LPNIX~y;{G6ZjbAHax`8hx5=lqQd zKj-KCoS*aaA8P!ZpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*aa!|-!{&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hxT`Nq%rIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e*W&^=lqQdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hxT?eKGc&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqQdKmW_{bAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6YEMC0fDoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqQdKmY5-&-pn&=jZ&KpYwBm&d>QdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKmWJzbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6YEdgJH(oS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqQdKVJ+#=jZ&KpYwBm&d>Qd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHaxmm5Fl=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`T1Vs=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hw|3P0!P{G6Zj zbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN`Im*C^K*XA z&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lp!T@pFF8 z&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqXUkkrMkyT-DS1zx0dx>sykiIm0Yc-y;jfZ|0Chw!I0y* z*MEn6&&?~$E6gjr&&?~$EAIVyh4;Dlx%avEx%avEx%avE`O~A$MxBj18+A77Y}DDP zvr%WG&PJV$IvaI1>TJ~6sIyULqs~U1jXE23HtKBD*{HMO=lqQdKj-KCoS*aa zmuvi-pYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN zIX~y;uN8jI&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqSz>ok7O&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqQdKj-KC zoS*Y^e$LPNIX~y;ZyA2h&-pn&|6lg*I=GT;yASYh>wu)AmPRw9(bZ*o&#ss;sl+m4 zVrD01Ok(DM8Fi$A3-8%Unc1$G*=4p&vSa2XCK&@}J4scjvsyY=Qk7nn{f}?Wz4z;? zuHM~e_wN2SzkB6W0bAHax`8hx5=lqQdKj-KCoS*Y^ ze$LPNIX~y;{QNB%Kj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqSzPc(kc&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=lqSzuZExVbAHax`8hx5=lqQd zKj-KCoS*Y^e$LPNIX~y;{QTV-Kj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqSze`x%ipYwBm&d>QdKj-H;+4|b%)!I5eO_%LVGs#M| zy5ZVroSixx^=;8(R_9&D)nisyc5C0P{kUIzT!*Fe=B3u***M8C;>G)h&aLhq)##IY zi{*O8YCUDGp0}QjT&X8r&9z*wx4lvCc~8B`y}2*<=Yc$!hw^Y9$)kBJkLN{sad-d^ z-~l{<2k-zMzyo*y58wejfCumZ9>4=WC_I1%@Bkjb19$)r5N}scr=Ctd9amFNr=Ctd z9Y5#i{G6ZjbAHax`8hx5=lqQdKj-KCoS*-*#?SdVKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqQdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC{C|d@^K*XA&-pn&=jZ&K zpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=luL%Hh#{}`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj^GWzQKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;%Z;D&bAHax z`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{CwW{IX~y; z{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LNtgrDQdKj-KC{By(4 z`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAEnL zQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=luM|;phCEpYwBm&d>QdKj-KCoS*;g`*}{bzP5R_woXseWjoVMvQn*XxHcMRrw&Jb zTlARKd6#kZnAMft+Ba)I?iU}|Vd=bisdacZPBM&m@xGyRtGh=vx;UXYp*W#9p*W#9 zp*W#9p*Z1JMVwHaP@GVlP@GVlP@GVlP@GVl@NYj(IL&g!`c^C2w^nh#^=#xyMf|Sj zTCP{r(E8l^-1^-5-1^-5{NQdKj-KCoS*Y^e$LPNIX~y;{QR}U&-pn&=jZ&KpYwBm z&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKmU7;pYwBm&d>Qd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`T3iMpYwBm z&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e*Wf- zpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^ ze$LN-CH$P9^K*XA&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=kMD1IX~y;{G6ZjbAHax`8hx5=lqQdKj-KC zoS*Y^e$LN-t?_ey&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQd zKj-KCoS*Y^e$LPTApD%4|Mh;JldZ39UahUu({$O+G?T1Us~fJ3#@VUEQQsClW_8|W zTs>xWWw-Xt+K>Ci$8}gbZ(eF0o{f_XBVN33=-le=QH{?1xF7f9e%z1yaX;?I{kR|Z z<9^(a`*A<+$Njh;_v3!tkNa^y?#KPO-#=~K@2_{iX_m|GR!i^J%JJ5-kt-#5S92}b zO992(#oNW(#oNW(#oNW(#oNW(#oNW(#oNW(#oNW(#oNW(#oNW(#oNW(#oNW(|8o;> z_dS)qr_%RSs#jC5re00G8b9af{G6ZjbAHax`8hx5=lqSz$AzEsbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN zIX~y;{QQ4}pYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqQd zKj-KCoS*Y^e$LN7EBu_F^K*XA&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=hMc|`8hx5=lqQdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAGQdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHaxx5Cf)IX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e*QV(=lqQdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hw|Zv32|^K*XA&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=l`bhbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Y^DEyqC^K*XA&-pn&=jZ&KpYwBm&d>QdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=U*9q&d>QdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=lq zYHgjKrptDwnPjC}-EeI*&Q2YU`nKpXtMe}7>M^S;yR~oDe%voUuEWxK^HS^ZY@B2m z@#1|$=T>)*YV=9H#d1AkwVtw8&s)z%uGEvR=31`T+uo@6yr++|-K@GQd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAJAvji2*#e$LPNIX~y;{G6Zj zbAHax`8hx5=lqQdKj-KC{7u5o`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAJA#;phCEpYwBm&d>Qd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAJBgji2*#e$LPN zIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC{GGzj`8hx5 z=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAJBI;phCE zpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAJA- zji2*#e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC z`~$+z`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAJB&;phCEpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAJAZji2*#e$LPNIX~y;FAP7=$=264uh!P-X}WA@nn_lw)eYB1Y`*A<+$Njh;_v3!tkNa^y z?#KPOANS*a+>iTlKkoOj;eOnY`-!)6KkoO!a6hiLmwEPckej)c+qsh`@?@UM(|Ji= z8g;YkX4TE|bAHax`8hx5=lqQdKmYIH=lq!bAHax`8hx5=lqQdKj-KCoS*aak8k{(pYwBm z&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqN{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN zIX}N1e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC zoS*aamxiD7bAHax`8hx5=lqQdKj-KCoS*Y^e$LPN zIX~y;{QO4a=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hxT+{VxOIX~y;{G6ZjbAHax`8hx5=lqQd zKj-KCoS*Y^e*Sp)IX~y;{G6ZjbAHax`8hx5=lqQd zKj-KCoS*Y^e$LOoBK(}6^K*XA&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=Pz#joS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-IP)%ZC-=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=lqQdKj-J)7Jkmp`8hx5=luNH@$;N)eQon-PwQtsb+%G<^!_s;4QtR++oMagB;(bHsR(FdNiW7_gl|K zu2jVDYOcll-1^-5-1^-5-1^-5-1^-5-1^-5{5zWUx%IjAx%IjAx$nX7Js7?R!}nnL z9t_`u;d?NA4~Fl-@I4s52Sa|l{C4^6^4s}2Kj-KCoS*Y^e$LPNIX~y;{QQl>&-pn& z=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKY!E4 z&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y; z{QMom&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKY!=O&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQdKj-KCoS*Y^ ze$LPNIX~y;{QUjH&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqQdKmWkS&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=lqQd zKj-KCoS*Y^e$LPNIX~y;{QP6W&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqiTlKkmo<{-*9X&2rh@YU$ltIo^6Ua-{_CYOY1RUA$eq zUA$eqUA$eqUA$eqUA$eqUA$eqUA$eqUA$eqUA$eqUA$eqUA$eqUA*1*RQjIEk88fC zQoWjbHT7!h)%ZC-=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=f3atGs4gLIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC zoS*Y^em-veoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-J4)%ZC-=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=luL7;phCEpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax9}7R{=lqQdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`T3WJpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{QN5$Kj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqSzYlNTkbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{QO(P&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=lq+UC{TIz3I7 z?MyStO0~M-+Gw1eIvn+F(PLKUUB=a8R#$dw->m((UwmALrSs;c*5TPW$uQ!@`-aY~ z?jF_XlX{Egdd6x!Wv!mKo{e0oCtZ#Bf%t*=f%t*=f%t*=f%t*=f%t*=!M8W@1Mvg# z1M!1rD}G>ozL$CSbC8?4mD{Tc4|sS0ArFUVXg!c=hq>f`0N^K*XA&-pn&=jZ&K zpYwBm&d>Sz4~L)gbAHax`8hx5=lqQdKj-KCoS*Y^ ze$LPNIX~y;{QONCKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqSzk2ZeJ&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=lqSzFNUAQd zKj-KCoS*Y^e$LPNIX~y;{QR97Kj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqSzFE@V9&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=lqSz?}nfAbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{QLtNKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqSz?>BzV&-pn&=jZ&KpYwBm&d>QdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqSzUxuIabAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{QP4ZKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqRI%QJqSldZ39UahUu({$O+ zG?T1Us~fJ3#@VUEQQsClW_8|WTs>xWWw-Xt+K>Ci$8}gbZ(eF0o{f_XBVN33=zRWu z(=69BR_iHi^}O|LiTl zKkmoQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=lqQd zKj-KCoS*Y^e$LPNIX~y;{M`S4{&e^`Kj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQdKj-KCoS*aamo|RR&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKYzUObAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6YEMdRoEoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=lqQdKY#V`bAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6YEOZYiI=jZ&KpYwBm&d>Qd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6YETjS^aoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=lqQdKYzpUbAHax`8hx5 z=lqQdKj-KCoS*Y^e$LPNIX~y;caxvzWb132S8MC^ zG+nkc%_J+;>V|8hadzr()VD>CS)F$oSC3g;*{yxE_Tzr>aUGVSoo=s+(0et8P}^th!lsv+8Em&8nMK zH>++|-K@MQdKj-KCoS*Y^e$LN-r15ip&d>QdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LN-A^e=5^K*XA z&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=kM6~IX~y; z{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LN-squ4u z&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LN- zC;XhB^K*XA&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=kMS6IX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^ ze$LN-ukmw!&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC zoS*Y^e$LPTBK(}6^K*XA&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=O5GfIX~y;{G6ZjbAHax`8hx5=lqQd zKj-KCoS*Y^e$LPTs_}Dv&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lquZ}=YwPqhUA8mLBrDbGhHIm7cIt4{w?&Uxop%{mk6B&W zt$nlh<9_jR9hT0Ums*Es<0Qj~7w;Q7^L@V0_xV2G=lgu0@AG}W&-eL0-{<>$pYQX1 zzR&mhKHum2e4p?0eZJ54`99z0`+WbR?@zN_M!#B0zgAwqo{d~7spo23jjM4ruEy24 z8du|LT#c)7HLk|hxEfdEYFv%0aW$^S)wmj0<7!-ut8ukYi@H>Gsp?YsIX~y;{G6Zj zbAHax`8hx5=lqQdKj-KCoS!c>e$LPNIX~y;{G6Zj zbAHax`8hx5=lqQdKj-KCoS*aa&uILdpYwBm&d>Qd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAEoU@pFF8 z&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS#1u ze$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*aa zFAYEE=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`T1jwpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`T47bpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQd zKj-KCoS*Y^e*PMbpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax?=C;j$=264uh!P-X}WA@nn_lw)eYB1TJ~6sIyULqs~U1jXE23HtKBD*{HKo zXQR$WosBvhbvEj3)Y-^~QdKj-KCoS*aa9|}L`=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`S}|)e$LPNIX~y;{G6Zj zbAHax`8hx5=lqQdKj-KCoS*aaA8!1dpYwBm&d>Qd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*aapASFh=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`T09Ee$LPN zIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*aaUu^uG zpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*aa zQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hxT-Nw)PIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC zoS*Y^e*Tf+=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hxT^YC+i&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hxT%f`?7IX~y;{G6Zj^XHeJ=Va?^n^$Y=^fX4UD>UDv-abD@o^oN&YPE7hiBs?!-yB}8#=eTdsL%O>MfS*8LRb_ zwR+xqHgct&bT!v?fD;ro1_@AG}W&-eL0-{<>$pYQX1 zzR&mhKHum2e4p?0eZJ54`99zO)bM@2&-b|+-{QdKj-KCoS*Y^e$LPN zIX~y;{G6Zj^G|R5oS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KE|8w8pezoy)e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS#44_&Go4=lqQdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax9}Yj~=lqQdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`T3WGpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=lqQdKj-KCoS*Y^e*R^RpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e*R72=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hv-^~TTnIX~y;{G6ZjbAHax`8hx5 z=lqQdKj-KCoS*Y^e*P_upYwBm&d>QdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e*S~u=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAEoe`FT#ZzP5R_woXseWjoVM zvQn*XxHcMRrw&JbTlARKd6#kZnAMft+Ba)I?iU}|Vd=bisdacZPBM&m@xGyRtGh=v z`lQ}sxt_6FPg$$yt!E?F71kBj71kBj71kBj71kBj71kBj6?faZ!n}Pi^X%s!H*+hu zb0<&4yxqLryxqLryxqLryxqLryxqLryj?w=db&4k>gm+esi#vpg#QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=luL_!_WCSKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX{2<#?SdV zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=luM= z!_WCSKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN zIX{2j#?SdVKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=luL5!q53RKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC zoS*Y^e$LPNIY0lX#?SdVKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=luLr!q53RKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqoft*z73blJ`{ldM#$8?KGU*{Q=(-xfV) zb>3xMJ!W-fxAx82kNd^PbyzxYUTPhljgt%`Uc7JU-0JR8jXtTjSgvQR)>GE%dF$B- z-{<>$pYQX1zR&mhKHum2e4p?0eZJ54`99z0`+T48^L@V0_xV2G=lgu0@AG}W|7q=o z=KK77FZ1l@AUAU>w{s^?grDQdKUaUw&o>)C=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=lqQdKj-KC{Gsr3e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS%Pj_&Go4=lqQd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHaxA8GuYpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHaxUnTsUpYwBm&d>QdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*aaS8M#7pYwBm&d>QdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHaxUoZTepYwBm&d>QdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*aaH)#BvpYwBm&d>Qd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;cb}iV|8hadzr()VD>CS)F$oSC3g;*{yxE_Tzr>aUGVD1Gyr&CX- zo=!cTdOG!V>gm+esi%|o%Fp>ZKj-KCoS*Y^e$LPNIX~y;{G6ZjbAJA_;phCEpYwBm z&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAJBwji2*# ze$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC{Jp}@ z`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAJ9? z;phCEpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAEo@_&Go4=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHaxKRo=LpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQd zKj-KCoS*aak8J#$pYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHaxKRNuIpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQdKj-KCoS*Y^e$LN7wefR)&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lq$pYQX1zR&mhKHum2e4p?0eZJ54`99z0`+T48^L@V0_xb({(f9fJ zUgp`)L2l+&Zs$(;IX~y;{G6ZjbAHax`8hx5=lqQd zKj-KCoS*Y^_2>NjO5^AJoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKYz0EbAHax`8hx5=lqQdKj-KC zoS*Y^e$LPNIX~y;{G6XZ7=F&r`8hx5=lqQdKj-KC zoS*Y^e$LPNIX~y;{G6Zj^DhcN=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=lqQdKj-KC{7V`?=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqQdKj-KC{2Rj0`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAJA+ji2*#e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQdKj-KC{F@p-=jZ&KpYwBm&d>QdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC{QJYt`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAJB%ji2*#e$LPNIX~y;{G6Zj zbAHax`8hx5=lqQdKj-KC{0AF9=jZ&KpYwBm&d>Qd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=luL`^z)o-eQon-PwQtsbT#LVFwxwqpFXMQ4Hcm2(c=5iWd&^QrHTtC9 zV!58NT2EQ4=UG=+S6EkAS6EkAS6EkAS6EkAS6Elvjq3{Y_PxxrpM%`Yt=x`zyLr2L zyLr2LyLr2LyLr2LyLr2LyLvkHbn5BU)4g@n)2XLZPp6(vJ)L?w^>pg#)YGY_Q%|R! zPCZ?vRJPWZ+Nh1&q)pp$TWPCp*4A3r*4swoYFv%0aW$^S)wmj0<7!-ut8q21#?`pm z&xEUSHLk|hxEfdEYFv%0aW$^S)wr7Psq}p^>QebRKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax-?s5{e$LPNIX~y;{G6ZjbAHax`8hx5=lqQd zKj-KCoS*+(QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*+@_&Go4=lqQdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax-@Ea1e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*-8QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqQdKj-KCoS*+m_&Go4=lqQdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHaxKcew-e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*+$QdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqQdKj-KCoS*-{@N<67&-pn&=jZ&KpYwBm&d>QdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~~h&-pn&e?IzoPPV?bd9}7qPt#>P z(@e5bt!}tB8fT{tM}1rLnALfgarKzhmEGDmYd@|pD|u#HdZzI*j)!OCB*Tao?;Elv%{l(l-E-$Q;6`91V}{5|C7d@u9t=O8z8E4Sn4+|Rk6bNPDm_2lcx z*ORX&Ur)ZCd_DPk^7Z8F$=8#wCtpv#o_sy|dh+$;>&e%XuP0wmzMgzN`Fis8QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC zoS*Y^e*QrCIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC zoS*Y^e$LOoF#MdK^K*XA&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=MOc0&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=dTog&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN`KvU3&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=eyzO{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIY0lt@N<67&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIY0k_#?SdVKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=lqKTu8qdosl!p<7CmNl-ep`p zW_4w^_RZRl`^Cp~SUPWBY8{@9lMEwXyl?2->h4jEKB>1@u4k;)Q`YKv>)FVadeYTg z%k_HO8}**|)SKL!`*MH$9`bw0@1f`B?;$_udzoiH2f3MBv97SLu&(g;+~0G5&;31@ zmn$zkuDm6!X|k(VnkS6;5XTzR?ja^>a9%axZaFIQf!yj*#?@^aQdKj-KCoS*aaw`u&GpYwBm&d>QdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax-y{5-pYwBm&d>QdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=lqe$LPNIX~y;{G6ZjbAHax z`8hx5=lqQdKj-KCoS*aa_iFr{pYwBm&d>QdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHaxKQ#QDpYwBm&d>Qd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*aa4{!XOpYwBm z&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHaxKQa8A zpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y; zpWOI4Kj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqf55ntj@cP ztH-Ra?AE?n`*FYcxDHF_%}cGrvvHDP#EbV0om<^Ks?jI)7R&XF)q0BghWW;NGv6@Z zFyAoWFyDA?&Nuu$-^)DvImpfUd+zV~o#yYkzvup*`+M&1x%_+i_ww)M-^;(3e=q-D z{=NKr`SEKj-KC{QmHBe$LPNIX~y; z{G6ZjbAHax`8hx5=lqQdKj-KCoS%O|_&Go4=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHaxA8h=bpYwBm z&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHaxUorfg zpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*aa zS8n{ApYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHaxUnl&WpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQd zKj-KCoS*aa*KPcqpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax-!lB1pYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQdKj-KCoS*aaw{HBLpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;&sRUs$=264 zuh!P-X}WA@nn_lw)eYB1&e%XuP0wmzMgzN`Fis8QdKj-KCoS*Y^e$LPNIX~y;{QTX+&-pn&=jZ&KpYwBm&d>Qd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lq*448oS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqQdKY!20&-pn&=jZ&KpYwBm z&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{QN`0&-pn&=jZ&K zpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKmV}C&-pn& z=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{QMKb z&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^ ze*Q^~pYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y; z{G6Zj^PA!4{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^ ze$LPNIX~|jKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqSz?Z(gfIX~y;&s9Iq$=264uh!P-X}WA@nn_lw)eYB13Hf6x6r_xD`>z5ILm_ww)M-^;(3e=q-D z{=NKr`SQd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*aaFKqmrpYwBm z&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*aauMI!v z=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`S~j~ ze$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*aa zuW$UEpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN zIX~y;-xGe$&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQdKj-KCoS*Y^ ze$LPNIX~y;{QMtQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC zoS*Y^e$LPNIX~y;{QRvNKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqSzPd9$f&-pn&=jZ&KpYwBm&d>QdKj-KCoS#2`{X8dIU)#J| zTc@Y#vYlxrS*cbxTpNwEQ-`CzEqcuAyvw+H%<9T+?VGh9_lu9~uyo$M)H*yHCmBY( zc;C>u)!m~SeNu0+T+djor>xcU*0Yf-^`xu0mh1JlH|jm_sW-Vde4p?0eZJ54`99z0 z`+T48^L@V0_xV2G=llO8e4p?0eZJ54`99z0`+T48^L?(y_xb+w*Z2APUgp`)LHwNi zIrnoePez`MJQ;a1@?_-6$di#LBTq)2j64~6GV)~P$;gwDCnHbhKSiF5JQ;a1@?_-6 z$di#LBTq)2j64~6GW?vM^K*XA&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LO|qw#Zo z&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hxT z;P7*P&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y; z{G6YEXyfPnoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKmX&#&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQd zKj-KCoS*Y^e*VA1&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqQdKj-KC zoS*Y^e$LPNIX~y;{CqY1oS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-HM;phCEpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqSzz2WEloS*Y^e$LPN`Q7X1IobN!=GEFdJx!PGOf$(! zwYuTjXq=ro9QAF{V^-%~#?@n1S9WXPto^uOd|Zd6^X8@2;n_IJFyh7ghR&_-9@XfR zdW+?HhINH?g>{8>g>}W}#k#_}!n(q`!n(q`;_h8nn78j`p8c4&o41>{o41>{-`~vJ z&D+h}&D+h}&D+(}si#vpg#)YGY_Q%@)F zm7nu-e$LPNIY0k`#?SdVKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAJBs!q53RKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=luNZ8b9af{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN`QHye=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqQdKj-KC{B;^X=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj^M4qA&d>QdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN`CB%A&d>QdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=l?kToS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqQdKj-KFG5nmL^K*XA&-pn& z=jZ(Vh3w}!+4|b%)!I5eO_%LVGs#M|y5ZVroSixx^=;8(R_9&D)nisyc5C0P{kUIz zT!*Fe=B3u***M8C;>G)h&aLhq)##IYi{*L--{<>$pYQX1zR&mhKHum2e4p?0eZJ54 z`99z0`+T48^L@Vm?v3yBeZJ54`99z0`+Wa}?ECzDFZ1k&pYwBm&d>QdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hxT^~TTnIX~y;{G6ZjbAGP=oS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;|26!apYwBm&d>QdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax|ETeEe$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQdKj-KCoS*-z@N<67&-pn&=jZ&KpYwBm&d>Qd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS%O}QdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHaxZ-k%obAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{CwK@IX~y;{G6ZjbAHax`8hx5 z=lqQdKj-KCoS*Y^e$LNtHGa;|`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj^PTW>e$LPNIX~y;{G6Zj zbAHax`8hx5=lqQdKj-KCoS%P5_&Go4=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHaxuQh(o&-pn&=jZpb zpXX@lYnfGR>v%U^wzHT?maEkb$42Aq)aIaX^B&XM?=r3)(>k(S>t?No{ruzFESxsa zwf0Z0lMGipd*0Bw)!C!E`lOy>sqV2-cUi6btz|tMb*Brtm`nAvm+LvN)RXx4Zshfh zH;gxoH;gxoH}2)}hTrqO%;G(v_k`XP8n=5-=sn>-YR2ux?Z)lK?Z)luY}DDPvr%WG z&PJV$IvaI1>TJ~6sIyULqs~U1jXE23HtKBD*{HLThvVn`oS*Y^e$LPN`J08G^K*XA z&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=luK|!q53R zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj z^LGkA=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQd zKj-KC{4X|s&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQd zKj-KCoS*Y^e$LPN`MWfJ&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=lqQdKj-Ja9DdHv`8hx5=lqQd zKj-KCoS*Y^e$LPNIX~y;{G6Zj^S{>kIX~y;{G6ZjbAHax`8hx5=luNs_VXMqeJ!(U zZ5{8X%XSts$#S*2;n--Lo!T7qZQf&A`(4J>V_HXcYu&8%u%CZin}yTnxz_&4b&}zV zXU`itw>o=NSD(~VEY&?$>MpBwzqPDqqwaJe7jvne_HsSvm3oq^xt8mBAP?rDJe)`J zXdcVsc_L5dMR{?!ANS*a+>iTlKkmoRqLAFn=MeY~8EoQ#}|oQ#}|oQ#}|oQ#}|oQ#}|oQ#}| zoQ#}|oXkf@PDV~fPDV~fPDV~fPDW0KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=luM4!_WCSKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqQdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=bsjS&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQd zKj-KCoS*Y^e$LPNIX~y;{G6Zj^S{^lIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LOA!q53RKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqQd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`T0uY=lqQd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hwo+4wm>=jZ&KpYwBm&d>QdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=lqQd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=luMO!_WCSKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqQdKj-KCoS*;0#?SdVKj-KCoS*Y^e*WD0 zd5)I8mRYs7j(5{#JByiQxmw+DY&5R_`ciVxw|S3g?RObhk7*s*t#z~3!+!p8Z5B?O z=UV$G*GYyeo;`2q-0JL6U42qd;r)U42i_lef8hOr_Xpk|cz^KXcz@vif%gaAA9#P@ z{ekxf&+YpI^Ygu!pPQeXpPQeXpPQeXpPQe5eaz3z&&|)x&&|)x&&|)($E%N5AFn=M zeZ2a3_3`TC)yJ!kS0ArFUVXg!c=hq}?fjgd^K*XA&-pn&=jZ(V&oq9{&-pn&=jZ&K zpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;?-+j0&-pn& z=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqSzUugWC zpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqt z@pFF8&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hv-=f=QdKj-KC zoS*Y^e*Ui;Kj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqSz`-h+NbAHax`8hx5=lqQdKj-KCoS*Y^ ze$LPNIX~y;{QQ@~&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqQdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`T2in{G6ZjbAHax`8hx5=lqQd zKj-KCoS*Y^e$LPNIX~y;9~FMi&-wYE?&mpL`dVhy+B)7%m+dTOlI3c3!?DpgJGD9J z+q}oL_PdO$$Fz>@*1B2iVL$)4HVdcCbFKZ8>miTlKkmoQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lq=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`T3`YpYwBm&d>Qd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{QU1We$LPN zIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*aa#qe`} z&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lp!R z@pFF8&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hxTqVRKm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=PwFB=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqoXK0p{oC=jP|%(9F-x&&|)x&&|)()2XLZPp6(vJ)L?w z^>pg#)YGY_Q%|R!PCcD^I`wq?oS*Y^e$LPNIX~y;{G6ZjbAJ8~;phCEpYwBm&d>Qd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS(l_QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAJBX@N<67 z&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax zf1>eoe$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC zoS(m6_&Go4=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHaxe=+==pYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHaxf4T8QdKj-KCoS%PW_&Go4=lqQdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHaxe>?n~pYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lq?fDc_feKu{@q9@?^Ll_v3!t zkNa^y?#KPOANS*a+>iTlKkmof_bNtB;qHk&}^=k&}^=k&}^=k&}^=k&}^=k&}^=k&}^=k&}^=k&}^=k(2q9$jQjb z$jR_?e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKmT9h z=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hxT zw8qc*IX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^ zem)65=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQd zKj-KCe5vtse$LPNIX~y;{G6ZjbAHax`8hx5=lqQd zKj-KCoS)AcKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqSzQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQd zKj-KCoS*Y^e$LOYG=9#{`8hx5=lqQdKj-KCoS*Y^ ze$LPNIX~y;{G6Zj^Dl1voS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-I9hM)6ue$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC{AQdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAJBf#?SdVKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqArSgE%FtMxu$E$i8+ zHvt!NF_-FXf^oZXyK%d5yK%d5yK%d5yK(z>$GF|N-MC%dth!lsv+8Em&8nMKH>++| z-K@GQdKj-KCoS*Y^e$LPNIX~y;{G6Zj^Y3l^oS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=lqQdKj-JK2|wrO{G6Zj zbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN`HzL4^K*XA z&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=luM2ji2*# ze$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC{Gjo3 ze$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS(mM z_&Go4=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax zeQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHaxf2r|ve$LPNIX~y;{G6ZjbAHax`8hx5=lqQd zKj-KCoS%P0_&Go4=lqQdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHaxe=Gc)pYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHaxf2Z+te$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS%Pk_&Go4=aU>SeJ!(UZq4_*Y-cf(ET^mg)iN4qr#1(DoA;R3 zewT6enAVZqS~qJw?B^fXX5qAXuC;%1on*M;+4F|Zt$pYQX1zR&mhKHum2e4p?0eZJ54`99z0`+T48|IhG!zR%D3IX~y;{G6ZjbAHax z`8hx5=lqQdKj-KCoS%PcQdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`S}-wpYwBm z&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hv- zBK(}6^K*XA&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=U)?k&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y; z{G6Zj^RH|CoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-If7kQdKj-KCoS*Y^e$LPN zIX~y;{G6Zj^Y02j=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqv%U^wzHT?maEkb$42Aq)aIaX^B&XM?=r3)(>k(S>t?No z{ruzFESxsawf0Z0lMGipd*0Bw)!C!E`lOy>sqV2-cUi6btz|tMb*Brtm`nAvm+LvN z)RSDzwOr2wc`y&<;XIN@^H?5_zeD~G`8)Ld_&el#zL!~UQdKj-KCoS*Y^e$LPN`K!av`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAJA#;phCEpYwBm&d>QdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIY0mL#?SdVKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIY0mT@N<67&-pn& z=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIY0l!#?SdV zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIY0l+ z@N<67&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN zIY0mH#?SdVKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lq8I{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^ ze$LPNIY0j&;phCEpYwBm&d>QdKj-KCoS*Y^e*XOM^BgUGEwgHE9q*>gb`~?qa<#hQ z*l3)c+8p$4-eX$(UB=a8T1R$k-K_PnpMPANh12G_*8a(LlHrPH&l@_oI(t-CpVU(< z)jd|~E~|CFwXA2O?sOp+bE%&8ay{pjdXlTTmg{*S59XmfoJaC#9?RqLcgWu%e}|qQ ze}{a}_cF_k+{~@q&Yj%NQ+Ybizv_a=CK3ax(m!pYwBm&d>QdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`T6HJe$LPNIX~y;{G6ZjbAHax z`8hx5=lqQdKj-KCoS*aaOX27IoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqQdKYwHRIX~y;{G6ZjbAHax z`8hx5=lqQdKj-KCoS*Y^e$LM?H-65~`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj^Dk`toS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=lqQdKj-IQdKj-KC{Hw#y`8hx5 z=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAJA0QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAJA| z;phCEpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC zoS(maQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqv%U^wzHT?maEkb$42Aq)aIaX^B&XM?=r3)(>k(S>t?No{ruzFESxsa zwf0Z0lMGipd*0Bw)!F71<`w1@<`w1@<`w1@<`w1@<`w1@&zE_{VwP%duu`*w)tV=) zWjz};L%5KO@q6y~-0!*HbHC?)&;6eJJy&O=&PJWhcQTJ~6sIyULqs~U1jXE23 zHtKBD*{HKoXQR$WosBvhc{q7Ec{q7Ee$LPNIX~y;{G6ZjbAHax`8hv-RronS=jZ&K zpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-JKY5bg@ z^K*XA&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=Rel? zIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LO| zJN%rV^K*XA&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=RX&I&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=fBYSIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC zoS*Y^e$LN7Ec~3G^K*XA&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=f4qt&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=fBnXIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LN7G5nmL^K*XA&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=YJf2&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lq<}0>)EKkC>L@uay)W8ay)W8ay)W8ay)W8ay)W8ay)W8ay)W8 zay)W8ay)W8ay)W8ay)W8ay)W8ay)W8ayQdKj-KCoS*Y^em-veoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqQdKj-J4*Z4U<=jZ&KpYwBm&d>QdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{QR--bAHax`8hx5 z=lqQdKj-KCoS*Y^e$LPNIX~y;{G6YERronS=jZ&K zpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-IP)A%_* z=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-J) z5q{3k`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj z^S5jKoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQd zKj-J))%ZC-=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lq>&R}co3$SH^N(w@aN0cA z+CRBYGFQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=luM=!q53RKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC zoS*Y^e$LPNIX{1&#?SdVKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=luLb!_WCSKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIY0mK#?SdVKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=luK=!q53RKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqQdKj-KCoS*Y^e$LPNIY0lT#?SdVKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=lqQdKj-KCoS*Y^e*W3v=lqA-?Cn=f3B@=W@An zxpKL3xpKL3xpKL3xpKL3xpKL3xpKL3xpKL3xpKL3xpKL3xpKL3xpKL3GIBCQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y; z{G6X(YW$p^^K*XA&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=WlHMoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-IQdKj-KC{42xH`8hx5=lqQdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAJAKQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAJBT;phCEpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqQdKj-KCoS(mKQdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=lqQd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAJAc@N<67&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{QOmopYwBm&d>QdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax?Zn|t|F_SD;s~e7u#@VUOLEq**rnTQ?Ts@|BWVhDM zS`YjA$F*5FZJulGpIj#yu6Xvmp|kOZ@rLn+@rLn+@rLon{WRWK%uQdKj-KCoS*Y^e$LO|GyI&N^K*XA z&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=RXsE&d>Qd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=Reo@ zIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LN7 zB>bG8^K*XA&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=f4(y&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=fBbTIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC zoS*Y^e$LN7KKz`Y^K*XA&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=YJS}&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=YQPzIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LN7EBu_F^K*XA&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=L?OW^K*XA&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=luM08b9af{G6ZPPkx@GrLSdHt*ztTblJ{gCRwglHyj&{ zvs0UczRi0~Yro65dQ9ucZmpZO9`^H(YqN0LJlEPkxlS@%@$7j+XaBtz|K5$fp7Dn9 zhVh2+#{D$jSjQdKj-KCoS*Y^e$LPNIX~y;{G6X(Z2X*`^K*XA&-pn& z=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=Wl5IoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=lqQdKj-IQdKj-KC{42uG`8hx5 z=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAJ9<QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAJ9- z;phCEpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC zoS(l<QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAJBv@N<67&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqQdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHaxf4K2;e$LPNIX~y;{G6ZjbAHax`8hx5=lq?fD z;ro1_@AG}W&-eL0-{<>$pYQYi_XywT`+T48^L@V0_xV2G=lgu0@AG}W&-b|+-{<@H zm+$lQz07hWH*+hub0>H6RGyCSx$n8}xtxrgjGT;|jGT;|jGT;|jGT;|jGT;|jGW9* zM@~jgMovafMovafMovafMovafMovafMovafMoxyG^K*XA&-pn&=jZ&KpYwBm&d>Qd zKj-KCoS(l}QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAJB8;phCEpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`T2)7e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*aaUvK=JpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqQdKj-KCoS*aaKL|hP=lqQdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`S~X_e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*aaKWhA(pYwBm&d>QdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqQdKj-KCoS*aa*7!L;=jZ&KpYwBm&d>QdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6X}grD~Uwv*!(+Tb(_st551Fmg*iWb(ht;-&)pVUh#CyE6gj*E6gj*E6gj*E6gj* zE6gkIt$BrU`(9?bk(;@d+qsjwc`C;37n*Uqal3K5al3K5al3K5al3K5al3jt^>pg# z)YGY_Q%|R!PCcD^I`wqw>D1Gyr&CX-o=!cTdOG!V>gnWO`8hv-w()a*&d>QdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN`J;`W^K*XA z&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=luLD8$ajg z{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN`CEpc z^K*XA&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=luLz z!_WCSKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=luNJ8$ajg{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^ ze$LPN`OCu3`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAJAV;phCEpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC zoS*Y^e$LPNIX~ZT{G6ZjbAHax`8hx5=lqQdKj-KC zoS*Y^e$LPNIX~y;?;d{6&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=lq$pYQX1zR&mhKHum2e4p>XXXE>P zpYQX1zR&mhKHum2e4p?0{TG_=^YgvTaw9i$E4Oneck@*EIX~y;{G6ZjbAHax`8hx5 z=lqQdKj-Hk6n@Un`8hx5=lqQdKj-KC zoS*Y^e$LPNIX~y;{G6Zj^Ir`==jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=lqG=9#{`8hx5=lqQd zKj-KCoS*Y^e$LPNIX~y;{G6Zj^IvQHoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-Hk8-C8u`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj^WP6Y=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj^FM6-oS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqQdKj-J45q{3k`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj^E7_W&-pn&=jZ&KpYwBm&d>Qd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;>*448oS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqQdKYuFxoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqQdKj-J0ji2*#e$LPNIX~y; z{G6ZPYkr=irLSdHt*ztTblJ{gCRwglHyj&{vs0UczRi0~Yro65dQ9ucZmpZO9`^H( zYqN0LJlEPkxlS@%@$7j+=T>Kr>gtnvilw^8O5J6(?za~63iAr{if5X6g?WW}g?WW} zg?WW}g?YujHm@*l-^(mFax=GbJ9lz7#_h)K#_bQqxZSwjxZSwjxZSwjxZSv2J)L?w z^>pg#)YGY_Q%|R!PCcD^I`wqw>D1Gyr&CX-o=!cTdOG!VaQdKj-KCoS*Y^e$LPNIX{1-@pFF8 z&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX{1k z@N<67&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQdKj-KCoS*Y^ ze$LPNIX{1C_&Go4=lqQdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHaxe<1vvpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC zoS*Y^e$LPNIX~y;Kh*d+Kj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQd zKj-KCoS*Y^e$LPNIX~y;KN)_`&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;Ki&8_Kj-KCoS*Y^e$LPNIX~y;FFZfb(bCs4tJc=> zZn|t|F_SD;s~e7u#@VUOLEq**rnTQ?Ts@|BWVhDMS`YjA$F*5FZJulGpIj#yu6Xvm zp>wOVM|Jf{J;hSpW2Nr0TK8KE-{<>$pYQX1zR&mhKHum2e4p?0eZJ54`99z0`+T48 ze_;4N-{<>$pYQX1zR&mhKHq=g`9442%Pco?Gq-X(cXBuUoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;hmD`}bAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{QP6W&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=lqQdKmWMK&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{QT3y&-pn&=jZ&KpYwBm&d>QdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC{4*Or=jZ&KpYwBm&d>QdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax?}nfAbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{CuPFbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6XZ-S{~_=jZ&KpYwBm z&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqG0e+l||e+l||e+tt&lr&CX- zo=!cTdOG!V>gm+esi#vQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hv-^YC+i z&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LO| zvhj0%&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hv-N%%QG=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-H!Yy6y_^K*XA&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=Rer^IX~y;{G6ZjbAHax`8hx5=lqQd zKj-KCoS*Y^e$LO|HT;~P^K*XA&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=l?$ZoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-J~-uO8`=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=lqQd zKj-KCoS*Y^e$LPNIX~y;{G6Zvjqr1R&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqv%U^ zwzHT?maEkb$42Aq)aIaX^B&XM?=r3)(>k(S>t?No^?#x_EemgI{0ZCrlj|hI70;eG zbZ=S6sIES#r&y|ctkhjr>wbKn@AG}W&-eL0-{<>$pYQX1zR&mhKHum2e4p?0eZJ54 z`ThqrzR&mhKHum2e4p?0{THI|^YgvTaw9i$E4One{G6ZjbAHax`8hx5=lqSzuQqQdKj-KCoS*Y^e$LPNIY0k9;phCEpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIY0l{#?SdVKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqQdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=luL1grDQdKj-KC{QrfY^K*XA&-pn&=jZ&KpYwBm&d>Qd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=luLL8b9af{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNc^7`p&-pn&=jZ&KpYwBm&d>Qd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hvdZ~UB}^K*XA&-pn&=jZ&K zpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=T9|$&d>QdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAJ9%M54$9xHX1)wvhMxBj18+kZ>&d>Qd zKj-KCoS*aae-?hu&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqQdKj-KC zoS*Y^e$LPNIX~y;-_rOwKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=lqQdKj-KCoS*Y^e*V(N&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{QR$kpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{QTV-Kj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqSzPd0we&-pn&=jZ&KpYwBm&d>QdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*aa{}g`C&-pn&=jZ&KpYwBm&d>Qd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;ztZ?QKj-KCoS*Y^e$LPN zIX~y;_otudXz6R2Rcq^bH(j=~m`Rqa)eXl+m z;S3G;((7Dywqq_Q}o?@x)u~K(gt^2KIJsWkW3%Qs}^|Y7kIj_`{ zT+Ovy58vnee4p?0eZJ54`99z0`+T48^L@V0_xV2G=lgu0@Bi)aeZJ54`99z0`+T3P z@qNC3fBHT@-^(mFax=GbJHF?>=f3B1GIBCQd zKj-KCoS*Y^e$LPNIX~y;{QP4YKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqSz?=^nT&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*aa{~LbJ&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;tKsMToS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqQdKffJ*&d>QdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqQdKj-KCoS$D0Kj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqSzKMp_V=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`T2v5pYwBm&d>QdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`T3s z{>gQc;fiO^8#=c-dsJ7S)Ke_gJyz;2<`w1@<`w1@<`w1@<`w1@<`w@m<`w1@_vXC9 zxP33P+{n$`igCMfyK%d5yK%d5yK%d5yK(!QH{*8WcJ*}X>D1Gyr&CX-o=!cTdOG!V z>gm+esi#vQdKj-KCoS*Y^e$LPNIX~y;{QNJ~zgqi}yfiP% z%kzr7GOx<3^P0RiugmN6&Uu&o<#08w#?`nQSL142jjM4ruEy248du|LT#c)7HLk|h zxEfdEYFv%0aW$^S)wtTfjJi~Hsp?YsIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS(m>@pFF8&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIY0N`KmWJk=lqQd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hv-*Y^L}x9{Itw)QUI-{rlx%l5dPod}5*34eB*8DOvkNx=$Kj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqSzzis@SpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqQdKj-KCoS*aaUkg9y=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`T1XM{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;|FQ9Ne$LPNIX~y;{G6ZjbAHax z`8hx5=lqQdKj-KCoS%Pk_&Go4=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax|1kWVpYwBm&d>QdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqJ*KtaWn4X`b!4yB&03H8`Omdk zIBQ;L9h{Dn3?rUDZ|L0W>`{$AsjFD3IaX>G&mqqt&!Nlu9CDxUXO?@pp9gWDyU*R{ z@_O=m@_O=m@_O=m@_O=m@_O=m@_O=m@_O=m@_O=m@_O=m@_O=m@_O=m@_PTJ$?M7M z$?M7M$;t3@e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y; z|1bQUpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQdKj-KC zoS*Y^e*Pf*oS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-INQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^etx_0bAHax`8hx5=lqQdKj-KC zoS*Y^e$LPNIX~y;{G6Zv!^Y3~IX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e*R|R=lqQdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hxTj_`AS&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqQdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=TEPn=XmLBnN@51WH()QvY1JhtJMw1M&s<(=CE(`9@En)B7B9pYLaud$}KfSNOZa-xc2H-sj%u-sf_;a=CK3 z|G3HJ%H_)C%H_)C%H_)C%H_)C%H_)C%H_)C%H_)C%H_)C%H_)C%H_(*@N<67&-pn& z=jZ&KpYwBm&d=X5{G6ZjbAHax`8hx5=lqQdKj-KC zoS*Y^e$LPNIX~y;KOBC}&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=lqQdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`T2W?pYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqQdKj-KCoS*Y^e*TvmKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqSzziIrOpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqQdKj-KCoS*aaUkyL!=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`T1XI{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;|Do}7e$LPNIX~y;{G6ZjbAHax z`8hx5=lqQdKj-KCoS%PE_&Go4=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax{~-LFpYwBm&d>QdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqSz)9dFsUiw;Q)!IJUO_!Z4W|HMnu?AyG@wD!Arp@dxi$-D%?quA({Ykv#PjD3om-tfs?jHP6-zb8O3kub^Q~n)8#U8buHvbhJ;yL6w>8K8M`r`PXu3WBMu3WBMu3WBM zu3WBMu3WBMu3WBMu3WBMu3WBMu3YYaZgRPDxpKL3xpFf6oS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN`Tq?+=jZ&KpYwBm&d>QdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=PTjo{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX}N2e$LPNIX~y;{G6ZjbAHax z`8hx5=lqQdKj-KCoS*aa)yB{HIX~y;{G6ZjbAHax z`8hx5=lqQdKj-KCoS*Y^e*UoWbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZP2tViN{G6ZjbAHax z`8hx5=lqQdKj-KCoS*Y^e$LPN`B#OX^K*XA&-pn& z=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=luLuQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=luLl!_WCS zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIY0kX zji2*#e$LPNIX~y;{G6ZjbAHax`8hx5=lqf z*D|Zt_Q`I#>|`;MELW==j*Z6It<7QI<~^ph-(_4qrgdbm*3DXv`uWebSvYH6XdRr6 zlMExCKX2&V>g-XCKB=o%syWmf>J9aVdPBXT-cWBm&Gm-&`F>`(7w>cLbMJHSbMJHS zbMJHcd-;3$`+wBr@8$31@8$31@8$31@8$31@8$31@8$31@8$31@8$31@8$31@8$31 z;rKZ}=jZ&KpYwBm&d>QdKj-J~5Pr_j`8hx5=lqQd zKj-KCoS*Y^e$LPNIX~y;{G6Zj^B)R7=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqQdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=l`tnbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Y^SNJ(U=jZ&KpYwBm&d>QdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-Ivsqu4u&d>QdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=lqQd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKmV2RbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6YEMC0fDoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=lqQdKmWDH&-pn&=jZ&K zpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKmYykbAHax z`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6YEa^vUx zoS*Y^e$LPNIX}P5exBo{uVq%P?UUVf*~wxiS*}($9286% zeeOP&*OS+i*OS+i*OS+i*OS+i*OS+i*OS+i*OS+i*OS+i*OS+i*OS+i*OS+i*ZadJ zuP3i3uP3i3C&SPAIX~y;{G6ZjbAHax`8hx5=lqQd zKmVNYbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y; z{G6ZvpYU^j&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hvd4nODT{G6ZjbAHax`8hx5=lqQdKj-KC zoS*Y^e$LPN`MvOSe$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS#n{Kj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqSzgT~MKIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e!d%i&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN`OU`9`8hx5=lqQdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAJBSji2*#e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC{7u5o`8hx5=lqQd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAJA9;phCEpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;Pq&}vcvUD>ch%&9|2I zY}8C!xst1Owb$x8uh*5_$j#i!?L3n^xtnM6T%OMhc`+~Lb$NaGKHum2-_iI!-{<>$ zpYQX1zR&mhKHum2e4p?0eZJ54`99z0`+VR3+xb4<=lfiZ@ALhq+xPkTerEAL_dfSN z_dfSN_dfSNmxq&wlZTUsd;7@4$-~LR$-~LR$-~LR$-~LR$-~LR$-~LR$-~LR$-~LR z$-~LR$;0t;e$LPNIX~y;{G6ZjbAHax`S}lqpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`S}kwe$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQdKj-KCoS*aa_Y6Ph=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`T0+VpYwBm&d>QdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`T0*be$LPNIX~y;{G6Zj zbAHax`8hx5=lqQdKj-KCoS*aa4-G%(=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`S~x0pYwBm&d>Qd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`T4Ile$LPN zIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*aaPY6Hf z=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`T6gK zpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqSzYunFry!5rqsV{*Z zadvBS*tdC)Y3+9zSC45O*{gN4)}wy@b8QyRnipCJr{g5Ui0986I=4D|RHIMoDwb-F zm6~O>=3C2pHfpA=T*=kC+G};4*Xv4dSzXNRBjbAHax`8hx5=lqQd zKj-KCoS*Y^e$LPNIX~y;{QQ51pYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqQdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`T0`#IX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LNl;phCEpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqSz_3(3k&d>QdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LNtG=9#{`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj^RH_BoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=lqQdKj-JK4?pMU{G6ZjbAHax`8hx5 z=lqQdKj-KCoS*Y^e$LPN`L~9j^K*XA&-pn&=jZ&K zpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=luLl8$ajg{G6ZjbAHax z`8hx5=lqQdKj-KCoS#4KexBo{uVq%P?UUVf*~wxi zS*}($92QdKj-KCoS*Y^ ze*Sjh=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hxTf$(#F&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hxTp~lboIX~y;{G6ZjbAHax`8hx5=lqQd zKj-KCoS*Y^e*PZe=lqQdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hv73_s`R{G6ZjbAHax`8hx5=lqQd zKj-KCoS*Y^e$LPN`Fk~f&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=N}S&&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQdKj-KCoS*Y^e$LPN`G+-r&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqQdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=N}(_&d>QdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN`6o7h&d>QdKj-KCoS*aa%kJknUiw;Q z)!IJUO_!Z4W|HMnu?AyG@wD!Arp@dxi$-D%?quA({Ykv z#PjD3om-tfs?jHP6`n($L!LvI{W;`5-;ev;eeOP&*OS+i*OS+i*OS+i*OS+i*OS+i z*OS+i*OS+i*OS+i*OS+i*OS+i*OS+i*Zch@uP3i3uP3i3C&SPAIX~y;{G6ZjbAHax z`8hx5=lqQdKmV-obAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zvukdqz&d>QdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hvd3_s`R{G6ZjbAHax`8hx5 z=lqQdKj-KCoS*Y^e$LPN`F{90Kj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=lqSzy~fY^IX~y;{G6Zj zbAHax`8hx5=lqQdKj-KCoS*Y^e!dfa&d>QdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN`EKLq{G6Zj zbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIY0l(#?SdV zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS%P7 z_&Go4=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z-=y(#e$LPNIX~y;{G6ZjbAHax`8hx5=lqSz)9>dw zUiw;Q)!IJUO_!Z4W|HMnu?AyG@wD!Arp@dxi$-D%?quA z({Ykv#PjD3om-tfs?jHP6-zb8O3kub^Q~n)8#U8buHvbhJax=GbJI~}! z?&jG%m*?|BUd&73`+T48e_P}Ge4p?0eZJ54`99z0`+T48^L@V0_xV2G=lgu0@AG~C zZ|D1bpYL-uzR&lce&6>#_dfSN_dfSN_dfSN_db`0lZTUslZSiT$ivCQ$-~LR$-~LR z$-~LR$-~LR$-~LR$-~LR$-~LR$-~LR$;0t;e$LPNIX~y;{G6ZjbAHax`8hxT{_t~t z&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hxT z!N$+|IX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^ ze*W&^=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hxTiSToN&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hxT$;Qw5IX~y;{G6ZjbAHax`8hx5=lqQd zKj-KCoS*Y^e*VGX=lqQdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hxT#qe`}&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hxT<;KtXIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e*SUc=lqQdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hxTo$zyh&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqQdKY#7} zd5)L9mRYs7Pj=H~CySY6xmw+DY&6bpZ4Uc3?=h|YF5~Jkts{H2Zq|C#&wsAX!dde| z>)>>pWEk=Mc|+$`XOC+1NnOQK&9PFmtk!&MSc%S?Kf&U-)|AD$)-L7s|x69wl z-^<_2-^<_2-^<_2-^<_2-^<_2-^<_2-^<_sUX#CQdKj-KCoS*Y^e$LPN`Tq<*=jZ&KpYwBm&d>QdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj^GWzQKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;OO2oNbAHax`8hx5 z=lqQdKj-KCoS*Y^e$LPNIX~y;{Cw8QdKj-KCoS*Y^e$LOYg`e|te$LPN zIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC{L8}6`8hx5 z=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAEok@pFF8 z&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX{0X z{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y; z-yD9<&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKferq zp5vvjWmc{2lihUL$zmp1u2wf38;!GDo5Q}%drWJ;%eZ<>>&RZMo3$SG^Pg+8aMrxg zIyfCC8Ad#R-q5+#*`pd=y`kPvZ>Tra8|n@9#$~8C7PC|}!%9^Rt5rX&Wjz~JIc()h zu2%KoK6jtH&)w&8xpKL3x!>C4a^-U6a^-U6a^-U6a^-U6a^-U6a^-U6a^-U6a^-U6 za^-U6a^-U6WaMPzWaMP{IX~y;{G6ZjbAHax-zNN=pYwBm&d>QdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*aaw`=^IpYwBm&d>QdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax-!1%{pYwBm&d>QdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*aa_h|f_pYwBm&d>Qd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e*O#L=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hxTkjBsX zIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e*R01 zpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^ ze*W9x=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hxT_{PuqIX~y;{G6ZjbAHax`8hx5=a<3HbG-Dm%&N70vYReDSK9oegOv(}@2{&Q^>&YBlm2dCpC!-(h48#=c-dsL&VH`E*I4fTe4 zL%pHixD55iVwS3ASgEREwd#kptY@Pthpk-6)v7++=k9a&x%*r$S1wmBS1wmBS1wmB zS1wmBS1wmBS1wmBS1wmBS1wmBS1wmB_jj9Iu3WBMu3WC1jGT;|jGPQV=jZ&KpYwBm z&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`T1vrpYwBm&d>QdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{QR>TKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqSzIQ*QS^K*XA&-pn&=jZ&K zpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=U*It&d>QdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKi_HmoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-IP-uO8` z=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC z{B@0=^K*XA&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHaxFNdG! zcvUD>ch%&9|2IY}8C!xst1Owb$x8uh*5_$j#i!?L3n^ zxtnM6T%OMh;ro1_?|)0<`+T48^L@V0_xV2G=lgu0@AG}W&-eL0-{<>$pYQX1|8M8} ze4p=gHNMaHFNg2*^Zm?nFZc5x5A!IG^CU0lmAslaQdKj-KC zoS*Y^e$LPN`P(*r&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=kFSR&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN`MWoM&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=N}k;&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqQdKj-KCoS*Y^e$LPN`3EQdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=lqQd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=N}V(&d>QdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN`NuVW&d>QdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hv-ZTNYPm%f%+wYE=o(`6@%nPjM^Y&d$n%XdeqNKrYV=86#Zt|&QnReqd}~?HM$NR9 zE4f-%d#$eXdR@tl+{~@q&NI1_yLmRx<@vl2^@e&wy`kPvZ>Tra8?TLe!~1+cv)s%5 zJjla5%HurA%XuZQ<_&pc{GZVO3H_f?-L7s|x2xOL?do=QySm+XHomj*osI8od}rf3 z8{gUZ&c=5(zO(V2&F?hd+4#=JcQ(GWk(2SAjqhxHXCn{C&-pn&=jZ&KpYwBm&d>Qd zKj-KCoS*Y^e$LPNIX~y;{G6Zj^G^>y=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN`DZqM&d>QdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHaxzbO2ipYwBm&d>QdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{QPS8IX~y;{G6ZjbAHax`8hx5 z=lqQdKj-KCoS*Y^e$LO|7=F&r`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj^J|Tt^K*XA&-pn&=jZ&K zpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=luN38b9af{G6ZjbAHax z`8hx5=lqQdKj-KCoS*Y^e$LPN`3vFa{G6ZjbAHax z`8hx5=lqQdKj-KCoS*Y^e$LPNIY0l#@N<67&-pn& z=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqM^Y&d$n%XdeqNK{H`E*I4fTe4L%pHixGeR?VwS3ASgEREwd#kptY@Pthpk+R``mr*K6jtX<;vyC z<$iOM%azNO%azNO%azNO%azNO%azNO%azNO%azNO%azNO%azNO%azNOlaZ5=laZ6* z=lqQdKj-KCoS%PB_&Go4=lqQdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax-=^_%e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS%PRQdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=lqQdKj-KCoS*+__&Go4=lqQd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax->va;e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*-AQdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*+(_&Go4=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHaxKdA9@e$LPNIX~y;{G6ZjbAHax z`8hx5=lqQdKj-KCoS*+fQdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*+@_&Go4=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHaxKeq96e$LPNIX~y; z{G6ZjbAHax`T1q>^BgaIEwgHEpX{c~P8KuCa<#hQ*l3*H+8p+6-eX$(UB=a8T1WP3 z-K_PfpZ{E&g|p^`*1_pG$uQ#i^M=l?&Q@=zH`E*I4fTe4L%nfX>W#%LRn4$cRl{o4 z4{KS^MpX`5xf1ue``mr*K9|dt%azNO%azNO%azNO%azNO%azNO%azNO%azNO%azNO z%azOh?IxEimn)Ymmn$bDCnF~#C&SPAIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC zoS*Y^e$LPNIY0lC@N<67&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=lqQd zKj-KCoS*Y^e$LPNIX_Sz7lxnnbAHax`8hx5=lqQd zKj-KCoS*Y^e$LPNIX~y;{CwQ_IX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LOoxbbs-&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqQdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqAKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=lqSzi;bW2bAHax`8hx5=lqQdKj-KCoS*Y^e$LM?kDupw>1&x)Yx`t3U3RjVNtUbC4aY{~ z?AGS6Z}T40+V3*19@9FqSL2i z`=%zZC$A^3C$A^3C$A^3C$A^3C$A^3C$A^3C$A^3C$A^3C$A^3C$GoP<@MzCQdKmYFVbAHax`8hx5=lqQdKj-KC zoS*Y^e$LPNIX~y;{G6Y^b>rv!oS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKmXpw&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqQdKmU>NbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Y^YvbqqoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQdKmW1D&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKmXbAbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6YEVB_cfoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=lqQdKmYm0&-pn&=jZ&KpYwBm&d>Qd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKmU#JbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6YEOylSLoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hv-di*@cOJB>ZTH7bP>9Ui>OtM_9Za6j?XSX(oeVg}~)_#|9 z^_bR?y;?VGJ?iH_*Jk0Yd7*W1I!-c-c>cVhbE~sQHTtBkVyWg>saaNQzO}4pqh{KQ zdPBXT-cWC-H`E*Iji;yH@IK$qEcbFh5AraN@;FcOa$d=+@jmxH_dfSN_dfSN_db`u zm%o?4m%o?4m%o?4m%o?4m%o?4m%o?4m%o?4|E(r}FMlt8FMls5BY!V{FMlr&$ItmW zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqSzABUgwbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{QT1!Kj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=lqRI3qR-Q{G6ZjbAHax z`8hx5=lqQdKj-KCoS*Y^e$LPN`4@zr^K*XA&-pn& z=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lp!s_&Go4=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHaxzo_wZe$LPN zIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS$!npYwBm z&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqeK zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-If zZ2X*`^K*XA&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=g)QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=luNZ!_WCSKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*aa z%jD-dUiw;Q)!IJUO_!Z4W|HMnu?AyG@wD!Arp@dxi$-D z%?quA({Ykv#PjD3oz)xa4fTe4L%pHiP;XqOdSfw5RWqzq)v#Li!&=s}QI&)H+QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqQdKj-KCoS*Y^e$LO|s_}Dv&d>QdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=lqQd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hv-r|@%r&d>QdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqQd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=luL#8b9af{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN`Hwb!&d>QdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN`Ok!(^K*XA&-pn&=jZ&K zpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=luKw8b9af{G6ZjbAHax z`8hx5=lqQdKj-KCoS*Y^e$LPN`Oh_e&d>QdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN`LBnc^K*XA z&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=luMm8$ajg z{G6ZjbAHax`8hx5=lqTra8|n@9hI->N)fQdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`T3`WpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e*S5VpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`T6IEpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqQdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hw|3_s`R{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN`77b){G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX}PB_&Go4=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax-_ZCuKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=lqKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=lqSz*M*<+bAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{QUXG&-pn&=jZ&KpYwBm&d>Qd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=aK9oegOv(}@2{&Q^>&YBlm2dCpC!-(h48#=c-dsL%O>ME9M zj+L5awdPyPdNyjNtz5~~y4q`Xo!9G1ZscZe<#wLQo!kxI=lgvB8yer|`+T48^L@V0 z_xV2G=lgu0@AG}W&-eL0-{<>$pYQX1zR&mhK3C)WeE)L!K0n{jEcbFh5AraN@;FcO za@^QdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{QSEcKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=lqSzUkpFz=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`T4&JKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqSzJ2!sL&-pn&=jZ&KpYwBm z&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;?;C#3&-pn&=jZ&K zpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*aa_iy~1pYwBm z&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax|9bd2 zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN`F{&P=jZ&KpYwBm&d>QdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj^FMCQdKj-IvKm44Z^K*XA&-pn&=jZ&K zpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=dJN`e$LPNIX~y;{G6Zj zbAHax`8hx5=lqQdKj-KCoS%O|Qd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`T5J?=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hw|YW$p^^K*XA z&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=dU(?&d>Qd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS#3} z_&Go4=lqQdKj-KCoS*Y^e$LPNIX~y;{QR={d5)L9 zmRYs7Pj=H~CySY6xmw+DY&6bpZ4Uc3?=h|YF5~Jkts{H2Zq|C#&wsAX!dde|>)>>p zWEk=Mc|+$`XOC+1NnOQK&9PFmtk!&MS&!$C=aA>nWqS^}&-XLSz1+`(Jj|m!&Xc&$ z-RJIec|CbOdA+Z1@_O=m@_O=m@_O=m@_O=m@_O=m@_O=m@_O=m@_O=m@_O=m@_O=m z@_O=m@_O=max(m!pYwBm&d>QdKmW7g=lqQdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hxTr{U-PoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQdKYxqH&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{QNJ3pYwBm&d>QdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{QR97Kj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqSzLF4EAoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqQdKYyR_bAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZvyYO><&d>QdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hxTna0ogIX~y; z{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e*V|O&-pn& z=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e*Te- zpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{QPP2^BgaIEwgHEpX{c~P8KuCa<#hQ*l3*H z+8p+6-eX$(UB=a8T1WP3-K_PfpZ{E&g|p^`*1_pG$uQ#i^M=l?&K}k1le&tfnq#GA zS*`ijvL5w@dPBXT-cWC-H`E(XTfO0ZzMom{<$fOIVIJjip2Yjy``r87``r87``r6n z{$BoG{$BoG{$BoG{$BoG{$BoG{$BoG{$BoG{{GjS{Js3W{Js3W{Js3W{Js3WJRCpg z=lqQdKj-KCoS*Y^e$LPTR`@wT=jZ&KpYwBm&d>Qd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LN7rSWrq&d>QdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hxTd*SE&oS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKTqT5{G6Zj zbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIY0mW#?SdV zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hw| zZ2X*`^K*XA&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=dUz=&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQd zKj-KCoS#42_&Go4=lqQdKj-KCoS*Y^e$LPNIX~y; z{QUCyd5)L9mRYs7Pj=H~CySY6xmw+DY&6bpZ4Uc3?=h|YF5~Jkts{H2Zq|C#&wsAX z!dde|>)>>pWEk=Mc|+$`XOC+1NnOQK&9PFmtk!&M@f`9T@*KK+&ms5uerCCs`+1Ou zd6dU-pS#c9=kj{;dh&W-*W~r&_2l*B_2l*B_2l*B_2l*B_2l*B_2l*B_2l*B_2l*B z_2l*B_2l*B_2l*B_2gvuIX~y;{G6ZjbAJA3!q53RKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAJBjji2*#e$LPNIX~y;{G6ZjbAHax z`8hx5=lqQdKj-KC{5u;z=jZ&KpYwBm&d>QdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN`M(H1|NrdV_j4@UdKTb! zX=W5gt+cncw)Wm9hkG5N0B6fUfpfxToD&o{Cn#`EP+(i59nLxT-0K{mz&Swy&H)hE zmNOJMCsg2^phylt^R+ZrQkDD%9I2kJ>gv@!t5^3g-Rsjl?}(rCbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;f4=c^e$LPNIX~y;{G6Zj zbAHax`8hx5=lqQdKj-KCoS*;e#?SdVKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIY0l2@N<67&-pn& z=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIY0mRji2*# ze$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC{KLY} z`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAJBI z;phCEpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC{Ob96PL{rwS+%xLchhAjiSz?}eZ9bAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{QPe=e$LPNIX~y;{G6Zj zbAHax`8hx5=lqQdKj-KCoS*aa|JL|9Kj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;efT*)=jZ&K zpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hvd3qR-Q z{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN`3vFa z{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~ZM z{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y; zFExJ7&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKYz0EbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN z`E~U3oGg7UvubUh?xxF57Bk6mwYuThXq?^J9QAG9V_N%N#?@n5NA_ynto67ii`Op; zuW$S@+rzVQl3~P)=MCLk7BZ^QCv_J~b&Zv}%4%KDd&qmpd+0j8hdk%|ndKn&^B@oN zD4uiAx#wJ7PhL-6?;kdKJ$XHOJ$XHOJ$XHOJ$XHOJ$XHOJ$XHOJ$XHOJ$XHOJ$XHO zJ$XHOJ$XHOJ$XGj8Gg>s`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;f3op&e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQdKj-KCoS*;G#?SdVKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIY0lt@N<67&-pn&=jZ&KpYwBm z&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAJA>8b9af{G6ZjbAHax z`8hx5=lqQdKj-KCoS*Y^e$LPN`Fn()^K*XA&-pn& z=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=luM~!_WCSKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=N}q= z&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN z`Cn=LoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=U36sbF%cc%&N70x|=RLSK9oegOv)1D(3SYl0yuR_rY!A=INrn+Go;P%FS;(kHpVVC} z)iqY?Dywxp^@e&wy`kPvZ>Tra8&|R3FrV*dmV?~SgFMWmn9t4U=5zD8`P_VNK9|3j zzn8z4zn8z4zn8z4zn8z4zn8z4zn8z4zn8!NPfh+_{$BoG{$BoG{=TACt+jQd zKj-Ja8-C8u`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y; z{G6Zj^S{yfIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC zoS*Y^e$LPTYvbqqoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKmY9TbAHax`8hx5=lqQdKj-KCoS*Y^ ze$LPNIX~y;{G6ZvzwmQ@&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=lqQdKj-KC{89KhKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqSzi;bW2bAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{QR--bAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6YEMff>C=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=lqQdKj-KFpj|~j&&kr)GOO11>2A90WHFO0SF0P2 zjmFun%~9XxJ*KtaWn4X`b!4yB&03H9`Ny?cIB#BR9iEMo3?p7VZ|L0W>`{$Ask>OJ zYpm2&R_l6eS_O$o)LX!}zQdKj-KC{7*D~&d>QdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=lqQdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAJB4;phCEpYwBm&d>QdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=luL%Hh#{}`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj^LGzF=jZ&KpYwBm&d>Qd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC{5=~#=jZ&KpYwBm z&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj^A8C>=jZ&K zpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC{KFbQ z=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lq2A90WHFO0SF0P2jmFun z%~9XxJ*KtaWn4X`b!4yB&03H9`Ny?cIB#BR9iEMo3?p7VZ|L0W>`{$Ask>OJYpm2& zR_l6eS_O$o)LX!}zwixm>wixm>wixm>wixm>wixm>wixm>wixm>y2kB?lgT&`TMT&`TM zT&`TMoD4tb=lqQdKj-KCoS*Y^e$LPN`R|0E^K*XA z&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=luK=8$ajg z{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN`R_G; z&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN z`Tq?+=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y; z{G6Zj^VRTke$LPNIX~y;{G6ZjbAHax`8hx5=lqQd zKj-KCoS#1oKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqSzTI1*ZoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKYyX|bAHax`8hx5=lqQdKj-KCoS*Y^ ze$LPNIX~y;{G6ZP3qR-Q{G6ZjbAHax`8hx5=lqQd zKj-KCoS*Y^e$LPN`Im>E^K*XA&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=luNf#?SdVKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=U3IwbF%cc%&N70x|=RLSK9oegOv)1E&{&8&<&YPE7hiBs?!-yBp8#=c-dsL%O>MoY*8Y^`b^@e&wy`kPv zZ>Tra8&|d7FrV*dmV?~SgP70F=jLQdKj-KCoS*Y^e$LO|wDEI(&d>Qd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hv-+wgOK z&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LO| zuJLnz&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hv-xA1d*&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC zoS*Y^e$LO|qw#Zo&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hxT;P7*P&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LN7wDEI(&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lq2A90 zWHFO0SF0P2jmFun%~9XxJ*KtaWn4X`b!4yB&03H9`Ny?cIB#BR9iEMo3?p7VZ|L0W z>`{$Ask>OJYpm2&yobDpyoavqd&qOXpIHuaKM&$L_ndpq<@MzCQd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=f53(&d>QdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=fB(dIX~y;{G6ZjbAHax`8hx5 z=lqQdKj-KCoS*Y^e$LN7GyI&N^K*XA&-pn&=jZ&K zpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=l>`CoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqQdKj-J4-S{~_=jZ&KpYwBm z&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX{08e$LPNIX~y; z{G6ZjbAHax`8hx5=lqQdKj-KCoS*aauJLnz&d>Qd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-In!_WCS zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX{1_ z@pFF8&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN z`PKFFoGg7UvubUh?xxF57Bk6mwYuThXq?^J9QAG9V_N%N#?@n5NA_ynto68`e_We| z^X8@2;n_IJFyh7YhR&_d9@XfRx{IZ{#!6jfwXV08^=#CYwsIpk>uzt=ecrA+xfAap z?;-D@tNR}EobP9rgWQk5EBsyI?+Wv|`P_VNK7XUg<;vyC<;vyC<;vyC<;vyC<;vyC z<;vyC<;vyC<;vyC<;vyC<;vyC<;vyC<;uzMbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX{1!@N<67 z&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqK6 zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqSz z_cngc&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqSzkA$D|bAHax`8hx5=lqQdKj-KCoS*Y^e$LPN zIX~y;{QTV;Kj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqSzk2QYI&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqSz&xfD$bAHax`8hx5=lqQdKj-KC zoS*Y^e$LPNIX~y;{QN^2Kj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqSzFE)P8&-pn&=jZ&KpYwBm&d>QdKj-KCoS$D^KhMe1*D|Zt z_UUfA>|`;MELW==j*Z6It<6#2<~^ph-(_4qrgdbm*3DXv`}xPUSvYTAY8{@9lMEwX zJa6dS>g-XCKB>D{s%xy&RaWbIYgx}mU1=*faSzZ-t-pbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN zIX~y;{QTn^Kj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqSz?=*hS&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqSzABCUubAHax`8hx5=lqQdKj-KC zoS*Y^e$LPNIX~y;{QR>TKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqSzO87ZH=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=lp!N@pFF8&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=lqQdKfl-bIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LOoyzz5>&d>QdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqM^Y&d$n%Xdfd-HuFb-E^HS^ZY@B2m@#1+y=T>KrYV=9n#Zp~E zy`kPvZ>Tra8|n@9##OF2%;)==QdKj-KCoS*aaH*Wl# zpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z-#Yx9pYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC zoS*aaw{85KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax-zEHQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*aacWeBdpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHaxKQR29pYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqQdKj-KCoS*aa4{rRNpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=lqxBcNtfYX&u?Cb+gvve*ST77S5ZOT8C%jB*Tao z&l@_oI(t;3PwFm~>Kfid-b3C)*ZDos`8hx5=lqQdKj-KCoS*aa-wZ$J=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`T55+e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQdKj-KCoS*aa-){VzpYwBm&d>QdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*aaKMX(T=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`T1uye$LPNIX~y;{G6Zj zbAHax`8hx5=lqQdKj-KCoS*aaQdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lp!y_&Go4=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHaxA2fc>&-pn& z=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6YEN%%QG z=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hwo z+J2ssrLSdHt?kp@blJ&bCRwglHyj&{vs;^^zRi0~Yro65dQ9ucUagz89{2N)YqN0P zywo~88z&h?ym;Qwxz*XD8huiCu~gSssjIBk_13bUjk?lSZsca&?X9}c+jS>*ayR$# zSRT(4c`{Gs={%EX^IV?KtMcl+LHIu3=lgu0@AG}W&-eL0-{<>$pYQX1zR&mhKHum2 ze4p?0eZKGecD~Q|`94?U`+Wau`#wM4&n)J1^SSxld~QBBpPSF+;l4idaPn~SaPn~S zaPn~SaPn~SaPn~SaPn~SaPn~SaPn~SaPn~SaPn~SaPn~SaQvK~^K*XA&-pn&=jZ(V zjT%4a=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`S~|Ce$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC zoS*aaw+cV!=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`T2K-pYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`T2J@e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*aacMd=2=lqQdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`S}lppYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`T37De$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*aa4+uZ!=lqQdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`T5U=pYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=lqQd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`T5T`e$LPNIX~y;{G6ZjbAHax`8hx5=lqo=NqfhECmg*WSb(Ph+-dfhPQCHf^johrey;b*l zyYA#p?&e+|%j0<>Pv)sSooDiFp3C!jRbCzMA@3pYp;z{M$aB7*S$u!s`vcz}n9qHG z;QIsLAE?{a?do=QyZpWUz5KoWz5KoWz5KoWz5KoWz5KoW{YOXsUjAPGUjAPGUjAPG zUQULe^K*XA&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{QNh<&-pn&=jZ&K zpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{QS2XKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqSzr-h&M zbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{QM8X z&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y; z{Cp|=oS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQd zKj-JO@N<67&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQdKj-In8b9af{G6ZjbAHax`8hx5=lqQd zKj-KCoS*Y^e$LPN`Ik0+&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lq!9&&kr)GOO11>2A90WHFO0SF0P2jmFun%~9XxJ*KtaWn4X` zb!4yB&03H9`Ny?cIB#BR9iEMo3?p7VZ|L0W>`{$Ask=~bs5jIb>J9aVdPBW&)$0xO z`F_mj=5zD8`P_VNJ~yAgI`a4O_wx7h_wx7h_wx7h_wx7h_wx7h_wx7h_wx7h_wx7h z_wx7h_wx7h_wx7haQvK~^K*XA&-pn&=jUG+e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*aaH*EZzpYwBm&d>QdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax-!lB1pYwBm&d>QdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*aaw{HBLpYwBm&d>QdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax-zof@pYwBm&d>Qd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hv-m&VWeIX~y; z{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e*VLapYwBm z&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e*QDz z=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hxT zz{bz{IX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^ ze*SZfpYwBm&d>QdKj-KCoS*Y^etzBkJSR(E%dA@4r@QI0lf_K3T&-?6HX3KQHb;G% z_n6jxmvQx&){(thH)}oa=O5Q*;krIoO{ka=kj{;dh&Ymdh&Ymdh&Ymdh&Ymdh&Ymdh&Ymdh&Ymdh&Ymdh&YmdLI>e zJ$XHOJ$XHOJ$XHOJvkYE&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqSz$25M<&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQdKj-KCoS*Y^ ze$LPNIX~y;pBjG7&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqSzr#F7i&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=lqQdKj-KC zoS*Y^e$LPNIX~y;{G6ZfhoAFve$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCe7W&+e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoSz>we$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*aao$zyh&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqQdKj-KCoS*Y^e$LN#8$ajg{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN`Ij_)&d>QdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=lq2A90WHFO0SF0P2jmFun z%~9XxJ*KtaWn4X`b!4yB&03H9`Ny?cIB#BR9iEMo3?p7VZ|L0W>`{$Ask>OJYpm2& zR_l6eS$pYQX1zR&mhzVF-lKHum2T#fJZ{j2Z$=5zD8`P_VN zJ~yA6&&}uZa9QdKj-KCoS*Y^e$LO|pz(8l&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqQdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hv-i|})P&d>QdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=lqQdKj-KCoS*Y^e$LO|s_}Dv&d>QdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hv-$MAE0&d>QdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LO|x$$#;&d>QdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hv-zwmQ@&d>QdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LN7pz(8l&d>Qd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS(n){X8d2U(2jo+o!wfvXjM3vRtihI5rw* zw>C$8oA;R3ewT6enAVZKS~qJw?&lxZX5qYfsdacZPBM&m@w}mPtFuQn`lRk+sjjh7 zS6Qv=tz|tMb)~J`$j!RjTXmne>rU?EZtmr=Jf0`=WS+{?c_z>1xjY~5A@3pYp;!KU z$aC)d1K%I`{=j_h`vcz}`2IlMu5MSitJ~%8QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=luNF!q53RKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=luLP8b9af{G6ZjbAHax`8hx5=lqQdKj-KC zoS*Y^e$LPN`KN@R^K*XA&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=luMChoAFve$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC{L>mg=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqQd zKj-KCoS*Y^e$LPNIX~y;pC5kC&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=eNSo`8hx5=lqQd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAJAX;phCEpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqQdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=luLS_<2s2zLr_FwoiA{WhaZ7WVu@1aBMWr zZf%bGHt#X5{VwC`F|8wewQkmW+|NI*&BA%}QtR++oMagB;(0^oR%efDboGXML%pHi zP;aO=)En2K-dN01)eI|DHLOwixm>wi zxm>wixm>wixm>wixm>wixm>wixm>wixm>wixm>wixm>wiIT<+_IT<+_e$LPNIX~y; zUlV@L&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC zoS*aauWS6ApYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQd zKj-KCoS*aaZwo)?=lqQdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`T1Kme$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*aa?`Zs-pYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQdKj-KCoS*aa9|%9^=lqQdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`T09Fe$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*aaqsGtqIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e*V7U=lqQdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hxTsqk}t&d>QdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=lqQd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hxTna0ogIX~y;{G6ZjbAHax`8hx5=lq@USKmWKk z3+K&Ct;4f%l3~P)=M9}(ojt12)f?&!^@e&wy`kPvZ(N6ZV=+rrGptnAuv+!QTGq2s zmBUtUQdKj-KCoS*Y^ ze*UZB=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hxTsK(FvIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC zoS*Y^e*Wu?pYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQd zKj-KCoS*Y^e*VA0&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqQdKj-KC zoS*Y^e$LPNIX~y;{CpgK&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN`C{Ye{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~ZT{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;+u`T@oS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKmUU8bAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZfG=9#{`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj^Dk=roS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQdKffM+o|C1oWmc{2)7^C0$zmp1u2wf38;!GD zo1?zXdrWJ;%eZ<>>&RZMo3$SI^N(w@aNfMsIy@UE8AiN#-q5+#*`peLQg^Xb*I22m ztk(6`vYw5)(pGNdX5H_&(p~`+T48^L@V0 z_xV2G=lgu0@AG}W&-eL0-{<>$pYQX1zVG{XzR&mhK3C)WeE)j*K0n{jEC;!t2YHxB zc_A<6rMxCD=e2oVJm;Qs&$*n;S4U1pPDV~fPDV~fPDV~fPDV~fPDV~fPDV~fPDV~f zPDV~fPDV~fPDV~fPDV~fPDV~fPDW0KpYwBm&d>Szs~SJ&=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`T5s2e$LPNIX~y;{G6ZjbAHax z`8hx5=lqQdKj-KCoS*aaHw!=K=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`T4hopYwBm&d>QdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`T4gue$LPNIX~y; z{G6ZjbAHax`8hx5=lqQdKj-KCoS*aaw+}z(=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`T6&UpYwBm z&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`S}kv ze$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*aa z_X$7e=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`T0+VpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`T0*be$LPNIX~y;{G6ZjbAHax`8hx5=lqAS*`1>Wjz~prLEk^&AQuLb)UEEPVVGx?&YyOo+t8T zp32jCCeKE_q25q$s5jIb>J9bAE2G{ppYLaugWS)9Jj|oKkQeh(UXz#e+Pp5lC-gm` z?+MlI>UMRzx?SC_ZdbRf+x^bQ?`-_e#_w$W&c^R-{LaSjY(6}GXXAG^erMx%HhyR0 zcQ$@!BPZi`HhyR0cQ*2H{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIY0l% z#?SdVKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=luMW!q53RKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAJ9Qji2*#e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCd=!4p&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqSzr15ip&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hwo8Gg>s`8hx5=lqQdKj-KC zoS*Y^e$LPNIX~y;{G6Zj^Vfx+^K*XA&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=luLuQdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=lqQd zKj-KCoS*Y^e$LPNIX~y;{G6Zj^Xua0Ia&HzX4Tq0-A$LBEM}7BYIVc0(Kx%cIqKWI z$F%mljH}1Aj_lRCS?h5>|F|{_=gmv4!?SUcVZ@8)4V_z^t=>>?s5jIb>J9aVdgHp( z8;e=0nqj4?hSjPc*0P?BsvNd*Bc5~5x#!$-{%quO<#Oe6<#Oe6<#Oe6<#Oe6<#Oe6 z<#Oe6<#Oe6<#Oe6<#Oe6<#Oe6<#Oe6QdKj-KCoS*Y^e$LPNIX~y;U(@(GKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqi$_&Go4 z=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax|3&yY zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN zIY0l4;phCEpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^ ze$LPNIX{1g#?SdVKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=luLHhoAFve$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC{6B`D^K*XA&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=luMA8$ajg{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN`A;=|&d>QdKj-KCoS*Y^e$LPNIX~y;{G6X(7eCL*($_Mp z*7oUcy6j{zlPp)O8;*^}*{#h{-{w80wcll2J*IVJuhz|4kNf$@wOKfCUTPhljgt%` zUOaE;-0E!ghI&K2q25q$s5jIb*QMTA%u>}1D^)eDR{gM+^=wq-u$3F}oO{ka=bm%9 zT)AAiT)AAiT)AAiT)AAiT)AAiT)AAiT)AAiT)Eu89=Tk(T)AAiT)AAiT)A91895m_ z895n#&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=l>=AoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqQdKj-Hk(fBz(=jZ&KpYwBm z&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zv?eKGc&d>Qd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6YEa^vUx zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKVJwx z=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN z`MB|Oe$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC zoS%PQQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`T6zn^PDVwEwgHEpYEp1P8KuCa<#hQ*l3*H+8p(5-eX$(UB=a8T1WP3 z-K_PvpMPANh4bd6*5TPW$uQ!@^M=l?&K}k1le&wgy2eUfWwoximi27ZmA2wN5%dh&YmdU7)SoS*Y^e*TZc&-pn&=jZ&K zpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKmY2+&-pn& z=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e*VwH z&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y; z{QO%RKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqSzUkE?v=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`T4&IKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqSz+c$pB&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqQdKj-KC zoS*Y^e$LPNIX~y;?;U>5&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=lqQdKj-KCoS*aa_i6l`pYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqQdKj-IH z$Io-J^tH^YwSBsqE<0JwB+J$6hGU~~c58Fgw|S3g?RObhk7*s*t97&1<9_~eZ5Ga( zms*Es<0Qj~7tb3yw>o=NqfhECmg*WSb(Ph+-dfhPQCHfEdPBXT-cWC-H`E*IjjK~{ zn9uh!%R%nvK_2E&UdW4iDX+=PF`t{y&FAKG^SSxld@g@4e=mP8e=mP8e=mP8e=mP8 ze=mP8fB)Aae=mP8e=mP8e=mP8e=jE^e=mP8e=iTm&-pn&=jZ&KpYwBm&d>QdKj-KC zoS*Y^e$LPNIX~y;|2h1epYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHaxf2Hwre$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*-#@N<67&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS%PEQdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqQdKmU8-=lqQd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8huyHGa;|`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj^UrPkoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqQdKj-IL;phCEpYwBm&d>QdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=lqwb|pYwBm&d>QdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=lqSz>l#1j=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHaxualqWWa(>}RcrfnH(hqJ zm`Rqa)eXl+W#%LRn4$cRl{o44{KS^MpX`;bI-Zw+;jePQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQd zKj-KCoS*-*@bmxQuKkC`dDj6P|6aYNw=Qe0={3D(X?ji5_L`>Y^>_OF(hUhiG_9gR zM6yjr5Rv!;2}2|sV}vmzh)5KXATq)j5@d`phJ-Oj5Rov31Q7{iNEk!H7!pM6p1TY> zyU&07Z@ixJ^m(3lo_mr%a_#qcefT*)=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e*RL9pYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=lqQdKj-KCoS*Y^e$LMy%<*%6&d>QdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hwonfN(B=jZ&KpYwBm&d>QdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-IP&hc}8&d>QdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=lqSzXUWe~bGw~w)7e>R>oT>bj?|LwNrmjJwb4G8Pvz{eax?eQ9Xqn? z)|Aig*pZbh*>@{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;kLCC|Kj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS(m+_&Go4=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHaxe~{zn{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIY0j($ItmWKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~Z)QdKj-KC zoS*Y^e$LPNIX~y;{G6YEIzPXiZ7JJ|+1{E;sU;P%V=>zsTRu(iOldsZ^V!uevwb!@ zZ)`JLhqQb<*F4+UxvhD;yAQWDUir!U=h|ArzS&j?`$%y|*yoE|Kk41yH|^T`LGSLP zyNcm+7Iw9Vz0h6?`&fHN*caQ&VK46P4Ey-*uCOod?hbpoR0;b;sVD5qrQWbt_8bZO zWp7Lt&rm{CwCqJBPzQ^1#DkpMT&BVNYFO4Et!;NZ1#;ZiT(D|0`i1 z+yB+DFYX@=d$IfLVIS}QM%b6SzZv%Ofw8bp9C$SB%Ll#{_DbdPuuoRT!@g4acG#;2 zzZ3SUgWnDN>cNSy*LuDm_UWD|PKOOeD!_S0$^YC=oM~*xj_W2{vg*_epY1l`P&V+s8=+DAlsQx_cW7S`TeX%+l z_TsT$g?;?kufx7{>^EUA_sxZUqVKn1U+()|*el1M5Buct`LM4Xe{XcVV^$nQrOo{yd3tyfyJ=T47?Kd^?_HzUO)NAu+N@c3j4;%*TOzrdp+!P zwKu}PSz8YK$f-Yteg4#+!=6t6CG4Z8SHiw<`pvKx&b$@&u`_=S`{J3^uout%E$riG z{~q?Gv;PQt`P^FAC(gYc_T_W$guQb9pJAUozaI9L^Y4bedf~mWPhI#|*jF!XguOQS ze%Pl6KM4EU;J?E@cyTlAGZ+68_VtVZ4SW64|H3|dDK!`TTmOotJL~^T+rmD4IUn}9 z%PnEwyj%$T$dw&opTE)?_H=b;*hjAx!@h8}J?w>RrLd1(>j?YewQ|^t*E_>Le!VN~ zOV_)@UanWdK2h%p`*OWE?3JM-VV@kThJ9tIFYMJD{b8TFF%b6E8@2!ID>s{$&EK`g z*bwilYO$T$CPZCd<=+><-eeKkTk2gY- literal 0 HcmV?d00001 diff --git a/tests/queries/0_stateless/maps b/tests/queries/0_stateless/maps new file mode 100644 index 00000000000..e69de29bb2d From ee167e9ee8033301515f3b0f29617733428fd798 Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Thu, 1 Jul 2021 21:01:49 +0300 Subject: [PATCH 159/599] Remove extra file --- .../nested_nested_arrow.cpp | 178 ------------------ 1 file changed, 178 deletions(-) delete mode 100644 tests/queries/0_stateless/data_orc_arrow_parquet_nested/nested_nested_arrow.cpp diff --git a/tests/queries/0_stateless/data_orc_arrow_parquet_nested/nested_nested_arrow.cpp b/tests/queries/0_stateless/data_orc_arrow_parquet_nested/nested_nested_arrow.cpp deleted file mode 100644 index 2eb1739453c..00000000000 --- a/tests/queries/0_stateless/data_orc_arrow_parquet_nested/nested_nested_arrow.cpp +++ /dev/null @@ -1,178 +0,0 @@ -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -void write_arrow(const arrow::Table & table) -{ - auto file = arrow::io::FileOutputStream::Open("nested_nested_table.arrow"); - - auto writer = arrow::ipc::MakeFileWriter(file->get(), table.schema()).ValueOrDie(); - - auto status = writer->WriteTable(table, 100000); - - if (!status.ok()) - throw std::runtime_error(status.ToString()); - - status = writer->Close(); - - if (!status.ok()) - throw std::runtime_error(status.ToString()); -} - -void write_parquet(const arrow::Table & table) -{ - auto file = arrow::io::FileOutputStream::Open("nested_nested_table.parquet"); - - std::unique_ptr writer; - parquet::WriterProperties::Builder prop_builder; - auto props = prop_builder.build(); - auto status = parquet::arrow::FileWriter::Open( - *table.schema(), - arrow::default_memory_pool(), - *file, - props, - &writer); - - - status = writer->WriteTable(table, 100000); - - if (!status.ok()) - throw std::runtime_error(status.ToString()); - - status = writer->Close(); - - if (!status.ok()) - throw std::runtime_error(status.ToString()); -} - -void write_orc(const arrow::Table & table) -{ - auto file = arrow::io::FileOutputStream::Open("nested_nested_table.orc"); - - auto writer = arrow::adapters::orc::ORCFileWriter::Open(file->get()).ValueOrDie(); - - auto status = writer->Write(table); - - if (!status.ok()) - throw std::runtime_error(status.ToString()); - - status = writer->Close(); - - if (!status.ok()) - throw std::runtime_error(status.ToString()); -} - - -void fillNested(arrow::ArrayBuilder * builder, bool nested) -{ - arrow::ListBuilder * list_builder = static_cast(builder); - arrow::StructBuilder * struct_builder = static_cast(list_builder->value_builder()); - arrow::Int32Builder * elem1_builder = static_cast(struct_builder->field_builder(0)); - arrow::BinaryBuilder * elem2_builder = static_cast(struct_builder->field_builder(1)); - arrow::FloatBuilder * elem3_builder = static_cast(struct_builder->field_builder(2)); - - arrow::ListBuilder * nested_list_builder = nullptr; - if (nested) - nested_list_builder = static_cast(struct_builder->field_builder(3)); - - arrow::Status status; - status = list_builder->Append(); - - std::vector elem1 = {1, 2, 3}; - std::vector elem2 = {"123", "456", "789"}; - std::vector elem3 = {9.8, 10.12, 11.14}; - status = elem1_builder->AppendValues(elem1); - status = elem2_builder->AppendValues(elem2); - status = elem3_builder->AppendValues(elem3); - if (nested) - fillNested(nested_list_builder, false); - - for (size_t i = 0; i != elem1.size(); ++i) - status = struct_builder->Append(); - - status = list_builder->Append(); - - elem1 = {4, 5, 6}; - elem2 = {"101112", "131415", "161718"}; - elem3 = {123.8, 10.2, 11.414}; - status = elem1_builder->AppendValues(elem1); - status = elem2_builder->AppendValues(elem2); - status = elem3_builder->AppendValues(elem3); - if (nested) - fillNested(nested_list_builder, false); - - for (size_t i = 0; i != elem1.size(); ++i) - status = struct_builder->Append(); - - status = list_builder->Append(); - - elem1 = {7, 8, 9}; - elem2 = {"101", "415", "118"}; - elem3 = {13.08, 1.12, 0.414}; - status = elem1_builder->AppendValues(elem1); - status = elem2_builder->AppendValues(elem2); - status = elem3_builder->AppendValues(elem3); - if (nested) - fillNested(nested_list_builder, false); - - for (size_t i = 0; i != elem1.size(); ++i) - status = struct_builder->Append(); -} - -int main() -{ - std::vector> nested_struct_fields; - nested_struct_fields.push_back(std::make_shared("elem1", arrow::int32())); - nested_struct_fields.push_back(std::make_shared("elem2", arrow::binary())); - nested_struct_fields.push_back(std::make_shared("elem3", arrow::float32())); - auto nested_struct_type = arrow::struct_(nested_struct_fields); - auto nested_field = std::make_shared("nested", nested_struct_type); - auto nested_list_type = arrow::list(nested_field); - auto nested_list_field = std::make_shared("nested", nested_list_type); - - std::vector> struct_fields; - struct_fields.push_back(std::make_shared("elem1", arrow::int32())); - struct_fields.push_back(std::make_shared("elem2", arrow::binary())); - struct_fields.push_back(std::make_shared("elem3", arrow::float32())); - struct_fields.push_back(std::make_shared("nested", nested_list_type)); - - - auto struct_type = arrow::struct_(struct_fields); - auto field = std::make_shared("table", struct_type); - auto list_type = arrow::list(field); - - arrow::MemoryPool* pool = arrow::default_memory_pool(); - std::unique_ptr tmp; - auto status = MakeBuilder(pool, list_type, &tmp); - - if (!status.ok()) - throw std::runtime_error(status.ToString()); - - fillNested(tmp.get(), true); - - std::shared_ptr array; - status = tmp->Finish(&array); - - if (!status.ok()) - throw std::runtime_error(status.ToString()); - - std::vector> fields_for_schema = {std::make_shared("table", list_type)}; - auto schema = std::make_shared(std::move(fields_for_schema)); - auto table = arrow::Table::Make(schema, {array}); - - if (!table) - throw std::runtime_error("WTF"); - - write_orc(*table); - write_arrow(*table); - write_parquet(*table); - - return 0; -} From 0f832fcf8bdcaf0b6cb564914fe7ab693dd27ac6 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 5 Aug 2021 14:45:38 +0300 Subject: [PATCH 160/599] Add missing file --- src/Common/clickhouse_malloc.cpp | 29 +++++++++++++++++++++++++++++ 1 file changed, 29 insertions(+) create mode 100644 src/Common/clickhouse_malloc.cpp diff --git a/src/Common/clickhouse_malloc.cpp b/src/Common/clickhouse_malloc.cpp new file mode 100644 index 00000000000..5e37c8b3a2d --- /dev/null +++ b/src/Common/clickhouse_malloc.cpp @@ -0,0 +1,29 @@ +#if 0 + +#include +#include + + +extern "C" void * clickhouse_malloc(size_t size) +{ + void * res = malloc(size); + if (res) + Memory::trackMemory(size); + return res; +} + +extern "C" void clickhouse_free(void * ptr) +{ + Memory::untrackMemory(ptr); + free(ptr); +} + +extern "C" int clickhouse_posix_memalign(void ** memptr, size_t alignment, size_t size) +{ + int res = posix_memalign(memptr, alignment, size); + if (res == 0) + Memory::trackMemory(size); + return res; +} + +#endif From 1a607af42920eb18016526b5682f00484ef888a5 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 3 Aug 2021 17:53:18 +0300 Subject: [PATCH 161/599] add test for parsing maps --- src/DataTypes/DataTypeMap.cpp | 2 +- .../0_stateless/02002_parse_map_int_key.reference | 1 + tests/queries/0_stateless/02002_parse_map_int_key.sql | 11 +++++++++++ 3 files changed, 13 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02002_parse_map_int_key.reference create mode 100644 tests/queries/0_stateless/02002_parse_map_int_key.sql diff --git a/src/DataTypes/DataTypeMap.cpp b/src/DataTypes/DataTypeMap.cpp index 8fd375aa86e..b0bf459b4ca 100644 --- a/src/DataTypes/DataTypeMap.cpp +++ b/src/DataTypes/DataTypeMap.cpp @@ -79,7 +79,7 @@ void DataTypeMap::assertKeyType() const std::string DataTypeMap::doGetName() const { WriteBufferFromOwnString s; - s << "Map(" << key_type->getName() << "," << value_type->getName() << ")"; + s << "Map(" << key_type->getName() << ", " << value_type->getName() << ")"; return s.str(); } diff --git a/tests/queries/0_stateless/02002_parse_map_int_key.reference b/tests/queries/0_stateless/02002_parse_map_int_key.reference new file mode 100644 index 00000000000..dc02589d4bc --- /dev/null +++ b/tests/queries/0_stateless/02002_parse_map_int_key.reference @@ -0,0 +1 @@ +{1:2,3:4,5:6,7:8} {'2021-05-20':1,'2021-05-21':2,'2021-05-22':3,'2021-05-23':4} diff --git a/tests/queries/0_stateless/02002_parse_map_int_key.sql b/tests/queries/0_stateless/02002_parse_map_int_key.sql new file mode 100644 index 00000000000..ecd2a090975 --- /dev/null +++ b/tests/queries/0_stateless/02002_parse_map_int_key.sql @@ -0,0 +1,11 @@ +SET allow_experimental_map_type = 1; + +DROP TABLE IF EXISTS t_map_int_key; +CREATE TABLE t_map_int_key (m1 Map(UInt32, UInt32), m2 Map(Date, UInt32)) ENGINE = Memory; + +INSERT INTO t_map_int_key FORMAT CSV "{1:2, 3: 4, 5 :6, 7 : 8}","{'2021-05-20':1, '2021-05-21': 2, '2021-05-22' :3, '2021-05-23' : 4}" +; + +SELECT m1, m2 FROM t_map_int_key; + +DROP TABLE t_map_int_key; From 607d4dcc0b71e38d4b0916906c69253c0e49722f Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Thu, 5 Aug 2021 18:09:48 +0300 Subject: [PATCH 162/599] Resolve conflicts, fix build and tests --- .../Formats/Impl/ArrowColumnToCHColumn.cpp | 901 +++++++++--------- .../Formats/Impl/ArrowColumnToCHColumn.h | 5 +- .../Formats/Impl/CHColumnToArrowColumn.cpp | 4 +- .../00900_long_parquet_load.reference | 8 +- .../nonnullable.impala.parquet.columns | 2 +- .../nullable.impala.parquet.columns | 2 +- 6 files changed, 477 insertions(+), 445 deletions(-) diff --git a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp index 93b49f332fc..54340213985 100644 --- a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp +++ b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp @@ -2,6 +2,7 @@ #include "ArrowColumnToCHColumn.h" #if USE_ARROW || USE_ORC || USE_PARQUET + #include #include #include @@ -10,7 +11,8 @@ #include #include #include -#include +#include +#include #include #include #include @@ -23,532 +25,561 @@ #include #include #include +#include #include #include -#include #include #include +#define FOR_ARROW_NUMERIC_TYPES(M) \ + M(arrow::Type::UINT8, DB::UInt8) \ + M(arrow::Type::INT8, DB::Int8) \ + M(arrow::Type::UINT16, DB::UInt16) \ + M(arrow::Type::INT16, DB::Int16) \ + M(arrow::Type::UINT32, DB::UInt32) \ + M(arrow::Type::INT32, DB::Int32) \ + M(arrow::Type::UINT64, DB::UInt64) \ + M(arrow::Type::INT64, DB::Int64) \ + M(arrow::Type::HALF_FLOAT, DB::Float32) \ + M(arrow::Type::FLOAT, DB::Float32) \ + M(arrow::Type::DOUBLE, DB::Float64) + +#define FOR_ARROW_INDEXES_TYPES(M) \ + M(arrow::Type::UINT8, DB::UInt8) \ + M(arrow::Type::INT8, DB::UInt8) \ + M(arrow::Type::UINT16, DB::UInt16) \ + M(arrow::Type::INT16, DB::UInt16) \ + M(arrow::Type::UINT32, DB::UInt32) \ + M(arrow::Type::INT32, DB::UInt32) \ + M(arrow::Type::UINT64, DB::UInt64) \ + M(arrow::Type::INT64, DB::UInt64) + namespace DB { - namespace ErrorCodes + +namespace ErrorCodes +{ + extern const int UNKNOWN_TYPE; + extern const int VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE; + extern const int CANNOT_CONVERT_TYPE; + extern const int CANNOT_INSERT_NULL_IN_ORDINARY_COLUMN; + extern const int THERE_IS_NO_COLUMN; + extern const int BAD_ARGUMENTS; + extern const int UNKNOWN_EXCEPTION; +} + + +/// Inserts numeric data right into internal column data to reduce an overhead +template > +static ColumnWithTypeAndName readColumnWithNumericData(std::shared_ptr & arrow_column, const String & column_name) +{ + auto internal_type = std::make_shared>(); + auto internal_column = internal_type->createColumn(); + auto & column_data = static_cast(*internal_column).getData(); + column_data.reserve(arrow_column->length()); + + for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i) { - extern const int UNKNOWN_TYPE; - extern const int VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE; - extern const int CANNOT_CONVERT_TYPE; - extern const int CANNOT_INSERT_NULL_IN_ORDINARY_COLUMN; - extern const int THERE_IS_NO_COLUMN; - extern const int BAD_ARGUMENTS; - extern const int UNKNOWN_EXCEPTION; + std::shared_ptr chunk = arrow_column->chunk(chunk_i); + /// buffers[0] is a null bitmap and buffers[1] are actual values + std::shared_ptr buffer = chunk->data()->buffers[1]; + + const auto * raw_data = reinterpret_cast(buffer->data()); + column_data.insert_assume_reserved(raw_data, raw_data + chunk->length()); } + return {std::move(internal_column), std::move(internal_type), column_name}; +} - static void checkStatus(const arrow::Status & status, const String & column_name, const String & format_name) +/// Inserts chars and offsets right into internal column data to reduce an overhead. +/// Internal offsets are shifted by one to the right in comparison with Arrow ones. So the last offset should map to the end of all chars. +/// Also internal strings are null terminated. +static ColumnWithTypeAndName readColumnWithStringData(std::shared_ptr & arrow_column, const String & column_name) +{ + auto internal_type = std::make_shared(); + auto internal_column = internal_type->createColumn(); + PaddedPODArray & column_chars_t = assert_cast(*internal_column).getChars(); + PaddedPODArray & column_offsets = assert_cast(*internal_column).getOffsets(); + + size_t chars_t_size = 0; + for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i) { - if (!status.ok()) - throw Exception{fmt::format("Error with a {} column \"{}\": {}.", format_name, column_name, status.ToString()), ErrorCodes::UNKNOWN_EXCEPTION}; - } + arrow::BinaryArray & chunk = dynamic_cast(*(arrow_column->chunk(chunk_i))); + const size_t chunk_length = chunk.length(); - /// Inserts numeric data right into internal column data to reduce an overhead - template > - static ColumnWithTypeAndName readColumnWithNumericData(std::shared_ptr & arrow_column, const String & column_name) - { - auto internal_type = std::make_shared>(); - auto internal_column = internal_type->createColumn(); - auto & column_data = static_cast(*internal_column).getData(); - column_data.reserve(arrow_column->length()); - - for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i) + if (chunk_length > 0) { - std::shared_ptr chunk = arrow_column->chunk(chunk_i); - /// buffers[0] is a null bitmap and buffers[1] are actual values - std::shared_ptr buffer = chunk->data()->buffers[1]; - - const auto * raw_data = reinterpret_cast(buffer->data()); - column_data.insert_assume_reserved(raw_data, raw_data + chunk->length()); + chars_t_size += chunk.value_offset(chunk_length - 1) + chunk.value_length(chunk_length - 1); + chars_t_size += chunk_length; /// additional space for null bytes } - return {std::move(internal_column), std::move(internal_type), column_name}; } - /// Inserts chars and offsets right into internal column data to reduce an overhead. - /// Internal offsets are shifted by one to the right in comparison with Arrow ones. So the last offset should map to the end of all chars. - /// Also internal strings are null terminated. - static ColumnWithTypeAndName readColumnWithStringData(std::shared_ptr & arrow_column, const String & column_name) + column_chars_t.reserve(chars_t_size); + column_offsets.reserve(arrow_column->length()); + + for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i) { - auto internal_type = std::make_shared(); - auto internal_column = internal_type->createColumn(); - PaddedPODArray & column_chars_t = assert_cast(*internal_column).getChars(); - PaddedPODArray & column_offsets = assert_cast(*internal_column).getOffsets(); + arrow::BinaryArray & chunk = dynamic_cast(*(arrow_column->chunk(chunk_i))); + std::shared_ptr buffer = chunk.value_data(); + const size_t chunk_length = chunk.length(); - size_t chars_t_size = 0; - for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i) + for (size_t offset_i = 0; offset_i != chunk_length; ++offset_i) { - arrow::BinaryArray & chunk = static_cast(*(arrow_column->chunk(chunk_i))); - const size_t chunk_length = chunk.length(); - - if (chunk_length > 0) + if (!chunk.IsNull(offset_i) && buffer) { - chars_t_size += chunk.value_offset(chunk_length - 1) + chunk.value_length(chunk_length - 1); - chars_t_size += chunk_length; /// additional space for null bytes + const auto * raw_data = buffer->data() + chunk.value_offset(offset_i); + column_chars_t.insert_assume_reserved(raw_data, raw_data + chunk.value_length(offset_i)); } + column_chars_t.emplace_back('\0'); + + column_offsets.emplace_back(column_chars_t.size()); } - - column_chars_t.reserve(chars_t_size); - column_offsets.reserve(arrow_column->length()); - - for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i) - { - arrow::BinaryArray & chunk = static_cast(*(arrow_column->chunk(chunk_i))); - std::shared_ptr buffer = chunk.value_data(); - const size_t chunk_length = chunk.length(); - - for (size_t offset_i = 0; offset_i != chunk_length; ++offset_i) - { - if (!chunk.IsNull(offset_i) && buffer) - { - const auto * raw_data = buffer->data() + chunk.value_offset(offset_i); - column_chars_t.insert_assume_reserved(raw_data, raw_data + chunk.value_length(offset_i)); - } - column_chars_t.emplace_back('\0'); - - column_offsets.emplace_back(column_chars_t.size()); - } - } - return {std::move(internal_column), std::move(internal_type), column_name}; } + return {std::move(internal_column), std::move(internal_type), column_name}; +} - static ColumnWithTypeAndName readColumnWithBooleanData(std::shared_ptr & arrow_column, const String & column_name) +static ColumnWithTypeAndName readColumnWithBooleanData(std::shared_ptr & arrow_column, const String & column_name) +{ + auto internal_type = std::make_shared(); + auto internal_column = internal_type->createColumn(); + auto & column_data = assert_cast &>(*internal_column).getData(); + column_data.reserve(arrow_column->length()); + + for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i) { - auto internal_type = std::make_shared(); - auto internal_column = internal_type->createColumn(); - auto & column_data = assert_cast &>(*internal_column).getData(); - column_data.reserve(arrow_column->length()); + arrow::BooleanArray & chunk = dynamic_cast(*(arrow_column->chunk(chunk_i))); + /// buffers[0] is a null bitmap and buffers[1] are actual values + std::shared_ptr buffer = chunk.data()->buffers[1]; - for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i) - { - arrow::BooleanArray & chunk = static_cast(*(arrow_column->chunk(chunk_i))); - /// buffers[0] is a null bitmap and buffers[1] are actual values - std::shared_ptr buffer = chunk.data()->buffers[1]; - - for (size_t bool_i = 0; bool_i != static_cast(chunk.length()); ++bool_i) - column_data.emplace_back(chunk.Value(bool_i)); - } - return {std::move(internal_column), std::move(internal_type), column_name}; + for (size_t bool_i = 0; bool_i != static_cast(chunk.length()); ++bool_i) + column_data.emplace_back(chunk.Value(bool_i)); } + return {std::move(internal_column), std::move(internal_type), column_name}; +} - static ColumnWithTypeAndName readColumnWithDate32Data(std::shared_ptr & arrow_column, const String & column_name) +static ColumnWithTypeAndName readColumnWithDate32Data(std::shared_ptr & arrow_column, const String & column_name) +{ + auto internal_type = std::make_shared(); + auto internal_column = internal_type->createColumn(); + PaddedPODArray & column_data = assert_cast &>(*internal_column).getData(); + column_data.reserve(arrow_column->length()); + + for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i) { - auto internal_type = std::make_shared(); - auto internal_column = internal_type->createColumn(); - PaddedPODArray & column_data = assert_cast &>(*internal_column).getData(); - column_data.reserve(arrow_column->length()); + arrow::Date32Array & chunk = dynamic_cast(*(arrow_column->chunk(chunk_i))); - for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i) + for (size_t value_i = 0, length = static_cast(chunk.length()); value_i < length; ++value_i) { - arrow::Date32Array & chunk = static_cast(*(arrow_column->chunk(chunk_i))); - - for (size_t value_i = 0, length = static_cast(chunk.length()); value_i < length; ++value_i) + Int32 days_num = static_cast(chunk.Value(value_i)); + if (days_num > DATE_LUT_MAX_EXTEND_DAY_NUM) { - Int32 days_num = static_cast(chunk.Value(value_i)); - if (days_num > DATE_LUT_MAX_EXTEND_DAY_NUM) - { - // TODO: will it rollback correctly? - throw Exception - { - fmt::format("Input value {} of a column \"{}\" is greater than max allowed Date value, which is {}", days_num, column_name, DATE_LUT_MAX_DAY_NUM), - ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE - }; - } - - column_data.emplace_back(days_num); - } - } - return {std::move(internal_column), std::move(internal_type), column_name}; - } - - /// Arrow stores Parquet::DATETIME in Int64, while ClickHouse stores DateTime in UInt32. Therefore, it should be checked before saving - static ColumnWithTypeAndName readColumnWithDate64Data(std::shared_ptr & arrow_column, const String & column_name) - { - auto internal_type = std::make_shared(); - auto internal_column = internal_type->createColumn(); - auto & column_data = assert_cast &>(*internal_column).getData(); - column_data.reserve(arrow_column->length()); - - for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i) - { - auto & chunk = static_cast(*(arrow_column->chunk(chunk_i))); - for (size_t value_i = 0, length = static_cast(chunk.length()); value_i < length; ++value_i) - { - auto timestamp = static_cast(chunk.Value(value_i) / 1000); // Always? in ms - column_data.emplace_back(timestamp); - } - } - return {std::move(internal_column), std::move(internal_type), column_name}; - } - - static ColumnWithTypeAndName readColumnWithTimestampData(std::shared_ptr & arrow_column, const String & column_name) - { - auto internal_type = std::make_shared(); - auto internal_column = internal_type->createColumn(); - auto & column_data = assert_cast &>(*internal_column).getData(); - column_data.reserve(arrow_column->length()); - - for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i) - { - auto & chunk = static_cast(*(arrow_column->chunk(chunk_i))); - const auto & type = static_cast(*chunk.type()); - - UInt32 divide = 1; - const auto unit = type.unit(); - switch (unit) - { - case arrow::TimeUnit::SECOND: - divide = 1; - break; - case arrow::TimeUnit::MILLI: - divide = 1000; - break; - case arrow::TimeUnit::MICRO: - divide = 1000000; - break; - case arrow::TimeUnit::NANO: - divide = 1000000000; - break; + // TODO: will it rollback correctly? + throw Exception + { + ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE, + "Input value {} of a column \"{}\" is greater than max allowed Date value, which is {}", days_num, column_name, DATE_LUT_MAX_DAY_NUM, + }; } - for (size_t value_i = 0, length = static_cast(chunk.length()); value_i < length; ++value_i) - { - auto timestamp = static_cast(chunk.Value(value_i) / divide); // ms! TODO: check other 's' 'ns' ... - column_data.emplace_back(timestamp); - } + column_data.emplace_back(days_num); } - return {std::move(internal_column), std::move(internal_type), column_name}; } + return {std::move(internal_column), std::move(internal_type), column_name}; +} - template - static ColumnWithTypeAndName readColumnWithDecimalData(std::shared_ptr & arrow_column, const String & column_name) +/// Arrow stores Parquet::DATETIME in Int64, while ClickHouse stores DateTime in UInt32. Therefore, it should be checked before saving +static ColumnWithTypeAndName readColumnWithDate64Data(std::shared_ptr & arrow_column, const String & column_name) +{ + auto internal_type = std::make_shared(); + auto internal_column = internal_type->createColumn(); + auto & column_data = assert_cast &>(*internal_column).getData(); + column_data.reserve(arrow_column->length()); + + for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i) { - const auto * arrow_decimal_type = static_cast(arrow_column->type().get()); - auto internal_type = std::make_shared>(arrow_decimal_type->precision(), arrow_decimal_type->scale()); - auto internal_column = internal_type->createColumn(); - auto & column = assert_cast &>(*internal_column); - auto & column_data = column.getData(); - column_data.reserve(arrow_column->length()); - - for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i) + auto & chunk = dynamic_cast(*(arrow_column->chunk(chunk_i))); + for (size_t value_i = 0, length = static_cast(chunk.length()); value_i < length; ++value_i) { - auto & chunk = static_cast(*(arrow_column->chunk(chunk_i))); - for (size_t value_i = 0, length = static_cast(chunk.length()); value_i < length; ++value_i) - { - column_data.emplace_back(chunk.IsNull(value_i) ? DecimalType(0) : *reinterpret_cast(chunk.Value(value_i))); // TODO: copy column - } + auto timestamp = static_cast(chunk.Value(value_i) / 1000); // Always? in ms + column_data.emplace_back(timestamp); } - return {std::move(internal_column), std::move(internal_type), column_name}; } + return {std::move(internal_column), std::move(internal_type), column_name}; +} - /// Creates a null bytemap from arrow's null bitmap - static ColumnPtr readByteMapFromArrowColumn(std::shared_ptr & arrow_column) +static ColumnWithTypeAndName readColumnWithTimestampData(std::shared_ptr & arrow_column, const String & column_name) +{ + auto internal_type = std::make_shared(); + auto internal_column = internal_type->createColumn(); + auto & column_data = assert_cast &>(*internal_column).getData(); + column_data.reserve(arrow_column->length()); + + for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i) { - auto nullmap_column = ColumnUInt8::create(); - PaddedPODArray & bytemap_data = assert_cast &>(*nullmap_column).getData(); - bytemap_data.reserve(arrow_column->length()); + auto & chunk = dynamic_cast(*(arrow_column->chunk(chunk_i))); + const auto & type = static_cast(*chunk.type()); - for (size_t chunk_i = 0; chunk_i != static_cast(arrow_column->num_chunks()); ++chunk_i) + UInt32 divide = 1; + const auto unit = type.unit(); + switch (unit) { - std::shared_ptr chunk = arrow_column->chunk(chunk_i); - - for (size_t value_i = 0; value_i != static_cast(chunk->length()); ++value_i) - bytemap_data.emplace_back(chunk->IsNull(value_i)); + case arrow::TimeUnit::SECOND: + divide = 1; + break; + case arrow::TimeUnit::MILLI: + divide = 1000; + break; + case arrow::TimeUnit::MICRO: + divide = 1000000; + break; + case arrow::TimeUnit::NANO: + divide = 1000000000; + break; } - return nullmap_column; - } - static ColumnPtr readOffsetsFromArrowListColumn(std::shared_ptr & arrow_column) - { - auto offsets_column = ColumnUInt64::create(); - ColumnArray::Offsets & offsets_data = assert_cast &>(*offsets_column).getData(); - offsets_data.reserve(arrow_column->length()); - - for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i) + for (size_t value_i = 0, length = static_cast(chunk.length()); value_i < length; ++value_i) { - arrow::ListArray & list_chunk = static_cast(*(arrow_column->chunk(chunk_i))); - auto arrow_offsets_array = list_chunk.offsets(); - auto & arrow_offsets = static_cast(*arrow_offsets_array); - auto start = offsets_data.back(); - for (int64_t i = 1; i < arrow_offsets.length(); ++i) - offsets_data.emplace_back(start + arrow_offsets.Value(i)); + auto timestamp = static_cast(chunk.Value(value_i) / divide); // ms! TODO: check other 's' 'ns' ... + column_data.emplace_back(timestamp); } - return offsets_column; } + return {std::move(internal_column), std::move(internal_type), column_name}; +} - static ColumnPtr readColumnWithIndexesData(std::shared_ptr & arrow_column) +template +static ColumnWithTypeAndName readColumnWithDecimalData(std::shared_ptr & arrow_column, const String & column_name) +{ + const auto * arrow_decimal_type = static_cast(arrow_column->type().get()); + auto internal_type = std::make_shared>(arrow_decimal_type->precision(), arrow_decimal_type->scale()); + auto internal_column = internal_type->createColumn(); + auto & column = assert_cast &>(*internal_column); + auto & column_data = column.getData(); + column_data.reserve(arrow_column->length()); + + for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i) { - switch (arrow_column->type()->id()) + auto & chunk = dynamic_cast(*(arrow_column->chunk(chunk_i))); + for (size_t value_i = 0, length = static_cast(chunk.length()); value_i < length; ++value_i) { + column_data.emplace_back(chunk.IsNull(value_i) ? DecimalType(0) : *reinterpret_cast(chunk.Value(value_i))); // TODO: copy column + } + } + return {std::move(internal_column), std::move(internal_type), column_name}; +} + +/// Creates a null bytemap from arrow's null bitmap +static ColumnPtr readByteMapFromArrowColumn(std::shared_ptr & arrow_column) +{ + auto nullmap_column = ColumnUInt8::create(); + PaddedPODArray & bytemap_data = assert_cast &>(*nullmap_column).getData(); + bytemap_data.reserve(arrow_column->length()); + + for (size_t chunk_i = 0; chunk_i != static_cast(arrow_column->num_chunks()); ++chunk_i) + { + std::shared_ptr chunk = arrow_column->chunk(chunk_i); + + for (size_t value_i = 0; value_i != static_cast(chunk->length()); ++value_i) + bytemap_data.emplace_back(chunk->IsNull(value_i)); + } + return nullmap_column; +} + +static ColumnPtr readOffsetsFromArrowListColumn(std::shared_ptr & arrow_column) +{ + auto offsets_column = ColumnUInt64::create(); + ColumnArray::Offsets & offsets_data = assert_cast &>(*offsets_column).getData(); + offsets_data.reserve(arrow_column->length()); + + for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i) + { + arrow::ListArray & list_chunk = dynamic_cast(*(arrow_column->chunk(chunk_i))); + auto arrow_offsets_array = list_chunk.offsets(); + auto & arrow_offsets = dynamic_cast(*arrow_offsets_array); + auto start = offsets_data.back(); + for (int64_t i = 1; i < arrow_offsets.length(); ++i) + offsets_data.emplace_back(start + arrow_offsets.Value(i)); + } + return offsets_column; +} + +static ColumnPtr readColumnWithIndexesData(std::shared_ptr & arrow_column) +{ + switch (arrow_column->type()->id()) + { # define DISPATCH(ARROW_NUMERIC_TYPE, CPP_NUMERIC_TYPE) \ case ARROW_NUMERIC_TYPE: \ { \ return readColumnWithNumericData(arrow_column, "").column; \ } - FOR_ARROW_INDEXES_TYPES(DISPATCH) + FOR_ARROW_INDEXES_TYPES(DISPATCH) # undef DISPATCH - default: - throw Exception(fmt::format("Unsupported type for indexes in LowCardinality: {}.", arrow_column->type()->name()), ErrorCodes::BAD_ARGUMENTS); - } + default: + throw Exception(fmt::format("Unsupported type for indexes in LowCardinality: {}.", arrow_column->type()->name()), ErrorCodes::BAD_ARGUMENTS); + } +} + +static std::shared_ptr getNestedArrowColumn(std::shared_ptr & arrow_column) +{ + arrow::ArrayVector array_vector; + array_vector.reserve(arrow_column->num_chunks()); + for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i) + { + arrow::ListArray & list_chunk = dynamic_cast(*(arrow_column->chunk(chunk_i))); + std::shared_ptr chunk = list_chunk.values(); + array_vector.emplace_back(std::move(chunk)); + } + return std::make_shared(array_vector); +} + +static ColumnWithTypeAndName readColumnFromArrowColumn( + std::shared_ptr & arrow_column, + const std::string & column_name, + const std::string & format_name, + bool is_nullable, + std::unordered_map> & dictionary_values) +{ + if (!is_nullable && arrow_column->null_count() && arrow_column->type()->id() != arrow::Type::LIST + && arrow_column->type()->id() != arrow::Type::MAP && arrow_column->type()->id() != arrow::Type::STRUCT) + { + auto nested_column = readColumnFromArrowColumn(arrow_column, column_name, format_name, true, dictionary_values); + auto nullmap_column = readByteMapFromArrowColumn(arrow_column); + auto nullable_type = std::make_shared(std::move(nested_column.type)); + auto nullable_column = ColumnNullable::create(std::move(nested_column.column), std::move(nullmap_column)); + return {std::move(nullable_column), std::move(nullable_type), column_name}; } - static std::shared_ptr getNestedArrowColumn(std::shared_ptr & arrow_column) + switch (arrow_column->type()->id()) { - arrow::ArrayVector array_vector; - array_vector.reserve(arrow_column->num_chunks()); - for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i) - { - arrow::ListArray & list_chunk = static_cast(*(arrow_column->chunk(chunk_i))); - std::shared_ptr chunk = list_chunk.values(); - array_vector.emplace_back(std::move(chunk)); - } - return std::make_shared(array_vector); - } - - static ColumnWithTypeAndName readColumnFromArrowColumn( - std::shared_ptr & arrow_column, - const std::string & column_name, - const std::string & format_name, - bool is_nullable, - std::unordered_map> & dictionary_values) - { - if (!is_nullable && arrow_column->null_count() && arrow_column->type()->id() != arrow::Type::LIST - && arrow_column->type()->id() != arrow::Type::MAP && arrow_column->type()->id() != arrow::Type::STRUCT) - { - auto nested_column = readColumnFromArrowColumn(arrow_column, column_name, format_name, true, dictionary_values); - auto nullmap_column = readByteMapFromArrowColumn(arrow_column); - auto nullable_type = std::make_shared(std::move(nested_column.type)); - auto nullable_column = ColumnNullable::create(std::move(nested_column.column), std::move(nullmap_column)); - return {std::move(nullable_column), std::move(nullable_type), column_name}; - } - - switch (arrow_column->type()->id()) - { - case arrow::Type::STRING: - case arrow::Type::BINARY: - //case arrow::Type::FIXED_SIZE_BINARY: - return readColumnWithStringData(arrow_column, column_name); - case arrow::Type::BOOL: - return readColumnWithBooleanData(arrow_column, column_name); - case arrow::Type::DATE32: - return readColumnWithDate32Data(arrow_column, column_name); - case arrow::Type::DATE64: - return readColumnWithDate64Data(arrow_column, column_name); - case arrow::Type::TIMESTAMP: - return readColumnWithTimestampData(arrow_column, column_name); + case arrow::Type::STRING: + case arrow::Type::BINARY: + //case arrow::Type::FIXED_SIZE_BINARY: + return readColumnWithStringData(arrow_column, column_name); + case arrow::Type::BOOL: + return readColumnWithBooleanData(arrow_column, column_name); + case arrow::Type::DATE32: + return readColumnWithDate32Data(arrow_column, column_name); + case arrow::Type::DATE64: + return readColumnWithDate64Data(arrow_column, column_name); + case arrow::Type::TIMESTAMP: + return readColumnWithTimestampData(arrow_column, column_name); #if defined(ARCADIA_BUILD) case arrow::Type::DECIMAL: return readColumnWithDecimalData(arrow_column, column_name); #else - case arrow::Type::DECIMAL128: - return readColumnWithDecimalData(arrow_column, column_name); - case arrow::Type::DECIMAL256: - return readColumnWithDecimalData(arrow_column, column_name); -#endif - case arrow::Type::MAP: - { - auto arrow_nested_column = getNestedArrowColumn(arrow_column); - auto nested_column = readColumnFromArrowColumn(arrow_nested_column, column_name, format_name, false, dictionary_values); - auto offsets_column = readOffsetsFromArrowListColumn(arrow_column); + case arrow::Type::DECIMAL128: + return readColumnWithDecimalData(arrow_column, column_name); + case arrow::Type::DECIMAL256: + return readColumnWithDecimalData(arrow_column, column_name); +#endif + case arrow::Type::MAP: + { + auto arrow_nested_column = getNestedArrowColumn(arrow_column); + auto nested_column = readColumnFromArrowColumn(arrow_nested_column, column_name, format_name, false, dictionary_values); + auto offsets_column = readOffsetsFromArrowListColumn(arrow_column); - const auto * tuple_column = assert_cast(nested_column.column.get()); - const auto * tuple_type = assert_cast(nested_column.type.get()); - auto map_column = ColumnMap::create(std::move(tuple_column->getColumnPtr(0)), std::move(tuple_column->getColumnPtr(1)), std::move(offsets_column)); - auto map_type = std::make_shared(tuple_type->getElements()[0], tuple_type->getElements()[1]); - return {std::move(map_column), std::move(map_type), column_name}; - } - case arrow::Type::LIST: + const auto * tuple_column = assert_cast(nested_column.column.get()); + const auto * tuple_type = assert_cast(nested_column.type.get()); + auto map_column = ColumnMap::create(std::move(tuple_column->getColumnPtr(0)), std::move(tuple_column->getColumnPtr(1)), std::move(offsets_column)); + auto map_type = std::make_shared(tuple_type->getElements()[0], tuple_type->getElements()[1]); + return {std::move(map_column), std::move(map_type), column_name}; + } + case arrow::Type::LIST: + { + auto arrow_nested_column = getNestedArrowColumn(arrow_column); + auto nested_column = readColumnFromArrowColumn(arrow_nested_column, column_name, format_name, false, dictionary_values); + auto offsets_column = readOffsetsFromArrowListColumn(arrow_column); + auto array_column = ColumnArray::create(std::move(nested_column.column), std::move(offsets_column)); + auto array_type = std::make_shared(nested_column.type); + return {std::move(array_column), std::move(array_type), column_name}; + } + case arrow::Type::STRUCT: + { + auto arrow_type = arrow_column->type(); + auto * arrow_struct_type = assert_cast(arrow_type.get()); + std::vector nested_arrow_columns(arrow_struct_type->num_fields()); + for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i) { - auto arrow_nested_column = getNestedArrowColumn(arrow_column); - auto nested_column = readColumnFromArrowColumn(arrow_nested_column, column_name, format_name, false, dictionary_values); - auto offsets_column = readOffsetsFromArrowListColumn(arrow_column); - auto array_column = ColumnArray::create(std::move(nested_column.column), std::move(offsets_column)); - auto array_type = std::make_shared(nested_column.type); - return {std::move(array_column), std::move(array_type), column_name}; + arrow::StructArray & struct_chunk = dynamic_cast(*(arrow_column->chunk(chunk_i))); + for (int i = 0; i < arrow_struct_type->num_fields(); ++i) + nested_arrow_columns[i].emplace_back(struct_chunk.field(i)); } - case arrow::Type::STRUCT: + + Columns tuple_elements; + DataTypes tuple_types; + std::vector tuple_names; + + for (int i = 0; i != arrow_struct_type->num_fields(); ++i) { - auto arrow_type = arrow_column->type(); - auto arrow_struct_type = assert_cast(arrow_type.get()); - std::vector nested_arrow_columns(arrow_struct_type->num_fields()); + auto nested_arrow_column = std::make_shared(nested_arrow_columns[i]); + auto element = readColumnFromArrowColumn(nested_arrow_column, arrow_struct_type->field(i)->name(), format_name, false, dictionary_values); + tuple_elements.emplace_back(std::move(element.column)); + tuple_types.emplace_back(std::move(element.type)); + tuple_names.emplace_back(std::move(element.name)); + } + + auto tuple_column = ColumnTuple::create(std::move(tuple_elements)); + auto tuple_type = std::make_shared(std::move(tuple_types), std::move(tuple_names)); + return {std::move(tuple_column), std::move(tuple_type), column_name}; + } + case arrow::Type::DICTIONARY: + { + auto & dict_values = dictionary_values[column_name]; + /// Load dictionary values only once and reuse it. + if (!dict_values) + { + arrow::ArrayVector dict_array; for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i) { - arrow::StructArray & struct_chunk = static_cast(*(arrow_column->chunk(chunk_i))); - for (int i = 0; i < arrow_struct_type->num_fields(); ++i) - nested_arrow_columns[i].emplace_back(struct_chunk.field(i)); + arrow::DictionaryArray & dict_chunk = dynamic_cast(*(arrow_column->chunk(chunk_i))); + dict_array.emplace_back(dict_chunk.dictionary()); } + auto arrow_dict_column = std::make_shared(dict_array); + auto dict_column = readColumnFromArrowColumn(arrow_dict_column, column_name, format_name, false, dictionary_values); - Columns tuple_elements; - DataTypes tuple_types; - std::vector tuple_names; - - for (int i = 0; i != arrow_struct_type->num_fields(); ++i) - { - auto nested_arrow_column = std::make_shared(nested_arrow_columns[i]); - auto element = readColumnFromArrowColumn(nested_arrow_column, arrow_struct_type->field(i)->name(), format_name, false, dictionary_values); - tuple_elements.emplace_back(std::move(element.column)); - tuple_types.emplace_back(std::move(element.type)); - tuple_names.emplace_back(std::move(element.name)); - } - - auto tuple_column = ColumnTuple::create(std::move(tuple_elements)); - auto tuple_type = std::make_shared(std::move(tuple_types), std::move(tuple_names)); - return {std::move(tuple_column), std::move(tuple_type), column_name}; + /// We should convert read column to ColumnUnique. + auto tmp_lc_column = DataTypeLowCardinality(dict_column.type).createColumn(); + auto tmp_dict_column = IColumn::mutate(assert_cast(tmp_lc_column.get())->getDictionaryPtr()); + static_cast(tmp_dict_column.get())->uniqueInsertRangeFrom(*dict_column.column, 0, dict_column.column->size()); + dict_column.column = std::move(tmp_dict_column); + dict_values = std::make_shared(std::move(dict_column)); } - case arrow::Type::DICTIONARY: + + arrow::ArrayVector indexes_array; + for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i) { - auto & dict_values = dictionary_values[column_name]; - /// Load dictionary values only once and reuse it. - if (!dict_values) - { - arrow::ArrayVector dict_array; - for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i) - { - arrow::DictionaryArray & dict_chunk = static_cast(*(arrow_column->chunk(chunk_i))); - dict_array.emplace_back(dict_chunk.dictionary()); - } - auto arrow_dict_column = std::make_shared(dict_array); - auto dict_column = readColumnFromArrowColumn(arrow_dict_column, column_name, format_name, false, dictionary_values); - - /// We should convert read column to ColumnUnique. - auto tmp_lc_column = DataTypeLowCardinality(dict_column.type).createColumn(); - auto tmp_dict_column = IColumn::mutate(assert_cast(tmp_lc_column.get())->getDictionaryPtr()); - static_cast(tmp_dict_column.get())->uniqueInsertRangeFrom(*dict_column.column, 0, dict_column.column->size()); - dict_column.column = std::move(tmp_dict_column); - dict_values = std::make_shared(std::move(dict_column)); - } - - arrow::ArrayVector indexes_array; - for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i) - { - arrow::DictionaryArray & dict_chunk = static_cast(*(arrow_column->chunk(chunk_i))); - indexes_array.emplace_back(dict_chunk.indices()); - } - - auto arrow_indexes_column = std::make_shared(indexes_array); - auto indexes_column = readColumnWithIndexesData(arrow_indexes_column); - auto lc_column = ColumnLowCardinality::create(dict_values->column, std::move(indexes_column)); - auto lc_type = std::make_shared(dict_values->type); - return {std::move(lc_column), std::move(lc_type), column_name}; + arrow::DictionaryArray & dict_chunk = dynamic_cast(*(arrow_column->chunk(chunk_i))); + indexes_array.emplace_back(dict_chunk.indices()); } + + auto arrow_indexes_column = std::make_shared(indexes_array); + auto indexes_column = readColumnWithIndexesData(arrow_indexes_column); + auto lc_column = ColumnLowCardinality::create(dict_values->column, std::move(indexes_column)); + auto lc_type = std::make_shared(dict_values->type); + return {std::move(lc_column), std::move(lc_type), column_name}; + } # define DISPATCH(ARROW_NUMERIC_TYPE, CPP_NUMERIC_TYPE) \ case ARROW_NUMERIC_TYPE: \ return readColumnWithNumericData(arrow_column, column_name); - FOR_ARROW_NUMERIC_TYPES(DISPATCH) + FOR_ARROW_NUMERIC_TYPES(DISPATCH) # undef DISPATCH - // TODO: support TIMESTAMP_MICROS and TIMESTAMP_MILLIS with truncated micro- and milliseconds? - // TODO: read JSON as a string? - // TODO: read UUID as a string? - default: - throw Exception - { - fmt::format(R"(Unsupported {} type "{}" of an input column "{}".)", format_name, arrow_column->type()->name(), column_name), - ErrorCodes::UNKNOWN_TYPE - }; - } - } - - static Block arrowSchemaToCHHeader(const arrow::Schema & schema, const std::string & format_name) - { - ColumnsWithTypeAndName sample_columns; - for (const auto & field : schema.fields()) - { - /// Create empty arrow column by it's type and convert it to ClickHouse column. - arrow::MemoryPool* pool = arrow::default_memory_pool(); - std::unique_ptr array_builder; - arrow::Status status = MakeBuilder(pool, field->type(), &array_builder); - checkStatus(status, field->name(), format_name); - std::shared_ptr arrow_array; - status = array_builder->Finish(&arrow_array); - checkStatus(status, field->name(), format_name); - arrow::ArrayVector array_vector = {arrow_array}; - auto arrow_column = std::make_shared(array_vector); - std::unordered_map> dict_values; - ColumnWithTypeAndName sample_column = readColumnFromArrowColumn(arrow_column, field->name(), format_name, false, dict_values); - sample_columns.emplace_back(std::move(sample_column)); - } - return Block(std::move(sample_columns)); - } - - ArrowColumnToCHColumn::ArrowColumnToCHColumn( - const Block & header_, const std::string & format_name_, bool import_nested_) - : header(header_), format_name(format_name_), import_nested(import_nested_) - { - } - - ArrowColumnToCHColumn::ArrowColumnToCHColumn( - const arrow::Schema & schema, const std::string & format_name_, bool import_nested_) - : header(arrowSchemaToCHHeader(schema, format_name_)), format_name(format_name_), import_nested(import_nested_) - { - } - - void ArrowColumnToCHColumn::arrowTableToCHChunk(Chunk & res, std::shared_ptr & table) - { - Columns columns_list; - UInt64 num_rows = 0; - - columns_list.reserve(header.rows()); - - using NameToColumnPtr = std::unordered_map>; - - NameToColumnPtr name_to_column_ptr; - for (const auto& column_name : table->ColumnNames()) - { - std::shared_ptr arrow_column = table->GetColumnByName(column_name); - name_to_column_ptr[column_name] = arrow_column; - } - - std::unordered_map nested_tables; - for (size_t column_i = 0, columns = header.columns(); column_i < columns; ++column_i) - { - const ColumnWithTypeAndName & header_column = header.getByPosition(column_i); - - bool read_from_nested = false; - String nested_table_name = Nested::extractTableName(header_column.name); - if (!name_to_column_ptr.contains(header_column.name)) - { - /// Check if it's a column from nested table. - if (import_nested && name_to_column_ptr.contains(nested_table_name)) + // TODO: support TIMESTAMP_MICROS and TIMESTAMP_MILLIS with truncated micro- and milliseconds? + // TODO: read JSON as a string? + // TODO: read UUID as a string? + default: + throw Exception { - if (!nested_tables.contains(nested_table_name)) - { - std::shared_ptr arrow_column = name_to_column_ptr[nested_table_name]; - ColumnsWithTypeAndName cols = {readColumnFromArrowColumn(arrow_column, nested_table_name, format_name, false, dictionary_values)}; - Block block(cols); - nested_tables[nested_table_name] = std::make_shared(Nested::flatten(block)); - } + ErrorCodes::UNKNOWN_TYPE, + "Unsupported {} type '{}' of an input column '{}'.", format_name, arrow_column->type()->name(), column_name, + }; + } +} - read_from_nested = nested_tables[nested_table_name]->has(header_column.name); + +// Creating CH header by arrow schema. Will be useful in task about inserting +// data from file without knowing table structure. +// +//static void checkStatus(const arrow::Status & status, const String & column_name, const String & format_name) +//{ +// if (!status.ok()) +// throw Exception{ErrorCodes::UNKNOWN_EXCEPTION, "Error with a {} column '{}': {}.", format_name, column_name, status.ToString()}; +//} +// +//static Block arrowSchemaToCHHeader(const arrow::Schema & schema, const std::string & format_name) +//{ +// ColumnsWithTypeAndName sample_columns; +// for (const auto & field : schema.fields()) +// { +// /// Create empty arrow column by it's type and convert it to ClickHouse column. +// arrow::MemoryPool* pool = arrow::default_memory_pool(); +// std::unique_ptr array_builder; +// arrow::Status status = MakeBuilder(pool, field->type(), &array_builder); +// checkStatus(status, field->name(), format_name); +// std::shared_ptr arrow_array; +// status = array_builder->Finish(&arrow_array); +// checkStatus(status, field->name(), format_name); +// arrow::ArrayVector array_vector = {arrow_array}; +// auto arrow_column = std::make_shared(array_vector); +// std::unordered_map> dict_values; +// ColumnWithTypeAndName sample_column = readColumnFromArrowColumn(arrow_column, field->name(), format_name, false, dict_values); +// sample_columns.emplace_back(std::move(sample_column)); +// } +// return Block(std::move(sample_columns)); +//} +// +//ArrowColumnToCHColumn::ArrowColumnToCHColumn( +// const arrow::Schema & schema, const std::string & format_name_, bool import_nested_) +// : header(arrowSchemaToCHHeader(schema, format_name_)), format_name(format_name_), import_nested(import_nested_) +//{ +//} + +ArrowColumnToCHColumn::ArrowColumnToCHColumn( + const Block & header_, const std::string & format_name_, bool import_nested_) + : header(header_), format_name(format_name_), import_nested(import_nested_) +{ +} + +void ArrowColumnToCHColumn::arrowTableToCHChunk(Chunk & res, std::shared_ptr & table) +{ + Columns columns_list; + UInt64 num_rows = 0; + + columns_list.reserve(header.rows()); + + using NameToColumnPtr = std::unordered_map>; + + NameToColumnPtr name_to_column_ptr; + for (const auto& column_name : table->ColumnNames()) + { + std::shared_ptr arrow_column = table->GetColumnByName(column_name); + name_to_column_ptr[column_name] = arrow_column; + } + + std::unordered_map nested_tables; + for (size_t column_i = 0, columns = header.columns(); column_i < columns; ++column_i) + { + const ColumnWithTypeAndName & header_column = header.getByPosition(column_i); + + bool read_from_nested = false; + String nested_table_name = Nested::extractTableName(header_column.name); + if (!name_to_column_ptr.contains(header_column.name)) + { + /// Check if it's a column from nested table. + if (import_nested && name_to_column_ptr.contains(nested_table_name)) + { + if (!nested_tables.contains(nested_table_name)) + { + std::shared_ptr arrow_column = name_to_column_ptr[nested_table_name]; + ColumnsWithTypeAndName cols = {readColumnFromArrowColumn(arrow_column, nested_table_name, format_name, false, dictionary_values)}; + Block block(cols); + nested_tables[nested_table_name] = std::make_shared(Nested::flatten(block)); } - - // TODO: What if some columns were not presented? Insert NULLs? What if a column is not nullable? - if (!read_from_nested) - throw Exception{ - fmt::format("Column \"{}\" is not presented in input data.", header_column.name), ErrorCodes::THERE_IS_NO_COLUMN}; + read_from_nested = nested_tables[nested_table_name]->has(header_column.name); } - std::shared_ptr arrow_column = name_to_column_ptr[header_column.name]; - ColumnWithTypeAndName column; - if (read_from_nested) - column = nested_tables[nested_table_name]->getByName(header_column.name); - else - column = readColumnFromArrowColumn(arrow_column, header_column.name, format_name, false, dictionary_values); - - column.column = castColumn(column, header_column.type); - column.type = header_column.type; - num_rows = column.column->size(); - columns_list.push_back(std::move(column.column)); + // TODO: What if some columns were not presented? Insert NULLs? What if a column is not nullable? + if (!read_from_nested) + throw Exception{ErrorCodes::THERE_IS_NO_COLUMN, "Column '{}' is not presented in input data.", header_column.name}; } - res.setColumns(columns_list, num_rows); + std::shared_ptr arrow_column = name_to_column_ptr[header_column.name]; + + ColumnWithTypeAndName column; + if (read_from_nested) + column = nested_tables[nested_table_name]->getByName(header_column.name); + else + column = readColumnFromArrowColumn(arrow_column, header_column.name, format_name, false, dictionary_values); + + column.column = castColumn(column, header_column.type); + column.type = header_column.type; + num_rows = column.column->size(); + columns_list.push_back(std::move(column.column)); } + + res.setColumns(columns_list, num_rows); +} + } #endif diff --git a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.h b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.h index 5917d9a81a7..3d2fae1420f 100644 --- a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.h +++ b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.h @@ -22,9 +22,9 @@ class ArrowColumnToCHColumn public: ArrowColumnToCHColumn(const Block & header_, const std::string & format_name_, bool import_nested_); - /// Create header by arrow schema. It will be useful for inserting + /// Constructor that create header by arrow schema. It will be useful for inserting /// data from file without knowing table structure. - ArrowColumnToCHColumn(const arrow::Schema & schema, const std::string & format_name, bool import_nested_); + /// ArrowColumnToCHColumn(const arrow::Schema & schema, const std::string & format_name, bool import_nested_); void arrowTableToCHChunk(Chunk & res, std::shared_ptr & table); @@ -32,6 +32,7 @@ private: const Block & header; const std::string format_name; bool import_nested; + /// Map {column name : dictionary column}. /// To avoid converting dictionary from Arrow Dictionary /// to LowCardinality every chunk we save it and reuse. diff --git a/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp b/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp index 7f85e26785f..5928a26a7f9 100644 --- a/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp +++ b/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp @@ -45,8 +45,8 @@ M(UINT64, arrow::UInt64Type) \ M(INT64, arrow::Int64Type) \ M(FLOAT, arrow::FloatType) \ - M(DOUBLE, arrow::DoubleType) \ - M(STRING, arrow::StringType) + M(DOUBLE, arrow::DoubleType) \ + M(BINARY, arrow::BinaryType) namespace DB { diff --git a/tests/queries/0_stateless/00900_long_parquet_load.reference b/tests/queries/0_stateless/00900_long_parquet_load.reference index e48df50bd98..f03f56c7125 100644 --- a/tests/queries/0_stateless/00900_long_parquet_load.reference +++ b/tests/queries/0_stateless/00900_long_parquet_load.reference @@ -315,15 +315,15 @@ Code: 33. DB::ParsingEx---tion: Error while reading Parquet data: IOError: Not y 1593604801 abc 42.125 1593604801 def 7.7 === Try load data from nonnullable.impala.parquet -8 [-1] [[-1,-2],[]] {'k1':-1} [{},{'k1':1},{},{}] (-1,[-1],([[(-1)]]),{}) +8 [-1] [[-1,-2],[]] {'k1':-1} [{},{'k1':1},{},{}] (-1,[-1],([[(-1,'nonnullable')]]),{}) === Try load data from nullable.impala.parquet -1 [1,2,3] [[1,2],[3,4]] {'k1':1,'k2':100} [{'k1':1}] (1,[1],([[(10),(-10)],[(11)]]),{'foo':(([1.1]))}) -2 [NULL,1,2,NULL,3,NULL] [[NULL,1,2,NULL],[3,NULL,4],[],[]] {'k1':2,'k2':NULL} [{'k3':NULL,'k1':1},{},{}] (NULL,[NULL],([[(NULL),(10),(NULL),(-10),(NULL)],[(11),(NULL)],[],[]]),{'g1':(([2.2,NULL])),'g2':(([])),'g3':(([])),'g4':(([])),'g5':(([]))}) +1 [1,2,3] [[1,2],[3,4]] {'k1':1,'k2':100} [{'k1':1}] (1,[1],([[(10,'aaa'),(-10,'bbb')],[(11,'c')]]),{'foo':(([1.1]))}) +2 [NULL,1,2,NULL,3,NULL] [[NULL,1,2,NULL],[3,NULL,4],[],[]] {'k1':2,'k2':NULL} [{'k3':NULL,'k1':1},{},{}] (NULL,[NULL],([[(NULL,NULL),(10,'aaa'),(NULL,NULL),(-10,'bbb'),(NULL,NULL)],[(11,'c'),(NULL,NULL)],[],[]]),{'g1':(([2.2,NULL])),'g2':(([])),'g3':(([])),'g4':(([])),'g5':(([]))}) 3 [] [[]] {} [{},{}] (NULL,[],([]),{}) 4 [] [] {} [] (NULL,[],([]),{}) 5 [] [] {} [] (NULL,[],([]),{'foo':(([2.2,3.3]))}) 6 [] [] {} [] (NULL,[],([]),{}) -7 [] [[],[5,6]] {'k1':NULL,'k3':NULL} [] (7,[2,3,NULL],([[],[(NULL)],[]]),{}) +7 [] [[],[5,6]] {'k1':NULL,'k3':NULL} [] (7,[2,3,NULL],([[],[(NULL,NULL)],[]]),{}) === Try load data from nullable_list.parquet [1,NULL,2] [NULL,'Some string',NULL] [0.00,NULL,42.42] [NULL] [NULL] [NULL] diff --git a/tests/queries/0_stateless/data_parquet/nonnullable.impala.parquet.columns b/tests/queries/0_stateless/data_parquet/nonnullable.impala.parquet.columns index 299ec3b6af2..3eafbc1bb02 100644 --- a/tests/queries/0_stateless/data_parquet/nonnullable.impala.parquet.columns +++ b/tests/queries/0_stateless/data_parquet/nonnullable.impala.parquet.columns @@ -1 +1 @@ -`ID` Nullable(Int64), `Int_Array` Array(Nullable(Int32)), `int_array_array` Array(Array(Nullable(Int32))), `Int_Map` Map(String, Nullable(Int32)), `int_map_array` Array(Map(String, Nullable(Int32))), `nested_Struct` Tuple(Nullable(Int32), Array(Nullable(Int32)), Tuple(Array(Array(Tuple(Nullable(Int32))))), Map(String, Tuple(Tuple(Array(Nullable(Float64)))))) +`ID` Nullable(Int64), `Int_Array` Array(Nullable(Int32)), `int_array_array` Array(Array(Nullable(Int32))), `Int_Map` Map(String, Nullable(Int32)), `int_map_array` Array(Map(String, Nullable(Int32))), `nested_Struct` Tuple(Nullable(Int32), Array(Nullable(Int32)), Tuple(Array(Array(Tuple(Nullable(Int32), Nullable(String))))), Map(String, Tuple(Tuple(Array(Nullable(Float64)))))) diff --git a/tests/queries/0_stateless/data_parquet/nullable.impala.parquet.columns b/tests/queries/0_stateless/data_parquet/nullable.impala.parquet.columns index 6fcbcdd4a0b..f8e51d8b43a 100644 --- a/tests/queries/0_stateless/data_parquet/nullable.impala.parquet.columns +++ b/tests/queries/0_stateless/data_parquet/nullable.impala.parquet.columns @@ -1 +1 @@ -`id` Nullable(Int64), `int_array` Array(Nullable(Int32)), `int_array_Array` Array(Array(Nullable(Int32))), `int_map` Map(String, Nullable(Int32)), `int_Map_Array` Array(Map(String, Nullable(Int32))), `nested_struct` Tuple(Nullable(Int32), Array(Nullable(Int32)), Tuple(Array(Array(Tuple(Nullable(Int32))))), Map(String, Tuple(Tuple(Array(Nullable(Float64)))))) +`id` Nullable(Int64), `int_array` Array(Nullable(Int32)), `int_array_Array` Array(Array(Nullable(Int32))), `int_map` Map(String, Nullable(Int32)), `int_Map_Array` Array(Map(String, Nullable(Int32))), `nested_struct` Tuple(Nullable(Int32), Array(Nullable(Int32)), Tuple(Array(Array(Tuple(Nullable(Int32), Nullable(String))))), Map(String, Tuple(Tuple(Array(Nullable(Float64)))))) From a0c10b546f1921ffaf677b53ed9190dac024e62d Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Thu, 5 Aug 2021 18:18:33 +0300 Subject: [PATCH 163/599] Small fixes --- .../Formats/Impl/ArrowColumnToCHColumn.cpp | 22 +++++-------------- 1 file changed, 6 insertions(+), 16 deletions(-) diff --git a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp index 54340213985..4d358b3e84b 100644 --- a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp +++ b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp @@ -1,4 +1,3 @@ -#include "config_formats.h" #include "ArrowColumnToCHColumn.h" #if USE_ARROW || USE_ORC || USE_PARQUET @@ -17,6 +16,7 @@ #include #include #include +#include #include #include #include @@ -25,7 +25,6 @@ #include #include #include -#include #include #include #include @@ -172,14 +171,8 @@ static ColumnWithTypeAndName readColumnWithDate32Data(std::shared_ptr(chunk.Value(value_i)); if (days_num > DATE_LUT_MAX_EXTEND_DAY_NUM) - { - // TODO: will it rollback correctly? - throw Exception - { - ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE, - "Input value {} of a column \"{}\" is greater than max allowed Date value, which is {}", days_num, column_name, DATE_LUT_MAX_DAY_NUM, - }; - } + throw Exception{ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE, + "Input value {} of a column \"{}\" is greater than max allowed Date value, which is {}", days_num, column_name, DATE_LUT_MAX_DAY_NUM}; column_data.emplace_back(days_num); } @@ -314,7 +307,7 @@ static ColumnPtr readColumnWithIndexesData(std::shared_ptr FOR_ARROW_INDEXES_TYPES(DISPATCH) # undef DISPATCH default: - throw Exception(fmt::format("Unsupported type for indexes in LowCardinality: {}.", arrow_column->type()->name()), ErrorCodes::BAD_ARGUMENTS); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unsupported type for indexes in LowCardinality: {}.", arrow_column->type()->name()); } } @@ -466,11 +459,8 @@ static ColumnWithTypeAndName readColumnFromArrowColumn( // TODO: read JSON as a string? // TODO: read UUID as a string? default: - throw Exception - { - ErrorCodes::UNKNOWN_TYPE, - "Unsupported {} type '{}' of an input column '{}'.", format_name, arrow_column->type()->name(), column_name, - }; + throw Exception(ErrorCodes::UNKNOWN_TYPE, + "Unsupported {} type '{}' of an input column '{}'.", format_name, arrow_column->type()->name(), column_name); } } From d2869d2e2b0216373d3504e875fce58726237b4c Mon Sep 17 00:00:00 2001 From: Artur <613623@mail.ru> Date: Thu, 5 Aug 2021 16:20:24 +0000 Subject: [PATCH 164/599] add test for memory tracker in client --- programs/client/Client.cpp | 2 +- src/Core/ya.make | 4 + src/DataStreams/ya.make | 1 + src/Parsers/ya.make | 2 + src/Processors/ya.make | 11 - src/Storages/ya.make | 1 + .../02003_memory_limit_in_client.reference | 60000 ++++++++++++++++ .../02003_memory_limit_in_client.sh | 14 + 8 files changed, 60023 insertions(+), 12 deletions(-) create mode 100644 tests/queries/0_stateless/02003_memory_limit_in_client.reference create mode 100755 tests/queries/0_stateless/02003_memory_limit_in_client.sh diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index f7bbdc15327..2e12c262605 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -461,7 +461,7 @@ private: {TokenType::ErrorSingleQuoteIsNotClosed, Replxx::Color::RED}, {TokenType::ErrorDoubleQuoteIsNotClosed, Replxx::Color::RED}, {TokenType::ErrorSinglePipeMark, Replxx::Color::RED}, - {TokenType::ErrorWrongNumber, Replxx::Color::RED}, + {TokenType::ErrorWrongNumber, Replxx::Color::RED}, {TokenType::ErrorMaxQuerySizeExceeded, Replxx::Color::RED }}; const Replxx::Color unknown_token_color = Replxx::Color::RED; diff --git a/src/Core/ya.make b/src/Core/ya.make index d1e352ee846..6946d7a47bb 100644 --- a/src/Core/ya.make +++ b/src/Core/ya.make @@ -31,6 +31,10 @@ SRCS( MySQL/PacketsProtocolText.cpp MySQL/PacketsReplication.cpp NamesAndTypes.cpp + PostgreSQL/Connection.cpp + PostgreSQL/PoolWithFailover.cpp + PostgreSQL/Utils.cpp + PostgreSQL/insertPostgreSQLValue.cpp PostgreSQLProtocol.cpp QueryProcessingStage.cpp Settings.cpp diff --git a/src/DataStreams/ya.make b/src/DataStreams/ya.make index 2012af76697..b1205828a7e 100644 --- a/src/DataStreams/ya.make +++ b/src/DataStreams/ya.make @@ -49,6 +49,7 @@ SRCS( TTLUpdateInfoAlgorithm.cpp copyData.cpp finalizeBlock.cpp + formatBlock.cpp materializeBlock.cpp narrowBlockInputStreams.cpp diff --git a/src/Parsers/ya.make b/src/Parsers/ya.make index 62e0c2b3225..3b8a9a19bce 100644 --- a/src/Parsers/ya.make +++ b/src/Parsers/ya.make @@ -21,6 +21,7 @@ SRCS( ASTCreateRowPolicyQuery.cpp ASTCreateSettingsProfileQuery.cpp ASTCreateUserQuery.cpp + ASTDatabaseOrNone.cpp ASTDictionary.cpp ASTDictionaryAttributeDeclaration.cpp ASTDropAccessEntityQuery.cpp @@ -95,6 +96,7 @@ SRCS( ParserCreateSettingsProfileQuery.cpp ParserCreateUserQuery.cpp ParserDataType.cpp + ParserDatabaseOrNone.cpp ParserDescribeTableQuery.cpp ParserDictionary.cpp ParserDictionaryAttributeDeclaration.cpp diff --git a/src/Processors/ya.make b/src/Processors/ya.make index 4b95484a828..543a08caca5 100644 --- a/src/Processors/ya.make +++ b/src/Processors/ya.make @@ -7,14 +7,8 @@ PEERDIR( clickhouse/src/Common contrib/libs/msgpack contrib/libs/protobuf - contrib/libs/arrow ) -ADDINCL( - contrib/libs/arrow/src -) - -CFLAGS(-DUSE_ARROW=1) SRCS( Chunk.cpp @@ -31,11 +25,6 @@ SRCS( Formats/IOutputFormat.cpp Formats/IRowInputFormat.cpp Formats/IRowOutputFormat.cpp - Formats/Impl/ArrowBlockInputFormat.cpp - Formats/Impl/ArrowBlockOutputFormat.cpp - Formats/Impl/ArrowBufferedStreams.cpp - Formats/Impl/ArrowColumnToCHColumn.cpp - Formats/Impl/CHColumnToArrowColumn.cpp Formats/Impl/BinaryRowInputFormat.cpp Formats/Impl/BinaryRowOutputFormat.cpp Formats/Impl/CSVRowInputFormat.cpp diff --git a/src/Storages/ya.make b/src/Storages/ya.make index e794124362b..9fb51a4d28f 100644 --- a/src/Storages/ya.make +++ b/src/Storages/ya.make @@ -140,6 +140,7 @@ SRCS( StorageMerge.cpp StorageMergeTree.cpp StorageMongoDB.cpp + StorageMongoDBSocketFactory.cpp StorageMySQL.cpp StorageNull.cpp StorageReplicatedMergeTree.cpp diff --git a/tests/queries/0_stateless/02003_memory_limit_in_client.reference b/tests/queries/0_stateless/02003_memory_limit_in_client.reference new file mode 100644 index 00000000000..9f074d19d33 --- /dev/null +++ b/tests/queries/0_stateless/02003_memory_limit_in_client.reference @@ -0,0 +1,60000 @@ +0 +1 +2 +3 +4 +5 +6 +7 +8 +9 +10 +11 +12 +13 +14 +15 +16 +17 +18 +19 +20 +21 +22 +23 +24 +25 +26 +27 +28 +29 +30 +31 +32 +33 +34 +35 +36 +37 +38 +39 +40 +41 +42 +43 +44 +45 +46 +47 +48 +49 +50 +51 +52 +53 +54 +55 +56 +57 +58 +59 +60 +61 +62 +63 +64 +65 +66 +67 +68 +69 +70 +71 +72 +73 +74 +75 +76 +77 +78 +79 +80 +81 +82 +83 +84 +85 +86 +87 +88 +89 +90 +91 +92 +93 +94 +95 +96 +97 +98 +99 +100 +101 +102 +103 +104 +105 +106 +107 +108 +109 +110 +111 +112 +113 +114 +115 +116 +117 +118 +119 +120 +121 +122 +123 +124 +125 +126 +127 +128 +129 +130 +131 +132 +133 +134 +135 +136 +137 +138 +139 +140 +141 +142 +143 +144 +145 +146 +147 +148 +149 +150 +151 +152 +153 +154 +155 +156 +157 +158 +159 +160 +161 +162 +163 +164 +165 +166 +167 +168 +169 +170 +171 +172 +173 +174 +175 +176 +177 +178 +179 +180 +181 +182 +183 +184 +185 +186 +187 +188 +189 +190 +191 +192 +193 +194 +195 +196 +197 +198 +199 +200 +201 +202 +203 +204 +205 +206 +207 +208 +209 +210 +211 +212 +213 +214 +215 +216 +217 +218 +219 +220 +221 +222 +223 +224 +225 +226 +227 +228 +229 +230 +231 +232 +233 +234 +235 +236 +237 +238 +239 +240 +241 +242 +243 +244 +245 +246 +247 +248 +249 +250 +251 +252 +253 +254 +255 +256 +257 +258 +259 +260 +261 +262 +263 +264 +265 +266 +267 +268 +269 +270 +271 +272 +273 +274 +275 +276 +277 +278 +279 +280 +281 +282 +283 +284 +285 +286 +287 +288 +289 +290 +291 +292 +293 +294 +295 +296 +297 +298 +299 +300 +301 +302 +303 +304 +305 +306 +307 +308 +309 +310 +311 +312 +313 +314 +315 +316 +317 +318 +319 +320 +321 +322 +323 +324 +325 +326 +327 +328 +329 +330 +331 +332 +333 +334 +335 +336 +337 +338 +339 +340 +341 +342 +343 +344 +345 +346 +347 +348 +349 +350 +351 +352 +353 +354 +355 +356 +357 +358 +359 +360 +361 +362 +363 +364 +365 +366 +367 +368 +369 +370 +371 +372 +373 +374 +375 +376 +377 +378 +379 +380 +381 +382 +383 +384 +385 +386 +387 +388 +389 +390 +391 +392 +393 +394 +395 +396 +397 +398 +399 +400 +401 +402 +403 +404 +405 +406 +407 +408 +409 +410 +411 +412 +413 +414 +415 +416 +417 +418 +419 +420 +421 +422 +423 +424 +425 +426 +427 +428 +429 +430 +431 +432 +433 +434 +435 +436 +437 +438 +439 +440 +441 +442 +443 +444 +445 +446 +447 +448 +449 +450 +451 +452 +453 +454 +455 +456 +457 +458 +459 +460 +461 +462 +463 +464 +465 +466 +467 +468 +469 +470 +471 +472 +473 +474 +475 +476 +477 +478 +479 +480 +481 +482 +483 +484 +485 +486 +487 +488 +489 +490 +491 +492 +493 +494 +495 +496 +497 +498 +499 +500 +501 +502 +503 +504 +505 +506 +507 +508 +509 +510 +511 +512 +513 +514 +515 +516 +517 +518 +519 +520 +521 +522 +523 +524 +525 +526 +527 +528 +529 +530 +531 +532 +533 +534 +535 +536 +537 +538 +539 +540 +541 +542 +543 +544 +545 +546 +547 +548 +549 +550 +551 +552 +553 +554 +555 +556 +557 +558 +559 +560 +561 +562 +563 +564 +565 +566 +567 +568 +569 +570 +571 +572 +573 +574 +575 +576 +577 +578 +579 +580 +581 +582 +583 +584 +585 +586 +587 +588 +589 +590 +591 +592 +593 +594 +595 +596 +597 +598 +599 +600 +601 +602 +603 +604 +605 +606 +607 +608 +609 +610 +611 +612 +613 +614 +615 +616 +617 +618 +619 +620 +621 +622 +623 +624 +625 +626 +627 +628 +629 +630 +631 +632 +633 +634 +635 +636 +637 +638 +639 +640 +641 +642 +643 +644 +645 +646 +647 +648 +649 +650 +651 +652 +653 +654 +655 +656 +657 +658 +659 +660 +661 +662 +663 +664 +665 +666 +667 +668 +669 +670 +671 +672 +673 +674 +675 +676 +677 +678 +679 +680 +681 +682 +683 +684 +685 +686 +687 +688 +689 +690 +691 +692 +693 +694 +695 +696 +697 +698 +699 +700 +701 +702 +703 +704 +705 +706 +707 +708 +709 +710 +711 +712 +713 +714 +715 +716 +717 +718 +719 +720 +721 +722 +723 +724 +725 +726 +727 +728 +729 +730 +731 +732 +733 +734 +735 +736 +737 +738 +739 +740 +741 +742 +743 +744 +745 +746 +747 +748 +749 +750 +751 +752 +753 +754 +755 +756 +757 +758 +759 +760 +761 +762 +763 +764 +765 +766 +767 +768 +769 +770 +771 +772 +773 +774 +775 +776 +777 +778 +779 +780 +781 +782 +783 +784 +785 +786 +787 +788 +789 +790 +791 +792 +793 +794 +795 +796 +797 +798 +799 +800 +801 +802 +803 +804 +805 +806 +807 +808 +809 +810 +811 +812 +813 +814 +815 +816 +817 +818 +819 +820 +821 +822 +823 +824 +825 +826 +827 +828 +829 +830 +831 +832 +833 +834 +835 +836 +837 +838 +839 +840 +841 +842 +843 +844 +845 +846 +847 +848 +849 +850 +851 +852 +853 +854 +855 +856 +857 +858 +859 +860 +861 +862 +863 +864 +865 +866 +867 +868 +869 +870 +871 +872 +873 +874 +875 +876 +877 +878 +879 +880 +881 +882 +883 +884 +885 +886 +887 +888 +889 +890 +891 +892 +893 +894 +895 +896 +897 +898 +899 +900 +901 +902 +903 +904 +905 +906 +907 +908 +909 +910 +911 +912 +913 +914 +915 +916 +917 +918 +919 +920 +921 +922 +923 +924 +925 +926 +927 +928 +929 +930 +931 +932 +933 +934 +935 +936 +937 +938 +939 +940 +941 +942 +943 +944 +945 +946 +947 +948 +949 +950 +951 +952 +953 +954 +955 +956 +957 +958 +959 +960 +961 +962 +963 +964 +965 +966 +967 +968 +969 +970 +971 +972 +973 +974 +975 +976 +977 +978 +979 +980 +981 +982 +983 +984 +985 +986 +987 +988 +989 +990 +991 +992 +993 +994 +995 +996 +997 +998 +999 +1000 +1001 +1002 +1003 +1004 +1005 +1006 +1007 +1008 +1009 +1010 +1011 +1012 +1013 +1014 +1015 +1016 +1017 +1018 +1019 +1020 +1021 +1022 +1023 +1024 +1025 +1026 +1027 +1028 +1029 +1030 +1031 +1032 +1033 +1034 +1035 +1036 +1037 +1038 +1039 +1040 +1041 +1042 +1043 +1044 +1045 +1046 +1047 +1048 +1049 +1050 +1051 +1052 +1053 +1054 +1055 +1056 +1057 +1058 +1059 +1060 +1061 +1062 +1063 +1064 +1065 +1066 +1067 +1068 +1069 +1070 +1071 +1072 +1073 +1074 +1075 +1076 +1077 +1078 +1079 +1080 +1081 +1082 +1083 +1084 +1085 +1086 +1087 +1088 +1089 +1090 +1091 +1092 +1093 +1094 +1095 +1096 +1097 +1098 +1099 +1100 +1101 +1102 +1103 +1104 +1105 +1106 +1107 +1108 +1109 +1110 +1111 +1112 +1113 +1114 +1115 +1116 +1117 +1118 +1119 +1120 +1121 +1122 +1123 +1124 +1125 +1126 +1127 +1128 +1129 +1130 +1131 +1132 +1133 +1134 +1135 +1136 +1137 +1138 +1139 +1140 +1141 +1142 +1143 +1144 +1145 +1146 +1147 +1148 +1149 +1150 +1151 +1152 +1153 +1154 +1155 +1156 +1157 +1158 +1159 +1160 +1161 +1162 +1163 +1164 +1165 +1166 +1167 +1168 +1169 +1170 +1171 +1172 +1173 +1174 +1175 +1176 +1177 +1178 +1179 +1180 +1181 +1182 +1183 +1184 +1185 +1186 +1187 +1188 +1189 +1190 +1191 +1192 +1193 +1194 +1195 +1196 +1197 +1198 +1199 +1200 +1201 +1202 +1203 +1204 +1205 +1206 +1207 +1208 +1209 +1210 +1211 +1212 +1213 +1214 +1215 +1216 +1217 +1218 +1219 +1220 +1221 +1222 +1223 +1224 +1225 +1226 +1227 +1228 +1229 +1230 +1231 +1232 +1233 +1234 +1235 +1236 +1237 +1238 +1239 +1240 +1241 +1242 +1243 +1244 +1245 +1246 +1247 +1248 +1249 +1250 +1251 +1252 +1253 +1254 +1255 +1256 +1257 +1258 +1259 +1260 +1261 +1262 +1263 +1264 +1265 +1266 +1267 +1268 +1269 +1270 +1271 +1272 +1273 +1274 +1275 +1276 +1277 +1278 +1279 +1280 +1281 +1282 +1283 +1284 +1285 +1286 +1287 +1288 +1289 +1290 +1291 +1292 +1293 +1294 +1295 +1296 +1297 +1298 +1299 +1300 +1301 +1302 +1303 +1304 +1305 +1306 +1307 +1308 +1309 +1310 +1311 +1312 +1313 +1314 +1315 +1316 +1317 +1318 +1319 +1320 +1321 +1322 +1323 +1324 +1325 +1326 +1327 +1328 +1329 +1330 +1331 +1332 +1333 +1334 +1335 +1336 +1337 +1338 +1339 +1340 +1341 +1342 +1343 +1344 +1345 +1346 +1347 +1348 +1349 +1350 +1351 +1352 +1353 +1354 +1355 +1356 +1357 +1358 +1359 +1360 +1361 +1362 +1363 +1364 +1365 +1366 +1367 +1368 +1369 +1370 +1371 +1372 +1373 +1374 +1375 +1376 +1377 +1378 +1379 +1380 +1381 +1382 +1383 +1384 +1385 +1386 +1387 +1388 +1389 +1390 +1391 +1392 +1393 +1394 +1395 +1396 +1397 +1398 +1399 +1400 +1401 +1402 +1403 +1404 +1405 +1406 +1407 +1408 +1409 +1410 +1411 +1412 +1413 +1414 +1415 +1416 +1417 +1418 +1419 +1420 +1421 +1422 +1423 +1424 +1425 +1426 +1427 +1428 +1429 +1430 +1431 +1432 +1433 +1434 +1435 +1436 +1437 +1438 +1439 +1440 +1441 +1442 +1443 +1444 +1445 +1446 +1447 +1448 +1449 +1450 +1451 +1452 +1453 +1454 +1455 +1456 +1457 +1458 +1459 +1460 +1461 +1462 +1463 +1464 +1465 +1466 +1467 +1468 +1469 +1470 +1471 +1472 +1473 +1474 +1475 +1476 +1477 +1478 +1479 +1480 +1481 +1482 +1483 +1484 +1485 +1486 +1487 +1488 +1489 +1490 +1491 +1492 +1493 +1494 +1495 +1496 +1497 +1498 +1499 +1500 +1501 +1502 +1503 +1504 +1505 +1506 +1507 +1508 +1509 +1510 +1511 +1512 +1513 +1514 +1515 +1516 +1517 +1518 +1519 +1520 +1521 +1522 +1523 +1524 +1525 +1526 +1527 +1528 +1529 +1530 +1531 +1532 +1533 +1534 +1535 +1536 +1537 +1538 +1539 +1540 +1541 +1542 +1543 +1544 +1545 +1546 +1547 +1548 +1549 +1550 +1551 +1552 +1553 +1554 +1555 +1556 +1557 +1558 +1559 +1560 +1561 +1562 +1563 +1564 +1565 +1566 +1567 +1568 +1569 +1570 +1571 +1572 +1573 +1574 +1575 +1576 +1577 +1578 +1579 +1580 +1581 +1582 +1583 +1584 +1585 +1586 +1587 +1588 +1589 +1590 +1591 +1592 +1593 +1594 +1595 +1596 +1597 +1598 +1599 +1600 +1601 +1602 +1603 +1604 +1605 +1606 +1607 +1608 +1609 +1610 +1611 +1612 +1613 +1614 +1615 +1616 +1617 +1618 +1619 +1620 +1621 +1622 +1623 +1624 +1625 +1626 +1627 +1628 +1629 +1630 +1631 +1632 +1633 +1634 +1635 +1636 +1637 +1638 +1639 +1640 +1641 +1642 +1643 +1644 +1645 +1646 +1647 +1648 +1649 +1650 +1651 +1652 +1653 +1654 +1655 +1656 +1657 +1658 +1659 +1660 +1661 +1662 +1663 +1664 +1665 +1666 +1667 +1668 +1669 +1670 +1671 +1672 +1673 +1674 +1675 +1676 +1677 +1678 +1679 +1680 +1681 +1682 +1683 +1684 +1685 +1686 +1687 +1688 +1689 +1690 +1691 +1692 +1693 +1694 +1695 +1696 +1697 +1698 +1699 +1700 +1701 +1702 +1703 +1704 +1705 +1706 +1707 +1708 +1709 +1710 +1711 +1712 +1713 +1714 +1715 +1716 +1717 +1718 +1719 +1720 +1721 +1722 +1723 +1724 +1725 +1726 +1727 +1728 +1729 +1730 +1731 +1732 +1733 +1734 +1735 +1736 +1737 +1738 +1739 +1740 +1741 +1742 +1743 +1744 +1745 +1746 +1747 +1748 +1749 +1750 +1751 +1752 +1753 +1754 +1755 +1756 +1757 +1758 +1759 +1760 +1761 +1762 +1763 +1764 +1765 +1766 +1767 +1768 +1769 +1770 +1771 +1772 +1773 +1774 +1775 +1776 +1777 +1778 +1779 +1780 +1781 +1782 +1783 +1784 +1785 +1786 +1787 +1788 +1789 +1790 +1791 +1792 +1793 +1794 +1795 +1796 +1797 +1798 +1799 +1800 +1801 +1802 +1803 +1804 +1805 +1806 +1807 +1808 +1809 +1810 +1811 +1812 +1813 +1814 +1815 +1816 +1817 +1818 +1819 +1820 +1821 +1822 +1823 +1824 +1825 +1826 +1827 +1828 +1829 +1830 +1831 +1832 +1833 +1834 +1835 +1836 +1837 +1838 +1839 +1840 +1841 +1842 +1843 +1844 +1845 +1846 +1847 +1848 +1849 +1850 +1851 +1852 +1853 +1854 +1855 +1856 +1857 +1858 +1859 +1860 +1861 +1862 +1863 +1864 +1865 +1866 +1867 +1868 +1869 +1870 +1871 +1872 +1873 +1874 +1875 +1876 +1877 +1878 +1879 +1880 +1881 +1882 +1883 +1884 +1885 +1886 +1887 +1888 +1889 +1890 +1891 +1892 +1893 +1894 +1895 +1896 +1897 +1898 +1899 +1900 +1901 +1902 +1903 +1904 +1905 +1906 +1907 +1908 +1909 +1910 +1911 +1912 +1913 +1914 +1915 +1916 +1917 +1918 +1919 +1920 +1921 +1922 +1923 +1924 +1925 +1926 +1927 +1928 +1929 +1930 +1931 +1932 +1933 +1934 +1935 +1936 +1937 +1938 +1939 +1940 +1941 +1942 +1943 +1944 +1945 +1946 +1947 +1948 +1949 +1950 +1951 +1952 +1953 +1954 +1955 +1956 +1957 +1958 +1959 +1960 +1961 +1962 +1963 +1964 +1965 +1966 +1967 +1968 +1969 +1970 +1971 +1972 +1973 +1974 +1975 +1976 +1977 +1978 +1979 +1980 +1981 +1982 +1983 +1984 +1985 +1986 +1987 +1988 +1989 +1990 +1991 +1992 +1993 +1994 +1995 +1996 +1997 +1998 +1999 +2000 +2001 +2002 +2003 +2004 +2005 +2006 +2007 +2008 +2009 +2010 +2011 +2012 +2013 +2014 +2015 +2016 +2017 +2018 +2019 +2020 +2021 +2022 +2023 +2024 +2025 +2026 +2027 +2028 +2029 +2030 +2031 +2032 +2033 +2034 +2035 +2036 +2037 +2038 +2039 +2040 +2041 +2042 +2043 +2044 +2045 +2046 +2047 +2048 +2049 +2050 +2051 +2052 +2053 +2054 +2055 +2056 +2057 +2058 +2059 +2060 +2061 +2062 +2063 +2064 +2065 +2066 +2067 +2068 +2069 +2070 +2071 +2072 +2073 +2074 +2075 +2076 +2077 +2078 +2079 +2080 +2081 +2082 +2083 +2084 +2085 +2086 +2087 +2088 +2089 +2090 +2091 +2092 +2093 +2094 +2095 +2096 +2097 +2098 +2099 +2100 +2101 +2102 +2103 +2104 +2105 +2106 +2107 +2108 +2109 +2110 +2111 +2112 +2113 +2114 +2115 +2116 +2117 +2118 +2119 +2120 +2121 +2122 +2123 +2124 +2125 +2126 +2127 +2128 +2129 +2130 +2131 +2132 +2133 +2134 +2135 +2136 +2137 +2138 +2139 +2140 +2141 +2142 +2143 +2144 +2145 +2146 +2147 +2148 +2149 +2150 +2151 +2152 +2153 +2154 +2155 +2156 +2157 +2158 +2159 +2160 +2161 +2162 +2163 +2164 +2165 +2166 +2167 +2168 +2169 +2170 +2171 +2172 +2173 +2174 +2175 +2176 +2177 +2178 +2179 +2180 +2181 +2182 +2183 +2184 +2185 +2186 +2187 +2188 +2189 +2190 +2191 +2192 +2193 +2194 +2195 +2196 +2197 +2198 +2199 +2200 +2201 +2202 +2203 +2204 +2205 +2206 +2207 +2208 +2209 +2210 +2211 +2212 +2213 +2214 +2215 +2216 +2217 +2218 +2219 +2220 +2221 +2222 +2223 +2224 +2225 +2226 +2227 +2228 +2229 +2230 +2231 +2232 +2233 +2234 +2235 +2236 +2237 +2238 +2239 +2240 +2241 +2242 +2243 +2244 +2245 +2246 +2247 +2248 +2249 +2250 +2251 +2252 +2253 +2254 +2255 +2256 +2257 +2258 +2259 +2260 +2261 +2262 +2263 +2264 +2265 +2266 +2267 +2268 +2269 +2270 +2271 +2272 +2273 +2274 +2275 +2276 +2277 +2278 +2279 +2280 +2281 +2282 +2283 +2284 +2285 +2286 +2287 +2288 +2289 +2290 +2291 +2292 +2293 +2294 +2295 +2296 +2297 +2298 +2299 +2300 +2301 +2302 +2303 +2304 +2305 +2306 +2307 +2308 +2309 +2310 +2311 +2312 +2313 +2314 +2315 +2316 +2317 +2318 +2319 +2320 +2321 +2322 +2323 +2324 +2325 +2326 +2327 +2328 +2329 +2330 +2331 +2332 +2333 +2334 +2335 +2336 +2337 +2338 +2339 +2340 +2341 +2342 +2343 +2344 +2345 +2346 +2347 +2348 +2349 +2350 +2351 +2352 +2353 +2354 +2355 +2356 +2357 +2358 +2359 +2360 +2361 +2362 +2363 +2364 +2365 +2366 +2367 +2368 +2369 +2370 +2371 +2372 +2373 +2374 +2375 +2376 +2377 +2378 +2379 +2380 +2381 +2382 +2383 +2384 +2385 +2386 +2387 +2388 +2389 +2390 +2391 +2392 +2393 +2394 +2395 +2396 +2397 +2398 +2399 +2400 +2401 +2402 +2403 +2404 +2405 +2406 +2407 +2408 +2409 +2410 +2411 +2412 +2413 +2414 +2415 +2416 +2417 +2418 +2419 +2420 +2421 +2422 +2423 +2424 +2425 +2426 +2427 +2428 +2429 +2430 +2431 +2432 +2433 +2434 +2435 +2436 +2437 +2438 +2439 +2440 +2441 +2442 +2443 +2444 +2445 +2446 +2447 +2448 +2449 +2450 +2451 +2452 +2453 +2454 +2455 +2456 +2457 +2458 +2459 +2460 +2461 +2462 +2463 +2464 +2465 +2466 +2467 +2468 +2469 +2470 +2471 +2472 +2473 +2474 +2475 +2476 +2477 +2478 +2479 +2480 +2481 +2482 +2483 +2484 +2485 +2486 +2487 +2488 +2489 +2490 +2491 +2492 +2493 +2494 +2495 +2496 +2497 +2498 +2499 +2500 +2501 +2502 +2503 +2504 +2505 +2506 +2507 +2508 +2509 +2510 +2511 +2512 +2513 +2514 +2515 +2516 +2517 +2518 +2519 +2520 +2521 +2522 +2523 +2524 +2525 +2526 +2527 +2528 +2529 +2530 +2531 +2532 +2533 +2534 +2535 +2536 +2537 +2538 +2539 +2540 +2541 +2542 +2543 +2544 +2545 +2546 +2547 +2548 +2549 +2550 +2551 +2552 +2553 +2554 +2555 +2556 +2557 +2558 +2559 +2560 +2561 +2562 +2563 +2564 +2565 +2566 +2567 +2568 +2569 +2570 +2571 +2572 +2573 +2574 +2575 +2576 +2577 +2578 +2579 +2580 +2581 +2582 +2583 +2584 +2585 +2586 +2587 +2588 +2589 +2590 +2591 +2592 +2593 +2594 +2595 +2596 +2597 +2598 +2599 +2600 +2601 +2602 +2603 +2604 +2605 +2606 +2607 +2608 +2609 +2610 +2611 +2612 +2613 +2614 +2615 +2616 +2617 +2618 +2619 +2620 +2621 +2622 +2623 +2624 +2625 +2626 +2627 +2628 +2629 +2630 +2631 +2632 +2633 +2634 +2635 +2636 +2637 +2638 +2639 +2640 +2641 +2642 +2643 +2644 +2645 +2646 +2647 +2648 +2649 +2650 +2651 +2652 +2653 +2654 +2655 +2656 +2657 +2658 +2659 +2660 +2661 +2662 +2663 +2664 +2665 +2666 +2667 +2668 +2669 +2670 +2671 +2672 +2673 +2674 +2675 +2676 +2677 +2678 +2679 +2680 +2681 +2682 +2683 +2684 +2685 +2686 +2687 +2688 +2689 +2690 +2691 +2692 +2693 +2694 +2695 +2696 +2697 +2698 +2699 +2700 +2701 +2702 +2703 +2704 +2705 +2706 +2707 +2708 +2709 +2710 +2711 +2712 +2713 +2714 +2715 +2716 +2717 +2718 +2719 +2720 +2721 +2722 +2723 +2724 +2725 +2726 +2727 +2728 +2729 +2730 +2731 +2732 +2733 +2734 +2735 +2736 +2737 +2738 +2739 +2740 +2741 +2742 +2743 +2744 +2745 +2746 +2747 +2748 +2749 +2750 +2751 +2752 +2753 +2754 +2755 +2756 +2757 +2758 +2759 +2760 +2761 +2762 +2763 +2764 +2765 +2766 +2767 +2768 +2769 +2770 +2771 +2772 +2773 +2774 +2775 +2776 +2777 +2778 +2779 +2780 +2781 +2782 +2783 +2784 +2785 +2786 +2787 +2788 +2789 +2790 +2791 +2792 +2793 +2794 +2795 +2796 +2797 +2798 +2799 +2800 +2801 +2802 +2803 +2804 +2805 +2806 +2807 +2808 +2809 +2810 +2811 +2812 +2813 +2814 +2815 +2816 +2817 +2818 +2819 +2820 +2821 +2822 +2823 +2824 +2825 +2826 +2827 +2828 +2829 +2830 +2831 +2832 +2833 +2834 +2835 +2836 +2837 +2838 +2839 +2840 +2841 +2842 +2843 +2844 +2845 +2846 +2847 +2848 +2849 +2850 +2851 +2852 +2853 +2854 +2855 +2856 +2857 +2858 +2859 +2860 +2861 +2862 +2863 +2864 +2865 +2866 +2867 +2868 +2869 +2870 +2871 +2872 +2873 +2874 +2875 +2876 +2877 +2878 +2879 +2880 +2881 +2882 +2883 +2884 +2885 +2886 +2887 +2888 +2889 +2890 +2891 +2892 +2893 +2894 +2895 +2896 +2897 +2898 +2899 +2900 +2901 +2902 +2903 +2904 +2905 +2906 +2907 +2908 +2909 +2910 +2911 +2912 +2913 +2914 +2915 +2916 +2917 +2918 +2919 +2920 +2921 +2922 +2923 +2924 +2925 +2926 +2927 +2928 +2929 +2930 +2931 +2932 +2933 +2934 +2935 +2936 +2937 +2938 +2939 +2940 +2941 +2942 +2943 +2944 +2945 +2946 +2947 +2948 +2949 +2950 +2951 +2952 +2953 +2954 +2955 +2956 +2957 +2958 +2959 +2960 +2961 +2962 +2963 +2964 +2965 +2966 +2967 +2968 +2969 +2970 +2971 +2972 +2973 +2974 +2975 +2976 +2977 +2978 +2979 +2980 +2981 +2982 +2983 +2984 +2985 +2986 +2987 +2988 +2989 +2990 +2991 +2992 +2993 +2994 +2995 +2996 +2997 +2998 +2999 +3000 +3001 +3002 +3003 +3004 +3005 +3006 +3007 +3008 +3009 +3010 +3011 +3012 +3013 +3014 +3015 +3016 +3017 +3018 +3019 +3020 +3021 +3022 +3023 +3024 +3025 +3026 +3027 +3028 +3029 +3030 +3031 +3032 +3033 +3034 +3035 +3036 +3037 +3038 +3039 +3040 +3041 +3042 +3043 +3044 +3045 +3046 +3047 +3048 +3049 +3050 +3051 +3052 +3053 +3054 +3055 +3056 +3057 +3058 +3059 +3060 +3061 +3062 +3063 +3064 +3065 +3066 +3067 +3068 +3069 +3070 +3071 +3072 +3073 +3074 +3075 +3076 +3077 +3078 +3079 +3080 +3081 +3082 +3083 +3084 +3085 +3086 +3087 +3088 +3089 +3090 +3091 +3092 +3093 +3094 +3095 +3096 +3097 +3098 +3099 +3100 +3101 +3102 +3103 +3104 +3105 +3106 +3107 +3108 +3109 +3110 +3111 +3112 +3113 +3114 +3115 +3116 +3117 +3118 +3119 +3120 +3121 +3122 +3123 +3124 +3125 +3126 +3127 +3128 +3129 +3130 +3131 +3132 +3133 +3134 +3135 +3136 +3137 +3138 +3139 +3140 +3141 +3142 +3143 +3144 +3145 +3146 +3147 +3148 +3149 +3150 +3151 +3152 +3153 +3154 +3155 +3156 +3157 +3158 +3159 +3160 +3161 +3162 +3163 +3164 +3165 +3166 +3167 +3168 +3169 +3170 +3171 +3172 +3173 +3174 +3175 +3176 +3177 +3178 +3179 +3180 +3181 +3182 +3183 +3184 +3185 +3186 +3187 +3188 +3189 +3190 +3191 +3192 +3193 +3194 +3195 +3196 +3197 +3198 +3199 +3200 +3201 +3202 +3203 +3204 +3205 +3206 +3207 +3208 +3209 +3210 +3211 +3212 +3213 +3214 +3215 +3216 +3217 +3218 +3219 +3220 +3221 +3222 +3223 +3224 +3225 +3226 +3227 +3228 +3229 +3230 +3231 +3232 +3233 +3234 +3235 +3236 +3237 +3238 +3239 +3240 +3241 +3242 +3243 +3244 +3245 +3246 +3247 +3248 +3249 +3250 +3251 +3252 +3253 +3254 +3255 +3256 +3257 +3258 +3259 +3260 +3261 +3262 +3263 +3264 +3265 +3266 +3267 +3268 +3269 +3270 +3271 +3272 +3273 +3274 +3275 +3276 +3277 +3278 +3279 +3280 +3281 +3282 +3283 +3284 +3285 +3286 +3287 +3288 +3289 +3290 +3291 +3292 +3293 +3294 +3295 +3296 +3297 +3298 +3299 +3300 +3301 +3302 +3303 +3304 +3305 +3306 +3307 +3308 +3309 +3310 +3311 +3312 +3313 +3314 +3315 +3316 +3317 +3318 +3319 +3320 +3321 +3322 +3323 +3324 +3325 +3326 +3327 +3328 +3329 +3330 +3331 +3332 +3333 +3334 +3335 +3336 +3337 +3338 +3339 +3340 +3341 +3342 +3343 +3344 +3345 +3346 +3347 +3348 +3349 +3350 +3351 +3352 +3353 +3354 +3355 +3356 +3357 +3358 +3359 +3360 +3361 +3362 +3363 +3364 +3365 +3366 +3367 +3368 +3369 +3370 +3371 +3372 +3373 +3374 +3375 +3376 +3377 +3378 +3379 +3380 +3381 +3382 +3383 +3384 +3385 +3386 +3387 +3388 +3389 +3390 +3391 +3392 +3393 +3394 +3395 +3396 +3397 +3398 +3399 +3400 +3401 +3402 +3403 +3404 +3405 +3406 +3407 +3408 +3409 +3410 +3411 +3412 +3413 +3414 +3415 +3416 +3417 +3418 +3419 +3420 +3421 +3422 +3423 +3424 +3425 +3426 +3427 +3428 +3429 +3430 +3431 +3432 +3433 +3434 +3435 +3436 +3437 +3438 +3439 +3440 +3441 +3442 +3443 +3444 +3445 +3446 +3447 +3448 +3449 +3450 +3451 +3452 +3453 +3454 +3455 +3456 +3457 +3458 +3459 +3460 +3461 +3462 +3463 +3464 +3465 +3466 +3467 +3468 +3469 +3470 +3471 +3472 +3473 +3474 +3475 +3476 +3477 +3478 +3479 +3480 +3481 +3482 +3483 +3484 +3485 +3486 +3487 +3488 +3489 +3490 +3491 +3492 +3493 +3494 +3495 +3496 +3497 +3498 +3499 +3500 +3501 +3502 +3503 +3504 +3505 +3506 +3507 +3508 +3509 +3510 +3511 +3512 +3513 +3514 +3515 +3516 +3517 +3518 +3519 +3520 +3521 +3522 +3523 +3524 +3525 +3526 +3527 +3528 +3529 +3530 +3531 +3532 +3533 +3534 +3535 +3536 +3537 +3538 +3539 +3540 +3541 +3542 +3543 +3544 +3545 +3546 +3547 +3548 +3549 +3550 +3551 +3552 +3553 +3554 +3555 +3556 +3557 +3558 +3559 +3560 +3561 +3562 +3563 +3564 +3565 +3566 +3567 +3568 +3569 +3570 +3571 +3572 +3573 +3574 +3575 +3576 +3577 +3578 +3579 +3580 +3581 +3582 +3583 +3584 +3585 +3586 +3587 +3588 +3589 +3590 +3591 +3592 +3593 +3594 +3595 +3596 +3597 +3598 +3599 +3600 +3601 +3602 +3603 +3604 +3605 +3606 +3607 +3608 +3609 +3610 +3611 +3612 +3613 +3614 +3615 +3616 +3617 +3618 +3619 +3620 +3621 +3622 +3623 +3624 +3625 +3626 +3627 +3628 +3629 +3630 +3631 +3632 +3633 +3634 +3635 +3636 +3637 +3638 +3639 +3640 +3641 +3642 +3643 +3644 +3645 +3646 +3647 +3648 +3649 +3650 +3651 +3652 +3653 +3654 +3655 +3656 +3657 +3658 +3659 +3660 +3661 +3662 +3663 +3664 +3665 +3666 +3667 +3668 +3669 +3670 +3671 +3672 +3673 +3674 +3675 +3676 +3677 +3678 +3679 +3680 +3681 +3682 +3683 +3684 +3685 +3686 +3687 +3688 +3689 +3690 +3691 +3692 +3693 +3694 +3695 +3696 +3697 +3698 +3699 +3700 +3701 +3702 +3703 +3704 +3705 +3706 +3707 +3708 +3709 +3710 +3711 +3712 +3713 +3714 +3715 +3716 +3717 +3718 +3719 +3720 +3721 +3722 +3723 +3724 +3725 +3726 +3727 +3728 +3729 +3730 +3731 +3732 +3733 +3734 +3735 +3736 +3737 +3738 +3739 +3740 +3741 +3742 +3743 +3744 +3745 +3746 +3747 +3748 +3749 +3750 +3751 +3752 +3753 +3754 +3755 +3756 +3757 +3758 +3759 +3760 +3761 +3762 +3763 +3764 +3765 +3766 +3767 +3768 +3769 +3770 +3771 +3772 +3773 +3774 +3775 +3776 +3777 +3778 +3779 +3780 +3781 +3782 +3783 +3784 +3785 +3786 +3787 +3788 +3789 +3790 +3791 +3792 +3793 +3794 +3795 +3796 +3797 +3798 +3799 +3800 +3801 +3802 +3803 +3804 +3805 +3806 +3807 +3808 +3809 +3810 +3811 +3812 +3813 +3814 +3815 +3816 +3817 +3818 +3819 +3820 +3821 +3822 +3823 +3824 +3825 +3826 +3827 +3828 +3829 +3830 +3831 +3832 +3833 +3834 +3835 +3836 +3837 +3838 +3839 +3840 +3841 +3842 +3843 +3844 +3845 +3846 +3847 +3848 +3849 +3850 +3851 +3852 +3853 +3854 +3855 +3856 +3857 +3858 +3859 +3860 +3861 +3862 +3863 +3864 +3865 +3866 +3867 +3868 +3869 +3870 +3871 +3872 +3873 +3874 +3875 +3876 +3877 +3878 +3879 +3880 +3881 +3882 +3883 +3884 +3885 +3886 +3887 +3888 +3889 +3890 +3891 +3892 +3893 +3894 +3895 +3896 +3897 +3898 +3899 +3900 +3901 +3902 +3903 +3904 +3905 +3906 +3907 +3908 +3909 +3910 +3911 +3912 +3913 +3914 +3915 +3916 +3917 +3918 +3919 +3920 +3921 +3922 +3923 +3924 +3925 +3926 +3927 +3928 +3929 +3930 +3931 +3932 +3933 +3934 +3935 +3936 +3937 +3938 +3939 +3940 +3941 +3942 +3943 +3944 +3945 +3946 +3947 +3948 +3949 +3950 +3951 +3952 +3953 +3954 +3955 +3956 +3957 +3958 +3959 +3960 +3961 +3962 +3963 +3964 +3965 +3966 +3967 +3968 +3969 +3970 +3971 +3972 +3973 +3974 +3975 +3976 +3977 +3978 +3979 +3980 +3981 +3982 +3983 +3984 +3985 +3986 +3987 +3988 +3989 +3990 +3991 +3992 +3993 +3994 +3995 +3996 +3997 +3998 +3999 +4000 +4001 +4002 +4003 +4004 +4005 +4006 +4007 +4008 +4009 +4010 +4011 +4012 +4013 +4014 +4015 +4016 +4017 +4018 +4019 +4020 +4021 +4022 +4023 +4024 +4025 +4026 +4027 +4028 +4029 +4030 +4031 +4032 +4033 +4034 +4035 +4036 +4037 +4038 +4039 +4040 +4041 +4042 +4043 +4044 +4045 +4046 +4047 +4048 +4049 +4050 +4051 +4052 +4053 +4054 +4055 +4056 +4057 +4058 +4059 +4060 +4061 +4062 +4063 +4064 +4065 +4066 +4067 +4068 +4069 +4070 +4071 +4072 +4073 +4074 +4075 +4076 +4077 +4078 +4079 +4080 +4081 +4082 +4083 +4084 +4085 +4086 +4087 +4088 +4089 +4090 +4091 +4092 +4093 +4094 +4095 +4096 +4097 +4098 +4099 +4100 +4101 +4102 +4103 +4104 +4105 +4106 +4107 +4108 +4109 +4110 +4111 +4112 +4113 +4114 +4115 +4116 +4117 +4118 +4119 +4120 +4121 +4122 +4123 +4124 +4125 +4126 +4127 +4128 +4129 +4130 +4131 +4132 +4133 +4134 +4135 +4136 +4137 +4138 +4139 +4140 +4141 +4142 +4143 +4144 +4145 +4146 +4147 +4148 +4149 +4150 +4151 +4152 +4153 +4154 +4155 +4156 +4157 +4158 +4159 +4160 +4161 +4162 +4163 +4164 +4165 +4166 +4167 +4168 +4169 +4170 +4171 +4172 +4173 +4174 +4175 +4176 +4177 +4178 +4179 +4180 +4181 +4182 +4183 +4184 +4185 +4186 +4187 +4188 +4189 +4190 +4191 +4192 +4193 +4194 +4195 +4196 +4197 +4198 +4199 +4200 +4201 +4202 +4203 +4204 +4205 +4206 +4207 +4208 +4209 +4210 +4211 +4212 +4213 +4214 +4215 +4216 +4217 +4218 +4219 +4220 +4221 +4222 +4223 +4224 +4225 +4226 +4227 +4228 +4229 +4230 +4231 +4232 +4233 +4234 +4235 +4236 +4237 +4238 +4239 +4240 +4241 +4242 +4243 +4244 +4245 +4246 +4247 +4248 +4249 +4250 +4251 +4252 +4253 +4254 +4255 +4256 +4257 +4258 +4259 +4260 +4261 +4262 +4263 +4264 +4265 +4266 +4267 +4268 +4269 +4270 +4271 +4272 +4273 +4274 +4275 +4276 +4277 +4278 +4279 +4280 +4281 +4282 +4283 +4284 +4285 +4286 +4287 +4288 +4289 +4290 +4291 +4292 +4293 +4294 +4295 +4296 +4297 +4298 +4299 +4300 +4301 +4302 +4303 +4304 +4305 +4306 +4307 +4308 +4309 +4310 +4311 +4312 +4313 +4314 +4315 +4316 +4317 +4318 +4319 +4320 +4321 +4322 +4323 +4324 +4325 +4326 +4327 +4328 +4329 +4330 +4331 +4332 +4333 +4334 +4335 +4336 +4337 +4338 +4339 +4340 +4341 +4342 +4343 +4344 +4345 +4346 +4347 +4348 +4349 +4350 +4351 +4352 +4353 +4354 +4355 +4356 +4357 +4358 +4359 +4360 +4361 +4362 +4363 +4364 +4365 +4366 +4367 +4368 +4369 +4370 +4371 +4372 +4373 +4374 +4375 +4376 +4377 +4378 +4379 +4380 +4381 +4382 +4383 +4384 +4385 +4386 +4387 +4388 +4389 +4390 +4391 +4392 +4393 +4394 +4395 +4396 +4397 +4398 +4399 +4400 +4401 +4402 +4403 +4404 +4405 +4406 +4407 +4408 +4409 +4410 +4411 +4412 +4413 +4414 +4415 +4416 +4417 +4418 +4419 +4420 +4421 +4422 +4423 +4424 +4425 +4426 +4427 +4428 +4429 +4430 +4431 +4432 +4433 +4434 +4435 +4436 +4437 +4438 +4439 +4440 +4441 +4442 +4443 +4444 +4445 +4446 +4447 +4448 +4449 +4450 +4451 +4452 +4453 +4454 +4455 +4456 +4457 +4458 +4459 +4460 +4461 +4462 +4463 +4464 +4465 +4466 +4467 +4468 +4469 +4470 +4471 +4472 +4473 +4474 +4475 +4476 +4477 +4478 +4479 +4480 +4481 +4482 +4483 +4484 +4485 +4486 +4487 +4488 +4489 +4490 +4491 +4492 +4493 +4494 +4495 +4496 +4497 +4498 +4499 +4500 +4501 +4502 +4503 +4504 +4505 +4506 +4507 +4508 +4509 +4510 +4511 +4512 +4513 +4514 +4515 +4516 +4517 +4518 +4519 +4520 +4521 +4522 +4523 +4524 +4525 +4526 +4527 +4528 +4529 +4530 +4531 +4532 +4533 +4534 +4535 +4536 +4537 +4538 +4539 +4540 +4541 +4542 +4543 +4544 +4545 +4546 +4547 +4548 +4549 +4550 +4551 +4552 +4553 +4554 +4555 +4556 +4557 +4558 +4559 +4560 +4561 +4562 +4563 +4564 +4565 +4566 +4567 +4568 +4569 +4570 +4571 +4572 +4573 +4574 +4575 +4576 +4577 +4578 +4579 +4580 +4581 +4582 +4583 +4584 +4585 +4586 +4587 +4588 +4589 +4590 +4591 +4592 +4593 +4594 +4595 +4596 +4597 +4598 +4599 +4600 +4601 +4602 +4603 +4604 +4605 +4606 +4607 +4608 +4609 +4610 +4611 +4612 +4613 +4614 +4615 +4616 +4617 +4618 +4619 +4620 +4621 +4622 +4623 +4624 +4625 +4626 +4627 +4628 +4629 +4630 +4631 +4632 +4633 +4634 +4635 +4636 +4637 +4638 +4639 +4640 +4641 +4642 +4643 +4644 +4645 +4646 +4647 +4648 +4649 +4650 +4651 +4652 +4653 +4654 +4655 +4656 +4657 +4658 +4659 +4660 +4661 +4662 +4663 +4664 +4665 +4666 +4667 +4668 +4669 +4670 +4671 +4672 +4673 +4674 +4675 +4676 +4677 +4678 +4679 +4680 +4681 +4682 +4683 +4684 +4685 +4686 +4687 +4688 +4689 +4690 +4691 +4692 +4693 +4694 +4695 +4696 +4697 +4698 +4699 +4700 +4701 +4702 +4703 +4704 +4705 +4706 +4707 +4708 +4709 +4710 +4711 +4712 +4713 +4714 +4715 +4716 +4717 +4718 +4719 +4720 +4721 +4722 +4723 +4724 +4725 +4726 +4727 +4728 +4729 +4730 +4731 +4732 +4733 +4734 +4735 +4736 +4737 +4738 +4739 +4740 +4741 +4742 +4743 +4744 +4745 +4746 +4747 +4748 +4749 +4750 +4751 +4752 +4753 +4754 +4755 +4756 +4757 +4758 +4759 +4760 +4761 +4762 +4763 +4764 +4765 +4766 +4767 +4768 +4769 +4770 +4771 +4772 +4773 +4774 +4775 +4776 +4777 +4778 +4779 +4780 +4781 +4782 +4783 +4784 +4785 +4786 +4787 +4788 +4789 +4790 +4791 +4792 +4793 +4794 +4795 +4796 +4797 +4798 +4799 +4800 +4801 +4802 +4803 +4804 +4805 +4806 +4807 +4808 +4809 +4810 +4811 +4812 +4813 +4814 +4815 +4816 +4817 +4818 +4819 +4820 +4821 +4822 +4823 +4824 +4825 +4826 +4827 +4828 +4829 +4830 +4831 +4832 +4833 +4834 +4835 +4836 +4837 +4838 +4839 +4840 +4841 +4842 +4843 +4844 +4845 +4846 +4847 +4848 +4849 +4850 +4851 +4852 +4853 +4854 +4855 +4856 +4857 +4858 +4859 +4860 +4861 +4862 +4863 +4864 +4865 +4866 +4867 +4868 +4869 +4870 +4871 +4872 +4873 +4874 +4875 +4876 +4877 +4878 +4879 +4880 +4881 +4882 +4883 +4884 +4885 +4886 +4887 +4888 +4889 +4890 +4891 +4892 +4893 +4894 +4895 +4896 +4897 +4898 +4899 +4900 +4901 +4902 +4903 +4904 +4905 +4906 +4907 +4908 +4909 +4910 +4911 +4912 +4913 +4914 +4915 +4916 +4917 +4918 +4919 +4920 +4921 +4922 +4923 +4924 +4925 +4926 +4927 +4928 +4929 +4930 +4931 +4932 +4933 +4934 +4935 +4936 +4937 +4938 +4939 +4940 +4941 +4942 +4943 +4944 +4945 +4946 +4947 +4948 +4949 +4950 +4951 +4952 +4953 +4954 +4955 +4956 +4957 +4958 +4959 +4960 +4961 +4962 +4963 +4964 +4965 +4966 +4967 +4968 +4969 +4970 +4971 +4972 +4973 +4974 +4975 +4976 +4977 +4978 +4979 +4980 +4981 +4982 +4983 +4984 +4985 +4986 +4987 +4988 +4989 +4990 +4991 +4992 +4993 +4994 +4995 +4996 +4997 +4998 +4999 +5000 +5001 +5002 +5003 +5004 +5005 +5006 +5007 +5008 +5009 +5010 +5011 +5012 +5013 +5014 +5015 +5016 +5017 +5018 +5019 +5020 +5021 +5022 +5023 +5024 +5025 +5026 +5027 +5028 +5029 +5030 +5031 +5032 +5033 +5034 +5035 +5036 +5037 +5038 +5039 +5040 +5041 +5042 +5043 +5044 +5045 +5046 +5047 +5048 +5049 +5050 +5051 +5052 +5053 +5054 +5055 +5056 +5057 +5058 +5059 +5060 +5061 +5062 +5063 +5064 +5065 +5066 +5067 +5068 +5069 +5070 +5071 +5072 +5073 +5074 +5075 +5076 +5077 +5078 +5079 +5080 +5081 +5082 +5083 +5084 +5085 +5086 +5087 +5088 +5089 +5090 +5091 +5092 +5093 +5094 +5095 +5096 +5097 +5098 +5099 +5100 +5101 +5102 +5103 +5104 +5105 +5106 +5107 +5108 +5109 +5110 +5111 +5112 +5113 +5114 +5115 +5116 +5117 +5118 +5119 +5120 +5121 +5122 +5123 +5124 +5125 +5126 +5127 +5128 +5129 +5130 +5131 +5132 +5133 +5134 +5135 +5136 +5137 +5138 +5139 +5140 +5141 +5142 +5143 +5144 +5145 +5146 +5147 +5148 +5149 +5150 +5151 +5152 +5153 +5154 +5155 +5156 +5157 +5158 +5159 +5160 +5161 +5162 +5163 +5164 +5165 +5166 +5167 +5168 +5169 +5170 +5171 +5172 +5173 +5174 +5175 +5176 +5177 +5178 +5179 +5180 +5181 +5182 +5183 +5184 +5185 +5186 +5187 +5188 +5189 +5190 +5191 +5192 +5193 +5194 +5195 +5196 +5197 +5198 +5199 +5200 +5201 +5202 +5203 +5204 +5205 +5206 +5207 +5208 +5209 +5210 +5211 +5212 +5213 +5214 +5215 +5216 +5217 +5218 +5219 +5220 +5221 +5222 +5223 +5224 +5225 +5226 +5227 +5228 +5229 +5230 +5231 +5232 +5233 +5234 +5235 +5236 +5237 +5238 +5239 +5240 +5241 +5242 +5243 +5244 +5245 +5246 +5247 +5248 +5249 +5250 +5251 +5252 +5253 +5254 +5255 +5256 +5257 +5258 +5259 +5260 +5261 +5262 +5263 +5264 +5265 +5266 +5267 +5268 +5269 +5270 +5271 +5272 +5273 +5274 +5275 +5276 +5277 +5278 +5279 +5280 +5281 +5282 +5283 +5284 +5285 +5286 +5287 +5288 +5289 +5290 +5291 +5292 +5293 +5294 +5295 +5296 +5297 +5298 +5299 +5300 +5301 +5302 +5303 +5304 +5305 +5306 +5307 +5308 +5309 +5310 +5311 +5312 +5313 +5314 +5315 +5316 +5317 +5318 +5319 +5320 +5321 +5322 +5323 +5324 +5325 +5326 +5327 +5328 +5329 +5330 +5331 +5332 +5333 +5334 +5335 +5336 +5337 +5338 +5339 +5340 +5341 +5342 +5343 +5344 +5345 +5346 +5347 +5348 +5349 +5350 +5351 +5352 +5353 +5354 +5355 +5356 +5357 +5358 +5359 +5360 +5361 +5362 +5363 +5364 +5365 +5366 +5367 +5368 +5369 +5370 +5371 +5372 +5373 +5374 +5375 +5376 +5377 +5378 +5379 +5380 +5381 +5382 +5383 +5384 +5385 +5386 +5387 +5388 +5389 +5390 +5391 +5392 +5393 +5394 +5395 +5396 +5397 +5398 +5399 +5400 +5401 +5402 +5403 +5404 +5405 +5406 +5407 +5408 +5409 +5410 +5411 +5412 +5413 +5414 +5415 +5416 +5417 +5418 +5419 +5420 +5421 +5422 +5423 +5424 +5425 +5426 +5427 +5428 +5429 +5430 +5431 +5432 +5433 +5434 +5435 +5436 +5437 +5438 +5439 +5440 +5441 +5442 +5443 +5444 +5445 +5446 +5447 +5448 +5449 +5450 +5451 +5452 +5453 +5454 +5455 +5456 +5457 +5458 +5459 +5460 +5461 +5462 +5463 +5464 +5465 +5466 +5467 +5468 +5469 +5470 +5471 +5472 +5473 +5474 +5475 +5476 +5477 +5478 +5479 +5480 +5481 +5482 +5483 +5484 +5485 +5486 +5487 +5488 +5489 +5490 +5491 +5492 +5493 +5494 +5495 +5496 +5497 +5498 +5499 +5500 +5501 +5502 +5503 +5504 +5505 +5506 +5507 +5508 +5509 +5510 +5511 +5512 +5513 +5514 +5515 +5516 +5517 +5518 +5519 +5520 +5521 +5522 +5523 +5524 +5525 +5526 +5527 +5528 +5529 +5530 +5531 +5532 +5533 +5534 +5535 +5536 +5537 +5538 +5539 +5540 +5541 +5542 +5543 +5544 +5545 +5546 +5547 +5548 +5549 +5550 +5551 +5552 +5553 +5554 +5555 +5556 +5557 +5558 +5559 +5560 +5561 +5562 +5563 +5564 +5565 +5566 +5567 +5568 +5569 +5570 +5571 +5572 +5573 +5574 +5575 +5576 +5577 +5578 +5579 +5580 +5581 +5582 +5583 +5584 +5585 +5586 +5587 +5588 +5589 +5590 +5591 +5592 +5593 +5594 +5595 +5596 +5597 +5598 +5599 +5600 +5601 +5602 +5603 +5604 +5605 +5606 +5607 +5608 +5609 +5610 +5611 +5612 +5613 +5614 +5615 +5616 +5617 +5618 +5619 +5620 +5621 +5622 +5623 +5624 +5625 +5626 +5627 +5628 +5629 +5630 +5631 +5632 +5633 +5634 +5635 +5636 +5637 +5638 +5639 +5640 +5641 +5642 +5643 +5644 +5645 +5646 +5647 +5648 +5649 +5650 +5651 +5652 +5653 +5654 +5655 +5656 +5657 +5658 +5659 +5660 +5661 +5662 +5663 +5664 +5665 +5666 +5667 +5668 +5669 +5670 +5671 +5672 +5673 +5674 +5675 +5676 +5677 +5678 +5679 +5680 +5681 +5682 +5683 +5684 +5685 +5686 +5687 +5688 +5689 +5690 +5691 +5692 +5693 +5694 +5695 +5696 +5697 +5698 +5699 +5700 +5701 +5702 +5703 +5704 +5705 +5706 +5707 +5708 +5709 +5710 +5711 +5712 +5713 +5714 +5715 +5716 +5717 +5718 +5719 +5720 +5721 +5722 +5723 +5724 +5725 +5726 +5727 +5728 +5729 +5730 +5731 +5732 +5733 +5734 +5735 +5736 +5737 +5738 +5739 +5740 +5741 +5742 +5743 +5744 +5745 +5746 +5747 +5748 +5749 +5750 +5751 +5752 +5753 +5754 +5755 +5756 +5757 +5758 +5759 +5760 +5761 +5762 +5763 +5764 +5765 +5766 +5767 +5768 +5769 +5770 +5771 +5772 +5773 +5774 +5775 +5776 +5777 +5778 +5779 +5780 +5781 +5782 +5783 +5784 +5785 +5786 +5787 +5788 +5789 +5790 +5791 +5792 +5793 +5794 +5795 +5796 +5797 +5798 +5799 +5800 +5801 +5802 +5803 +5804 +5805 +5806 +5807 +5808 +5809 +5810 +5811 +5812 +5813 +5814 +5815 +5816 +5817 +5818 +5819 +5820 +5821 +5822 +5823 +5824 +5825 +5826 +5827 +5828 +5829 +5830 +5831 +5832 +5833 +5834 +5835 +5836 +5837 +5838 +5839 +5840 +5841 +5842 +5843 +5844 +5845 +5846 +5847 +5848 +5849 +5850 +5851 +5852 +5853 +5854 +5855 +5856 +5857 +5858 +5859 +5860 +5861 +5862 +5863 +5864 +5865 +5866 +5867 +5868 +5869 +5870 +5871 +5872 +5873 +5874 +5875 +5876 +5877 +5878 +5879 +5880 +5881 +5882 +5883 +5884 +5885 +5886 +5887 +5888 +5889 +5890 +5891 +5892 +5893 +5894 +5895 +5896 +5897 +5898 +5899 +5900 +5901 +5902 +5903 +5904 +5905 +5906 +5907 +5908 +5909 +5910 +5911 +5912 +5913 +5914 +5915 +5916 +5917 +5918 +5919 +5920 +5921 +5922 +5923 +5924 +5925 +5926 +5927 +5928 +5929 +5930 +5931 +5932 +5933 +5934 +5935 +5936 +5937 +5938 +5939 +5940 +5941 +5942 +5943 +5944 +5945 +5946 +5947 +5948 +5949 +5950 +5951 +5952 +5953 +5954 +5955 +5956 +5957 +5958 +5959 +5960 +5961 +5962 +5963 +5964 +5965 +5966 +5967 +5968 +5969 +5970 +5971 +5972 +5973 +5974 +5975 +5976 +5977 +5978 +5979 +5980 +5981 +5982 +5983 +5984 +5985 +5986 +5987 +5988 +5989 +5990 +5991 +5992 +5993 +5994 +5995 +5996 +5997 +5998 +5999 +6000 +6001 +6002 +6003 +6004 +6005 +6006 +6007 +6008 +6009 +6010 +6011 +6012 +6013 +6014 +6015 +6016 +6017 +6018 +6019 +6020 +6021 +6022 +6023 +6024 +6025 +6026 +6027 +6028 +6029 +6030 +6031 +6032 +6033 +6034 +6035 +6036 +6037 +6038 +6039 +6040 +6041 +6042 +6043 +6044 +6045 +6046 +6047 +6048 +6049 +6050 +6051 +6052 +6053 +6054 +6055 +6056 +6057 +6058 +6059 +6060 +6061 +6062 +6063 +6064 +6065 +6066 +6067 +6068 +6069 +6070 +6071 +6072 +6073 +6074 +6075 +6076 +6077 +6078 +6079 +6080 +6081 +6082 +6083 +6084 +6085 +6086 +6087 +6088 +6089 +6090 +6091 +6092 +6093 +6094 +6095 +6096 +6097 +6098 +6099 +6100 +6101 +6102 +6103 +6104 +6105 +6106 +6107 +6108 +6109 +6110 +6111 +6112 +6113 +6114 +6115 +6116 +6117 +6118 +6119 +6120 +6121 +6122 +6123 +6124 +6125 +6126 +6127 +6128 +6129 +6130 +6131 +6132 +6133 +6134 +6135 +6136 +6137 +6138 +6139 +6140 +6141 +6142 +6143 +6144 +6145 +6146 +6147 +6148 +6149 +6150 +6151 +6152 +6153 +6154 +6155 +6156 +6157 +6158 +6159 +6160 +6161 +6162 +6163 +6164 +6165 +6166 +6167 +6168 +6169 +6170 +6171 +6172 +6173 +6174 +6175 +6176 +6177 +6178 +6179 +6180 +6181 +6182 +6183 +6184 +6185 +6186 +6187 +6188 +6189 +6190 +6191 +6192 +6193 +6194 +6195 +6196 +6197 +6198 +6199 +6200 +6201 +6202 +6203 +6204 +6205 +6206 +6207 +6208 +6209 +6210 +6211 +6212 +6213 +6214 +6215 +6216 +6217 +6218 +6219 +6220 +6221 +6222 +6223 +6224 +6225 +6226 +6227 +6228 +6229 +6230 +6231 +6232 +6233 +6234 +6235 +6236 +6237 +6238 +6239 +6240 +6241 +6242 +6243 +6244 +6245 +6246 +6247 +6248 +6249 +6250 +6251 +6252 +6253 +6254 +6255 +6256 +6257 +6258 +6259 +6260 +6261 +6262 +6263 +6264 +6265 +6266 +6267 +6268 +6269 +6270 +6271 +6272 +6273 +6274 +6275 +6276 +6277 +6278 +6279 +6280 +6281 +6282 +6283 +6284 +6285 +6286 +6287 +6288 +6289 +6290 +6291 +6292 +6293 +6294 +6295 +6296 +6297 +6298 +6299 +6300 +6301 +6302 +6303 +6304 +6305 +6306 +6307 +6308 +6309 +6310 +6311 +6312 +6313 +6314 +6315 +6316 +6317 +6318 +6319 +6320 +6321 +6322 +6323 +6324 +6325 +6326 +6327 +6328 +6329 +6330 +6331 +6332 +6333 +6334 +6335 +6336 +6337 +6338 +6339 +6340 +6341 +6342 +6343 +6344 +6345 +6346 +6347 +6348 +6349 +6350 +6351 +6352 +6353 +6354 +6355 +6356 +6357 +6358 +6359 +6360 +6361 +6362 +6363 +6364 +6365 +6366 +6367 +6368 +6369 +6370 +6371 +6372 +6373 +6374 +6375 +6376 +6377 +6378 +6379 +6380 +6381 +6382 +6383 +6384 +6385 +6386 +6387 +6388 +6389 +6390 +6391 +6392 +6393 +6394 +6395 +6396 +6397 +6398 +6399 +6400 +6401 +6402 +6403 +6404 +6405 +6406 +6407 +6408 +6409 +6410 +6411 +6412 +6413 +6414 +6415 +6416 +6417 +6418 +6419 +6420 +6421 +6422 +6423 +6424 +6425 +6426 +6427 +6428 +6429 +6430 +6431 +6432 +6433 +6434 +6435 +6436 +6437 +6438 +6439 +6440 +6441 +6442 +6443 +6444 +6445 +6446 +6447 +6448 +6449 +6450 +6451 +6452 +6453 +6454 +6455 +6456 +6457 +6458 +6459 +6460 +6461 +6462 +6463 +6464 +6465 +6466 +6467 +6468 +6469 +6470 +6471 +6472 +6473 +6474 +6475 +6476 +6477 +6478 +6479 +6480 +6481 +6482 +6483 +6484 +6485 +6486 +6487 +6488 +6489 +6490 +6491 +6492 +6493 +6494 +6495 +6496 +6497 +6498 +6499 +6500 +6501 +6502 +6503 +6504 +6505 +6506 +6507 +6508 +6509 +6510 +6511 +6512 +6513 +6514 +6515 +6516 +6517 +6518 +6519 +6520 +6521 +6522 +6523 +6524 +6525 +6526 +6527 +6528 +6529 +6530 +6531 +6532 +6533 +6534 +6535 +6536 +6537 +6538 +6539 +6540 +6541 +6542 +6543 +6544 +6545 +6546 +6547 +6548 +6549 +6550 +6551 +6552 +6553 +6554 +6555 +6556 +6557 +6558 +6559 +6560 +6561 +6562 +6563 +6564 +6565 +6566 +6567 +6568 +6569 +6570 +6571 +6572 +6573 +6574 +6575 +6576 +6577 +6578 +6579 +6580 +6581 +6582 +6583 +6584 +6585 +6586 +6587 +6588 +6589 +6590 +6591 +6592 +6593 +6594 +6595 +6596 +6597 +6598 +6599 +6600 +6601 +6602 +6603 +6604 +6605 +6606 +6607 +6608 +6609 +6610 +6611 +6612 +6613 +6614 +6615 +6616 +6617 +6618 +6619 +6620 +6621 +6622 +6623 +6624 +6625 +6626 +6627 +6628 +6629 +6630 +6631 +6632 +6633 +6634 +6635 +6636 +6637 +6638 +6639 +6640 +6641 +6642 +6643 +6644 +6645 +6646 +6647 +6648 +6649 +6650 +6651 +6652 +6653 +6654 +6655 +6656 +6657 +6658 +6659 +6660 +6661 +6662 +6663 +6664 +6665 +6666 +6667 +6668 +6669 +6670 +6671 +6672 +6673 +6674 +6675 +6676 +6677 +6678 +6679 +6680 +6681 +6682 +6683 +6684 +6685 +6686 +6687 +6688 +6689 +6690 +6691 +6692 +6693 +6694 +6695 +6696 +6697 +6698 +6699 +6700 +6701 +6702 +6703 +6704 +6705 +6706 +6707 +6708 +6709 +6710 +6711 +6712 +6713 +6714 +6715 +6716 +6717 +6718 +6719 +6720 +6721 +6722 +6723 +6724 +6725 +6726 +6727 +6728 +6729 +6730 +6731 +6732 +6733 +6734 +6735 +6736 +6737 +6738 +6739 +6740 +6741 +6742 +6743 +6744 +6745 +6746 +6747 +6748 +6749 +6750 +6751 +6752 +6753 +6754 +6755 +6756 +6757 +6758 +6759 +6760 +6761 +6762 +6763 +6764 +6765 +6766 +6767 +6768 +6769 +6770 +6771 +6772 +6773 +6774 +6775 +6776 +6777 +6778 +6779 +6780 +6781 +6782 +6783 +6784 +6785 +6786 +6787 +6788 +6789 +6790 +6791 +6792 +6793 +6794 +6795 +6796 +6797 +6798 +6799 +6800 +6801 +6802 +6803 +6804 +6805 +6806 +6807 +6808 +6809 +6810 +6811 +6812 +6813 +6814 +6815 +6816 +6817 +6818 +6819 +6820 +6821 +6822 +6823 +6824 +6825 +6826 +6827 +6828 +6829 +6830 +6831 +6832 +6833 +6834 +6835 +6836 +6837 +6838 +6839 +6840 +6841 +6842 +6843 +6844 +6845 +6846 +6847 +6848 +6849 +6850 +6851 +6852 +6853 +6854 +6855 +6856 +6857 +6858 +6859 +6860 +6861 +6862 +6863 +6864 +6865 +6866 +6867 +6868 +6869 +6870 +6871 +6872 +6873 +6874 +6875 +6876 +6877 +6878 +6879 +6880 +6881 +6882 +6883 +6884 +6885 +6886 +6887 +6888 +6889 +6890 +6891 +6892 +6893 +6894 +6895 +6896 +6897 +6898 +6899 +6900 +6901 +6902 +6903 +6904 +6905 +6906 +6907 +6908 +6909 +6910 +6911 +6912 +6913 +6914 +6915 +6916 +6917 +6918 +6919 +6920 +6921 +6922 +6923 +6924 +6925 +6926 +6927 +6928 +6929 +6930 +6931 +6932 +6933 +6934 +6935 +6936 +6937 +6938 +6939 +6940 +6941 +6942 +6943 +6944 +6945 +6946 +6947 +6948 +6949 +6950 +6951 +6952 +6953 +6954 +6955 +6956 +6957 +6958 +6959 +6960 +6961 +6962 +6963 +6964 +6965 +6966 +6967 +6968 +6969 +6970 +6971 +6972 +6973 +6974 +6975 +6976 +6977 +6978 +6979 +6980 +6981 +6982 +6983 +6984 +6985 +6986 +6987 +6988 +6989 +6990 +6991 +6992 +6993 +6994 +6995 +6996 +6997 +6998 +6999 +7000 +7001 +7002 +7003 +7004 +7005 +7006 +7007 +7008 +7009 +7010 +7011 +7012 +7013 +7014 +7015 +7016 +7017 +7018 +7019 +7020 +7021 +7022 +7023 +7024 +7025 +7026 +7027 +7028 +7029 +7030 +7031 +7032 +7033 +7034 +7035 +7036 +7037 +7038 +7039 +7040 +7041 +7042 +7043 +7044 +7045 +7046 +7047 +7048 +7049 +7050 +7051 +7052 +7053 +7054 +7055 +7056 +7057 +7058 +7059 +7060 +7061 +7062 +7063 +7064 +7065 +7066 +7067 +7068 +7069 +7070 +7071 +7072 +7073 +7074 +7075 +7076 +7077 +7078 +7079 +7080 +7081 +7082 +7083 +7084 +7085 +7086 +7087 +7088 +7089 +7090 +7091 +7092 +7093 +7094 +7095 +7096 +7097 +7098 +7099 +7100 +7101 +7102 +7103 +7104 +7105 +7106 +7107 +7108 +7109 +7110 +7111 +7112 +7113 +7114 +7115 +7116 +7117 +7118 +7119 +7120 +7121 +7122 +7123 +7124 +7125 +7126 +7127 +7128 +7129 +7130 +7131 +7132 +7133 +7134 +7135 +7136 +7137 +7138 +7139 +7140 +7141 +7142 +7143 +7144 +7145 +7146 +7147 +7148 +7149 +7150 +7151 +7152 +7153 +7154 +7155 +7156 +7157 +7158 +7159 +7160 +7161 +7162 +7163 +7164 +7165 +7166 +7167 +7168 +7169 +7170 +7171 +7172 +7173 +7174 +7175 +7176 +7177 +7178 +7179 +7180 +7181 +7182 +7183 +7184 +7185 +7186 +7187 +7188 +7189 +7190 +7191 +7192 +7193 +7194 +7195 +7196 +7197 +7198 +7199 +7200 +7201 +7202 +7203 +7204 +7205 +7206 +7207 +7208 +7209 +7210 +7211 +7212 +7213 +7214 +7215 +7216 +7217 +7218 +7219 +7220 +7221 +7222 +7223 +7224 +7225 +7226 +7227 +7228 +7229 +7230 +7231 +7232 +7233 +7234 +7235 +7236 +7237 +7238 +7239 +7240 +7241 +7242 +7243 +7244 +7245 +7246 +7247 +7248 +7249 +7250 +7251 +7252 +7253 +7254 +7255 +7256 +7257 +7258 +7259 +7260 +7261 +7262 +7263 +7264 +7265 +7266 +7267 +7268 +7269 +7270 +7271 +7272 +7273 +7274 +7275 +7276 +7277 +7278 +7279 +7280 +7281 +7282 +7283 +7284 +7285 +7286 +7287 +7288 +7289 +7290 +7291 +7292 +7293 +7294 +7295 +7296 +7297 +7298 +7299 +7300 +7301 +7302 +7303 +7304 +7305 +7306 +7307 +7308 +7309 +7310 +7311 +7312 +7313 +7314 +7315 +7316 +7317 +7318 +7319 +7320 +7321 +7322 +7323 +7324 +7325 +7326 +7327 +7328 +7329 +7330 +7331 +7332 +7333 +7334 +7335 +7336 +7337 +7338 +7339 +7340 +7341 +7342 +7343 +7344 +7345 +7346 +7347 +7348 +7349 +7350 +7351 +7352 +7353 +7354 +7355 +7356 +7357 +7358 +7359 +7360 +7361 +7362 +7363 +7364 +7365 +7366 +7367 +7368 +7369 +7370 +7371 +7372 +7373 +7374 +7375 +7376 +7377 +7378 +7379 +7380 +7381 +7382 +7383 +7384 +7385 +7386 +7387 +7388 +7389 +7390 +7391 +7392 +7393 +7394 +7395 +7396 +7397 +7398 +7399 +7400 +7401 +7402 +7403 +7404 +7405 +7406 +7407 +7408 +7409 +7410 +7411 +7412 +7413 +7414 +7415 +7416 +7417 +7418 +7419 +7420 +7421 +7422 +7423 +7424 +7425 +7426 +7427 +7428 +7429 +7430 +7431 +7432 +7433 +7434 +7435 +7436 +7437 +7438 +7439 +7440 +7441 +7442 +7443 +7444 +7445 +7446 +7447 +7448 +7449 +7450 +7451 +7452 +7453 +7454 +7455 +7456 +7457 +7458 +7459 +7460 +7461 +7462 +7463 +7464 +7465 +7466 +7467 +7468 +7469 +7470 +7471 +7472 +7473 +7474 +7475 +7476 +7477 +7478 +7479 +7480 +7481 +7482 +7483 +7484 +7485 +7486 +7487 +7488 +7489 +7490 +7491 +7492 +7493 +7494 +7495 +7496 +7497 +7498 +7499 +7500 +7501 +7502 +7503 +7504 +7505 +7506 +7507 +7508 +7509 +7510 +7511 +7512 +7513 +7514 +7515 +7516 +7517 +7518 +7519 +7520 +7521 +7522 +7523 +7524 +7525 +7526 +7527 +7528 +7529 +7530 +7531 +7532 +7533 +7534 +7535 +7536 +7537 +7538 +7539 +7540 +7541 +7542 +7543 +7544 +7545 +7546 +7547 +7548 +7549 +7550 +7551 +7552 +7553 +7554 +7555 +7556 +7557 +7558 +7559 +7560 +7561 +7562 +7563 +7564 +7565 +7566 +7567 +7568 +7569 +7570 +7571 +7572 +7573 +7574 +7575 +7576 +7577 +7578 +7579 +7580 +7581 +7582 +7583 +7584 +7585 +7586 +7587 +7588 +7589 +7590 +7591 +7592 +7593 +7594 +7595 +7596 +7597 +7598 +7599 +7600 +7601 +7602 +7603 +7604 +7605 +7606 +7607 +7608 +7609 +7610 +7611 +7612 +7613 +7614 +7615 +7616 +7617 +7618 +7619 +7620 +7621 +7622 +7623 +7624 +7625 +7626 +7627 +7628 +7629 +7630 +7631 +7632 +7633 +7634 +7635 +7636 +7637 +7638 +7639 +7640 +7641 +7642 +7643 +7644 +7645 +7646 +7647 +7648 +7649 +7650 +7651 +7652 +7653 +7654 +7655 +7656 +7657 +7658 +7659 +7660 +7661 +7662 +7663 +7664 +7665 +7666 +7667 +7668 +7669 +7670 +7671 +7672 +7673 +7674 +7675 +7676 +7677 +7678 +7679 +7680 +7681 +7682 +7683 +7684 +7685 +7686 +7687 +7688 +7689 +7690 +7691 +7692 +7693 +7694 +7695 +7696 +7697 +7698 +7699 +7700 +7701 +7702 +7703 +7704 +7705 +7706 +7707 +7708 +7709 +7710 +7711 +7712 +7713 +7714 +7715 +7716 +7717 +7718 +7719 +7720 +7721 +7722 +7723 +7724 +7725 +7726 +7727 +7728 +7729 +7730 +7731 +7732 +7733 +7734 +7735 +7736 +7737 +7738 +7739 +7740 +7741 +7742 +7743 +7744 +7745 +7746 +7747 +7748 +7749 +7750 +7751 +7752 +7753 +7754 +7755 +7756 +7757 +7758 +7759 +7760 +7761 +7762 +7763 +7764 +7765 +7766 +7767 +7768 +7769 +7770 +7771 +7772 +7773 +7774 +7775 +7776 +7777 +7778 +7779 +7780 +7781 +7782 +7783 +7784 +7785 +7786 +7787 +7788 +7789 +7790 +7791 +7792 +7793 +7794 +7795 +7796 +7797 +7798 +7799 +7800 +7801 +7802 +7803 +7804 +7805 +7806 +7807 +7808 +7809 +7810 +7811 +7812 +7813 +7814 +7815 +7816 +7817 +7818 +7819 +7820 +7821 +7822 +7823 +7824 +7825 +7826 +7827 +7828 +7829 +7830 +7831 +7832 +7833 +7834 +7835 +7836 +7837 +7838 +7839 +7840 +7841 +7842 +7843 +7844 +7845 +7846 +7847 +7848 +7849 +7850 +7851 +7852 +7853 +7854 +7855 +7856 +7857 +7858 +7859 +7860 +7861 +7862 +7863 +7864 +7865 +7866 +7867 +7868 +7869 +7870 +7871 +7872 +7873 +7874 +7875 +7876 +7877 +7878 +7879 +7880 +7881 +7882 +7883 +7884 +7885 +7886 +7887 +7888 +7889 +7890 +7891 +7892 +7893 +7894 +7895 +7896 +7897 +7898 +7899 +7900 +7901 +7902 +7903 +7904 +7905 +7906 +7907 +7908 +7909 +7910 +7911 +7912 +7913 +7914 +7915 +7916 +7917 +7918 +7919 +7920 +7921 +7922 +7923 +7924 +7925 +7926 +7927 +7928 +7929 +7930 +7931 +7932 +7933 +7934 +7935 +7936 +7937 +7938 +7939 +7940 +7941 +7942 +7943 +7944 +7945 +7946 +7947 +7948 +7949 +7950 +7951 +7952 +7953 +7954 +7955 +7956 +7957 +7958 +7959 +7960 +7961 +7962 +7963 +7964 +7965 +7966 +7967 +7968 +7969 +7970 +7971 +7972 +7973 +7974 +7975 +7976 +7977 +7978 +7979 +7980 +7981 +7982 +7983 +7984 +7985 +7986 +7987 +7988 +7989 +7990 +7991 +7992 +7993 +7994 +7995 +7996 +7997 +7998 +7999 +8000 +8001 +8002 +8003 +8004 +8005 +8006 +8007 +8008 +8009 +8010 +8011 +8012 +8013 +8014 +8015 +8016 +8017 +8018 +8019 +8020 +8021 +8022 +8023 +8024 +8025 +8026 +8027 +8028 +8029 +8030 +8031 +8032 +8033 +8034 +8035 +8036 +8037 +8038 +8039 +8040 +8041 +8042 +8043 +8044 +8045 +8046 +8047 +8048 +8049 +8050 +8051 +8052 +8053 +8054 +8055 +8056 +8057 +8058 +8059 +8060 +8061 +8062 +8063 +8064 +8065 +8066 +8067 +8068 +8069 +8070 +8071 +8072 +8073 +8074 +8075 +8076 +8077 +8078 +8079 +8080 +8081 +8082 +8083 +8084 +8085 +8086 +8087 +8088 +8089 +8090 +8091 +8092 +8093 +8094 +8095 +8096 +8097 +8098 +8099 +8100 +8101 +8102 +8103 +8104 +8105 +8106 +8107 +8108 +8109 +8110 +8111 +8112 +8113 +8114 +8115 +8116 +8117 +8118 +8119 +8120 +8121 +8122 +8123 +8124 +8125 +8126 +8127 +8128 +8129 +8130 +8131 +8132 +8133 +8134 +8135 +8136 +8137 +8138 +8139 +8140 +8141 +8142 +8143 +8144 +8145 +8146 +8147 +8148 +8149 +8150 +8151 +8152 +8153 +8154 +8155 +8156 +8157 +8158 +8159 +8160 +8161 +8162 +8163 +8164 +8165 +8166 +8167 +8168 +8169 +8170 +8171 +8172 +8173 +8174 +8175 +8176 +8177 +8178 +8179 +8180 +8181 +8182 +8183 +8184 +8185 +8186 +8187 +8188 +8189 +8190 +8191 +8192 +8193 +8194 +8195 +8196 +8197 +8198 +8199 +8200 +8201 +8202 +8203 +8204 +8205 +8206 +8207 +8208 +8209 +8210 +8211 +8212 +8213 +8214 +8215 +8216 +8217 +8218 +8219 +8220 +8221 +8222 +8223 +8224 +8225 +8226 +8227 +8228 +8229 +8230 +8231 +8232 +8233 +8234 +8235 +8236 +8237 +8238 +8239 +8240 +8241 +8242 +8243 +8244 +8245 +8246 +8247 +8248 +8249 +8250 +8251 +8252 +8253 +8254 +8255 +8256 +8257 +8258 +8259 +8260 +8261 +8262 +8263 +8264 +8265 +8266 +8267 +8268 +8269 +8270 +8271 +8272 +8273 +8274 +8275 +8276 +8277 +8278 +8279 +8280 +8281 +8282 +8283 +8284 +8285 +8286 +8287 +8288 +8289 +8290 +8291 +8292 +8293 +8294 +8295 +8296 +8297 +8298 +8299 +8300 +8301 +8302 +8303 +8304 +8305 +8306 +8307 +8308 +8309 +8310 +8311 +8312 +8313 +8314 +8315 +8316 +8317 +8318 +8319 +8320 +8321 +8322 +8323 +8324 +8325 +8326 +8327 +8328 +8329 +8330 +8331 +8332 +8333 +8334 +8335 +8336 +8337 +8338 +8339 +8340 +8341 +8342 +8343 +8344 +8345 +8346 +8347 +8348 +8349 +8350 +8351 +8352 +8353 +8354 +8355 +8356 +8357 +8358 +8359 +8360 +8361 +8362 +8363 +8364 +8365 +8366 +8367 +8368 +8369 +8370 +8371 +8372 +8373 +8374 +8375 +8376 +8377 +8378 +8379 +8380 +8381 +8382 +8383 +8384 +8385 +8386 +8387 +8388 +8389 +8390 +8391 +8392 +8393 +8394 +8395 +8396 +8397 +8398 +8399 +8400 +8401 +8402 +8403 +8404 +8405 +8406 +8407 +8408 +8409 +8410 +8411 +8412 +8413 +8414 +8415 +8416 +8417 +8418 +8419 +8420 +8421 +8422 +8423 +8424 +8425 +8426 +8427 +8428 +8429 +8430 +8431 +8432 +8433 +8434 +8435 +8436 +8437 +8438 +8439 +8440 +8441 +8442 +8443 +8444 +8445 +8446 +8447 +8448 +8449 +8450 +8451 +8452 +8453 +8454 +8455 +8456 +8457 +8458 +8459 +8460 +8461 +8462 +8463 +8464 +8465 +8466 +8467 +8468 +8469 +8470 +8471 +8472 +8473 +8474 +8475 +8476 +8477 +8478 +8479 +8480 +8481 +8482 +8483 +8484 +8485 +8486 +8487 +8488 +8489 +8490 +8491 +8492 +8493 +8494 +8495 +8496 +8497 +8498 +8499 +8500 +8501 +8502 +8503 +8504 +8505 +8506 +8507 +8508 +8509 +8510 +8511 +8512 +8513 +8514 +8515 +8516 +8517 +8518 +8519 +8520 +8521 +8522 +8523 +8524 +8525 +8526 +8527 +8528 +8529 +8530 +8531 +8532 +8533 +8534 +8535 +8536 +8537 +8538 +8539 +8540 +8541 +8542 +8543 +8544 +8545 +8546 +8547 +8548 +8549 +8550 +8551 +8552 +8553 +8554 +8555 +8556 +8557 +8558 +8559 +8560 +8561 +8562 +8563 +8564 +8565 +8566 +8567 +8568 +8569 +8570 +8571 +8572 +8573 +8574 +8575 +8576 +8577 +8578 +8579 +8580 +8581 +8582 +8583 +8584 +8585 +8586 +8587 +8588 +8589 +8590 +8591 +8592 +8593 +8594 +8595 +8596 +8597 +8598 +8599 +8600 +8601 +8602 +8603 +8604 +8605 +8606 +8607 +8608 +8609 +8610 +8611 +8612 +8613 +8614 +8615 +8616 +8617 +8618 +8619 +8620 +8621 +8622 +8623 +8624 +8625 +8626 +8627 +8628 +8629 +8630 +8631 +8632 +8633 +8634 +8635 +8636 +8637 +8638 +8639 +8640 +8641 +8642 +8643 +8644 +8645 +8646 +8647 +8648 +8649 +8650 +8651 +8652 +8653 +8654 +8655 +8656 +8657 +8658 +8659 +8660 +8661 +8662 +8663 +8664 +8665 +8666 +8667 +8668 +8669 +8670 +8671 +8672 +8673 +8674 +8675 +8676 +8677 +8678 +8679 +8680 +8681 +8682 +8683 +8684 +8685 +8686 +8687 +8688 +8689 +8690 +8691 +8692 +8693 +8694 +8695 +8696 +8697 +8698 +8699 +8700 +8701 +8702 +8703 +8704 +8705 +8706 +8707 +8708 +8709 +8710 +8711 +8712 +8713 +8714 +8715 +8716 +8717 +8718 +8719 +8720 +8721 +8722 +8723 +8724 +8725 +8726 +8727 +8728 +8729 +8730 +8731 +8732 +8733 +8734 +8735 +8736 +8737 +8738 +8739 +8740 +8741 +8742 +8743 +8744 +8745 +8746 +8747 +8748 +8749 +8750 +8751 +8752 +8753 +8754 +8755 +8756 +8757 +8758 +8759 +8760 +8761 +8762 +8763 +8764 +8765 +8766 +8767 +8768 +8769 +8770 +8771 +8772 +8773 +8774 +8775 +8776 +8777 +8778 +8779 +8780 +8781 +8782 +8783 +8784 +8785 +8786 +8787 +8788 +8789 +8790 +8791 +8792 +8793 +8794 +8795 +8796 +8797 +8798 +8799 +8800 +8801 +8802 +8803 +8804 +8805 +8806 +8807 +8808 +8809 +8810 +8811 +8812 +8813 +8814 +8815 +8816 +8817 +8818 +8819 +8820 +8821 +8822 +8823 +8824 +8825 +8826 +8827 +8828 +8829 +8830 +8831 +8832 +8833 +8834 +8835 +8836 +8837 +8838 +8839 +8840 +8841 +8842 +8843 +8844 +8845 +8846 +8847 +8848 +8849 +8850 +8851 +8852 +8853 +8854 +8855 +8856 +8857 +8858 +8859 +8860 +8861 +8862 +8863 +8864 +8865 +8866 +8867 +8868 +8869 +8870 +8871 +8872 +8873 +8874 +8875 +8876 +8877 +8878 +8879 +8880 +8881 +8882 +8883 +8884 +8885 +8886 +8887 +8888 +8889 +8890 +8891 +8892 +8893 +8894 +8895 +8896 +8897 +8898 +8899 +8900 +8901 +8902 +8903 +8904 +8905 +8906 +8907 +8908 +8909 +8910 +8911 +8912 +8913 +8914 +8915 +8916 +8917 +8918 +8919 +8920 +8921 +8922 +8923 +8924 +8925 +8926 +8927 +8928 +8929 +8930 +8931 +8932 +8933 +8934 +8935 +8936 +8937 +8938 +8939 +8940 +8941 +8942 +8943 +8944 +8945 +8946 +8947 +8948 +8949 +8950 +8951 +8952 +8953 +8954 +8955 +8956 +8957 +8958 +8959 +8960 +8961 +8962 +8963 +8964 +8965 +8966 +8967 +8968 +8969 +8970 +8971 +8972 +8973 +8974 +8975 +8976 +8977 +8978 +8979 +8980 +8981 +8982 +8983 +8984 +8985 +8986 +8987 +8988 +8989 +8990 +8991 +8992 +8993 +8994 +8995 +8996 +8997 +8998 +8999 +9000 +9001 +9002 +9003 +9004 +9005 +9006 +9007 +9008 +9009 +9010 +9011 +9012 +9013 +9014 +9015 +9016 +9017 +9018 +9019 +9020 +9021 +9022 +9023 +9024 +9025 +9026 +9027 +9028 +9029 +9030 +9031 +9032 +9033 +9034 +9035 +9036 +9037 +9038 +9039 +9040 +9041 +9042 +9043 +9044 +9045 +9046 +9047 +9048 +9049 +9050 +9051 +9052 +9053 +9054 +9055 +9056 +9057 +9058 +9059 +9060 +9061 +9062 +9063 +9064 +9065 +9066 +9067 +9068 +9069 +9070 +9071 +9072 +9073 +9074 +9075 +9076 +9077 +9078 +9079 +9080 +9081 +9082 +9083 +9084 +9085 +9086 +9087 +9088 +9089 +9090 +9091 +9092 +9093 +9094 +9095 +9096 +9097 +9098 +9099 +9100 +9101 +9102 +9103 +9104 +9105 +9106 +9107 +9108 +9109 +9110 +9111 +9112 +9113 +9114 +9115 +9116 +9117 +9118 +9119 +9120 +9121 +9122 +9123 +9124 +9125 +9126 +9127 +9128 +9129 +9130 +9131 +9132 +9133 +9134 +9135 +9136 +9137 +9138 +9139 +9140 +9141 +9142 +9143 +9144 +9145 +9146 +9147 +9148 +9149 +9150 +9151 +9152 +9153 +9154 +9155 +9156 +9157 +9158 +9159 +9160 +9161 +9162 +9163 +9164 +9165 +9166 +9167 +9168 +9169 +9170 +9171 +9172 +9173 +9174 +9175 +9176 +9177 +9178 +9179 +9180 +9181 +9182 +9183 +9184 +9185 +9186 +9187 +9188 +9189 +9190 +9191 +9192 +9193 +9194 +9195 +9196 +9197 +9198 +9199 +9200 +9201 +9202 +9203 +9204 +9205 +9206 +9207 +9208 +9209 +9210 +9211 +9212 +9213 +9214 +9215 +9216 +9217 +9218 +9219 +9220 +9221 +9222 +9223 +9224 +9225 +9226 +9227 +9228 +9229 +9230 +9231 +9232 +9233 +9234 +9235 +9236 +9237 +9238 +9239 +9240 +9241 +9242 +9243 +9244 +9245 +9246 +9247 +9248 +9249 +9250 +9251 +9252 +9253 +9254 +9255 +9256 +9257 +9258 +9259 +9260 +9261 +9262 +9263 +9264 +9265 +9266 +9267 +9268 +9269 +9270 +9271 +9272 +9273 +9274 +9275 +9276 +9277 +9278 +9279 +9280 +9281 +9282 +9283 +9284 +9285 +9286 +9287 +9288 +9289 +9290 +9291 +9292 +9293 +9294 +9295 +9296 +9297 +9298 +9299 +9300 +9301 +9302 +9303 +9304 +9305 +9306 +9307 +9308 +9309 +9310 +9311 +9312 +9313 +9314 +9315 +9316 +9317 +9318 +9319 +9320 +9321 +9322 +9323 +9324 +9325 +9326 +9327 +9328 +9329 +9330 +9331 +9332 +9333 +9334 +9335 +9336 +9337 +9338 +9339 +9340 +9341 +9342 +9343 +9344 +9345 +9346 +9347 +9348 +9349 +9350 +9351 +9352 +9353 +9354 +9355 +9356 +9357 +9358 +9359 +9360 +9361 +9362 +9363 +9364 +9365 +9366 +9367 +9368 +9369 +9370 +9371 +9372 +9373 +9374 +9375 +9376 +9377 +9378 +9379 +9380 +9381 +9382 +9383 +9384 +9385 +9386 +9387 +9388 +9389 +9390 +9391 +9392 +9393 +9394 +9395 +9396 +9397 +9398 +9399 +9400 +9401 +9402 +9403 +9404 +9405 +9406 +9407 +9408 +9409 +9410 +9411 +9412 +9413 +9414 +9415 +9416 +9417 +9418 +9419 +9420 +9421 +9422 +9423 +9424 +9425 +9426 +9427 +9428 +9429 +9430 +9431 +9432 +9433 +9434 +9435 +9436 +9437 +9438 +9439 +9440 +9441 +9442 +9443 +9444 +9445 +9446 +9447 +9448 +9449 +9450 +9451 +9452 +9453 +9454 +9455 +9456 +9457 +9458 +9459 +9460 +9461 +9462 +9463 +9464 +9465 +9466 +9467 +9468 +9469 +9470 +9471 +9472 +9473 +9474 +9475 +9476 +9477 +9478 +9479 +9480 +9481 +9482 +9483 +9484 +9485 +9486 +9487 +9488 +9489 +9490 +9491 +9492 +9493 +9494 +9495 +9496 +9497 +9498 +9499 +9500 +9501 +9502 +9503 +9504 +9505 +9506 +9507 +9508 +9509 +9510 +9511 +9512 +9513 +9514 +9515 +9516 +9517 +9518 +9519 +9520 +9521 +9522 +9523 +9524 +9525 +9526 +9527 +9528 +9529 +9530 +9531 +9532 +9533 +9534 +9535 +9536 +9537 +9538 +9539 +9540 +9541 +9542 +9543 +9544 +9545 +9546 +9547 +9548 +9549 +9550 +9551 +9552 +9553 +9554 +9555 +9556 +9557 +9558 +9559 +9560 +9561 +9562 +9563 +9564 +9565 +9566 +9567 +9568 +9569 +9570 +9571 +9572 +9573 +9574 +9575 +9576 +9577 +9578 +9579 +9580 +9581 +9582 +9583 +9584 +9585 +9586 +9587 +9588 +9589 +9590 +9591 +9592 +9593 +9594 +9595 +9596 +9597 +9598 +9599 +9600 +9601 +9602 +9603 +9604 +9605 +9606 +9607 +9608 +9609 +9610 +9611 +9612 +9613 +9614 +9615 +9616 +9617 +9618 +9619 +9620 +9621 +9622 +9623 +9624 +9625 +9626 +9627 +9628 +9629 +9630 +9631 +9632 +9633 +9634 +9635 +9636 +9637 +9638 +9639 +9640 +9641 +9642 +9643 +9644 +9645 +9646 +9647 +9648 +9649 +9650 +9651 +9652 +9653 +9654 +9655 +9656 +9657 +9658 +9659 +9660 +9661 +9662 +9663 +9664 +9665 +9666 +9667 +9668 +9669 +9670 +9671 +9672 +9673 +9674 +9675 +9676 +9677 +9678 +9679 +9680 +9681 +9682 +9683 +9684 +9685 +9686 +9687 +9688 +9689 +9690 +9691 +9692 +9693 +9694 +9695 +9696 +9697 +9698 +9699 +9700 +9701 +9702 +9703 +9704 +9705 +9706 +9707 +9708 +9709 +9710 +9711 +9712 +9713 +9714 +9715 +9716 +9717 +9718 +9719 +9720 +9721 +9722 +9723 +9724 +9725 +9726 +9727 +9728 +9729 +9730 +9731 +9732 +9733 +9734 +9735 +9736 +9737 +9738 +9739 +9740 +9741 +9742 +9743 +9744 +9745 +9746 +9747 +9748 +9749 +9750 +9751 +9752 +9753 +9754 +9755 +9756 +9757 +9758 +9759 +9760 +9761 +9762 +9763 +9764 +9765 +9766 +9767 +9768 +9769 +9770 +9771 +9772 +9773 +9774 +9775 +9776 +9777 +9778 +9779 +9780 +9781 +9782 +9783 +9784 +9785 +9786 +9787 +9788 +9789 +9790 +9791 +9792 +9793 +9794 +9795 +9796 +9797 +9798 +9799 +9800 +9801 +9802 +9803 +9804 +9805 +9806 +9807 +9808 +9809 +9810 +9811 +9812 +9813 +9814 +9815 +9816 +9817 +9818 +9819 +9820 +9821 +9822 +9823 +9824 +9825 +9826 +9827 +9828 +9829 +9830 +9831 +9832 +9833 +9834 +9835 +9836 +9837 +9838 +9839 +9840 +9841 +9842 +9843 +9844 +9845 +9846 +9847 +9848 +9849 +9850 +9851 +9852 +9853 +9854 +9855 +9856 +9857 +9858 +9859 +9860 +9861 +9862 +9863 +9864 +9865 +9866 +9867 +9868 +9869 +9870 +9871 +9872 +9873 +9874 +9875 +9876 +9877 +9878 +9879 +9880 +9881 +9882 +9883 +9884 +9885 +9886 +9887 +9888 +9889 +9890 +9891 +9892 +9893 +9894 +9895 +9896 +9897 +9898 +9899 +9900 +9901 +9902 +9903 +9904 +9905 +9906 +9907 +9908 +9909 +9910 +9911 +9912 +9913 +9914 +9915 +9916 +9917 +9918 +9919 +9920 +9921 +9922 +9923 +9924 +9925 +9926 +9927 +9928 +9929 +9930 +9931 +9932 +9933 +9934 +9935 +9936 +9937 +9938 +9939 +9940 +9941 +9942 +9943 +9944 +9945 +9946 +9947 +9948 +9949 +9950 +9951 +9952 +9953 +9954 +9955 +9956 +9957 +9958 +9959 +9960 +9961 +9962 +9963 +9964 +9965 +9966 +9967 +9968 +9969 +9970 +9971 +9972 +9973 +9974 +9975 +9976 +9977 +9978 +9979 +9980 +9981 +9982 +9983 +9984 +9985 +9986 +9987 +9988 +9989 +9990 +9991 +9992 +9993 +9994 +9995 +9996 +9997 +9998 +9999 +10000 +10001 +10002 +10003 +10004 +10005 +10006 +10007 +10008 +10009 +10010 +10011 +10012 +10013 +10014 +10015 +10016 +10017 +10018 +10019 +10020 +10021 +10022 +10023 +10024 +10025 +10026 +10027 +10028 +10029 +10030 +10031 +10032 +10033 +10034 +10035 +10036 +10037 +10038 +10039 +10040 +10041 +10042 +10043 +10044 +10045 +10046 +10047 +10048 +10049 +10050 +10051 +10052 +10053 +10054 +10055 +10056 +10057 +10058 +10059 +10060 +10061 +10062 +10063 +10064 +10065 +10066 +10067 +10068 +10069 +10070 +10071 +10072 +10073 +10074 +10075 +10076 +10077 +10078 +10079 +10080 +10081 +10082 +10083 +10084 +10085 +10086 +10087 +10088 +10089 +10090 +10091 +10092 +10093 +10094 +10095 +10096 +10097 +10098 +10099 +10100 +10101 +10102 +10103 +10104 +10105 +10106 +10107 +10108 +10109 +10110 +10111 +10112 +10113 +10114 +10115 +10116 +10117 +10118 +10119 +10120 +10121 +10122 +10123 +10124 +10125 +10126 +10127 +10128 +10129 +10130 +10131 +10132 +10133 +10134 +10135 +10136 +10137 +10138 +10139 +10140 +10141 +10142 +10143 +10144 +10145 +10146 +10147 +10148 +10149 +10150 +10151 +10152 +10153 +10154 +10155 +10156 +10157 +10158 +10159 +10160 +10161 +10162 +10163 +10164 +10165 +10166 +10167 +10168 +10169 +10170 +10171 +10172 +10173 +10174 +10175 +10176 +10177 +10178 +10179 +10180 +10181 +10182 +10183 +10184 +10185 +10186 +10187 +10188 +10189 +10190 +10191 +10192 +10193 +10194 +10195 +10196 +10197 +10198 +10199 +10200 +10201 +10202 +10203 +10204 +10205 +10206 +10207 +10208 +10209 +10210 +10211 +10212 +10213 +10214 +10215 +10216 +10217 +10218 +10219 +10220 +10221 +10222 +10223 +10224 +10225 +10226 +10227 +10228 +10229 +10230 +10231 +10232 +10233 +10234 +10235 +10236 +10237 +10238 +10239 +10240 +10241 +10242 +10243 +10244 +10245 +10246 +10247 +10248 +10249 +10250 +10251 +10252 +10253 +10254 +10255 +10256 +10257 +10258 +10259 +10260 +10261 +10262 +10263 +10264 +10265 +10266 +10267 +10268 +10269 +10270 +10271 +10272 +10273 +10274 +10275 +10276 +10277 +10278 +10279 +10280 +10281 +10282 +10283 +10284 +10285 +10286 +10287 +10288 +10289 +10290 +10291 +10292 +10293 +10294 +10295 +10296 +10297 +10298 +10299 +10300 +10301 +10302 +10303 +10304 +10305 +10306 +10307 +10308 +10309 +10310 +10311 +10312 +10313 +10314 +10315 +10316 +10317 +10318 +10319 +10320 +10321 +10322 +10323 +10324 +10325 +10326 +10327 +10328 +10329 +10330 +10331 +10332 +10333 +10334 +10335 +10336 +10337 +10338 +10339 +10340 +10341 +10342 +10343 +10344 +10345 +10346 +10347 +10348 +10349 +10350 +10351 +10352 +10353 +10354 +10355 +10356 +10357 +10358 +10359 +10360 +10361 +10362 +10363 +10364 +10365 +10366 +10367 +10368 +10369 +10370 +10371 +10372 +10373 +10374 +10375 +10376 +10377 +10378 +10379 +10380 +10381 +10382 +10383 +10384 +10385 +10386 +10387 +10388 +10389 +10390 +10391 +10392 +10393 +10394 +10395 +10396 +10397 +10398 +10399 +10400 +10401 +10402 +10403 +10404 +10405 +10406 +10407 +10408 +10409 +10410 +10411 +10412 +10413 +10414 +10415 +10416 +10417 +10418 +10419 +10420 +10421 +10422 +10423 +10424 +10425 +10426 +10427 +10428 +10429 +10430 +10431 +10432 +10433 +10434 +10435 +10436 +10437 +10438 +10439 +10440 +10441 +10442 +10443 +10444 +10445 +10446 +10447 +10448 +10449 +10450 +10451 +10452 +10453 +10454 +10455 +10456 +10457 +10458 +10459 +10460 +10461 +10462 +10463 +10464 +10465 +10466 +10467 +10468 +10469 +10470 +10471 +10472 +10473 +10474 +10475 +10476 +10477 +10478 +10479 +10480 +10481 +10482 +10483 +10484 +10485 +10486 +10487 +10488 +10489 +10490 +10491 +10492 +10493 +10494 +10495 +10496 +10497 +10498 +10499 +10500 +10501 +10502 +10503 +10504 +10505 +10506 +10507 +10508 +10509 +10510 +10511 +10512 +10513 +10514 +10515 +10516 +10517 +10518 +10519 +10520 +10521 +10522 +10523 +10524 +10525 +10526 +10527 +10528 +10529 +10530 +10531 +10532 +10533 +10534 +10535 +10536 +10537 +10538 +10539 +10540 +10541 +10542 +10543 +10544 +10545 +10546 +10547 +10548 +10549 +10550 +10551 +10552 +10553 +10554 +10555 +10556 +10557 +10558 +10559 +10560 +10561 +10562 +10563 +10564 +10565 +10566 +10567 +10568 +10569 +10570 +10571 +10572 +10573 +10574 +10575 +10576 +10577 +10578 +10579 +10580 +10581 +10582 +10583 +10584 +10585 +10586 +10587 +10588 +10589 +10590 +10591 +10592 +10593 +10594 +10595 +10596 +10597 +10598 +10599 +10600 +10601 +10602 +10603 +10604 +10605 +10606 +10607 +10608 +10609 +10610 +10611 +10612 +10613 +10614 +10615 +10616 +10617 +10618 +10619 +10620 +10621 +10622 +10623 +10624 +10625 +10626 +10627 +10628 +10629 +10630 +10631 +10632 +10633 +10634 +10635 +10636 +10637 +10638 +10639 +10640 +10641 +10642 +10643 +10644 +10645 +10646 +10647 +10648 +10649 +10650 +10651 +10652 +10653 +10654 +10655 +10656 +10657 +10658 +10659 +10660 +10661 +10662 +10663 +10664 +10665 +10666 +10667 +10668 +10669 +10670 +10671 +10672 +10673 +10674 +10675 +10676 +10677 +10678 +10679 +10680 +10681 +10682 +10683 +10684 +10685 +10686 +10687 +10688 +10689 +10690 +10691 +10692 +10693 +10694 +10695 +10696 +10697 +10698 +10699 +10700 +10701 +10702 +10703 +10704 +10705 +10706 +10707 +10708 +10709 +10710 +10711 +10712 +10713 +10714 +10715 +10716 +10717 +10718 +10719 +10720 +10721 +10722 +10723 +10724 +10725 +10726 +10727 +10728 +10729 +10730 +10731 +10732 +10733 +10734 +10735 +10736 +10737 +10738 +10739 +10740 +10741 +10742 +10743 +10744 +10745 +10746 +10747 +10748 +10749 +10750 +10751 +10752 +10753 +10754 +10755 +10756 +10757 +10758 +10759 +10760 +10761 +10762 +10763 +10764 +10765 +10766 +10767 +10768 +10769 +10770 +10771 +10772 +10773 +10774 +10775 +10776 +10777 +10778 +10779 +10780 +10781 +10782 +10783 +10784 +10785 +10786 +10787 +10788 +10789 +10790 +10791 +10792 +10793 +10794 +10795 +10796 +10797 +10798 +10799 +10800 +10801 +10802 +10803 +10804 +10805 +10806 +10807 +10808 +10809 +10810 +10811 +10812 +10813 +10814 +10815 +10816 +10817 +10818 +10819 +10820 +10821 +10822 +10823 +10824 +10825 +10826 +10827 +10828 +10829 +10830 +10831 +10832 +10833 +10834 +10835 +10836 +10837 +10838 +10839 +10840 +10841 +10842 +10843 +10844 +10845 +10846 +10847 +10848 +10849 +10850 +10851 +10852 +10853 +10854 +10855 +10856 +10857 +10858 +10859 +10860 +10861 +10862 +10863 +10864 +10865 +10866 +10867 +10868 +10869 +10870 +10871 +10872 +10873 +10874 +10875 +10876 +10877 +10878 +10879 +10880 +10881 +10882 +10883 +10884 +10885 +10886 +10887 +10888 +10889 +10890 +10891 +10892 +10893 +10894 +10895 +10896 +10897 +10898 +10899 +10900 +10901 +10902 +10903 +10904 +10905 +10906 +10907 +10908 +10909 +10910 +10911 +10912 +10913 +10914 +10915 +10916 +10917 +10918 +10919 +10920 +10921 +10922 +10923 +10924 +10925 +10926 +10927 +10928 +10929 +10930 +10931 +10932 +10933 +10934 +10935 +10936 +10937 +10938 +10939 +10940 +10941 +10942 +10943 +10944 +10945 +10946 +10947 +10948 +10949 +10950 +10951 +10952 +10953 +10954 +10955 +10956 +10957 +10958 +10959 +10960 +10961 +10962 +10963 +10964 +10965 +10966 +10967 +10968 +10969 +10970 +10971 +10972 +10973 +10974 +10975 +10976 +10977 +10978 +10979 +10980 +10981 +10982 +10983 +10984 +10985 +10986 +10987 +10988 +10989 +10990 +10991 +10992 +10993 +10994 +10995 +10996 +10997 +10998 +10999 +11000 +11001 +11002 +11003 +11004 +11005 +11006 +11007 +11008 +11009 +11010 +11011 +11012 +11013 +11014 +11015 +11016 +11017 +11018 +11019 +11020 +11021 +11022 +11023 +11024 +11025 +11026 +11027 +11028 +11029 +11030 +11031 +11032 +11033 +11034 +11035 +11036 +11037 +11038 +11039 +11040 +11041 +11042 +11043 +11044 +11045 +11046 +11047 +11048 +11049 +11050 +11051 +11052 +11053 +11054 +11055 +11056 +11057 +11058 +11059 +11060 +11061 +11062 +11063 +11064 +11065 +11066 +11067 +11068 +11069 +11070 +11071 +11072 +11073 +11074 +11075 +11076 +11077 +11078 +11079 +11080 +11081 +11082 +11083 +11084 +11085 +11086 +11087 +11088 +11089 +11090 +11091 +11092 +11093 +11094 +11095 +11096 +11097 +11098 +11099 +11100 +11101 +11102 +11103 +11104 +11105 +11106 +11107 +11108 +11109 +11110 +11111 +11112 +11113 +11114 +11115 +11116 +11117 +11118 +11119 +11120 +11121 +11122 +11123 +11124 +11125 +11126 +11127 +11128 +11129 +11130 +11131 +11132 +11133 +11134 +11135 +11136 +11137 +11138 +11139 +11140 +11141 +11142 +11143 +11144 +11145 +11146 +11147 +11148 +11149 +11150 +11151 +11152 +11153 +11154 +11155 +11156 +11157 +11158 +11159 +11160 +11161 +11162 +11163 +11164 +11165 +11166 +11167 +11168 +11169 +11170 +11171 +11172 +11173 +11174 +11175 +11176 +11177 +11178 +11179 +11180 +11181 +11182 +11183 +11184 +11185 +11186 +11187 +11188 +11189 +11190 +11191 +11192 +11193 +11194 +11195 +11196 +11197 +11198 +11199 +11200 +11201 +11202 +11203 +11204 +11205 +11206 +11207 +11208 +11209 +11210 +11211 +11212 +11213 +11214 +11215 +11216 +11217 +11218 +11219 +11220 +11221 +11222 +11223 +11224 +11225 +11226 +11227 +11228 +11229 +11230 +11231 +11232 +11233 +11234 +11235 +11236 +11237 +11238 +11239 +11240 +11241 +11242 +11243 +11244 +11245 +11246 +11247 +11248 +11249 +11250 +11251 +11252 +11253 +11254 +11255 +11256 +11257 +11258 +11259 +11260 +11261 +11262 +11263 +11264 +11265 +11266 +11267 +11268 +11269 +11270 +11271 +11272 +11273 +11274 +11275 +11276 +11277 +11278 +11279 +11280 +11281 +11282 +11283 +11284 +11285 +11286 +11287 +11288 +11289 +11290 +11291 +11292 +11293 +11294 +11295 +11296 +11297 +11298 +11299 +11300 +11301 +11302 +11303 +11304 +11305 +11306 +11307 +11308 +11309 +11310 +11311 +11312 +11313 +11314 +11315 +11316 +11317 +11318 +11319 +11320 +11321 +11322 +11323 +11324 +11325 +11326 +11327 +11328 +11329 +11330 +11331 +11332 +11333 +11334 +11335 +11336 +11337 +11338 +11339 +11340 +11341 +11342 +11343 +11344 +11345 +11346 +11347 +11348 +11349 +11350 +11351 +11352 +11353 +11354 +11355 +11356 +11357 +11358 +11359 +11360 +11361 +11362 +11363 +11364 +11365 +11366 +11367 +11368 +11369 +11370 +11371 +11372 +11373 +11374 +11375 +11376 +11377 +11378 +11379 +11380 +11381 +11382 +11383 +11384 +11385 +11386 +11387 +11388 +11389 +11390 +11391 +11392 +11393 +11394 +11395 +11396 +11397 +11398 +11399 +11400 +11401 +11402 +11403 +11404 +11405 +11406 +11407 +11408 +11409 +11410 +11411 +11412 +11413 +11414 +11415 +11416 +11417 +11418 +11419 +11420 +11421 +11422 +11423 +11424 +11425 +11426 +11427 +11428 +11429 +11430 +11431 +11432 +11433 +11434 +11435 +11436 +11437 +11438 +11439 +11440 +11441 +11442 +11443 +11444 +11445 +11446 +11447 +11448 +11449 +11450 +11451 +11452 +11453 +11454 +11455 +11456 +11457 +11458 +11459 +11460 +11461 +11462 +11463 +11464 +11465 +11466 +11467 +11468 +11469 +11470 +11471 +11472 +11473 +11474 +11475 +11476 +11477 +11478 +11479 +11480 +11481 +11482 +11483 +11484 +11485 +11486 +11487 +11488 +11489 +11490 +11491 +11492 +11493 +11494 +11495 +11496 +11497 +11498 +11499 +11500 +11501 +11502 +11503 +11504 +11505 +11506 +11507 +11508 +11509 +11510 +11511 +11512 +11513 +11514 +11515 +11516 +11517 +11518 +11519 +11520 +11521 +11522 +11523 +11524 +11525 +11526 +11527 +11528 +11529 +11530 +11531 +11532 +11533 +11534 +11535 +11536 +11537 +11538 +11539 +11540 +11541 +11542 +11543 +11544 +11545 +11546 +11547 +11548 +11549 +11550 +11551 +11552 +11553 +11554 +11555 +11556 +11557 +11558 +11559 +11560 +11561 +11562 +11563 +11564 +11565 +11566 +11567 +11568 +11569 +11570 +11571 +11572 +11573 +11574 +11575 +11576 +11577 +11578 +11579 +11580 +11581 +11582 +11583 +11584 +11585 +11586 +11587 +11588 +11589 +11590 +11591 +11592 +11593 +11594 +11595 +11596 +11597 +11598 +11599 +11600 +11601 +11602 +11603 +11604 +11605 +11606 +11607 +11608 +11609 +11610 +11611 +11612 +11613 +11614 +11615 +11616 +11617 +11618 +11619 +11620 +11621 +11622 +11623 +11624 +11625 +11626 +11627 +11628 +11629 +11630 +11631 +11632 +11633 +11634 +11635 +11636 +11637 +11638 +11639 +11640 +11641 +11642 +11643 +11644 +11645 +11646 +11647 +11648 +11649 +11650 +11651 +11652 +11653 +11654 +11655 +11656 +11657 +11658 +11659 +11660 +11661 +11662 +11663 +11664 +11665 +11666 +11667 +11668 +11669 +11670 +11671 +11672 +11673 +11674 +11675 +11676 +11677 +11678 +11679 +11680 +11681 +11682 +11683 +11684 +11685 +11686 +11687 +11688 +11689 +11690 +11691 +11692 +11693 +11694 +11695 +11696 +11697 +11698 +11699 +11700 +11701 +11702 +11703 +11704 +11705 +11706 +11707 +11708 +11709 +11710 +11711 +11712 +11713 +11714 +11715 +11716 +11717 +11718 +11719 +11720 +11721 +11722 +11723 +11724 +11725 +11726 +11727 +11728 +11729 +11730 +11731 +11732 +11733 +11734 +11735 +11736 +11737 +11738 +11739 +11740 +11741 +11742 +11743 +11744 +11745 +11746 +11747 +11748 +11749 +11750 +11751 +11752 +11753 +11754 +11755 +11756 +11757 +11758 +11759 +11760 +11761 +11762 +11763 +11764 +11765 +11766 +11767 +11768 +11769 +11770 +11771 +11772 +11773 +11774 +11775 +11776 +11777 +11778 +11779 +11780 +11781 +11782 +11783 +11784 +11785 +11786 +11787 +11788 +11789 +11790 +11791 +11792 +11793 +11794 +11795 +11796 +11797 +11798 +11799 +11800 +11801 +11802 +11803 +11804 +11805 +11806 +11807 +11808 +11809 +11810 +11811 +11812 +11813 +11814 +11815 +11816 +11817 +11818 +11819 +11820 +11821 +11822 +11823 +11824 +11825 +11826 +11827 +11828 +11829 +11830 +11831 +11832 +11833 +11834 +11835 +11836 +11837 +11838 +11839 +11840 +11841 +11842 +11843 +11844 +11845 +11846 +11847 +11848 +11849 +11850 +11851 +11852 +11853 +11854 +11855 +11856 +11857 +11858 +11859 +11860 +11861 +11862 +11863 +11864 +11865 +11866 +11867 +11868 +11869 +11870 +11871 +11872 +11873 +11874 +11875 +11876 +11877 +11878 +11879 +11880 +11881 +11882 +11883 +11884 +11885 +11886 +11887 +11888 +11889 +11890 +11891 +11892 +11893 +11894 +11895 +11896 +11897 +11898 +11899 +11900 +11901 +11902 +11903 +11904 +11905 +11906 +11907 +11908 +11909 +11910 +11911 +11912 +11913 +11914 +11915 +11916 +11917 +11918 +11919 +11920 +11921 +11922 +11923 +11924 +11925 +11926 +11927 +11928 +11929 +11930 +11931 +11932 +11933 +11934 +11935 +11936 +11937 +11938 +11939 +11940 +11941 +11942 +11943 +11944 +11945 +11946 +11947 +11948 +11949 +11950 +11951 +11952 +11953 +11954 +11955 +11956 +11957 +11958 +11959 +11960 +11961 +11962 +11963 +11964 +11965 +11966 +11967 +11968 +11969 +11970 +11971 +11972 +11973 +11974 +11975 +11976 +11977 +11978 +11979 +11980 +11981 +11982 +11983 +11984 +11985 +11986 +11987 +11988 +11989 +11990 +11991 +11992 +11993 +11994 +11995 +11996 +11997 +11998 +11999 +12000 +12001 +12002 +12003 +12004 +12005 +12006 +12007 +12008 +12009 +12010 +12011 +12012 +12013 +12014 +12015 +12016 +12017 +12018 +12019 +12020 +12021 +12022 +12023 +12024 +12025 +12026 +12027 +12028 +12029 +12030 +12031 +12032 +12033 +12034 +12035 +12036 +12037 +12038 +12039 +12040 +12041 +12042 +12043 +12044 +12045 +12046 +12047 +12048 +12049 +12050 +12051 +12052 +12053 +12054 +12055 +12056 +12057 +12058 +12059 +12060 +12061 +12062 +12063 +12064 +12065 +12066 +12067 +12068 +12069 +12070 +12071 +12072 +12073 +12074 +12075 +12076 +12077 +12078 +12079 +12080 +12081 +12082 +12083 +12084 +12085 +12086 +12087 +12088 +12089 +12090 +12091 +12092 +12093 +12094 +12095 +12096 +12097 +12098 +12099 +12100 +12101 +12102 +12103 +12104 +12105 +12106 +12107 +12108 +12109 +12110 +12111 +12112 +12113 +12114 +12115 +12116 +12117 +12118 +12119 +12120 +12121 +12122 +12123 +12124 +12125 +12126 +12127 +12128 +12129 +12130 +12131 +12132 +12133 +12134 +12135 +12136 +12137 +12138 +12139 +12140 +12141 +12142 +12143 +12144 +12145 +12146 +12147 +12148 +12149 +12150 +12151 +12152 +12153 +12154 +12155 +12156 +12157 +12158 +12159 +12160 +12161 +12162 +12163 +12164 +12165 +12166 +12167 +12168 +12169 +12170 +12171 +12172 +12173 +12174 +12175 +12176 +12177 +12178 +12179 +12180 +12181 +12182 +12183 +12184 +12185 +12186 +12187 +12188 +12189 +12190 +12191 +12192 +12193 +12194 +12195 +12196 +12197 +12198 +12199 +12200 +12201 +12202 +12203 +12204 +12205 +12206 +12207 +12208 +12209 +12210 +12211 +12212 +12213 +12214 +12215 +12216 +12217 +12218 +12219 +12220 +12221 +12222 +12223 +12224 +12225 +12226 +12227 +12228 +12229 +12230 +12231 +12232 +12233 +12234 +12235 +12236 +12237 +12238 +12239 +12240 +12241 +12242 +12243 +12244 +12245 +12246 +12247 +12248 +12249 +12250 +12251 +12252 +12253 +12254 +12255 +12256 +12257 +12258 +12259 +12260 +12261 +12262 +12263 +12264 +12265 +12266 +12267 +12268 +12269 +12270 +12271 +12272 +12273 +12274 +12275 +12276 +12277 +12278 +12279 +12280 +12281 +12282 +12283 +12284 +12285 +12286 +12287 +12288 +12289 +12290 +12291 +12292 +12293 +12294 +12295 +12296 +12297 +12298 +12299 +12300 +12301 +12302 +12303 +12304 +12305 +12306 +12307 +12308 +12309 +12310 +12311 +12312 +12313 +12314 +12315 +12316 +12317 +12318 +12319 +12320 +12321 +12322 +12323 +12324 +12325 +12326 +12327 +12328 +12329 +12330 +12331 +12332 +12333 +12334 +12335 +12336 +12337 +12338 +12339 +12340 +12341 +12342 +12343 +12344 +12345 +12346 +12347 +12348 +12349 +12350 +12351 +12352 +12353 +12354 +12355 +12356 +12357 +12358 +12359 +12360 +12361 +12362 +12363 +12364 +12365 +12366 +12367 +12368 +12369 +12370 +12371 +12372 +12373 +12374 +12375 +12376 +12377 +12378 +12379 +12380 +12381 +12382 +12383 +12384 +12385 +12386 +12387 +12388 +12389 +12390 +12391 +12392 +12393 +12394 +12395 +12396 +12397 +12398 +12399 +12400 +12401 +12402 +12403 +12404 +12405 +12406 +12407 +12408 +12409 +12410 +12411 +12412 +12413 +12414 +12415 +12416 +12417 +12418 +12419 +12420 +12421 +12422 +12423 +12424 +12425 +12426 +12427 +12428 +12429 +12430 +12431 +12432 +12433 +12434 +12435 +12436 +12437 +12438 +12439 +12440 +12441 +12442 +12443 +12444 +12445 +12446 +12447 +12448 +12449 +12450 +12451 +12452 +12453 +12454 +12455 +12456 +12457 +12458 +12459 +12460 +12461 +12462 +12463 +12464 +12465 +12466 +12467 +12468 +12469 +12470 +12471 +12472 +12473 +12474 +12475 +12476 +12477 +12478 +12479 +12480 +12481 +12482 +12483 +12484 +12485 +12486 +12487 +12488 +12489 +12490 +12491 +12492 +12493 +12494 +12495 +12496 +12497 +12498 +12499 +12500 +12501 +12502 +12503 +12504 +12505 +12506 +12507 +12508 +12509 +12510 +12511 +12512 +12513 +12514 +12515 +12516 +12517 +12518 +12519 +12520 +12521 +12522 +12523 +12524 +12525 +12526 +12527 +12528 +12529 +12530 +12531 +12532 +12533 +12534 +12535 +12536 +12537 +12538 +12539 +12540 +12541 +12542 +12543 +12544 +12545 +12546 +12547 +12548 +12549 +12550 +12551 +12552 +12553 +12554 +12555 +12556 +12557 +12558 +12559 +12560 +12561 +12562 +12563 +12564 +12565 +12566 +12567 +12568 +12569 +12570 +12571 +12572 +12573 +12574 +12575 +12576 +12577 +12578 +12579 +12580 +12581 +12582 +12583 +12584 +12585 +12586 +12587 +12588 +12589 +12590 +12591 +12592 +12593 +12594 +12595 +12596 +12597 +12598 +12599 +12600 +12601 +12602 +12603 +12604 +12605 +12606 +12607 +12608 +12609 +12610 +12611 +12612 +12613 +12614 +12615 +12616 +12617 +12618 +12619 +12620 +12621 +12622 +12623 +12624 +12625 +12626 +12627 +12628 +12629 +12630 +12631 +12632 +12633 +12634 +12635 +12636 +12637 +12638 +12639 +12640 +12641 +12642 +12643 +12644 +12645 +12646 +12647 +12648 +12649 +12650 +12651 +12652 +12653 +12654 +12655 +12656 +12657 +12658 +12659 +12660 +12661 +12662 +12663 +12664 +12665 +12666 +12667 +12668 +12669 +12670 +12671 +12672 +12673 +12674 +12675 +12676 +12677 +12678 +12679 +12680 +12681 +12682 +12683 +12684 +12685 +12686 +12687 +12688 +12689 +12690 +12691 +12692 +12693 +12694 +12695 +12696 +12697 +12698 +12699 +12700 +12701 +12702 +12703 +12704 +12705 +12706 +12707 +12708 +12709 +12710 +12711 +12712 +12713 +12714 +12715 +12716 +12717 +12718 +12719 +12720 +12721 +12722 +12723 +12724 +12725 +12726 +12727 +12728 +12729 +12730 +12731 +12732 +12733 +12734 +12735 +12736 +12737 +12738 +12739 +12740 +12741 +12742 +12743 +12744 +12745 +12746 +12747 +12748 +12749 +12750 +12751 +12752 +12753 +12754 +12755 +12756 +12757 +12758 +12759 +12760 +12761 +12762 +12763 +12764 +12765 +12766 +12767 +12768 +12769 +12770 +12771 +12772 +12773 +12774 +12775 +12776 +12777 +12778 +12779 +12780 +12781 +12782 +12783 +12784 +12785 +12786 +12787 +12788 +12789 +12790 +12791 +12792 +12793 +12794 +12795 +12796 +12797 +12798 +12799 +12800 +12801 +12802 +12803 +12804 +12805 +12806 +12807 +12808 +12809 +12810 +12811 +12812 +12813 +12814 +12815 +12816 +12817 +12818 +12819 +12820 +12821 +12822 +12823 +12824 +12825 +12826 +12827 +12828 +12829 +12830 +12831 +12832 +12833 +12834 +12835 +12836 +12837 +12838 +12839 +12840 +12841 +12842 +12843 +12844 +12845 +12846 +12847 +12848 +12849 +12850 +12851 +12852 +12853 +12854 +12855 +12856 +12857 +12858 +12859 +12860 +12861 +12862 +12863 +12864 +12865 +12866 +12867 +12868 +12869 +12870 +12871 +12872 +12873 +12874 +12875 +12876 +12877 +12878 +12879 +12880 +12881 +12882 +12883 +12884 +12885 +12886 +12887 +12888 +12889 +12890 +12891 +12892 +12893 +12894 +12895 +12896 +12897 +12898 +12899 +12900 +12901 +12902 +12903 +12904 +12905 +12906 +12907 +12908 +12909 +12910 +12911 +12912 +12913 +12914 +12915 +12916 +12917 +12918 +12919 +12920 +12921 +12922 +12923 +12924 +12925 +12926 +12927 +12928 +12929 +12930 +12931 +12932 +12933 +12934 +12935 +12936 +12937 +12938 +12939 +12940 +12941 +12942 +12943 +12944 +12945 +12946 +12947 +12948 +12949 +12950 +12951 +12952 +12953 +12954 +12955 +12956 +12957 +12958 +12959 +12960 +12961 +12962 +12963 +12964 +12965 +12966 +12967 +12968 +12969 +12970 +12971 +12972 +12973 +12974 +12975 +12976 +12977 +12978 +12979 +12980 +12981 +12982 +12983 +12984 +12985 +12986 +12987 +12988 +12989 +12990 +12991 +12992 +12993 +12994 +12995 +12996 +12997 +12998 +12999 +13000 +13001 +13002 +13003 +13004 +13005 +13006 +13007 +13008 +13009 +13010 +13011 +13012 +13013 +13014 +13015 +13016 +13017 +13018 +13019 +13020 +13021 +13022 +13023 +13024 +13025 +13026 +13027 +13028 +13029 +13030 +13031 +13032 +13033 +13034 +13035 +13036 +13037 +13038 +13039 +13040 +13041 +13042 +13043 +13044 +13045 +13046 +13047 +13048 +13049 +13050 +13051 +13052 +13053 +13054 +13055 +13056 +13057 +13058 +13059 +13060 +13061 +13062 +13063 +13064 +13065 +13066 +13067 +13068 +13069 +13070 +13071 +13072 +13073 +13074 +13075 +13076 +13077 +13078 +13079 +13080 +13081 +13082 +13083 +13084 +13085 +13086 +13087 +13088 +13089 +13090 +13091 +13092 +13093 +13094 +13095 +13096 +13097 +13098 +13099 +13100 +13101 +13102 +13103 +13104 +13105 +13106 +13107 +13108 +13109 +13110 +13111 +13112 +13113 +13114 +13115 +13116 +13117 +13118 +13119 +13120 +13121 +13122 +13123 +13124 +13125 +13126 +13127 +13128 +13129 +13130 +13131 +13132 +13133 +13134 +13135 +13136 +13137 +13138 +13139 +13140 +13141 +13142 +13143 +13144 +13145 +13146 +13147 +13148 +13149 +13150 +13151 +13152 +13153 +13154 +13155 +13156 +13157 +13158 +13159 +13160 +13161 +13162 +13163 +13164 +13165 +13166 +13167 +13168 +13169 +13170 +13171 +13172 +13173 +13174 +13175 +13176 +13177 +13178 +13179 +13180 +13181 +13182 +13183 +13184 +13185 +13186 +13187 +13188 +13189 +13190 +13191 +13192 +13193 +13194 +13195 +13196 +13197 +13198 +13199 +13200 +13201 +13202 +13203 +13204 +13205 +13206 +13207 +13208 +13209 +13210 +13211 +13212 +13213 +13214 +13215 +13216 +13217 +13218 +13219 +13220 +13221 +13222 +13223 +13224 +13225 +13226 +13227 +13228 +13229 +13230 +13231 +13232 +13233 +13234 +13235 +13236 +13237 +13238 +13239 +13240 +13241 +13242 +13243 +13244 +13245 +13246 +13247 +13248 +13249 +13250 +13251 +13252 +13253 +13254 +13255 +13256 +13257 +13258 +13259 +13260 +13261 +13262 +13263 +13264 +13265 +13266 +13267 +13268 +13269 +13270 +13271 +13272 +13273 +13274 +13275 +13276 +13277 +13278 +13279 +13280 +13281 +13282 +13283 +13284 +13285 +13286 +13287 +13288 +13289 +13290 +13291 +13292 +13293 +13294 +13295 +13296 +13297 +13298 +13299 +13300 +13301 +13302 +13303 +13304 +13305 +13306 +13307 +13308 +13309 +13310 +13311 +13312 +13313 +13314 +13315 +13316 +13317 +13318 +13319 +13320 +13321 +13322 +13323 +13324 +13325 +13326 +13327 +13328 +13329 +13330 +13331 +13332 +13333 +13334 +13335 +13336 +13337 +13338 +13339 +13340 +13341 +13342 +13343 +13344 +13345 +13346 +13347 +13348 +13349 +13350 +13351 +13352 +13353 +13354 +13355 +13356 +13357 +13358 +13359 +13360 +13361 +13362 +13363 +13364 +13365 +13366 +13367 +13368 +13369 +13370 +13371 +13372 +13373 +13374 +13375 +13376 +13377 +13378 +13379 +13380 +13381 +13382 +13383 +13384 +13385 +13386 +13387 +13388 +13389 +13390 +13391 +13392 +13393 +13394 +13395 +13396 +13397 +13398 +13399 +13400 +13401 +13402 +13403 +13404 +13405 +13406 +13407 +13408 +13409 +13410 +13411 +13412 +13413 +13414 +13415 +13416 +13417 +13418 +13419 +13420 +13421 +13422 +13423 +13424 +13425 +13426 +13427 +13428 +13429 +13430 +13431 +13432 +13433 +13434 +13435 +13436 +13437 +13438 +13439 +13440 +13441 +13442 +13443 +13444 +13445 +13446 +13447 +13448 +13449 +13450 +13451 +13452 +13453 +13454 +13455 +13456 +13457 +13458 +13459 +13460 +13461 +13462 +13463 +13464 +13465 +13466 +13467 +13468 +13469 +13470 +13471 +13472 +13473 +13474 +13475 +13476 +13477 +13478 +13479 +13480 +13481 +13482 +13483 +13484 +13485 +13486 +13487 +13488 +13489 +13490 +13491 +13492 +13493 +13494 +13495 +13496 +13497 +13498 +13499 +13500 +13501 +13502 +13503 +13504 +13505 +13506 +13507 +13508 +13509 +13510 +13511 +13512 +13513 +13514 +13515 +13516 +13517 +13518 +13519 +13520 +13521 +13522 +13523 +13524 +13525 +13526 +13527 +13528 +13529 +13530 +13531 +13532 +13533 +13534 +13535 +13536 +13537 +13538 +13539 +13540 +13541 +13542 +13543 +13544 +13545 +13546 +13547 +13548 +13549 +13550 +13551 +13552 +13553 +13554 +13555 +13556 +13557 +13558 +13559 +13560 +13561 +13562 +13563 +13564 +13565 +13566 +13567 +13568 +13569 +13570 +13571 +13572 +13573 +13574 +13575 +13576 +13577 +13578 +13579 +13580 +13581 +13582 +13583 +13584 +13585 +13586 +13587 +13588 +13589 +13590 +13591 +13592 +13593 +13594 +13595 +13596 +13597 +13598 +13599 +13600 +13601 +13602 +13603 +13604 +13605 +13606 +13607 +13608 +13609 +13610 +13611 +13612 +13613 +13614 +13615 +13616 +13617 +13618 +13619 +13620 +13621 +13622 +13623 +13624 +13625 +13626 +13627 +13628 +13629 +13630 +13631 +13632 +13633 +13634 +13635 +13636 +13637 +13638 +13639 +13640 +13641 +13642 +13643 +13644 +13645 +13646 +13647 +13648 +13649 +13650 +13651 +13652 +13653 +13654 +13655 +13656 +13657 +13658 +13659 +13660 +13661 +13662 +13663 +13664 +13665 +13666 +13667 +13668 +13669 +13670 +13671 +13672 +13673 +13674 +13675 +13676 +13677 +13678 +13679 +13680 +13681 +13682 +13683 +13684 +13685 +13686 +13687 +13688 +13689 +13690 +13691 +13692 +13693 +13694 +13695 +13696 +13697 +13698 +13699 +13700 +13701 +13702 +13703 +13704 +13705 +13706 +13707 +13708 +13709 +13710 +13711 +13712 +13713 +13714 +13715 +13716 +13717 +13718 +13719 +13720 +13721 +13722 +13723 +13724 +13725 +13726 +13727 +13728 +13729 +13730 +13731 +13732 +13733 +13734 +13735 +13736 +13737 +13738 +13739 +13740 +13741 +13742 +13743 +13744 +13745 +13746 +13747 +13748 +13749 +13750 +13751 +13752 +13753 +13754 +13755 +13756 +13757 +13758 +13759 +13760 +13761 +13762 +13763 +13764 +13765 +13766 +13767 +13768 +13769 +13770 +13771 +13772 +13773 +13774 +13775 +13776 +13777 +13778 +13779 +13780 +13781 +13782 +13783 +13784 +13785 +13786 +13787 +13788 +13789 +13790 +13791 +13792 +13793 +13794 +13795 +13796 +13797 +13798 +13799 +13800 +13801 +13802 +13803 +13804 +13805 +13806 +13807 +13808 +13809 +13810 +13811 +13812 +13813 +13814 +13815 +13816 +13817 +13818 +13819 +13820 +13821 +13822 +13823 +13824 +13825 +13826 +13827 +13828 +13829 +13830 +13831 +13832 +13833 +13834 +13835 +13836 +13837 +13838 +13839 +13840 +13841 +13842 +13843 +13844 +13845 +13846 +13847 +13848 +13849 +13850 +13851 +13852 +13853 +13854 +13855 +13856 +13857 +13858 +13859 +13860 +13861 +13862 +13863 +13864 +13865 +13866 +13867 +13868 +13869 +13870 +13871 +13872 +13873 +13874 +13875 +13876 +13877 +13878 +13879 +13880 +13881 +13882 +13883 +13884 +13885 +13886 +13887 +13888 +13889 +13890 +13891 +13892 +13893 +13894 +13895 +13896 +13897 +13898 +13899 +13900 +13901 +13902 +13903 +13904 +13905 +13906 +13907 +13908 +13909 +13910 +13911 +13912 +13913 +13914 +13915 +13916 +13917 +13918 +13919 +13920 +13921 +13922 +13923 +13924 +13925 +13926 +13927 +13928 +13929 +13930 +13931 +13932 +13933 +13934 +13935 +13936 +13937 +13938 +13939 +13940 +13941 +13942 +13943 +13944 +13945 +13946 +13947 +13948 +13949 +13950 +13951 +13952 +13953 +13954 +13955 +13956 +13957 +13958 +13959 +13960 +13961 +13962 +13963 +13964 +13965 +13966 +13967 +13968 +13969 +13970 +13971 +13972 +13973 +13974 +13975 +13976 +13977 +13978 +13979 +13980 +13981 +13982 +13983 +13984 +13985 +13986 +13987 +13988 +13989 +13990 +13991 +13992 +13993 +13994 +13995 +13996 +13997 +13998 +13999 +14000 +14001 +14002 +14003 +14004 +14005 +14006 +14007 +14008 +14009 +14010 +14011 +14012 +14013 +14014 +14015 +14016 +14017 +14018 +14019 +14020 +14021 +14022 +14023 +14024 +14025 +14026 +14027 +14028 +14029 +14030 +14031 +14032 +14033 +14034 +14035 +14036 +14037 +14038 +14039 +14040 +14041 +14042 +14043 +14044 +14045 +14046 +14047 +14048 +14049 +14050 +14051 +14052 +14053 +14054 +14055 +14056 +14057 +14058 +14059 +14060 +14061 +14062 +14063 +14064 +14065 +14066 +14067 +14068 +14069 +14070 +14071 +14072 +14073 +14074 +14075 +14076 +14077 +14078 +14079 +14080 +14081 +14082 +14083 +14084 +14085 +14086 +14087 +14088 +14089 +14090 +14091 +14092 +14093 +14094 +14095 +14096 +14097 +14098 +14099 +14100 +14101 +14102 +14103 +14104 +14105 +14106 +14107 +14108 +14109 +14110 +14111 +14112 +14113 +14114 +14115 +14116 +14117 +14118 +14119 +14120 +14121 +14122 +14123 +14124 +14125 +14126 +14127 +14128 +14129 +14130 +14131 +14132 +14133 +14134 +14135 +14136 +14137 +14138 +14139 +14140 +14141 +14142 +14143 +14144 +14145 +14146 +14147 +14148 +14149 +14150 +14151 +14152 +14153 +14154 +14155 +14156 +14157 +14158 +14159 +14160 +14161 +14162 +14163 +14164 +14165 +14166 +14167 +14168 +14169 +14170 +14171 +14172 +14173 +14174 +14175 +14176 +14177 +14178 +14179 +14180 +14181 +14182 +14183 +14184 +14185 +14186 +14187 +14188 +14189 +14190 +14191 +14192 +14193 +14194 +14195 +14196 +14197 +14198 +14199 +14200 +14201 +14202 +14203 +14204 +14205 +14206 +14207 +14208 +14209 +14210 +14211 +14212 +14213 +14214 +14215 +14216 +14217 +14218 +14219 +14220 +14221 +14222 +14223 +14224 +14225 +14226 +14227 +14228 +14229 +14230 +14231 +14232 +14233 +14234 +14235 +14236 +14237 +14238 +14239 +14240 +14241 +14242 +14243 +14244 +14245 +14246 +14247 +14248 +14249 +14250 +14251 +14252 +14253 +14254 +14255 +14256 +14257 +14258 +14259 +14260 +14261 +14262 +14263 +14264 +14265 +14266 +14267 +14268 +14269 +14270 +14271 +14272 +14273 +14274 +14275 +14276 +14277 +14278 +14279 +14280 +14281 +14282 +14283 +14284 +14285 +14286 +14287 +14288 +14289 +14290 +14291 +14292 +14293 +14294 +14295 +14296 +14297 +14298 +14299 +14300 +14301 +14302 +14303 +14304 +14305 +14306 +14307 +14308 +14309 +14310 +14311 +14312 +14313 +14314 +14315 +14316 +14317 +14318 +14319 +14320 +14321 +14322 +14323 +14324 +14325 +14326 +14327 +14328 +14329 +14330 +14331 +14332 +14333 +14334 +14335 +14336 +14337 +14338 +14339 +14340 +14341 +14342 +14343 +14344 +14345 +14346 +14347 +14348 +14349 +14350 +14351 +14352 +14353 +14354 +14355 +14356 +14357 +14358 +14359 +14360 +14361 +14362 +14363 +14364 +14365 +14366 +14367 +14368 +14369 +14370 +14371 +14372 +14373 +14374 +14375 +14376 +14377 +14378 +14379 +14380 +14381 +14382 +14383 +14384 +14385 +14386 +14387 +14388 +14389 +14390 +14391 +14392 +14393 +14394 +14395 +14396 +14397 +14398 +14399 +14400 +14401 +14402 +14403 +14404 +14405 +14406 +14407 +14408 +14409 +14410 +14411 +14412 +14413 +14414 +14415 +14416 +14417 +14418 +14419 +14420 +14421 +14422 +14423 +14424 +14425 +14426 +14427 +14428 +14429 +14430 +14431 +14432 +14433 +14434 +14435 +14436 +14437 +14438 +14439 +14440 +14441 +14442 +14443 +14444 +14445 +14446 +14447 +14448 +14449 +14450 +14451 +14452 +14453 +14454 +14455 +14456 +14457 +14458 +14459 +14460 +14461 +14462 +14463 +14464 +14465 +14466 +14467 +14468 +14469 +14470 +14471 +14472 +14473 +14474 +14475 +14476 +14477 +14478 +14479 +14480 +14481 +14482 +14483 +14484 +14485 +14486 +14487 +14488 +14489 +14490 +14491 +14492 +14493 +14494 +14495 +14496 +14497 +14498 +14499 +14500 +14501 +14502 +14503 +14504 +14505 +14506 +14507 +14508 +14509 +14510 +14511 +14512 +14513 +14514 +14515 +14516 +14517 +14518 +14519 +14520 +14521 +14522 +14523 +14524 +14525 +14526 +14527 +14528 +14529 +14530 +14531 +14532 +14533 +14534 +14535 +14536 +14537 +14538 +14539 +14540 +14541 +14542 +14543 +14544 +14545 +14546 +14547 +14548 +14549 +14550 +14551 +14552 +14553 +14554 +14555 +14556 +14557 +14558 +14559 +14560 +14561 +14562 +14563 +14564 +14565 +14566 +14567 +14568 +14569 +14570 +14571 +14572 +14573 +14574 +14575 +14576 +14577 +14578 +14579 +14580 +14581 +14582 +14583 +14584 +14585 +14586 +14587 +14588 +14589 +14590 +14591 +14592 +14593 +14594 +14595 +14596 +14597 +14598 +14599 +14600 +14601 +14602 +14603 +14604 +14605 +14606 +14607 +14608 +14609 +14610 +14611 +14612 +14613 +14614 +14615 +14616 +14617 +14618 +14619 +14620 +14621 +14622 +14623 +14624 +14625 +14626 +14627 +14628 +14629 +14630 +14631 +14632 +14633 +14634 +14635 +14636 +14637 +14638 +14639 +14640 +14641 +14642 +14643 +14644 +14645 +14646 +14647 +14648 +14649 +14650 +14651 +14652 +14653 +14654 +14655 +14656 +14657 +14658 +14659 +14660 +14661 +14662 +14663 +14664 +14665 +14666 +14667 +14668 +14669 +14670 +14671 +14672 +14673 +14674 +14675 +14676 +14677 +14678 +14679 +14680 +14681 +14682 +14683 +14684 +14685 +14686 +14687 +14688 +14689 +14690 +14691 +14692 +14693 +14694 +14695 +14696 +14697 +14698 +14699 +14700 +14701 +14702 +14703 +14704 +14705 +14706 +14707 +14708 +14709 +14710 +14711 +14712 +14713 +14714 +14715 +14716 +14717 +14718 +14719 +14720 +14721 +14722 +14723 +14724 +14725 +14726 +14727 +14728 +14729 +14730 +14731 +14732 +14733 +14734 +14735 +14736 +14737 +14738 +14739 +14740 +14741 +14742 +14743 +14744 +14745 +14746 +14747 +14748 +14749 +14750 +14751 +14752 +14753 +14754 +14755 +14756 +14757 +14758 +14759 +14760 +14761 +14762 +14763 +14764 +14765 +14766 +14767 +14768 +14769 +14770 +14771 +14772 +14773 +14774 +14775 +14776 +14777 +14778 +14779 +14780 +14781 +14782 +14783 +14784 +14785 +14786 +14787 +14788 +14789 +14790 +14791 +14792 +14793 +14794 +14795 +14796 +14797 +14798 +14799 +14800 +14801 +14802 +14803 +14804 +14805 +14806 +14807 +14808 +14809 +14810 +14811 +14812 +14813 +14814 +14815 +14816 +14817 +14818 +14819 +14820 +14821 +14822 +14823 +14824 +14825 +14826 +14827 +14828 +14829 +14830 +14831 +14832 +14833 +14834 +14835 +14836 +14837 +14838 +14839 +14840 +14841 +14842 +14843 +14844 +14845 +14846 +14847 +14848 +14849 +14850 +14851 +14852 +14853 +14854 +14855 +14856 +14857 +14858 +14859 +14860 +14861 +14862 +14863 +14864 +14865 +14866 +14867 +14868 +14869 +14870 +14871 +14872 +14873 +14874 +14875 +14876 +14877 +14878 +14879 +14880 +14881 +14882 +14883 +14884 +14885 +14886 +14887 +14888 +14889 +14890 +14891 +14892 +14893 +14894 +14895 +14896 +14897 +14898 +14899 +14900 +14901 +14902 +14903 +14904 +14905 +14906 +14907 +14908 +14909 +14910 +14911 +14912 +14913 +14914 +14915 +14916 +14917 +14918 +14919 +14920 +14921 +14922 +14923 +14924 +14925 +14926 +14927 +14928 +14929 +14930 +14931 +14932 +14933 +14934 +14935 +14936 +14937 +14938 +14939 +14940 +14941 +14942 +14943 +14944 +14945 +14946 +14947 +14948 +14949 +14950 +14951 +14952 +14953 +14954 +14955 +14956 +14957 +14958 +14959 +14960 +14961 +14962 +14963 +14964 +14965 +14966 +14967 +14968 +14969 +14970 +14971 +14972 +14973 +14974 +14975 +14976 +14977 +14978 +14979 +14980 +14981 +14982 +14983 +14984 +14985 +14986 +14987 +14988 +14989 +14990 +14991 +14992 +14993 +14994 +14995 +14996 +14997 +14998 +14999 +15000 +15001 +15002 +15003 +15004 +15005 +15006 +15007 +15008 +15009 +15010 +15011 +15012 +15013 +15014 +15015 +15016 +15017 +15018 +15019 +15020 +15021 +15022 +15023 +15024 +15025 +15026 +15027 +15028 +15029 +15030 +15031 +15032 +15033 +15034 +15035 +15036 +15037 +15038 +15039 +15040 +15041 +15042 +15043 +15044 +15045 +15046 +15047 +15048 +15049 +15050 +15051 +15052 +15053 +15054 +15055 +15056 +15057 +15058 +15059 +15060 +15061 +15062 +15063 +15064 +15065 +15066 +15067 +15068 +15069 +15070 +15071 +15072 +15073 +15074 +15075 +15076 +15077 +15078 +15079 +15080 +15081 +15082 +15083 +15084 +15085 +15086 +15087 +15088 +15089 +15090 +15091 +15092 +15093 +15094 +15095 +15096 +15097 +15098 +15099 +15100 +15101 +15102 +15103 +15104 +15105 +15106 +15107 +15108 +15109 +15110 +15111 +15112 +15113 +15114 +15115 +15116 +15117 +15118 +15119 +15120 +15121 +15122 +15123 +15124 +15125 +15126 +15127 +15128 +15129 +15130 +15131 +15132 +15133 +15134 +15135 +15136 +15137 +15138 +15139 +15140 +15141 +15142 +15143 +15144 +15145 +15146 +15147 +15148 +15149 +15150 +15151 +15152 +15153 +15154 +15155 +15156 +15157 +15158 +15159 +15160 +15161 +15162 +15163 +15164 +15165 +15166 +15167 +15168 +15169 +15170 +15171 +15172 +15173 +15174 +15175 +15176 +15177 +15178 +15179 +15180 +15181 +15182 +15183 +15184 +15185 +15186 +15187 +15188 +15189 +15190 +15191 +15192 +15193 +15194 +15195 +15196 +15197 +15198 +15199 +15200 +15201 +15202 +15203 +15204 +15205 +15206 +15207 +15208 +15209 +15210 +15211 +15212 +15213 +15214 +15215 +15216 +15217 +15218 +15219 +15220 +15221 +15222 +15223 +15224 +15225 +15226 +15227 +15228 +15229 +15230 +15231 +15232 +15233 +15234 +15235 +15236 +15237 +15238 +15239 +15240 +15241 +15242 +15243 +15244 +15245 +15246 +15247 +15248 +15249 +15250 +15251 +15252 +15253 +15254 +15255 +15256 +15257 +15258 +15259 +15260 +15261 +15262 +15263 +15264 +15265 +15266 +15267 +15268 +15269 +15270 +15271 +15272 +15273 +15274 +15275 +15276 +15277 +15278 +15279 +15280 +15281 +15282 +15283 +15284 +15285 +15286 +15287 +15288 +15289 +15290 +15291 +15292 +15293 +15294 +15295 +15296 +15297 +15298 +15299 +15300 +15301 +15302 +15303 +15304 +15305 +15306 +15307 +15308 +15309 +15310 +15311 +15312 +15313 +15314 +15315 +15316 +15317 +15318 +15319 +15320 +15321 +15322 +15323 +15324 +15325 +15326 +15327 +15328 +15329 +15330 +15331 +15332 +15333 +15334 +15335 +15336 +15337 +15338 +15339 +15340 +15341 +15342 +15343 +15344 +15345 +15346 +15347 +15348 +15349 +15350 +15351 +15352 +15353 +15354 +15355 +15356 +15357 +15358 +15359 +15360 +15361 +15362 +15363 +15364 +15365 +15366 +15367 +15368 +15369 +15370 +15371 +15372 +15373 +15374 +15375 +15376 +15377 +15378 +15379 +15380 +15381 +15382 +15383 +15384 +15385 +15386 +15387 +15388 +15389 +15390 +15391 +15392 +15393 +15394 +15395 +15396 +15397 +15398 +15399 +15400 +15401 +15402 +15403 +15404 +15405 +15406 +15407 +15408 +15409 +15410 +15411 +15412 +15413 +15414 +15415 +15416 +15417 +15418 +15419 +15420 +15421 +15422 +15423 +15424 +15425 +15426 +15427 +15428 +15429 +15430 +15431 +15432 +15433 +15434 +15435 +15436 +15437 +15438 +15439 +15440 +15441 +15442 +15443 +15444 +15445 +15446 +15447 +15448 +15449 +15450 +15451 +15452 +15453 +15454 +15455 +15456 +15457 +15458 +15459 +15460 +15461 +15462 +15463 +15464 +15465 +15466 +15467 +15468 +15469 +15470 +15471 +15472 +15473 +15474 +15475 +15476 +15477 +15478 +15479 +15480 +15481 +15482 +15483 +15484 +15485 +15486 +15487 +15488 +15489 +15490 +15491 +15492 +15493 +15494 +15495 +15496 +15497 +15498 +15499 +15500 +15501 +15502 +15503 +15504 +15505 +15506 +15507 +15508 +15509 +15510 +15511 +15512 +15513 +15514 +15515 +15516 +15517 +15518 +15519 +15520 +15521 +15522 +15523 +15524 +15525 +15526 +15527 +15528 +15529 +15530 +15531 +15532 +15533 +15534 +15535 +15536 +15537 +15538 +15539 +15540 +15541 +15542 +15543 +15544 +15545 +15546 +15547 +15548 +15549 +15550 +15551 +15552 +15553 +15554 +15555 +15556 +15557 +15558 +15559 +15560 +15561 +15562 +15563 +15564 +15565 +15566 +15567 +15568 +15569 +15570 +15571 +15572 +15573 +15574 +15575 +15576 +15577 +15578 +15579 +15580 +15581 +15582 +15583 +15584 +15585 +15586 +15587 +15588 +15589 +15590 +15591 +15592 +15593 +15594 +15595 +15596 +15597 +15598 +15599 +15600 +15601 +15602 +15603 +15604 +15605 +15606 +15607 +15608 +15609 +15610 +15611 +15612 +15613 +15614 +15615 +15616 +15617 +15618 +15619 +15620 +15621 +15622 +15623 +15624 +15625 +15626 +15627 +15628 +15629 +15630 +15631 +15632 +15633 +15634 +15635 +15636 +15637 +15638 +15639 +15640 +15641 +15642 +15643 +15644 +15645 +15646 +15647 +15648 +15649 +15650 +15651 +15652 +15653 +15654 +15655 +15656 +15657 +15658 +15659 +15660 +15661 +15662 +15663 +15664 +15665 +15666 +15667 +15668 +15669 +15670 +15671 +15672 +15673 +15674 +15675 +15676 +15677 +15678 +15679 +15680 +15681 +15682 +15683 +15684 +15685 +15686 +15687 +15688 +15689 +15690 +15691 +15692 +15693 +15694 +15695 +15696 +15697 +15698 +15699 +15700 +15701 +15702 +15703 +15704 +15705 +15706 +15707 +15708 +15709 +15710 +15711 +15712 +15713 +15714 +15715 +15716 +15717 +15718 +15719 +15720 +15721 +15722 +15723 +15724 +15725 +15726 +15727 +15728 +15729 +15730 +15731 +15732 +15733 +15734 +15735 +15736 +15737 +15738 +15739 +15740 +15741 +15742 +15743 +15744 +15745 +15746 +15747 +15748 +15749 +15750 +15751 +15752 +15753 +15754 +15755 +15756 +15757 +15758 +15759 +15760 +15761 +15762 +15763 +15764 +15765 +15766 +15767 +15768 +15769 +15770 +15771 +15772 +15773 +15774 +15775 +15776 +15777 +15778 +15779 +15780 +15781 +15782 +15783 +15784 +15785 +15786 +15787 +15788 +15789 +15790 +15791 +15792 +15793 +15794 +15795 +15796 +15797 +15798 +15799 +15800 +15801 +15802 +15803 +15804 +15805 +15806 +15807 +15808 +15809 +15810 +15811 +15812 +15813 +15814 +15815 +15816 +15817 +15818 +15819 +15820 +15821 +15822 +15823 +15824 +15825 +15826 +15827 +15828 +15829 +15830 +15831 +15832 +15833 +15834 +15835 +15836 +15837 +15838 +15839 +15840 +15841 +15842 +15843 +15844 +15845 +15846 +15847 +15848 +15849 +15850 +15851 +15852 +15853 +15854 +15855 +15856 +15857 +15858 +15859 +15860 +15861 +15862 +15863 +15864 +15865 +15866 +15867 +15868 +15869 +15870 +15871 +15872 +15873 +15874 +15875 +15876 +15877 +15878 +15879 +15880 +15881 +15882 +15883 +15884 +15885 +15886 +15887 +15888 +15889 +15890 +15891 +15892 +15893 +15894 +15895 +15896 +15897 +15898 +15899 +15900 +15901 +15902 +15903 +15904 +15905 +15906 +15907 +15908 +15909 +15910 +15911 +15912 +15913 +15914 +15915 +15916 +15917 +15918 +15919 +15920 +15921 +15922 +15923 +15924 +15925 +15926 +15927 +15928 +15929 +15930 +15931 +15932 +15933 +15934 +15935 +15936 +15937 +15938 +15939 +15940 +15941 +15942 +15943 +15944 +15945 +15946 +15947 +15948 +15949 +15950 +15951 +15952 +15953 +15954 +15955 +15956 +15957 +15958 +15959 +15960 +15961 +15962 +15963 +15964 +15965 +15966 +15967 +15968 +15969 +15970 +15971 +15972 +15973 +15974 +15975 +15976 +15977 +15978 +15979 +15980 +15981 +15982 +15983 +15984 +15985 +15986 +15987 +15988 +15989 +15990 +15991 +15992 +15993 +15994 +15995 +15996 +15997 +15998 +15999 +16000 +16001 +16002 +16003 +16004 +16005 +16006 +16007 +16008 +16009 +16010 +16011 +16012 +16013 +16014 +16015 +16016 +16017 +16018 +16019 +16020 +16021 +16022 +16023 +16024 +16025 +16026 +16027 +16028 +16029 +16030 +16031 +16032 +16033 +16034 +16035 +16036 +16037 +16038 +16039 +16040 +16041 +16042 +16043 +16044 +16045 +16046 +16047 +16048 +16049 +16050 +16051 +16052 +16053 +16054 +16055 +16056 +16057 +16058 +16059 +16060 +16061 +16062 +16063 +16064 +16065 +16066 +16067 +16068 +16069 +16070 +16071 +16072 +16073 +16074 +16075 +16076 +16077 +16078 +16079 +16080 +16081 +16082 +16083 +16084 +16085 +16086 +16087 +16088 +16089 +16090 +16091 +16092 +16093 +16094 +16095 +16096 +16097 +16098 +16099 +16100 +16101 +16102 +16103 +16104 +16105 +16106 +16107 +16108 +16109 +16110 +16111 +16112 +16113 +16114 +16115 +16116 +16117 +16118 +16119 +16120 +16121 +16122 +16123 +16124 +16125 +16126 +16127 +16128 +16129 +16130 +16131 +16132 +16133 +16134 +16135 +16136 +16137 +16138 +16139 +16140 +16141 +16142 +16143 +16144 +16145 +16146 +16147 +16148 +16149 +16150 +16151 +16152 +16153 +16154 +16155 +16156 +16157 +16158 +16159 +16160 +16161 +16162 +16163 +16164 +16165 +16166 +16167 +16168 +16169 +16170 +16171 +16172 +16173 +16174 +16175 +16176 +16177 +16178 +16179 +16180 +16181 +16182 +16183 +16184 +16185 +16186 +16187 +16188 +16189 +16190 +16191 +16192 +16193 +16194 +16195 +16196 +16197 +16198 +16199 +16200 +16201 +16202 +16203 +16204 +16205 +16206 +16207 +16208 +16209 +16210 +16211 +16212 +16213 +16214 +16215 +16216 +16217 +16218 +16219 +16220 +16221 +16222 +16223 +16224 +16225 +16226 +16227 +16228 +16229 +16230 +16231 +16232 +16233 +16234 +16235 +16236 +16237 +16238 +16239 +16240 +16241 +16242 +16243 +16244 +16245 +16246 +16247 +16248 +16249 +16250 +16251 +16252 +16253 +16254 +16255 +16256 +16257 +16258 +16259 +16260 +16261 +16262 +16263 +16264 +16265 +16266 +16267 +16268 +16269 +16270 +16271 +16272 +16273 +16274 +16275 +16276 +16277 +16278 +16279 +16280 +16281 +16282 +16283 +16284 +16285 +16286 +16287 +16288 +16289 +16290 +16291 +16292 +16293 +16294 +16295 +16296 +16297 +16298 +16299 +16300 +16301 +16302 +16303 +16304 +16305 +16306 +16307 +16308 +16309 +16310 +16311 +16312 +16313 +16314 +16315 +16316 +16317 +16318 +16319 +16320 +16321 +16322 +16323 +16324 +16325 +16326 +16327 +16328 +16329 +16330 +16331 +16332 +16333 +16334 +16335 +16336 +16337 +16338 +16339 +16340 +16341 +16342 +16343 +16344 +16345 +16346 +16347 +16348 +16349 +16350 +16351 +16352 +16353 +16354 +16355 +16356 +16357 +16358 +16359 +16360 +16361 +16362 +16363 +16364 +16365 +16366 +16367 +16368 +16369 +16370 +16371 +16372 +16373 +16374 +16375 +16376 +16377 +16378 +16379 +16380 +16381 +16382 +16383 +16384 +16385 +16386 +16387 +16388 +16389 +16390 +16391 +16392 +16393 +16394 +16395 +16396 +16397 +16398 +16399 +16400 +16401 +16402 +16403 +16404 +16405 +16406 +16407 +16408 +16409 +16410 +16411 +16412 +16413 +16414 +16415 +16416 +16417 +16418 +16419 +16420 +16421 +16422 +16423 +16424 +16425 +16426 +16427 +16428 +16429 +16430 +16431 +16432 +16433 +16434 +16435 +16436 +16437 +16438 +16439 +16440 +16441 +16442 +16443 +16444 +16445 +16446 +16447 +16448 +16449 +16450 +16451 +16452 +16453 +16454 +16455 +16456 +16457 +16458 +16459 +16460 +16461 +16462 +16463 +16464 +16465 +16466 +16467 +16468 +16469 +16470 +16471 +16472 +16473 +16474 +16475 +16476 +16477 +16478 +16479 +16480 +16481 +16482 +16483 +16484 +16485 +16486 +16487 +16488 +16489 +16490 +16491 +16492 +16493 +16494 +16495 +16496 +16497 +16498 +16499 +16500 +16501 +16502 +16503 +16504 +16505 +16506 +16507 +16508 +16509 +16510 +16511 +16512 +16513 +16514 +16515 +16516 +16517 +16518 +16519 +16520 +16521 +16522 +16523 +16524 +16525 +16526 +16527 +16528 +16529 +16530 +16531 +16532 +16533 +16534 +16535 +16536 +16537 +16538 +16539 +16540 +16541 +16542 +16543 +16544 +16545 +16546 +16547 +16548 +16549 +16550 +16551 +16552 +16553 +16554 +16555 +16556 +16557 +16558 +16559 +16560 +16561 +16562 +16563 +16564 +16565 +16566 +16567 +16568 +16569 +16570 +16571 +16572 +16573 +16574 +16575 +16576 +16577 +16578 +16579 +16580 +16581 +16582 +16583 +16584 +16585 +16586 +16587 +16588 +16589 +16590 +16591 +16592 +16593 +16594 +16595 +16596 +16597 +16598 +16599 +16600 +16601 +16602 +16603 +16604 +16605 +16606 +16607 +16608 +16609 +16610 +16611 +16612 +16613 +16614 +16615 +16616 +16617 +16618 +16619 +16620 +16621 +16622 +16623 +16624 +16625 +16626 +16627 +16628 +16629 +16630 +16631 +16632 +16633 +16634 +16635 +16636 +16637 +16638 +16639 +16640 +16641 +16642 +16643 +16644 +16645 +16646 +16647 +16648 +16649 +16650 +16651 +16652 +16653 +16654 +16655 +16656 +16657 +16658 +16659 +16660 +16661 +16662 +16663 +16664 +16665 +16666 +16667 +16668 +16669 +16670 +16671 +16672 +16673 +16674 +16675 +16676 +16677 +16678 +16679 +16680 +16681 +16682 +16683 +16684 +16685 +16686 +16687 +16688 +16689 +16690 +16691 +16692 +16693 +16694 +16695 +16696 +16697 +16698 +16699 +16700 +16701 +16702 +16703 +16704 +16705 +16706 +16707 +16708 +16709 +16710 +16711 +16712 +16713 +16714 +16715 +16716 +16717 +16718 +16719 +16720 +16721 +16722 +16723 +16724 +16725 +16726 +16727 +16728 +16729 +16730 +16731 +16732 +16733 +16734 +16735 +16736 +16737 +16738 +16739 +16740 +16741 +16742 +16743 +16744 +16745 +16746 +16747 +16748 +16749 +16750 +16751 +16752 +16753 +16754 +16755 +16756 +16757 +16758 +16759 +16760 +16761 +16762 +16763 +16764 +16765 +16766 +16767 +16768 +16769 +16770 +16771 +16772 +16773 +16774 +16775 +16776 +16777 +16778 +16779 +16780 +16781 +16782 +16783 +16784 +16785 +16786 +16787 +16788 +16789 +16790 +16791 +16792 +16793 +16794 +16795 +16796 +16797 +16798 +16799 +16800 +16801 +16802 +16803 +16804 +16805 +16806 +16807 +16808 +16809 +16810 +16811 +16812 +16813 +16814 +16815 +16816 +16817 +16818 +16819 +16820 +16821 +16822 +16823 +16824 +16825 +16826 +16827 +16828 +16829 +16830 +16831 +16832 +16833 +16834 +16835 +16836 +16837 +16838 +16839 +16840 +16841 +16842 +16843 +16844 +16845 +16846 +16847 +16848 +16849 +16850 +16851 +16852 +16853 +16854 +16855 +16856 +16857 +16858 +16859 +16860 +16861 +16862 +16863 +16864 +16865 +16866 +16867 +16868 +16869 +16870 +16871 +16872 +16873 +16874 +16875 +16876 +16877 +16878 +16879 +16880 +16881 +16882 +16883 +16884 +16885 +16886 +16887 +16888 +16889 +16890 +16891 +16892 +16893 +16894 +16895 +16896 +16897 +16898 +16899 +16900 +16901 +16902 +16903 +16904 +16905 +16906 +16907 +16908 +16909 +16910 +16911 +16912 +16913 +16914 +16915 +16916 +16917 +16918 +16919 +16920 +16921 +16922 +16923 +16924 +16925 +16926 +16927 +16928 +16929 +16930 +16931 +16932 +16933 +16934 +16935 +16936 +16937 +16938 +16939 +16940 +16941 +16942 +16943 +16944 +16945 +16946 +16947 +16948 +16949 +16950 +16951 +16952 +16953 +16954 +16955 +16956 +16957 +16958 +16959 +16960 +16961 +16962 +16963 +16964 +16965 +16966 +16967 +16968 +16969 +16970 +16971 +16972 +16973 +16974 +16975 +16976 +16977 +16978 +16979 +16980 +16981 +16982 +16983 +16984 +16985 +16986 +16987 +16988 +16989 +16990 +16991 +16992 +16993 +16994 +16995 +16996 +16997 +16998 +16999 +17000 +17001 +17002 +17003 +17004 +17005 +17006 +17007 +17008 +17009 +17010 +17011 +17012 +17013 +17014 +17015 +17016 +17017 +17018 +17019 +17020 +17021 +17022 +17023 +17024 +17025 +17026 +17027 +17028 +17029 +17030 +17031 +17032 +17033 +17034 +17035 +17036 +17037 +17038 +17039 +17040 +17041 +17042 +17043 +17044 +17045 +17046 +17047 +17048 +17049 +17050 +17051 +17052 +17053 +17054 +17055 +17056 +17057 +17058 +17059 +17060 +17061 +17062 +17063 +17064 +17065 +17066 +17067 +17068 +17069 +17070 +17071 +17072 +17073 +17074 +17075 +17076 +17077 +17078 +17079 +17080 +17081 +17082 +17083 +17084 +17085 +17086 +17087 +17088 +17089 +17090 +17091 +17092 +17093 +17094 +17095 +17096 +17097 +17098 +17099 +17100 +17101 +17102 +17103 +17104 +17105 +17106 +17107 +17108 +17109 +17110 +17111 +17112 +17113 +17114 +17115 +17116 +17117 +17118 +17119 +17120 +17121 +17122 +17123 +17124 +17125 +17126 +17127 +17128 +17129 +17130 +17131 +17132 +17133 +17134 +17135 +17136 +17137 +17138 +17139 +17140 +17141 +17142 +17143 +17144 +17145 +17146 +17147 +17148 +17149 +17150 +17151 +17152 +17153 +17154 +17155 +17156 +17157 +17158 +17159 +17160 +17161 +17162 +17163 +17164 +17165 +17166 +17167 +17168 +17169 +17170 +17171 +17172 +17173 +17174 +17175 +17176 +17177 +17178 +17179 +17180 +17181 +17182 +17183 +17184 +17185 +17186 +17187 +17188 +17189 +17190 +17191 +17192 +17193 +17194 +17195 +17196 +17197 +17198 +17199 +17200 +17201 +17202 +17203 +17204 +17205 +17206 +17207 +17208 +17209 +17210 +17211 +17212 +17213 +17214 +17215 +17216 +17217 +17218 +17219 +17220 +17221 +17222 +17223 +17224 +17225 +17226 +17227 +17228 +17229 +17230 +17231 +17232 +17233 +17234 +17235 +17236 +17237 +17238 +17239 +17240 +17241 +17242 +17243 +17244 +17245 +17246 +17247 +17248 +17249 +17250 +17251 +17252 +17253 +17254 +17255 +17256 +17257 +17258 +17259 +17260 +17261 +17262 +17263 +17264 +17265 +17266 +17267 +17268 +17269 +17270 +17271 +17272 +17273 +17274 +17275 +17276 +17277 +17278 +17279 +17280 +17281 +17282 +17283 +17284 +17285 +17286 +17287 +17288 +17289 +17290 +17291 +17292 +17293 +17294 +17295 +17296 +17297 +17298 +17299 +17300 +17301 +17302 +17303 +17304 +17305 +17306 +17307 +17308 +17309 +17310 +17311 +17312 +17313 +17314 +17315 +17316 +17317 +17318 +17319 +17320 +17321 +17322 +17323 +17324 +17325 +17326 +17327 +17328 +17329 +17330 +17331 +17332 +17333 +17334 +17335 +17336 +17337 +17338 +17339 +17340 +17341 +17342 +17343 +17344 +17345 +17346 +17347 +17348 +17349 +17350 +17351 +17352 +17353 +17354 +17355 +17356 +17357 +17358 +17359 +17360 +17361 +17362 +17363 +17364 +17365 +17366 +17367 +17368 +17369 +17370 +17371 +17372 +17373 +17374 +17375 +17376 +17377 +17378 +17379 +17380 +17381 +17382 +17383 +17384 +17385 +17386 +17387 +17388 +17389 +17390 +17391 +17392 +17393 +17394 +17395 +17396 +17397 +17398 +17399 +17400 +17401 +17402 +17403 +17404 +17405 +17406 +17407 +17408 +17409 +17410 +17411 +17412 +17413 +17414 +17415 +17416 +17417 +17418 +17419 +17420 +17421 +17422 +17423 +17424 +17425 +17426 +17427 +17428 +17429 +17430 +17431 +17432 +17433 +17434 +17435 +17436 +17437 +17438 +17439 +17440 +17441 +17442 +17443 +17444 +17445 +17446 +17447 +17448 +17449 +17450 +17451 +17452 +17453 +17454 +17455 +17456 +17457 +17458 +17459 +17460 +17461 +17462 +17463 +17464 +17465 +17466 +17467 +17468 +17469 +17470 +17471 +17472 +17473 +17474 +17475 +17476 +17477 +17478 +17479 +17480 +17481 +17482 +17483 +17484 +17485 +17486 +17487 +17488 +17489 +17490 +17491 +17492 +17493 +17494 +17495 +17496 +17497 +17498 +17499 +17500 +17501 +17502 +17503 +17504 +17505 +17506 +17507 +17508 +17509 +17510 +17511 +17512 +17513 +17514 +17515 +17516 +17517 +17518 +17519 +17520 +17521 +17522 +17523 +17524 +17525 +17526 +17527 +17528 +17529 +17530 +17531 +17532 +17533 +17534 +17535 +17536 +17537 +17538 +17539 +17540 +17541 +17542 +17543 +17544 +17545 +17546 +17547 +17548 +17549 +17550 +17551 +17552 +17553 +17554 +17555 +17556 +17557 +17558 +17559 +17560 +17561 +17562 +17563 +17564 +17565 +17566 +17567 +17568 +17569 +17570 +17571 +17572 +17573 +17574 +17575 +17576 +17577 +17578 +17579 +17580 +17581 +17582 +17583 +17584 +17585 +17586 +17587 +17588 +17589 +17590 +17591 +17592 +17593 +17594 +17595 +17596 +17597 +17598 +17599 +17600 +17601 +17602 +17603 +17604 +17605 +17606 +17607 +17608 +17609 +17610 +17611 +17612 +17613 +17614 +17615 +17616 +17617 +17618 +17619 +17620 +17621 +17622 +17623 +17624 +17625 +17626 +17627 +17628 +17629 +17630 +17631 +17632 +17633 +17634 +17635 +17636 +17637 +17638 +17639 +17640 +17641 +17642 +17643 +17644 +17645 +17646 +17647 +17648 +17649 +17650 +17651 +17652 +17653 +17654 +17655 +17656 +17657 +17658 +17659 +17660 +17661 +17662 +17663 +17664 +17665 +17666 +17667 +17668 +17669 +17670 +17671 +17672 +17673 +17674 +17675 +17676 +17677 +17678 +17679 +17680 +17681 +17682 +17683 +17684 +17685 +17686 +17687 +17688 +17689 +17690 +17691 +17692 +17693 +17694 +17695 +17696 +17697 +17698 +17699 +17700 +17701 +17702 +17703 +17704 +17705 +17706 +17707 +17708 +17709 +17710 +17711 +17712 +17713 +17714 +17715 +17716 +17717 +17718 +17719 +17720 +17721 +17722 +17723 +17724 +17725 +17726 +17727 +17728 +17729 +17730 +17731 +17732 +17733 +17734 +17735 +17736 +17737 +17738 +17739 +17740 +17741 +17742 +17743 +17744 +17745 +17746 +17747 +17748 +17749 +17750 +17751 +17752 +17753 +17754 +17755 +17756 +17757 +17758 +17759 +17760 +17761 +17762 +17763 +17764 +17765 +17766 +17767 +17768 +17769 +17770 +17771 +17772 +17773 +17774 +17775 +17776 +17777 +17778 +17779 +17780 +17781 +17782 +17783 +17784 +17785 +17786 +17787 +17788 +17789 +17790 +17791 +17792 +17793 +17794 +17795 +17796 +17797 +17798 +17799 +17800 +17801 +17802 +17803 +17804 +17805 +17806 +17807 +17808 +17809 +17810 +17811 +17812 +17813 +17814 +17815 +17816 +17817 +17818 +17819 +17820 +17821 +17822 +17823 +17824 +17825 +17826 +17827 +17828 +17829 +17830 +17831 +17832 +17833 +17834 +17835 +17836 +17837 +17838 +17839 +17840 +17841 +17842 +17843 +17844 +17845 +17846 +17847 +17848 +17849 +17850 +17851 +17852 +17853 +17854 +17855 +17856 +17857 +17858 +17859 +17860 +17861 +17862 +17863 +17864 +17865 +17866 +17867 +17868 +17869 +17870 +17871 +17872 +17873 +17874 +17875 +17876 +17877 +17878 +17879 +17880 +17881 +17882 +17883 +17884 +17885 +17886 +17887 +17888 +17889 +17890 +17891 +17892 +17893 +17894 +17895 +17896 +17897 +17898 +17899 +17900 +17901 +17902 +17903 +17904 +17905 +17906 +17907 +17908 +17909 +17910 +17911 +17912 +17913 +17914 +17915 +17916 +17917 +17918 +17919 +17920 +17921 +17922 +17923 +17924 +17925 +17926 +17927 +17928 +17929 +17930 +17931 +17932 +17933 +17934 +17935 +17936 +17937 +17938 +17939 +17940 +17941 +17942 +17943 +17944 +17945 +17946 +17947 +17948 +17949 +17950 +17951 +17952 +17953 +17954 +17955 +17956 +17957 +17958 +17959 +17960 +17961 +17962 +17963 +17964 +17965 +17966 +17967 +17968 +17969 +17970 +17971 +17972 +17973 +17974 +17975 +17976 +17977 +17978 +17979 +17980 +17981 +17982 +17983 +17984 +17985 +17986 +17987 +17988 +17989 +17990 +17991 +17992 +17993 +17994 +17995 +17996 +17997 +17998 +17999 +18000 +18001 +18002 +18003 +18004 +18005 +18006 +18007 +18008 +18009 +18010 +18011 +18012 +18013 +18014 +18015 +18016 +18017 +18018 +18019 +18020 +18021 +18022 +18023 +18024 +18025 +18026 +18027 +18028 +18029 +18030 +18031 +18032 +18033 +18034 +18035 +18036 +18037 +18038 +18039 +18040 +18041 +18042 +18043 +18044 +18045 +18046 +18047 +18048 +18049 +18050 +18051 +18052 +18053 +18054 +18055 +18056 +18057 +18058 +18059 +18060 +18061 +18062 +18063 +18064 +18065 +18066 +18067 +18068 +18069 +18070 +18071 +18072 +18073 +18074 +18075 +18076 +18077 +18078 +18079 +18080 +18081 +18082 +18083 +18084 +18085 +18086 +18087 +18088 +18089 +18090 +18091 +18092 +18093 +18094 +18095 +18096 +18097 +18098 +18099 +18100 +18101 +18102 +18103 +18104 +18105 +18106 +18107 +18108 +18109 +18110 +18111 +18112 +18113 +18114 +18115 +18116 +18117 +18118 +18119 +18120 +18121 +18122 +18123 +18124 +18125 +18126 +18127 +18128 +18129 +18130 +18131 +18132 +18133 +18134 +18135 +18136 +18137 +18138 +18139 +18140 +18141 +18142 +18143 +18144 +18145 +18146 +18147 +18148 +18149 +18150 +18151 +18152 +18153 +18154 +18155 +18156 +18157 +18158 +18159 +18160 +18161 +18162 +18163 +18164 +18165 +18166 +18167 +18168 +18169 +18170 +18171 +18172 +18173 +18174 +18175 +18176 +18177 +18178 +18179 +18180 +18181 +18182 +18183 +18184 +18185 +18186 +18187 +18188 +18189 +18190 +18191 +18192 +18193 +18194 +18195 +18196 +18197 +18198 +18199 +18200 +18201 +18202 +18203 +18204 +18205 +18206 +18207 +18208 +18209 +18210 +18211 +18212 +18213 +18214 +18215 +18216 +18217 +18218 +18219 +18220 +18221 +18222 +18223 +18224 +18225 +18226 +18227 +18228 +18229 +18230 +18231 +18232 +18233 +18234 +18235 +18236 +18237 +18238 +18239 +18240 +18241 +18242 +18243 +18244 +18245 +18246 +18247 +18248 +18249 +18250 +18251 +18252 +18253 +18254 +18255 +18256 +18257 +18258 +18259 +18260 +18261 +18262 +18263 +18264 +18265 +18266 +18267 +18268 +18269 +18270 +18271 +18272 +18273 +18274 +18275 +18276 +18277 +18278 +18279 +18280 +18281 +18282 +18283 +18284 +18285 +18286 +18287 +18288 +18289 +18290 +18291 +18292 +18293 +18294 +18295 +18296 +18297 +18298 +18299 +18300 +18301 +18302 +18303 +18304 +18305 +18306 +18307 +18308 +18309 +18310 +18311 +18312 +18313 +18314 +18315 +18316 +18317 +18318 +18319 +18320 +18321 +18322 +18323 +18324 +18325 +18326 +18327 +18328 +18329 +18330 +18331 +18332 +18333 +18334 +18335 +18336 +18337 +18338 +18339 +18340 +18341 +18342 +18343 +18344 +18345 +18346 +18347 +18348 +18349 +18350 +18351 +18352 +18353 +18354 +18355 +18356 +18357 +18358 +18359 +18360 +18361 +18362 +18363 +18364 +18365 +18366 +18367 +18368 +18369 +18370 +18371 +18372 +18373 +18374 +18375 +18376 +18377 +18378 +18379 +18380 +18381 +18382 +18383 +18384 +18385 +18386 +18387 +18388 +18389 +18390 +18391 +18392 +18393 +18394 +18395 +18396 +18397 +18398 +18399 +18400 +18401 +18402 +18403 +18404 +18405 +18406 +18407 +18408 +18409 +18410 +18411 +18412 +18413 +18414 +18415 +18416 +18417 +18418 +18419 +18420 +18421 +18422 +18423 +18424 +18425 +18426 +18427 +18428 +18429 +18430 +18431 +18432 +18433 +18434 +18435 +18436 +18437 +18438 +18439 +18440 +18441 +18442 +18443 +18444 +18445 +18446 +18447 +18448 +18449 +18450 +18451 +18452 +18453 +18454 +18455 +18456 +18457 +18458 +18459 +18460 +18461 +18462 +18463 +18464 +18465 +18466 +18467 +18468 +18469 +18470 +18471 +18472 +18473 +18474 +18475 +18476 +18477 +18478 +18479 +18480 +18481 +18482 +18483 +18484 +18485 +18486 +18487 +18488 +18489 +18490 +18491 +18492 +18493 +18494 +18495 +18496 +18497 +18498 +18499 +18500 +18501 +18502 +18503 +18504 +18505 +18506 +18507 +18508 +18509 +18510 +18511 +18512 +18513 +18514 +18515 +18516 +18517 +18518 +18519 +18520 +18521 +18522 +18523 +18524 +18525 +18526 +18527 +18528 +18529 +18530 +18531 +18532 +18533 +18534 +18535 +18536 +18537 +18538 +18539 +18540 +18541 +18542 +18543 +18544 +18545 +18546 +18547 +18548 +18549 +18550 +18551 +18552 +18553 +18554 +18555 +18556 +18557 +18558 +18559 +18560 +18561 +18562 +18563 +18564 +18565 +18566 +18567 +18568 +18569 +18570 +18571 +18572 +18573 +18574 +18575 +18576 +18577 +18578 +18579 +18580 +18581 +18582 +18583 +18584 +18585 +18586 +18587 +18588 +18589 +18590 +18591 +18592 +18593 +18594 +18595 +18596 +18597 +18598 +18599 +18600 +18601 +18602 +18603 +18604 +18605 +18606 +18607 +18608 +18609 +18610 +18611 +18612 +18613 +18614 +18615 +18616 +18617 +18618 +18619 +18620 +18621 +18622 +18623 +18624 +18625 +18626 +18627 +18628 +18629 +18630 +18631 +18632 +18633 +18634 +18635 +18636 +18637 +18638 +18639 +18640 +18641 +18642 +18643 +18644 +18645 +18646 +18647 +18648 +18649 +18650 +18651 +18652 +18653 +18654 +18655 +18656 +18657 +18658 +18659 +18660 +18661 +18662 +18663 +18664 +18665 +18666 +18667 +18668 +18669 +18670 +18671 +18672 +18673 +18674 +18675 +18676 +18677 +18678 +18679 +18680 +18681 +18682 +18683 +18684 +18685 +18686 +18687 +18688 +18689 +18690 +18691 +18692 +18693 +18694 +18695 +18696 +18697 +18698 +18699 +18700 +18701 +18702 +18703 +18704 +18705 +18706 +18707 +18708 +18709 +18710 +18711 +18712 +18713 +18714 +18715 +18716 +18717 +18718 +18719 +18720 +18721 +18722 +18723 +18724 +18725 +18726 +18727 +18728 +18729 +18730 +18731 +18732 +18733 +18734 +18735 +18736 +18737 +18738 +18739 +18740 +18741 +18742 +18743 +18744 +18745 +18746 +18747 +18748 +18749 +18750 +18751 +18752 +18753 +18754 +18755 +18756 +18757 +18758 +18759 +18760 +18761 +18762 +18763 +18764 +18765 +18766 +18767 +18768 +18769 +18770 +18771 +18772 +18773 +18774 +18775 +18776 +18777 +18778 +18779 +18780 +18781 +18782 +18783 +18784 +18785 +18786 +18787 +18788 +18789 +18790 +18791 +18792 +18793 +18794 +18795 +18796 +18797 +18798 +18799 +18800 +18801 +18802 +18803 +18804 +18805 +18806 +18807 +18808 +18809 +18810 +18811 +18812 +18813 +18814 +18815 +18816 +18817 +18818 +18819 +18820 +18821 +18822 +18823 +18824 +18825 +18826 +18827 +18828 +18829 +18830 +18831 +18832 +18833 +18834 +18835 +18836 +18837 +18838 +18839 +18840 +18841 +18842 +18843 +18844 +18845 +18846 +18847 +18848 +18849 +18850 +18851 +18852 +18853 +18854 +18855 +18856 +18857 +18858 +18859 +18860 +18861 +18862 +18863 +18864 +18865 +18866 +18867 +18868 +18869 +18870 +18871 +18872 +18873 +18874 +18875 +18876 +18877 +18878 +18879 +18880 +18881 +18882 +18883 +18884 +18885 +18886 +18887 +18888 +18889 +18890 +18891 +18892 +18893 +18894 +18895 +18896 +18897 +18898 +18899 +18900 +18901 +18902 +18903 +18904 +18905 +18906 +18907 +18908 +18909 +18910 +18911 +18912 +18913 +18914 +18915 +18916 +18917 +18918 +18919 +18920 +18921 +18922 +18923 +18924 +18925 +18926 +18927 +18928 +18929 +18930 +18931 +18932 +18933 +18934 +18935 +18936 +18937 +18938 +18939 +18940 +18941 +18942 +18943 +18944 +18945 +18946 +18947 +18948 +18949 +18950 +18951 +18952 +18953 +18954 +18955 +18956 +18957 +18958 +18959 +18960 +18961 +18962 +18963 +18964 +18965 +18966 +18967 +18968 +18969 +18970 +18971 +18972 +18973 +18974 +18975 +18976 +18977 +18978 +18979 +18980 +18981 +18982 +18983 +18984 +18985 +18986 +18987 +18988 +18989 +18990 +18991 +18992 +18993 +18994 +18995 +18996 +18997 +18998 +18999 +19000 +19001 +19002 +19003 +19004 +19005 +19006 +19007 +19008 +19009 +19010 +19011 +19012 +19013 +19014 +19015 +19016 +19017 +19018 +19019 +19020 +19021 +19022 +19023 +19024 +19025 +19026 +19027 +19028 +19029 +19030 +19031 +19032 +19033 +19034 +19035 +19036 +19037 +19038 +19039 +19040 +19041 +19042 +19043 +19044 +19045 +19046 +19047 +19048 +19049 +19050 +19051 +19052 +19053 +19054 +19055 +19056 +19057 +19058 +19059 +19060 +19061 +19062 +19063 +19064 +19065 +19066 +19067 +19068 +19069 +19070 +19071 +19072 +19073 +19074 +19075 +19076 +19077 +19078 +19079 +19080 +19081 +19082 +19083 +19084 +19085 +19086 +19087 +19088 +19089 +19090 +19091 +19092 +19093 +19094 +19095 +19096 +19097 +19098 +19099 +19100 +19101 +19102 +19103 +19104 +19105 +19106 +19107 +19108 +19109 +19110 +19111 +19112 +19113 +19114 +19115 +19116 +19117 +19118 +19119 +19120 +19121 +19122 +19123 +19124 +19125 +19126 +19127 +19128 +19129 +19130 +19131 +19132 +19133 +19134 +19135 +19136 +19137 +19138 +19139 +19140 +19141 +19142 +19143 +19144 +19145 +19146 +19147 +19148 +19149 +19150 +19151 +19152 +19153 +19154 +19155 +19156 +19157 +19158 +19159 +19160 +19161 +19162 +19163 +19164 +19165 +19166 +19167 +19168 +19169 +19170 +19171 +19172 +19173 +19174 +19175 +19176 +19177 +19178 +19179 +19180 +19181 +19182 +19183 +19184 +19185 +19186 +19187 +19188 +19189 +19190 +19191 +19192 +19193 +19194 +19195 +19196 +19197 +19198 +19199 +19200 +19201 +19202 +19203 +19204 +19205 +19206 +19207 +19208 +19209 +19210 +19211 +19212 +19213 +19214 +19215 +19216 +19217 +19218 +19219 +19220 +19221 +19222 +19223 +19224 +19225 +19226 +19227 +19228 +19229 +19230 +19231 +19232 +19233 +19234 +19235 +19236 +19237 +19238 +19239 +19240 +19241 +19242 +19243 +19244 +19245 +19246 +19247 +19248 +19249 +19250 +19251 +19252 +19253 +19254 +19255 +19256 +19257 +19258 +19259 +19260 +19261 +19262 +19263 +19264 +19265 +19266 +19267 +19268 +19269 +19270 +19271 +19272 +19273 +19274 +19275 +19276 +19277 +19278 +19279 +19280 +19281 +19282 +19283 +19284 +19285 +19286 +19287 +19288 +19289 +19290 +19291 +19292 +19293 +19294 +19295 +19296 +19297 +19298 +19299 +19300 +19301 +19302 +19303 +19304 +19305 +19306 +19307 +19308 +19309 +19310 +19311 +19312 +19313 +19314 +19315 +19316 +19317 +19318 +19319 +19320 +19321 +19322 +19323 +19324 +19325 +19326 +19327 +19328 +19329 +19330 +19331 +19332 +19333 +19334 +19335 +19336 +19337 +19338 +19339 +19340 +19341 +19342 +19343 +19344 +19345 +19346 +19347 +19348 +19349 +19350 +19351 +19352 +19353 +19354 +19355 +19356 +19357 +19358 +19359 +19360 +19361 +19362 +19363 +19364 +19365 +19366 +19367 +19368 +19369 +19370 +19371 +19372 +19373 +19374 +19375 +19376 +19377 +19378 +19379 +19380 +19381 +19382 +19383 +19384 +19385 +19386 +19387 +19388 +19389 +19390 +19391 +19392 +19393 +19394 +19395 +19396 +19397 +19398 +19399 +19400 +19401 +19402 +19403 +19404 +19405 +19406 +19407 +19408 +19409 +19410 +19411 +19412 +19413 +19414 +19415 +19416 +19417 +19418 +19419 +19420 +19421 +19422 +19423 +19424 +19425 +19426 +19427 +19428 +19429 +19430 +19431 +19432 +19433 +19434 +19435 +19436 +19437 +19438 +19439 +19440 +19441 +19442 +19443 +19444 +19445 +19446 +19447 +19448 +19449 +19450 +19451 +19452 +19453 +19454 +19455 +19456 +19457 +19458 +19459 +19460 +19461 +19462 +19463 +19464 +19465 +19466 +19467 +19468 +19469 +19470 +19471 +19472 +19473 +19474 +19475 +19476 +19477 +19478 +19479 +19480 +19481 +19482 +19483 +19484 +19485 +19486 +19487 +19488 +19489 +19490 +19491 +19492 +19493 +19494 +19495 +19496 +19497 +19498 +19499 +19500 +19501 +19502 +19503 +19504 +19505 +19506 +19507 +19508 +19509 +19510 +19511 +19512 +19513 +19514 +19515 +19516 +19517 +19518 +19519 +19520 +19521 +19522 +19523 +19524 +19525 +19526 +19527 +19528 +19529 +19530 +19531 +19532 +19533 +19534 +19535 +19536 +19537 +19538 +19539 +19540 +19541 +19542 +19543 +19544 +19545 +19546 +19547 +19548 +19549 +19550 +19551 +19552 +19553 +19554 +19555 +19556 +19557 +19558 +19559 +19560 +19561 +19562 +19563 +19564 +19565 +19566 +19567 +19568 +19569 +19570 +19571 +19572 +19573 +19574 +19575 +19576 +19577 +19578 +19579 +19580 +19581 +19582 +19583 +19584 +19585 +19586 +19587 +19588 +19589 +19590 +19591 +19592 +19593 +19594 +19595 +19596 +19597 +19598 +19599 +19600 +19601 +19602 +19603 +19604 +19605 +19606 +19607 +19608 +19609 +19610 +19611 +19612 +19613 +19614 +19615 +19616 +19617 +19618 +19619 +19620 +19621 +19622 +19623 +19624 +19625 +19626 +19627 +19628 +19629 +19630 +19631 +19632 +19633 +19634 +19635 +19636 +19637 +19638 +19639 +19640 +19641 +19642 +19643 +19644 +19645 +19646 +19647 +19648 +19649 +19650 +19651 +19652 +19653 +19654 +19655 +19656 +19657 +19658 +19659 +19660 +19661 +19662 +19663 +19664 +19665 +19666 +19667 +19668 +19669 +19670 +19671 +19672 +19673 +19674 +19675 +19676 +19677 +19678 +19679 +19680 +19681 +19682 +19683 +19684 +19685 +19686 +19687 +19688 +19689 +19690 +19691 +19692 +19693 +19694 +19695 +19696 +19697 +19698 +19699 +19700 +19701 +19702 +19703 +19704 +19705 +19706 +19707 +19708 +19709 +19710 +19711 +19712 +19713 +19714 +19715 +19716 +19717 +19718 +19719 +19720 +19721 +19722 +19723 +19724 +19725 +19726 +19727 +19728 +19729 +19730 +19731 +19732 +19733 +19734 +19735 +19736 +19737 +19738 +19739 +19740 +19741 +19742 +19743 +19744 +19745 +19746 +19747 +19748 +19749 +19750 +19751 +19752 +19753 +19754 +19755 +19756 +19757 +19758 +19759 +19760 +19761 +19762 +19763 +19764 +19765 +19766 +19767 +19768 +19769 +19770 +19771 +19772 +19773 +19774 +19775 +19776 +19777 +19778 +19779 +19780 +19781 +19782 +19783 +19784 +19785 +19786 +19787 +19788 +19789 +19790 +19791 +19792 +19793 +19794 +19795 +19796 +19797 +19798 +19799 +19800 +19801 +19802 +19803 +19804 +19805 +19806 +19807 +19808 +19809 +19810 +19811 +19812 +19813 +19814 +19815 +19816 +19817 +19818 +19819 +19820 +19821 +19822 +19823 +19824 +19825 +19826 +19827 +19828 +19829 +19830 +19831 +19832 +19833 +19834 +19835 +19836 +19837 +19838 +19839 +19840 +19841 +19842 +19843 +19844 +19845 +19846 +19847 +19848 +19849 +19850 +19851 +19852 +19853 +19854 +19855 +19856 +19857 +19858 +19859 +19860 +19861 +19862 +19863 +19864 +19865 +19866 +19867 +19868 +19869 +19870 +19871 +19872 +19873 +19874 +19875 +19876 +19877 +19878 +19879 +19880 +19881 +19882 +19883 +19884 +19885 +19886 +19887 +19888 +19889 +19890 +19891 +19892 +19893 +19894 +19895 +19896 +19897 +19898 +19899 +19900 +19901 +19902 +19903 +19904 +19905 +19906 +19907 +19908 +19909 +19910 +19911 +19912 +19913 +19914 +19915 +19916 +19917 +19918 +19919 +19920 +19921 +19922 +19923 +19924 +19925 +19926 +19927 +19928 +19929 +19930 +19931 +19932 +19933 +19934 +19935 +19936 +19937 +19938 +19939 +19940 +19941 +19942 +19943 +19944 +19945 +19946 +19947 +19948 +19949 +19950 +19951 +19952 +19953 +19954 +19955 +19956 +19957 +19958 +19959 +19960 +19961 +19962 +19963 +19964 +19965 +19966 +19967 +19968 +19969 +19970 +19971 +19972 +19973 +19974 +19975 +19976 +19977 +19978 +19979 +19980 +19981 +19982 +19983 +19984 +19985 +19986 +19987 +19988 +19989 +19990 +19991 +19992 +19993 +19994 +19995 +19996 +19997 +19998 +19999 +20000 +20001 +20002 +20003 +20004 +20005 +20006 +20007 +20008 +20009 +20010 +20011 +20012 +20013 +20014 +20015 +20016 +20017 +20018 +20019 +20020 +20021 +20022 +20023 +20024 +20025 +20026 +20027 +20028 +20029 +20030 +20031 +20032 +20033 +20034 +20035 +20036 +20037 +20038 +20039 +20040 +20041 +20042 +20043 +20044 +20045 +20046 +20047 +20048 +20049 +20050 +20051 +20052 +20053 +20054 +20055 +20056 +20057 +20058 +20059 +20060 +20061 +20062 +20063 +20064 +20065 +20066 +20067 +20068 +20069 +20070 +20071 +20072 +20073 +20074 +20075 +20076 +20077 +20078 +20079 +20080 +20081 +20082 +20083 +20084 +20085 +20086 +20087 +20088 +20089 +20090 +20091 +20092 +20093 +20094 +20095 +20096 +20097 +20098 +20099 +20100 +20101 +20102 +20103 +20104 +20105 +20106 +20107 +20108 +20109 +20110 +20111 +20112 +20113 +20114 +20115 +20116 +20117 +20118 +20119 +20120 +20121 +20122 +20123 +20124 +20125 +20126 +20127 +20128 +20129 +20130 +20131 +20132 +20133 +20134 +20135 +20136 +20137 +20138 +20139 +20140 +20141 +20142 +20143 +20144 +20145 +20146 +20147 +20148 +20149 +20150 +20151 +20152 +20153 +20154 +20155 +20156 +20157 +20158 +20159 +20160 +20161 +20162 +20163 +20164 +20165 +20166 +20167 +20168 +20169 +20170 +20171 +20172 +20173 +20174 +20175 +20176 +20177 +20178 +20179 +20180 +20181 +20182 +20183 +20184 +20185 +20186 +20187 +20188 +20189 +20190 +20191 +20192 +20193 +20194 +20195 +20196 +20197 +20198 +20199 +20200 +20201 +20202 +20203 +20204 +20205 +20206 +20207 +20208 +20209 +20210 +20211 +20212 +20213 +20214 +20215 +20216 +20217 +20218 +20219 +20220 +20221 +20222 +20223 +20224 +20225 +20226 +20227 +20228 +20229 +20230 +20231 +20232 +20233 +20234 +20235 +20236 +20237 +20238 +20239 +20240 +20241 +20242 +20243 +20244 +20245 +20246 +20247 +20248 +20249 +20250 +20251 +20252 +20253 +20254 +20255 +20256 +20257 +20258 +20259 +20260 +20261 +20262 +20263 +20264 +20265 +20266 +20267 +20268 +20269 +20270 +20271 +20272 +20273 +20274 +20275 +20276 +20277 +20278 +20279 +20280 +20281 +20282 +20283 +20284 +20285 +20286 +20287 +20288 +20289 +20290 +20291 +20292 +20293 +20294 +20295 +20296 +20297 +20298 +20299 +20300 +20301 +20302 +20303 +20304 +20305 +20306 +20307 +20308 +20309 +20310 +20311 +20312 +20313 +20314 +20315 +20316 +20317 +20318 +20319 +20320 +20321 +20322 +20323 +20324 +20325 +20326 +20327 +20328 +20329 +20330 +20331 +20332 +20333 +20334 +20335 +20336 +20337 +20338 +20339 +20340 +20341 +20342 +20343 +20344 +20345 +20346 +20347 +20348 +20349 +20350 +20351 +20352 +20353 +20354 +20355 +20356 +20357 +20358 +20359 +20360 +20361 +20362 +20363 +20364 +20365 +20366 +20367 +20368 +20369 +20370 +20371 +20372 +20373 +20374 +20375 +20376 +20377 +20378 +20379 +20380 +20381 +20382 +20383 +20384 +20385 +20386 +20387 +20388 +20389 +20390 +20391 +20392 +20393 +20394 +20395 +20396 +20397 +20398 +20399 +20400 +20401 +20402 +20403 +20404 +20405 +20406 +20407 +20408 +20409 +20410 +20411 +20412 +20413 +20414 +20415 +20416 +20417 +20418 +20419 +20420 +20421 +20422 +20423 +20424 +20425 +20426 +20427 +20428 +20429 +20430 +20431 +20432 +20433 +20434 +20435 +20436 +20437 +20438 +20439 +20440 +20441 +20442 +20443 +20444 +20445 +20446 +20447 +20448 +20449 +20450 +20451 +20452 +20453 +20454 +20455 +20456 +20457 +20458 +20459 +20460 +20461 +20462 +20463 +20464 +20465 +20466 +20467 +20468 +20469 +20470 +20471 +20472 +20473 +20474 +20475 +20476 +20477 +20478 +20479 +20480 +20481 +20482 +20483 +20484 +20485 +20486 +20487 +20488 +20489 +20490 +20491 +20492 +20493 +20494 +20495 +20496 +20497 +20498 +20499 +20500 +20501 +20502 +20503 +20504 +20505 +20506 +20507 +20508 +20509 +20510 +20511 +20512 +20513 +20514 +20515 +20516 +20517 +20518 +20519 +20520 +20521 +20522 +20523 +20524 +20525 +20526 +20527 +20528 +20529 +20530 +20531 +20532 +20533 +20534 +20535 +20536 +20537 +20538 +20539 +20540 +20541 +20542 +20543 +20544 +20545 +20546 +20547 +20548 +20549 +20550 +20551 +20552 +20553 +20554 +20555 +20556 +20557 +20558 +20559 +20560 +20561 +20562 +20563 +20564 +20565 +20566 +20567 +20568 +20569 +20570 +20571 +20572 +20573 +20574 +20575 +20576 +20577 +20578 +20579 +20580 +20581 +20582 +20583 +20584 +20585 +20586 +20587 +20588 +20589 +20590 +20591 +20592 +20593 +20594 +20595 +20596 +20597 +20598 +20599 +20600 +20601 +20602 +20603 +20604 +20605 +20606 +20607 +20608 +20609 +20610 +20611 +20612 +20613 +20614 +20615 +20616 +20617 +20618 +20619 +20620 +20621 +20622 +20623 +20624 +20625 +20626 +20627 +20628 +20629 +20630 +20631 +20632 +20633 +20634 +20635 +20636 +20637 +20638 +20639 +20640 +20641 +20642 +20643 +20644 +20645 +20646 +20647 +20648 +20649 +20650 +20651 +20652 +20653 +20654 +20655 +20656 +20657 +20658 +20659 +20660 +20661 +20662 +20663 +20664 +20665 +20666 +20667 +20668 +20669 +20670 +20671 +20672 +20673 +20674 +20675 +20676 +20677 +20678 +20679 +20680 +20681 +20682 +20683 +20684 +20685 +20686 +20687 +20688 +20689 +20690 +20691 +20692 +20693 +20694 +20695 +20696 +20697 +20698 +20699 +20700 +20701 +20702 +20703 +20704 +20705 +20706 +20707 +20708 +20709 +20710 +20711 +20712 +20713 +20714 +20715 +20716 +20717 +20718 +20719 +20720 +20721 +20722 +20723 +20724 +20725 +20726 +20727 +20728 +20729 +20730 +20731 +20732 +20733 +20734 +20735 +20736 +20737 +20738 +20739 +20740 +20741 +20742 +20743 +20744 +20745 +20746 +20747 +20748 +20749 +20750 +20751 +20752 +20753 +20754 +20755 +20756 +20757 +20758 +20759 +20760 +20761 +20762 +20763 +20764 +20765 +20766 +20767 +20768 +20769 +20770 +20771 +20772 +20773 +20774 +20775 +20776 +20777 +20778 +20779 +20780 +20781 +20782 +20783 +20784 +20785 +20786 +20787 +20788 +20789 +20790 +20791 +20792 +20793 +20794 +20795 +20796 +20797 +20798 +20799 +20800 +20801 +20802 +20803 +20804 +20805 +20806 +20807 +20808 +20809 +20810 +20811 +20812 +20813 +20814 +20815 +20816 +20817 +20818 +20819 +20820 +20821 +20822 +20823 +20824 +20825 +20826 +20827 +20828 +20829 +20830 +20831 +20832 +20833 +20834 +20835 +20836 +20837 +20838 +20839 +20840 +20841 +20842 +20843 +20844 +20845 +20846 +20847 +20848 +20849 +20850 +20851 +20852 +20853 +20854 +20855 +20856 +20857 +20858 +20859 +20860 +20861 +20862 +20863 +20864 +20865 +20866 +20867 +20868 +20869 +20870 +20871 +20872 +20873 +20874 +20875 +20876 +20877 +20878 +20879 +20880 +20881 +20882 +20883 +20884 +20885 +20886 +20887 +20888 +20889 +20890 +20891 +20892 +20893 +20894 +20895 +20896 +20897 +20898 +20899 +20900 +20901 +20902 +20903 +20904 +20905 +20906 +20907 +20908 +20909 +20910 +20911 +20912 +20913 +20914 +20915 +20916 +20917 +20918 +20919 +20920 +20921 +20922 +20923 +20924 +20925 +20926 +20927 +20928 +20929 +20930 +20931 +20932 +20933 +20934 +20935 +20936 +20937 +20938 +20939 +20940 +20941 +20942 +20943 +20944 +20945 +20946 +20947 +20948 +20949 +20950 +20951 +20952 +20953 +20954 +20955 +20956 +20957 +20958 +20959 +20960 +20961 +20962 +20963 +20964 +20965 +20966 +20967 +20968 +20969 +20970 +20971 +20972 +20973 +20974 +20975 +20976 +20977 +20978 +20979 +20980 +20981 +20982 +20983 +20984 +20985 +20986 +20987 +20988 +20989 +20990 +20991 +20992 +20993 +20994 +20995 +20996 +20997 +20998 +20999 +21000 +21001 +21002 +21003 +21004 +21005 +21006 +21007 +21008 +21009 +21010 +21011 +21012 +21013 +21014 +21015 +21016 +21017 +21018 +21019 +21020 +21021 +21022 +21023 +21024 +21025 +21026 +21027 +21028 +21029 +21030 +21031 +21032 +21033 +21034 +21035 +21036 +21037 +21038 +21039 +21040 +21041 +21042 +21043 +21044 +21045 +21046 +21047 +21048 +21049 +21050 +21051 +21052 +21053 +21054 +21055 +21056 +21057 +21058 +21059 +21060 +21061 +21062 +21063 +21064 +21065 +21066 +21067 +21068 +21069 +21070 +21071 +21072 +21073 +21074 +21075 +21076 +21077 +21078 +21079 +21080 +21081 +21082 +21083 +21084 +21085 +21086 +21087 +21088 +21089 +21090 +21091 +21092 +21093 +21094 +21095 +21096 +21097 +21098 +21099 +21100 +21101 +21102 +21103 +21104 +21105 +21106 +21107 +21108 +21109 +21110 +21111 +21112 +21113 +21114 +21115 +21116 +21117 +21118 +21119 +21120 +21121 +21122 +21123 +21124 +21125 +21126 +21127 +21128 +21129 +21130 +21131 +21132 +21133 +21134 +21135 +21136 +21137 +21138 +21139 +21140 +21141 +21142 +21143 +21144 +21145 +21146 +21147 +21148 +21149 +21150 +21151 +21152 +21153 +21154 +21155 +21156 +21157 +21158 +21159 +21160 +21161 +21162 +21163 +21164 +21165 +21166 +21167 +21168 +21169 +21170 +21171 +21172 +21173 +21174 +21175 +21176 +21177 +21178 +21179 +21180 +21181 +21182 +21183 +21184 +21185 +21186 +21187 +21188 +21189 +21190 +21191 +21192 +21193 +21194 +21195 +21196 +21197 +21198 +21199 +21200 +21201 +21202 +21203 +21204 +21205 +21206 +21207 +21208 +21209 +21210 +21211 +21212 +21213 +21214 +21215 +21216 +21217 +21218 +21219 +21220 +21221 +21222 +21223 +21224 +21225 +21226 +21227 +21228 +21229 +21230 +21231 +21232 +21233 +21234 +21235 +21236 +21237 +21238 +21239 +21240 +21241 +21242 +21243 +21244 +21245 +21246 +21247 +21248 +21249 +21250 +21251 +21252 +21253 +21254 +21255 +21256 +21257 +21258 +21259 +21260 +21261 +21262 +21263 +21264 +21265 +21266 +21267 +21268 +21269 +21270 +21271 +21272 +21273 +21274 +21275 +21276 +21277 +21278 +21279 +21280 +21281 +21282 +21283 +21284 +21285 +21286 +21287 +21288 +21289 +21290 +21291 +21292 +21293 +21294 +21295 +21296 +21297 +21298 +21299 +21300 +21301 +21302 +21303 +21304 +21305 +21306 +21307 +21308 +21309 +21310 +21311 +21312 +21313 +21314 +21315 +21316 +21317 +21318 +21319 +21320 +21321 +21322 +21323 +21324 +21325 +21326 +21327 +21328 +21329 +21330 +21331 +21332 +21333 +21334 +21335 +21336 +21337 +21338 +21339 +21340 +21341 +21342 +21343 +21344 +21345 +21346 +21347 +21348 +21349 +21350 +21351 +21352 +21353 +21354 +21355 +21356 +21357 +21358 +21359 +21360 +21361 +21362 +21363 +21364 +21365 +21366 +21367 +21368 +21369 +21370 +21371 +21372 +21373 +21374 +21375 +21376 +21377 +21378 +21379 +21380 +21381 +21382 +21383 +21384 +21385 +21386 +21387 +21388 +21389 +21390 +21391 +21392 +21393 +21394 +21395 +21396 +21397 +21398 +21399 +21400 +21401 +21402 +21403 +21404 +21405 +21406 +21407 +21408 +21409 +21410 +21411 +21412 +21413 +21414 +21415 +21416 +21417 +21418 +21419 +21420 +21421 +21422 +21423 +21424 +21425 +21426 +21427 +21428 +21429 +21430 +21431 +21432 +21433 +21434 +21435 +21436 +21437 +21438 +21439 +21440 +21441 +21442 +21443 +21444 +21445 +21446 +21447 +21448 +21449 +21450 +21451 +21452 +21453 +21454 +21455 +21456 +21457 +21458 +21459 +21460 +21461 +21462 +21463 +21464 +21465 +21466 +21467 +21468 +21469 +21470 +21471 +21472 +21473 +21474 +21475 +21476 +21477 +21478 +21479 +21480 +21481 +21482 +21483 +21484 +21485 +21486 +21487 +21488 +21489 +21490 +21491 +21492 +21493 +21494 +21495 +21496 +21497 +21498 +21499 +21500 +21501 +21502 +21503 +21504 +21505 +21506 +21507 +21508 +21509 +21510 +21511 +21512 +21513 +21514 +21515 +21516 +21517 +21518 +21519 +21520 +21521 +21522 +21523 +21524 +21525 +21526 +21527 +21528 +21529 +21530 +21531 +21532 +21533 +21534 +21535 +21536 +21537 +21538 +21539 +21540 +21541 +21542 +21543 +21544 +21545 +21546 +21547 +21548 +21549 +21550 +21551 +21552 +21553 +21554 +21555 +21556 +21557 +21558 +21559 +21560 +21561 +21562 +21563 +21564 +21565 +21566 +21567 +21568 +21569 +21570 +21571 +21572 +21573 +21574 +21575 +21576 +21577 +21578 +21579 +21580 +21581 +21582 +21583 +21584 +21585 +21586 +21587 +21588 +21589 +21590 +21591 +21592 +21593 +21594 +21595 +21596 +21597 +21598 +21599 +21600 +21601 +21602 +21603 +21604 +21605 +21606 +21607 +21608 +21609 +21610 +21611 +21612 +21613 +21614 +21615 +21616 +21617 +21618 +21619 +21620 +21621 +21622 +21623 +21624 +21625 +21626 +21627 +21628 +21629 +21630 +21631 +21632 +21633 +21634 +21635 +21636 +21637 +21638 +21639 +21640 +21641 +21642 +21643 +21644 +21645 +21646 +21647 +21648 +21649 +21650 +21651 +21652 +21653 +21654 +21655 +21656 +21657 +21658 +21659 +21660 +21661 +21662 +21663 +21664 +21665 +21666 +21667 +21668 +21669 +21670 +21671 +21672 +21673 +21674 +21675 +21676 +21677 +21678 +21679 +21680 +21681 +21682 +21683 +21684 +21685 +21686 +21687 +21688 +21689 +21690 +21691 +21692 +21693 +21694 +21695 +21696 +21697 +21698 +21699 +21700 +21701 +21702 +21703 +21704 +21705 +21706 +21707 +21708 +21709 +21710 +21711 +21712 +21713 +21714 +21715 +21716 +21717 +21718 +21719 +21720 +21721 +21722 +21723 +21724 +21725 +21726 +21727 +21728 +21729 +21730 +21731 +21732 +21733 +21734 +21735 +21736 +21737 +21738 +21739 +21740 +21741 +21742 +21743 +21744 +21745 +21746 +21747 +21748 +21749 +21750 +21751 +21752 +21753 +21754 +21755 +21756 +21757 +21758 +21759 +21760 +21761 +21762 +21763 +21764 +21765 +21766 +21767 +21768 +21769 +21770 +21771 +21772 +21773 +21774 +21775 +21776 +21777 +21778 +21779 +21780 +21781 +21782 +21783 +21784 +21785 +21786 +21787 +21788 +21789 +21790 +21791 +21792 +21793 +21794 +21795 +21796 +21797 +21798 +21799 +21800 +21801 +21802 +21803 +21804 +21805 +21806 +21807 +21808 +21809 +21810 +21811 +21812 +21813 +21814 +21815 +21816 +21817 +21818 +21819 +21820 +21821 +21822 +21823 +21824 +21825 +21826 +21827 +21828 +21829 +21830 +21831 +21832 +21833 +21834 +21835 +21836 +21837 +21838 +21839 +21840 +21841 +21842 +21843 +21844 +21845 +21846 +21847 +21848 +21849 +21850 +21851 +21852 +21853 +21854 +21855 +21856 +21857 +21858 +21859 +21860 +21861 +21862 +21863 +21864 +21865 +21866 +21867 +21868 +21869 +21870 +21871 +21872 +21873 +21874 +21875 +21876 +21877 +21878 +21879 +21880 +21881 +21882 +21883 +21884 +21885 +21886 +21887 +21888 +21889 +21890 +21891 +21892 +21893 +21894 +21895 +21896 +21897 +21898 +21899 +21900 +21901 +21902 +21903 +21904 +21905 +21906 +21907 +21908 +21909 +21910 +21911 +21912 +21913 +21914 +21915 +21916 +21917 +21918 +21919 +21920 +21921 +21922 +21923 +21924 +21925 +21926 +21927 +21928 +21929 +21930 +21931 +21932 +21933 +21934 +21935 +21936 +21937 +21938 +21939 +21940 +21941 +21942 +21943 +21944 +21945 +21946 +21947 +21948 +21949 +21950 +21951 +21952 +21953 +21954 +21955 +21956 +21957 +21958 +21959 +21960 +21961 +21962 +21963 +21964 +21965 +21966 +21967 +21968 +21969 +21970 +21971 +21972 +21973 +21974 +21975 +21976 +21977 +21978 +21979 +21980 +21981 +21982 +21983 +21984 +21985 +21986 +21987 +21988 +21989 +21990 +21991 +21992 +21993 +21994 +21995 +21996 +21997 +21998 +21999 +22000 +22001 +22002 +22003 +22004 +22005 +22006 +22007 +22008 +22009 +22010 +22011 +22012 +22013 +22014 +22015 +22016 +22017 +22018 +22019 +22020 +22021 +22022 +22023 +22024 +22025 +22026 +22027 +22028 +22029 +22030 +22031 +22032 +22033 +22034 +22035 +22036 +22037 +22038 +22039 +22040 +22041 +22042 +22043 +22044 +22045 +22046 +22047 +22048 +22049 +22050 +22051 +22052 +22053 +22054 +22055 +22056 +22057 +22058 +22059 +22060 +22061 +22062 +22063 +22064 +22065 +22066 +22067 +22068 +22069 +22070 +22071 +22072 +22073 +22074 +22075 +22076 +22077 +22078 +22079 +22080 +22081 +22082 +22083 +22084 +22085 +22086 +22087 +22088 +22089 +22090 +22091 +22092 +22093 +22094 +22095 +22096 +22097 +22098 +22099 +22100 +22101 +22102 +22103 +22104 +22105 +22106 +22107 +22108 +22109 +22110 +22111 +22112 +22113 +22114 +22115 +22116 +22117 +22118 +22119 +22120 +22121 +22122 +22123 +22124 +22125 +22126 +22127 +22128 +22129 +22130 +22131 +22132 +22133 +22134 +22135 +22136 +22137 +22138 +22139 +22140 +22141 +22142 +22143 +22144 +22145 +22146 +22147 +22148 +22149 +22150 +22151 +22152 +22153 +22154 +22155 +22156 +22157 +22158 +22159 +22160 +22161 +22162 +22163 +22164 +22165 +22166 +22167 +22168 +22169 +22170 +22171 +22172 +22173 +22174 +22175 +22176 +22177 +22178 +22179 +22180 +22181 +22182 +22183 +22184 +22185 +22186 +22187 +22188 +22189 +22190 +22191 +22192 +22193 +22194 +22195 +22196 +22197 +22198 +22199 +22200 +22201 +22202 +22203 +22204 +22205 +22206 +22207 +22208 +22209 +22210 +22211 +22212 +22213 +22214 +22215 +22216 +22217 +22218 +22219 +22220 +22221 +22222 +22223 +22224 +22225 +22226 +22227 +22228 +22229 +22230 +22231 +22232 +22233 +22234 +22235 +22236 +22237 +22238 +22239 +22240 +22241 +22242 +22243 +22244 +22245 +22246 +22247 +22248 +22249 +22250 +22251 +22252 +22253 +22254 +22255 +22256 +22257 +22258 +22259 +22260 +22261 +22262 +22263 +22264 +22265 +22266 +22267 +22268 +22269 +22270 +22271 +22272 +22273 +22274 +22275 +22276 +22277 +22278 +22279 +22280 +22281 +22282 +22283 +22284 +22285 +22286 +22287 +22288 +22289 +22290 +22291 +22292 +22293 +22294 +22295 +22296 +22297 +22298 +22299 +22300 +22301 +22302 +22303 +22304 +22305 +22306 +22307 +22308 +22309 +22310 +22311 +22312 +22313 +22314 +22315 +22316 +22317 +22318 +22319 +22320 +22321 +22322 +22323 +22324 +22325 +22326 +22327 +22328 +22329 +22330 +22331 +22332 +22333 +22334 +22335 +22336 +22337 +22338 +22339 +22340 +22341 +22342 +22343 +22344 +22345 +22346 +22347 +22348 +22349 +22350 +22351 +22352 +22353 +22354 +22355 +22356 +22357 +22358 +22359 +22360 +22361 +22362 +22363 +22364 +22365 +22366 +22367 +22368 +22369 +22370 +22371 +22372 +22373 +22374 +22375 +22376 +22377 +22378 +22379 +22380 +22381 +22382 +22383 +22384 +22385 +22386 +22387 +22388 +22389 +22390 +22391 +22392 +22393 +22394 +22395 +22396 +22397 +22398 +22399 +22400 +22401 +22402 +22403 +22404 +22405 +22406 +22407 +22408 +22409 +22410 +22411 +22412 +22413 +22414 +22415 +22416 +22417 +22418 +22419 +22420 +22421 +22422 +22423 +22424 +22425 +22426 +22427 +22428 +22429 +22430 +22431 +22432 +22433 +22434 +22435 +22436 +22437 +22438 +22439 +22440 +22441 +22442 +22443 +22444 +22445 +22446 +22447 +22448 +22449 +22450 +22451 +22452 +22453 +22454 +22455 +22456 +22457 +22458 +22459 +22460 +22461 +22462 +22463 +22464 +22465 +22466 +22467 +22468 +22469 +22470 +22471 +22472 +22473 +22474 +22475 +22476 +22477 +22478 +22479 +22480 +22481 +22482 +22483 +22484 +22485 +22486 +22487 +22488 +22489 +22490 +22491 +22492 +22493 +22494 +22495 +22496 +22497 +22498 +22499 +22500 +22501 +22502 +22503 +22504 +22505 +22506 +22507 +22508 +22509 +22510 +22511 +22512 +22513 +22514 +22515 +22516 +22517 +22518 +22519 +22520 +22521 +22522 +22523 +22524 +22525 +22526 +22527 +22528 +22529 +22530 +22531 +22532 +22533 +22534 +22535 +22536 +22537 +22538 +22539 +22540 +22541 +22542 +22543 +22544 +22545 +22546 +22547 +22548 +22549 +22550 +22551 +22552 +22553 +22554 +22555 +22556 +22557 +22558 +22559 +22560 +22561 +22562 +22563 +22564 +22565 +22566 +22567 +22568 +22569 +22570 +22571 +22572 +22573 +22574 +22575 +22576 +22577 +22578 +22579 +22580 +22581 +22582 +22583 +22584 +22585 +22586 +22587 +22588 +22589 +22590 +22591 +22592 +22593 +22594 +22595 +22596 +22597 +22598 +22599 +22600 +22601 +22602 +22603 +22604 +22605 +22606 +22607 +22608 +22609 +22610 +22611 +22612 +22613 +22614 +22615 +22616 +22617 +22618 +22619 +22620 +22621 +22622 +22623 +22624 +22625 +22626 +22627 +22628 +22629 +22630 +22631 +22632 +22633 +22634 +22635 +22636 +22637 +22638 +22639 +22640 +22641 +22642 +22643 +22644 +22645 +22646 +22647 +22648 +22649 +22650 +22651 +22652 +22653 +22654 +22655 +22656 +22657 +22658 +22659 +22660 +22661 +22662 +22663 +22664 +22665 +22666 +22667 +22668 +22669 +22670 +22671 +22672 +22673 +22674 +22675 +22676 +22677 +22678 +22679 +22680 +22681 +22682 +22683 +22684 +22685 +22686 +22687 +22688 +22689 +22690 +22691 +22692 +22693 +22694 +22695 +22696 +22697 +22698 +22699 +22700 +22701 +22702 +22703 +22704 +22705 +22706 +22707 +22708 +22709 +22710 +22711 +22712 +22713 +22714 +22715 +22716 +22717 +22718 +22719 +22720 +22721 +22722 +22723 +22724 +22725 +22726 +22727 +22728 +22729 +22730 +22731 +22732 +22733 +22734 +22735 +22736 +22737 +22738 +22739 +22740 +22741 +22742 +22743 +22744 +22745 +22746 +22747 +22748 +22749 +22750 +22751 +22752 +22753 +22754 +22755 +22756 +22757 +22758 +22759 +22760 +22761 +22762 +22763 +22764 +22765 +22766 +22767 +22768 +22769 +22770 +22771 +22772 +22773 +22774 +22775 +22776 +22777 +22778 +22779 +22780 +22781 +22782 +22783 +22784 +22785 +22786 +22787 +22788 +22789 +22790 +22791 +22792 +22793 +22794 +22795 +22796 +22797 +22798 +22799 +22800 +22801 +22802 +22803 +22804 +22805 +22806 +22807 +22808 +22809 +22810 +22811 +22812 +22813 +22814 +22815 +22816 +22817 +22818 +22819 +22820 +22821 +22822 +22823 +22824 +22825 +22826 +22827 +22828 +22829 +22830 +22831 +22832 +22833 +22834 +22835 +22836 +22837 +22838 +22839 +22840 +22841 +22842 +22843 +22844 +22845 +22846 +22847 +22848 +22849 +22850 +22851 +22852 +22853 +22854 +22855 +22856 +22857 +22858 +22859 +22860 +22861 +22862 +22863 +22864 +22865 +22866 +22867 +22868 +22869 +22870 +22871 +22872 +22873 +22874 +22875 +22876 +22877 +22878 +22879 +22880 +22881 +22882 +22883 +22884 +22885 +22886 +22887 +22888 +22889 +22890 +22891 +22892 +22893 +22894 +22895 +22896 +22897 +22898 +22899 +22900 +22901 +22902 +22903 +22904 +22905 +22906 +22907 +22908 +22909 +22910 +22911 +22912 +22913 +22914 +22915 +22916 +22917 +22918 +22919 +22920 +22921 +22922 +22923 +22924 +22925 +22926 +22927 +22928 +22929 +22930 +22931 +22932 +22933 +22934 +22935 +22936 +22937 +22938 +22939 +22940 +22941 +22942 +22943 +22944 +22945 +22946 +22947 +22948 +22949 +22950 +22951 +22952 +22953 +22954 +22955 +22956 +22957 +22958 +22959 +22960 +22961 +22962 +22963 +22964 +22965 +22966 +22967 +22968 +22969 +22970 +22971 +22972 +22973 +22974 +22975 +22976 +22977 +22978 +22979 +22980 +22981 +22982 +22983 +22984 +22985 +22986 +22987 +22988 +22989 +22990 +22991 +22992 +22993 +22994 +22995 +22996 +22997 +22998 +22999 +23000 +23001 +23002 +23003 +23004 +23005 +23006 +23007 +23008 +23009 +23010 +23011 +23012 +23013 +23014 +23015 +23016 +23017 +23018 +23019 +23020 +23021 +23022 +23023 +23024 +23025 +23026 +23027 +23028 +23029 +23030 +23031 +23032 +23033 +23034 +23035 +23036 +23037 +23038 +23039 +23040 +23041 +23042 +23043 +23044 +23045 +23046 +23047 +23048 +23049 +23050 +23051 +23052 +23053 +23054 +23055 +23056 +23057 +23058 +23059 +23060 +23061 +23062 +23063 +23064 +23065 +23066 +23067 +23068 +23069 +23070 +23071 +23072 +23073 +23074 +23075 +23076 +23077 +23078 +23079 +23080 +23081 +23082 +23083 +23084 +23085 +23086 +23087 +23088 +23089 +23090 +23091 +23092 +23093 +23094 +23095 +23096 +23097 +23098 +23099 +23100 +23101 +23102 +23103 +23104 +23105 +23106 +23107 +23108 +23109 +23110 +23111 +23112 +23113 +23114 +23115 +23116 +23117 +23118 +23119 +23120 +23121 +23122 +23123 +23124 +23125 +23126 +23127 +23128 +23129 +23130 +23131 +23132 +23133 +23134 +23135 +23136 +23137 +23138 +23139 +23140 +23141 +23142 +23143 +23144 +23145 +23146 +23147 +23148 +23149 +23150 +23151 +23152 +23153 +23154 +23155 +23156 +23157 +23158 +23159 +23160 +23161 +23162 +23163 +23164 +23165 +23166 +23167 +23168 +23169 +23170 +23171 +23172 +23173 +23174 +23175 +23176 +23177 +23178 +23179 +23180 +23181 +23182 +23183 +23184 +23185 +23186 +23187 +23188 +23189 +23190 +23191 +23192 +23193 +23194 +23195 +23196 +23197 +23198 +23199 +23200 +23201 +23202 +23203 +23204 +23205 +23206 +23207 +23208 +23209 +23210 +23211 +23212 +23213 +23214 +23215 +23216 +23217 +23218 +23219 +23220 +23221 +23222 +23223 +23224 +23225 +23226 +23227 +23228 +23229 +23230 +23231 +23232 +23233 +23234 +23235 +23236 +23237 +23238 +23239 +23240 +23241 +23242 +23243 +23244 +23245 +23246 +23247 +23248 +23249 +23250 +23251 +23252 +23253 +23254 +23255 +23256 +23257 +23258 +23259 +23260 +23261 +23262 +23263 +23264 +23265 +23266 +23267 +23268 +23269 +23270 +23271 +23272 +23273 +23274 +23275 +23276 +23277 +23278 +23279 +23280 +23281 +23282 +23283 +23284 +23285 +23286 +23287 +23288 +23289 +23290 +23291 +23292 +23293 +23294 +23295 +23296 +23297 +23298 +23299 +23300 +23301 +23302 +23303 +23304 +23305 +23306 +23307 +23308 +23309 +23310 +23311 +23312 +23313 +23314 +23315 +23316 +23317 +23318 +23319 +23320 +23321 +23322 +23323 +23324 +23325 +23326 +23327 +23328 +23329 +23330 +23331 +23332 +23333 +23334 +23335 +23336 +23337 +23338 +23339 +23340 +23341 +23342 +23343 +23344 +23345 +23346 +23347 +23348 +23349 +23350 +23351 +23352 +23353 +23354 +23355 +23356 +23357 +23358 +23359 +23360 +23361 +23362 +23363 +23364 +23365 +23366 +23367 +23368 +23369 +23370 +23371 +23372 +23373 +23374 +23375 +23376 +23377 +23378 +23379 +23380 +23381 +23382 +23383 +23384 +23385 +23386 +23387 +23388 +23389 +23390 +23391 +23392 +23393 +23394 +23395 +23396 +23397 +23398 +23399 +23400 +23401 +23402 +23403 +23404 +23405 +23406 +23407 +23408 +23409 +23410 +23411 +23412 +23413 +23414 +23415 +23416 +23417 +23418 +23419 +23420 +23421 +23422 +23423 +23424 +23425 +23426 +23427 +23428 +23429 +23430 +23431 +23432 +23433 +23434 +23435 +23436 +23437 +23438 +23439 +23440 +23441 +23442 +23443 +23444 +23445 +23446 +23447 +23448 +23449 +23450 +23451 +23452 +23453 +23454 +23455 +23456 +23457 +23458 +23459 +23460 +23461 +23462 +23463 +23464 +23465 +23466 +23467 +23468 +23469 +23470 +23471 +23472 +23473 +23474 +23475 +23476 +23477 +23478 +23479 +23480 +23481 +23482 +23483 +23484 +23485 +23486 +23487 +23488 +23489 +23490 +23491 +23492 +23493 +23494 +23495 +23496 +23497 +23498 +23499 +23500 +23501 +23502 +23503 +23504 +23505 +23506 +23507 +23508 +23509 +23510 +23511 +23512 +23513 +23514 +23515 +23516 +23517 +23518 +23519 +23520 +23521 +23522 +23523 +23524 +23525 +23526 +23527 +23528 +23529 +23530 +23531 +23532 +23533 +23534 +23535 +23536 +23537 +23538 +23539 +23540 +23541 +23542 +23543 +23544 +23545 +23546 +23547 +23548 +23549 +23550 +23551 +23552 +23553 +23554 +23555 +23556 +23557 +23558 +23559 +23560 +23561 +23562 +23563 +23564 +23565 +23566 +23567 +23568 +23569 +23570 +23571 +23572 +23573 +23574 +23575 +23576 +23577 +23578 +23579 +23580 +23581 +23582 +23583 +23584 +23585 +23586 +23587 +23588 +23589 +23590 +23591 +23592 +23593 +23594 +23595 +23596 +23597 +23598 +23599 +23600 +23601 +23602 +23603 +23604 +23605 +23606 +23607 +23608 +23609 +23610 +23611 +23612 +23613 +23614 +23615 +23616 +23617 +23618 +23619 +23620 +23621 +23622 +23623 +23624 +23625 +23626 +23627 +23628 +23629 +23630 +23631 +23632 +23633 +23634 +23635 +23636 +23637 +23638 +23639 +23640 +23641 +23642 +23643 +23644 +23645 +23646 +23647 +23648 +23649 +23650 +23651 +23652 +23653 +23654 +23655 +23656 +23657 +23658 +23659 +23660 +23661 +23662 +23663 +23664 +23665 +23666 +23667 +23668 +23669 +23670 +23671 +23672 +23673 +23674 +23675 +23676 +23677 +23678 +23679 +23680 +23681 +23682 +23683 +23684 +23685 +23686 +23687 +23688 +23689 +23690 +23691 +23692 +23693 +23694 +23695 +23696 +23697 +23698 +23699 +23700 +23701 +23702 +23703 +23704 +23705 +23706 +23707 +23708 +23709 +23710 +23711 +23712 +23713 +23714 +23715 +23716 +23717 +23718 +23719 +23720 +23721 +23722 +23723 +23724 +23725 +23726 +23727 +23728 +23729 +23730 +23731 +23732 +23733 +23734 +23735 +23736 +23737 +23738 +23739 +23740 +23741 +23742 +23743 +23744 +23745 +23746 +23747 +23748 +23749 +23750 +23751 +23752 +23753 +23754 +23755 +23756 +23757 +23758 +23759 +23760 +23761 +23762 +23763 +23764 +23765 +23766 +23767 +23768 +23769 +23770 +23771 +23772 +23773 +23774 +23775 +23776 +23777 +23778 +23779 +23780 +23781 +23782 +23783 +23784 +23785 +23786 +23787 +23788 +23789 +23790 +23791 +23792 +23793 +23794 +23795 +23796 +23797 +23798 +23799 +23800 +23801 +23802 +23803 +23804 +23805 +23806 +23807 +23808 +23809 +23810 +23811 +23812 +23813 +23814 +23815 +23816 +23817 +23818 +23819 +23820 +23821 +23822 +23823 +23824 +23825 +23826 +23827 +23828 +23829 +23830 +23831 +23832 +23833 +23834 +23835 +23836 +23837 +23838 +23839 +23840 +23841 +23842 +23843 +23844 +23845 +23846 +23847 +23848 +23849 +23850 +23851 +23852 +23853 +23854 +23855 +23856 +23857 +23858 +23859 +23860 +23861 +23862 +23863 +23864 +23865 +23866 +23867 +23868 +23869 +23870 +23871 +23872 +23873 +23874 +23875 +23876 +23877 +23878 +23879 +23880 +23881 +23882 +23883 +23884 +23885 +23886 +23887 +23888 +23889 +23890 +23891 +23892 +23893 +23894 +23895 +23896 +23897 +23898 +23899 +23900 +23901 +23902 +23903 +23904 +23905 +23906 +23907 +23908 +23909 +23910 +23911 +23912 +23913 +23914 +23915 +23916 +23917 +23918 +23919 +23920 +23921 +23922 +23923 +23924 +23925 +23926 +23927 +23928 +23929 +23930 +23931 +23932 +23933 +23934 +23935 +23936 +23937 +23938 +23939 +23940 +23941 +23942 +23943 +23944 +23945 +23946 +23947 +23948 +23949 +23950 +23951 +23952 +23953 +23954 +23955 +23956 +23957 +23958 +23959 +23960 +23961 +23962 +23963 +23964 +23965 +23966 +23967 +23968 +23969 +23970 +23971 +23972 +23973 +23974 +23975 +23976 +23977 +23978 +23979 +23980 +23981 +23982 +23983 +23984 +23985 +23986 +23987 +23988 +23989 +23990 +23991 +23992 +23993 +23994 +23995 +23996 +23997 +23998 +23999 +24000 +24001 +24002 +24003 +24004 +24005 +24006 +24007 +24008 +24009 +24010 +24011 +24012 +24013 +24014 +24015 +24016 +24017 +24018 +24019 +24020 +24021 +24022 +24023 +24024 +24025 +24026 +24027 +24028 +24029 +24030 +24031 +24032 +24033 +24034 +24035 +24036 +24037 +24038 +24039 +24040 +24041 +24042 +24043 +24044 +24045 +24046 +24047 +24048 +24049 +24050 +24051 +24052 +24053 +24054 +24055 +24056 +24057 +24058 +24059 +24060 +24061 +24062 +24063 +24064 +24065 +24066 +24067 +24068 +24069 +24070 +24071 +24072 +24073 +24074 +24075 +24076 +24077 +24078 +24079 +24080 +24081 +24082 +24083 +24084 +24085 +24086 +24087 +24088 +24089 +24090 +24091 +24092 +24093 +24094 +24095 +24096 +24097 +24098 +24099 +24100 +24101 +24102 +24103 +24104 +24105 +24106 +24107 +24108 +24109 +24110 +24111 +24112 +24113 +24114 +24115 +24116 +24117 +24118 +24119 +24120 +24121 +24122 +24123 +24124 +24125 +24126 +24127 +24128 +24129 +24130 +24131 +24132 +24133 +24134 +24135 +24136 +24137 +24138 +24139 +24140 +24141 +24142 +24143 +24144 +24145 +24146 +24147 +24148 +24149 +24150 +24151 +24152 +24153 +24154 +24155 +24156 +24157 +24158 +24159 +24160 +24161 +24162 +24163 +24164 +24165 +24166 +24167 +24168 +24169 +24170 +24171 +24172 +24173 +24174 +24175 +24176 +24177 +24178 +24179 +24180 +24181 +24182 +24183 +24184 +24185 +24186 +24187 +24188 +24189 +24190 +24191 +24192 +24193 +24194 +24195 +24196 +24197 +24198 +24199 +24200 +24201 +24202 +24203 +24204 +24205 +24206 +24207 +24208 +24209 +24210 +24211 +24212 +24213 +24214 +24215 +24216 +24217 +24218 +24219 +24220 +24221 +24222 +24223 +24224 +24225 +24226 +24227 +24228 +24229 +24230 +24231 +24232 +24233 +24234 +24235 +24236 +24237 +24238 +24239 +24240 +24241 +24242 +24243 +24244 +24245 +24246 +24247 +24248 +24249 +24250 +24251 +24252 +24253 +24254 +24255 +24256 +24257 +24258 +24259 +24260 +24261 +24262 +24263 +24264 +24265 +24266 +24267 +24268 +24269 +24270 +24271 +24272 +24273 +24274 +24275 +24276 +24277 +24278 +24279 +24280 +24281 +24282 +24283 +24284 +24285 +24286 +24287 +24288 +24289 +24290 +24291 +24292 +24293 +24294 +24295 +24296 +24297 +24298 +24299 +24300 +24301 +24302 +24303 +24304 +24305 +24306 +24307 +24308 +24309 +24310 +24311 +24312 +24313 +24314 +24315 +24316 +24317 +24318 +24319 +24320 +24321 +24322 +24323 +24324 +24325 +24326 +24327 +24328 +24329 +24330 +24331 +24332 +24333 +24334 +24335 +24336 +24337 +24338 +24339 +24340 +24341 +24342 +24343 +24344 +24345 +24346 +24347 +24348 +24349 +24350 +24351 +24352 +24353 +24354 +24355 +24356 +24357 +24358 +24359 +24360 +24361 +24362 +24363 +24364 +24365 +24366 +24367 +24368 +24369 +24370 +24371 +24372 +24373 +24374 +24375 +24376 +24377 +24378 +24379 +24380 +24381 +24382 +24383 +24384 +24385 +24386 +24387 +24388 +24389 +24390 +24391 +24392 +24393 +24394 +24395 +24396 +24397 +24398 +24399 +24400 +24401 +24402 +24403 +24404 +24405 +24406 +24407 +24408 +24409 +24410 +24411 +24412 +24413 +24414 +24415 +24416 +24417 +24418 +24419 +24420 +24421 +24422 +24423 +24424 +24425 +24426 +24427 +24428 +24429 +24430 +24431 +24432 +24433 +24434 +24435 +24436 +24437 +24438 +24439 +24440 +24441 +24442 +24443 +24444 +24445 +24446 +24447 +24448 +24449 +24450 +24451 +24452 +24453 +24454 +24455 +24456 +24457 +24458 +24459 +24460 +24461 +24462 +24463 +24464 +24465 +24466 +24467 +24468 +24469 +24470 +24471 +24472 +24473 +24474 +24475 +24476 +24477 +24478 +24479 +24480 +24481 +24482 +24483 +24484 +24485 +24486 +24487 +24488 +24489 +24490 +24491 +24492 +24493 +24494 +24495 +24496 +24497 +24498 +24499 +24500 +24501 +24502 +24503 +24504 +24505 +24506 +24507 +24508 +24509 +24510 +24511 +24512 +24513 +24514 +24515 +24516 +24517 +24518 +24519 +24520 +24521 +24522 +24523 +24524 +24525 +24526 +24527 +24528 +24529 +24530 +24531 +24532 +24533 +24534 +24535 +24536 +24537 +24538 +24539 +24540 +24541 +24542 +24543 +24544 +24545 +24546 +24547 +24548 +24549 +24550 +24551 +24552 +24553 +24554 +24555 +24556 +24557 +24558 +24559 +24560 +24561 +24562 +24563 +24564 +24565 +24566 +24567 +24568 +24569 +24570 +24571 +24572 +24573 +24574 +24575 +24576 +24577 +24578 +24579 +24580 +24581 +24582 +24583 +24584 +24585 +24586 +24587 +24588 +24589 +24590 +24591 +24592 +24593 +24594 +24595 +24596 +24597 +24598 +24599 +24600 +24601 +24602 +24603 +24604 +24605 +24606 +24607 +24608 +24609 +24610 +24611 +24612 +24613 +24614 +24615 +24616 +24617 +24618 +24619 +24620 +24621 +24622 +24623 +24624 +24625 +24626 +24627 +24628 +24629 +24630 +24631 +24632 +24633 +24634 +24635 +24636 +24637 +24638 +24639 +24640 +24641 +24642 +24643 +24644 +24645 +24646 +24647 +24648 +24649 +24650 +24651 +24652 +24653 +24654 +24655 +24656 +24657 +24658 +24659 +24660 +24661 +24662 +24663 +24664 +24665 +24666 +24667 +24668 +24669 +24670 +24671 +24672 +24673 +24674 +24675 +24676 +24677 +24678 +24679 +24680 +24681 +24682 +24683 +24684 +24685 +24686 +24687 +24688 +24689 +24690 +24691 +24692 +24693 +24694 +24695 +24696 +24697 +24698 +24699 +24700 +24701 +24702 +24703 +24704 +24705 +24706 +24707 +24708 +24709 +24710 +24711 +24712 +24713 +24714 +24715 +24716 +24717 +24718 +24719 +24720 +24721 +24722 +24723 +24724 +24725 +24726 +24727 +24728 +24729 +24730 +24731 +24732 +24733 +24734 +24735 +24736 +24737 +24738 +24739 +24740 +24741 +24742 +24743 +24744 +24745 +24746 +24747 +24748 +24749 +24750 +24751 +24752 +24753 +24754 +24755 +24756 +24757 +24758 +24759 +24760 +24761 +24762 +24763 +24764 +24765 +24766 +24767 +24768 +24769 +24770 +24771 +24772 +24773 +24774 +24775 +24776 +24777 +24778 +24779 +24780 +24781 +24782 +24783 +24784 +24785 +24786 +24787 +24788 +24789 +24790 +24791 +24792 +24793 +24794 +24795 +24796 +24797 +24798 +24799 +24800 +24801 +24802 +24803 +24804 +24805 +24806 +24807 +24808 +24809 +24810 +24811 +24812 +24813 +24814 +24815 +24816 +24817 +24818 +24819 +24820 +24821 +24822 +24823 +24824 +24825 +24826 +24827 +24828 +24829 +24830 +24831 +24832 +24833 +24834 +24835 +24836 +24837 +24838 +24839 +24840 +24841 +24842 +24843 +24844 +24845 +24846 +24847 +24848 +24849 +24850 +24851 +24852 +24853 +24854 +24855 +24856 +24857 +24858 +24859 +24860 +24861 +24862 +24863 +24864 +24865 +24866 +24867 +24868 +24869 +24870 +24871 +24872 +24873 +24874 +24875 +24876 +24877 +24878 +24879 +24880 +24881 +24882 +24883 +24884 +24885 +24886 +24887 +24888 +24889 +24890 +24891 +24892 +24893 +24894 +24895 +24896 +24897 +24898 +24899 +24900 +24901 +24902 +24903 +24904 +24905 +24906 +24907 +24908 +24909 +24910 +24911 +24912 +24913 +24914 +24915 +24916 +24917 +24918 +24919 +24920 +24921 +24922 +24923 +24924 +24925 +24926 +24927 +24928 +24929 +24930 +24931 +24932 +24933 +24934 +24935 +24936 +24937 +24938 +24939 +24940 +24941 +24942 +24943 +24944 +24945 +24946 +24947 +24948 +24949 +24950 +24951 +24952 +24953 +24954 +24955 +24956 +24957 +24958 +24959 +24960 +24961 +24962 +24963 +24964 +24965 +24966 +24967 +24968 +24969 +24970 +24971 +24972 +24973 +24974 +24975 +24976 +24977 +24978 +24979 +24980 +24981 +24982 +24983 +24984 +24985 +24986 +24987 +24988 +24989 +24990 +24991 +24992 +24993 +24994 +24995 +24996 +24997 +24998 +24999 +25000 +25001 +25002 +25003 +25004 +25005 +25006 +25007 +25008 +25009 +25010 +25011 +25012 +25013 +25014 +25015 +25016 +25017 +25018 +25019 +25020 +25021 +25022 +25023 +25024 +25025 +25026 +25027 +25028 +25029 +25030 +25031 +25032 +25033 +25034 +25035 +25036 +25037 +25038 +25039 +25040 +25041 +25042 +25043 +25044 +25045 +25046 +25047 +25048 +25049 +25050 +25051 +25052 +25053 +25054 +25055 +25056 +25057 +25058 +25059 +25060 +25061 +25062 +25063 +25064 +25065 +25066 +25067 +25068 +25069 +25070 +25071 +25072 +25073 +25074 +25075 +25076 +25077 +25078 +25079 +25080 +25081 +25082 +25083 +25084 +25085 +25086 +25087 +25088 +25089 +25090 +25091 +25092 +25093 +25094 +25095 +25096 +25097 +25098 +25099 +25100 +25101 +25102 +25103 +25104 +25105 +25106 +25107 +25108 +25109 +25110 +25111 +25112 +25113 +25114 +25115 +25116 +25117 +25118 +25119 +25120 +25121 +25122 +25123 +25124 +25125 +25126 +25127 +25128 +25129 +25130 +25131 +25132 +25133 +25134 +25135 +25136 +25137 +25138 +25139 +25140 +25141 +25142 +25143 +25144 +25145 +25146 +25147 +25148 +25149 +25150 +25151 +25152 +25153 +25154 +25155 +25156 +25157 +25158 +25159 +25160 +25161 +25162 +25163 +25164 +25165 +25166 +25167 +25168 +25169 +25170 +25171 +25172 +25173 +25174 +25175 +25176 +25177 +25178 +25179 +25180 +25181 +25182 +25183 +25184 +25185 +25186 +25187 +25188 +25189 +25190 +25191 +25192 +25193 +25194 +25195 +25196 +25197 +25198 +25199 +25200 +25201 +25202 +25203 +25204 +25205 +25206 +25207 +25208 +25209 +25210 +25211 +25212 +25213 +25214 +25215 +25216 +25217 +25218 +25219 +25220 +25221 +25222 +25223 +25224 +25225 +25226 +25227 +25228 +25229 +25230 +25231 +25232 +25233 +25234 +25235 +25236 +25237 +25238 +25239 +25240 +25241 +25242 +25243 +25244 +25245 +25246 +25247 +25248 +25249 +25250 +25251 +25252 +25253 +25254 +25255 +25256 +25257 +25258 +25259 +25260 +25261 +25262 +25263 +25264 +25265 +25266 +25267 +25268 +25269 +25270 +25271 +25272 +25273 +25274 +25275 +25276 +25277 +25278 +25279 +25280 +25281 +25282 +25283 +25284 +25285 +25286 +25287 +25288 +25289 +25290 +25291 +25292 +25293 +25294 +25295 +25296 +25297 +25298 +25299 +25300 +25301 +25302 +25303 +25304 +25305 +25306 +25307 +25308 +25309 +25310 +25311 +25312 +25313 +25314 +25315 +25316 +25317 +25318 +25319 +25320 +25321 +25322 +25323 +25324 +25325 +25326 +25327 +25328 +25329 +25330 +25331 +25332 +25333 +25334 +25335 +25336 +25337 +25338 +25339 +25340 +25341 +25342 +25343 +25344 +25345 +25346 +25347 +25348 +25349 +25350 +25351 +25352 +25353 +25354 +25355 +25356 +25357 +25358 +25359 +25360 +25361 +25362 +25363 +25364 +25365 +25366 +25367 +25368 +25369 +25370 +25371 +25372 +25373 +25374 +25375 +25376 +25377 +25378 +25379 +25380 +25381 +25382 +25383 +25384 +25385 +25386 +25387 +25388 +25389 +25390 +25391 +25392 +25393 +25394 +25395 +25396 +25397 +25398 +25399 +25400 +25401 +25402 +25403 +25404 +25405 +25406 +25407 +25408 +25409 +25410 +25411 +25412 +25413 +25414 +25415 +25416 +25417 +25418 +25419 +25420 +25421 +25422 +25423 +25424 +25425 +25426 +25427 +25428 +25429 +25430 +25431 +25432 +25433 +25434 +25435 +25436 +25437 +25438 +25439 +25440 +25441 +25442 +25443 +25444 +25445 +25446 +25447 +25448 +25449 +25450 +25451 +25452 +25453 +25454 +25455 +25456 +25457 +25458 +25459 +25460 +25461 +25462 +25463 +25464 +25465 +25466 +25467 +25468 +25469 +25470 +25471 +25472 +25473 +25474 +25475 +25476 +25477 +25478 +25479 +25480 +25481 +25482 +25483 +25484 +25485 +25486 +25487 +25488 +25489 +25490 +25491 +25492 +25493 +25494 +25495 +25496 +25497 +25498 +25499 +25500 +25501 +25502 +25503 +25504 +25505 +25506 +25507 +25508 +25509 +25510 +25511 +25512 +25513 +25514 +25515 +25516 +25517 +25518 +25519 +25520 +25521 +25522 +25523 +25524 +25525 +25526 +25527 +25528 +25529 +25530 +25531 +25532 +25533 +25534 +25535 +25536 +25537 +25538 +25539 +25540 +25541 +25542 +25543 +25544 +25545 +25546 +25547 +25548 +25549 +25550 +25551 +25552 +25553 +25554 +25555 +25556 +25557 +25558 +25559 +25560 +25561 +25562 +25563 +25564 +25565 +25566 +25567 +25568 +25569 +25570 +25571 +25572 +25573 +25574 +25575 +25576 +25577 +25578 +25579 +25580 +25581 +25582 +25583 +25584 +25585 +25586 +25587 +25588 +25589 +25590 +25591 +25592 +25593 +25594 +25595 +25596 +25597 +25598 +25599 +25600 +25601 +25602 +25603 +25604 +25605 +25606 +25607 +25608 +25609 +25610 +25611 +25612 +25613 +25614 +25615 +25616 +25617 +25618 +25619 +25620 +25621 +25622 +25623 +25624 +25625 +25626 +25627 +25628 +25629 +25630 +25631 +25632 +25633 +25634 +25635 +25636 +25637 +25638 +25639 +25640 +25641 +25642 +25643 +25644 +25645 +25646 +25647 +25648 +25649 +25650 +25651 +25652 +25653 +25654 +25655 +25656 +25657 +25658 +25659 +25660 +25661 +25662 +25663 +25664 +25665 +25666 +25667 +25668 +25669 +25670 +25671 +25672 +25673 +25674 +25675 +25676 +25677 +25678 +25679 +25680 +25681 +25682 +25683 +25684 +25685 +25686 +25687 +25688 +25689 +25690 +25691 +25692 +25693 +25694 +25695 +25696 +25697 +25698 +25699 +25700 +25701 +25702 +25703 +25704 +25705 +25706 +25707 +25708 +25709 +25710 +25711 +25712 +25713 +25714 +25715 +25716 +25717 +25718 +25719 +25720 +25721 +25722 +25723 +25724 +25725 +25726 +25727 +25728 +25729 +25730 +25731 +25732 +25733 +25734 +25735 +25736 +25737 +25738 +25739 +25740 +25741 +25742 +25743 +25744 +25745 +25746 +25747 +25748 +25749 +25750 +25751 +25752 +25753 +25754 +25755 +25756 +25757 +25758 +25759 +25760 +25761 +25762 +25763 +25764 +25765 +25766 +25767 +25768 +25769 +25770 +25771 +25772 +25773 +25774 +25775 +25776 +25777 +25778 +25779 +25780 +25781 +25782 +25783 +25784 +25785 +25786 +25787 +25788 +25789 +25790 +25791 +25792 +25793 +25794 +25795 +25796 +25797 +25798 +25799 +25800 +25801 +25802 +25803 +25804 +25805 +25806 +25807 +25808 +25809 +25810 +25811 +25812 +25813 +25814 +25815 +25816 +25817 +25818 +25819 +25820 +25821 +25822 +25823 +25824 +25825 +25826 +25827 +25828 +25829 +25830 +25831 +25832 +25833 +25834 +25835 +25836 +25837 +25838 +25839 +25840 +25841 +25842 +25843 +25844 +25845 +25846 +25847 +25848 +25849 +25850 +25851 +25852 +25853 +25854 +25855 +25856 +25857 +25858 +25859 +25860 +25861 +25862 +25863 +25864 +25865 +25866 +25867 +25868 +25869 +25870 +25871 +25872 +25873 +25874 +25875 +25876 +25877 +25878 +25879 +25880 +25881 +25882 +25883 +25884 +25885 +25886 +25887 +25888 +25889 +25890 +25891 +25892 +25893 +25894 +25895 +25896 +25897 +25898 +25899 +25900 +25901 +25902 +25903 +25904 +25905 +25906 +25907 +25908 +25909 +25910 +25911 +25912 +25913 +25914 +25915 +25916 +25917 +25918 +25919 +25920 +25921 +25922 +25923 +25924 +25925 +25926 +25927 +25928 +25929 +25930 +25931 +25932 +25933 +25934 +25935 +25936 +25937 +25938 +25939 +25940 +25941 +25942 +25943 +25944 +25945 +25946 +25947 +25948 +25949 +25950 +25951 +25952 +25953 +25954 +25955 +25956 +25957 +25958 +25959 +25960 +25961 +25962 +25963 +25964 +25965 +25966 +25967 +25968 +25969 +25970 +25971 +25972 +25973 +25974 +25975 +25976 +25977 +25978 +25979 +25980 +25981 +25982 +25983 +25984 +25985 +25986 +25987 +25988 +25989 +25990 +25991 +25992 +25993 +25994 +25995 +25996 +25997 +25998 +25999 +26000 +26001 +26002 +26003 +26004 +26005 +26006 +26007 +26008 +26009 +26010 +26011 +26012 +26013 +26014 +26015 +26016 +26017 +26018 +26019 +26020 +26021 +26022 +26023 +26024 +26025 +26026 +26027 +26028 +26029 +26030 +26031 +26032 +26033 +26034 +26035 +26036 +26037 +26038 +26039 +26040 +26041 +26042 +26043 +26044 +26045 +26046 +26047 +26048 +26049 +26050 +26051 +26052 +26053 +26054 +26055 +26056 +26057 +26058 +26059 +26060 +26061 +26062 +26063 +26064 +26065 +26066 +26067 +26068 +26069 +26070 +26071 +26072 +26073 +26074 +26075 +26076 +26077 +26078 +26079 +26080 +26081 +26082 +26083 +26084 +26085 +26086 +26087 +26088 +26089 +26090 +26091 +26092 +26093 +26094 +26095 +26096 +26097 +26098 +26099 +26100 +26101 +26102 +26103 +26104 +26105 +26106 +26107 +26108 +26109 +26110 +26111 +26112 +26113 +26114 +26115 +26116 +26117 +26118 +26119 +26120 +26121 +26122 +26123 +26124 +26125 +26126 +26127 +26128 +26129 +26130 +26131 +26132 +26133 +26134 +26135 +26136 +26137 +26138 +26139 +26140 +26141 +26142 +26143 +26144 +26145 +26146 +26147 +26148 +26149 +26150 +26151 +26152 +26153 +26154 +26155 +26156 +26157 +26158 +26159 +26160 +26161 +26162 +26163 +26164 +26165 +26166 +26167 +26168 +26169 +26170 +26171 +26172 +26173 +26174 +26175 +26176 +26177 +26178 +26179 +26180 +26181 +26182 +26183 +26184 +26185 +26186 +26187 +26188 +26189 +26190 +26191 +26192 +26193 +26194 +26195 +26196 +26197 +26198 +26199 +26200 +26201 +26202 +26203 +26204 +26205 +26206 +26207 +26208 +26209 +26210 +26211 +26212 +26213 +26214 +26215 +26216 +26217 +26218 +26219 +26220 +26221 +26222 +26223 +26224 +26225 +26226 +26227 +26228 +26229 +26230 +26231 +26232 +26233 +26234 +26235 +26236 +26237 +26238 +26239 +26240 +26241 +26242 +26243 +26244 +26245 +26246 +26247 +26248 +26249 +26250 +26251 +26252 +26253 +26254 +26255 +26256 +26257 +26258 +26259 +26260 +26261 +26262 +26263 +26264 +26265 +26266 +26267 +26268 +26269 +26270 +26271 +26272 +26273 +26274 +26275 +26276 +26277 +26278 +26279 +26280 +26281 +26282 +26283 +26284 +26285 +26286 +26287 +26288 +26289 +26290 +26291 +26292 +26293 +26294 +26295 +26296 +26297 +26298 +26299 +26300 +26301 +26302 +26303 +26304 +26305 +26306 +26307 +26308 +26309 +26310 +26311 +26312 +26313 +26314 +26315 +26316 +26317 +26318 +26319 +26320 +26321 +26322 +26323 +26324 +26325 +26326 +26327 +26328 +26329 +26330 +26331 +26332 +26333 +26334 +26335 +26336 +26337 +26338 +26339 +26340 +26341 +26342 +26343 +26344 +26345 +26346 +26347 +26348 +26349 +26350 +26351 +26352 +26353 +26354 +26355 +26356 +26357 +26358 +26359 +26360 +26361 +26362 +26363 +26364 +26365 +26366 +26367 +26368 +26369 +26370 +26371 +26372 +26373 +26374 +26375 +26376 +26377 +26378 +26379 +26380 +26381 +26382 +26383 +26384 +26385 +26386 +26387 +26388 +26389 +26390 +26391 +26392 +26393 +26394 +26395 +26396 +26397 +26398 +26399 +26400 +26401 +26402 +26403 +26404 +26405 +26406 +26407 +26408 +26409 +26410 +26411 +26412 +26413 +26414 +26415 +26416 +26417 +26418 +26419 +26420 +26421 +26422 +26423 +26424 +26425 +26426 +26427 +26428 +26429 +26430 +26431 +26432 +26433 +26434 +26435 +26436 +26437 +26438 +26439 +26440 +26441 +26442 +26443 +26444 +26445 +26446 +26447 +26448 +26449 +26450 +26451 +26452 +26453 +26454 +26455 +26456 +26457 +26458 +26459 +26460 +26461 +26462 +26463 +26464 +26465 +26466 +26467 +26468 +26469 +26470 +26471 +26472 +26473 +26474 +26475 +26476 +26477 +26478 +26479 +26480 +26481 +26482 +26483 +26484 +26485 +26486 +26487 +26488 +26489 +26490 +26491 +26492 +26493 +26494 +26495 +26496 +26497 +26498 +26499 +26500 +26501 +26502 +26503 +26504 +26505 +26506 +26507 +26508 +26509 +26510 +26511 +26512 +26513 +26514 +26515 +26516 +26517 +26518 +26519 +26520 +26521 +26522 +26523 +26524 +26525 +26526 +26527 +26528 +26529 +26530 +26531 +26532 +26533 +26534 +26535 +26536 +26537 +26538 +26539 +26540 +26541 +26542 +26543 +26544 +26545 +26546 +26547 +26548 +26549 +26550 +26551 +26552 +26553 +26554 +26555 +26556 +26557 +26558 +26559 +26560 +26561 +26562 +26563 +26564 +26565 +26566 +26567 +26568 +26569 +26570 +26571 +26572 +26573 +26574 +26575 +26576 +26577 +26578 +26579 +26580 +26581 +26582 +26583 +26584 +26585 +26586 +26587 +26588 +26589 +26590 +26591 +26592 +26593 +26594 +26595 +26596 +26597 +26598 +26599 +26600 +26601 +26602 +26603 +26604 +26605 +26606 +26607 +26608 +26609 +26610 +26611 +26612 +26613 +26614 +26615 +26616 +26617 +26618 +26619 +26620 +26621 +26622 +26623 +26624 +26625 +26626 +26627 +26628 +26629 +26630 +26631 +26632 +26633 +26634 +26635 +26636 +26637 +26638 +26639 +26640 +26641 +26642 +26643 +26644 +26645 +26646 +26647 +26648 +26649 +26650 +26651 +26652 +26653 +26654 +26655 +26656 +26657 +26658 +26659 +26660 +26661 +26662 +26663 +26664 +26665 +26666 +26667 +26668 +26669 +26670 +26671 +26672 +26673 +26674 +26675 +26676 +26677 +26678 +26679 +26680 +26681 +26682 +26683 +26684 +26685 +26686 +26687 +26688 +26689 +26690 +26691 +26692 +26693 +26694 +26695 +26696 +26697 +26698 +26699 +26700 +26701 +26702 +26703 +26704 +26705 +26706 +26707 +26708 +26709 +26710 +26711 +26712 +26713 +26714 +26715 +26716 +26717 +26718 +26719 +26720 +26721 +26722 +26723 +26724 +26725 +26726 +26727 +26728 +26729 +26730 +26731 +26732 +26733 +26734 +26735 +26736 +26737 +26738 +26739 +26740 +26741 +26742 +26743 +26744 +26745 +26746 +26747 +26748 +26749 +26750 +26751 +26752 +26753 +26754 +26755 +26756 +26757 +26758 +26759 +26760 +26761 +26762 +26763 +26764 +26765 +26766 +26767 +26768 +26769 +26770 +26771 +26772 +26773 +26774 +26775 +26776 +26777 +26778 +26779 +26780 +26781 +26782 +26783 +26784 +26785 +26786 +26787 +26788 +26789 +26790 +26791 +26792 +26793 +26794 +26795 +26796 +26797 +26798 +26799 +26800 +26801 +26802 +26803 +26804 +26805 +26806 +26807 +26808 +26809 +26810 +26811 +26812 +26813 +26814 +26815 +26816 +26817 +26818 +26819 +26820 +26821 +26822 +26823 +26824 +26825 +26826 +26827 +26828 +26829 +26830 +26831 +26832 +26833 +26834 +26835 +26836 +26837 +26838 +26839 +26840 +26841 +26842 +26843 +26844 +26845 +26846 +26847 +26848 +26849 +26850 +26851 +26852 +26853 +26854 +26855 +26856 +26857 +26858 +26859 +26860 +26861 +26862 +26863 +26864 +26865 +26866 +26867 +26868 +26869 +26870 +26871 +26872 +26873 +26874 +26875 +26876 +26877 +26878 +26879 +26880 +26881 +26882 +26883 +26884 +26885 +26886 +26887 +26888 +26889 +26890 +26891 +26892 +26893 +26894 +26895 +26896 +26897 +26898 +26899 +26900 +26901 +26902 +26903 +26904 +26905 +26906 +26907 +26908 +26909 +26910 +26911 +26912 +26913 +26914 +26915 +26916 +26917 +26918 +26919 +26920 +26921 +26922 +26923 +26924 +26925 +26926 +26927 +26928 +26929 +26930 +26931 +26932 +26933 +26934 +26935 +26936 +26937 +26938 +26939 +26940 +26941 +26942 +26943 +26944 +26945 +26946 +26947 +26948 +26949 +26950 +26951 +26952 +26953 +26954 +26955 +26956 +26957 +26958 +26959 +26960 +26961 +26962 +26963 +26964 +26965 +26966 +26967 +26968 +26969 +26970 +26971 +26972 +26973 +26974 +26975 +26976 +26977 +26978 +26979 +26980 +26981 +26982 +26983 +26984 +26985 +26986 +26987 +26988 +26989 +26990 +26991 +26992 +26993 +26994 +26995 +26996 +26997 +26998 +26999 +27000 +27001 +27002 +27003 +27004 +27005 +27006 +27007 +27008 +27009 +27010 +27011 +27012 +27013 +27014 +27015 +27016 +27017 +27018 +27019 +27020 +27021 +27022 +27023 +27024 +27025 +27026 +27027 +27028 +27029 +27030 +27031 +27032 +27033 +27034 +27035 +27036 +27037 +27038 +27039 +27040 +27041 +27042 +27043 +27044 +27045 +27046 +27047 +27048 +27049 +27050 +27051 +27052 +27053 +27054 +27055 +27056 +27057 +27058 +27059 +27060 +27061 +27062 +27063 +27064 +27065 +27066 +27067 +27068 +27069 +27070 +27071 +27072 +27073 +27074 +27075 +27076 +27077 +27078 +27079 +27080 +27081 +27082 +27083 +27084 +27085 +27086 +27087 +27088 +27089 +27090 +27091 +27092 +27093 +27094 +27095 +27096 +27097 +27098 +27099 +27100 +27101 +27102 +27103 +27104 +27105 +27106 +27107 +27108 +27109 +27110 +27111 +27112 +27113 +27114 +27115 +27116 +27117 +27118 +27119 +27120 +27121 +27122 +27123 +27124 +27125 +27126 +27127 +27128 +27129 +27130 +27131 +27132 +27133 +27134 +27135 +27136 +27137 +27138 +27139 +27140 +27141 +27142 +27143 +27144 +27145 +27146 +27147 +27148 +27149 +27150 +27151 +27152 +27153 +27154 +27155 +27156 +27157 +27158 +27159 +27160 +27161 +27162 +27163 +27164 +27165 +27166 +27167 +27168 +27169 +27170 +27171 +27172 +27173 +27174 +27175 +27176 +27177 +27178 +27179 +27180 +27181 +27182 +27183 +27184 +27185 +27186 +27187 +27188 +27189 +27190 +27191 +27192 +27193 +27194 +27195 +27196 +27197 +27198 +27199 +27200 +27201 +27202 +27203 +27204 +27205 +27206 +27207 +27208 +27209 +27210 +27211 +27212 +27213 +27214 +27215 +27216 +27217 +27218 +27219 +27220 +27221 +27222 +27223 +27224 +27225 +27226 +27227 +27228 +27229 +27230 +27231 +27232 +27233 +27234 +27235 +27236 +27237 +27238 +27239 +27240 +27241 +27242 +27243 +27244 +27245 +27246 +27247 +27248 +27249 +27250 +27251 +27252 +27253 +27254 +27255 +27256 +27257 +27258 +27259 +27260 +27261 +27262 +27263 +27264 +27265 +27266 +27267 +27268 +27269 +27270 +27271 +27272 +27273 +27274 +27275 +27276 +27277 +27278 +27279 +27280 +27281 +27282 +27283 +27284 +27285 +27286 +27287 +27288 +27289 +27290 +27291 +27292 +27293 +27294 +27295 +27296 +27297 +27298 +27299 +27300 +27301 +27302 +27303 +27304 +27305 +27306 +27307 +27308 +27309 +27310 +27311 +27312 +27313 +27314 +27315 +27316 +27317 +27318 +27319 +27320 +27321 +27322 +27323 +27324 +27325 +27326 +27327 +27328 +27329 +27330 +27331 +27332 +27333 +27334 +27335 +27336 +27337 +27338 +27339 +27340 +27341 +27342 +27343 +27344 +27345 +27346 +27347 +27348 +27349 +27350 +27351 +27352 +27353 +27354 +27355 +27356 +27357 +27358 +27359 +27360 +27361 +27362 +27363 +27364 +27365 +27366 +27367 +27368 +27369 +27370 +27371 +27372 +27373 +27374 +27375 +27376 +27377 +27378 +27379 +27380 +27381 +27382 +27383 +27384 +27385 +27386 +27387 +27388 +27389 +27390 +27391 +27392 +27393 +27394 +27395 +27396 +27397 +27398 +27399 +27400 +27401 +27402 +27403 +27404 +27405 +27406 +27407 +27408 +27409 +27410 +27411 +27412 +27413 +27414 +27415 +27416 +27417 +27418 +27419 +27420 +27421 +27422 +27423 +27424 +27425 +27426 +27427 +27428 +27429 +27430 +27431 +27432 +27433 +27434 +27435 +27436 +27437 +27438 +27439 +27440 +27441 +27442 +27443 +27444 +27445 +27446 +27447 +27448 +27449 +27450 +27451 +27452 +27453 +27454 +27455 +27456 +27457 +27458 +27459 +27460 +27461 +27462 +27463 +27464 +27465 +27466 +27467 +27468 +27469 +27470 +27471 +27472 +27473 +27474 +27475 +27476 +27477 +27478 +27479 +27480 +27481 +27482 +27483 +27484 +27485 +27486 +27487 +27488 +27489 +27490 +27491 +27492 +27493 +27494 +27495 +27496 +27497 +27498 +27499 +27500 +27501 +27502 +27503 +27504 +27505 +27506 +27507 +27508 +27509 +27510 +27511 +27512 +27513 +27514 +27515 +27516 +27517 +27518 +27519 +27520 +27521 +27522 +27523 +27524 +27525 +27526 +27527 +27528 +27529 +27530 +27531 +27532 +27533 +27534 +27535 +27536 +27537 +27538 +27539 +27540 +27541 +27542 +27543 +27544 +27545 +27546 +27547 +27548 +27549 +27550 +27551 +27552 +27553 +27554 +27555 +27556 +27557 +27558 +27559 +27560 +27561 +27562 +27563 +27564 +27565 +27566 +27567 +27568 +27569 +27570 +27571 +27572 +27573 +27574 +27575 +27576 +27577 +27578 +27579 +27580 +27581 +27582 +27583 +27584 +27585 +27586 +27587 +27588 +27589 +27590 +27591 +27592 +27593 +27594 +27595 +27596 +27597 +27598 +27599 +27600 +27601 +27602 +27603 +27604 +27605 +27606 +27607 +27608 +27609 +27610 +27611 +27612 +27613 +27614 +27615 +27616 +27617 +27618 +27619 +27620 +27621 +27622 +27623 +27624 +27625 +27626 +27627 +27628 +27629 +27630 +27631 +27632 +27633 +27634 +27635 +27636 +27637 +27638 +27639 +27640 +27641 +27642 +27643 +27644 +27645 +27646 +27647 +27648 +27649 +27650 +27651 +27652 +27653 +27654 +27655 +27656 +27657 +27658 +27659 +27660 +27661 +27662 +27663 +27664 +27665 +27666 +27667 +27668 +27669 +27670 +27671 +27672 +27673 +27674 +27675 +27676 +27677 +27678 +27679 +27680 +27681 +27682 +27683 +27684 +27685 +27686 +27687 +27688 +27689 +27690 +27691 +27692 +27693 +27694 +27695 +27696 +27697 +27698 +27699 +27700 +27701 +27702 +27703 +27704 +27705 +27706 +27707 +27708 +27709 +27710 +27711 +27712 +27713 +27714 +27715 +27716 +27717 +27718 +27719 +27720 +27721 +27722 +27723 +27724 +27725 +27726 +27727 +27728 +27729 +27730 +27731 +27732 +27733 +27734 +27735 +27736 +27737 +27738 +27739 +27740 +27741 +27742 +27743 +27744 +27745 +27746 +27747 +27748 +27749 +27750 +27751 +27752 +27753 +27754 +27755 +27756 +27757 +27758 +27759 +27760 +27761 +27762 +27763 +27764 +27765 +27766 +27767 +27768 +27769 +27770 +27771 +27772 +27773 +27774 +27775 +27776 +27777 +27778 +27779 +27780 +27781 +27782 +27783 +27784 +27785 +27786 +27787 +27788 +27789 +27790 +27791 +27792 +27793 +27794 +27795 +27796 +27797 +27798 +27799 +27800 +27801 +27802 +27803 +27804 +27805 +27806 +27807 +27808 +27809 +27810 +27811 +27812 +27813 +27814 +27815 +27816 +27817 +27818 +27819 +27820 +27821 +27822 +27823 +27824 +27825 +27826 +27827 +27828 +27829 +27830 +27831 +27832 +27833 +27834 +27835 +27836 +27837 +27838 +27839 +27840 +27841 +27842 +27843 +27844 +27845 +27846 +27847 +27848 +27849 +27850 +27851 +27852 +27853 +27854 +27855 +27856 +27857 +27858 +27859 +27860 +27861 +27862 +27863 +27864 +27865 +27866 +27867 +27868 +27869 +27870 +27871 +27872 +27873 +27874 +27875 +27876 +27877 +27878 +27879 +27880 +27881 +27882 +27883 +27884 +27885 +27886 +27887 +27888 +27889 +27890 +27891 +27892 +27893 +27894 +27895 +27896 +27897 +27898 +27899 +27900 +27901 +27902 +27903 +27904 +27905 +27906 +27907 +27908 +27909 +27910 +27911 +27912 +27913 +27914 +27915 +27916 +27917 +27918 +27919 +27920 +27921 +27922 +27923 +27924 +27925 +27926 +27927 +27928 +27929 +27930 +27931 +27932 +27933 +27934 +27935 +27936 +27937 +27938 +27939 +27940 +27941 +27942 +27943 +27944 +27945 +27946 +27947 +27948 +27949 +27950 +27951 +27952 +27953 +27954 +27955 +27956 +27957 +27958 +27959 +27960 +27961 +27962 +27963 +27964 +27965 +27966 +27967 +27968 +27969 +27970 +27971 +27972 +27973 +27974 +27975 +27976 +27977 +27978 +27979 +27980 +27981 +27982 +27983 +27984 +27985 +27986 +27987 +27988 +27989 +27990 +27991 +27992 +27993 +27994 +27995 +27996 +27997 +27998 +27999 +28000 +28001 +28002 +28003 +28004 +28005 +28006 +28007 +28008 +28009 +28010 +28011 +28012 +28013 +28014 +28015 +28016 +28017 +28018 +28019 +28020 +28021 +28022 +28023 +28024 +28025 +28026 +28027 +28028 +28029 +28030 +28031 +28032 +28033 +28034 +28035 +28036 +28037 +28038 +28039 +28040 +28041 +28042 +28043 +28044 +28045 +28046 +28047 +28048 +28049 +28050 +28051 +28052 +28053 +28054 +28055 +28056 +28057 +28058 +28059 +28060 +28061 +28062 +28063 +28064 +28065 +28066 +28067 +28068 +28069 +28070 +28071 +28072 +28073 +28074 +28075 +28076 +28077 +28078 +28079 +28080 +28081 +28082 +28083 +28084 +28085 +28086 +28087 +28088 +28089 +28090 +28091 +28092 +28093 +28094 +28095 +28096 +28097 +28098 +28099 +28100 +28101 +28102 +28103 +28104 +28105 +28106 +28107 +28108 +28109 +28110 +28111 +28112 +28113 +28114 +28115 +28116 +28117 +28118 +28119 +28120 +28121 +28122 +28123 +28124 +28125 +28126 +28127 +28128 +28129 +28130 +28131 +28132 +28133 +28134 +28135 +28136 +28137 +28138 +28139 +28140 +28141 +28142 +28143 +28144 +28145 +28146 +28147 +28148 +28149 +28150 +28151 +28152 +28153 +28154 +28155 +28156 +28157 +28158 +28159 +28160 +28161 +28162 +28163 +28164 +28165 +28166 +28167 +28168 +28169 +28170 +28171 +28172 +28173 +28174 +28175 +28176 +28177 +28178 +28179 +28180 +28181 +28182 +28183 +28184 +28185 +28186 +28187 +28188 +28189 +28190 +28191 +28192 +28193 +28194 +28195 +28196 +28197 +28198 +28199 +28200 +28201 +28202 +28203 +28204 +28205 +28206 +28207 +28208 +28209 +28210 +28211 +28212 +28213 +28214 +28215 +28216 +28217 +28218 +28219 +28220 +28221 +28222 +28223 +28224 +28225 +28226 +28227 +28228 +28229 +28230 +28231 +28232 +28233 +28234 +28235 +28236 +28237 +28238 +28239 +28240 +28241 +28242 +28243 +28244 +28245 +28246 +28247 +28248 +28249 +28250 +28251 +28252 +28253 +28254 +28255 +28256 +28257 +28258 +28259 +28260 +28261 +28262 +28263 +28264 +28265 +28266 +28267 +28268 +28269 +28270 +28271 +28272 +28273 +28274 +28275 +28276 +28277 +28278 +28279 +28280 +28281 +28282 +28283 +28284 +28285 +28286 +28287 +28288 +28289 +28290 +28291 +28292 +28293 +28294 +28295 +28296 +28297 +28298 +28299 +28300 +28301 +28302 +28303 +28304 +28305 +28306 +28307 +28308 +28309 +28310 +28311 +28312 +28313 +28314 +28315 +28316 +28317 +28318 +28319 +28320 +28321 +28322 +28323 +28324 +28325 +28326 +28327 +28328 +28329 +28330 +28331 +28332 +28333 +28334 +28335 +28336 +28337 +28338 +28339 +28340 +28341 +28342 +28343 +28344 +28345 +28346 +28347 +28348 +28349 +28350 +28351 +28352 +28353 +28354 +28355 +28356 +28357 +28358 +28359 +28360 +28361 +28362 +28363 +28364 +28365 +28366 +28367 +28368 +28369 +28370 +28371 +28372 +28373 +28374 +28375 +28376 +28377 +28378 +28379 +28380 +28381 +28382 +28383 +28384 +28385 +28386 +28387 +28388 +28389 +28390 +28391 +28392 +28393 +28394 +28395 +28396 +28397 +28398 +28399 +28400 +28401 +28402 +28403 +28404 +28405 +28406 +28407 +28408 +28409 +28410 +28411 +28412 +28413 +28414 +28415 +28416 +28417 +28418 +28419 +28420 +28421 +28422 +28423 +28424 +28425 +28426 +28427 +28428 +28429 +28430 +28431 +28432 +28433 +28434 +28435 +28436 +28437 +28438 +28439 +28440 +28441 +28442 +28443 +28444 +28445 +28446 +28447 +28448 +28449 +28450 +28451 +28452 +28453 +28454 +28455 +28456 +28457 +28458 +28459 +28460 +28461 +28462 +28463 +28464 +28465 +28466 +28467 +28468 +28469 +28470 +28471 +28472 +28473 +28474 +28475 +28476 +28477 +28478 +28479 +28480 +28481 +28482 +28483 +28484 +28485 +28486 +28487 +28488 +28489 +28490 +28491 +28492 +28493 +28494 +28495 +28496 +28497 +28498 +28499 +28500 +28501 +28502 +28503 +28504 +28505 +28506 +28507 +28508 +28509 +28510 +28511 +28512 +28513 +28514 +28515 +28516 +28517 +28518 +28519 +28520 +28521 +28522 +28523 +28524 +28525 +28526 +28527 +28528 +28529 +28530 +28531 +28532 +28533 +28534 +28535 +28536 +28537 +28538 +28539 +28540 +28541 +28542 +28543 +28544 +28545 +28546 +28547 +28548 +28549 +28550 +28551 +28552 +28553 +28554 +28555 +28556 +28557 +28558 +28559 +28560 +28561 +28562 +28563 +28564 +28565 +28566 +28567 +28568 +28569 +28570 +28571 +28572 +28573 +28574 +28575 +28576 +28577 +28578 +28579 +28580 +28581 +28582 +28583 +28584 +28585 +28586 +28587 +28588 +28589 +28590 +28591 +28592 +28593 +28594 +28595 +28596 +28597 +28598 +28599 +28600 +28601 +28602 +28603 +28604 +28605 +28606 +28607 +28608 +28609 +28610 +28611 +28612 +28613 +28614 +28615 +28616 +28617 +28618 +28619 +28620 +28621 +28622 +28623 +28624 +28625 +28626 +28627 +28628 +28629 +28630 +28631 +28632 +28633 +28634 +28635 +28636 +28637 +28638 +28639 +28640 +28641 +28642 +28643 +28644 +28645 +28646 +28647 +28648 +28649 +28650 +28651 +28652 +28653 +28654 +28655 +28656 +28657 +28658 +28659 +28660 +28661 +28662 +28663 +28664 +28665 +28666 +28667 +28668 +28669 +28670 +28671 +28672 +28673 +28674 +28675 +28676 +28677 +28678 +28679 +28680 +28681 +28682 +28683 +28684 +28685 +28686 +28687 +28688 +28689 +28690 +28691 +28692 +28693 +28694 +28695 +28696 +28697 +28698 +28699 +28700 +28701 +28702 +28703 +28704 +28705 +28706 +28707 +28708 +28709 +28710 +28711 +28712 +28713 +28714 +28715 +28716 +28717 +28718 +28719 +28720 +28721 +28722 +28723 +28724 +28725 +28726 +28727 +28728 +28729 +28730 +28731 +28732 +28733 +28734 +28735 +28736 +28737 +28738 +28739 +28740 +28741 +28742 +28743 +28744 +28745 +28746 +28747 +28748 +28749 +28750 +28751 +28752 +28753 +28754 +28755 +28756 +28757 +28758 +28759 +28760 +28761 +28762 +28763 +28764 +28765 +28766 +28767 +28768 +28769 +28770 +28771 +28772 +28773 +28774 +28775 +28776 +28777 +28778 +28779 +28780 +28781 +28782 +28783 +28784 +28785 +28786 +28787 +28788 +28789 +28790 +28791 +28792 +28793 +28794 +28795 +28796 +28797 +28798 +28799 +28800 +28801 +28802 +28803 +28804 +28805 +28806 +28807 +28808 +28809 +28810 +28811 +28812 +28813 +28814 +28815 +28816 +28817 +28818 +28819 +28820 +28821 +28822 +28823 +28824 +28825 +28826 +28827 +28828 +28829 +28830 +28831 +28832 +28833 +28834 +28835 +28836 +28837 +28838 +28839 +28840 +28841 +28842 +28843 +28844 +28845 +28846 +28847 +28848 +28849 +28850 +28851 +28852 +28853 +28854 +28855 +28856 +28857 +28858 +28859 +28860 +28861 +28862 +28863 +28864 +28865 +28866 +28867 +28868 +28869 +28870 +28871 +28872 +28873 +28874 +28875 +28876 +28877 +28878 +28879 +28880 +28881 +28882 +28883 +28884 +28885 +28886 +28887 +28888 +28889 +28890 +28891 +28892 +28893 +28894 +28895 +28896 +28897 +28898 +28899 +28900 +28901 +28902 +28903 +28904 +28905 +28906 +28907 +28908 +28909 +28910 +28911 +28912 +28913 +28914 +28915 +28916 +28917 +28918 +28919 +28920 +28921 +28922 +28923 +28924 +28925 +28926 +28927 +28928 +28929 +28930 +28931 +28932 +28933 +28934 +28935 +28936 +28937 +28938 +28939 +28940 +28941 +28942 +28943 +28944 +28945 +28946 +28947 +28948 +28949 +28950 +28951 +28952 +28953 +28954 +28955 +28956 +28957 +28958 +28959 +28960 +28961 +28962 +28963 +28964 +28965 +28966 +28967 +28968 +28969 +28970 +28971 +28972 +28973 +28974 +28975 +28976 +28977 +28978 +28979 +28980 +28981 +28982 +28983 +28984 +28985 +28986 +28987 +28988 +28989 +28990 +28991 +28992 +28993 +28994 +28995 +28996 +28997 +28998 +28999 +29000 +29001 +29002 +29003 +29004 +29005 +29006 +29007 +29008 +29009 +29010 +29011 +29012 +29013 +29014 +29015 +29016 +29017 +29018 +29019 +29020 +29021 +29022 +29023 +29024 +29025 +29026 +29027 +29028 +29029 +29030 +29031 +29032 +29033 +29034 +29035 +29036 +29037 +29038 +29039 +29040 +29041 +29042 +29043 +29044 +29045 +29046 +29047 +29048 +29049 +29050 +29051 +29052 +29053 +29054 +29055 +29056 +29057 +29058 +29059 +29060 +29061 +29062 +29063 +29064 +29065 +29066 +29067 +29068 +29069 +29070 +29071 +29072 +29073 +29074 +29075 +29076 +29077 +29078 +29079 +29080 +29081 +29082 +29083 +29084 +29085 +29086 +29087 +29088 +29089 +29090 +29091 +29092 +29093 +29094 +29095 +29096 +29097 +29098 +29099 +29100 +29101 +29102 +29103 +29104 +29105 +29106 +29107 +29108 +29109 +29110 +29111 +29112 +29113 +29114 +29115 +29116 +29117 +29118 +29119 +29120 +29121 +29122 +29123 +29124 +29125 +29126 +29127 +29128 +29129 +29130 +29131 +29132 +29133 +29134 +29135 +29136 +29137 +29138 +29139 +29140 +29141 +29142 +29143 +29144 +29145 +29146 +29147 +29148 +29149 +29150 +29151 +29152 +29153 +29154 +29155 +29156 +29157 +29158 +29159 +29160 +29161 +29162 +29163 +29164 +29165 +29166 +29167 +29168 +29169 +29170 +29171 +29172 +29173 +29174 +29175 +29176 +29177 +29178 +29179 +29180 +29181 +29182 +29183 +29184 +29185 +29186 +29187 +29188 +29189 +29190 +29191 +29192 +29193 +29194 +29195 +29196 +29197 +29198 +29199 +29200 +29201 +29202 +29203 +29204 +29205 +29206 +29207 +29208 +29209 +29210 +29211 +29212 +29213 +29214 +29215 +29216 +29217 +29218 +29219 +29220 +29221 +29222 +29223 +29224 +29225 +29226 +29227 +29228 +29229 +29230 +29231 +29232 +29233 +29234 +29235 +29236 +29237 +29238 +29239 +29240 +29241 +29242 +29243 +29244 +29245 +29246 +29247 +29248 +29249 +29250 +29251 +29252 +29253 +29254 +29255 +29256 +29257 +29258 +29259 +29260 +29261 +29262 +29263 +29264 +29265 +29266 +29267 +29268 +29269 +29270 +29271 +29272 +29273 +29274 +29275 +29276 +29277 +29278 +29279 +29280 +29281 +29282 +29283 +29284 +29285 +29286 +29287 +29288 +29289 +29290 +29291 +29292 +29293 +29294 +29295 +29296 +29297 +29298 +29299 +29300 +29301 +29302 +29303 +29304 +29305 +29306 +29307 +29308 +29309 +29310 +29311 +29312 +29313 +29314 +29315 +29316 +29317 +29318 +29319 +29320 +29321 +29322 +29323 +29324 +29325 +29326 +29327 +29328 +29329 +29330 +29331 +29332 +29333 +29334 +29335 +29336 +29337 +29338 +29339 +29340 +29341 +29342 +29343 +29344 +29345 +29346 +29347 +29348 +29349 +29350 +29351 +29352 +29353 +29354 +29355 +29356 +29357 +29358 +29359 +29360 +29361 +29362 +29363 +29364 +29365 +29366 +29367 +29368 +29369 +29370 +29371 +29372 +29373 +29374 +29375 +29376 +29377 +29378 +29379 +29380 +29381 +29382 +29383 +29384 +29385 +29386 +29387 +29388 +29389 +29390 +29391 +29392 +29393 +29394 +29395 +29396 +29397 +29398 +29399 +29400 +29401 +29402 +29403 +29404 +29405 +29406 +29407 +29408 +29409 +29410 +29411 +29412 +29413 +29414 +29415 +29416 +29417 +29418 +29419 +29420 +29421 +29422 +29423 +29424 +29425 +29426 +29427 +29428 +29429 +29430 +29431 +29432 +29433 +29434 +29435 +29436 +29437 +29438 +29439 +29440 +29441 +29442 +29443 +29444 +29445 +29446 +29447 +29448 +29449 +29450 +29451 +29452 +29453 +29454 +29455 +29456 +29457 +29458 +29459 +29460 +29461 +29462 +29463 +29464 +29465 +29466 +29467 +29468 +29469 +29470 +29471 +29472 +29473 +29474 +29475 +29476 +29477 +29478 +29479 +29480 +29481 +29482 +29483 +29484 +29485 +29486 +29487 +29488 +29489 +29490 +29491 +29492 +29493 +29494 +29495 +29496 +29497 +29498 +29499 +29500 +29501 +29502 +29503 +29504 +29505 +29506 +29507 +29508 +29509 +29510 +29511 +29512 +29513 +29514 +29515 +29516 +29517 +29518 +29519 +29520 +29521 +29522 +29523 +29524 +29525 +29526 +29527 +29528 +29529 +29530 +29531 +29532 +29533 +29534 +29535 +29536 +29537 +29538 +29539 +29540 +29541 +29542 +29543 +29544 +29545 +29546 +29547 +29548 +29549 +29550 +29551 +29552 +29553 +29554 +29555 +29556 +29557 +29558 +29559 +29560 +29561 +29562 +29563 +29564 +29565 +29566 +29567 +29568 +29569 +29570 +29571 +29572 +29573 +29574 +29575 +29576 +29577 +29578 +29579 +29580 +29581 +29582 +29583 +29584 +29585 +29586 +29587 +29588 +29589 +29590 +29591 +29592 +29593 +29594 +29595 +29596 +29597 +29598 +29599 +29600 +29601 +29602 +29603 +29604 +29605 +29606 +29607 +29608 +29609 +29610 +29611 +29612 +29613 +29614 +29615 +29616 +29617 +29618 +29619 +29620 +29621 +29622 +29623 +29624 +29625 +29626 +29627 +29628 +29629 +29630 +29631 +29632 +29633 +29634 +29635 +29636 +29637 +29638 +29639 +29640 +29641 +29642 +29643 +29644 +29645 +29646 +29647 +29648 +29649 +29650 +29651 +29652 +29653 +29654 +29655 +29656 +29657 +29658 +29659 +29660 +29661 +29662 +29663 +29664 +29665 +29666 +29667 +29668 +29669 +29670 +29671 +29672 +29673 +29674 +29675 +29676 +29677 +29678 +29679 +29680 +29681 +29682 +29683 +29684 +29685 +29686 +29687 +29688 +29689 +29690 +29691 +29692 +29693 +29694 +29695 +29696 +29697 +29698 +29699 +29700 +29701 +29702 +29703 +29704 +29705 +29706 +29707 +29708 +29709 +29710 +29711 +29712 +29713 +29714 +29715 +29716 +29717 +29718 +29719 +29720 +29721 +29722 +29723 +29724 +29725 +29726 +29727 +29728 +29729 +29730 +29731 +29732 +29733 +29734 +29735 +29736 +29737 +29738 +29739 +29740 +29741 +29742 +29743 +29744 +29745 +29746 +29747 +29748 +29749 +29750 +29751 +29752 +29753 +29754 +29755 +29756 +29757 +29758 +29759 +29760 +29761 +29762 +29763 +29764 +29765 +29766 +29767 +29768 +29769 +29770 +29771 +29772 +29773 +29774 +29775 +29776 +29777 +29778 +29779 +29780 +29781 +29782 +29783 +29784 +29785 +29786 +29787 +29788 +29789 +29790 +29791 +29792 +29793 +29794 +29795 +29796 +29797 +29798 +29799 +29800 +29801 +29802 +29803 +29804 +29805 +29806 +29807 +29808 +29809 +29810 +29811 +29812 +29813 +29814 +29815 +29816 +29817 +29818 +29819 +29820 +29821 +29822 +29823 +29824 +29825 +29826 +29827 +29828 +29829 +29830 +29831 +29832 +29833 +29834 +29835 +29836 +29837 +29838 +29839 +29840 +29841 +29842 +29843 +29844 +29845 +29846 +29847 +29848 +29849 +29850 +29851 +29852 +29853 +29854 +29855 +29856 +29857 +29858 +29859 +29860 +29861 +29862 +29863 +29864 +29865 +29866 +29867 +29868 +29869 +29870 +29871 +29872 +29873 +29874 +29875 +29876 +29877 +29878 +29879 +29880 +29881 +29882 +29883 +29884 +29885 +29886 +29887 +29888 +29889 +29890 +29891 +29892 +29893 +29894 +29895 +29896 +29897 +29898 +29899 +29900 +29901 +29902 +29903 +29904 +29905 +29906 +29907 +29908 +29909 +29910 +29911 +29912 +29913 +29914 +29915 +29916 +29917 +29918 +29919 +29920 +29921 +29922 +29923 +29924 +29925 +29926 +29927 +29928 +29929 +29930 +29931 +29932 +29933 +29934 +29935 +29936 +29937 +29938 +29939 +29940 +29941 +29942 +29943 +29944 +29945 +29946 +29947 +29948 +29949 +29950 +29951 +29952 +29953 +29954 +29955 +29956 +29957 +29958 +29959 +29960 +29961 +29962 +29963 +29964 +29965 +29966 +29967 +29968 +29969 +29970 +29971 +29972 +29973 +29974 +29975 +29976 +29977 +29978 +29979 +29980 +29981 +29982 +29983 +29984 +29985 +29986 +29987 +29988 +29989 +29990 +29991 +29992 +29993 +29994 +29995 +29996 +29997 +29998 +29999 +30000 +30001 +30002 +30003 +30004 +30005 +30006 +30007 +30008 +30009 +30010 +30011 +30012 +30013 +30014 +30015 +30016 +30017 +30018 +30019 +30020 +30021 +30022 +30023 +30024 +30025 +30026 +30027 +30028 +30029 +30030 +30031 +30032 +30033 +30034 +30035 +30036 +30037 +30038 +30039 +30040 +30041 +30042 +30043 +30044 +30045 +30046 +30047 +30048 +30049 +30050 +30051 +30052 +30053 +30054 +30055 +30056 +30057 +30058 +30059 +30060 +30061 +30062 +30063 +30064 +30065 +30066 +30067 +30068 +30069 +30070 +30071 +30072 +30073 +30074 +30075 +30076 +30077 +30078 +30079 +30080 +30081 +30082 +30083 +30084 +30085 +30086 +30087 +30088 +30089 +30090 +30091 +30092 +30093 +30094 +30095 +30096 +30097 +30098 +30099 +30100 +30101 +30102 +30103 +30104 +30105 +30106 +30107 +30108 +30109 +30110 +30111 +30112 +30113 +30114 +30115 +30116 +30117 +30118 +30119 +30120 +30121 +30122 +30123 +30124 +30125 +30126 +30127 +30128 +30129 +30130 +30131 +30132 +30133 +30134 +30135 +30136 +30137 +30138 +30139 +30140 +30141 +30142 +30143 +30144 +30145 +30146 +30147 +30148 +30149 +30150 +30151 +30152 +30153 +30154 +30155 +30156 +30157 +30158 +30159 +30160 +30161 +30162 +30163 +30164 +30165 +30166 +30167 +30168 +30169 +30170 +30171 +30172 +30173 +30174 +30175 +30176 +30177 +30178 +30179 +30180 +30181 +30182 +30183 +30184 +30185 +30186 +30187 +30188 +30189 +30190 +30191 +30192 +30193 +30194 +30195 +30196 +30197 +30198 +30199 +30200 +30201 +30202 +30203 +30204 +30205 +30206 +30207 +30208 +30209 +30210 +30211 +30212 +30213 +30214 +30215 +30216 +30217 +30218 +30219 +30220 +30221 +30222 +30223 +30224 +30225 +30226 +30227 +30228 +30229 +30230 +30231 +30232 +30233 +30234 +30235 +30236 +30237 +30238 +30239 +30240 +30241 +30242 +30243 +30244 +30245 +30246 +30247 +30248 +30249 +30250 +30251 +30252 +30253 +30254 +30255 +30256 +30257 +30258 +30259 +30260 +30261 +30262 +30263 +30264 +30265 +30266 +30267 +30268 +30269 +30270 +30271 +30272 +30273 +30274 +30275 +30276 +30277 +30278 +30279 +30280 +30281 +30282 +30283 +30284 +30285 +30286 +30287 +30288 +30289 +30290 +30291 +30292 +30293 +30294 +30295 +30296 +30297 +30298 +30299 +30300 +30301 +30302 +30303 +30304 +30305 +30306 +30307 +30308 +30309 +30310 +30311 +30312 +30313 +30314 +30315 +30316 +30317 +30318 +30319 +30320 +30321 +30322 +30323 +30324 +30325 +30326 +30327 +30328 +30329 +30330 +30331 +30332 +30333 +30334 +30335 +30336 +30337 +30338 +30339 +30340 +30341 +30342 +30343 +30344 +30345 +30346 +30347 +30348 +30349 +30350 +30351 +30352 +30353 +30354 +30355 +30356 +30357 +30358 +30359 +30360 +30361 +30362 +30363 +30364 +30365 +30366 +30367 +30368 +30369 +30370 +30371 +30372 +30373 +30374 +30375 +30376 +30377 +30378 +30379 +30380 +30381 +30382 +30383 +30384 +30385 +30386 +30387 +30388 +30389 +30390 +30391 +30392 +30393 +30394 +30395 +30396 +30397 +30398 +30399 +30400 +30401 +30402 +30403 +30404 +30405 +30406 +30407 +30408 +30409 +30410 +30411 +30412 +30413 +30414 +30415 +30416 +30417 +30418 +30419 +30420 +30421 +30422 +30423 +30424 +30425 +30426 +30427 +30428 +30429 +30430 +30431 +30432 +30433 +30434 +30435 +30436 +30437 +30438 +30439 +30440 +30441 +30442 +30443 +30444 +30445 +30446 +30447 +30448 +30449 +30450 +30451 +30452 +30453 +30454 +30455 +30456 +30457 +30458 +30459 +30460 +30461 +30462 +30463 +30464 +30465 +30466 +30467 +30468 +30469 +30470 +30471 +30472 +30473 +30474 +30475 +30476 +30477 +30478 +30479 +30480 +30481 +30482 +30483 +30484 +30485 +30486 +30487 +30488 +30489 +30490 +30491 +30492 +30493 +30494 +30495 +30496 +30497 +30498 +30499 +30500 +30501 +30502 +30503 +30504 +30505 +30506 +30507 +30508 +30509 +30510 +30511 +30512 +30513 +30514 +30515 +30516 +30517 +30518 +30519 +30520 +30521 +30522 +30523 +30524 +30525 +30526 +30527 +30528 +30529 +30530 +30531 +30532 +30533 +30534 +30535 +30536 +30537 +30538 +30539 +30540 +30541 +30542 +30543 +30544 +30545 +30546 +30547 +30548 +30549 +30550 +30551 +30552 +30553 +30554 +30555 +30556 +30557 +30558 +30559 +30560 +30561 +30562 +30563 +30564 +30565 +30566 +30567 +30568 +30569 +30570 +30571 +30572 +30573 +30574 +30575 +30576 +30577 +30578 +30579 +30580 +30581 +30582 +30583 +30584 +30585 +30586 +30587 +30588 +30589 +30590 +30591 +30592 +30593 +30594 +30595 +30596 +30597 +30598 +30599 +30600 +30601 +30602 +30603 +30604 +30605 +30606 +30607 +30608 +30609 +30610 +30611 +30612 +30613 +30614 +30615 +30616 +30617 +30618 +30619 +30620 +30621 +30622 +30623 +30624 +30625 +30626 +30627 +30628 +30629 +30630 +30631 +30632 +30633 +30634 +30635 +30636 +30637 +30638 +30639 +30640 +30641 +30642 +30643 +30644 +30645 +30646 +30647 +30648 +30649 +30650 +30651 +30652 +30653 +30654 +30655 +30656 +30657 +30658 +30659 +30660 +30661 +30662 +30663 +30664 +30665 +30666 +30667 +30668 +30669 +30670 +30671 +30672 +30673 +30674 +30675 +30676 +30677 +30678 +30679 +30680 +30681 +30682 +30683 +30684 +30685 +30686 +30687 +30688 +30689 +30690 +30691 +30692 +30693 +30694 +30695 +30696 +30697 +30698 +30699 +30700 +30701 +30702 +30703 +30704 +30705 +30706 +30707 +30708 +30709 +30710 +30711 +30712 +30713 +30714 +30715 +30716 +30717 +30718 +30719 +30720 +30721 +30722 +30723 +30724 +30725 +30726 +30727 +30728 +30729 +30730 +30731 +30732 +30733 +30734 +30735 +30736 +30737 +30738 +30739 +30740 +30741 +30742 +30743 +30744 +30745 +30746 +30747 +30748 +30749 +30750 +30751 +30752 +30753 +30754 +30755 +30756 +30757 +30758 +30759 +30760 +30761 +30762 +30763 +30764 +30765 +30766 +30767 +30768 +30769 +30770 +30771 +30772 +30773 +30774 +30775 +30776 +30777 +30778 +30779 +30780 +30781 +30782 +30783 +30784 +30785 +30786 +30787 +30788 +30789 +30790 +30791 +30792 +30793 +30794 +30795 +30796 +30797 +30798 +30799 +30800 +30801 +30802 +30803 +30804 +30805 +30806 +30807 +30808 +30809 +30810 +30811 +30812 +30813 +30814 +30815 +30816 +30817 +30818 +30819 +30820 +30821 +30822 +30823 +30824 +30825 +30826 +30827 +30828 +30829 +30830 +30831 +30832 +30833 +30834 +30835 +30836 +30837 +30838 +30839 +30840 +30841 +30842 +30843 +30844 +30845 +30846 +30847 +30848 +30849 +30850 +30851 +30852 +30853 +30854 +30855 +30856 +30857 +30858 +30859 +30860 +30861 +30862 +30863 +30864 +30865 +30866 +30867 +30868 +30869 +30870 +30871 +30872 +30873 +30874 +30875 +30876 +30877 +30878 +30879 +30880 +30881 +30882 +30883 +30884 +30885 +30886 +30887 +30888 +30889 +30890 +30891 +30892 +30893 +30894 +30895 +30896 +30897 +30898 +30899 +30900 +30901 +30902 +30903 +30904 +30905 +30906 +30907 +30908 +30909 +30910 +30911 +30912 +30913 +30914 +30915 +30916 +30917 +30918 +30919 +30920 +30921 +30922 +30923 +30924 +30925 +30926 +30927 +30928 +30929 +30930 +30931 +30932 +30933 +30934 +30935 +30936 +30937 +30938 +30939 +30940 +30941 +30942 +30943 +30944 +30945 +30946 +30947 +30948 +30949 +30950 +30951 +30952 +30953 +30954 +30955 +30956 +30957 +30958 +30959 +30960 +30961 +30962 +30963 +30964 +30965 +30966 +30967 +30968 +30969 +30970 +30971 +30972 +30973 +30974 +30975 +30976 +30977 +30978 +30979 +30980 +30981 +30982 +30983 +30984 +30985 +30986 +30987 +30988 +30989 +30990 +30991 +30992 +30993 +30994 +30995 +30996 +30997 +30998 +30999 +31000 +31001 +31002 +31003 +31004 +31005 +31006 +31007 +31008 +31009 +31010 +31011 +31012 +31013 +31014 +31015 +31016 +31017 +31018 +31019 +31020 +31021 +31022 +31023 +31024 +31025 +31026 +31027 +31028 +31029 +31030 +31031 +31032 +31033 +31034 +31035 +31036 +31037 +31038 +31039 +31040 +31041 +31042 +31043 +31044 +31045 +31046 +31047 +31048 +31049 +31050 +31051 +31052 +31053 +31054 +31055 +31056 +31057 +31058 +31059 +31060 +31061 +31062 +31063 +31064 +31065 +31066 +31067 +31068 +31069 +31070 +31071 +31072 +31073 +31074 +31075 +31076 +31077 +31078 +31079 +31080 +31081 +31082 +31083 +31084 +31085 +31086 +31087 +31088 +31089 +31090 +31091 +31092 +31093 +31094 +31095 +31096 +31097 +31098 +31099 +31100 +31101 +31102 +31103 +31104 +31105 +31106 +31107 +31108 +31109 +31110 +31111 +31112 +31113 +31114 +31115 +31116 +31117 +31118 +31119 +31120 +31121 +31122 +31123 +31124 +31125 +31126 +31127 +31128 +31129 +31130 +31131 +31132 +31133 +31134 +31135 +31136 +31137 +31138 +31139 +31140 +31141 +31142 +31143 +31144 +31145 +31146 +31147 +31148 +31149 +31150 +31151 +31152 +31153 +31154 +31155 +31156 +31157 +31158 +31159 +31160 +31161 +31162 +31163 +31164 +31165 +31166 +31167 +31168 +31169 +31170 +31171 +31172 +31173 +31174 +31175 +31176 +31177 +31178 +31179 +31180 +31181 +31182 +31183 +31184 +31185 +31186 +31187 +31188 +31189 +31190 +31191 +31192 +31193 +31194 +31195 +31196 +31197 +31198 +31199 +31200 +31201 +31202 +31203 +31204 +31205 +31206 +31207 +31208 +31209 +31210 +31211 +31212 +31213 +31214 +31215 +31216 +31217 +31218 +31219 +31220 +31221 +31222 +31223 +31224 +31225 +31226 +31227 +31228 +31229 +31230 +31231 +31232 +31233 +31234 +31235 +31236 +31237 +31238 +31239 +31240 +31241 +31242 +31243 +31244 +31245 +31246 +31247 +31248 +31249 +31250 +31251 +31252 +31253 +31254 +31255 +31256 +31257 +31258 +31259 +31260 +31261 +31262 +31263 +31264 +31265 +31266 +31267 +31268 +31269 +31270 +31271 +31272 +31273 +31274 +31275 +31276 +31277 +31278 +31279 +31280 +31281 +31282 +31283 +31284 +31285 +31286 +31287 +31288 +31289 +31290 +31291 +31292 +31293 +31294 +31295 +31296 +31297 +31298 +31299 +31300 +31301 +31302 +31303 +31304 +31305 +31306 +31307 +31308 +31309 +31310 +31311 +31312 +31313 +31314 +31315 +31316 +31317 +31318 +31319 +31320 +31321 +31322 +31323 +31324 +31325 +31326 +31327 +31328 +31329 +31330 +31331 +31332 +31333 +31334 +31335 +31336 +31337 +31338 +31339 +31340 +31341 +31342 +31343 +31344 +31345 +31346 +31347 +31348 +31349 +31350 +31351 +31352 +31353 +31354 +31355 +31356 +31357 +31358 +31359 +31360 +31361 +31362 +31363 +31364 +31365 +31366 +31367 +31368 +31369 +31370 +31371 +31372 +31373 +31374 +31375 +31376 +31377 +31378 +31379 +31380 +31381 +31382 +31383 +31384 +31385 +31386 +31387 +31388 +31389 +31390 +31391 +31392 +31393 +31394 +31395 +31396 +31397 +31398 +31399 +31400 +31401 +31402 +31403 +31404 +31405 +31406 +31407 +31408 +31409 +31410 +31411 +31412 +31413 +31414 +31415 +31416 +31417 +31418 +31419 +31420 +31421 +31422 +31423 +31424 +31425 +31426 +31427 +31428 +31429 +31430 +31431 +31432 +31433 +31434 +31435 +31436 +31437 +31438 +31439 +31440 +31441 +31442 +31443 +31444 +31445 +31446 +31447 +31448 +31449 +31450 +31451 +31452 +31453 +31454 +31455 +31456 +31457 +31458 +31459 +31460 +31461 +31462 +31463 +31464 +31465 +31466 +31467 +31468 +31469 +31470 +31471 +31472 +31473 +31474 +31475 +31476 +31477 +31478 +31479 +31480 +31481 +31482 +31483 +31484 +31485 +31486 +31487 +31488 +31489 +31490 +31491 +31492 +31493 +31494 +31495 +31496 +31497 +31498 +31499 +31500 +31501 +31502 +31503 +31504 +31505 +31506 +31507 +31508 +31509 +31510 +31511 +31512 +31513 +31514 +31515 +31516 +31517 +31518 +31519 +31520 +31521 +31522 +31523 +31524 +31525 +31526 +31527 +31528 +31529 +31530 +31531 +31532 +31533 +31534 +31535 +31536 +31537 +31538 +31539 +31540 +31541 +31542 +31543 +31544 +31545 +31546 +31547 +31548 +31549 +31550 +31551 +31552 +31553 +31554 +31555 +31556 +31557 +31558 +31559 +31560 +31561 +31562 +31563 +31564 +31565 +31566 +31567 +31568 +31569 +31570 +31571 +31572 +31573 +31574 +31575 +31576 +31577 +31578 +31579 +31580 +31581 +31582 +31583 +31584 +31585 +31586 +31587 +31588 +31589 +31590 +31591 +31592 +31593 +31594 +31595 +31596 +31597 +31598 +31599 +31600 +31601 +31602 +31603 +31604 +31605 +31606 +31607 +31608 +31609 +31610 +31611 +31612 +31613 +31614 +31615 +31616 +31617 +31618 +31619 +31620 +31621 +31622 +31623 +31624 +31625 +31626 +31627 +31628 +31629 +31630 +31631 +31632 +31633 +31634 +31635 +31636 +31637 +31638 +31639 +31640 +31641 +31642 +31643 +31644 +31645 +31646 +31647 +31648 +31649 +31650 +31651 +31652 +31653 +31654 +31655 +31656 +31657 +31658 +31659 +31660 +31661 +31662 +31663 +31664 +31665 +31666 +31667 +31668 +31669 +31670 +31671 +31672 +31673 +31674 +31675 +31676 +31677 +31678 +31679 +31680 +31681 +31682 +31683 +31684 +31685 +31686 +31687 +31688 +31689 +31690 +31691 +31692 +31693 +31694 +31695 +31696 +31697 +31698 +31699 +31700 +31701 +31702 +31703 +31704 +31705 +31706 +31707 +31708 +31709 +31710 +31711 +31712 +31713 +31714 +31715 +31716 +31717 +31718 +31719 +31720 +31721 +31722 +31723 +31724 +31725 +31726 +31727 +31728 +31729 +31730 +31731 +31732 +31733 +31734 +31735 +31736 +31737 +31738 +31739 +31740 +31741 +31742 +31743 +31744 +31745 +31746 +31747 +31748 +31749 +31750 +31751 +31752 +31753 +31754 +31755 +31756 +31757 +31758 +31759 +31760 +31761 +31762 +31763 +31764 +31765 +31766 +31767 +31768 +31769 +31770 +31771 +31772 +31773 +31774 +31775 +31776 +31777 +31778 +31779 +31780 +31781 +31782 +31783 +31784 +31785 +31786 +31787 +31788 +31789 +31790 +31791 +31792 +31793 +31794 +31795 +31796 +31797 +31798 +31799 +31800 +31801 +31802 +31803 +31804 +31805 +31806 +31807 +31808 +31809 +31810 +31811 +31812 +31813 +31814 +31815 +31816 +31817 +31818 +31819 +31820 +31821 +31822 +31823 +31824 +31825 +31826 +31827 +31828 +31829 +31830 +31831 +31832 +31833 +31834 +31835 +31836 +31837 +31838 +31839 +31840 +31841 +31842 +31843 +31844 +31845 +31846 +31847 +31848 +31849 +31850 +31851 +31852 +31853 +31854 +31855 +31856 +31857 +31858 +31859 +31860 +31861 +31862 +31863 +31864 +31865 +31866 +31867 +31868 +31869 +31870 +31871 +31872 +31873 +31874 +31875 +31876 +31877 +31878 +31879 +31880 +31881 +31882 +31883 +31884 +31885 +31886 +31887 +31888 +31889 +31890 +31891 +31892 +31893 +31894 +31895 +31896 +31897 +31898 +31899 +31900 +31901 +31902 +31903 +31904 +31905 +31906 +31907 +31908 +31909 +31910 +31911 +31912 +31913 +31914 +31915 +31916 +31917 +31918 +31919 +31920 +31921 +31922 +31923 +31924 +31925 +31926 +31927 +31928 +31929 +31930 +31931 +31932 +31933 +31934 +31935 +31936 +31937 +31938 +31939 +31940 +31941 +31942 +31943 +31944 +31945 +31946 +31947 +31948 +31949 +31950 +31951 +31952 +31953 +31954 +31955 +31956 +31957 +31958 +31959 +31960 +31961 +31962 +31963 +31964 +31965 +31966 +31967 +31968 +31969 +31970 +31971 +31972 +31973 +31974 +31975 +31976 +31977 +31978 +31979 +31980 +31981 +31982 +31983 +31984 +31985 +31986 +31987 +31988 +31989 +31990 +31991 +31992 +31993 +31994 +31995 +31996 +31997 +31998 +31999 +32000 +32001 +32002 +32003 +32004 +32005 +32006 +32007 +32008 +32009 +32010 +32011 +32012 +32013 +32014 +32015 +32016 +32017 +32018 +32019 +32020 +32021 +32022 +32023 +32024 +32025 +32026 +32027 +32028 +32029 +32030 +32031 +32032 +32033 +32034 +32035 +32036 +32037 +32038 +32039 +32040 +32041 +32042 +32043 +32044 +32045 +32046 +32047 +32048 +32049 +32050 +32051 +32052 +32053 +32054 +32055 +32056 +32057 +32058 +32059 +32060 +32061 +32062 +32063 +32064 +32065 +32066 +32067 +32068 +32069 +32070 +32071 +32072 +32073 +32074 +32075 +32076 +32077 +32078 +32079 +32080 +32081 +32082 +32083 +32084 +32085 +32086 +32087 +32088 +32089 +32090 +32091 +32092 +32093 +32094 +32095 +32096 +32097 +32098 +32099 +32100 +32101 +32102 +32103 +32104 +32105 +32106 +32107 +32108 +32109 +32110 +32111 +32112 +32113 +32114 +32115 +32116 +32117 +32118 +32119 +32120 +32121 +32122 +32123 +32124 +32125 +32126 +32127 +32128 +32129 +32130 +32131 +32132 +32133 +32134 +32135 +32136 +32137 +32138 +32139 +32140 +32141 +32142 +32143 +32144 +32145 +32146 +32147 +32148 +32149 +32150 +32151 +32152 +32153 +32154 +32155 +32156 +32157 +32158 +32159 +32160 +32161 +32162 +32163 +32164 +32165 +32166 +32167 +32168 +32169 +32170 +32171 +32172 +32173 +32174 +32175 +32176 +32177 +32178 +32179 +32180 +32181 +32182 +32183 +32184 +32185 +32186 +32187 +32188 +32189 +32190 +32191 +32192 +32193 +32194 +32195 +32196 +32197 +32198 +32199 +32200 +32201 +32202 +32203 +32204 +32205 +32206 +32207 +32208 +32209 +32210 +32211 +32212 +32213 +32214 +32215 +32216 +32217 +32218 +32219 +32220 +32221 +32222 +32223 +32224 +32225 +32226 +32227 +32228 +32229 +32230 +32231 +32232 +32233 +32234 +32235 +32236 +32237 +32238 +32239 +32240 +32241 +32242 +32243 +32244 +32245 +32246 +32247 +32248 +32249 +32250 +32251 +32252 +32253 +32254 +32255 +32256 +32257 +32258 +32259 +32260 +32261 +32262 +32263 +32264 +32265 +32266 +32267 +32268 +32269 +32270 +32271 +32272 +32273 +32274 +32275 +32276 +32277 +32278 +32279 +32280 +32281 +32282 +32283 +32284 +32285 +32286 +32287 +32288 +32289 +32290 +32291 +32292 +32293 +32294 +32295 +32296 +32297 +32298 +32299 +32300 +32301 +32302 +32303 +32304 +32305 +32306 +32307 +32308 +32309 +32310 +32311 +32312 +32313 +32314 +32315 +32316 +32317 +32318 +32319 +32320 +32321 +32322 +32323 +32324 +32325 +32326 +32327 +32328 +32329 +32330 +32331 +32332 +32333 +32334 +32335 +32336 +32337 +32338 +32339 +32340 +32341 +32342 +32343 +32344 +32345 +32346 +32347 +32348 +32349 +32350 +32351 +32352 +32353 +32354 +32355 +32356 +32357 +32358 +32359 +32360 +32361 +32362 +32363 +32364 +32365 +32366 +32367 +32368 +32369 +32370 +32371 +32372 +32373 +32374 +32375 +32376 +32377 +32378 +32379 +32380 +32381 +32382 +32383 +32384 +32385 +32386 +32387 +32388 +32389 +32390 +32391 +32392 +32393 +32394 +32395 +32396 +32397 +32398 +32399 +32400 +32401 +32402 +32403 +32404 +32405 +32406 +32407 +32408 +32409 +32410 +32411 +32412 +32413 +32414 +32415 +32416 +32417 +32418 +32419 +32420 +32421 +32422 +32423 +32424 +32425 +32426 +32427 +32428 +32429 +32430 +32431 +32432 +32433 +32434 +32435 +32436 +32437 +32438 +32439 +32440 +32441 +32442 +32443 +32444 +32445 +32446 +32447 +32448 +32449 +32450 +32451 +32452 +32453 +32454 +32455 +32456 +32457 +32458 +32459 +32460 +32461 +32462 +32463 +32464 +32465 +32466 +32467 +32468 +32469 +32470 +32471 +32472 +32473 +32474 +32475 +32476 +32477 +32478 +32479 +32480 +32481 +32482 +32483 +32484 +32485 +32486 +32487 +32488 +32489 +32490 +32491 +32492 +32493 +32494 +32495 +32496 +32497 +32498 +32499 +32500 +32501 +32502 +32503 +32504 +32505 +32506 +32507 +32508 +32509 +32510 +32511 +32512 +32513 +32514 +32515 +32516 +32517 +32518 +32519 +32520 +32521 +32522 +32523 +32524 +32525 +32526 +32527 +32528 +32529 +32530 +32531 +32532 +32533 +32534 +32535 +32536 +32537 +32538 +32539 +32540 +32541 +32542 +32543 +32544 +32545 +32546 +32547 +32548 +32549 +32550 +32551 +32552 +32553 +32554 +32555 +32556 +32557 +32558 +32559 +32560 +32561 +32562 +32563 +32564 +32565 +32566 +32567 +32568 +32569 +32570 +32571 +32572 +32573 +32574 +32575 +32576 +32577 +32578 +32579 +32580 +32581 +32582 +32583 +32584 +32585 +32586 +32587 +32588 +32589 +32590 +32591 +32592 +32593 +32594 +32595 +32596 +32597 +32598 +32599 +32600 +32601 +32602 +32603 +32604 +32605 +32606 +32607 +32608 +32609 +32610 +32611 +32612 +32613 +32614 +32615 +32616 +32617 +32618 +32619 +32620 +32621 +32622 +32623 +32624 +32625 +32626 +32627 +32628 +32629 +32630 +32631 +32632 +32633 +32634 +32635 +32636 +32637 +32638 +32639 +32640 +32641 +32642 +32643 +32644 +32645 +32646 +32647 +32648 +32649 +32650 +32651 +32652 +32653 +32654 +32655 +32656 +32657 +32658 +32659 +32660 +32661 +32662 +32663 +32664 +32665 +32666 +32667 +32668 +32669 +32670 +32671 +32672 +32673 +32674 +32675 +32676 +32677 +32678 +32679 +32680 +32681 +32682 +32683 +32684 +32685 +32686 +32687 +32688 +32689 +32690 +32691 +32692 +32693 +32694 +32695 +32696 +32697 +32698 +32699 +32700 +32701 +32702 +32703 +32704 +32705 +32706 +32707 +32708 +32709 +32710 +32711 +32712 +32713 +32714 +32715 +32716 +32717 +32718 +32719 +32720 +32721 +32722 +32723 +32724 +32725 +32726 +32727 +32728 +32729 +32730 +32731 +32732 +32733 +32734 +32735 +32736 +32737 +32738 +32739 +32740 +32741 +32742 +32743 +32744 +32745 +32746 +32747 +32748 +32749 +32750 +32751 +32752 +32753 +32754 +32755 +32756 +32757 +32758 +32759 +32760 +32761 +32762 +32763 +32764 +32765 +32766 +32767 +32768 +32769 +32770 +32771 +32772 +32773 +32774 +32775 +32776 +32777 +32778 +32779 +32780 +32781 +32782 +32783 +32784 +32785 +32786 +32787 +32788 +32789 +32790 +32791 +32792 +32793 +32794 +32795 +32796 +32797 +32798 +32799 +32800 +32801 +32802 +32803 +32804 +32805 +32806 +32807 +32808 +32809 +32810 +32811 +32812 +32813 +32814 +32815 +32816 +32817 +32818 +32819 +32820 +32821 +32822 +32823 +32824 +32825 +32826 +32827 +32828 +32829 +32830 +32831 +32832 +32833 +32834 +32835 +32836 +32837 +32838 +32839 +32840 +32841 +32842 +32843 +32844 +32845 +32846 +32847 +32848 +32849 +32850 +32851 +32852 +32853 +32854 +32855 +32856 +32857 +32858 +32859 +32860 +32861 +32862 +32863 +32864 +32865 +32866 +32867 +32868 +32869 +32870 +32871 +32872 +32873 +32874 +32875 +32876 +32877 +32878 +32879 +32880 +32881 +32882 +32883 +32884 +32885 +32886 +32887 +32888 +32889 +32890 +32891 +32892 +32893 +32894 +32895 +32896 +32897 +32898 +32899 +32900 +32901 +32902 +32903 +32904 +32905 +32906 +32907 +32908 +32909 +32910 +32911 +32912 +32913 +32914 +32915 +32916 +32917 +32918 +32919 +32920 +32921 +32922 +32923 +32924 +32925 +32926 +32927 +32928 +32929 +32930 +32931 +32932 +32933 +32934 +32935 +32936 +32937 +32938 +32939 +32940 +32941 +32942 +32943 +32944 +32945 +32946 +32947 +32948 +32949 +32950 +32951 +32952 +32953 +32954 +32955 +32956 +32957 +32958 +32959 +32960 +32961 +32962 +32963 +32964 +32965 +32966 +32967 +32968 +32969 +32970 +32971 +32972 +32973 +32974 +32975 +32976 +32977 +32978 +32979 +32980 +32981 +32982 +32983 +32984 +32985 +32986 +32987 +32988 +32989 +32990 +32991 +32992 +32993 +32994 +32995 +32996 +32997 +32998 +32999 +33000 +33001 +33002 +33003 +33004 +33005 +33006 +33007 +33008 +33009 +33010 +33011 +33012 +33013 +33014 +33015 +33016 +33017 +33018 +33019 +33020 +33021 +33022 +33023 +33024 +33025 +33026 +33027 +33028 +33029 +33030 +33031 +33032 +33033 +33034 +33035 +33036 +33037 +33038 +33039 +33040 +33041 +33042 +33043 +33044 +33045 +33046 +33047 +33048 +33049 +33050 +33051 +33052 +33053 +33054 +33055 +33056 +33057 +33058 +33059 +33060 +33061 +33062 +33063 +33064 +33065 +33066 +33067 +33068 +33069 +33070 +33071 +33072 +33073 +33074 +33075 +33076 +33077 +33078 +33079 +33080 +33081 +33082 +33083 +33084 +33085 +33086 +33087 +33088 +33089 +33090 +33091 +33092 +33093 +33094 +33095 +33096 +33097 +33098 +33099 +33100 +33101 +33102 +33103 +33104 +33105 +33106 +33107 +33108 +33109 +33110 +33111 +33112 +33113 +33114 +33115 +33116 +33117 +33118 +33119 +33120 +33121 +33122 +33123 +33124 +33125 +33126 +33127 +33128 +33129 +33130 +33131 +33132 +33133 +33134 +33135 +33136 +33137 +33138 +33139 +33140 +33141 +33142 +33143 +33144 +33145 +33146 +33147 +33148 +33149 +33150 +33151 +33152 +33153 +33154 +33155 +33156 +33157 +33158 +33159 +33160 +33161 +33162 +33163 +33164 +33165 +33166 +33167 +33168 +33169 +33170 +33171 +33172 +33173 +33174 +33175 +33176 +33177 +33178 +33179 +33180 +33181 +33182 +33183 +33184 +33185 +33186 +33187 +33188 +33189 +33190 +33191 +33192 +33193 +33194 +33195 +33196 +33197 +33198 +33199 +33200 +33201 +33202 +33203 +33204 +33205 +33206 +33207 +33208 +33209 +33210 +33211 +33212 +33213 +33214 +33215 +33216 +33217 +33218 +33219 +33220 +33221 +33222 +33223 +33224 +33225 +33226 +33227 +33228 +33229 +33230 +33231 +33232 +33233 +33234 +33235 +33236 +33237 +33238 +33239 +33240 +33241 +33242 +33243 +33244 +33245 +33246 +33247 +33248 +33249 +33250 +33251 +33252 +33253 +33254 +33255 +33256 +33257 +33258 +33259 +33260 +33261 +33262 +33263 +33264 +33265 +33266 +33267 +33268 +33269 +33270 +33271 +33272 +33273 +33274 +33275 +33276 +33277 +33278 +33279 +33280 +33281 +33282 +33283 +33284 +33285 +33286 +33287 +33288 +33289 +33290 +33291 +33292 +33293 +33294 +33295 +33296 +33297 +33298 +33299 +33300 +33301 +33302 +33303 +33304 +33305 +33306 +33307 +33308 +33309 +33310 +33311 +33312 +33313 +33314 +33315 +33316 +33317 +33318 +33319 +33320 +33321 +33322 +33323 +33324 +33325 +33326 +33327 +33328 +33329 +33330 +33331 +33332 +33333 +33334 +33335 +33336 +33337 +33338 +33339 +33340 +33341 +33342 +33343 +33344 +33345 +33346 +33347 +33348 +33349 +33350 +33351 +33352 +33353 +33354 +33355 +33356 +33357 +33358 +33359 +33360 +33361 +33362 +33363 +33364 +33365 +33366 +33367 +33368 +33369 +33370 +33371 +33372 +33373 +33374 +33375 +33376 +33377 +33378 +33379 +33380 +33381 +33382 +33383 +33384 +33385 +33386 +33387 +33388 +33389 +33390 +33391 +33392 +33393 +33394 +33395 +33396 +33397 +33398 +33399 +33400 +33401 +33402 +33403 +33404 +33405 +33406 +33407 +33408 +33409 +33410 +33411 +33412 +33413 +33414 +33415 +33416 +33417 +33418 +33419 +33420 +33421 +33422 +33423 +33424 +33425 +33426 +33427 +33428 +33429 +33430 +33431 +33432 +33433 +33434 +33435 +33436 +33437 +33438 +33439 +33440 +33441 +33442 +33443 +33444 +33445 +33446 +33447 +33448 +33449 +33450 +33451 +33452 +33453 +33454 +33455 +33456 +33457 +33458 +33459 +33460 +33461 +33462 +33463 +33464 +33465 +33466 +33467 +33468 +33469 +33470 +33471 +33472 +33473 +33474 +33475 +33476 +33477 +33478 +33479 +33480 +33481 +33482 +33483 +33484 +33485 +33486 +33487 +33488 +33489 +33490 +33491 +33492 +33493 +33494 +33495 +33496 +33497 +33498 +33499 +33500 +33501 +33502 +33503 +33504 +33505 +33506 +33507 +33508 +33509 +33510 +33511 +33512 +33513 +33514 +33515 +33516 +33517 +33518 +33519 +33520 +33521 +33522 +33523 +33524 +33525 +33526 +33527 +33528 +33529 +33530 +33531 +33532 +33533 +33534 +33535 +33536 +33537 +33538 +33539 +33540 +33541 +33542 +33543 +33544 +33545 +33546 +33547 +33548 +33549 +33550 +33551 +33552 +33553 +33554 +33555 +33556 +33557 +33558 +33559 +33560 +33561 +33562 +33563 +33564 +33565 +33566 +33567 +33568 +33569 +33570 +33571 +33572 +33573 +33574 +33575 +33576 +33577 +33578 +33579 +33580 +33581 +33582 +33583 +33584 +33585 +33586 +33587 +33588 +33589 +33590 +33591 +33592 +33593 +33594 +33595 +33596 +33597 +33598 +33599 +33600 +33601 +33602 +33603 +33604 +33605 +33606 +33607 +33608 +33609 +33610 +33611 +33612 +33613 +33614 +33615 +33616 +33617 +33618 +33619 +33620 +33621 +33622 +33623 +33624 +33625 +33626 +33627 +33628 +33629 +33630 +33631 +33632 +33633 +33634 +33635 +33636 +33637 +33638 +33639 +33640 +33641 +33642 +33643 +33644 +33645 +33646 +33647 +33648 +33649 +33650 +33651 +33652 +33653 +33654 +33655 +33656 +33657 +33658 +33659 +33660 +33661 +33662 +33663 +33664 +33665 +33666 +33667 +33668 +33669 +33670 +33671 +33672 +33673 +33674 +33675 +33676 +33677 +33678 +33679 +33680 +33681 +33682 +33683 +33684 +33685 +33686 +33687 +33688 +33689 +33690 +33691 +33692 +33693 +33694 +33695 +33696 +33697 +33698 +33699 +33700 +33701 +33702 +33703 +33704 +33705 +33706 +33707 +33708 +33709 +33710 +33711 +33712 +33713 +33714 +33715 +33716 +33717 +33718 +33719 +33720 +33721 +33722 +33723 +33724 +33725 +33726 +33727 +33728 +33729 +33730 +33731 +33732 +33733 +33734 +33735 +33736 +33737 +33738 +33739 +33740 +33741 +33742 +33743 +33744 +33745 +33746 +33747 +33748 +33749 +33750 +33751 +33752 +33753 +33754 +33755 +33756 +33757 +33758 +33759 +33760 +33761 +33762 +33763 +33764 +33765 +33766 +33767 +33768 +33769 +33770 +33771 +33772 +33773 +33774 +33775 +33776 +33777 +33778 +33779 +33780 +33781 +33782 +33783 +33784 +33785 +33786 +33787 +33788 +33789 +33790 +33791 +33792 +33793 +33794 +33795 +33796 +33797 +33798 +33799 +33800 +33801 +33802 +33803 +33804 +33805 +33806 +33807 +33808 +33809 +33810 +33811 +33812 +33813 +33814 +33815 +33816 +33817 +33818 +33819 +33820 +33821 +33822 +33823 +33824 +33825 +33826 +33827 +33828 +33829 +33830 +33831 +33832 +33833 +33834 +33835 +33836 +33837 +33838 +33839 +33840 +33841 +33842 +33843 +33844 +33845 +33846 +33847 +33848 +33849 +33850 +33851 +33852 +33853 +33854 +33855 +33856 +33857 +33858 +33859 +33860 +33861 +33862 +33863 +33864 +33865 +33866 +33867 +33868 +33869 +33870 +33871 +33872 +33873 +33874 +33875 +33876 +33877 +33878 +33879 +33880 +33881 +33882 +33883 +33884 +33885 +33886 +33887 +33888 +33889 +33890 +33891 +33892 +33893 +33894 +33895 +33896 +33897 +33898 +33899 +33900 +33901 +33902 +33903 +33904 +33905 +33906 +33907 +33908 +33909 +33910 +33911 +33912 +33913 +33914 +33915 +33916 +33917 +33918 +33919 +33920 +33921 +33922 +33923 +33924 +33925 +33926 +33927 +33928 +33929 +33930 +33931 +33932 +33933 +33934 +33935 +33936 +33937 +33938 +33939 +33940 +33941 +33942 +33943 +33944 +33945 +33946 +33947 +33948 +33949 +33950 +33951 +33952 +33953 +33954 +33955 +33956 +33957 +33958 +33959 +33960 +33961 +33962 +33963 +33964 +33965 +33966 +33967 +33968 +33969 +33970 +33971 +33972 +33973 +33974 +33975 +33976 +33977 +33978 +33979 +33980 +33981 +33982 +33983 +33984 +33985 +33986 +33987 +33988 +33989 +33990 +33991 +33992 +33993 +33994 +33995 +33996 +33997 +33998 +33999 +34000 +34001 +34002 +34003 +34004 +34005 +34006 +34007 +34008 +34009 +34010 +34011 +34012 +34013 +34014 +34015 +34016 +34017 +34018 +34019 +34020 +34021 +34022 +34023 +34024 +34025 +34026 +34027 +34028 +34029 +34030 +34031 +34032 +34033 +34034 +34035 +34036 +34037 +34038 +34039 +34040 +34041 +34042 +34043 +34044 +34045 +34046 +34047 +34048 +34049 +34050 +34051 +34052 +34053 +34054 +34055 +34056 +34057 +34058 +34059 +34060 +34061 +34062 +34063 +34064 +34065 +34066 +34067 +34068 +34069 +34070 +34071 +34072 +34073 +34074 +34075 +34076 +34077 +34078 +34079 +34080 +34081 +34082 +34083 +34084 +34085 +34086 +34087 +34088 +34089 +34090 +34091 +34092 +34093 +34094 +34095 +34096 +34097 +34098 +34099 +34100 +34101 +34102 +34103 +34104 +34105 +34106 +34107 +34108 +34109 +34110 +34111 +34112 +34113 +34114 +34115 +34116 +34117 +34118 +34119 +34120 +34121 +34122 +34123 +34124 +34125 +34126 +34127 +34128 +34129 +34130 +34131 +34132 +34133 +34134 +34135 +34136 +34137 +34138 +34139 +34140 +34141 +34142 +34143 +34144 +34145 +34146 +34147 +34148 +34149 +34150 +34151 +34152 +34153 +34154 +34155 +34156 +34157 +34158 +34159 +34160 +34161 +34162 +34163 +34164 +34165 +34166 +34167 +34168 +34169 +34170 +34171 +34172 +34173 +34174 +34175 +34176 +34177 +34178 +34179 +34180 +34181 +34182 +34183 +34184 +34185 +34186 +34187 +34188 +34189 +34190 +34191 +34192 +34193 +34194 +34195 +34196 +34197 +34198 +34199 +34200 +34201 +34202 +34203 +34204 +34205 +34206 +34207 +34208 +34209 +34210 +34211 +34212 +34213 +34214 +34215 +34216 +34217 +34218 +34219 +34220 +34221 +34222 +34223 +34224 +34225 +34226 +34227 +34228 +34229 +34230 +34231 +34232 +34233 +34234 +34235 +34236 +34237 +34238 +34239 +34240 +34241 +34242 +34243 +34244 +34245 +34246 +34247 +34248 +34249 +34250 +34251 +34252 +34253 +34254 +34255 +34256 +34257 +34258 +34259 +34260 +34261 +34262 +34263 +34264 +34265 +34266 +34267 +34268 +34269 +34270 +34271 +34272 +34273 +34274 +34275 +34276 +34277 +34278 +34279 +34280 +34281 +34282 +34283 +34284 +34285 +34286 +34287 +34288 +34289 +34290 +34291 +34292 +34293 +34294 +34295 +34296 +34297 +34298 +34299 +34300 +34301 +34302 +34303 +34304 +34305 +34306 +34307 +34308 +34309 +34310 +34311 +34312 +34313 +34314 +34315 +34316 +34317 +34318 +34319 +34320 +34321 +34322 +34323 +34324 +34325 +34326 +34327 +34328 +34329 +34330 +34331 +34332 +34333 +34334 +34335 +34336 +34337 +34338 +34339 +34340 +34341 +34342 +34343 +34344 +34345 +34346 +34347 +34348 +34349 +34350 +34351 +34352 +34353 +34354 +34355 +34356 +34357 +34358 +34359 +34360 +34361 +34362 +34363 +34364 +34365 +34366 +34367 +34368 +34369 +34370 +34371 +34372 +34373 +34374 +34375 +34376 +34377 +34378 +34379 +34380 +34381 +34382 +34383 +34384 +34385 +34386 +34387 +34388 +34389 +34390 +34391 +34392 +34393 +34394 +34395 +34396 +34397 +34398 +34399 +34400 +34401 +34402 +34403 +34404 +34405 +34406 +34407 +34408 +34409 +34410 +34411 +34412 +34413 +34414 +34415 +34416 +34417 +34418 +34419 +34420 +34421 +34422 +34423 +34424 +34425 +34426 +34427 +34428 +34429 +34430 +34431 +34432 +34433 +34434 +34435 +34436 +34437 +34438 +34439 +34440 +34441 +34442 +34443 +34444 +34445 +34446 +34447 +34448 +34449 +34450 +34451 +34452 +34453 +34454 +34455 +34456 +34457 +34458 +34459 +34460 +34461 +34462 +34463 +34464 +34465 +34466 +34467 +34468 +34469 +34470 +34471 +34472 +34473 +34474 +34475 +34476 +34477 +34478 +34479 +34480 +34481 +34482 +34483 +34484 +34485 +34486 +34487 +34488 +34489 +34490 +34491 +34492 +34493 +34494 +34495 +34496 +34497 +34498 +34499 +34500 +34501 +34502 +34503 +34504 +34505 +34506 +34507 +34508 +34509 +34510 +34511 +34512 +34513 +34514 +34515 +34516 +34517 +34518 +34519 +34520 +34521 +34522 +34523 +34524 +34525 +34526 +34527 +34528 +34529 +34530 +34531 +34532 +34533 +34534 +34535 +34536 +34537 +34538 +34539 +34540 +34541 +34542 +34543 +34544 +34545 +34546 +34547 +34548 +34549 +34550 +34551 +34552 +34553 +34554 +34555 +34556 +34557 +34558 +34559 +34560 +34561 +34562 +34563 +34564 +34565 +34566 +34567 +34568 +34569 +34570 +34571 +34572 +34573 +34574 +34575 +34576 +34577 +34578 +34579 +34580 +34581 +34582 +34583 +34584 +34585 +34586 +34587 +34588 +34589 +34590 +34591 +34592 +34593 +34594 +34595 +34596 +34597 +34598 +34599 +34600 +34601 +34602 +34603 +34604 +34605 +34606 +34607 +34608 +34609 +34610 +34611 +34612 +34613 +34614 +34615 +34616 +34617 +34618 +34619 +34620 +34621 +34622 +34623 +34624 +34625 +34626 +34627 +34628 +34629 +34630 +34631 +34632 +34633 +34634 +34635 +34636 +34637 +34638 +34639 +34640 +34641 +34642 +34643 +34644 +34645 +34646 +34647 +34648 +34649 +34650 +34651 +34652 +34653 +34654 +34655 +34656 +34657 +34658 +34659 +34660 +34661 +34662 +34663 +34664 +34665 +34666 +34667 +34668 +34669 +34670 +34671 +34672 +34673 +34674 +34675 +34676 +34677 +34678 +34679 +34680 +34681 +34682 +34683 +34684 +34685 +34686 +34687 +34688 +34689 +34690 +34691 +34692 +34693 +34694 +34695 +34696 +34697 +34698 +34699 +34700 +34701 +34702 +34703 +34704 +34705 +34706 +34707 +34708 +34709 +34710 +34711 +34712 +34713 +34714 +34715 +34716 +34717 +34718 +34719 +34720 +34721 +34722 +34723 +34724 +34725 +34726 +34727 +34728 +34729 +34730 +34731 +34732 +34733 +34734 +34735 +34736 +34737 +34738 +34739 +34740 +34741 +34742 +34743 +34744 +34745 +34746 +34747 +34748 +34749 +34750 +34751 +34752 +34753 +34754 +34755 +34756 +34757 +34758 +34759 +34760 +34761 +34762 +34763 +34764 +34765 +34766 +34767 +34768 +34769 +34770 +34771 +34772 +34773 +34774 +34775 +34776 +34777 +34778 +34779 +34780 +34781 +34782 +34783 +34784 +34785 +34786 +34787 +34788 +34789 +34790 +34791 +34792 +34793 +34794 +34795 +34796 +34797 +34798 +34799 +34800 +34801 +34802 +34803 +34804 +34805 +34806 +34807 +34808 +34809 +34810 +34811 +34812 +34813 +34814 +34815 +34816 +34817 +34818 +34819 +34820 +34821 +34822 +34823 +34824 +34825 +34826 +34827 +34828 +34829 +34830 +34831 +34832 +34833 +34834 +34835 +34836 +34837 +34838 +34839 +34840 +34841 +34842 +34843 +34844 +34845 +34846 +34847 +34848 +34849 +34850 +34851 +34852 +34853 +34854 +34855 +34856 +34857 +34858 +34859 +34860 +34861 +34862 +34863 +34864 +34865 +34866 +34867 +34868 +34869 +34870 +34871 +34872 +34873 +34874 +34875 +34876 +34877 +34878 +34879 +34880 +34881 +34882 +34883 +34884 +34885 +34886 +34887 +34888 +34889 +34890 +34891 +34892 +34893 +34894 +34895 +34896 +34897 +34898 +34899 +34900 +34901 +34902 +34903 +34904 +34905 +34906 +34907 +34908 +34909 +34910 +34911 +34912 +34913 +34914 +34915 +34916 +34917 +34918 +34919 +34920 +34921 +34922 +34923 +34924 +34925 +34926 +34927 +34928 +34929 +34930 +34931 +34932 +34933 +34934 +34935 +34936 +34937 +34938 +34939 +34940 +34941 +34942 +34943 +34944 +34945 +34946 +34947 +34948 +34949 +34950 +34951 +34952 +34953 +34954 +34955 +34956 +34957 +34958 +34959 +34960 +34961 +34962 +34963 +34964 +34965 +34966 +34967 +34968 +34969 +34970 +34971 +34972 +34973 +34974 +34975 +34976 +34977 +34978 +34979 +34980 +34981 +34982 +34983 +34984 +34985 +34986 +34987 +34988 +34989 +34990 +34991 +34992 +34993 +34994 +34995 +34996 +34997 +34998 +34999 +35000 +35001 +35002 +35003 +35004 +35005 +35006 +35007 +35008 +35009 +35010 +35011 +35012 +35013 +35014 +35015 +35016 +35017 +35018 +35019 +35020 +35021 +35022 +35023 +35024 +35025 +35026 +35027 +35028 +35029 +35030 +35031 +35032 +35033 +35034 +35035 +35036 +35037 +35038 +35039 +35040 +35041 +35042 +35043 +35044 +35045 +35046 +35047 +35048 +35049 +35050 +35051 +35052 +35053 +35054 +35055 +35056 +35057 +35058 +35059 +35060 +35061 +35062 +35063 +35064 +35065 +35066 +35067 +35068 +35069 +35070 +35071 +35072 +35073 +35074 +35075 +35076 +35077 +35078 +35079 +35080 +35081 +35082 +35083 +35084 +35085 +35086 +35087 +35088 +35089 +35090 +35091 +35092 +35093 +35094 +35095 +35096 +35097 +35098 +35099 +35100 +35101 +35102 +35103 +35104 +35105 +35106 +35107 +35108 +35109 +35110 +35111 +35112 +35113 +35114 +35115 +35116 +35117 +35118 +35119 +35120 +35121 +35122 +35123 +35124 +35125 +35126 +35127 +35128 +35129 +35130 +35131 +35132 +35133 +35134 +35135 +35136 +35137 +35138 +35139 +35140 +35141 +35142 +35143 +35144 +35145 +35146 +35147 +35148 +35149 +35150 +35151 +35152 +35153 +35154 +35155 +35156 +35157 +35158 +35159 +35160 +35161 +35162 +35163 +35164 +35165 +35166 +35167 +35168 +35169 +35170 +35171 +35172 +35173 +35174 +35175 +35176 +35177 +35178 +35179 +35180 +35181 +35182 +35183 +35184 +35185 +35186 +35187 +35188 +35189 +35190 +35191 +35192 +35193 +35194 +35195 +35196 +35197 +35198 +35199 +35200 +35201 +35202 +35203 +35204 +35205 +35206 +35207 +35208 +35209 +35210 +35211 +35212 +35213 +35214 +35215 +35216 +35217 +35218 +35219 +35220 +35221 +35222 +35223 +35224 +35225 +35226 +35227 +35228 +35229 +35230 +35231 +35232 +35233 +35234 +35235 +35236 +35237 +35238 +35239 +35240 +35241 +35242 +35243 +35244 +35245 +35246 +35247 +35248 +35249 +35250 +35251 +35252 +35253 +35254 +35255 +35256 +35257 +35258 +35259 +35260 +35261 +35262 +35263 +35264 +35265 +35266 +35267 +35268 +35269 +35270 +35271 +35272 +35273 +35274 +35275 +35276 +35277 +35278 +35279 +35280 +35281 +35282 +35283 +35284 +35285 +35286 +35287 +35288 +35289 +35290 +35291 +35292 +35293 +35294 +35295 +35296 +35297 +35298 +35299 +35300 +35301 +35302 +35303 +35304 +35305 +35306 +35307 +35308 +35309 +35310 +35311 +35312 +35313 +35314 +35315 +35316 +35317 +35318 +35319 +35320 +35321 +35322 +35323 +35324 +35325 +35326 +35327 +35328 +35329 +35330 +35331 +35332 +35333 +35334 +35335 +35336 +35337 +35338 +35339 +35340 +35341 +35342 +35343 +35344 +35345 +35346 +35347 +35348 +35349 +35350 +35351 +35352 +35353 +35354 +35355 +35356 +35357 +35358 +35359 +35360 +35361 +35362 +35363 +35364 +35365 +35366 +35367 +35368 +35369 +35370 +35371 +35372 +35373 +35374 +35375 +35376 +35377 +35378 +35379 +35380 +35381 +35382 +35383 +35384 +35385 +35386 +35387 +35388 +35389 +35390 +35391 +35392 +35393 +35394 +35395 +35396 +35397 +35398 +35399 +35400 +35401 +35402 +35403 +35404 +35405 +35406 +35407 +35408 +35409 +35410 +35411 +35412 +35413 +35414 +35415 +35416 +35417 +35418 +35419 +35420 +35421 +35422 +35423 +35424 +35425 +35426 +35427 +35428 +35429 +35430 +35431 +35432 +35433 +35434 +35435 +35436 +35437 +35438 +35439 +35440 +35441 +35442 +35443 +35444 +35445 +35446 +35447 +35448 +35449 +35450 +35451 +35452 +35453 +35454 +35455 +35456 +35457 +35458 +35459 +35460 +35461 +35462 +35463 +35464 +35465 +35466 +35467 +35468 +35469 +35470 +35471 +35472 +35473 +35474 +35475 +35476 +35477 +35478 +35479 +35480 +35481 +35482 +35483 +35484 +35485 +35486 +35487 +35488 +35489 +35490 +35491 +35492 +35493 +35494 +35495 +35496 +35497 +35498 +35499 +35500 +35501 +35502 +35503 +35504 +35505 +35506 +35507 +35508 +35509 +35510 +35511 +35512 +35513 +35514 +35515 +35516 +35517 +35518 +35519 +35520 +35521 +35522 +35523 +35524 +35525 +35526 +35527 +35528 +35529 +35530 +35531 +35532 +35533 +35534 +35535 +35536 +35537 +35538 +35539 +35540 +35541 +35542 +35543 +35544 +35545 +35546 +35547 +35548 +35549 +35550 +35551 +35552 +35553 +35554 +35555 +35556 +35557 +35558 +35559 +35560 +35561 +35562 +35563 +35564 +35565 +35566 +35567 +35568 +35569 +35570 +35571 +35572 +35573 +35574 +35575 +35576 +35577 +35578 +35579 +35580 +35581 +35582 +35583 +35584 +35585 +35586 +35587 +35588 +35589 +35590 +35591 +35592 +35593 +35594 +35595 +35596 +35597 +35598 +35599 +35600 +35601 +35602 +35603 +35604 +35605 +35606 +35607 +35608 +35609 +35610 +35611 +35612 +35613 +35614 +35615 +35616 +35617 +35618 +35619 +35620 +35621 +35622 +35623 +35624 +35625 +35626 +35627 +35628 +35629 +35630 +35631 +35632 +35633 +35634 +35635 +35636 +35637 +35638 +35639 +35640 +35641 +35642 +35643 +35644 +35645 +35646 +35647 +35648 +35649 +35650 +35651 +35652 +35653 +35654 +35655 +35656 +35657 +35658 +35659 +35660 +35661 +35662 +35663 +35664 +35665 +35666 +35667 +35668 +35669 +35670 +35671 +35672 +35673 +35674 +35675 +35676 +35677 +35678 +35679 +35680 +35681 +35682 +35683 +35684 +35685 +35686 +35687 +35688 +35689 +35690 +35691 +35692 +35693 +35694 +35695 +35696 +35697 +35698 +35699 +35700 +35701 +35702 +35703 +35704 +35705 +35706 +35707 +35708 +35709 +35710 +35711 +35712 +35713 +35714 +35715 +35716 +35717 +35718 +35719 +35720 +35721 +35722 +35723 +35724 +35725 +35726 +35727 +35728 +35729 +35730 +35731 +35732 +35733 +35734 +35735 +35736 +35737 +35738 +35739 +35740 +35741 +35742 +35743 +35744 +35745 +35746 +35747 +35748 +35749 +35750 +35751 +35752 +35753 +35754 +35755 +35756 +35757 +35758 +35759 +35760 +35761 +35762 +35763 +35764 +35765 +35766 +35767 +35768 +35769 +35770 +35771 +35772 +35773 +35774 +35775 +35776 +35777 +35778 +35779 +35780 +35781 +35782 +35783 +35784 +35785 +35786 +35787 +35788 +35789 +35790 +35791 +35792 +35793 +35794 +35795 +35796 +35797 +35798 +35799 +35800 +35801 +35802 +35803 +35804 +35805 +35806 +35807 +35808 +35809 +35810 +35811 +35812 +35813 +35814 +35815 +35816 +35817 +35818 +35819 +35820 +35821 +35822 +35823 +35824 +35825 +35826 +35827 +35828 +35829 +35830 +35831 +35832 +35833 +35834 +35835 +35836 +35837 +35838 +35839 +35840 +35841 +35842 +35843 +35844 +35845 +35846 +35847 +35848 +35849 +35850 +35851 +35852 +35853 +35854 +35855 +35856 +35857 +35858 +35859 +35860 +35861 +35862 +35863 +35864 +35865 +35866 +35867 +35868 +35869 +35870 +35871 +35872 +35873 +35874 +35875 +35876 +35877 +35878 +35879 +35880 +35881 +35882 +35883 +35884 +35885 +35886 +35887 +35888 +35889 +35890 +35891 +35892 +35893 +35894 +35895 +35896 +35897 +35898 +35899 +35900 +35901 +35902 +35903 +35904 +35905 +35906 +35907 +35908 +35909 +35910 +35911 +35912 +35913 +35914 +35915 +35916 +35917 +35918 +35919 +35920 +35921 +35922 +35923 +35924 +35925 +35926 +35927 +35928 +35929 +35930 +35931 +35932 +35933 +35934 +35935 +35936 +35937 +35938 +35939 +35940 +35941 +35942 +35943 +35944 +35945 +35946 +35947 +35948 +35949 +35950 +35951 +35952 +35953 +35954 +35955 +35956 +35957 +35958 +35959 +35960 +35961 +35962 +35963 +35964 +35965 +35966 +35967 +35968 +35969 +35970 +35971 +35972 +35973 +35974 +35975 +35976 +35977 +35978 +35979 +35980 +35981 +35982 +35983 +35984 +35985 +35986 +35987 +35988 +35989 +35990 +35991 +35992 +35993 +35994 +35995 +35996 +35997 +35998 +35999 +36000 +36001 +36002 +36003 +36004 +36005 +36006 +36007 +36008 +36009 +36010 +36011 +36012 +36013 +36014 +36015 +36016 +36017 +36018 +36019 +36020 +36021 +36022 +36023 +36024 +36025 +36026 +36027 +36028 +36029 +36030 +36031 +36032 +36033 +36034 +36035 +36036 +36037 +36038 +36039 +36040 +36041 +36042 +36043 +36044 +36045 +36046 +36047 +36048 +36049 +36050 +36051 +36052 +36053 +36054 +36055 +36056 +36057 +36058 +36059 +36060 +36061 +36062 +36063 +36064 +36065 +36066 +36067 +36068 +36069 +36070 +36071 +36072 +36073 +36074 +36075 +36076 +36077 +36078 +36079 +36080 +36081 +36082 +36083 +36084 +36085 +36086 +36087 +36088 +36089 +36090 +36091 +36092 +36093 +36094 +36095 +36096 +36097 +36098 +36099 +36100 +36101 +36102 +36103 +36104 +36105 +36106 +36107 +36108 +36109 +36110 +36111 +36112 +36113 +36114 +36115 +36116 +36117 +36118 +36119 +36120 +36121 +36122 +36123 +36124 +36125 +36126 +36127 +36128 +36129 +36130 +36131 +36132 +36133 +36134 +36135 +36136 +36137 +36138 +36139 +36140 +36141 +36142 +36143 +36144 +36145 +36146 +36147 +36148 +36149 +36150 +36151 +36152 +36153 +36154 +36155 +36156 +36157 +36158 +36159 +36160 +36161 +36162 +36163 +36164 +36165 +36166 +36167 +36168 +36169 +36170 +36171 +36172 +36173 +36174 +36175 +36176 +36177 +36178 +36179 +36180 +36181 +36182 +36183 +36184 +36185 +36186 +36187 +36188 +36189 +36190 +36191 +36192 +36193 +36194 +36195 +36196 +36197 +36198 +36199 +36200 +36201 +36202 +36203 +36204 +36205 +36206 +36207 +36208 +36209 +36210 +36211 +36212 +36213 +36214 +36215 +36216 +36217 +36218 +36219 +36220 +36221 +36222 +36223 +36224 +36225 +36226 +36227 +36228 +36229 +36230 +36231 +36232 +36233 +36234 +36235 +36236 +36237 +36238 +36239 +36240 +36241 +36242 +36243 +36244 +36245 +36246 +36247 +36248 +36249 +36250 +36251 +36252 +36253 +36254 +36255 +36256 +36257 +36258 +36259 +36260 +36261 +36262 +36263 +36264 +36265 +36266 +36267 +36268 +36269 +36270 +36271 +36272 +36273 +36274 +36275 +36276 +36277 +36278 +36279 +36280 +36281 +36282 +36283 +36284 +36285 +36286 +36287 +36288 +36289 +36290 +36291 +36292 +36293 +36294 +36295 +36296 +36297 +36298 +36299 +36300 +36301 +36302 +36303 +36304 +36305 +36306 +36307 +36308 +36309 +36310 +36311 +36312 +36313 +36314 +36315 +36316 +36317 +36318 +36319 +36320 +36321 +36322 +36323 +36324 +36325 +36326 +36327 +36328 +36329 +36330 +36331 +36332 +36333 +36334 +36335 +36336 +36337 +36338 +36339 +36340 +36341 +36342 +36343 +36344 +36345 +36346 +36347 +36348 +36349 +36350 +36351 +36352 +36353 +36354 +36355 +36356 +36357 +36358 +36359 +36360 +36361 +36362 +36363 +36364 +36365 +36366 +36367 +36368 +36369 +36370 +36371 +36372 +36373 +36374 +36375 +36376 +36377 +36378 +36379 +36380 +36381 +36382 +36383 +36384 +36385 +36386 +36387 +36388 +36389 +36390 +36391 +36392 +36393 +36394 +36395 +36396 +36397 +36398 +36399 +36400 +36401 +36402 +36403 +36404 +36405 +36406 +36407 +36408 +36409 +36410 +36411 +36412 +36413 +36414 +36415 +36416 +36417 +36418 +36419 +36420 +36421 +36422 +36423 +36424 +36425 +36426 +36427 +36428 +36429 +36430 +36431 +36432 +36433 +36434 +36435 +36436 +36437 +36438 +36439 +36440 +36441 +36442 +36443 +36444 +36445 +36446 +36447 +36448 +36449 +36450 +36451 +36452 +36453 +36454 +36455 +36456 +36457 +36458 +36459 +36460 +36461 +36462 +36463 +36464 +36465 +36466 +36467 +36468 +36469 +36470 +36471 +36472 +36473 +36474 +36475 +36476 +36477 +36478 +36479 +36480 +36481 +36482 +36483 +36484 +36485 +36486 +36487 +36488 +36489 +36490 +36491 +36492 +36493 +36494 +36495 +36496 +36497 +36498 +36499 +36500 +36501 +36502 +36503 +36504 +36505 +36506 +36507 +36508 +36509 +36510 +36511 +36512 +36513 +36514 +36515 +36516 +36517 +36518 +36519 +36520 +36521 +36522 +36523 +36524 +36525 +36526 +36527 +36528 +36529 +36530 +36531 +36532 +36533 +36534 +36535 +36536 +36537 +36538 +36539 +36540 +36541 +36542 +36543 +36544 +36545 +36546 +36547 +36548 +36549 +36550 +36551 +36552 +36553 +36554 +36555 +36556 +36557 +36558 +36559 +36560 +36561 +36562 +36563 +36564 +36565 +36566 +36567 +36568 +36569 +36570 +36571 +36572 +36573 +36574 +36575 +36576 +36577 +36578 +36579 +36580 +36581 +36582 +36583 +36584 +36585 +36586 +36587 +36588 +36589 +36590 +36591 +36592 +36593 +36594 +36595 +36596 +36597 +36598 +36599 +36600 +36601 +36602 +36603 +36604 +36605 +36606 +36607 +36608 +36609 +36610 +36611 +36612 +36613 +36614 +36615 +36616 +36617 +36618 +36619 +36620 +36621 +36622 +36623 +36624 +36625 +36626 +36627 +36628 +36629 +36630 +36631 +36632 +36633 +36634 +36635 +36636 +36637 +36638 +36639 +36640 +36641 +36642 +36643 +36644 +36645 +36646 +36647 +36648 +36649 +36650 +36651 +36652 +36653 +36654 +36655 +36656 +36657 +36658 +36659 +36660 +36661 +36662 +36663 +36664 +36665 +36666 +36667 +36668 +36669 +36670 +36671 +36672 +36673 +36674 +36675 +36676 +36677 +36678 +36679 +36680 +36681 +36682 +36683 +36684 +36685 +36686 +36687 +36688 +36689 +36690 +36691 +36692 +36693 +36694 +36695 +36696 +36697 +36698 +36699 +36700 +36701 +36702 +36703 +36704 +36705 +36706 +36707 +36708 +36709 +36710 +36711 +36712 +36713 +36714 +36715 +36716 +36717 +36718 +36719 +36720 +36721 +36722 +36723 +36724 +36725 +36726 +36727 +36728 +36729 +36730 +36731 +36732 +36733 +36734 +36735 +36736 +36737 +36738 +36739 +36740 +36741 +36742 +36743 +36744 +36745 +36746 +36747 +36748 +36749 +36750 +36751 +36752 +36753 +36754 +36755 +36756 +36757 +36758 +36759 +36760 +36761 +36762 +36763 +36764 +36765 +36766 +36767 +36768 +36769 +36770 +36771 +36772 +36773 +36774 +36775 +36776 +36777 +36778 +36779 +36780 +36781 +36782 +36783 +36784 +36785 +36786 +36787 +36788 +36789 +36790 +36791 +36792 +36793 +36794 +36795 +36796 +36797 +36798 +36799 +36800 +36801 +36802 +36803 +36804 +36805 +36806 +36807 +36808 +36809 +36810 +36811 +36812 +36813 +36814 +36815 +36816 +36817 +36818 +36819 +36820 +36821 +36822 +36823 +36824 +36825 +36826 +36827 +36828 +36829 +36830 +36831 +36832 +36833 +36834 +36835 +36836 +36837 +36838 +36839 +36840 +36841 +36842 +36843 +36844 +36845 +36846 +36847 +36848 +36849 +36850 +36851 +36852 +36853 +36854 +36855 +36856 +36857 +36858 +36859 +36860 +36861 +36862 +36863 +36864 +36865 +36866 +36867 +36868 +36869 +36870 +36871 +36872 +36873 +36874 +36875 +36876 +36877 +36878 +36879 +36880 +36881 +36882 +36883 +36884 +36885 +36886 +36887 +36888 +36889 +36890 +36891 +36892 +36893 +36894 +36895 +36896 +36897 +36898 +36899 +36900 +36901 +36902 +36903 +36904 +36905 +36906 +36907 +36908 +36909 +36910 +36911 +36912 +36913 +36914 +36915 +36916 +36917 +36918 +36919 +36920 +36921 +36922 +36923 +36924 +36925 +36926 +36927 +36928 +36929 +36930 +36931 +36932 +36933 +36934 +36935 +36936 +36937 +36938 +36939 +36940 +36941 +36942 +36943 +36944 +36945 +36946 +36947 +36948 +36949 +36950 +36951 +36952 +36953 +36954 +36955 +36956 +36957 +36958 +36959 +36960 +36961 +36962 +36963 +36964 +36965 +36966 +36967 +36968 +36969 +36970 +36971 +36972 +36973 +36974 +36975 +36976 +36977 +36978 +36979 +36980 +36981 +36982 +36983 +36984 +36985 +36986 +36987 +36988 +36989 +36990 +36991 +36992 +36993 +36994 +36995 +36996 +36997 +36998 +36999 +37000 +37001 +37002 +37003 +37004 +37005 +37006 +37007 +37008 +37009 +37010 +37011 +37012 +37013 +37014 +37015 +37016 +37017 +37018 +37019 +37020 +37021 +37022 +37023 +37024 +37025 +37026 +37027 +37028 +37029 +37030 +37031 +37032 +37033 +37034 +37035 +37036 +37037 +37038 +37039 +37040 +37041 +37042 +37043 +37044 +37045 +37046 +37047 +37048 +37049 +37050 +37051 +37052 +37053 +37054 +37055 +37056 +37057 +37058 +37059 +37060 +37061 +37062 +37063 +37064 +37065 +37066 +37067 +37068 +37069 +37070 +37071 +37072 +37073 +37074 +37075 +37076 +37077 +37078 +37079 +37080 +37081 +37082 +37083 +37084 +37085 +37086 +37087 +37088 +37089 +37090 +37091 +37092 +37093 +37094 +37095 +37096 +37097 +37098 +37099 +37100 +37101 +37102 +37103 +37104 +37105 +37106 +37107 +37108 +37109 +37110 +37111 +37112 +37113 +37114 +37115 +37116 +37117 +37118 +37119 +37120 +37121 +37122 +37123 +37124 +37125 +37126 +37127 +37128 +37129 +37130 +37131 +37132 +37133 +37134 +37135 +37136 +37137 +37138 +37139 +37140 +37141 +37142 +37143 +37144 +37145 +37146 +37147 +37148 +37149 +37150 +37151 +37152 +37153 +37154 +37155 +37156 +37157 +37158 +37159 +37160 +37161 +37162 +37163 +37164 +37165 +37166 +37167 +37168 +37169 +37170 +37171 +37172 +37173 +37174 +37175 +37176 +37177 +37178 +37179 +37180 +37181 +37182 +37183 +37184 +37185 +37186 +37187 +37188 +37189 +37190 +37191 +37192 +37193 +37194 +37195 +37196 +37197 +37198 +37199 +37200 +37201 +37202 +37203 +37204 +37205 +37206 +37207 +37208 +37209 +37210 +37211 +37212 +37213 +37214 +37215 +37216 +37217 +37218 +37219 +37220 +37221 +37222 +37223 +37224 +37225 +37226 +37227 +37228 +37229 +37230 +37231 +37232 +37233 +37234 +37235 +37236 +37237 +37238 +37239 +37240 +37241 +37242 +37243 +37244 +37245 +37246 +37247 +37248 +37249 +37250 +37251 +37252 +37253 +37254 +37255 +37256 +37257 +37258 +37259 +37260 +37261 +37262 +37263 +37264 +37265 +37266 +37267 +37268 +37269 +37270 +37271 +37272 +37273 +37274 +37275 +37276 +37277 +37278 +37279 +37280 +37281 +37282 +37283 +37284 +37285 +37286 +37287 +37288 +37289 +37290 +37291 +37292 +37293 +37294 +37295 +37296 +37297 +37298 +37299 +37300 +37301 +37302 +37303 +37304 +37305 +37306 +37307 +37308 +37309 +37310 +37311 +37312 +37313 +37314 +37315 +37316 +37317 +37318 +37319 +37320 +37321 +37322 +37323 +37324 +37325 +37326 +37327 +37328 +37329 +37330 +37331 +37332 +37333 +37334 +37335 +37336 +37337 +37338 +37339 +37340 +37341 +37342 +37343 +37344 +37345 +37346 +37347 +37348 +37349 +37350 +37351 +37352 +37353 +37354 +37355 +37356 +37357 +37358 +37359 +37360 +37361 +37362 +37363 +37364 +37365 +37366 +37367 +37368 +37369 +37370 +37371 +37372 +37373 +37374 +37375 +37376 +37377 +37378 +37379 +37380 +37381 +37382 +37383 +37384 +37385 +37386 +37387 +37388 +37389 +37390 +37391 +37392 +37393 +37394 +37395 +37396 +37397 +37398 +37399 +37400 +37401 +37402 +37403 +37404 +37405 +37406 +37407 +37408 +37409 +37410 +37411 +37412 +37413 +37414 +37415 +37416 +37417 +37418 +37419 +37420 +37421 +37422 +37423 +37424 +37425 +37426 +37427 +37428 +37429 +37430 +37431 +37432 +37433 +37434 +37435 +37436 +37437 +37438 +37439 +37440 +37441 +37442 +37443 +37444 +37445 +37446 +37447 +37448 +37449 +37450 +37451 +37452 +37453 +37454 +37455 +37456 +37457 +37458 +37459 +37460 +37461 +37462 +37463 +37464 +37465 +37466 +37467 +37468 +37469 +37470 +37471 +37472 +37473 +37474 +37475 +37476 +37477 +37478 +37479 +37480 +37481 +37482 +37483 +37484 +37485 +37486 +37487 +37488 +37489 +37490 +37491 +37492 +37493 +37494 +37495 +37496 +37497 +37498 +37499 +37500 +37501 +37502 +37503 +37504 +37505 +37506 +37507 +37508 +37509 +37510 +37511 +37512 +37513 +37514 +37515 +37516 +37517 +37518 +37519 +37520 +37521 +37522 +37523 +37524 +37525 +37526 +37527 +37528 +37529 +37530 +37531 +37532 +37533 +37534 +37535 +37536 +37537 +37538 +37539 +37540 +37541 +37542 +37543 +37544 +37545 +37546 +37547 +37548 +37549 +37550 +37551 +37552 +37553 +37554 +37555 +37556 +37557 +37558 +37559 +37560 +37561 +37562 +37563 +37564 +37565 +37566 +37567 +37568 +37569 +37570 +37571 +37572 +37573 +37574 +37575 +37576 +37577 +37578 +37579 +37580 +37581 +37582 +37583 +37584 +37585 +37586 +37587 +37588 +37589 +37590 +37591 +37592 +37593 +37594 +37595 +37596 +37597 +37598 +37599 +37600 +37601 +37602 +37603 +37604 +37605 +37606 +37607 +37608 +37609 +37610 +37611 +37612 +37613 +37614 +37615 +37616 +37617 +37618 +37619 +37620 +37621 +37622 +37623 +37624 +37625 +37626 +37627 +37628 +37629 +37630 +37631 +37632 +37633 +37634 +37635 +37636 +37637 +37638 +37639 +37640 +37641 +37642 +37643 +37644 +37645 +37646 +37647 +37648 +37649 +37650 +37651 +37652 +37653 +37654 +37655 +37656 +37657 +37658 +37659 +37660 +37661 +37662 +37663 +37664 +37665 +37666 +37667 +37668 +37669 +37670 +37671 +37672 +37673 +37674 +37675 +37676 +37677 +37678 +37679 +37680 +37681 +37682 +37683 +37684 +37685 +37686 +37687 +37688 +37689 +37690 +37691 +37692 +37693 +37694 +37695 +37696 +37697 +37698 +37699 +37700 +37701 +37702 +37703 +37704 +37705 +37706 +37707 +37708 +37709 +37710 +37711 +37712 +37713 +37714 +37715 +37716 +37717 +37718 +37719 +37720 +37721 +37722 +37723 +37724 +37725 +37726 +37727 +37728 +37729 +37730 +37731 +37732 +37733 +37734 +37735 +37736 +37737 +37738 +37739 +37740 +37741 +37742 +37743 +37744 +37745 +37746 +37747 +37748 +37749 +37750 +37751 +37752 +37753 +37754 +37755 +37756 +37757 +37758 +37759 +37760 +37761 +37762 +37763 +37764 +37765 +37766 +37767 +37768 +37769 +37770 +37771 +37772 +37773 +37774 +37775 +37776 +37777 +37778 +37779 +37780 +37781 +37782 +37783 +37784 +37785 +37786 +37787 +37788 +37789 +37790 +37791 +37792 +37793 +37794 +37795 +37796 +37797 +37798 +37799 +37800 +37801 +37802 +37803 +37804 +37805 +37806 +37807 +37808 +37809 +37810 +37811 +37812 +37813 +37814 +37815 +37816 +37817 +37818 +37819 +37820 +37821 +37822 +37823 +37824 +37825 +37826 +37827 +37828 +37829 +37830 +37831 +37832 +37833 +37834 +37835 +37836 +37837 +37838 +37839 +37840 +37841 +37842 +37843 +37844 +37845 +37846 +37847 +37848 +37849 +37850 +37851 +37852 +37853 +37854 +37855 +37856 +37857 +37858 +37859 +37860 +37861 +37862 +37863 +37864 +37865 +37866 +37867 +37868 +37869 +37870 +37871 +37872 +37873 +37874 +37875 +37876 +37877 +37878 +37879 +37880 +37881 +37882 +37883 +37884 +37885 +37886 +37887 +37888 +37889 +37890 +37891 +37892 +37893 +37894 +37895 +37896 +37897 +37898 +37899 +37900 +37901 +37902 +37903 +37904 +37905 +37906 +37907 +37908 +37909 +37910 +37911 +37912 +37913 +37914 +37915 +37916 +37917 +37918 +37919 +37920 +37921 +37922 +37923 +37924 +37925 +37926 +37927 +37928 +37929 +37930 +37931 +37932 +37933 +37934 +37935 +37936 +37937 +37938 +37939 +37940 +37941 +37942 +37943 +37944 +37945 +37946 +37947 +37948 +37949 +37950 +37951 +37952 +37953 +37954 +37955 +37956 +37957 +37958 +37959 +37960 +37961 +37962 +37963 +37964 +37965 +37966 +37967 +37968 +37969 +37970 +37971 +37972 +37973 +37974 +37975 +37976 +37977 +37978 +37979 +37980 +37981 +37982 +37983 +37984 +37985 +37986 +37987 +37988 +37989 +37990 +37991 +37992 +37993 +37994 +37995 +37996 +37997 +37998 +37999 +38000 +38001 +38002 +38003 +38004 +38005 +38006 +38007 +38008 +38009 +38010 +38011 +38012 +38013 +38014 +38015 +38016 +38017 +38018 +38019 +38020 +38021 +38022 +38023 +38024 +38025 +38026 +38027 +38028 +38029 +38030 +38031 +38032 +38033 +38034 +38035 +38036 +38037 +38038 +38039 +38040 +38041 +38042 +38043 +38044 +38045 +38046 +38047 +38048 +38049 +38050 +38051 +38052 +38053 +38054 +38055 +38056 +38057 +38058 +38059 +38060 +38061 +38062 +38063 +38064 +38065 +38066 +38067 +38068 +38069 +38070 +38071 +38072 +38073 +38074 +38075 +38076 +38077 +38078 +38079 +38080 +38081 +38082 +38083 +38084 +38085 +38086 +38087 +38088 +38089 +38090 +38091 +38092 +38093 +38094 +38095 +38096 +38097 +38098 +38099 +38100 +38101 +38102 +38103 +38104 +38105 +38106 +38107 +38108 +38109 +38110 +38111 +38112 +38113 +38114 +38115 +38116 +38117 +38118 +38119 +38120 +38121 +38122 +38123 +38124 +38125 +38126 +38127 +38128 +38129 +38130 +38131 +38132 +38133 +38134 +38135 +38136 +38137 +38138 +38139 +38140 +38141 +38142 +38143 +38144 +38145 +38146 +38147 +38148 +38149 +38150 +38151 +38152 +38153 +38154 +38155 +38156 +38157 +38158 +38159 +38160 +38161 +38162 +38163 +38164 +38165 +38166 +38167 +38168 +38169 +38170 +38171 +38172 +38173 +38174 +38175 +38176 +38177 +38178 +38179 +38180 +38181 +38182 +38183 +38184 +38185 +38186 +38187 +38188 +38189 +38190 +38191 +38192 +38193 +38194 +38195 +38196 +38197 +38198 +38199 +38200 +38201 +38202 +38203 +38204 +38205 +38206 +38207 +38208 +38209 +38210 +38211 +38212 +38213 +38214 +38215 +38216 +38217 +38218 +38219 +38220 +38221 +38222 +38223 +38224 +38225 +38226 +38227 +38228 +38229 +38230 +38231 +38232 +38233 +38234 +38235 +38236 +38237 +38238 +38239 +38240 +38241 +38242 +38243 +38244 +38245 +38246 +38247 +38248 +38249 +38250 +38251 +38252 +38253 +38254 +38255 +38256 +38257 +38258 +38259 +38260 +38261 +38262 +38263 +38264 +38265 +38266 +38267 +38268 +38269 +38270 +38271 +38272 +38273 +38274 +38275 +38276 +38277 +38278 +38279 +38280 +38281 +38282 +38283 +38284 +38285 +38286 +38287 +38288 +38289 +38290 +38291 +38292 +38293 +38294 +38295 +38296 +38297 +38298 +38299 +38300 +38301 +38302 +38303 +38304 +38305 +38306 +38307 +38308 +38309 +38310 +38311 +38312 +38313 +38314 +38315 +38316 +38317 +38318 +38319 +38320 +38321 +38322 +38323 +38324 +38325 +38326 +38327 +38328 +38329 +38330 +38331 +38332 +38333 +38334 +38335 +38336 +38337 +38338 +38339 +38340 +38341 +38342 +38343 +38344 +38345 +38346 +38347 +38348 +38349 +38350 +38351 +38352 +38353 +38354 +38355 +38356 +38357 +38358 +38359 +38360 +38361 +38362 +38363 +38364 +38365 +38366 +38367 +38368 +38369 +38370 +38371 +38372 +38373 +38374 +38375 +38376 +38377 +38378 +38379 +38380 +38381 +38382 +38383 +38384 +38385 +38386 +38387 +38388 +38389 +38390 +38391 +38392 +38393 +38394 +38395 +38396 +38397 +38398 +38399 +38400 +38401 +38402 +38403 +38404 +38405 +38406 +38407 +38408 +38409 +38410 +38411 +38412 +38413 +38414 +38415 +38416 +38417 +38418 +38419 +38420 +38421 +38422 +38423 +38424 +38425 +38426 +38427 +38428 +38429 +38430 +38431 +38432 +38433 +38434 +38435 +38436 +38437 +38438 +38439 +38440 +38441 +38442 +38443 +38444 +38445 +38446 +38447 +38448 +38449 +38450 +38451 +38452 +38453 +38454 +38455 +38456 +38457 +38458 +38459 +38460 +38461 +38462 +38463 +38464 +38465 +38466 +38467 +38468 +38469 +38470 +38471 +38472 +38473 +38474 +38475 +38476 +38477 +38478 +38479 +38480 +38481 +38482 +38483 +38484 +38485 +38486 +38487 +38488 +38489 +38490 +38491 +38492 +38493 +38494 +38495 +38496 +38497 +38498 +38499 +38500 +38501 +38502 +38503 +38504 +38505 +38506 +38507 +38508 +38509 +38510 +38511 +38512 +38513 +38514 +38515 +38516 +38517 +38518 +38519 +38520 +38521 +38522 +38523 +38524 +38525 +38526 +38527 +38528 +38529 +38530 +38531 +38532 +38533 +38534 +38535 +38536 +38537 +38538 +38539 +38540 +38541 +38542 +38543 +38544 +38545 +38546 +38547 +38548 +38549 +38550 +38551 +38552 +38553 +38554 +38555 +38556 +38557 +38558 +38559 +38560 +38561 +38562 +38563 +38564 +38565 +38566 +38567 +38568 +38569 +38570 +38571 +38572 +38573 +38574 +38575 +38576 +38577 +38578 +38579 +38580 +38581 +38582 +38583 +38584 +38585 +38586 +38587 +38588 +38589 +38590 +38591 +38592 +38593 +38594 +38595 +38596 +38597 +38598 +38599 +38600 +38601 +38602 +38603 +38604 +38605 +38606 +38607 +38608 +38609 +38610 +38611 +38612 +38613 +38614 +38615 +38616 +38617 +38618 +38619 +38620 +38621 +38622 +38623 +38624 +38625 +38626 +38627 +38628 +38629 +38630 +38631 +38632 +38633 +38634 +38635 +38636 +38637 +38638 +38639 +38640 +38641 +38642 +38643 +38644 +38645 +38646 +38647 +38648 +38649 +38650 +38651 +38652 +38653 +38654 +38655 +38656 +38657 +38658 +38659 +38660 +38661 +38662 +38663 +38664 +38665 +38666 +38667 +38668 +38669 +38670 +38671 +38672 +38673 +38674 +38675 +38676 +38677 +38678 +38679 +38680 +38681 +38682 +38683 +38684 +38685 +38686 +38687 +38688 +38689 +38690 +38691 +38692 +38693 +38694 +38695 +38696 +38697 +38698 +38699 +38700 +38701 +38702 +38703 +38704 +38705 +38706 +38707 +38708 +38709 +38710 +38711 +38712 +38713 +38714 +38715 +38716 +38717 +38718 +38719 +38720 +38721 +38722 +38723 +38724 +38725 +38726 +38727 +38728 +38729 +38730 +38731 +38732 +38733 +38734 +38735 +38736 +38737 +38738 +38739 +38740 +38741 +38742 +38743 +38744 +38745 +38746 +38747 +38748 +38749 +38750 +38751 +38752 +38753 +38754 +38755 +38756 +38757 +38758 +38759 +38760 +38761 +38762 +38763 +38764 +38765 +38766 +38767 +38768 +38769 +38770 +38771 +38772 +38773 +38774 +38775 +38776 +38777 +38778 +38779 +38780 +38781 +38782 +38783 +38784 +38785 +38786 +38787 +38788 +38789 +38790 +38791 +38792 +38793 +38794 +38795 +38796 +38797 +38798 +38799 +38800 +38801 +38802 +38803 +38804 +38805 +38806 +38807 +38808 +38809 +38810 +38811 +38812 +38813 +38814 +38815 +38816 +38817 +38818 +38819 +38820 +38821 +38822 +38823 +38824 +38825 +38826 +38827 +38828 +38829 +38830 +38831 +38832 +38833 +38834 +38835 +38836 +38837 +38838 +38839 +38840 +38841 +38842 +38843 +38844 +38845 +38846 +38847 +38848 +38849 +38850 +38851 +38852 +38853 +38854 +38855 +38856 +38857 +38858 +38859 +38860 +38861 +38862 +38863 +38864 +38865 +38866 +38867 +38868 +38869 +38870 +38871 +38872 +38873 +38874 +38875 +38876 +38877 +38878 +38879 +38880 +38881 +38882 +38883 +38884 +38885 +38886 +38887 +38888 +38889 +38890 +38891 +38892 +38893 +38894 +38895 +38896 +38897 +38898 +38899 +38900 +38901 +38902 +38903 +38904 +38905 +38906 +38907 +38908 +38909 +38910 +38911 +38912 +38913 +38914 +38915 +38916 +38917 +38918 +38919 +38920 +38921 +38922 +38923 +38924 +38925 +38926 +38927 +38928 +38929 +38930 +38931 +38932 +38933 +38934 +38935 +38936 +38937 +38938 +38939 +38940 +38941 +38942 +38943 +38944 +38945 +38946 +38947 +38948 +38949 +38950 +38951 +38952 +38953 +38954 +38955 +38956 +38957 +38958 +38959 +38960 +38961 +38962 +38963 +38964 +38965 +38966 +38967 +38968 +38969 +38970 +38971 +38972 +38973 +38974 +38975 +38976 +38977 +38978 +38979 +38980 +38981 +38982 +38983 +38984 +38985 +38986 +38987 +38988 +38989 +38990 +38991 +38992 +38993 +38994 +38995 +38996 +38997 +38998 +38999 +39000 +39001 +39002 +39003 +39004 +39005 +39006 +39007 +39008 +39009 +39010 +39011 +39012 +39013 +39014 +39015 +39016 +39017 +39018 +39019 +39020 +39021 +39022 +39023 +39024 +39025 +39026 +39027 +39028 +39029 +39030 +39031 +39032 +39033 +39034 +39035 +39036 +39037 +39038 +39039 +39040 +39041 +39042 +39043 +39044 +39045 +39046 +39047 +39048 +39049 +39050 +39051 +39052 +39053 +39054 +39055 +39056 +39057 +39058 +39059 +39060 +39061 +39062 +39063 +39064 +39065 +39066 +39067 +39068 +39069 +39070 +39071 +39072 +39073 +39074 +39075 +39076 +39077 +39078 +39079 +39080 +39081 +39082 +39083 +39084 +39085 +39086 +39087 +39088 +39089 +39090 +39091 +39092 +39093 +39094 +39095 +39096 +39097 +39098 +39099 +39100 +39101 +39102 +39103 +39104 +39105 +39106 +39107 +39108 +39109 +39110 +39111 +39112 +39113 +39114 +39115 +39116 +39117 +39118 +39119 +39120 +39121 +39122 +39123 +39124 +39125 +39126 +39127 +39128 +39129 +39130 +39131 +39132 +39133 +39134 +39135 +39136 +39137 +39138 +39139 +39140 +39141 +39142 +39143 +39144 +39145 +39146 +39147 +39148 +39149 +39150 +39151 +39152 +39153 +39154 +39155 +39156 +39157 +39158 +39159 +39160 +39161 +39162 +39163 +39164 +39165 +39166 +39167 +39168 +39169 +39170 +39171 +39172 +39173 +39174 +39175 +39176 +39177 +39178 +39179 +39180 +39181 +39182 +39183 +39184 +39185 +39186 +39187 +39188 +39189 +39190 +39191 +39192 +39193 +39194 +39195 +39196 +39197 +39198 +39199 +39200 +39201 +39202 +39203 +39204 +39205 +39206 +39207 +39208 +39209 +39210 +39211 +39212 +39213 +39214 +39215 +39216 +39217 +39218 +39219 +39220 +39221 +39222 +39223 +39224 +39225 +39226 +39227 +39228 +39229 +39230 +39231 +39232 +39233 +39234 +39235 +39236 +39237 +39238 +39239 +39240 +39241 +39242 +39243 +39244 +39245 +39246 +39247 +39248 +39249 +39250 +39251 +39252 +39253 +39254 +39255 +39256 +39257 +39258 +39259 +39260 +39261 +39262 +39263 +39264 +39265 +39266 +39267 +39268 +39269 +39270 +39271 +39272 +39273 +39274 +39275 +39276 +39277 +39278 +39279 +39280 +39281 +39282 +39283 +39284 +39285 +39286 +39287 +39288 +39289 +39290 +39291 +39292 +39293 +39294 +39295 +39296 +39297 +39298 +39299 +39300 +39301 +39302 +39303 +39304 +39305 +39306 +39307 +39308 +39309 +39310 +39311 +39312 +39313 +39314 +39315 +39316 +39317 +39318 +39319 +39320 +39321 +39322 +39323 +39324 +39325 +39326 +39327 +39328 +39329 +39330 +39331 +39332 +39333 +39334 +39335 +39336 +39337 +39338 +39339 +39340 +39341 +39342 +39343 +39344 +39345 +39346 +39347 +39348 +39349 +39350 +39351 +39352 +39353 +39354 +39355 +39356 +39357 +39358 +39359 +39360 +39361 +39362 +39363 +39364 +39365 +39366 +39367 +39368 +39369 +39370 +39371 +39372 +39373 +39374 +39375 +39376 +39377 +39378 +39379 +39380 +39381 +39382 +39383 +39384 +39385 +39386 +39387 +39388 +39389 +39390 +39391 +39392 +39393 +39394 +39395 +39396 +39397 +39398 +39399 +39400 +39401 +39402 +39403 +39404 +39405 +39406 +39407 +39408 +39409 +39410 +39411 +39412 +39413 +39414 +39415 +39416 +39417 +39418 +39419 +39420 +39421 +39422 +39423 +39424 +39425 +39426 +39427 +39428 +39429 +39430 +39431 +39432 +39433 +39434 +39435 +39436 +39437 +39438 +39439 +39440 +39441 +39442 +39443 +39444 +39445 +39446 +39447 +39448 +39449 +39450 +39451 +39452 +39453 +39454 +39455 +39456 +39457 +39458 +39459 +39460 +39461 +39462 +39463 +39464 +39465 +39466 +39467 +39468 +39469 +39470 +39471 +39472 +39473 +39474 +39475 +39476 +39477 +39478 +39479 +39480 +39481 +39482 +39483 +39484 +39485 +39486 +39487 +39488 +39489 +39490 +39491 +39492 +39493 +39494 +39495 +39496 +39497 +39498 +39499 +39500 +39501 +39502 +39503 +39504 +39505 +39506 +39507 +39508 +39509 +39510 +39511 +39512 +39513 +39514 +39515 +39516 +39517 +39518 +39519 +39520 +39521 +39522 +39523 +39524 +39525 +39526 +39527 +39528 +39529 +39530 +39531 +39532 +39533 +39534 +39535 +39536 +39537 +39538 +39539 +39540 +39541 +39542 +39543 +39544 +39545 +39546 +39547 +39548 +39549 +39550 +39551 +39552 +39553 +39554 +39555 +39556 +39557 +39558 +39559 +39560 +39561 +39562 +39563 +39564 +39565 +39566 +39567 +39568 +39569 +39570 +39571 +39572 +39573 +39574 +39575 +39576 +39577 +39578 +39579 +39580 +39581 +39582 +39583 +39584 +39585 +39586 +39587 +39588 +39589 +39590 +39591 +39592 +39593 +39594 +39595 +39596 +39597 +39598 +39599 +39600 +39601 +39602 +39603 +39604 +39605 +39606 +39607 +39608 +39609 +39610 +39611 +39612 +39613 +39614 +39615 +39616 +39617 +39618 +39619 +39620 +39621 +39622 +39623 +39624 +39625 +39626 +39627 +39628 +39629 +39630 +39631 +39632 +39633 +39634 +39635 +39636 +39637 +39638 +39639 +39640 +39641 +39642 +39643 +39644 +39645 +39646 +39647 +39648 +39649 +39650 +39651 +39652 +39653 +39654 +39655 +39656 +39657 +39658 +39659 +39660 +39661 +39662 +39663 +39664 +39665 +39666 +39667 +39668 +39669 +39670 +39671 +39672 +39673 +39674 +39675 +39676 +39677 +39678 +39679 +39680 +39681 +39682 +39683 +39684 +39685 +39686 +39687 +39688 +39689 +39690 +39691 +39692 +39693 +39694 +39695 +39696 +39697 +39698 +39699 +39700 +39701 +39702 +39703 +39704 +39705 +39706 +39707 +39708 +39709 +39710 +39711 +39712 +39713 +39714 +39715 +39716 +39717 +39718 +39719 +39720 +39721 +39722 +39723 +39724 +39725 +39726 +39727 +39728 +39729 +39730 +39731 +39732 +39733 +39734 +39735 +39736 +39737 +39738 +39739 +39740 +39741 +39742 +39743 +39744 +39745 +39746 +39747 +39748 +39749 +39750 +39751 +39752 +39753 +39754 +39755 +39756 +39757 +39758 +39759 +39760 +39761 +39762 +39763 +39764 +39765 +39766 +39767 +39768 +39769 +39770 +39771 +39772 +39773 +39774 +39775 +39776 +39777 +39778 +39779 +39780 +39781 +39782 +39783 +39784 +39785 +39786 +39787 +39788 +39789 +39790 +39791 +39792 +39793 +39794 +39795 +39796 +39797 +39798 +39799 +39800 +39801 +39802 +39803 +39804 +39805 +39806 +39807 +39808 +39809 +39810 +39811 +39812 +39813 +39814 +39815 +39816 +39817 +39818 +39819 +39820 +39821 +39822 +39823 +39824 +39825 +39826 +39827 +39828 +39829 +39830 +39831 +39832 +39833 +39834 +39835 +39836 +39837 +39838 +39839 +39840 +39841 +39842 +39843 +39844 +39845 +39846 +39847 +39848 +39849 +39850 +39851 +39852 +39853 +39854 +39855 +39856 +39857 +39858 +39859 +39860 +39861 +39862 +39863 +39864 +39865 +39866 +39867 +39868 +39869 +39870 +39871 +39872 +39873 +39874 +39875 +39876 +39877 +39878 +39879 +39880 +39881 +39882 +39883 +39884 +39885 +39886 +39887 +39888 +39889 +39890 +39891 +39892 +39893 +39894 +39895 +39896 +39897 +39898 +39899 +39900 +39901 +39902 +39903 +39904 +39905 +39906 +39907 +39908 +39909 +39910 +39911 +39912 +39913 +39914 +39915 +39916 +39917 +39918 +39919 +39920 +39921 +39922 +39923 +39924 +39925 +39926 +39927 +39928 +39929 +39930 +39931 +39932 +39933 +39934 +39935 +39936 +39937 +39938 +39939 +39940 +39941 +39942 +39943 +39944 +39945 +39946 +39947 +39948 +39949 +39950 +39951 +39952 +39953 +39954 +39955 +39956 +39957 +39958 +39959 +39960 +39961 +39962 +39963 +39964 +39965 +39966 +39967 +39968 +39969 +39970 +39971 +39972 +39973 +39974 +39975 +39976 +39977 +39978 +39979 +39980 +39981 +39982 +39983 +39984 +39985 +39986 +39987 +39988 +39989 +39990 +39991 +39992 +39993 +39994 +39995 +39996 +39997 +39998 +39999 +40000 +40001 +40002 +40003 +40004 +40005 +40006 +40007 +40008 +40009 +40010 +40011 +40012 +40013 +40014 +40015 +40016 +40017 +40018 +40019 +40020 +40021 +40022 +40023 +40024 +40025 +40026 +40027 +40028 +40029 +40030 +40031 +40032 +40033 +40034 +40035 +40036 +40037 +40038 +40039 +40040 +40041 +40042 +40043 +40044 +40045 +40046 +40047 +40048 +40049 +40050 +40051 +40052 +40053 +40054 +40055 +40056 +40057 +40058 +40059 +40060 +40061 +40062 +40063 +40064 +40065 +40066 +40067 +40068 +40069 +40070 +40071 +40072 +40073 +40074 +40075 +40076 +40077 +40078 +40079 +40080 +40081 +40082 +40083 +40084 +40085 +40086 +40087 +40088 +40089 +40090 +40091 +40092 +40093 +40094 +40095 +40096 +40097 +40098 +40099 +40100 +40101 +40102 +40103 +40104 +40105 +40106 +40107 +40108 +40109 +40110 +40111 +40112 +40113 +40114 +40115 +40116 +40117 +40118 +40119 +40120 +40121 +40122 +40123 +40124 +40125 +40126 +40127 +40128 +40129 +40130 +40131 +40132 +40133 +40134 +40135 +40136 +40137 +40138 +40139 +40140 +40141 +40142 +40143 +40144 +40145 +40146 +40147 +40148 +40149 +40150 +40151 +40152 +40153 +40154 +40155 +40156 +40157 +40158 +40159 +40160 +40161 +40162 +40163 +40164 +40165 +40166 +40167 +40168 +40169 +40170 +40171 +40172 +40173 +40174 +40175 +40176 +40177 +40178 +40179 +40180 +40181 +40182 +40183 +40184 +40185 +40186 +40187 +40188 +40189 +40190 +40191 +40192 +40193 +40194 +40195 +40196 +40197 +40198 +40199 +40200 +40201 +40202 +40203 +40204 +40205 +40206 +40207 +40208 +40209 +40210 +40211 +40212 +40213 +40214 +40215 +40216 +40217 +40218 +40219 +40220 +40221 +40222 +40223 +40224 +40225 +40226 +40227 +40228 +40229 +40230 +40231 +40232 +40233 +40234 +40235 +40236 +40237 +40238 +40239 +40240 +40241 +40242 +40243 +40244 +40245 +40246 +40247 +40248 +40249 +40250 +40251 +40252 +40253 +40254 +40255 +40256 +40257 +40258 +40259 +40260 +40261 +40262 +40263 +40264 +40265 +40266 +40267 +40268 +40269 +40270 +40271 +40272 +40273 +40274 +40275 +40276 +40277 +40278 +40279 +40280 +40281 +40282 +40283 +40284 +40285 +40286 +40287 +40288 +40289 +40290 +40291 +40292 +40293 +40294 +40295 +40296 +40297 +40298 +40299 +40300 +40301 +40302 +40303 +40304 +40305 +40306 +40307 +40308 +40309 +40310 +40311 +40312 +40313 +40314 +40315 +40316 +40317 +40318 +40319 +40320 +40321 +40322 +40323 +40324 +40325 +40326 +40327 +40328 +40329 +40330 +40331 +40332 +40333 +40334 +40335 +40336 +40337 +40338 +40339 +40340 +40341 +40342 +40343 +40344 +40345 +40346 +40347 +40348 +40349 +40350 +40351 +40352 +40353 +40354 +40355 +40356 +40357 +40358 +40359 +40360 +40361 +40362 +40363 +40364 +40365 +40366 +40367 +40368 +40369 +40370 +40371 +40372 +40373 +40374 +40375 +40376 +40377 +40378 +40379 +40380 +40381 +40382 +40383 +40384 +40385 +40386 +40387 +40388 +40389 +40390 +40391 +40392 +40393 +40394 +40395 +40396 +40397 +40398 +40399 +40400 +40401 +40402 +40403 +40404 +40405 +40406 +40407 +40408 +40409 +40410 +40411 +40412 +40413 +40414 +40415 +40416 +40417 +40418 +40419 +40420 +40421 +40422 +40423 +40424 +40425 +40426 +40427 +40428 +40429 +40430 +40431 +40432 +40433 +40434 +40435 +40436 +40437 +40438 +40439 +40440 +40441 +40442 +40443 +40444 +40445 +40446 +40447 +40448 +40449 +40450 +40451 +40452 +40453 +40454 +40455 +40456 +40457 +40458 +40459 +40460 +40461 +40462 +40463 +40464 +40465 +40466 +40467 +40468 +40469 +40470 +40471 +40472 +40473 +40474 +40475 +40476 +40477 +40478 +40479 +40480 +40481 +40482 +40483 +40484 +40485 +40486 +40487 +40488 +40489 +40490 +40491 +40492 +40493 +40494 +40495 +40496 +40497 +40498 +40499 +40500 +40501 +40502 +40503 +40504 +40505 +40506 +40507 +40508 +40509 +40510 +40511 +40512 +40513 +40514 +40515 +40516 +40517 +40518 +40519 +40520 +40521 +40522 +40523 +40524 +40525 +40526 +40527 +40528 +40529 +40530 +40531 +40532 +40533 +40534 +40535 +40536 +40537 +40538 +40539 +40540 +40541 +40542 +40543 +40544 +40545 +40546 +40547 +40548 +40549 +40550 +40551 +40552 +40553 +40554 +40555 +40556 +40557 +40558 +40559 +40560 +40561 +40562 +40563 +40564 +40565 +40566 +40567 +40568 +40569 +40570 +40571 +40572 +40573 +40574 +40575 +40576 +40577 +40578 +40579 +40580 +40581 +40582 +40583 +40584 +40585 +40586 +40587 +40588 +40589 +40590 +40591 +40592 +40593 +40594 +40595 +40596 +40597 +40598 +40599 +40600 +40601 +40602 +40603 +40604 +40605 +40606 +40607 +40608 +40609 +40610 +40611 +40612 +40613 +40614 +40615 +40616 +40617 +40618 +40619 +40620 +40621 +40622 +40623 +40624 +40625 +40626 +40627 +40628 +40629 +40630 +40631 +40632 +40633 +40634 +40635 +40636 +40637 +40638 +40639 +40640 +40641 +40642 +40643 +40644 +40645 +40646 +40647 +40648 +40649 +40650 +40651 +40652 +40653 +40654 +40655 +40656 +40657 +40658 +40659 +40660 +40661 +40662 +40663 +40664 +40665 +40666 +40667 +40668 +40669 +40670 +40671 +40672 +40673 +40674 +40675 +40676 +40677 +40678 +40679 +40680 +40681 +40682 +40683 +40684 +40685 +40686 +40687 +40688 +40689 +40690 +40691 +40692 +40693 +40694 +40695 +40696 +40697 +40698 +40699 +40700 +40701 +40702 +40703 +40704 +40705 +40706 +40707 +40708 +40709 +40710 +40711 +40712 +40713 +40714 +40715 +40716 +40717 +40718 +40719 +40720 +40721 +40722 +40723 +40724 +40725 +40726 +40727 +40728 +40729 +40730 +40731 +40732 +40733 +40734 +40735 +40736 +40737 +40738 +40739 +40740 +40741 +40742 +40743 +40744 +40745 +40746 +40747 +40748 +40749 +40750 +40751 +40752 +40753 +40754 +40755 +40756 +40757 +40758 +40759 +40760 +40761 +40762 +40763 +40764 +40765 +40766 +40767 +40768 +40769 +40770 +40771 +40772 +40773 +40774 +40775 +40776 +40777 +40778 +40779 +40780 +40781 +40782 +40783 +40784 +40785 +40786 +40787 +40788 +40789 +40790 +40791 +40792 +40793 +40794 +40795 +40796 +40797 +40798 +40799 +40800 +40801 +40802 +40803 +40804 +40805 +40806 +40807 +40808 +40809 +40810 +40811 +40812 +40813 +40814 +40815 +40816 +40817 +40818 +40819 +40820 +40821 +40822 +40823 +40824 +40825 +40826 +40827 +40828 +40829 +40830 +40831 +40832 +40833 +40834 +40835 +40836 +40837 +40838 +40839 +40840 +40841 +40842 +40843 +40844 +40845 +40846 +40847 +40848 +40849 +40850 +40851 +40852 +40853 +40854 +40855 +40856 +40857 +40858 +40859 +40860 +40861 +40862 +40863 +40864 +40865 +40866 +40867 +40868 +40869 +40870 +40871 +40872 +40873 +40874 +40875 +40876 +40877 +40878 +40879 +40880 +40881 +40882 +40883 +40884 +40885 +40886 +40887 +40888 +40889 +40890 +40891 +40892 +40893 +40894 +40895 +40896 +40897 +40898 +40899 +40900 +40901 +40902 +40903 +40904 +40905 +40906 +40907 +40908 +40909 +40910 +40911 +40912 +40913 +40914 +40915 +40916 +40917 +40918 +40919 +40920 +40921 +40922 +40923 +40924 +40925 +40926 +40927 +40928 +40929 +40930 +40931 +40932 +40933 +40934 +40935 +40936 +40937 +40938 +40939 +40940 +40941 +40942 +40943 +40944 +40945 +40946 +40947 +40948 +40949 +40950 +40951 +40952 +40953 +40954 +40955 +40956 +40957 +40958 +40959 +40960 +40961 +40962 +40963 +40964 +40965 +40966 +40967 +40968 +40969 +40970 +40971 +40972 +40973 +40974 +40975 +40976 +40977 +40978 +40979 +40980 +40981 +40982 +40983 +40984 +40985 +40986 +40987 +40988 +40989 +40990 +40991 +40992 +40993 +40994 +40995 +40996 +40997 +40998 +40999 +41000 +41001 +41002 +41003 +41004 +41005 +41006 +41007 +41008 +41009 +41010 +41011 +41012 +41013 +41014 +41015 +41016 +41017 +41018 +41019 +41020 +41021 +41022 +41023 +41024 +41025 +41026 +41027 +41028 +41029 +41030 +41031 +41032 +41033 +41034 +41035 +41036 +41037 +41038 +41039 +41040 +41041 +41042 +41043 +41044 +41045 +41046 +41047 +41048 +41049 +41050 +41051 +41052 +41053 +41054 +41055 +41056 +41057 +41058 +41059 +41060 +41061 +41062 +41063 +41064 +41065 +41066 +41067 +41068 +41069 +41070 +41071 +41072 +41073 +41074 +41075 +41076 +41077 +41078 +41079 +41080 +41081 +41082 +41083 +41084 +41085 +41086 +41087 +41088 +41089 +41090 +41091 +41092 +41093 +41094 +41095 +41096 +41097 +41098 +41099 +41100 +41101 +41102 +41103 +41104 +41105 +41106 +41107 +41108 +41109 +41110 +41111 +41112 +41113 +41114 +41115 +41116 +41117 +41118 +41119 +41120 +41121 +41122 +41123 +41124 +41125 +41126 +41127 +41128 +41129 +41130 +41131 +41132 +41133 +41134 +41135 +41136 +41137 +41138 +41139 +41140 +41141 +41142 +41143 +41144 +41145 +41146 +41147 +41148 +41149 +41150 +41151 +41152 +41153 +41154 +41155 +41156 +41157 +41158 +41159 +41160 +41161 +41162 +41163 +41164 +41165 +41166 +41167 +41168 +41169 +41170 +41171 +41172 +41173 +41174 +41175 +41176 +41177 +41178 +41179 +41180 +41181 +41182 +41183 +41184 +41185 +41186 +41187 +41188 +41189 +41190 +41191 +41192 +41193 +41194 +41195 +41196 +41197 +41198 +41199 +41200 +41201 +41202 +41203 +41204 +41205 +41206 +41207 +41208 +41209 +41210 +41211 +41212 +41213 +41214 +41215 +41216 +41217 +41218 +41219 +41220 +41221 +41222 +41223 +41224 +41225 +41226 +41227 +41228 +41229 +41230 +41231 +41232 +41233 +41234 +41235 +41236 +41237 +41238 +41239 +41240 +41241 +41242 +41243 +41244 +41245 +41246 +41247 +41248 +41249 +41250 +41251 +41252 +41253 +41254 +41255 +41256 +41257 +41258 +41259 +41260 +41261 +41262 +41263 +41264 +41265 +41266 +41267 +41268 +41269 +41270 +41271 +41272 +41273 +41274 +41275 +41276 +41277 +41278 +41279 +41280 +41281 +41282 +41283 +41284 +41285 +41286 +41287 +41288 +41289 +41290 +41291 +41292 +41293 +41294 +41295 +41296 +41297 +41298 +41299 +41300 +41301 +41302 +41303 +41304 +41305 +41306 +41307 +41308 +41309 +41310 +41311 +41312 +41313 +41314 +41315 +41316 +41317 +41318 +41319 +41320 +41321 +41322 +41323 +41324 +41325 +41326 +41327 +41328 +41329 +41330 +41331 +41332 +41333 +41334 +41335 +41336 +41337 +41338 +41339 +41340 +41341 +41342 +41343 +41344 +41345 +41346 +41347 +41348 +41349 +41350 +41351 +41352 +41353 +41354 +41355 +41356 +41357 +41358 +41359 +41360 +41361 +41362 +41363 +41364 +41365 +41366 +41367 +41368 +41369 +41370 +41371 +41372 +41373 +41374 +41375 +41376 +41377 +41378 +41379 +41380 +41381 +41382 +41383 +41384 +41385 +41386 +41387 +41388 +41389 +41390 +41391 +41392 +41393 +41394 +41395 +41396 +41397 +41398 +41399 +41400 +41401 +41402 +41403 +41404 +41405 +41406 +41407 +41408 +41409 +41410 +41411 +41412 +41413 +41414 +41415 +41416 +41417 +41418 +41419 +41420 +41421 +41422 +41423 +41424 +41425 +41426 +41427 +41428 +41429 +41430 +41431 +41432 +41433 +41434 +41435 +41436 +41437 +41438 +41439 +41440 +41441 +41442 +41443 +41444 +41445 +41446 +41447 +41448 +41449 +41450 +41451 +41452 +41453 +41454 +41455 +41456 +41457 +41458 +41459 +41460 +41461 +41462 +41463 +41464 +41465 +41466 +41467 +41468 +41469 +41470 +41471 +41472 +41473 +41474 +41475 +41476 +41477 +41478 +41479 +41480 +41481 +41482 +41483 +41484 +41485 +41486 +41487 +41488 +41489 +41490 +41491 +41492 +41493 +41494 +41495 +41496 +41497 +41498 +41499 +41500 +41501 +41502 +41503 +41504 +41505 +41506 +41507 +41508 +41509 +41510 +41511 +41512 +41513 +41514 +41515 +41516 +41517 +41518 +41519 +41520 +41521 +41522 +41523 +41524 +41525 +41526 +41527 +41528 +41529 +41530 +41531 +41532 +41533 +41534 +41535 +41536 +41537 +41538 +41539 +41540 +41541 +41542 +41543 +41544 +41545 +41546 +41547 +41548 +41549 +41550 +41551 +41552 +41553 +41554 +41555 +41556 +41557 +41558 +41559 +41560 +41561 +41562 +41563 +41564 +41565 +41566 +41567 +41568 +41569 +41570 +41571 +41572 +41573 +41574 +41575 +41576 +41577 +41578 +41579 +41580 +41581 +41582 +41583 +41584 +41585 +41586 +41587 +41588 +41589 +41590 +41591 +41592 +41593 +41594 +41595 +41596 +41597 +41598 +41599 +41600 +41601 +41602 +41603 +41604 +41605 +41606 +41607 +41608 +41609 +41610 +41611 +41612 +41613 +41614 +41615 +41616 +41617 +41618 +41619 +41620 +41621 +41622 +41623 +41624 +41625 +41626 +41627 +41628 +41629 +41630 +41631 +41632 +41633 +41634 +41635 +41636 +41637 +41638 +41639 +41640 +41641 +41642 +41643 +41644 +41645 +41646 +41647 +41648 +41649 +41650 +41651 +41652 +41653 +41654 +41655 +41656 +41657 +41658 +41659 +41660 +41661 +41662 +41663 +41664 +41665 +41666 +41667 +41668 +41669 +41670 +41671 +41672 +41673 +41674 +41675 +41676 +41677 +41678 +41679 +41680 +41681 +41682 +41683 +41684 +41685 +41686 +41687 +41688 +41689 +41690 +41691 +41692 +41693 +41694 +41695 +41696 +41697 +41698 +41699 +41700 +41701 +41702 +41703 +41704 +41705 +41706 +41707 +41708 +41709 +41710 +41711 +41712 +41713 +41714 +41715 +41716 +41717 +41718 +41719 +41720 +41721 +41722 +41723 +41724 +41725 +41726 +41727 +41728 +41729 +41730 +41731 +41732 +41733 +41734 +41735 +41736 +41737 +41738 +41739 +41740 +41741 +41742 +41743 +41744 +41745 +41746 +41747 +41748 +41749 +41750 +41751 +41752 +41753 +41754 +41755 +41756 +41757 +41758 +41759 +41760 +41761 +41762 +41763 +41764 +41765 +41766 +41767 +41768 +41769 +41770 +41771 +41772 +41773 +41774 +41775 +41776 +41777 +41778 +41779 +41780 +41781 +41782 +41783 +41784 +41785 +41786 +41787 +41788 +41789 +41790 +41791 +41792 +41793 +41794 +41795 +41796 +41797 +41798 +41799 +41800 +41801 +41802 +41803 +41804 +41805 +41806 +41807 +41808 +41809 +41810 +41811 +41812 +41813 +41814 +41815 +41816 +41817 +41818 +41819 +41820 +41821 +41822 +41823 +41824 +41825 +41826 +41827 +41828 +41829 +41830 +41831 +41832 +41833 +41834 +41835 +41836 +41837 +41838 +41839 +41840 +41841 +41842 +41843 +41844 +41845 +41846 +41847 +41848 +41849 +41850 +41851 +41852 +41853 +41854 +41855 +41856 +41857 +41858 +41859 +41860 +41861 +41862 +41863 +41864 +41865 +41866 +41867 +41868 +41869 +41870 +41871 +41872 +41873 +41874 +41875 +41876 +41877 +41878 +41879 +41880 +41881 +41882 +41883 +41884 +41885 +41886 +41887 +41888 +41889 +41890 +41891 +41892 +41893 +41894 +41895 +41896 +41897 +41898 +41899 +41900 +41901 +41902 +41903 +41904 +41905 +41906 +41907 +41908 +41909 +41910 +41911 +41912 +41913 +41914 +41915 +41916 +41917 +41918 +41919 +41920 +41921 +41922 +41923 +41924 +41925 +41926 +41927 +41928 +41929 +41930 +41931 +41932 +41933 +41934 +41935 +41936 +41937 +41938 +41939 +41940 +41941 +41942 +41943 +41944 +41945 +41946 +41947 +41948 +41949 +41950 +41951 +41952 +41953 +41954 +41955 +41956 +41957 +41958 +41959 +41960 +41961 +41962 +41963 +41964 +41965 +41966 +41967 +41968 +41969 +41970 +41971 +41972 +41973 +41974 +41975 +41976 +41977 +41978 +41979 +41980 +41981 +41982 +41983 +41984 +41985 +41986 +41987 +41988 +41989 +41990 +41991 +41992 +41993 +41994 +41995 +41996 +41997 +41998 +41999 +42000 +42001 +42002 +42003 +42004 +42005 +42006 +42007 +42008 +42009 +42010 +42011 +42012 +42013 +42014 +42015 +42016 +42017 +42018 +42019 +42020 +42021 +42022 +42023 +42024 +42025 +42026 +42027 +42028 +42029 +42030 +42031 +42032 +42033 +42034 +42035 +42036 +42037 +42038 +42039 +42040 +42041 +42042 +42043 +42044 +42045 +42046 +42047 +42048 +42049 +42050 +42051 +42052 +42053 +42054 +42055 +42056 +42057 +42058 +42059 +42060 +42061 +42062 +42063 +42064 +42065 +42066 +42067 +42068 +42069 +42070 +42071 +42072 +42073 +42074 +42075 +42076 +42077 +42078 +42079 +42080 +42081 +42082 +42083 +42084 +42085 +42086 +42087 +42088 +42089 +42090 +42091 +42092 +42093 +42094 +42095 +42096 +42097 +42098 +42099 +42100 +42101 +42102 +42103 +42104 +42105 +42106 +42107 +42108 +42109 +42110 +42111 +42112 +42113 +42114 +42115 +42116 +42117 +42118 +42119 +42120 +42121 +42122 +42123 +42124 +42125 +42126 +42127 +42128 +42129 +42130 +42131 +42132 +42133 +42134 +42135 +42136 +42137 +42138 +42139 +42140 +42141 +42142 +42143 +42144 +42145 +42146 +42147 +42148 +42149 +42150 +42151 +42152 +42153 +42154 +42155 +42156 +42157 +42158 +42159 +42160 +42161 +42162 +42163 +42164 +42165 +42166 +42167 +42168 +42169 +42170 +42171 +42172 +42173 +42174 +42175 +42176 +42177 +42178 +42179 +42180 +42181 +42182 +42183 +42184 +42185 +42186 +42187 +42188 +42189 +42190 +42191 +42192 +42193 +42194 +42195 +42196 +42197 +42198 +42199 +42200 +42201 +42202 +42203 +42204 +42205 +42206 +42207 +42208 +42209 +42210 +42211 +42212 +42213 +42214 +42215 +42216 +42217 +42218 +42219 +42220 +42221 +42222 +42223 +42224 +42225 +42226 +42227 +42228 +42229 +42230 +42231 +42232 +42233 +42234 +42235 +42236 +42237 +42238 +42239 +42240 +42241 +42242 +42243 +42244 +42245 +42246 +42247 +42248 +42249 +42250 +42251 +42252 +42253 +42254 +42255 +42256 +42257 +42258 +42259 +42260 +42261 +42262 +42263 +42264 +42265 +42266 +42267 +42268 +42269 +42270 +42271 +42272 +42273 +42274 +42275 +42276 +42277 +42278 +42279 +42280 +42281 +42282 +42283 +42284 +42285 +42286 +42287 +42288 +42289 +42290 +42291 +42292 +42293 +42294 +42295 +42296 +42297 +42298 +42299 +42300 +42301 +42302 +42303 +42304 +42305 +42306 +42307 +42308 +42309 +42310 +42311 +42312 +42313 +42314 +42315 +42316 +42317 +42318 +42319 +42320 +42321 +42322 +42323 +42324 +42325 +42326 +42327 +42328 +42329 +42330 +42331 +42332 +42333 +42334 +42335 +42336 +42337 +42338 +42339 +42340 +42341 +42342 +42343 +42344 +42345 +42346 +42347 +42348 +42349 +42350 +42351 +42352 +42353 +42354 +42355 +42356 +42357 +42358 +42359 +42360 +42361 +42362 +42363 +42364 +42365 +42366 +42367 +42368 +42369 +42370 +42371 +42372 +42373 +42374 +42375 +42376 +42377 +42378 +42379 +42380 +42381 +42382 +42383 +42384 +42385 +42386 +42387 +42388 +42389 +42390 +42391 +42392 +42393 +42394 +42395 +42396 +42397 +42398 +42399 +42400 +42401 +42402 +42403 +42404 +42405 +42406 +42407 +42408 +42409 +42410 +42411 +42412 +42413 +42414 +42415 +42416 +42417 +42418 +42419 +42420 +42421 +42422 +42423 +42424 +42425 +42426 +42427 +42428 +42429 +42430 +42431 +42432 +42433 +42434 +42435 +42436 +42437 +42438 +42439 +42440 +42441 +42442 +42443 +42444 +42445 +42446 +42447 +42448 +42449 +42450 +42451 +42452 +42453 +42454 +42455 +42456 +42457 +42458 +42459 +42460 +42461 +42462 +42463 +42464 +42465 +42466 +42467 +42468 +42469 +42470 +42471 +42472 +42473 +42474 +42475 +42476 +42477 +42478 +42479 +42480 +42481 +42482 +42483 +42484 +42485 +42486 +42487 +42488 +42489 +42490 +42491 +42492 +42493 +42494 +42495 +42496 +42497 +42498 +42499 +42500 +42501 +42502 +42503 +42504 +42505 +42506 +42507 +42508 +42509 +42510 +42511 +42512 +42513 +42514 +42515 +42516 +42517 +42518 +42519 +42520 +42521 +42522 +42523 +42524 +42525 +42526 +42527 +42528 +42529 +42530 +42531 +42532 +42533 +42534 +42535 +42536 +42537 +42538 +42539 +42540 +42541 +42542 +42543 +42544 +42545 +42546 +42547 +42548 +42549 +42550 +42551 +42552 +42553 +42554 +42555 +42556 +42557 +42558 +42559 +42560 +42561 +42562 +42563 +42564 +42565 +42566 +42567 +42568 +42569 +42570 +42571 +42572 +42573 +42574 +42575 +42576 +42577 +42578 +42579 +42580 +42581 +42582 +42583 +42584 +42585 +42586 +42587 +42588 +42589 +42590 +42591 +42592 +42593 +42594 +42595 +42596 +42597 +42598 +42599 +42600 +42601 +42602 +42603 +42604 +42605 +42606 +42607 +42608 +42609 +42610 +42611 +42612 +42613 +42614 +42615 +42616 +42617 +42618 +42619 +42620 +42621 +42622 +42623 +42624 +42625 +42626 +42627 +42628 +42629 +42630 +42631 +42632 +42633 +42634 +42635 +42636 +42637 +42638 +42639 +42640 +42641 +42642 +42643 +42644 +42645 +42646 +42647 +42648 +42649 +42650 +42651 +42652 +42653 +42654 +42655 +42656 +42657 +42658 +42659 +42660 +42661 +42662 +42663 +42664 +42665 +42666 +42667 +42668 +42669 +42670 +42671 +42672 +42673 +42674 +42675 +42676 +42677 +42678 +42679 +42680 +42681 +42682 +42683 +42684 +42685 +42686 +42687 +42688 +42689 +42690 +42691 +42692 +42693 +42694 +42695 +42696 +42697 +42698 +42699 +42700 +42701 +42702 +42703 +42704 +42705 +42706 +42707 +42708 +42709 +42710 +42711 +42712 +42713 +42714 +42715 +42716 +42717 +42718 +42719 +42720 +42721 +42722 +42723 +42724 +42725 +42726 +42727 +42728 +42729 +42730 +42731 +42732 +42733 +42734 +42735 +42736 +42737 +42738 +42739 +42740 +42741 +42742 +42743 +42744 +42745 +42746 +42747 +42748 +42749 +42750 +42751 +42752 +42753 +42754 +42755 +42756 +42757 +42758 +42759 +42760 +42761 +42762 +42763 +42764 +42765 +42766 +42767 +42768 +42769 +42770 +42771 +42772 +42773 +42774 +42775 +42776 +42777 +42778 +42779 +42780 +42781 +42782 +42783 +42784 +42785 +42786 +42787 +42788 +42789 +42790 +42791 +42792 +42793 +42794 +42795 +42796 +42797 +42798 +42799 +42800 +42801 +42802 +42803 +42804 +42805 +42806 +42807 +42808 +42809 +42810 +42811 +42812 +42813 +42814 +42815 +42816 +42817 +42818 +42819 +42820 +42821 +42822 +42823 +42824 +42825 +42826 +42827 +42828 +42829 +42830 +42831 +42832 +42833 +42834 +42835 +42836 +42837 +42838 +42839 +42840 +42841 +42842 +42843 +42844 +42845 +42846 +42847 +42848 +42849 +42850 +42851 +42852 +42853 +42854 +42855 +42856 +42857 +42858 +42859 +42860 +42861 +42862 +42863 +42864 +42865 +42866 +42867 +42868 +42869 +42870 +42871 +42872 +42873 +42874 +42875 +42876 +42877 +42878 +42879 +42880 +42881 +42882 +42883 +42884 +42885 +42886 +42887 +42888 +42889 +42890 +42891 +42892 +42893 +42894 +42895 +42896 +42897 +42898 +42899 +42900 +42901 +42902 +42903 +42904 +42905 +42906 +42907 +42908 +42909 +42910 +42911 +42912 +42913 +42914 +42915 +42916 +42917 +42918 +42919 +42920 +42921 +42922 +42923 +42924 +42925 +42926 +42927 +42928 +42929 +42930 +42931 +42932 +42933 +42934 +42935 +42936 +42937 +42938 +42939 +42940 +42941 +42942 +42943 +42944 +42945 +42946 +42947 +42948 +42949 +42950 +42951 +42952 +42953 +42954 +42955 +42956 +42957 +42958 +42959 +42960 +42961 +42962 +42963 +42964 +42965 +42966 +42967 +42968 +42969 +42970 +42971 +42972 +42973 +42974 +42975 +42976 +42977 +42978 +42979 +42980 +42981 +42982 +42983 +42984 +42985 +42986 +42987 +42988 +42989 +42990 +42991 +42992 +42993 +42994 +42995 +42996 +42997 +42998 +42999 +43000 +43001 +43002 +43003 +43004 +43005 +43006 +43007 +43008 +43009 +43010 +43011 +43012 +43013 +43014 +43015 +43016 +43017 +43018 +43019 +43020 +43021 +43022 +43023 +43024 +43025 +43026 +43027 +43028 +43029 +43030 +43031 +43032 +43033 +43034 +43035 +43036 +43037 +43038 +43039 +43040 +43041 +43042 +43043 +43044 +43045 +43046 +43047 +43048 +43049 +43050 +43051 +43052 +43053 +43054 +43055 +43056 +43057 +43058 +43059 +43060 +43061 +43062 +43063 +43064 +43065 +43066 +43067 +43068 +43069 +43070 +43071 +43072 +43073 +43074 +43075 +43076 +43077 +43078 +43079 +43080 +43081 +43082 +43083 +43084 +43085 +43086 +43087 +43088 +43089 +43090 +43091 +43092 +43093 +43094 +43095 +43096 +43097 +43098 +43099 +43100 +43101 +43102 +43103 +43104 +43105 +43106 +43107 +43108 +43109 +43110 +43111 +43112 +43113 +43114 +43115 +43116 +43117 +43118 +43119 +43120 +43121 +43122 +43123 +43124 +43125 +43126 +43127 +43128 +43129 +43130 +43131 +43132 +43133 +43134 +43135 +43136 +43137 +43138 +43139 +43140 +43141 +43142 +43143 +43144 +43145 +43146 +43147 +43148 +43149 +43150 +43151 +43152 +43153 +43154 +43155 +43156 +43157 +43158 +43159 +43160 +43161 +43162 +43163 +43164 +43165 +43166 +43167 +43168 +43169 +43170 +43171 +43172 +43173 +43174 +43175 +43176 +43177 +43178 +43179 +43180 +43181 +43182 +43183 +43184 +43185 +43186 +43187 +43188 +43189 +43190 +43191 +43192 +43193 +43194 +43195 +43196 +43197 +43198 +43199 +43200 +43201 +43202 +43203 +43204 +43205 +43206 +43207 +43208 +43209 +43210 +43211 +43212 +43213 +43214 +43215 +43216 +43217 +43218 +43219 +43220 +43221 +43222 +43223 +43224 +43225 +43226 +43227 +43228 +43229 +43230 +43231 +43232 +43233 +43234 +43235 +43236 +43237 +43238 +43239 +43240 +43241 +43242 +43243 +43244 +43245 +43246 +43247 +43248 +43249 +43250 +43251 +43252 +43253 +43254 +43255 +43256 +43257 +43258 +43259 +43260 +43261 +43262 +43263 +43264 +43265 +43266 +43267 +43268 +43269 +43270 +43271 +43272 +43273 +43274 +43275 +43276 +43277 +43278 +43279 +43280 +43281 +43282 +43283 +43284 +43285 +43286 +43287 +43288 +43289 +43290 +43291 +43292 +43293 +43294 +43295 +43296 +43297 +43298 +43299 +43300 +43301 +43302 +43303 +43304 +43305 +43306 +43307 +43308 +43309 +43310 +43311 +43312 +43313 +43314 +43315 +43316 +43317 +43318 +43319 +43320 +43321 +43322 +43323 +43324 +43325 +43326 +43327 +43328 +43329 +43330 +43331 +43332 +43333 +43334 +43335 +43336 +43337 +43338 +43339 +43340 +43341 +43342 +43343 +43344 +43345 +43346 +43347 +43348 +43349 +43350 +43351 +43352 +43353 +43354 +43355 +43356 +43357 +43358 +43359 +43360 +43361 +43362 +43363 +43364 +43365 +43366 +43367 +43368 +43369 +43370 +43371 +43372 +43373 +43374 +43375 +43376 +43377 +43378 +43379 +43380 +43381 +43382 +43383 +43384 +43385 +43386 +43387 +43388 +43389 +43390 +43391 +43392 +43393 +43394 +43395 +43396 +43397 +43398 +43399 +43400 +43401 +43402 +43403 +43404 +43405 +43406 +43407 +43408 +43409 +43410 +43411 +43412 +43413 +43414 +43415 +43416 +43417 +43418 +43419 +43420 +43421 +43422 +43423 +43424 +43425 +43426 +43427 +43428 +43429 +43430 +43431 +43432 +43433 +43434 +43435 +43436 +43437 +43438 +43439 +43440 +43441 +43442 +43443 +43444 +43445 +43446 +43447 +43448 +43449 +43450 +43451 +43452 +43453 +43454 +43455 +43456 +43457 +43458 +43459 +43460 +43461 +43462 +43463 +43464 +43465 +43466 +43467 +43468 +43469 +43470 +43471 +43472 +43473 +43474 +43475 +43476 +43477 +43478 +43479 +43480 +43481 +43482 +43483 +43484 +43485 +43486 +43487 +43488 +43489 +43490 +43491 +43492 +43493 +43494 +43495 +43496 +43497 +43498 +43499 +43500 +43501 +43502 +43503 +43504 +43505 +43506 +43507 +43508 +43509 +43510 +43511 +43512 +43513 +43514 +43515 +43516 +43517 +43518 +43519 +43520 +43521 +43522 +43523 +43524 +43525 +43526 +43527 +43528 +43529 +43530 +43531 +43532 +43533 +43534 +43535 +43536 +43537 +43538 +43539 +43540 +43541 +43542 +43543 +43544 +43545 +43546 +43547 +43548 +43549 +43550 +43551 +43552 +43553 +43554 +43555 +43556 +43557 +43558 +43559 +43560 +43561 +43562 +43563 +43564 +43565 +43566 +43567 +43568 +43569 +43570 +43571 +43572 +43573 +43574 +43575 +43576 +43577 +43578 +43579 +43580 +43581 +43582 +43583 +43584 +43585 +43586 +43587 +43588 +43589 +43590 +43591 +43592 +43593 +43594 +43595 +43596 +43597 +43598 +43599 +43600 +43601 +43602 +43603 +43604 +43605 +43606 +43607 +43608 +43609 +43610 +43611 +43612 +43613 +43614 +43615 +43616 +43617 +43618 +43619 +43620 +43621 +43622 +43623 +43624 +43625 +43626 +43627 +43628 +43629 +43630 +43631 +43632 +43633 +43634 +43635 +43636 +43637 +43638 +43639 +43640 +43641 +43642 +43643 +43644 +43645 +43646 +43647 +43648 +43649 +43650 +43651 +43652 +43653 +43654 +43655 +43656 +43657 +43658 +43659 +43660 +43661 +43662 +43663 +43664 +43665 +43666 +43667 +43668 +43669 +43670 +43671 +43672 +43673 +43674 +43675 +43676 +43677 +43678 +43679 +43680 +43681 +43682 +43683 +43684 +43685 +43686 +43687 +43688 +43689 +43690 +43691 +43692 +43693 +43694 +43695 +43696 +43697 +43698 +43699 +43700 +43701 +43702 +43703 +43704 +43705 +43706 +43707 +43708 +43709 +43710 +43711 +43712 +43713 +43714 +43715 +43716 +43717 +43718 +43719 +43720 +43721 +43722 +43723 +43724 +43725 +43726 +43727 +43728 +43729 +43730 +43731 +43732 +43733 +43734 +43735 +43736 +43737 +43738 +43739 +43740 +43741 +43742 +43743 +43744 +43745 +43746 +43747 +43748 +43749 +43750 +43751 +43752 +43753 +43754 +43755 +43756 +43757 +43758 +43759 +43760 +43761 +43762 +43763 +43764 +43765 +43766 +43767 +43768 +43769 +43770 +43771 +43772 +43773 +43774 +43775 +43776 +43777 +43778 +43779 +43780 +43781 +43782 +43783 +43784 +43785 +43786 +43787 +43788 +43789 +43790 +43791 +43792 +43793 +43794 +43795 +43796 +43797 +43798 +43799 +43800 +43801 +43802 +43803 +43804 +43805 +43806 +43807 +43808 +43809 +43810 +43811 +43812 +43813 +43814 +43815 +43816 +43817 +43818 +43819 +43820 +43821 +43822 +43823 +43824 +43825 +43826 +43827 +43828 +43829 +43830 +43831 +43832 +43833 +43834 +43835 +43836 +43837 +43838 +43839 +43840 +43841 +43842 +43843 +43844 +43845 +43846 +43847 +43848 +43849 +43850 +43851 +43852 +43853 +43854 +43855 +43856 +43857 +43858 +43859 +43860 +43861 +43862 +43863 +43864 +43865 +43866 +43867 +43868 +43869 +43870 +43871 +43872 +43873 +43874 +43875 +43876 +43877 +43878 +43879 +43880 +43881 +43882 +43883 +43884 +43885 +43886 +43887 +43888 +43889 +43890 +43891 +43892 +43893 +43894 +43895 +43896 +43897 +43898 +43899 +43900 +43901 +43902 +43903 +43904 +43905 +43906 +43907 +43908 +43909 +43910 +43911 +43912 +43913 +43914 +43915 +43916 +43917 +43918 +43919 +43920 +43921 +43922 +43923 +43924 +43925 +43926 +43927 +43928 +43929 +43930 +43931 +43932 +43933 +43934 +43935 +43936 +43937 +43938 +43939 +43940 +43941 +43942 +43943 +43944 +43945 +43946 +43947 +43948 +43949 +43950 +43951 +43952 +43953 +43954 +43955 +43956 +43957 +43958 +43959 +43960 +43961 +43962 +43963 +43964 +43965 +43966 +43967 +43968 +43969 +43970 +43971 +43972 +43973 +43974 +43975 +43976 +43977 +43978 +43979 +43980 +43981 +43982 +43983 +43984 +43985 +43986 +43987 +43988 +43989 +43990 +43991 +43992 +43993 +43994 +43995 +43996 +43997 +43998 +43999 +44000 +44001 +44002 +44003 +44004 +44005 +44006 +44007 +44008 +44009 +44010 +44011 +44012 +44013 +44014 +44015 +44016 +44017 +44018 +44019 +44020 +44021 +44022 +44023 +44024 +44025 +44026 +44027 +44028 +44029 +44030 +44031 +44032 +44033 +44034 +44035 +44036 +44037 +44038 +44039 +44040 +44041 +44042 +44043 +44044 +44045 +44046 +44047 +44048 +44049 +44050 +44051 +44052 +44053 +44054 +44055 +44056 +44057 +44058 +44059 +44060 +44061 +44062 +44063 +44064 +44065 +44066 +44067 +44068 +44069 +44070 +44071 +44072 +44073 +44074 +44075 +44076 +44077 +44078 +44079 +44080 +44081 +44082 +44083 +44084 +44085 +44086 +44087 +44088 +44089 +44090 +44091 +44092 +44093 +44094 +44095 +44096 +44097 +44098 +44099 +44100 +44101 +44102 +44103 +44104 +44105 +44106 +44107 +44108 +44109 +44110 +44111 +44112 +44113 +44114 +44115 +44116 +44117 +44118 +44119 +44120 +44121 +44122 +44123 +44124 +44125 +44126 +44127 +44128 +44129 +44130 +44131 +44132 +44133 +44134 +44135 +44136 +44137 +44138 +44139 +44140 +44141 +44142 +44143 +44144 +44145 +44146 +44147 +44148 +44149 +44150 +44151 +44152 +44153 +44154 +44155 +44156 +44157 +44158 +44159 +44160 +44161 +44162 +44163 +44164 +44165 +44166 +44167 +44168 +44169 +44170 +44171 +44172 +44173 +44174 +44175 +44176 +44177 +44178 +44179 +44180 +44181 +44182 +44183 +44184 +44185 +44186 +44187 +44188 +44189 +44190 +44191 +44192 +44193 +44194 +44195 +44196 +44197 +44198 +44199 +44200 +44201 +44202 +44203 +44204 +44205 +44206 +44207 +44208 +44209 +44210 +44211 +44212 +44213 +44214 +44215 +44216 +44217 +44218 +44219 +44220 +44221 +44222 +44223 +44224 +44225 +44226 +44227 +44228 +44229 +44230 +44231 +44232 +44233 +44234 +44235 +44236 +44237 +44238 +44239 +44240 +44241 +44242 +44243 +44244 +44245 +44246 +44247 +44248 +44249 +44250 +44251 +44252 +44253 +44254 +44255 +44256 +44257 +44258 +44259 +44260 +44261 +44262 +44263 +44264 +44265 +44266 +44267 +44268 +44269 +44270 +44271 +44272 +44273 +44274 +44275 +44276 +44277 +44278 +44279 +44280 +44281 +44282 +44283 +44284 +44285 +44286 +44287 +44288 +44289 +44290 +44291 +44292 +44293 +44294 +44295 +44296 +44297 +44298 +44299 +44300 +44301 +44302 +44303 +44304 +44305 +44306 +44307 +44308 +44309 +44310 +44311 +44312 +44313 +44314 +44315 +44316 +44317 +44318 +44319 +44320 +44321 +44322 +44323 +44324 +44325 +44326 +44327 +44328 +44329 +44330 +44331 +44332 +44333 +44334 +44335 +44336 +44337 +44338 +44339 +44340 +44341 +44342 +44343 +44344 +44345 +44346 +44347 +44348 +44349 +44350 +44351 +44352 +44353 +44354 +44355 +44356 +44357 +44358 +44359 +44360 +44361 +44362 +44363 +44364 +44365 +44366 +44367 +44368 +44369 +44370 +44371 +44372 +44373 +44374 +44375 +44376 +44377 +44378 +44379 +44380 +44381 +44382 +44383 +44384 +44385 +44386 +44387 +44388 +44389 +44390 +44391 +44392 +44393 +44394 +44395 +44396 +44397 +44398 +44399 +44400 +44401 +44402 +44403 +44404 +44405 +44406 +44407 +44408 +44409 +44410 +44411 +44412 +44413 +44414 +44415 +44416 +44417 +44418 +44419 +44420 +44421 +44422 +44423 +44424 +44425 +44426 +44427 +44428 +44429 +44430 +44431 +44432 +44433 +44434 +44435 +44436 +44437 +44438 +44439 +44440 +44441 +44442 +44443 +44444 +44445 +44446 +44447 +44448 +44449 +44450 +44451 +44452 +44453 +44454 +44455 +44456 +44457 +44458 +44459 +44460 +44461 +44462 +44463 +44464 +44465 +44466 +44467 +44468 +44469 +44470 +44471 +44472 +44473 +44474 +44475 +44476 +44477 +44478 +44479 +44480 +44481 +44482 +44483 +44484 +44485 +44486 +44487 +44488 +44489 +44490 +44491 +44492 +44493 +44494 +44495 +44496 +44497 +44498 +44499 +44500 +44501 +44502 +44503 +44504 +44505 +44506 +44507 +44508 +44509 +44510 +44511 +44512 +44513 +44514 +44515 +44516 +44517 +44518 +44519 +44520 +44521 +44522 +44523 +44524 +44525 +44526 +44527 +44528 +44529 +44530 +44531 +44532 +44533 +44534 +44535 +44536 +44537 +44538 +44539 +44540 +44541 +44542 +44543 +44544 +44545 +44546 +44547 +44548 +44549 +44550 +44551 +44552 +44553 +44554 +44555 +44556 +44557 +44558 +44559 +44560 +44561 +44562 +44563 +44564 +44565 +44566 +44567 +44568 +44569 +44570 +44571 +44572 +44573 +44574 +44575 +44576 +44577 +44578 +44579 +44580 +44581 +44582 +44583 +44584 +44585 +44586 +44587 +44588 +44589 +44590 +44591 +44592 +44593 +44594 +44595 +44596 +44597 +44598 +44599 +44600 +44601 +44602 +44603 +44604 +44605 +44606 +44607 +44608 +44609 +44610 +44611 +44612 +44613 +44614 +44615 +44616 +44617 +44618 +44619 +44620 +44621 +44622 +44623 +44624 +44625 +44626 +44627 +44628 +44629 +44630 +44631 +44632 +44633 +44634 +44635 +44636 +44637 +44638 +44639 +44640 +44641 +44642 +44643 +44644 +44645 +44646 +44647 +44648 +44649 +44650 +44651 +44652 +44653 +44654 +44655 +44656 +44657 +44658 +44659 +44660 +44661 +44662 +44663 +44664 +44665 +44666 +44667 +44668 +44669 +44670 +44671 +44672 +44673 +44674 +44675 +44676 +44677 +44678 +44679 +44680 +44681 +44682 +44683 +44684 +44685 +44686 +44687 +44688 +44689 +44690 +44691 +44692 +44693 +44694 +44695 +44696 +44697 +44698 +44699 +44700 +44701 +44702 +44703 +44704 +44705 +44706 +44707 +44708 +44709 +44710 +44711 +44712 +44713 +44714 +44715 +44716 +44717 +44718 +44719 +44720 +44721 +44722 +44723 +44724 +44725 +44726 +44727 +44728 +44729 +44730 +44731 +44732 +44733 +44734 +44735 +44736 +44737 +44738 +44739 +44740 +44741 +44742 +44743 +44744 +44745 +44746 +44747 +44748 +44749 +44750 +44751 +44752 +44753 +44754 +44755 +44756 +44757 +44758 +44759 +44760 +44761 +44762 +44763 +44764 +44765 +44766 +44767 +44768 +44769 +44770 +44771 +44772 +44773 +44774 +44775 +44776 +44777 +44778 +44779 +44780 +44781 +44782 +44783 +44784 +44785 +44786 +44787 +44788 +44789 +44790 +44791 +44792 +44793 +44794 +44795 +44796 +44797 +44798 +44799 +44800 +44801 +44802 +44803 +44804 +44805 +44806 +44807 +44808 +44809 +44810 +44811 +44812 +44813 +44814 +44815 +44816 +44817 +44818 +44819 +44820 +44821 +44822 +44823 +44824 +44825 +44826 +44827 +44828 +44829 +44830 +44831 +44832 +44833 +44834 +44835 +44836 +44837 +44838 +44839 +44840 +44841 +44842 +44843 +44844 +44845 +44846 +44847 +44848 +44849 +44850 +44851 +44852 +44853 +44854 +44855 +44856 +44857 +44858 +44859 +44860 +44861 +44862 +44863 +44864 +44865 +44866 +44867 +44868 +44869 +44870 +44871 +44872 +44873 +44874 +44875 +44876 +44877 +44878 +44879 +44880 +44881 +44882 +44883 +44884 +44885 +44886 +44887 +44888 +44889 +44890 +44891 +44892 +44893 +44894 +44895 +44896 +44897 +44898 +44899 +44900 +44901 +44902 +44903 +44904 +44905 +44906 +44907 +44908 +44909 +44910 +44911 +44912 +44913 +44914 +44915 +44916 +44917 +44918 +44919 +44920 +44921 +44922 +44923 +44924 +44925 +44926 +44927 +44928 +44929 +44930 +44931 +44932 +44933 +44934 +44935 +44936 +44937 +44938 +44939 +44940 +44941 +44942 +44943 +44944 +44945 +44946 +44947 +44948 +44949 +44950 +44951 +44952 +44953 +44954 +44955 +44956 +44957 +44958 +44959 +44960 +44961 +44962 +44963 +44964 +44965 +44966 +44967 +44968 +44969 +44970 +44971 +44972 +44973 +44974 +44975 +44976 +44977 +44978 +44979 +44980 +44981 +44982 +44983 +44984 +44985 +44986 +44987 +44988 +44989 +44990 +44991 +44992 +44993 +44994 +44995 +44996 +44997 +44998 +44999 +45000 +45001 +45002 +45003 +45004 +45005 +45006 +45007 +45008 +45009 +45010 +45011 +45012 +45013 +45014 +45015 +45016 +45017 +45018 +45019 +45020 +45021 +45022 +45023 +45024 +45025 +45026 +45027 +45028 +45029 +45030 +45031 +45032 +45033 +45034 +45035 +45036 +45037 +45038 +45039 +45040 +45041 +45042 +45043 +45044 +45045 +45046 +45047 +45048 +45049 +45050 +45051 +45052 +45053 +45054 +45055 +45056 +45057 +45058 +45059 +45060 +45061 +45062 +45063 +45064 +45065 +45066 +45067 +45068 +45069 +45070 +45071 +45072 +45073 +45074 +45075 +45076 +45077 +45078 +45079 +45080 +45081 +45082 +45083 +45084 +45085 +45086 +45087 +45088 +45089 +45090 +45091 +45092 +45093 +45094 +45095 +45096 +45097 +45098 +45099 +45100 +45101 +45102 +45103 +45104 +45105 +45106 +45107 +45108 +45109 +45110 +45111 +45112 +45113 +45114 +45115 +45116 +45117 +45118 +45119 +45120 +45121 +45122 +45123 +45124 +45125 +45126 +45127 +45128 +45129 +45130 +45131 +45132 +45133 +45134 +45135 +45136 +45137 +45138 +45139 +45140 +45141 +45142 +45143 +45144 +45145 +45146 +45147 +45148 +45149 +45150 +45151 +45152 +45153 +45154 +45155 +45156 +45157 +45158 +45159 +45160 +45161 +45162 +45163 +45164 +45165 +45166 +45167 +45168 +45169 +45170 +45171 +45172 +45173 +45174 +45175 +45176 +45177 +45178 +45179 +45180 +45181 +45182 +45183 +45184 +45185 +45186 +45187 +45188 +45189 +45190 +45191 +45192 +45193 +45194 +45195 +45196 +45197 +45198 +45199 +45200 +45201 +45202 +45203 +45204 +45205 +45206 +45207 +45208 +45209 +45210 +45211 +45212 +45213 +45214 +45215 +45216 +45217 +45218 +45219 +45220 +45221 +45222 +45223 +45224 +45225 +45226 +45227 +45228 +45229 +45230 +45231 +45232 +45233 +45234 +45235 +45236 +45237 +45238 +45239 +45240 +45241 +45242 +45243 +45244 +45245 +45246 +45247 +45248 +45249 +45250 +45251 +45252 +45253 +45254 +45255 +45256 +45257 +45258 +45259 +45260 +45261 +45262 +45263 +45264 +45265 +45266 +45267 +45268 +45269 +45270 +45271 +45272 +45273 +45274 +45275 +45276 +45277 +45278 +45279 +45280 +45281 +45282 +45283 +45284 +45285 +45286 +45287 +45288 +45289 +45290 +45291 +45292 +45293 +45294 +45295 +45296 +45297 +45298 +45299 +45300 +45301 +45302 +45303 +45304 +45305 +45306 +45307 +45308 +45309 +45310 +45311 +45312 +45313 +45314 +45315 +45316 +45317 +45318 +45319 +45320 +45321 +45322 +45323 +45324 +45325 +45326 +45327 +45328 +45329 +45330 +45331 +45332 +45333 +45334 +45335 +45336 +45337 +45338 +45339 +45340 +45341 +45342 +45343 +45344 +45345 +45346 +45347 +45348 +45349 +45350 +45351 +45352 +45353 +45354 +45355 +45356 +45357 +45358 +45359 +45360 +45361 +45362 +45363 +45364 +45365 +45366 +45367 +45368 +45369 +45370 +45371 +45372 +45373 +45374 +45375 +45376 +45377 +45378 +45379 +45380 +45381 +45382 +45383 +45384 +45385 +45386 +45387 +45388 +45389 +45390 +45391 +45392 +45393 +45394 +45395 +45396 +45397 +45398 +45399 +45400 +45401 +45402 +45403 +45404 +45405 +45406 +45407 +45408 +45409 +45410 +45411 +45412 +45413 +45414 +45415 +45416 +45417 +45418 +45419 +45420 +45421 +45422 +45423 +45424 +45425 +45426 +45427 +45428 +45429 +45430 +45431 +45432 +45433 +45434 +45435 +45436 +45437 +45438 +45439 +45440 +45441 +45442 +45443 +45444 +45445 +45446 +45447 +45448 +45449 +45450 +45451 +45452 +45453 +45454 +45455 +45456 +45457 +45458 +45459 +45460 +45461 +45462 +45463 +45464 +45465 +45466 +45467 +45468 +45469 +45470 +45471 +45472 +45473 +45474 +45475 +45476 +45477 +45478 +45479 +45480 +45481 +45482 +45483 +45484 +45485 +45486 +45487 +45488 +45489 +45490 +45491 +45492 +45493 +45494 +45495 +45496 +45497 +45498 +45499 +45500 +45501 +45502 +45503 +45504 +45505 +45506 +45507 +45508 +45509 +45510 +45511 +45512 +45513 +45514 +45515 +45516 +45517 +45518 +45519 +45520 +45521 +45522 +45523 +45524 +45525 +45526 +45527 +45528 +45529 +45530 +45531 +45532 +45533 +45534 +45535 +45536 +45537 +45538 +45539 +45540 +45541 +45542 +45543 +45544 +45545 +45546 +45547 +45548 +45549 +45550 +45551 +45552 +45553 +45554 +45555 +45556 +45557 +45558 +45559 +45560 +45561 +45562 +45563 +45564 +45565 +45566 +45567 +45568 +45569 +45570 +45571 +45572 +45573 +45574 +45575 +45576 +45577 +45578 +45579 +45580 +45581 +45582 +45583 +45584 +45585 +45586 +45587 +45588 +45589 +45590 +45591 +45592 +45593 +45594 +45595 +45596 +45597 +45598 +45599 +45600 +45601 +45602 +45603 +45604 +45605 +45606 +45607 +45608 +45609 +45610 +45611 +45612 +45613 +45614 +45615 +45616 +45617 +45618 +45619 +45620 +45621 +45622 +45623 +45624 +45625 +45626 +45627 +45628 +45629 +45630 +45631 +45632 +45633 +45634 +45635 +45636 +45637 +45638 +45639 +45640 +45641 +45642 +45643 +45644 +45645 +45646 +45647 +45648 +45649 +45650 +45651 +45652 +45653 +45654 +45655 +45656 +45657 +45658 +45659 +45660 +45661 +45662 +45663 +45664 +45665 +45666 +45667 +45668 +45669 +45670 +45671 +45672 +45673 +45674 +45675 +45676 +45677 +45678 +45679 +45680 +45681 +45682 +45683 +45684 +45685 +45686 +45687 +45688 +45689 +45690 +45691 +45692 +45693 +45694 +45695 +45696 +45697 +45698 +45699 +45700 +45701 +45702 +45703 +45704 +45705 +45706 +45707 +45708 +45709 +45710 +45711 +45712 +45713 +45714 +45715 +45716 +45717 +45718 +45719 +45720 +45721 +45722 +45723 +45724 +45725 +45726 +45727 +45728 +45729 +45730 +45731 +45732 +45733 +45734 +45735 +45736 +45737 +45738 +45739 +45740 +45741 +45742 +45743 +45744 +45745 +45746 +45747 +45748 +45749 +45750 +45751 +45752 +45753 +45754 +45755 +45756 +45757 +45758 +45759 +45760 +45761 +45762 +45763 +45764 +45765 +45766 +45767 +45768 +45769 +45770 +45771 +45772 +45773 +45774 +45775 +45776 +45777 +45778 +45779 +45780 +45781 +45782 +45783 +45784 +45785 +45786 +45787 +45788 +45789 +45790 +45791 +45792 +45793 +45794 +45795 +45796 +45797 +45798 +45799 +45800 +45801 +45802 +45803 +45804 +45805 +45806 +45807 +45808 +45809 +45810 +45811 +45812 +45813 +45814 +45815 +45816 +45817 +45818 +45819 +45820 +45821 +45822 +45823 +45824 +45825 +45826 +45827 +45828 +45829 +45830 +45831 +45832 +45833 +45834 +45835 +45836 +45837 +45838 +45839 +45840 +45841 +45842 +45843 +45844 +45845 +45846 +45847 +45848 +45849 +45850 +45851 +45852 +45853 +45854 +45855 +45856 +45857 +45858 +45859 +45860 +45861 +45862 +45863 +45864 +45865 +45866 +45867 +45868 +45869 +45870 +45871 +45872 +45873 +45874 +45875 +45876 +45877 +45878 +45879 +45880 +45881 +45882 +45883 +45884 +45885 +45886 +45887 +45888 +45889 +45890 +45891 +45892 +45893 +45894 +45895 +45896 +45897 +45898 +45899 +45900 +45901 +45902 +45903 +45904 +45905 +45906 +45907 +45908 +45909 +45910 +45911 +45912 +45913 +45914 +45915 +45916 +45917 +45918 +45919 +45920 +45921 +45922 +45923 +45924 +45925 +45926 +45927 +45928 +45929 +45930 +45931 +45932 +45933 +45934 +45935 +45936 +45937 +45938 +45939 +45940 +45941 +45942 +45943 +45944 +45945 +45946 +45947 +45948 +45949 +45950 +45951 +45952 +45953 +45954 +45955 +45956 +45957 +45958 +45959 +45960 +45961 +45962 +45963 +45964 +45965 +45966 +45967 +45968 +45969 +45970 +45971 +45972 +45973 +45974 +45975 +45976 +45977 +45978 +45979 +45980 +45981 +45982 +45983 +45984 +45985 +45986 +45987 +45988 +45989 +45990 +45991 +45992 +45993 +45994 +45995 +45996 +45997 +45998 +45999 +46000 +46001 +46002 +46003 +46004 +46005 +46006 +46007 +46008 +46009 +46010 +46011 +46012 +46013 +46014 +46015 +46016 +46017 +46018 +46019 +46020 +46021 +46022 +46023 +46024 +46025 +46026 +46027 +46028 +46029 +46030 +46031 +46032 +46033 +46034 +46035 +46036 +46037 +46038 +46039 +46040 +46041 +46042 +46043 +46044 +46045 +46046 +46047 +46048 +46049 +46050 +46051 +46052 +46053 +46054 +46055 +46056 +46057 +46058 +46059 +46060 +46061 +46062 +46063 +46064 +46065 +46066 +46067 +46068 +46069 +46070 +46071 +46072 +46073 +46074 +46075 +46076 +46077 +46078 +46079 +46080 +46081 +46082 +46083 +46084 +46085 +46086 +46087 +46088 +46089 +46090 +46091 +46092 +46093 +46094 +46095 +46096 +46097 +46098 +46099 +46100 +46101 +46102 +46103 +46104 +46105 +46106 +46107 +46108 +46109 +46110 +46111 +46112 +46113 +46114 +46115 +46116 +46117 +46118 +46119 +46120 +46121 +46122 +46123 +46124 +46125 +46126 +46127 +46128 +46129 +46130 +46131 +46132 +46133 +46134 +46135 +46136 +46137 +46138 +46139 +46140 +46141 +46142 +46143 +46144 +46145 +46146 +46147 +46148 +46149 +46150 +46151 +46152 +46153 +46154 +46155 +46156 +46157 +46158 +46159 +46160 +46161 +46162 +46163 +46164 +46165 +46166 +46167 +46168 +46169 +46170 +46171 +46172 +46173 +46174 +46175 +46176 +46177 +46178 +46179 +46180 +46181 +46182 +46183 +46184 +46185 +46186 +46187 +46188 +46189 +46190 +46191 +46192 +46193 +46194 +46195 +46196 +46197 +46198 +46199 +46200 +46201 +46202 +46203 +46204 +46205 +46206 +46207 +46208 +46209 +46210 +46211 +46212 +46213 +46214 +46215 +46216 +46217 +46218 +46219 +46220 +46221 +46222 +46223 +46224 +46225 +46226 +46227 +46228 +46229 +46230 +46231 +46232 +46233 +46234 +46235 +46236 +46237 +46238 +46239 +46240 +46241 +46242 +46243 +46244 +46245 +46246 +46247 +46248 +46249 +46250 +46251 +46252 +46253 +46254 +46255 +46256 +46257 +46258 +46259 +46260 +46261 +46262 +46263 +46264 +46265 +46266 +46267 +46268 +46269 +46270 +46271 +46272 +46273 +46274 +46275 +46276 +46277 +46278 +46279 +46280 +46281 +46282 +46283 +46284 +46285 +46286 +46287 +46288 +46289 +46290 +46291 +46292 +46293 +46294 +46295 +46296 +46297 +46298 +46299 +46300 +46301 +46302 +46303 +46304 +46305 +46306 +46307 +46308 +46309 +46310 +46311 +46312 +46313 +46314 +46315 +46316 +46317 +46318 +46319 +46320 +46321 +46322 +46323 +46324 +46325 +46326 +46327 +46328 +46329 +46330 +46331 +46332 +46333 +46334 +46335 +46336 +46337 +46338 +46339 +46340 +46341 +46342 +46343 +46344 +46345 +46346 +46347 +46348 +46349 +46350 +46351 +46352 +46353 +46354 +46355 +46356 +46357 +46358 +46359 +46360 +46361 +46362 +46363 +46364 +46365 +46366 +46367 +46368 +46369 +46370 +46371 +46372 +46373 +46374 +46375 +46376 +46377 +46378 +46379 +46380 +46381 +46382 +46383 +46384 +46385 +46386 +46387 +46388 +46389 +46390 +46391 +46392 +46393 +46394 +46395 +46396 +46397 +46398 +46399 +46400 +46401 +46402 +46403 +46404 +46405 +46406 +46407 +46408 +46409 +46410 +46411 +46412 +46413 +46414 +46415 +46416 +46417 +46418 +46419 +46420 +46421 +46422 +46423 +46424 +46425 +46426 +46427 +46428 +46429 +46430 +46431 +46432 +46433 +46434 +46435 +46436 +46437 +46438 +46439 +46440 +46441 +46442 +46443 +46444 +46445 +46446 +46447 +46448 +46449 +46450 +46451 +46452 +46453 +46454 +46455 +46456 +46457 +46458 +46459 +46460 +46461 +46462 +46463 +46464 +46465 +46466 +46467 +46468 +46469 +46470 +46471 +46472 +46473 +46474 +46475 +46476 +46477 +46478 +46479 +46480 +46481 +46482 +46483 +46484 +46485 +46486 +46487 +46488 +46489 +46490 +46491 +46492 +46493 +46494 +46495 +46496 +46497 +46498 +46499 +46500 +46501 +46502 +46503 +46504 +46505 +46506 +46507 +46508 +46509 +46510 +46511 +46512 +46513 +46514 +46515 +46516 +46517 +46518 +46519 +46520 +46521 +46522 +46523 +46524 +46525 +46526 +46527 +46528 +46529 +46530 +46531 +46532 +46533 +46534 +46535 +46536 +46537 +46538 +46539 +46540 +46541 +46542 +46543 +46544 +46545 +46546 +46547 +46548 +46549 +46550 +46551 +46552 +46553 +46554 +46555 +46556 +46557 +46558 +46559 +46560 +46561 +46562 +46563 +46564 +46565 +46566 +46567 +46568 +46569 +46570 +46571 +46572 +46573 +46574 +46575 +46576 +46577 +46578 +46579 +46580 +46581 +46582 +46583 +46584 +46585 +46586 +46587 +46588 +46589 +46590 +46591 +46592 +46593 +46594 +46595 +46596 +46597 +46598 +46599 +46600 +46601 +46602 +46603 +46604 +46605 +46606 +46607 +46608 +46609 +46610 +46611 +46612 +46613 +46614 +46615 +46616 +46617 +46618 +46619 +46620 +46621 +46622 +46623 +46624 +46625 +46626 +46627 +46628 +46629 +46630 +46631 +46632 +46633 +46634 +46635 +46636 +46637 +46638 +46639 +46640 +46641 +46642 +46643 +46644 +46645 +46646 +46647 +46648 +46649 +46650 +46651 +46652 +46653 +46654 +46655 +46656 +46657 +46658 +46659 +46660 +46661 +46662 +46663 +46664 +46665 +46666 +46667 +46668 +46669 +46670 +46671 +46672 +46673 +46674 +46675 +46676 +46677 +46678 +46679 +46680 +46681 +46682 +46683 +46684 +46685 +46686 +46687 +46688 +46689 +46690 +46691 +46692 +46693 +46694 +46695 +46696 +46697 +46698 +46699 +46700 +46701 +46702 +46703 +46704 +46705 +46706 +46707 +46708 +46709 +46710 +46711 +46712 +46713 +46714 +46715 +46716 +46717 +46718 +46719 +46720 +46721 +46722 +46723 +46724 +46725 +46726 +46727 +46728 +46729 +46730 +46731 +46732 +46733 +46734 +46735 +46736 +46737 +46738 +46739 +46740 +46741 +46742 +46743 +46744 +46745 +46746 +46747 +46748 +46749 +46750 +46751 +46752 +46753 +46754 +46755 +46756 +46757 +46758 +46759 +46760 +46761 +46762 +46763 +46764 +46765 +46766 +46767 +46768 +46769 +46770 +46771 +46772 +46773 +46774 +46775 +46776 +46777 +46778 +46779 +46780 +46781 +46782 +46783 +46784 +46785 +46786 +46787 +46788 +46789 +46790 +46791 +46792 +46793 +46794 +46795 +46796 +46797 +46798 +46799 +46800 +46801 +46802 +46803 +46804 +46805 +46806 +46807 +46808 +46809 +46810 +46811 +46812 +46813 +46814 +46815 +46816 +46817 +46818 +46819 +46820 +46821 +46822 +46823 +46824 +46825 +46826 +46827 +46828 +46829 +46830 +46831 +46832 +46833 +46834 +46835 +46836 +46837 +46838 +46839 +46840 +46841 +46842 +46843 +46844 +46845 +46846 +46847 +46848 +46849 +46850 +46851 +46852 +46853 +46854 +46855 +46856 +46857 +46858 +46859 +46860 +46861 +46862 +46863 +46864 +46865 +46866 +46867 +46868 +46869 +46870 +46871 +46872 +46873 +46874 +46875 +46876 +46877 +46878 +46879 +46880 +46881 +46882 +46883 +46884 +46885 +46886 +46887 +46888 +46889 +46890 +46891 +46892 +46893 +46894 +46895 +46896 +46897 +46898 +46899 +46900 +46901 +46902 +46903 +46904 +46905 +46906 +46907 +46908 +46909 +46910 +46911 +46912 +46913 +46914 +46915 +46916 +46917 +46918 +46919 +46920 +46921 +46922 +46923 +46924 +46925 +46926 +46927 +46928 +46929 +46930 +46931 +46932 +46933 +46934 +46935 +46936 +46937 +46938 +46939 +46940 +46941 +46942 +46943 +46944 +46945 +46946 +46947 +46948 +46949 +46950 +46951 +46952 +46953 +46954 +46955 +46956 +46957 +46958 +46959 +46960 +46961 +46962 +46963 +46964 +46965 +46966 +46967 +46968 +46969 +46970 +46971 +46972 +46973 +46974 +46975 +46976 +46977 +46978 +46979 +46980 +46981 +46982 +46983 +46984 +46985 +46986 +46987 +46988 +46989 +46990 +46991 +46992 +46993 +46994 +46995 +46996 +46997 +46998 +46999 +47000 +47001 +47002 +47003 +47004 +47005 +47006 +47007 +47008 +47009 +47010 +47011 +47012 +47013 +47014 +47015 +47016 +47017 +47018 +47019 +47020 +47021 +47022 +47023 +47024 +47025 +47026 +47027 +47028 +47029 +47030 +47031 +47032 +47033 +47034 +47035 +47036 +47037 +47038 +47039 +47040 +47041 +47042 +47043 +47044 +47045 +47046 +47047 +47048 +47049 +47050 +47051 +47052 +47053 +47054 +47055 +47056 +47057 +47058 +47059 +47060 +47061 +47062 +47063 +47064 +47065 +47066 +47067 +47068 +47069 +47070 +47071 +47072 +47073 +47074 +47075 +47076 +47077 +47078 +47079 +47080 +47081 +47082 +47083 +47084 +47085 +47086 +47087 +47088 +47089 +47090 +47091 +47092 +47093 +47094 +47095 +47096 +47097 +47098 +47099 +47100 +47101 +47102 +47103 +47104 +47105 +47106 +47107 +47108 +47109 +47110 +47111 +47112 +47113 +47114 +47115 +47116 +47117 +47118 +47119 +47120 +47121 +47122 +47123 +47124 +47125 +47126 +47127 +47128 +47129 +47130 +47131 +47132 +47133 +47134 +47135 +47136 +47137 +47138 +47139 +47140 +47141 +47142 +47143 +47144 +47145 +47146 +47147 +47148 +47149 +47150 +47151 +47152 +47153 +47154 +47155 +47156 +47157 +47158 +47159 +47160 +47161 +47162 +47163 +47164 +47165 +47166 +47167 +47168 +47169 +47170 +47171 +47172 +47173 +47174 +47175 +47176 +47177 +47178 +47179 +47180 +47181 +47182 +47183 +47184 +47185 +47186 +47187 +47188 +47189 +47190 +47191 +47192 +47193 +47194 +47195 +47196 +47197 +47198 +47199 +47200 +47201 +47202 +47203 +47204 +47205 +47206 +47207 +47208 +47209 +47210 +47211 +47212 +47213 +47214 +47215 +47216 +47217 +47218 +47219 +47220 +47221 +47222 +47223 +47224 +47225 +47226 +47227 +47228 +47229 +47230 +47231 +47232 +47233 +47234 +47235 +47236 +47237 +47238 +47239 +47240 +47241 +47242 +47243 +47244 +47245 +47246 +47247 +47248 +47249 +47250 +47251 +47252 +47253 +47254 +47255 +47256 +47257 +47258 +47259 +47260 +47261 +47262 +47263 +47264 +47265 +47266 +47267 +47268 +47269 +47270 +47271 +47272 +47273 +47274 +47275 +47276 +47277 +47278 +47279 +47280 +47281 +47282 +47283 +47284 +47285 +47286 +47287 +47288 +47289 +47290 +47291 +47292 +47293 +47294 +47295 +47296 +47297 +47298 +47299 +47300 +47301 +47302 +47303 +47304 +47305 +47306 +47307 +47308 +47309 +47310 +47311 +47312 +47313 +47314 +47315 +47316 +47317 +47318 +47319 +47320 +47321 +47322 +47323 +47324 +47325 +47326 +47327 +47328 +47329 +47330 +47331 +47332 +47333 +47334 +47335 +47336 +47337 +47338 +47339 +47340 +47341 +47342 +47343 +47344 +47345 +47346 +47347 +47348 +47349 +47350 +47351 +47352 +47353 +47354 +47355 +47356 +47357 +47358 +47359 +47360 +47361 +47362 +47363 +47364 +47365 +47366 +47367 +47368 +47369 +47370 +47371 +47372 +47373 +47374 +47375 +47376 +47377 +47378 +47379 +47380 +47381 +47382 +47383 +47384 +47385 +47386 +47387 +47388 +47389 +47390 +47391 +47392 +47393 +47394 +47395 +47396 +47397 +47398 +47399 +47400 +47401 +47402 +47403 +47404 +47405 +47406 +47407 +47408 +47409 +47410 +47411 +47412 +47413 +47414 +47415 +47416 +47417 +47418 +47419 +47420 +47421 +47422 +47423 +47424 +47425 +47426 +47427 +47428 +47429 +47430 +47431 +47432 +47433 +47434 +47435 +47436 +47437 +47438 +47439 +47440 +47441 +47442 +47443 +47444 +47445 +47446 +47447 +47448 +47449 +47450 +47451 +47452 +47453 +47454 +47455 +47456 +47457 +47458 +47459 +47460 +47461 +47462 +47463 +47464 +47465 +47466 +47467 +47468 +47469 +47470 +47471 +47472 +47473 +47474 +47475 +47476 +47477 +47478 +47479 +47480 +47481 +47482 +47483 +47484 +47485 +47486 +47487 +47488 +47489 +47490 +47491 +47492 +47493 +47494 +47495 +47496 +47497 +47498 +47499 +47500 +47501 +47502 +47503 +47504 +47505 +47506 +47507 +47508 +47509 +47510 +47511 +47512 +47513 +47514 +47515 +47516 +47517 +47518 +47519 +47520 +47521 +47522 +47523 +47524 +47525 +47526 +47527 +47528 +47529 +47530 +47531 +47532 +47533 +47534 +47535 +47536 +47537 +47538 +47539 +47540 +47541 +47542 +47543 +47544 +47545 +47546 +47547 +47548 +47549 +47550 +47551 +47552 +47553 +47554 +47555 +47556 +47557 +47558 +47559 +47560 +47561 +47562 +47563 +47564 +47565 +47566 +47567 +47568 +47569 +47570 +47571 +47572 +47573 +47574 +47575 +47576 +47577 +47578 +47579 +47580 +47581 +47582 +47583 +47584 +47585 +47586 +47587 +47588 +47589 +47590 +47591 +47592 +47593 +47594 +47595 +47596 +47597 +47598 +47599 +47600 +47601 +47602 +47603 +47604 +47605 +47606 +47607 +47608 +47609 +47610 +47611 +47612 +47613 +47614 +47615 +47616 +47617 +47618 +47619 +47620 +47621 +47622 +47623 +47624 +47625 +47626 +47627 +47628 +47629 +47630 +47631 +47632 +47633 +47634 +47635 +47636 +47637 +47638 +47639 +47640 +47641 +47642 +47643 +47644 +47645 +47646 +47647 +47648 +47649 +47650 +47651 +47652 +47653 +47654 +47655 +47656 +47657 +47658 +47659 +47660 +47661 +47662 +47663 +47664 +47665 +47666 +47667 +47668 +47669 +47670 +47671 +47672 +47673 +47674 +47675 +47676 +47677 +47678 +47679 +47680 +47681 +47682 +47683 +47684 +47685 +47686 +47687 +47688 +47689 +47690 +47691 +47692 +47693 +47694 +47695 +47696 +47697 +47698 +47699 +47700 +47701 +47702 +47703 +47704 +47705 +47706 +47707 +47708 +47709 +47710 +47711 +47712 +47713 +47714 +47715 +47716 +47717 +47718 +47719 +47720 +47721 +47722 +47723 +47724 +47725 +47726 +47727 +47728 +47729 +47730 +47731 +47732 +47733 +47734 +47735 +47736 +47737 +47738 +47739 +47740 +47741 +47742 +47743 +47744 +47745 +47746 +47747 +47748 +47749 +47750 +47751 +47752 +47753 +47754 +47755 +47756 +47757 +47758 +47759 +47760 +47761 +47762 +47763 +47764 +47765 +47766 +47767 +47768 +47769 +47770 +47771 +47772 +47773 +47774 +47775 +47776 +47777 +47778 +47779 +47780 +47781 +47782 +47783 +47784 +47785 +47786 +47787 +47788 +47789 +47790 +47791 +47792 +47793 +47794 +47795 +47796 +47797 +47798 +47799 +47800 +47801 +47802 +47803 +47804 +47805 +47806 +47807 +47808 +47809 +47810 +47811 +47812 +47813 +47814 +47815 +47816 +47817 +47818 +47819 +47820 +47821 +47822 +47823 +47824 +47825 +47826 +47827 +47828 +47829 +47830 +47831 +47832 +47833 +47834 +47835 +47836 +47837 +47838 +47839 +47840 +47841 +47842 +47843 +47844 +47845 +47846 +47847 +47848 +47849 +47850 +47851 +47852 +47853 +47854 +47855 +47856 +47857 +47858 +47859 +47860 +47861 +47862 +47863 +47864 +47865 +47866 +47867 +47868 +47869 +47870 +47871 +47872 +47873 +47874 +47875 +47876 +47877 +47878 +47879 +47880 +47881 +47882 +47883 +47884 +47885 +47886 +47887 +47888 +47889 +47890 +47891 +47892 +47893 +47894 +47895 +47896 +47897 +47898 +47899 +47900 +47901 +47902 +47903 +47904 +47905 +47906 +47907 +47908 +47909 +47910 +47911 +47912 +47913 +47914 +47915 +47916 +47917 +47918 +47919 +47920 +47921 +47922 +47923 +47924 +47925 +47926 +47927 +47928 +47929 +47930 +47931 +47932 +47933 +47934 +47935 +47936 +47937 +47938 +47939 +47940 +47941 +47942 +47943 +47944 +47945 +47946 +47947 +47948 +47949 +47950 +47951 +47952 +47953 +47954 +47955 +47956 +47957 +47958 +47959 +47960 +47961 +47962 +47963 +47964 +47965 +47966 +47967 +47968 +47969 +47970 +47971 +47972 +47973 +47974 +47975 +47976 +47977 +47978 +47979 +47980 +47981 +47982 +47983 +47984 +47985 +47986 +47987 +47988 +47989 +47990 +47991 +47992 +47993 +47994 +47995 +47996 +47997 +47998 +47999 +48000 +48001 +48002 +48003 +48004 +48005 +48006 +48007 +48008 +48009 +48010 +48011 +48012 +48013 +48014 +48015 +48016 +48017 +48018 +48019 +48020 +48021 +48022 +48023 +48024 +48025 +48026 +48027 +48028 +48029 +48030 +48031 +48032 +48033 +48034 +48035 +48036 +48037 +48038 +48039 +48040 +48041 +48042 +48043 +48044 +48045 +48046 +48047 +48048 +48049 +48050 +48051 +48052 +48053 +48054 +48055 +48056 +48057 +48058 +48059 +48060 +48061 +48062 +48063 +48064 +48065 +48066 +48067 +48068 +48069 +48070 +48071 +48072 +48073 +48074 +48075 +48076 +48077 +48078 +48079 +48080 +48081 +48082 +48083 +48084 +48085 +48086 +48087 +48088 +48089 +48090 +48091 +48092 +48093 +48094 +48095 +48096 +48097 +48098 +48099 +48100 +48101 +48102 +48103 +48104 +48105 +48106 +48107 +48108 +48109 +48110 +48111 +48112 +48113 +48114 +48115 +48116 +48117 +48118 +48119 +48120 +48121 +48122 +48123 +48124 +48125 +48126 +48127 +48128 +48129 +48130 +48131 +48132 +48133 +48134 +48135 +48136 +48137 +48138 +48139 +48140 +48141 +48142 +48143 +48144 +48145 +48146 +48147 +48148 +48149 +48150 +48151 +48152 +48153 +48154 +48155 +48156 +48157 +48158 +48159 +48160 +48161 +48162 +48163 +48164 +48165 +48166 +48167 +48168 +48169 +48170 +48171 +48172 +48173 +48174 +48175 +48176 +48177 +48178 +48179 +48180 +48181 +48182 +48183 +48184 +48185 +48186 +48187 +48188 +48189 +48190 +48191 +48192 +48193 +48194 +48195 +48196 +48197 +48198 +48199 +48200 +48201 +48202 +48203 +48204 +48205 +48206 +48207 +48208 +48209 +48210 +48211 +48212 +48213 +48214 +48215 +48216 +48217 +48218 +48219 +48220 +48221 +48222 +48223 +48224 +48225 +48226 +48227 +48228 +48229 +48230 +48231 +48232 +48233 +48234 +48235 +48236 +48237 +48238 +48239 +48240 +48241 +48242 +48243 +48244 +48245 +48246 +48247 +48248 +48249 +48250 +48251 +48252 +48253 +48254 +48255 +48256 +48257 +48258 +48259 +48260 +48261 +48262 +48263 +48264 +48265 +48266 +48267 +48268 +48269 +48270 +48271 +48272 +48273 +48274 +48275 +48276 +48277 +48278 +48279 +48280 +48281 +48282 +48283 +48284 +48285 +48286 +48287 +48288 +48289 +48290 +48291 +48292 +48293 +48294 +48295 +48296 +48297 +48298 +48299 +48300 +48301 +48302 +48303 +48304 +48305 +48306 +48307 +48308 +48309 +48310 +48311 +48312 +48313 +48314 +48315 +48316 +48317 +48318 +48319 +48320 +48321 +48322 +48323 +48324 +48325 +48326 +48327 +48328 +48329 +48330 +48331 +48332 +48333 +48334 +48335 +48336 +48337 +48338 +48339 +48340 +48341 +48342 +48343 +48344 +48345 +48346 +48347 +48348 +48349 +48350 +48351 +48352 +48353 +48354 +48355 +48356 +48357 +48358 +48359 +48360 +48361 +48362 +48363 +48364 +48365 +48366 +48367 +48368 +48369 +48370 +48371 +48372 +48373 +48374 +48375 +48376 +48377 +48378 +48379 +48380 +48381 +48382 +48383 +48384 +48385 +48386 +48387 +48388 +48389 +48390 +48391 +48392 +48393 +48394 +48395 +48396 +48397 +48398 +48399 +48400 +48401 +48402 +48403 +48404 +48405 +48406 +48407 +48408 +48409 +48410 +48411 +48412 +48413 +48414 +48415 +48416 +48417 +48418 +48419 +48420 +48421 +48422 +48423 +48424 +48425 +48426 +48427 +48428 +48429 +48430 +48431 +48432 +48433 +48434 +48435 +48436 +48437 +48438 +48439 +48440 +48441 +48442 +48443 +48444 +48445 +48446 +48447 +48448 +48449 +48450 +48451 +48452 +48453 +48454 +48455 +48456 +48457 +48458 +48459 +48460 +48461 +48462 +48463 +48464 +48465 +48466 +48467 +48468 +48469 +48470 +48471 +48472 +48473 +48474 +48475 +48476 +48477 +48478 +48479 +48480 +48481 +48482 +48483 +48484 +48485 +48486 +48487 +48488 +48489 +48490 +48491 +48492 +48493 +48494 +48495 +48496 +48497 +48498 +48499 +48500 +48501 +48502 +48503 +48504 +48505 +48506 +48507 +48508 +48509 +48510 +48511 +48512 +48513 +48514 +48515 +48516 +48517 +48518 +48519 +48520 +48521 +48522 +48523 +48524 +48525 +48526 +48527 +48528 +48529 +48530 +48531 +48532 +48533 +48534 +48535 +48536 +48537 +48538 +48539 +48540 +48541 +48542 +48543 +48544 +48545 +48546 +48547 +48548 +48549 +48550 +48551 +48552 +48553 +48554 +48555 +48556 +48557 +48558 +48559 +48560 +48561 +48562 +48563 +48564 +48565 +48566 +48567 +48568 +48569 +48570 +48571 +48572 +48573 +48574 +48575 +48576 +48577 +48578 +48579 +48580 +48581 +48582 +48583 +48584 +48585 +48586 +48587 +48588 +48589 +48590 +48591 +48592 +48593 +48594 +48595 +48596 +48597 +48598 +48599 +48600 +48601 +48602 +48603 +48604 +48605 +48606 +48607 +48608 +48609 +48610 +48611 +48612 +48613 +48614 +48615 +48616 +48617 +48618 +48619 +48620 +48621 +48622 +48623 +48624 +48625 +48626 +48627 +48628 +48629 +48630 +48631 +48632 +48633 +48634 +48635 +48636 +48637 +48638 +48639 +48640 +48641 +48642 +48643 +48644 +48645 +48646 +48647 +48648 +48649 +48650 +48651 +48652 +48653 +48654 +48655 +48656 +48657 +48658 +48659 +48660 +48661 +48662 +48663 +48664 +48665 +48666 +48667 +48668 +48669 +48670 +48671 +48672 +48673 +48674 +48675 +48676 +48677 +48678 +48679 +48680 +48681 +48682 +48683 +48684 +48685 +48686 +48687 +48688 +48689 +48690 +48691 +48692 +48693 +48694 +48695 +48696 +48697 +48698 +48699 +48700 +48701 +48702 +48703 +48704 +48705 +48706 +48707 +48708 +48709 +48710 +48711 +48712 +48713 +48714 +48715 +48716 +48717 +48718 +48719 +48720 +48721 +48722 +48723 +48724 +48725 +48726 +48727 +48728 +48729 +48730 +48731 +48732 +48733 +48734 +48735 +48736 +48737 +48738 +48739 +48740 +48741 +48742 +48743 +48744 +48745 +48746 +48747 +48748 +48749 +48750 +48751 +48752 +48753 +48754 +48755 +48756 +48757 +48758 +48759 +48760 +48761 +48762 +48763 +48764 +48765 +48766 +48767 +48768 +48769 +48770 +48771 +48772 +48773 +48774 +48775 +48776 +48777 +48778 +48779 +48780 +48781 +48782 +48783 +48784 +48785 +48786 +48787 +48788 +48789 +48790 +48791 +48792 +48793 +48794 +48795 +48796 +48797 +48798 +48799 +48800 +48801 +48802 +48803 +48804 +48805 +48806 +48807 +48808 +48809 +48810 +48811 +48812 +48813 +48814 +48815 +48816 +48817 +48818 +48819 +48820 +48821 +48822 +48823 +48824 +48825 +48826 +48827 +48828 +48829 +48830 +48831 +48832 +48833 +48834 +48835 +48836 +48837 +48838 +48839 +48840 +48841 +48842 +48843 +48844 +48845 +48846 +48847 +48848 +48849 +48850 +48851 +48852 +48853 +48854 +48855 +48856 +48857 +48858 +48859 +48860 +48861 +48862 +48863 +48864 +48865 +48866 +48867 +48868 +48869 +48870 +48871 +48872 +48873 +48874 +48875 +48876 +48877 +48878 +48879 +48880 +48881 +48882 +48883 +48884 +48885 +48886 +48887 +48888 +48889 +48890 +48891 +48892 +48893 +48894 +48895 +48896 +48897 +48898 +48899 +48900 +48901 +48902 +48903 +48904 +48905 +48906 +48907 +48908 +48909 +48910 +48911 +48912 +48913 +48914 +48915 +48916 +48917 +48918 +48919 +48920 +48921 +48922 +48923 +48924 +48925 +48926 +48927 +48928 +48929 +48930 +48931 +48932 +48933 +48934 +48935 +48936 +48937 +48938 +48939 +48940 +48941 +48942 +48943 +48944 +48945 +48946 +48947 +48948 +48949 +48950 +48951 +48952 +48953 +48954 +48955 +48956 +48957 +48958 +48959 +48960 +48961 +48962 +48963 +48964 +48965 +48966 +48967 +48968 +48969 +48970 +48971 +48972 +48973 +48974 +48975 +48976 +48977 +48978 +48979 +48980 +48981 +48982 +48983 +48984 +48985 +48986 +48987 +48988 +48989 +48990 +48991 +48992 +48993 +48994 +48995 +48996 +48997 +48998 +48999 +49000 +49001 +49002 +49003 +49004 +49005 +49006 +49007 +49008 +49009 +49010 +49011 +49012 +49013 +49014 +49015 +49016 +49017 +49018 +49019 +49020 +49021 +49022 +49023 +49024 +49025 +49026 +49027 +49028 +49029 +49030 +49031 +49032 +49033 +49034 +49035 +49036 +49037 +49038 +49039 +49040 +49041 +49042 +49043 +49044 +49045 +49046 +49047 +49048 +49049 +49050 +49051 +49052 +49053 +49054 +49055 +49056 +49057 +49058 +49059 +49060 +49061 +49062 +49063 +49064 +49065 +49066 +49067 +49068 +49069 +49070 +49071 +49072 +49073 +49074 +49075 +49076 +49077 +49078 +49079 +49080 +49081 +49082 +49083 +49084 +49085 +49086 +49087 +49088 +49089 +49090 +49091 +49092 +49093 +49094 +49095 +49096 +49097 +49098 +49099 +49100 +49101 +49102 +49103 +49104 +49105 +49106 +49107 +49108 +49109 +49110 +49111 +49112 +49113 +49114 +49115 +49116 +49117 +49118 +49119 +49120 +49121 +49122 +49123 +49124 +49125 +49126 +49127 +49128 +49129 +49130 +49131 +49132 +49133 +49134 +49135 +49136 +49137 +49138 +49139 +49140 +49141 +49142 +49143 +49144 +49145 +49146 +49147 +49148 +49149 +49150 +49151 +49152 +49153 +49154 +49155 +49156 +49157 +49158 +49159 +49160 +49161 +49162 +49163 +49164 +49165 +49166 +49167 +49168 +49169 +49170 +49171 +49172 +49173 +49174 +49175 +49176 +49177 +49178 +49179 +49180 +49181 +49182 +49183 +49184 +49185 +49186 +49187 +49188 +49189 +49190 +49191 +49192 +49193 +49194 +49195 +49196 +49197 +49198 +49199 +49200 +49201 +49202 +49203 +49204 +49205 +49206 +49207 +49208 +49209 +49210 +49211 +49212 +49213 +49214 +49215 +49216 +49217 +49218 +49219 +49220 +49221 +49222 +49223 +49224 +49225 +49226 +49227 +49228 +49229 +49230 +49231 +49232 +49233 +49234 +49235 +49236 +49237 +49238 +49239 +49240 +49241 +49242 +49243 +49244 +49245 +49246 +49247 +49248 +49249 +49250 +49251 +49252 +49253 +49254 +49255 +49256 +49257 +49258 +49259 +49260 +49261 +49262 +49263 +49264 +49265 +49266 +49267 +49268 +49269 +49270 +49271 +49272 +49273 +49274 +49275 +49276 +49277 +49278 +49279 +49280 +49281 +49282 +49283 +49284 +49285 +49286 +49287 +49288 +49289 +49290 +49291 +49292 +49293 +49294 +49295 +49296 +49297 +49298 +49299 +49300 +49301 +49302 +49303 +49304 +49305 +49306 +49307 +49308 +49309 +49310 +49311 +49312 +49313 +49314 +49315 +49316 +49317 +49318 +49319 +49320 +49321 +49322 +49323 +49324 +49325 +49326 +49327 +49328 +49329 +49330 +49331 +49332 +49333 +49334 +49335 +49336 +49337 +49338 +49339 +49340 +49341 +49342 +49343 +49344 +49345 +49346 +49347 +49348 +49349 +49350 +49351 +49352 +49353 +49354 +49355 +49356 +49357 +49358 +49359 +49360 +49361 +49362 +49363 +49364 +49365 +49366 +49367 +49368 +49369 +49370 +49371 +49372 +49373 +49374 +49375 +49376 +49377 +49378 +49379 +49380 +49381 +49382 +49383 +49384 +49385 +49386 +49387 +49388 +49389 +49390 +49391 +49392 +49393 +49394 +49395 +49396 +49397 +49398 +49399 +49400 +49401 +49402 +49403 +49404 +49405 +49406 +49407 +49408 +49409 +49410 +49411 +49412 +49413 +49414 +49415 +49416 +49417 +49418 +49419 +49420 +49421 +49422 +49423 +49424 +49425 +49426 +49427 +49428 +49429 +49430 +49431 +49432 +49433 +49434 +49435 +49436 +49437 +49438 +49439 +49440 +49441 +49442 +49443 +49444 +49445 +49446 +49447 +49448 +49449 +49450 +49451 +49452 +49453 +49454 +49455 +49456 +49457 +49458 +49459 +49460 +49461 +49462 +49463 +49464 +49465 +49466 +49467 +49468 +49469 +49470 +49471 +49472 +49473 +49474 +49475 +49476 +49477 +49478 +49479 +49480 +49481 +49482 +49483 +49484 +49485 +49486 +49487 +49488 +49489 +49490 +49491 +49492 +49493 +49494 +49495 +49496 +49497 +49498 +49499 +49500 +49501 +49502 +49503 +49504 +49505 +49506 +49507 +49508 +49509 +49510 +49511 +49512 +49513 +49514 +49515 +49516 +49517 +49518 +49519 +49520 +49521 +49522 +49523 +49524 +49525 +49526 +49527 +49528 +49529 +49530 +49531 +49532 +49533 +49534 +49535 +49536 +49537 +49538 +49539 +49540 +49541 +49542 +49543 +49544 +49545 +49546 +49547 +49548 +49549 +49550 +49551 +49552 +49553 +49554 +49555 +49556 +49557 +49558 +49559 +49560 +49561 +49562 +49563 +49564 +49565 +49566 +49567 +49568 +49569 +49570 +49571 +49572 +49573 +49574 +49575 +49576 +49577 +49578 +49579 +49580 +49581 +49582 +49583 +49584 +49585 +49586 +49587 +49588 +49589 +49590 +49591 +49592 +49593 +49594 +49595 +49596 +49597 +49598 +49599 +49600 +49601 +49602 +49603 +49604 +49605 +49606 +49607 +49608 +49609 +49610 +49611 +49612 +49613 +49614 +49615 +49616 +49617 +49618 +49619 +49620 +49621 +49622 +49623 +49624 +49625 +49626 +49627 +49628 +49629 +49630 +49631 +49632 +49633 +49634 +49635 +49636 +49637 +49638 +49639 +49640 +49641 +49642 +49643 +49644 +49645 +49646 +49647 +49648 +49649 +49650 +49651 +49652 +49653 +49654 +49655 +49656 +49657 +49658 +49659 +49660 +49661 +49662 +49663 +49664 +49665 +49666 +49667 +49668 +49669 +49670 +49671 +49672 +49673 +49674 +49675 +49676 +49677 +49678 +49679 +49680 +49681 +49682 +49683 +49684 +49685 +49686 +49687 +49688 +49689 +49690 +49691 +49692 +49693 +49694 +49695 +49696 +49697 +49698 +49699 +49700 +49701 +49702 +49703 +49704 +49705 +49706 +49707 +49708 +49709 +49710 +49711 +49712 +49713 +49714 +49715 +49716 +49717 +49718 +49719 +49720 +49721 +49722 +49723 +49724 +49725 +49726 +49727 +49728 +49729 +49730 +49731 +49732 +49733 +49734 +49735 +49736 +49737 +49738 +49739 +49740 +49741 +49742 +49743 +49744 +49745 +49746 +49747 +49748 +49749 +49750 +49751 +49752 +49753 +49754 +49755 +49756 +49757 +49758 +49759 +49760 +49761 +49762 +49763 +49764 +49765 +49766 +49767 +49768 +49769 +49770 +49771 +49772 +49773 +49774 +49775 +49776 +49777 +49778 +49779 +49780 +49781 +49782 +49783 +49784 +49785 +49786 +49787 +49788 +49789 +49790 +49791 +49792 +49793 +49794 +49795 +49796 +49797 +49798 +49799 +49800 +49801 +49802 +49803 +49804 +49805 +49806 +49807 +49808 +49809 +49810 +49811 +49812 +49813 +49814 +49815 +49816 +49817 +49818 +49819 +49820 +49821 +49822 +49823 +49824 +49825 +49826 +49827 +49828 +49829 +49830 +49831 +49832 +49833 +49834 +49835 +49836 +49837 +49838 +49839 +49840 +49841 +49842 +49843 +49844 +49845 +49846 +49847 +49848 +49849 +49850 +49851 +49852 +49853 +49854 +49855 +49856 +49857 +49858 +49859 +49860 +49861 +49862 +49863 +49864 +49865 +49866 +49867 +49868 +49869 +49870 +49871 +49872 +49873 +49874 +49875 +49876 +49877 +49878 +49879 +49880 +49881 +49882 +49883 +49884 +49885 +49886 +49887 +49888 +49889 +49890 +49891 +49892 +49893 +49894 +49895 +49896 +49897 +49898 +49899 +49900 +49901 +49902 +49903 +49904 +49905 +49906 +49907 +49908 +49909 +49910 +49911 +49912 +49913 +49914 +49915 +49916 +49917 +49918 +49919 +49920 +49921 +49922 +49923 +49924 +49925 +49926 +49927 +49928 +49929 +49930 +49931 +49932 +49933 +49934 +49935 +49936 +49937 +49938 +49939 +49940 +49941 +49942 +49943 +49944 +49945 +49946 +49947 +49948 +49949 +49950 +49951 +49952 +49953 +49954 +49955 +49956 +49957 +49958 +49959 +49960 +49961 +49962 +49963 +49964 +49965 +49966 +49967 +49968 +49969 +49970 +49971 +49972 +49973 +49974 +49975 +49976 +49977 +49978 +49979 +49980 +49981 +49982 +49983 +49984 +49985 +49986 +49987 +49988 +49989 +49990 +49991 +49992 +49993 +49994 +49995 +49996 +49997 +49998 +49999 +50000 +50001 +50002 +50003 +50004 +50005 +50006 +50007 +50008 +50009 +50010 +50011 +50012 +50013 +50014 +50015 +50016 +50017 +50018 +50019 +50020 +50021 +50022 +50023 +50024 +50025 +50026 +50027 +50028 +50029 +50030 +50031 +50032 +50033 +50034 +50035 +50036 +50037 +50038 +50039 +50040 +50041 +50042 +50043 +50044 +50045 +50046 +50047 +50048 +50049 +50050 +50051 +50052 +50053 +50054 +50055 +50056 +50057 +50058 +50059 +50060 +50061 +50062 +50063 +50064 +50065 +50066 +50067 +50068 +50069 +50070 +50071 +50072 +50073 +50074 +50075 +50076 +50077 +50078 +50079 +50080 +50081 +50082 +50083 +50084 +50085 +50086 +50087 +50088 +50089 +50090 +50091 +50092 +50093 +50094 +50095 +50096 +50097 +50098 +50099 +50100 +50101 +50102 +50103 +50104 +50105 +50106 +50107 +50108 +50109 +50110 +50111 +50112 +50113 +50114 +50115 +50116 +50117 +50118 +50119 +50120 +50121 +50122 +50123 +50124 +50125 +50126 +50127 +50128 +50129 +50130 +50131 +50132 +50133 +50134 +50135 +50136 +50137 +50138 +50139 +50140 +50141 +50142 +50143 +50144 +50145 +50146 +50147 +50148 +50149 +50150 +50151 +50152 +50153 +50154 +50155 +50156 +50157 +50158 +50159 +50160 +50161 +50162 +50163 +50164 +50165 +50166 +50167 +50168 +50169 +50170 +50171 +50172 +50173 +50174 +50175 +50176 +50177 +50178 +50179 +50180 +50181 +50182 +50183 +50184 +50185 +50186 +50187 +50188 +50189 +50190 +50191 +50192 +50193 +50194 +50195 +50196 +50197 +50198 +50199 +50200 +50201 +50202 +50203 +50204 +50205 +50206 +50207 +50208 +50209 +50210 +50211 +50212 +50213 +50214 +50215 +50216 +50217 +50218 +50219 +50220 +50221 +50222 +50223 +50224 +50225 +50226 +50227 +50228 +50229 +50230 +50231 +50232 +50233 +50234 +50235 +50236 +50237 +50238 +50239 +50240 +50241 +50242 +50243 +50244 +50245 +50246 +50247 +50248 +50249 +50250 +50251 +50252 +50253 +50254 +50255 +50256 +50257 +50258 +50259 +50260 +50261 +50262 +50263 +50264 +50265 +50266 +50267 +50268 +50269 +50270 +50271 +50272 +50273 +50274 +50275 +50276 +50277 +50278 +50279 +50280 +50281 +50282 +50283 +50284 +50285 +50286 +50287 +50288 +50289 +50290 +50291 +50292 +50293 +50294 +50295 +50296 +50297 +50298 +50299 +50300 +50301 +50302 +50303 +50304 +50305 +50306 +50307 +50308 +50309 +50310 +50311 +50312 +50313 +50314 +50315 +50316 +50317 +50318 +50319 +50320 +50321 +50322 +50323 +50324 +50325 +50326 +50327 +50328 +50329 +50330 +50331 +50332 +50333 +50334 +50335 +50336 +50337 +50338 +50339 +50340 +50341 +50342 +50343 +50344 +50345 +50346 +50347 +50348 +50349 +50350 +50351 +50352 +50353 +50354 +50355 +50356 +50357 +50358 +50359 +50360 +50361 +50362 +50363 +50364 +50365 +50366 +50367 +50368 +50369 +50370 +50371 +50372 +50373 +50374 +50375 +50376 +50377 +50378 +50379 +50380 +50381 +50382 +50383 +50384 +50385 +50386 +50387 +50388 +50389 +50390 +50391 +50392 +50393 +50394 +50395 +50396 +50397 +50398 +50399 +50400 +50401 +50402 +50403 +50404 +50405 +50406 +50407 +50408 +50409 +50410 +50411 +50412 +50413 +50414 +50415 +50416 +50417 +50418 +50419 +50420 +50421 +50422 +50423 +50424 +50425 +50426 +50427 +50428 +50429 +50430 +50431 +50432 +50433 +50434 +50435 +50436 +50437 +50438 +50439 +50440 +50441 +50442 +50443 +50444 +50445 +50446 +50447 +50448 +50449 +50450 +50451 +50452 +50453 +50454 +50455 +50456 +50457 +50458 +50459 +50460 +50461 +50462 +50463 +50464 +50465 +50466 +50467 +50468 +50469 +50470 +50471 +50472 +50473 +50474 +50475 +50476 +50477 +50478 +50479 +50480 +50481 +50482 +50483 +50484 +50485 +50486 +50487 +50488 +50489 +50490 +50491 +50492 +50493 +50494 +50495 +50496 +50497 +50498 +50499 +50500 +50501 +50502 +50503 +50504 +50505 +50506 +50507 +50508 +50509 +50510 +50511 +50512 +50513 +50514 +50515 +50516 +50517 +50518 +50519 +50520 +50521 +50522 +50523 +50524 +50525 +50526 +50527 +50528 +50529 +50530 +50531 +50532 +50533 +50534 +50535 +50536 +50537 +50538 +50539 +50540 +50541 +50542 +50543 +50544 +50545 +50546 +50547 +50548 +50549 +50550 +50551 +50552 +50553 +50554 +50555 +50556 +50557 +50558 +50559 +50560 +50561 +50562 +50563 +50564 +50565 +50566 +50567 +50568 +50569 +50570 +50571 +50572 +50573 +50574 +50575 +50576 +50577 +50578 +50579 +50580 +50581 +50582 +50583 +50584 +50585 +50586 +50587 +50588 +50589 +50590 +50591 +50592 +50593 +50594 +50595 +50596 +50597 +50598 +50599 +50600 +50601 +50602 +50603 +50604 +50605 +50606 +50607 +50608 +50609 +50610 +50611 +50612 +50613 +50614 +50615 +50616 +50617 +50618 +50619 +50620 +50621 +50622 +50623 +50624 +50625 +50626 +50627 +50628 +50629 +50630 +50631 +50632 +50633 +50634 +50635 +50636 +50637 +50638 +50639 +50640 +50641 +50642 +50643 +50644 +50645 +50646 +50647 +50648 +50649 +50650 +50651 +50652 +50653 +50654 +50655 +50656 +50657 +50658 +50659 +50660 +50661 +50662 +50663 +50664 +50665 +50666 +50667 +50668 +50669 +50670 +50671 +50672 +50673 +50674 +50675 +50676 +50677 +50678 +50679 +50680 +50681 +50682 +50683 +50684 +50685 +50686 +50687 +50688 +50689 +50690 +50691 +50692 +50693 +50694 +50695 +50696 +50697 +50698 +50699 +50700 +50701 +50702 +50703 +50704 +50705 +50706 +50707 +50708 +50709 +50710 +50711 +50712 +50713 +50714 +50715 +50716 +50717 +50718 +50719 +50720 +50721 +50722 +50723 +50724 +50725 +50726 +50727 +50728 +50729 +50730 +50731 +50732 +50733 +50734 +50735 +50736 +50737 +50738 +50739 +50740 +50741 +50742 +50743 +50744 +50745 +50746 +50747 +50748 +50749 +50750 +50751 +50752 +50753 +50754 +50755 +50756 +50757 +50758 +50759 +50760 +50761 +50762 +50763 +50764 +50765 +50766 +50767 +50768 +50769 +50770 +50771 +50772 +50773 +50774 +50775 +50776 +50777 +50778 +50779 +50780 +50781 +50782 +50783 +50784 +50785 +50786 +50787 +50788 +50789 +50790 +50791 +50792 +50793 +50794 +50795 +50796 +50797 +50798 +50799 +50800 +50801 +50802 +50803 +50804 +50805 +50806 +50807 +50808 +50809 +50810 +50811 +50812 +50813 +50814 +50815 +50816 +50817 +50818 +50819 +50820 +50821 +50822 +50823 +50824 +50825 +50826 +50827 +50828 +50829 +50830 +50831 +50832 +50833 +50834 +50835 +50836 +50837 +50838 +50839 +50840 +50841 +50842 +50843 +50844 +50845 +50846 +50847 +50848 +50849 +50850 +50851 +50852 +50853 +50854 +50855 +50856 +50857 +50858 +50859 +50860 +50861 +50862 +50863 +50864 +50865 +50866 +50867 +50868 +50869 +50870 +50871 +50872 +50873 +50874 +50875 +50876 +50877 +50878 +50879 +50880 +50881 +50882 +50883 +50884 +50885 +50886 +50887 +50888 +50889 +50890 +50891 +50892 +50893 +50894 +50895 +50896 +50897 +50898 +50899 +50900 +50901 +50902 +50903 +50904 +50905 +50906 +50907 +50908 +50909 +50910 +50911 +50912 +50913 +50914 +50915 +50916 +50917 +50918 +50919 +50920 +50921 +50922 +50923 +50924 +50925 +50926 +50927 +50928 +50929 +50930 +50931 +50932 +50933 +50934 +50935 +50936 +50937 +50938 +50939 +50940 +50941 +50942 +50943 +50944 +50945 +50946 +50947 +50948 +50949 +50950 +50951 +50952 +50953 +50954 +50955 +50956 +50957 +50958 +50959 +50960 +50961 +50962 +50963 +50964 +50965 +50966 +50967 +50968 +50969 +50970 +50971 +50972 +50973 +50974 +50975 +50976 +50977 +50978 +50979 +50980 +50981 +50982 +50983 +50984 +50985 +50986 +50987 +50988 +50989 +50990 +50991 +50992 +50993 +50994 +50995 +50996 +50997 +50998 +50999 +51000 +51001 +51002 +51003 +51004 +51005 +51006 +51007 +51008 +51009 +51010 +51011 +51012 +51013 +51014 +51015 +51016 +51017 +51018 +51019 +51020 +51021 +51022 +51023 +51024 +51025 +51026 +51027 +51028 +51029 +51030 +51031 +51032 +51033 +51034 +51035 +51036 +51037 +51038 +51039 +51040 +51041 +51042 +51043 +51044 +51045 +51046 +51047 +51048 +51049 +51050 +51051 +51052 +51053 +51054 +51055 +51056 +51057 +51058 +51059 +51060 +51061 +51062 +51063 +51064 +51065 +51066 +51067 +51068 +51069 +51070 +51071 +51072 +51073 +51074 +51075 +51076 +51077 +51078 +51079 +51080 +51081 +51082 +51083 +51084 +51085 +51086 +51087 +51088 +51089 +51090 +51091 +51092 +51093 +51094 +51095 +51096 +51097 +51098 +51099 +51100 +51101 +51102 +51103 +51104 +51105 +51106 +51107 +51108 +51109 +51110 +51111 +51112 +51113 +51114 +51115 +51116 +51117 +51118 +51119 +51120 +51121 +51122 +51123 +51124 +51125 +51126 +51127 +51128 +51129 +51130 +51131 +51132 +51133 +51134 +51135 +51136 +51137 +51138 +51139 +51140 +51141 +51142 +51143 +51144 +51145 +51146 +51147 +51148 +51149 +51150 +51151 +51152 +51153 +51154 +51155 +51156 +51157 +51158 +51159 +51160 +51161 +51162 +51163 +51164 +51165 +51166 +51167 +51168 +51169 +51170 +51171 +51172 +51173 +51174 +51175 +51176 +51177 +51178 +51179 +51180 +51181 +51182 +51183 +51184 +51185 +51186 +51187 +51188 +51189 +51190 +51191 +51192 +51193 +51194 +51195 +51196 +51197 +51198 +51199 +51200 +51201 +51202 +51203 +51204 +51205 +51206 +51207 +51208 +51209 +51210 +51211 +51212 +51213 +51214 +51215 +51216 +51217 +51218 +51219 +51220 +51221 +51222 +51223 +51224 +51225 +51226 +51227 +51228 +51229 +51230 +51231 +51232 +51233 +51234 +51235 +51236 +51237 +51238 +51239 +51240 +51241 +51242 +51243 +51244 +51245 +51246 +51247 +51248 +51249 +51250 +51251 +51252 +51253 +51254 +51255 +51256 +51257 +51258 +51259 +51260 +51261 +51262 +51263 +51264 +51265 +51266 +51267 +51268 +51269 +51270 +51271 +51272 +51273 +51274 +51275 +51276 +51277 +51278 +51279 +51280 +51281 +51282 +51283 +51284 +51285 +51286 +51287 +51288 +51289 +51290 +51291 +51292 +51293 +51294 +51295 +51296 +51297 +51298 +51299 +51300 +51301 +51302 +51303 +51304 +51305 +51306 +51307 +51308 +51309 +51310 +51311 +51312 +51313 +51314 +51315 +51316 +51317 +51318 +51319 +51320 +51321 +51322 +51323 +51324 +51325 +51326 +51327 +51328 +51329 +51330 +51331 +51332 +51333 +51334 +51335 +51336 +51337 +51338 +51339 +51340 +51341 +51342 +51343 +51344 +51345 +51346 +51347 +51348 +51349 +51350 +51351 +51352 +51353 +51354 +51355 +51356 +51357 +51358 +51359 +51360 +51361 +51362 +51363 +51364 +51365 +51366 +51367 +51368 +51369 +51370 +51371 +51372 +51373 +51374 +51375 +51376 +51377 +51378 +51379 +51380 +51381 +51382 +51383 +51384 +51385 +51386 +51387 +51388 +51389 +51390 +51391 +51392 +51393 +51394 +51395 +51396 +51397 +51398 +51399 +51400 +51401 +51402 +51403 +51404 +51405 +51406 +51407 +51408 +51409 +51410 +51411 +51412 +51413 +51414 +51415 +51416 +51417 +51418 +51419 +51420 +51421 +51422 +51423 +51424 +51425 +51426 +51427 +51428 +51429 +51430 +51431 +51432 +51433 +51434 +51435 +51436 +51437 +51438 +51439 +51440 +51441 +51442 +51443 +51444 +51445 +51446 +51447 +51448 +51449 +51450 +51451 +51452 +51453 +51454 +51455 +51456 +51457 +51458 +51459 +51460 +51461 +51462 +51463 +51464 +51465 +51466 +51467 +51468 +51469 +51470 +51471 +51472 +51473 +51474 +51475 +51476 +51477 +51478 +51479 +51480 +51481 +51482 +51483 +51484 +51485 +51486 +51487 +51488 +51489 +51490 +51491 +51492 +51493 +51494 +51495 +51496 +51497 +51498 +51499 +51500 +51501 +51502 +51503 +51504 +51505 +51506 +51507 +51508 +51509 +51510 +51511 +51512 +51513 +51514 +51515 +51516 +51517 +51518 +51519 +51520 +51521 +51522 +51523 +51524 +51525 +51526 +51527 +51528 +51529 +51530 +51531 +51532 +51533 +51534 +51535 +51536 +51537 +51538 +51539 +51540 +51541 +51542 +51543 +51544 +51545 +51546 +51547 +51548 +51549 +51550 +51551 +51552 +51553 +51554 +51555 +51556 +51557 +51558 +51559 +51560 +51561 +51562 +51563 +51564 +51565 +51566 +51567 +51568 +51569 +51570 +51571 +51572 +51573 +51574 +51575 +51576 +51577 +51578 +51579 +51580 +51581 +51582 +51583 +51584 +51585 +51586 +51587 +51588 +51589 +51590 +51591 +51592 +51593 +51594 +51595 +51596 +51597 +51598 +51599 +51600 +51601 +51602 +51603 +51604 +51605 +51606 +51607 +51608 +51609 +51610 +51611 +51612 +51613 +51614 +51615 +51616 +51617 +51618 +51619 +51620 +51621 +51622 +51623 +51624 +51625 +51626 +51627 +51628 +51629 +51630 +51631 +51632 +51633 +51634 +51635 +51636 +51637 +51638 +51639 +51640 +51641 +51642 +51643 +51644 +51645 +51646 +51647 +51648 +51649 +51650 +51651 +51652 +51653 +51654 +51655 +51656 +51657 +51658 +51659 +51660 +51661 +51662 +51663 +51664 +51665 +51666 +51667 +51668 +51669 +51670 +51671 +51672 +51673 +51674 +51675 +51676 +51677 +51678 +51679 +51680 +51681 +51682 +51683 +51684 +51685 +51686 +51687 +51688 +51689 +51690 +51691 +51692 +51693 +51694 +51695 +51696 +51697 +51698 +51699 +51700 +51701 +51702 +51703 +51704 +51705 +51706 +51707 +51708 +51709 +51710 +51711 +51712 +51713 +51714 +51715 +51716 +51717 +51718 +51719 +51720 +51721 +51722 +51723 +51724 +51725 +51726 +51727 +51728 +51729 +51730 +51731 +51732 +51733 +51734 +51735 +51736 +51737 +51738 +51739 +51740 +51741 +51742 +51743 +51744 +51745 +51746 +51747 +51748 +51749 +51750 +51751 +51752 +51753 +51754 +51755 +51756 +51757 +51758 +51759 +51760 +51761 +51762 +51763 +51764 +51765 +51766 +51767 +51768 +51769 +51770 +51771 +51772 +51773 +51774 +51775 +51776 +51777 +51778 +51779 +51780 +51781 +51782 +51783 +51784 +51785 +51786 +51787 +51788 +51789 +51790 +51791 +51792 +51793 +51794 +51795 +51796 +51797 +51798 +51799 +51800 +51801 +51802 +51803 +51804 +51805 +51806 +51807 +51808 +51809 +51810 +51811 +51812 +51813 +51814 +51815 +51816 +51817 +51818 +51819 +51820 +51821 +51822 +51823 +51824 +51825 +51826 +51827 +51828 +51829 +51830 +51831 +51832 +51833 +51834 +51835 +51836 +51837 +51838 +51839 +51840 +51841 +51842 +51843 +51844 +51845 +51846 +51847 +51848 +51849 +51850 +51851 +51852 +51853 +51854 +51855 +51856 +51857 +51858 +51859 +51860 +51861 +51862 +51863 +51864 +51865 +51866 +51867 +51868 +51869 +51870 +51871 +51872 +51873 +51874 +51875 +51876 +51877 +51878 +51879 +51880 +51881 +51882 +51883 +51884 +51885 +51886 +51887 +51888 +51889 +51890 +51891 +51892 +51893 +51894 +51895 +51896 +51897 +51898 +51899 +51900 +51901 +51902 +51903 +51904 +51905 +51906 +51907 +51908 +51909 +51910 +51911 +51912 +51913 +51914 +51915 +51916 +51917 +51918 +51919 +51920 +51921 +51922 +51923 +51924 +51925 +51926 +51927 +51928 +51929 +51930 +51931 +51932 +51933 +51934 +51935 +51936 +51937 +51938 +51939 +51940 +51941 +51942 +51943 +51944 +51945 +51946 +51947 +51948 +51949 +51950 +51951 +51952 +51953 +51954 +51955 +51956 +51957 +51958 +51959 +51960 +51961 +51962 +51963 +51964 +51965 +51966 +51967 +51968 +51969 +51970 +51971 +51972 +51973 +51974 +51975 +51976 +51977 +51978 +51979 +51980 +51981 +51982 +51983 +51984 +51985 +51986 +51987 +51988 +51989 +51990 +51991 +51992 +51993 +51994 +51995 +51996 +51997 +51998 +51999 +52000 +52001 +52002 +52003 +52004 +52005 +52006 +52007 +52008 +52009 +52010 +52011 +52012 +52013 +52014 +52015 +52016 +52017 +52018 +52019 +52020 +52021 +52022 +52023 +52024 +52025 +52026 +52027 +52028 +52029 +52030 +52031 +52032 +52033 +52034 +52035 +52036 +52037 +52038 +52039 +52040 +52041 +52042 +52043 +52044 +52045 +52046 +52047 +52048 +52049 +52050 +52051 +52052 +52053 +52054 +52055 +52056 +52057 +52058 +52059 +52060 +52061 +52062 +52063 +52064 +52065 +52066 +52067 +52068 +52069 +52070 +52071 +52072 +52073 +52074 +52075 +52076 +52077 +52078 +52079 +52080 +52081 +52082 +52083 +52084 +52085 +52086 +52087 +52088 +52089 +52090 +52091 +52092 +52093 +52094 +52095 +52096 +52097 +52098 +52099 +52100 +52101 +52102 +52103 +52104 +52105 +52106 +52107 +52108 +52109 +52110 +52111 +52112 +52113 +52114 +52115 +52116 +52117 +52118 +52119 +52120 +52121 +52122 +52123 +52124 +52125 +52126 +52127 +52128 +52129 +52130 +52131 +52132 +52133 +52134 +52135 +52136 +52137 +52138 +52139 +52140 +52141 +52142 +52143 +52144 +52145 +52146 +52147 +52148 +52149 +52150 +52151 +52152 +52153 +52154 +52155 +52156 +52157 +52158 +52159 +52160 +52161 +52162 +52163 +52164 +52165 +52166 +52167 +52168 +52169 +52170 +52171 +52172 +52173 +52174 +52175 +52176 +52177 +52178 +52179 +52180 +52181 +52182 +52183 +52184 +52185 +52186 +52187 +52188 +52189 +52190 +52191 +52192 +52193 +52194 +52195 +52196 +52197 +52198 +52199 +52200 +52201 +52202 +52203 +52204 +52205 +52206 +52207 +52208 +52209 +52210 +52211 +52212 +52213 +52214 +52215 +52216 +52217 +52218 +52219 +52220 +52221 +52222 +52223 +52224 +52225 +52226 +52227 +52228 +52229 +52230 +52231 +52232 +52233 +52234 +52235 +52236 +52237 +52238 +52239 +52240 +52241 +52242 +52243 +52244 +52245 +52246 +52247 +52248 +52249 +52250 +52251 +52252 +52253 +52254 +52255 +52256 +52257 +52258 +52259 +52260 +52261 +52262 +52263 +52264 +52265 +52266 +52267 +52268 +52269 +52270 +52271 +52272 +52273 +52274 +52275 +52276 +52277 +52278 +52279 +52280 +52281 +52282 +52283 +52284 +52285 +52286 +52287 +52288 +52289 +52290 +52291 +52292 +52293 +52294 +52295 +52296 +52297 +52298 +52299 +52300 +52301 +52302 +52303 +52304 +52305 +52306 +52307 +52308 +52309 +52310 +52311 +52312 +52313 +52314 +52315 +52316 +52317 +52318 +52319 +52320 +52321 +52322 +52323 +52324 +52325 +52326 +52327 +52328 +52329 +52330 +52331 +52332 +52333 +52334 +52335 +52336 +52337 +52338 +52339 +52340 +52341 +52342 +52343 +52344 +52345 +52346 +52347 +52348 +52349 +52350 +52351 +52352 +52353 +52354 +52355 +52356 +52357 +52358 +52359 +52360 +52361 +52362 +52363 +52364 +52365 +52366 +52367 +52368 +52369 +52370 +52371 +52372 +52373 +52374 +52375 +52376 +52377 +52378 +52379 +52380 +52381 +52382 +52383 +52384 +52385 +52386 +52387 +52388 +52389 +52390 +52391 +52392 +52393 +52394 +52395 +52396 +52397 +52398 +52399 +52400 +52401 +52402 +52403 +52404 +52405 +52406 +52407 +52408 +52409 +52410 +52411 +52412 +52413 +52414 +52415 +52416 +52417 +52418 +52419 +52420 +52421 +52422 +52423 +52424 +52425 +52426 +52427 +52428 +52429 +52430 +52431 +52432 +52433 +52434 +52435 +52436 +52437 +52438 +52439 +52440 +52441 +52442 +52443 +52444 +52445 +52446 +52447 +52448 +52449 +52450 +52451 +52452 +52453 +52454 +52455 +52456 +52457 +52458 +52459 +52460 +52461 +52462 +52463 +52464 +52465 +52466 +52467 +52468 +52469 +52470 +52471 +52472 +52473 +52474 +52475 +52476 +52477 +52478 +52479 +52480 +52481 +52482 +52483 +52484 +52485 +52486 +52487 +52488 +52489 +52490 +52491 +52492 +52493 +52494 +52495 +52496 +52497 +52498 +52499 +52500 +52501 +52502 +52503 +52504 +52505 +52506 +52507 +52508 +52509 +52510 +52511 +52512 +52513 +52514 +52515 +52516 +52517 +52518 +52519 +52520 +52521 +52522 +52523 +52524 +52525 +52526 +52527 +52528 +52529 +52530 +52531 +52532 +52533 +52534 +52535 +52536 +52537 +52538 +52539 +52540 +52541 +52542 +52543 +52544 +52545 +52546 +52547 +52548 +52549 +52550 +52551 +52552 +52553 +52554 +52555 +52556 +52557 +52558 +52559 +52560 +52561 +52562 +52563 +52564 +52565 +52566 +52567 +52568 +52569 +52570 +52571 +52572 +52573 +52574 +52575 +52576 +52577 +52578 +52579 +52580 +52581 +52582 +52583 +52584 +52585 +52586 +52587 +52588 +52589 +52590 +52591 +52592 +52593 +52594 +52595 +52596 +52597 +52598 +52599 +52600 +52601 +52602 +52603 +52604 +52605 +52606 +52607 +52608 +52609 +52610 +52611 +52612 +52613 +52614 +52615 +52616 +52617 +52618 +52619 +52620 +52621 +52622 +52623 +52624 +52625 +52626 +52627 +52628 +52629 +52630 +52631 +52632 +52633 +52634 +52635 +52636 +52637 +52638 +52639 +52640 +52641 +52642 +52643 +52644 +52645 +52646 +52647 +52648 +52649 +52650 +52651 +52652 +52653 +52654 +52655 +52656 +52657 +52658 +52659 +52660 +52661 +52662 +52663 +52664 +52665 +52666 +52667 +52668 +52669 +52670 +52671 +52672 +52673 +52674 +52675 +52676 +52677 +52678 +52679 +52680 +52681 +52682 +52683 +52684 +52685 +52686 +52687 +52688 +52689 +52690 +52691 +52692 +52693 +52694 +52695 +52696 +52697 +52698 +52699 +52700 +52701 +52702 +52703 +52704 +52705 +52706 +52707 +52708 +52709 +52710 +52711 +52712 +52713 +52714 +52715 +52716 +52717 +52718 +52719 +52720 +52721 +52722 +52723 +52724 +52725 +52726 +52727 +52728 +52729 +52730 +52731 +52732 +52733 +52734 +52735 +52736 +52737 +52738 +52739 +52740 +52741 +52742 +52743 +52744 +52745 +52746 +52747 +52748 +52749 +52750 +52751 +52752 +52753 +52754 +52755 +52756 +52757 +52758 +52759 +52760 +52761 +52762 +52763 +52764 +52765 +52766 +52767 +52768 +52769 +52770 +52771 +52772 +52773 +52774 +52775 +52776 +52777 +52778 +52779 +52780 +52781 +52782 +52783 +52784 +52785 +52786 +52787 +52788 +52789 +52790 +52791 +52792 +52793 +52794 +52795 +52796 +52797 +52798 +52799 +52800 +52801 +52802 +52803 +52804 +52805 +52806 +52807 +52808 +52809 +52810 +52811 +52812 +52813 +52814 +52815 +52816 +52817 +52818 +52819 +52820 +52821 +52822 +52823 +52824 +52825 +52826 +52827 +52828 +52829 +52830 +52831 +52832 +52833 +52834 +52835 +52836 +52837 +52838 +52839 +52840 +52841 +52842 +52843 +52844 +52845 +52846 +52847 +52848 +52849 +52850 +52851 +52852 +52853 +52854 +52855 +52856 +52857 +52858 +52859 +52860 +52861 +52862 +52863 +52864 +52865 +52866 +52867 +52868 +52869 +52870 +52871 +52872 +52873 +52874 +52875 +52876 +52877 +52878 +52879 +52880 +52881 +52882 +52883 +52884 +52885 +52886 +52887 +52888 +52889 +52890 +52891 +52892 +52893 +52894 +52895 +52896 +52897 +52898 +52899 +52900 +52901 +52902 +52903 +52904 +52905 +52906 +52907 +52908 +52909 +52910 +52911 +52912 +52913 +52914 +52915 +52916 +52917 +52918 +52919 +52920 +52921 +52922 +52923 +52924 +52925 +52926 +52927 +52928 +52929 +52930 +52931 +52932 +52933 +52934 +52935 +52936 +52937 +52938 +52939 +52940 +52941 +52942 +52943 +52944 +52945 +52946 +52947 +52948 +52949 +52950 +52951 +52952 +52953 +52954 +52955 +52956 +52957 +52958 +52959 +52960 +52961 +52962 +52963 +52964 +52965 +52966 +52967 +52968 +52969 +52970 +52971 +52972 +52973 +52974 +52975 +52976 +52977 +52978 +52979 +52980 +52981 +52982 +52983 +52984 +52985 +52986 +52987 +52988 +52989 +52990 +52991 +52992 +52993 +52994 +52995 +52996 +52997 +52998 +52999 +53000 +53001 +53002 +53003 +53004 +53005 +53006 +53007 +53008 +53009 +53010 +53011 +53012 +53013 +53014 +53015 +53016 +53017 +53018 +53019 +53020 +53021 +53022 +53023 +53024 +53025 +53026 +53027 +53028 +53029 +53030 +53031 +53032 +53033 +53034 +53035 +53036 +53037 +53038 +53039 +53040 +53041 +53042 +53043 +53044 +53045 +53046 +53047 +53048 +53049 +53050 +53051 +53052 +53053 +53054 +53055 +53056 +53057 +53058 +53059 +53060 +53061 +53062 +53063 +53064 +53065 +53066 +53067 +53068 +53069 +53070 +53071 +53072 +53073 +53074 +53075 +53076 +53077 +53078 +53079 +53080 +53081 +53082 +53083 +53084 +53085 +53086 +53087 +53088 +53089 +53090 +53091 +53092 +53093 +53094 +53095 +53096 +53097 +53098 +53099 +53100 +53101 +53102 +53103 +53104 +53105 +53106 +53107 +53108 +53109 +53110 +53111 +53112 +53113 +53114 +53115 +53116 +53117 +53118 +53119 +53120 +53121 +53122 +53123 +53124 +53125 +53126 +53127 +53128 +53129 +53130 +53131 +53132 +53133 +53134 +53135 +53136 +53137 +53138 +53139 +53140 +53141 +53142 +53143 +53144 +53145 +53146 +53147 +53148 +53149 +53150 +53151 +53152 +53153 +53154 +53155 +53156 +53157 +53158 +53159 +53160 +53161 +53162 +53163 +53164 +53165 +53166 +53167 +53168 +53169 +53170 +53171 +53172 +53173 +53174 +53175 +53176 +53177 +53178 +53179 +53180 +53181 +53182 +53183 +53184 +53185 +53186 +53187 +53188 +53189 +53190 +53191 +53192 +53193 +53194 +53195 +53196 +53197 +53198 +53199 +53200 +53201 +53202 +53203 +53204 +53205 +53206 +53207 +53208 +53209 +53210 +53211 +53212 +53213 +53214 +53215 +53216 +53217 +53218 +53219 +53220 +53221 +53222 +53223 +53224 +53225 +53226 +53227 +53228 +53229 +53230 +53231 +53232 +53233 +53234 +53235 +53236 +53237 +53238 +53239 +53240 +53241 +53242 +53243 +53244 +53245 +53246 +53247 +53248 +53249 +53250 +53251 +53252 +53253 +53254 +53255 +53256 +53257 +53258 +53259 +53260 +53261 +53262 +53263 +53264 +53265 +53266 +53267 +53268 +53269 +53270 +53271 +53272 +53273 +53274 +53275 +53276 +53277 +53278 +53279 +53280 +53281 +53282 +53283 +53284 +53285 +53286 +53287 +53288 +53289 +53290 +53291 +53292 +53293 +53294 +53295 +53296 +53297 +53298 +53299 +53300 +53301 +53302 +53303 +53304 +53305 +53306 +53307 +53308 +53309 +53310 +53311 +53312 +53313 +53314 +53315 +53316 +53317 +53318 +53319 +53320 +53321 +53322 +53323 +53324 +53325 +53326 +53327 +53328 +53329 +53330 +53331 +53332 +53333 +53334 +53335 +53336 +53337 +53338 +53339 +53340 +53341 +53342 +53343 +53344 +53345 +53346 +53347 +53348 +53349 +53350 +53351 +53352 +53353 +53354 +53355 +53356 +53357 +53358 +53359 +53360 +53361 +53362 +53363 +53364 +53365 +53366 +53367 +53368 +53369 +53370 +53371 +53372 +53373 +53374 +53375 +53376 +53377 +53378 +53379 +53380 +53381 +53382 +53383 +53384 +53385 +53386 +53387 +53388 +53389 +53390 +53391 +53392 +53393 +53394 +53395 +53396 +53397 +53398 +53399 +53400 +53401 +53402 +53403 +53404 +53405 +53406 +53407 +53408 +53409 +53410 +53411 +53412 +53413 +53414 +53415 +53416 +53417 +53418 +53419 +53420 +53421 +53422 +53423 +53424 +53425 +53426 +53427 +53428 +53429 +53430 +53431 +53432 +53433 +53434 +53435 +53436 +53437 +53438 +53439 +53440 +53441 +53442 +53443 +53444 +53445 +53446 +53447 +53448 +53449 +53450 +53451 +53452 +53453 +53454 +53455 +53456 +53457 +53458 +53459 +53460 +53461 +53462 +53463 +53464 +53465 +53466 +53467 +53468 +53469 +53470 +53471 +53472 +53473 +53474 +53475 +53476 +53477 +53478 +53479 +53480 +53481 +53482 +53483 +53484 +53485 +53486 +53487 +53488 +53489 +53490 +53491 +53492 +53493 +53494 +53495 +53496 +53497 +53498 +53499 +53500 +53501 +53502 +53503 +53504 +53505 +53506 +53507 +53508 +53509 +53510 +53511 +53512 +53513 +53514 +53515 +53516 +53517 +53518 +53519 +53520 +53521 +53522 +53523 +53524 +53525 +53526 +53527 +53528 +53529 +53530 +53531 +53532 +53533 +53534 +53535 +53536 +53537 +53538 +53539 +53540 +53541 +53542 +53543 +53544 +53545 +53546 +53547 +53548 +53549 +53550 +53551 +53552 +53553 +53554 +53555 +53556 +53557 +53558 +53559 +53560 +53561 +53562 +53563 +53564 +53565 +53566 +53567 +53568 +53569 +53570 +53571 +53572 +53573 +53574 +53575 +53576 +53577 +53578 +53579 +53580 +53581 +53582 +53583 +53584 +53585 +53586 +53587 +53588 +53589 +53590 +53591 +53592 +53593 +53594 +53595 +53596 +53597 +53598 +53599 +53600 +53601 +53602 +53603 +53604 +53605 +53606 +53607 +53608 +53609 +53610 +53611 +53612 +53613 +53614 +53615 +53616 +53617 +53618 +53619 +53620 +53621 +53622 +53623 +53624 +53625 +53626 +53627 +53628 +53629 +53630 +53631 +53632 +53633 +53634 +53635 +53636 +53637 +53638 +53639 +53640 +53641 +53642 +53643 +53644 +53645 +53646 +53647 +53648 +53649 +53650 +53651 +53652 +53653 +53654 +53655 +53656 +53657 +53658 +53659 +53660 +53661 +53662 +53663 +53664 +53665 +53666 +53667 +53668 +53669 +53670 +53671 +53672 +53673 +53674 +53675 +53676 +53677 +53678 +53679 +53680 +53681 +53682 +53683 +53684 +53685 +53686 +53687 +53688 +53689 +53690 +53691 +53692 +53693 +53694 +53695 +53696 +53697 +53698 +53699 +53700 +53701 +53702 +53703 +53704 +53705 +53706 +53707 +53708 +53709 +53710 +53711 +53712 +53713 +53714 +53715 +53716 +53717 +53718 +53719 +53720 +53721 +53722 +53723 +53724 +53725 +53726 +53727 +53728 +53729 +53730 +53731 +53732 +53733 +53734 +53735 +53736 +53737 +53738 +53739 +53740 +53741 +53742 +53743 +53744 +53745 +53746 +53747 +53748 +53749 +53750 +53751 +53752 +53753 +53754 +53755 +53756 +53757 +53758 +53759 +53760 +53761 +53762 +53763 +53764 +53765 +53766 +53767 +53768 +53769 +53770 +53771 +53772 +53773 +53774 +53775 +53776 +53777 +53778 +53779 +53780 +53781 +53782 +53783 +53784 +53785 +53786 +53787 +53788 +53789 +53790 +53791 +53792 +53793 +53794 +53795 +53796 +53797 +53798 +53799 +53800 +53801 +53802 +53803 +53804 +53805 +53806 +53807 +53808 +53809 +53810 +53811 +53812 +53813 +53814 +53815 +53816 +53817 +53818 +53819 +53820 +53821 +53822 +53823 +53824 +53825 +53826 +53827 +53828 +53829 +53830 +53831 +53832 +53833 +53834 +53835 +53836 +53837 +53838 +53839 +53840 +53841 +53842 +53843 +53844 +53845 +53846 +53847 +53848 +53849 +53850 +53851 +53852 +53853 +53854 +53855 +53856 +53857 +53858 +53859 +53860 +53861 +53862 +53863 +53864 +53865 +53866 +53867 +53868 +53869 +53870 +53871 +53872 +53873 +53874 +53875 +53876 +53877 +53878 +53879 +53880 +53881 +53882 +53883 +53884 +53885 +53886 +53887 +53888 +53889 +53890 +53891 +53892 +53893 +53894 +53895 +53896 +53897 +53898 +53899 +53900 +53901 +53902 +53903 +53904 +53905 +53906 +53907 +53908 +53909 +53910 +53911 +53912 +53913 +53914 +53915 +53916 +53917 +53918 +53919 +53920 +53921 +53922 +53923 +53924 +53925 +53926 +53927 +53928 +53929 +53930 +53931 +53932 +53933 +53934 +53935 +53936 +53937 +53938 +53939 +53940 +53941 +53942 +53943 +53944 +53945 +53946 +53947 +53948 +53949 +53950 +53951 +53952 +53953 +53954 +53955 +53956 +53957 +53958 +53959 +53960 +53961 +53962 +53963 +53964 +53965 +53966 +53967 +53968 +53969 +53970 +53971 +53972 +53973 +53974 +53975 +53976 +53977 +53978 +53979 +53980 +53981 +53982 +53983 +53984 +53985 +53986 +53987 +53988 +53989 +53990 +53991 +53992 +53993 +53994 +53995 +53996 +53997 +53998 +53999 +54000 +54001 +54002 +54003 +54004 +54005 +54006 +54007 +54008 +54009 +54010 +54011 +54012 +54013 +54014 +54015 +54016 +54017 +54018 +54019 +54020 +54021 +54022 +54023 +54024 +54025 +54026 +54027 +54028 +54029 +54030 +54031 +54032 +54033 +54034 +54035 +54036 +54037 +54038 +54039 +54040 +54041 +54042 +54043 +54044 +54045 +54046 +54047 +54048 +54049 +54050 +54051 +54052 +54053 +54054 +54055 +54056 +54057 +54058 +54059 +54060 +54061 +54062 +54063 +54064 +54065 +54066 +54067 +54068 +54069 +54070 +54071 +54072 +54073 +54074 +54075 +54076 +54077 +54078 +54079 +54080 +54081 +54082 +54083 +54084 +54085 +54086 +54087 +54088 +54089 +54090 +54091 +54092 +54093 +54094 +54095 +54096 +54097 +54098 +54099 +54100 +54101 +54102 +54103 +54104 +54105 +54106 +54107 +54108 +54109 +54110 +54111 +54112 +54113 +54114 +54115 +54116 +54117 +54118 +54119 +54120 +54121 +54122 +54123 +54124 +54125 +54126 +54127 +54128 +54129 +54130 +54131 +54132 +54133 +54134 +54135 +54136 +54137 +54138 +54139 +54140 +54141 +54142 +54143 +54144 +54145 +54146 +54147 +54148 +54149 +54150 +54151 +54152 +54153 +54154 +54155 +54156 +54157 +54158 +54159 +54160 +54161 +54162 +54163 +54164 +54165 +54166 +54167 +54168 +54169 +54170 +54171 +54172 +54173 +54174 +54175 +54176 +54177 +54178 +54179 +54180 +54181 +54182 +54183 +54184 +54185 +54186 +54187 +54188 +54189 +54190 +54191 +54192 +54193 +54194 +54195 +54196 +54197 +54198 +54199 +54200 +54201 +54202 +54203 +54204 +54205 +54206 +54207 +54208 +54209 +54210 +54211 +54212 +54213 +54214 +54215 +54216 +54217 +54218 +54219 +54220 +54221 +54222 +54223 +54224 +54225 +54226 +54227 +54228 +54229 +54230 +54231 +54232 +54233 +54234 +54235 +54236 +54237 +54238 +54239 +54240 +54241 +54242 +54243 +54244 +54245 +54246 +54247 +54248 +54249 +54250 +54251 +54252 +54253 +54254 +54255 +54256 +54257 +54258 +54259 +54260 +54261 +54262 +54263 +54264 +54265 +54266 +54267 +54268 +54269 +54270 +54271 +54272 +54273 +54274 +54275 +54276 +54277 +54278 +54279 +54280 +54281 +54282 +54283 +54284 +54285 +54286 +54287 +54288 +54289 +54290 +54291 +54292 +54293 +54294 +54295 +54296 +54297 +54298 +54299 +54300 +54301 +54302 +54303 +54304 +54305 +54306 +54307 +54308 +54309 +54310 +54311 +54312 +54313 +54314 +54315 +54316 +54317 +54318 +54319 +54320 +54321 +54322 +54323 +54324 +54325 +54326 +54327 +54328 +54329 +54330 +54331 +54332 +54333 +54334 +54335 +54336 +54337 +54338 +54339 +54340 +54341 +54342 +54343 +54344 +54345 +54346 +54347 +54348 +54349 +54350 +54351 +54352 +54353 +54354 +54355 +54356 +54357 +54358 +54359 +54360 +54361 +54362 +54363 +54364 +54365 +54366 +54367 +54368 +54369 +54370 +54371 +54372 +54373 +54374 +54375 +54376 +54377 +54378 +54379 +54380 +54381 +54382 +54383 +54384 +54385 +54386 +54387 +54388 +54389 +54390 +54391 +54392 +54393 +54394 +54395 +54396 +54397 +54398 +54399 +54400 +54401 +54402 +54403 +54404 +54405 +54406 +54407 +54408 +54409 +54410 +54411 +54412 +54413 +54414 +54415 +54416 +54417 +54418 +54419 +54420 +54421 +54422 +54423 +54424 +54425 +54426 +54427 +54428 +54429 +54430 +54431 +54432 +54433 +54434 +54435 +54436 +54437 +54438 +54439 +54440 +54441 +54442 +54443 +54444 +54445 +54446 +54447 +54448 +54449 +54450 +54451 +54452 +54453 +54454 +54455 +54456 +54457 +54458 +54459 +54460 +54461 +54462 +54463 +54464 +54465 +54466 +54467 +54468 +54469 +54470 +54471 +54472 +54473 +54474 +54475 +54476 +54477 +54478 +54479 +54480 +54481 +54482 +54483 +54484 +54485 +54486 +54487 +54488 +54489 +54490 +54491 +54492 +54493 +54494 +54495 +54496 +54497 +54498 +54499 +54500 +54501 +54502 +54503 +54504 +54505 +54506 +54507 +54508 +54509 +54510 +54511 +54512 +54513 +54514 +54515 +54516 +54517 +54518 +54519 +54520 +54521 +54522 +54523 +54524 +54525 +54526 +54527 +54528 +54529 +54530 +54531 +54532 +54533 +54534 +54535 +54536 +54537 +54538 +54539 +54540 +54541 +54542 +54543 +54544 +54545 +54546 +54547 +54548 +54549 +54550 +54551 +54552 +54553 +54554 +54555 +54556 +54557 +54558 +54559 +54560 +54561 +54562 +54563 +54564 +54565 +54566 +54567 +54568 +54569 +54570 +54571 +54572 +54573 +54574 +54575 +54576 +54577 +54578 +54579 +54580 +54581 +54582 +54583 +54584 +54585 +54586 +54587 +54588 +54589 +54590 +54591 +54592 +54593 +54594 +54595 +54596 +54597 +54598 +54599 +54600 +54601 +54602 +54603 +54604 +54605 +54606 +54607 +54608 +54609 +54610 +54611 +54612 +54613 +54614 +54615 +54616 +54617 +54618 +54619 +54620 +54621 +54622 +54623 +54624 +54625 +54626 +54627 +54628 +54629 +54630 +54631 +54632 +54633 +54634 +54635 +54636 +54637 +54638 +54639 +54640 +54641 +54642 +54643 +54644 +54645 +54646 +54647 +54648 +54649 +54650 +54651 +54652 +54653 +54654 +54655 +54656 +54657 +54658 +54659 +54660 +54661 +54662 +54663 +54664 +54665 +54666 +54667 +54668 +54669 +54670 +54671 +54672 +54673 +54674 +54675 +54676 +54677 +54678 +54679 +54680 +54681 +54682 +54683 +54684 +54685 +54686 +54687 +54688 +54689 +54690 +54691 +54692 +54693 +54694 +54695 +54696 +54697 +54698 +54699 +54700 +54701 +54702 +54703 +54704 +54705 +54706 +54707 +54708 +54709 +54710 +54711 +54712 +54713 +54714 +54715 +54716 +54717 +54718 +54719 +54720 +54721 +54722 +54723 +54724 +54725 +54726 +54727 +54728 +54729 +54730 +54731 +54732 +54733 +54734 +54735 +54736 +54737 +54738 +54739 +54740 +54741 +54742 +54743 +54744 +54745 +54746 +54747 +54748 +54749 +54750 +54751 +54752 +54753 +54754 +54755 +54756 +54757 +54758 +54759 +54760 +54761 +54762 +54763 +54764 +54765 +54766 +54767 +54768 +54769 +54770 +54771 +54772 +54773 +54774 +54775 +54776 +54777 +54778 +54779 +54780 +54781 +54782 +54783 +54784 +54785 +54786 +54787 +54788 +54789 +54790 +54791 +54792 +54793 +54794 +54795 +54796 +54797 +54798 +54799 +54800 +54801 +54802 +54803 +54804 +54805 +54806 +54807 +54808 +54809 +54810 +54811 +54812 +54813 +54814 +54815 +54816 +54817 +54818 +54819 +54820 +54821 +54822 +54823 +54824 +54825 +54826 +54827 +54828 +54829 +54830 +54831 +54832 +54833 +54834 +54835 +54836 +54837 +54838 +54839 +54840 +54841 +54842 +54843 +54844 +54845 +54846 +54847 +54848 +54849 +54850 +54851 +54852 +54853 +54854 +54855 +54856 +54857 +54858 +54859 +54860 +54861 +54862 +54863 +54864 +54865 +54866 +54867 +54868 +54869 +54870 +54871 +54872 +54873 +54874 +54875 +54876 +54877 +54878 +54879 +54880 +54881 +54882 +54883 +54884 +54885 +54886 +54887 +54888 +54889 +54890 +54891 +54892 +54893 +54894 +54895 +54896 +54897 +54898 +54899 +54900 +54901 +54902 +54903 +54904 +54905 +54906 +54907 +54908 +54909 +54910 +54911 +54912 +54913 +54914 +54915 +54916 +54917 +54918 +54919 +54920 +54921 +54922 +54923 +54924 +54925 +54926 +54927 +54928 +54929 +54930 +54931 +54932 +54933 +54934 +54935 +54936 +54937 +54938 +54939 +54940 +54941 +54942 +54943 +54944 +54945 +54946 +54947 +54948 +54949 +54950 +54951 +54952 +54953 +54954 +54955 +54956 +54957 +54958 +54959 +54960 +54961 +54962 +54963 +54964 +54965 +54966 +54967 +54968 +54969 +54970 +54971 +54972 +54973 +54974 +54975 +54976 +54977 +54978 +54979 +54980 +54981 +54982 +54983 +54984 +54985 +54986 +54987 +54988 +54989 +54990 +54991 +54992 +54993 +54994 +54995 +54996 +54997 +54998 +54999 +55000 +55001 +55002 +55003 +55004 +55005 +55006 +55007 +55008 +55009 +55010 +55011 +55012 +55013 +55014 +55015 +55016 +55017 +55018 +55019 +55020 +55021 +55022 +55023 +55024 +55025 +55026 +55027 +55028 +55029 +55030 +55031 +55032 +55033 +55034 +55035 +55036 +55037 +55038 +55039 +55040 +55041 +55042 +55043 +55044 +55045 +55046 +55047 +55048 +55049 +55050 +55051 +55052 +55053 +55054 +55055 +55056 +55057 +55058 +55059 +55060 +55061 +55062 +55063 +55064 +55065 +55066 +55067 +55068 +55069 +55070 +55071 +55072 +55073 +55074 +55075 +55076 +55077 +55078 +55079 +55080 +55081 +55082 +55083 +55084 +55085 +55086 +55087 +55088 +55089 +55090 +55091 +55092 +55093 +55094 +55095 +55096 +55097 +55098 +55099 +55100 +55101 +55102 +55103 +55104 +55105 +55106 +55107 +55108 +55109 +55110 +55111 +55112 +55113 +55114 +55115 +55116 +55117 +55118 +55119 +55120 +55121 +55122 +55123 +55124 +55125 +55126 +55127 +55128 +55129 +55130 +55131 +55132 +55133 +55134 +55135 +55136 +55137 +55138 +55139 +55140 +55141 +55142 +55143 +55144 +55145 +55146 +55147 +55148 +55149 +55150 +55151 +55152 +55153 +55154 +55155 +55156 +55157 +55158 +55159 +55160 +55161 +55162 +55163 +55164 +55165 +55166 +55167 +55168 +55169 +55170 +55171 +55172 +55173 +55174 +55175 +55176 +55177 +55178 +55179 +55180 +55181 +55182 +55183 +55184 +55185 +55186 +55187 +55188 +55189 +55190 +55191 +55192 +55193 +55194 +55195 +55196 +55197 +55198 +55199 +55200 +55201 +55202 +55203 +55204 +55205 +55206 +55207 +55208 +55209 +55210 +55211 +55212 +55213 +55214 +55215 +55216 +55217 +55218 +55219 +55220 +55221 +55222 +55223 +55224 +55225 +55226 +55227 +55228 +55229 +55230 +55231 +55232 +55233 +55234 +55235 +55236 +55237 +55238 +55239 +55240 +55241 +55242 +55243 +55244 +55245 +55246 +55247 +55248 +55249 +55250 +55251 +55252 +55253 +55254 +55255 +55256 +55257 +55258 +55259 +55260 +55261 +55262 +55263 +55264 +55265 +55266 +55267 +55268 +55269 +55270 +55271 +55272 +55273 +55274 +55275 +55276 +55277 +55278 +55279 +55280 +55281 +55282 +55283 +55284 +55285 +55286 +55287 +55288 +55289 +55290 +55291 +55292 +55293 +55294 +55295 +55296 +55297 +55298 +55299 +55300 +55301 +55302 +55303 +55304 +55305 +55306 +55307 +55308 +55309 +55310 +55311 +55312 +55313 +55314 +55315 +55316 +55317 +55318 +55319 +55320 +55321 +55322 +55323 +55324 +55325 +55326 +55327 +55328 +55329 +55330 +55331 +55332 +55333 +55334 +55335 +55336 +55337 +55338 +55339 +55340 +55341 +55342 +55343 +55344 +55345 +55346 +55347 +55348 +55349 +55350 +55351 +55352 +55353 +55354 +55355 +55356 +55357 +55358 +55359 +55360 +55361 +55362 +55363 +55364 +55365 +55366 +55367 +55368 +55369 +55370 +55371 +55372 +55373 +55374 +55375 +55376 +55377 +55378 +55379 +55380 +55381 +55382 +55383 +55384 +55385 +55386 +55387 +55388 +55389 +55390 +55391 +55392 +55393 +55394 +55395 +55396 +55397 +55398 +55399 +55400 +55401 +55402 +55403 +55404 +55405 +55406 +55407 +55408 +55409 +55410 +55411 +55412 +55413 +55414 +55415 +55416 +55417 +55418 +55419 +55420 +55421 +55422 +55423 +55424 +55425 +55426 +55427 +55428 +55429 +55430 +55431 +55432 +55433 +55434 +55435 +55436 +55437 +55438 +55439 +55440 +55441 +55442 +55443 +55444 +55445 +55446 +55447 +55448 +55449 +55450 +55451 +55452 +55453 +55454 +55455 +55456 +55457 +55458 +55459 +55460 +55461 +55462 +55463 +55464 +55465 +55466 +55467 +55468 +55469 +55470 +55471 +55472 +55473 +55474 +55475 +55476 +55477 +55478 +55479 +55480 +55481 +55482 +55483 +55484 +55485 +55486 +55487 +55488 +55489 +55490 +55491 +55492 +55493 +55494 +55495 +55496 +55497 +55498 +55499 +55500 +55501 +55502 +55503 +55504 +55505 +55506 +55507 +55508 +55509 +55510 +55511 +55512 +55513 +55514 +55515 +55516 +55517 +55518 +55519 +55520 +55521 +55522 +55523 +55524 +55525 +55526 +55527 +55528 +55529 +55530 +55531 +55532 +55533 +55534 +55535 +55536 +55537 +55538 +55539 +55540 +55541 +55542 +55543 +55544 +55545 +55546 +55547 +55548 +55549 +55550 +55551 +55552 +55553 +55554 +55555 +55556 +55557 +55558 +55559 +55560 +55561 +55562 +55563 +55564 +55565 +55566 +55567 +55568 +55569 +55570 +55571 +55572 +55573 +55574 +55575 +55576 +55577 +55578 +55579 +55580 +55581 +55582 +55583 +55584 +55585 +55586 +55587 +55588 +55589 +55590 +55591 +55592 +55593 +55594 +55595 +55596 +55597 +55598 +55599 +55600 +55601 +55602 +55603 +55604 +55605 +55606 +55607 +55608 +55609 +55610 +55611 +55612 +55613 +55614 +55615 +55616 +55617 +55618 +55619 +55620 +55621 +55622 +55623 +55624 +55625 +55626 +55627 +55628 +55629 +55630 +55631 +55632 +55633 +55634 +55635 +55636 +55637 +55638 +55639 +55640 +55641 +55642 +55643 +55644 +55645 +55646 +55647 +55648 +55649 +55650 +55651 +55652 +55653 +55654 +55655 +55656 +55657 +55658 +55659 +55660 +55661 +55662 +55663 +55664 +55665 +55666 +55667 +55668 +55669 +55670 +55671 +55672 +55673 +55674 +55675 +55676 +55677 +55678 +55679 +55680 +55681 +55682 +55683 +55684 +55685 +55686 +55687 +55688 +55689 +55690 +55691 +55692 +55693 +55694 +55695 +55696 +55697 +55698 +55699 +55700 +55701 +55702 +55703 +55704 +55705 +55706 +55707 +55708 +55709 +55710 +55711 +55712 +55713 +55714 +55715 +55716 +55717 +55718 +55719 +55720 +55721 +55722 +55723 +55724 +55725 +55726 +55727 +55728 +55729 +55730 +55731 +55732 +55733 +55734 +55735 +55736 +55737 +55738 +55739 +55740 +55741 +55742 +55743 +55744 +55745 +55746 +55747 +55748 +55749 +55750 +55751 +55752 +55753 +55754 +55755 +55756 +55757 +55758 +55759 +55760 +55761 +55762 +55763 +55764 +55765 +55766 +55767 +55768 +55769 +55770 +55771 +55772 +55773 +55774 +55775 +55776 +55777 +55778 +55779 +55780 +55781 +55782 +55783 +55784 +55785 +55786 +55787 +55788 +55789 +55790 +55791 +55792 +55793 +55794 +55795 +55796 +55797 +55798 +55799 +55800 +55801 +55802 +55803 +55804 +55805 +55806 +55807 +55808 +55809 +55810 +55811 +55812 +55813 +55814 +55815 +55816 +55817 +55818 +55819 +55820 +55821 +55822 +55823 +55824 +55825 +55826 +55827 +55828 +55829 +55830 +55831 +55832 +55833 +55834 +55835 +55836 +55837 +55838 +55839 +55840 +55841 +55842 +55843 +55844 +55845 +55846 +55847 +55848 +55849 +55850 +55851 +55852 +55853 +55854 +55855 +55856 +55857 +55858 +55859 +55860 +55861 +55862 +55863 +55864 +55865 +55866 +55867 +55868 +55869 +55870 +55871 +55872 +55873 +55874 +55875 +55876 +55877 +55878 +55879 +55880 +55881 +55882 +55883 +55884 +55885 +55886 +55887 +55888 +55889 +55890 +55891 +55892 +55893 +55894 +55895 +55896 +55897 +55898 +55899 +55900 +55901 +55902 +55903 +55904 +55905 +55906 +55907 +55908 +55909 +55910 +55911 +55912 +55913 +55914 +55915 +55916 +55917 +55918 +55919 +55920 +55921 +55922 +55923 +55924 +55925 +55926 +55927 +55928 +55929 +55930 +55931 +55932 +55933 +55934 +55935 +55936 +55937 +55938 +55939 +55940 +55941 +55942 +55943 +55944 +55945 +55946 +55947 +55948 +55949 +55950 +55951 +55952 +55953 +55954 +55955 +55956 +55957 +55958 +55959 +55960 +55961 +55962 +55963 +55964 +55965 +55966 +55967 +55968 +55969 +55970 +55971 +55972 +55973 +55974 +55975 +55976 +55977 +55978 +55979 +55980 +55981 +55982 +55983 +55984 +55985 +55986 +55987 +55988 +55989 +55990 +55991 +55992 +55993 +55994 +55995 +55996 +55997 +55998 +55999 +56000 +56001 +56002 +56003 +56004 +56005 +56006 +56007 +56008 +56009 +56010 +56011 +56012 +56013 +56014 +56015 +56016 +56017 +56018 +56019 +56020 +56021 +56022 +56023 +56024 +56025 +56026 +56027 +56028 +56029 +56030 +56031 +56032 +56033 +56034 +56035 +56036 +56037 +56038 +56039 +56040 +56041 +56042 +56043 +56044 +56045 +56046 +56047 +56048 +56049 +56050 +56051 +56052 +56053 +56054 +56055 +56056 +56057 +56058 +56059 +56060 +56061 +56062 +56063 +56064 +56065 +56066 +56067 +56068 +56069 +56070 +56071 +56072 +56073 +56074 +56075 +56076 +56077 +56078 +56079 +56080 +56081 +56082 +56083 +56084 +56085 +56086 +56087 +56088 +56089 +56090 +56091 +56092 +56093 +56094 +56095 +56096 +56097 +56098 +56099 +56100 +56101 +56102 +56103 +56104 +56105 +56106 +56107 +56108 +56109 +56110 +56111 +56112 +56113 +56114 +56115 +56116 +56117 +56118 +56119 +56120 +56121 +56122 +56123 +56124 +56125 +56126 +56127 +56128 +56129 +56130 +56131 +56132 +56133 +56134 +56135 +56136 +56137 +56138 +56139 +56140 +56141 +56142 +56143 +56144 +56145 +56146 +56147 +56148 +56149 +56150 +56151 +56152 +56153 +56154 +56155 +56156 +56157 +56158 +56159 +56160 +56161 +56162 +56163 +56164 +56165 +56166 +56167 +56168 +56169 +56170 +56171 +56172 +56173 +56174 +56175 +56176 +56177 +56178 +56179 +56180 +56181 +56182 +56183 +56184 +56185 +56186 +56187 +56188 +56189 +56190 +56191 +56192 +56193 +56194 +56195 +56196 +56197 +56198 +56199 +56200 +56201 +56202 +56203 +56204 +56205 +56206 +56207 +56208 +56209 +56210 +56211 +56212 +56213 +56214 +56215 +56216 +56217 +56218 +56219 +56220 +56221 +56222 +56223 +56224 +56225 +56226 +56227 +56228 +56229 +56230 +56231 +56232 +56233 +56234 +56235 +56236 +56237 +56238 +56239 +56240 +56241 +56242 +56243 +56244 +56245 +56246 +56247 +56248 +56249 +56250 +56251 +56252 +56253 +56254 +56255 +56256 +56257 +56258 +56259 +56260 +56261 +56262 +56263 +56264 +56265 +56266 +56267 +56268 +56269 +56270 +56271 +56272 +56273 +56274 +56275 +56276 +56277 +56278 +56279 +56280 +56281 +56282 +56283 +56284 +56285 +56286 +56287 +56288 +56289 +56290 +56291 +56292 +56293 +56294 +56295 +56296 +56297 +56298 +56299 +56300 +56301 +56302 +56303 +56304 +56305 +56306 +56307 +56308 +56309 +56310 +56311 +56312 +56313 +56314 +56315 +56316 +56317 +56318 +56319 +56320 +56321 +56322 +56323 +56324 +56325 +56326 +56327 +56328 +56329 +56330 +56331 +56332 +56333 +56334 +56335 +56336 +56337 +56338 +56339 +56340 +56341 +56342 +56343 +56344 +56345 +56346 +56347 +56348 +56349 +56350 +56351 +56352 +56353 +56354 +56355 +56356 +56357 +56358 +56359 +56360 +56361 +56362 +56363 +56364 +56365 +56366 +56367 +56368 +56369 +56370 +56371 +56372 +56373 +56374 +56375 +56376 +56377 +56378 +56379 +56380 +56381 +56382 +56383 +56384 +56385 +56386 +56387 +56388 +56389 +56390 +56391 +56392 +56393 +56394 +56395 +56396 +56397 +56398 +56399 +56400 +56401 +56402 +56403 +56404 +56405 +56406 +56407 +56408 +56409 +56410 +56411 +56412 +56413 +56414 +56415 +56416 +56417 +56418 +56419 +56420 +56421 +56422 +56423 +56424 +56425 +56426 +56427 +56428 +56429 +56430 +56431 +56432 +56433 +56434 +56435 +56436 +56437 +56438 +56439 +56440 +56441 +56442 +56443 +56444 +56445 +56446 +56447 +56448 +56449 +56450 +56451 +56452 +56453 +56454 +56455 +56456 +56457 +56458 +56459 +56460 +56461 +56462 +56463 +56464 +56465 +56466 +56467 +56468 +56469 +56470 +56471 +56472 +56473 +56474 +56475 +56476 +56477 +56478 +56479 +56480 +56481 +56482 +56483 +56484 +56485 +56486 +56487 +56488 +56489 +56490 +56491 +56492 +56493 +56494 +56495 +56496 +56497 +56498 +56499 +56500 +56501 +56502 +56503 +56504 +56505 +56506 +56507 +56508 +56509 +56510 +56511 +56512 +56513 +56514 +56515 +56516 +56517 +56518 +56519 +56520 +56521 +56522 +56523 +56524 +56525 +56526 +56527 +56528 +56529 +56530 +56531 +56532 +56533 +56534 +56535 +56536 +56537 +56538 +56539 +56540 +56541 +56542 +56543 +56544 +56545 +56546 +56547 +56548 +56549 +56550 +56551 +56552 +56553 +56554 +56555 +56556 +56557 +56558 +56559 +56560 +56561 +56562 +56563 +56564 +56565 +56566 +56567 +56568 +56569 +56570 +56571 +56572 +56573 +56574 +56575 +56576 +56577 +56578 +56579 +56580 +56581 +56582 +56583 +56584 +56585 +56586 +56587 +56588 +56589 +56590 +56591 +56592 +56593 +56594 +56595 +56596 +56597 +56598 +56599 +56600 +56601 +56602 +56603 +56604 +56605 +56606 +56607 +56608 +56609 +56610 +56611 +56612 +56613 +56614 +56615 +56616 +56617 +56618 +56619 +56620 +56621 +56622 +56623 +56624 +56625 +56626 +56627 +56628 +56629 +56630 +56631 +56632 +56633 +56634 +56635 +56636 +56637 +56638 +56639 +56640 +56641 +56642 +56643 +56644 +56645 +56646 +56647 +56648 +56649 +56650 +56651 +56652 +56653 +56654 +56655 +56656 +56657 +56658 +56659 +56660 +56661 +56662 +56663 +56664 +56665 +56666 +56667 +56668 +56669 +56670 +56671 +56672 +56673 +56674 +56675 +56676 +56677 +56678 +56679 +56680 +56681 +56682 +56683 +56684 +56685 +56686 +56687 +56688 +56689 +56690 +56691 +56692 +56693 +56694 +56695 +56696 +56697 +56698 +56699 +56700 +56701 +56702 +56703 +56704 +56705 +56706 +56707 +56708 +56709 +56710 +56711 +56712 +56713 +56714 +56715 +56716 +56717 +56718 +56719 +56720 +56721 +56722 +56723 +56724 +56725 +56726 +56727 +56728 +56729 +56730 +56731 +56732 +56733 +56734 +56735 +56736 +56737 +56738 +56739 +56740 +56741 +56742 +56743 +56744 +56745 +56746 +56747 +56748 +56749 +56750 +56751 +56752 +56753 +56754 +56755 +56756 +56757 +56758 +56759 +56760 +56761 +56762 +56763 +56764 +56765 +56766 +56767 +56768 +56769 +56770 +56771 +56772 +56773 +56774 +56775 +56776 +56777 +56778 +56779 +56780 +56781 +56782 +56783 +56784 +56785 +56786 +56787 +56788 +56789 +56790 +56791 +56792 +56793 +56794 +56795 +56796 +56797 +56798 +56799 +56800 +56801 +56802 +56803 +56804 +56805 +56806 +56807 +56808 +56809 +56810 +56811 +56812 +56813 +56814 +56815 +56816 +56817 +56818 +56819 +56820 +56821 +56822 +56823 +56824 +56825 +56826 +56827 +56828 +56829 +56830 +56831 +56832 +56833 +56834 +56835 +56836 +56837 +56838 +56839 +56840 +56841 +56842 +56843 +56844 +56845 +56846 +56847 +56848 +56849 +56850 +56851 +56852 +56853 +56854 +56855 +56856 +56857 +56858 +56859 +56860 +56861 +56862 +56863 +56864 +56865 +56866 +56867 +56868 +56869 +56870 +56871 +56872 +56873 +56874 +56875 +56876 +56877 +56878 +56879 +56880 +56881 +56882 +56883 +56884 +56885 +56886 +56887 +56888 +56889 +56890 +56891 +56892 +56893 +56894 +56895 +56896 +56897 +56898 +56899 +56900 +56901 +56902 +56903 +56904 +56905 +56906 +56907 +56908 +56909 +56910 +56911 +56912 +56913 +56914 +56915 +56916 +56917 +56918 +56919 +56920 +56921 +56922 +56923 +56924 +56925 +56926 +56927 +56928 +56929 +56930 +56931 +56932 +56933 +56934 +56935 +56936 +56937 +56938 +56939 +56940 +56941 +56942 +56943 +56944 +56945 +56946 +56947 +56948 +56949 +56950 +56951 +56952 +56953 +56954 +56955 +56956 +56957 +56958 +56959 +56960 +56961 +56962 +56963 +56964 +56965 +56966 +56967 +56968 +56969 +56970 +56971 +56972 +56973 +56974 +56975 +56976 +56977 +56978 +56979 +56980 +56981 +56982 +56983 +56984 +56985 +56986 +56987 +56988 +56989 +56990 +56991 +56992 +56993 +56994 +56995 +56996 +56997 +56998 +56999 +57000 +57001 +57002 +57003 +57004 +57005 +57006 +57007 +57008 +57009 +57010 +57011 +57012 +57013 +57014 +57015 +57016 +57017 +57018 +57019 +57020 +57021 +57022 +57023 +57024 +57025 +57026 +57027 +57028 +57029 +57030 +57031 +57032 +57033 +57034 +57035 +57036 +57037 +57038 +57039 +57040 +57041 +57042 +57043 +57044 +57045 +57046 +57047 +57048 +57049 +57050 +57051 +57052 +57053 +57054 +57055 +57056 +57057 +57058 +57059 +57060 +57061 +57062 +57063 +57064 +57065 +57066 +57067 +57068 +57069 +57070 +57071 +57072 +57073 +57074 +57075 +57076 +57077 +57078 +57079 +57080 +57081 +57082 +57083 +57084 +57085 +57086 +57087 +57088 +57089 +57090 +57091 +57092 +57093 +57094 +57095 +57096 +57097 +57098 +57099 +57100 +57101 +57102 +57103 +57104 +57105 +57106 +57107 +57108 +57109 +57110 +57111 +57112 +57113 +57114 +57115 +57116 +57117 +57118 +57119 +57120 +57121 +57122 +57123 +57124 +57125 +57126 +57127 +57128 +57129 +57130 +57131 +57132 +57133 +57134 +57135 +57136 +57137 +57138 +57139 +57140 +57141 +57142 +57143 +57144 +57145 +57146 +57147 +57148 +57149 +57150 +57151 +57152 +57153 +57154 +57155 +57156 +57157 +57158 +57159 +57160 +57161 +57162 +57163 +57164 +57165 +57166 +57167 +57168 +57169 +57170 +57171 +57172 +57173 +57174 +57175 +57176 +57177 +57178 +57179 +57180 +57181 +57182 +57183 +57184 +57185 +57186 +57187 +57188 +57189 +57190 +57191 +57192 +57193 +57194 +57195 +57196 +57197 +57198 +57199 +57200 +57201 +57202 +57203 +57204 +57205 +57206 +57207 +57208 +57209 +57210 +57211 +57212 +57213 +57214 +57215 +57216 +57217 +57218 +57219 +57220 +57221 +57222 +57223 +57224 +57225 +57226 +57227 +57228 +57229 +57230 +57231 +57232 +57233 +57234 +57235 +57236 +57237 +57238 +57239 +57240 +57241 +57242 +57243 +57244 +57245 +57246 +57247 +57248 +57249 +57250 +57251 +57252 +57253 +57254 +57255 +57256 +57257 +57258 +57259 +57260 +57261 +57262 +57263 +57264 +57265 +57266 +57267 +57268 +57269 +57270 +57271 +57272 +57273 +57274 +57275 +57276 +57277 +57278 +57279 +57280 +57281 +57282 +57283 +57284 +57285 +57286 +57287 +57288 +57289 +57290 +57291 +57292 +57293 +57294 +57295 +57296 +57297 +57298 +57299 +57300 +57301 +57302 +57303 +57304 +57305 +57306 +57307 +57308 +57309 +57310 +57311 +57312 +57313 +57314 +57315 +57316 +57317 +57318 +57319 +57320 +57321 +57322 +57323 +57324 +57325 +57326 +57327 +57328 +57329 +57330 +57331 +57332 +57333 +57334 +57335 +57336 +57337 +57338 +57339 +57340 +57341 +57342 +57343 +57344 +57345 +57346 +57347 +57348 +57349 +57350 +57351 +57352 +57353 +57354 +57355 +57356 +57357 +57358 +57359 +57360 +57361 +57362 +57363 +57364 +57365 +57366 +57367 +57368 +57369 +57370 +57371 +57372 +57373 +57374 +57375 +57376 +57377 +57378 +57379 +57380 +57381 +57382 +57383 +57384 +57385 +57386 +57387 +57388 +57389 +57390 +57391 +57392 +57393 +57394 +57395 +57396 +57397 +57398 +57399 +57400 +57401 +57402 +57403 +57404 +57405 +57406 +57407 +57408 +57409 +57410 +57411 +57412 +57413 +57414 +57415 +57416 +57417 +57418 +57419 +57420 +57421 +57422 +57423 +57424 +57425 +57426 +57427 +57428 +57429 +57430 +57431 +57432 +57433 +57434 +57435 +57436 +57437 +57438 +57439 +57440 +57441 +57442 +57443 +57444 +57445 +57446 +57447 +57448 +57449 +57450 +57451 +57452 +57453 +57454 +57455 +57456 +57457 +57458 +57459 +57460 +57461 +57462 +57463 +57464 +57465 +57466 +57467 +57468 +57469 +57470 +57471 +57472 +57473 +57474 +57475 +57476 +57477 +57478 +57479 +57480 +57481 +57482 +57483 +57484 +57485 +57486 +57487 +57488 +57489 +57490 +57491 +57492 +57493 +57494 +57495 +57496 +57497 +57498 +57499 +57500 +57501 +57502 +57503 +57504 +57505 +57506 +57507 +57508 +57509 +57510 +57511 +57512 +57513 +57514 +57515 +57516 +57517 +57518 +57519 +57520 +57521 +57522 +57523 +57524 +57525 +57526 +57527 +57528 +57529 +57530 +57531 +57532 +57533 +57534 +57535 +57536 +57537 +57538 +57539 +57540 +57541 +57542 +57543 +57544 +57545 +57546 +57547 +57548 +57549 +57550 +57551 +57552 +57553 +57554 +57555 +57556 +57557 +57558 +57559 +57560 +57561 +57562 +57563 +57564 +57565 +57566 +57567 +57568 +57569 +57570 +57571 +57572 +57573 +57574 +57575 +57576 +57577 +57578 +57579 +57580 +57581 +57582 +57583 +57584 +57585 +57586 +57587 +57588 +57589 +57590 +57591 +57592 +57593 +57594 +57595 +57596 +57597 +57598 +57599 +57600 +57601 +57602 +57603 +57604 +57605 +57606 +57607 +57608 +57609 +57610 +57611 +57612 +57613 +57614 +57615 +57616 +57617 +57618 +57619 +57620 +57621 +57622 +57623 +57624 +57625 +57626 +57627 +57628 +57629 +57630 +57631 +57632 +57633 +57634 +57635 +57636 +57637 +57638 +57639 +57640 +57641 +57642 +57643 +57644 +57645 +57646 +57647 +57648 +57649 +57650 +57651 +57652 +57653 +57654 +57655 +57656 +57657 +57658 +57659 +57660 +57661 +57662 +57663 +57664 +57665 +57666 +57667 +57668 +57669 +57670 +57671 +57672 +57673 +57674 +57675 +57676 +57677 +57678 +57679 +57680 +57681 +57682 +57683 +57684 +57685 +57686 +57687 +57688 +57689 +57690 +57691 +57692 +57693 +57694 +57695 +57696 +57697 +57698 +57699 +57700 +57701 +57702 +57703 +57704 +57705 +57706 +57707 +57708 +57709 +57710 +57711 +57712 +57713 +57714 +57715 +57716 +57717 +57718 +57719 +57720 +57721 +57722 +57723 +57724 +57725 +57726 +57727 +57728 +57729 +57730 +57731 +57732 +57733 +57734 +57735 +57736 +57737 +57738 +57739 +57740 +57741 +57742 +57743 +57744 +57745 +57746 +57747 +57748 +57749 +57750 +57751 +57752 +57753 +57754 +57755 +57756 +57757 +57758 +57759 +57760 +57761 +57762 +57763 +57764 +57765 +57766 +57767 +57768 +57769 +57770 +57771 +57772 +57773 +57774 +57775 +57776 +57777 +57778 +57779 +57780 +57781 +57782 +57783 +57784 +57785 +57786 +57787 +57788 +57789 +57790 +57791 +57792 +57793 +57794 +57795 +57796 +57797 +57798 +57799 +57800 +57801 +57802 +57803 +57804 +57805 +57806 +57807 +57808 +57809 +57810 +57811 +57812 +57813 +57814 +57815 +57816 +57817 +57818 +57819 +57820 +57821 +57822 +57823 +57824 +57825 +57826 +57827 +57828 +57829 +57830 +57831 +57832 +57833 +57834 +57835 +57836 +57837 +57838 +57839 +57840 +57841 +57842 +57843 +57844 +57845 +57846 +57847 +57848 +57849 +57850 +57851 +57852 +57853 +57854 +57855 +57856 +57857 +57858 +57859 +57860 +57861 +57862 +57863 +57864 +57865 +57866 +57867 +57868 +57869 +57870 +57871 +57872 +57873 +57874 +57875 +57876 +57877 +57878 +57879 +57880 +57881 +57882 +57883 +57884 +57885 +57886 +57887 +57888 +57889 +57890 +57891 +57892 +57893 +57894 +57895 +57896 +57897 +57898 +57899 +57900 +57901 +57902 +57903 +57904 +57905 +57906 +57907 +57908 +57909 +57910 +57911 +57912 +57913 +57914 +57915 +57916 +57917 +57918 +57919 +57920 +57921 +57922 +57923 +57924 +57925 +57926 +57927 +57928 +57929 +57930 +57931 +57932 +57933 +57934 +57935 +57936 +57937 +57938 +57939 +57940 +57941 +57942 +57943 +57944 +57945 +57946 +57947 +57948 +57949 +57950 +57951 +57952 +57953 +57954 +57955 +57956 +57957 +57958 +57959 +57960 +57961 +57962 +57963 +57964 +57965 +57966 +57967 +57968 +57969 +57970 +57971 +57972 +57973 +57974 +57975 +57976 +57977 +57978 +57979 +57980 +57981 +57982 +57983 +57984 +57985 +57986 +57987 +57988 +57989 +57990 +57991 +57992 +57993 +57994 +57995 +57996 +57997 +57998 +57999 +58000 +58001 +58002 +58003 +58004 +58005 +58006 +58007 +58008 +58009 +58010 +58011 +58012 +58013 +58014 +58015 +58016 +58017 +58018 +58019 +58020 +58021 +58022 +58023 +58024 +58025 +58026 +58027 +58028 +58029 +58030 +58031 +58032 +58033 +58034 +58035 +58036 +58037 +58038 +58039 +58040 +58041 +58042 +58043 +58044 +58045 +58046 +58047 +58048 +58049 +58050 +58051 +58052 +58053 +58054 +58055 +58056 +58057 +58058 +58059 +58060 +58061 +58062 +58063 +58064 +58065 +58066 +58067 +58068 +58069 +58070 +58071 +58072 +58073 +58074 +58075 +58076 +58077 +58078 +58079 +58080 +58081 +58082 +58083 +58084 +58085 +58086 +58087 +58088 +58089 +58090 +58091 +58092 +58093 +58094 +58095 +58096 +58097 +58098 +58099 +58100 +58101 +58102 +58103 +58104 +58105 +58106 +58107 +58108 +58109 +58110 +58111 +58112 +58113 +58114 +58115 +58116 +58117 +58118 +58119 +58120 +58121 +58122 +58123 +58124 +58125 +58126 +58127 +58128 +58129 +58130 +58131 +58132 +58133 +58134 +58135 +58136 +58137 +58138 +58139 +58140 +58141 +58142 +58143 +58144 +58145 +58146 +58147 +58148 +58149 +58150 +58151 +58152 +58153 +58154 +58155 +58156 +58157 +58158 +58159 +58160 +58161 +58162 +58163 +58164 +58165 +58166 +58167 +58168 +58169 +58170 +58171 +58172 +58173 +58174 +58175 +58176 +58177 +58178 +58179 +58180 +58181 +58182 +58183 +58184 +58185 +58186 +58187 +58188 +58189 +58190 +58191 +58192 +58193 +58194 +58195 +58196 +58197 +58198 +58199 +58200 +58201 +58202 +58203 +58204 +58205 +58206 +58207 +58208 +58209 +58210 +58211 +58212 +58213 +58214 +58215 +58216 +58217 +58218 +58219 +58220 +58221 +58222 +58223 +58224 +58225 +58226 +58227 +58228 +58229 +58230 +58231 +58232 +58233 +58234 +58235 +58236 +58237 +58238 +58239 +58240 +58241 +58242 +58243 +58244 +58245 +58246 +58247 +58248 +58249 +58250 +58251 +58252 +58253 +58254 +58255 +58256 +58257 +58258 +58259 +58260 +58261 +58262 +58263 +58264 +58265 +58266 +58267 +58268 +58269 +58270 +58271 +58272 +58273 +58274 +58275 +58276 +58277 +58278 +58279 +58280 +58281 +58282 +58283 +58284 +58285 +58286 +58287 +58288 +58289 +58290 +58291 +58292 +58293 +58294 +58295 +58296 +58297 +58298 +58299 +58300 +58301 +58302 +58303 +58304 +58305 +58306 +58307 +58308 +58309 +58310 +58311 +58312 +58313 +58314 +58315 +58316 +58317 +58318 +58319 +58320 +58321 +58322 +58323 +58324 +58325 +58326 +58327 +58328 +58329 +58330 +58331 +58332 +58333 +58334 +58335 +58336 +58337 +58338 +58339 +58340 +58341 +58342 +58343 +58344 +58345 +58346 +58347 +58348 +58349 +58350 +58351 +58352 +58353 +58354 +58355 +58356 +58357 +58358 +58359 +58360 +58361 +58362 +58363 +58364 +58365 +58366 +58367 +58368 +58369 +58370 +58371 +58372 +58373 +58374 +58375 +58376 +58377 +58378 +58379 +58380 +58381 +58382 +58383 +58384 +58385 +58386 +58387 +58388 +58389 +58390 +58391 +58392 +58393 +58394 +58395 +58396 +58397 +58398 +58399 +58400 +58401 +58402 +58403 +58404 +58405 +58406 +58407 +58408 +58409 +58410 +58411 +58412 +58413 +58414 +58415 +58416 +58417 +58418 +58419 +58420 +58421 +58422 +58423 +58424 +58425 +58426 +58427 +58428 +58429 +58430 +58431 +58432 +58433 +58434 +58435 +58436 +58437 +58438 +58439 +58440 +58441 +58442 +58443 +58444 +58445 +58446 +58447 +58448 +58449 +58450 +58451 +58452 +58453 +58454 +58455 +58456 +58457 +58458 +58459 +58460 +58461 +58462 +58463 +58464 +58465 +58466 +58467 +58468 +58469 +58470 +58471 +58472 +58473 +58474 +58475 +58476 +58477 +58478 +58479 +58480 +58481 +58482 +58483 +58484 +58485 +58486 +58487 +58488 +58489 +58490 +58491 +58492 +58493 +58494 +58495 +58496 +58497 +58498 +58499 +58500 +58501 +58502 +58503 +58504 +58505 +58506 +58507 +58508 +58509 +58510 +58511 +58512 +58513 +58514 +58515 +58516 +58517 +58518 +58519 +58520 +58521 +58522 +58523 +58524 +58525 +58526 +58527 +58528 +58529 +58530 +58531 +58532 +58533 +58534 +58535 +58536 +58537 +58538 +58539 +58540 +58541 +58542 +58543 +58544 +58545 +58546 +58547 +58548 +58549 +58550 +58551 +58552 +58553 +58554 +58555 +58556 +58557 +58558 +58559 +58560 +58561 +58562 +58563 +58564 +58565 +58566 +58567 +58568 +58569 +58570 +58571 +58572 +58573 +58574 +58575 +58576 +58577 +58578 +58579 +58580 +58581 +58582 +58583 +58584 +58585 +58586 +58587 +58588 +58589 +58590 +58591 +58592 +58593 +58594 +58595 +58596 +58597 +58598 +58599 +58600 +58601 +58602 +58603 +58604 +58605 +58606 +58607 +58608 +58609 +58610 +58611 +58612 +58613 +58614 +58615 +58616 +58617 +58618 +58619 +58620 +58621 +58622 +58623 +58624 +58625 +58626 +58627 +58628 +58629 +58630 +58631 +58632 +58633 +58634 +58635 +58636 +58637 +58638 +58639 +58640 +58641 +58642 +58643 +58644 +58645 +58646 +58647 +58648 +58649 +58650 +58651 +58652 +58653 +58654 +58655 +58656 +58657 +58658 +58659 +58660 +58661 +58662 +58663 +58664 +58665 +58666 +58667 +58668 +58669 +58670 +58671 +58672 +58673 +58674 +58675 +58676 +58677 +58678 +58679 +58680 +58681 +58682 +58683 +58684 +58685 +58686 +58687 +58688 +58689 +58690 +58691 +58692 +58693 +58694 +58695 +58696 +58697 +58698 +58699 +58700 +58701 +58702 +58703 +58704 +58705 +58706 +58707 +58708 +58709 +58710 +58711 +58712 +58713 +58714 +58715 +58716 +58717 +58718 +58719 +58720 +58721 +58722 +58723 +58724 +58725 +58726 +58727 +58728 +58729 +58730 +58731 +58732 +58733 +58734 +58735 +58736 +58737 +58738 +58739 +58740 +58741 +58742 +58743 +58744 +58745 +58746 +58747 +58748 +58749 +58750 +58751 +58752 +58753 +58754 +58755 +58756 +58757 +58758 +58759 +58760 +58761 +58762 +58763 +58764 +58765 +58766 +58767 +58768 +58769 +58770 +58771 +58772 +58773 +58774 +58775 +58776 +58777 +58778 +58779 +58780 +58781 +58782 +58783 +58784 +58785 +58786 +58787 +58788 +58789 +58790 +58791 +58792 +58793 +58794 +58795 +58796 +58797 +58798 +58799 +58800 +58801 +58802 +58803 +58804 +58805 +58806 +58807 +58808 +58809 +58810 +58811 +58812 +58813 +58814 +58815 +58816 +58817 +58818 +58819 +58820 +58821 +58822 +58823 +58824 +58825 +58826 +58827 +58828 +58829 +58830 +58831 +58832 +58833 +58834 +58835 +58836 +58837 +58838 +58839 +58840 +58841 +58842 +58843 +58844 +58845 +58846 +58847 +58848 +58849 +58850 +58851 +58852 +58853 +58854 +58855 +58856 +58857 +58858 +58859 +58860 +58861 +58862 +58863 +58864 +58865 +58866 +58867 +58868 +58869 +58870 +58871 +58872 +58873 +58874 +58875 +58876 +58877 +58878 +58879 +58880 +58881 +58882 +58883 +58884 +58885 +58886 +58887 +58888 +58889 +58890 +58891 +58892 +58893 +58894 +58895 +58896 +58897 +58898 +58899 +58900 +58901 +58902 +58903 +58904 +58905 +58906 +58907 +58908 +58909 +58910 +58911 +58912 +58913 +58914 +58915 +58916 +58917 +58918 +58919 +58920 +58921 +58922 +58923 +58924 +58925 +58926 +58927 +58928 +58929 +58930 +58931 +58932 +58933 +58934 +58935 +58936 +58937 +58938 +58939 +58940 +58941 +58942 +58943 +58944 +58945 +58946 +58947 +58948 +58949 +58950 +58951 +58952 +58953 +58954 +58955 +58956 +58957 +58958 +58959 +58960 +58961 +58962 +58963 +58964 +58965 +58966 +58967 +58968 +58969 +58970 +58971 +58972 +58973 +58974 +58975 +58976 +58977 +58978 +58979 +58980 +58981 +58982 +58983 +58984 +58985 +58986 +58987 +58988 +58989 +58990 +58991 +58992 +58993 +58994 +58995 +58996 +58997 +58998 +58999 +59000 +59001 +59002 +59003 +59004 +59005 +59006 +59007 +59008 +59009 +59010 +59011 +59012 +59013 +59014 +59015 +59016 +59017 +59018 +59019 +59020 +59021 +59022 +59023 +59024 +59025 +59026 +59027 +59028 +59029 +59030 +59031 +59032 +59033 +59034 +59035 +59036 +59037 +59038 +59039 +59040 +59041 +59042 +59043 +59044 +59045 +59046 +59047 +59048 +59049 +59050 +59051 +59052 +59053 +59054 +59055 +59056 +59057 +59058 +59059 +59060 +59061 +59062 +59063 +59064 +59065 +59066 +59067 +59068 +59069 +59070 +59071 +59072 +59073 +59074 +59075 +59076 +59077 +59078 +59079 +59080 +59081 +59082 +59083 +59084 +59085 +59086 +59087 +59088 +59089 +59090 +59091 +59092 +59093 +59094 +59095 +59096 +59097 +59098 +59099 +59100 +59101 +59102 +59103 +59104 +59105 +59106 +59107 +59108 +59109 +59110 +59111 +59112 +59113 +59114 +59115 +59116 +59117 +59118 +59119 +59120 +59121 +59122 +59123 +59124 +59125 +59126 +59127 +59128 +59129 +59130 +59131 +59132 +59133 +59134 +59135 +59136 +59137 +59138 +59139 +59140 +59141 +59142 +59143 +59144 +59145 +59146 +59147 +59148 +59149 +59150 +59151 +59152 +59153 +59154 +59155 +59156 +59157 +59158 +59159 +59160 +59161 +59162 +59163 +59164 +59165 +59166 +59167 +59168 +59169 +59170 +59171 +59172 +59173 +59174 +59175 +59176 +59177 +59178 +59179 +59180 +59181 +59182 +59183 +59184 +59185 +59186 +59187 +59188 +59189 +59190 +59191 +59192 +59193 +59194 +59195 +59196 +59197 +59198 +59199 +59200 +59201 +59202 +59203 +59204 +59205 +59206 +59207 +59208 +59209 +59210 +59211 +59212 +59213 +59214 +59215 +59216 +59217 +59218 +59219 +59220 +59221 +59222 +59223 +59224 +59225 +59226 +59227 +59228 +59229 +59230 +59231 +59232 +59233 +59234 +59235 +59236 +59237 +59238 +59239 +59240 +59241 +59242 +59243 +59244 +59245 +59246 +59247 +59248 +59249 +59250 +59251 +59252 +59253 +59254 +59255 +59256 +59257 +59258 +59259 +59260 +59261 +59262 +59263 +59264 +59265 +59266 +59267 +59268 +59269 +59270 +59271 +59272 +59273 +59274 +59275 +59276 +59277 +59278 +59279 +59280 +59281 +59282 +59283 +59284 +59285 +59286 +59287 +59288 +59289 +59290 +59291 +59292 +59293 +59294 +59295 +59296 +59297 +59298 +59299 +59300 +59301 +59302 +59303 +59304 +59305 +59306 +59307 +59308 +59309 +59310 +59311 +59312 +59313 +59314 +59315 +59316 +59317 +59318 +59319 +59320 +59321 +59322 +59323 +59324 +59325 +59326 +59327 +59328 +59329 +59330 +59331 +59332 +59333 +59334 +59335 +59336 +59337 +59338 +59339 +59340 +59341 +59342 +59343 +59344 +59345 +59346 +59347 +59348 +59349 +59350 +59351 +59352 +59353 +59354 +59355 +59356 +59357 +59358 +59359 +59360 +59361 +59362 +59363 +59364 +59365 +59366 +59367 +59368 +59369 +59370 +59371 +59372 +59373 +59374 +59375 +59376 +59377 +59378 +59379 +59380 +59381 +59382 +59383 +59384 +59385 +59386 +59387 +59388 +59389 +59390 +59391 +59392 +59393 +59394 +59395 +59396 +59397 +59398 +59399 +59400 +59401 +59402 +59403 +59404 +59405 +59406 +59407 +59408 +59409 +59410 +59411 +59412 +59413 +59414 +59415 +59416 +59417 +59418 +59419 +59420 +59421 +59422 +59423 +59424 +59425 +59426 +59427 +59428 +59429 +59430 +59431 +59432 +59433 +59434 +59435 +59436 +59437 +59438 +59439 +59440 +59441 +59442 +59443 +59444 +59445 +59446 +59447 +59448 +59449 +59450 +59451 +59452 +59453 +59454 +59455 +59456 +59457 +59458 +59459 +59460 +59461 +59462 +59463 +59464 +59465 +59466 +59467 +59468 +59469 +59470 +59471 +59472 +59473 +59474 +59475 +59476 +59477 +59478 +59479 +59480 +59481 +59482 +59483 +59484 +59485 +59486 +59487 +59488 +59489 +59490 +59491 +59492 +59493 +59494 +59495 +59496 +59497 +59498 +59499 +59500 +59501 +59502 +59503 +59504 +59505 +59506 +59507 +59508 +59509 +59510 +59511 +59512 +59513 +59514 +59515 +59516 +59517 +59518 +59519 +59520 +59521 +59522 +59523 +59524 +59525 +59526 +59527 +59528 +59529 +59530 +59531 +59532 +59533 +59534 +59535 +59536 +59537 +59538 +59539 +59540 +59541 +59542 +59543 +59544 +59545 +59546 +59547 +59548 +59549 +59550 +59551 +59552 +59553 +59554 +59555 +59556 +59557 +59558 +59559 +59560 +59561 +59562 +59563 +59564 +59565 +59566 +59567 +59568 +59569 +59570 +59571 +59572 +59573 +59574 +59575 +59576 +59577 +59578 +59579 +59580 +59581 +59582 +59583 +59584 +59585 +59586 +59587 +59588 +59589 +59590 +59591 +59592 +59593 +59594 +59595 +59596 +59597 +59598 +59599 +59600 +59601 +59602 +59603 +59604 +59605 +59606 +59607 +59608 +59609 +59610 +59611 +59612 +59613 +59614 +59615 +59616 +59617 +59618 +59619 +59620 +59621 +59622 +59623 +59624 +59625 +59626 +59627 +59628 +59629 +59630 +59631 +59632 +59633 +59634 +59635 +59636 +59637 +59638 +59639 +59640 +59641 +59642 +59643 +59644 +59645 +59646 +59647 +59648 +59649 +59650 +59651 +59652 +59653 +59654 +59655 +59656 +59657 +59658 +59659 +59660 +59661 +59662 +59663 +59664 +59665 +59666 +59667 +59668 +59669 +59670 +59671 +59672 +59673 +59674 +59675 +59676 +59677 +59678 +59679 +59680 +59681 +59682 +59683 +59684 +59685 +59686 +59687 +59688 +59689 +59690 +59691 +59692 +59693 +59694 +59695 +59696 +59697 +59698 +59699 +59700 +59701 +59702 +59703 +59704 +59705 +59706 +59707 +59708 +59709 +59710 +59711 +59712 +59713 +59714 +59715 +59716 +59717 +59718 +59719 +59720 +59721 +59722 +59723 +59724 +59725 +59726 +59727 +59728 +59729 +59730 +59731 +59732 +59733 +59734 +59735 +59736 +59737 +59738 +59739 +59740 +59741 +59742 +59743 +59744 +59745 +59746 +59747 +59748 +59749 +59750 +59751 +59752 +59753 +59754 +59755 +59756 +59757 +59758 +59759 +59760 +59761 +59762 +59763 +59764 +59765 +59766 +59767 +59768 +59769 +59770 +59771 +59772 +59773 +59774 +59775 +59776 +59777 +59778 +59779 +59780 +59781 +59782 +59783 +59784 +59785 +59786 +59787 +59788 +59789 +59790 +59791 +59792 +59793 +59794 +59795 +59796 +59797 +59798 +59799 +59800 +59801 +59802 +59803 +59804 +59805 +59806 +59807 +59808 +59809 +59810 +59811 +59812 +59813 +59814 +59815 +59816 +59817 +59818 +59819 +59820 +59821 +59822 +59823 +59824 +59825 +59826 +59827 +59828 +59829 +59830 +59831 +59832 +59833 +59834 +59835 +59836 +59837 +59838 +59839 +59840 +59841 +59842 +59843 +59844 +59845 +59846 +59847 +59848 +59849 +59850 +59851 +59852 +59853 +59854 +59855 +59856 +59857 +59858 +59859 +59860 +59861 +59862 +59863 +59864 +59865 +59866 +59867 +59868 +59869 +59870 +59871 +59872 +59873 +59874 +59875 +59876 +59877 +59878 +59879 +59880 +59881 +59882 +59883 +59884 +59885 +59886 +59887 +59888 +59889 +59890 +59891 +59892 +59893 +59894 +59895 +59896 +59897 +59898 +59899 +59900 +59901 +59902 +59903 +59904 +59905 +59906 +59907 +59908 +59909 +59910 +59911 +59912 +59913 +59914 +59915 +59916 +59917 +59918 +59919 +59920 +59921 +59922 +59923 +59924 +59925 +59926 +59927 +59928 +59929 +59930 +59931 +59932 +59933 +59934 +59935 +59936 +59937 +59938 +59939 +59940 +59941 +59942 +59943 +59944 +59945 +59946 +59947 +59948 +59949 +59950 +59951 +59952 +59953 +59954 +59955 +59956 +59957 +59958 +59959 +59960 +59961 +59962 +59963 +59964 +59965 +59966 +59967 +59968 +59969 +59970 +59971 +59972 +59973 +59974 +59975 +59976 +59977 +59978 +59979 +59980 +59981 +59982 +59983 +59984 +59985 +59986 +59987 +59988 +59989 +59990 +59991 +59992 +59993 +59994 +59995 +59996 +59997 +59998 +59999 diff --git a/tests/queries/0_stateless/02003_memory_limit_in_client.sh b/tests/queries/0_stateless/02003_memory_limit_in_client.sh new file mode 100755 index 00000000000..1cb3014c38f --- /dev/null +++ b/tests/queries/0_stateless/02003_memory_limit_in_client.sh @@ -0,0 +1,14 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +# Big quiery. Memory limit +${CLICKHOUSE_CLIENT} --max_memory_usage_in_client=1 --query "SELECT arrayMap(x -> range(x), range(number)) FROM numbers(1000) -- { serverError 241; }" + +# small quiery. In max untracked memory range +${CLICKHOUSE_CLIENT} --max_memory_usage_in_client=1 --query "SELECT * FROM system.numbers LIMIT 60000" + +# Have anough memory +${CLICKHOUSE_CLIENT} --max_memory_usage_in_client=20000000 --query "SELECT arrayMap(x -> range(x), range(number)) FROM numbers(1000)" From e8b6c0a22777c14a196bc09d76cb2abb43c85936 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E8=91=A3=E6=B5=B7=E9=95=94?= Date: Fri, 6 Aug 2021 00:44:07 +0800 Subject: [PATCH 165/599] add bitmapSubsetOffsetLimit function --- .../functions/bitmap-functions.md | 38 ++++++++++++++ .../functions/bitmap-functions.md | 24 +++++++++ .../AggregateFunctionGroupBitmapData.h | 51 +++++++++++++++++++ src/Functions/FunctionsBitmap.cpp | 1 + src/Functions/FunctionsBitmap.h | 17 ++++++- .../00829_bitmap_function.reference | 8 +++ .../0_stateless/00829_bitmap_function.sql | 19 +++++++ 7 files changed, 157 insertions(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/bitmap-functions.md b/docs/en/sql-reference/functions/bitmap-functions.md index c695c894784..2becaefbc53 100644 --- a/docs/en/sql-reference/functions/bitmap-functions.md +++ b/docs/en/sql-reference/functions/bitmap-functions.md @@ -125,6 +125,44 @@ Result: └───────────────────────────┘ ``` +## bitmapSubsetOffsetLimit {#bitmapsubsetoffsetlimit} + +Creates a subset of bitmap limit the results to `cardinality_limit` with offset of `offset`. + +**Syntax** + +``` sql +bitmapSubsetOffsetLimit(bitmap, offset, cardinality_limit) +``` + +**Arguments** + +- `bitmap` – [Bitmap object](#bitmap_functions-bitmapbuild). +- `offset` – the number of offsets. Type: [UInt32](../../sql-reference/data-types/int-uint.md). +- `cardinality_limit` – The subset cardinality upper limit. Type: [UInt32](../../sql-reference/data-types/int-uint.md). + +**Returned value** + +The subset. + +Type: `Bitmap object`. + +**Example** + +Query: + +``` sql +SELECT bitmapToArray(bitmapSubsetOffsetLimit(bitmapBuild([0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,100,200,500]), toUInt32(10), toUInt32(10))) AS res; +``` + +Result: + +``` text +┌─res─────────────────────────────┐ +│ [10,11,12,13,14,15,16,17,18,19] │ +└─────────────────────────────────┘ +``` + ## bitmapContains {#bitmap_functions-bitmapcontains} Checks whether the bitmap contains an element. diff --git a/docs/zh/sql-reference/functions/bitmap-functions.md b/docs/zh/sql-reference/functions/bitmap-functions.md index 5a9a88c5be1..e9480d3e411 100644 --- a/docs/zh/sql-reference/functions/bitmap-functions.md +++ b/docs/zh/sql-reference/functions/bitmap-functions.md @@ -88,6 +88,30 @@ SELECT bitmapToArray(bitmapSubsetLimit(bitmapBuild([0,1,2,3,4,5,6,7,8,9,10,11,12 │ [30,31,32,33,100,200,500] │ └───────────────────────────┘ +## bitmapSubsetOffsetLimit {#bitmapsubsetoffsetlimit} + +将位图跳过`offset`个元素,限制大小为`limit`个的结果转换为另一个位图。 + + bitmapSubsetOffsetLimit(bitmap, offset, limit) + +**参数** + +- `bitmap` – 位图对象. +- `offset` – 跳过多少个元素. +- `limit` – 子位图基数上限. + +**示例** + +``` sql +SELECT bitmapToArray(bitmapSubsetOffsetLimit(bitmapBuild([0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,100,200,500]), toUInt32(10), toUInt32(10))) AS res +``` + +```text +┌─res─────────────────────────────┐ +│ [10,11,12,13,14,15,16,17,18,19] │ +└─────────────────────────────────┘ +``` + ## bitmapContains {#bitmapcontains} 检查位图是否包含指定元素。 diff --git a/src/AggregateFunctions/AggregateFunctionGroupBitmapData.h b/src/AggregateFunctions/AggregateFunctionGroupBitmapData.h index 067daf6dc3a..8e5a1b34104 100644 --- a/src/AggregateFunctions/AggregateFunctionGroupBitmapData.h +++ b/src/AggregateFunctions/AggregateFunctionGroupBitmapData.h @@ -579,6 +579,57 @@ public: } } + UInt64 rb_offset_limit(UInt64 offset, UInt64 limit, RoaringBitmapWithSmallSet & r1) const + { + if (limit == 0 || offset >= size()) + return 0; + + if (isSmall()) + { + UInt64 offset_count = 0; + std::vector answer; + for (const auto & x : small) + { + T val = x.getValue(); + if (offset_count >= offset) + { + answer.push_back(val); + } else { + offset_count++; + } + } + if (limit < answer.size()) + { + std::nth_element(answer.begin(), answer.begin() + limit, answer.end()); + answer.resize(limit); + } + + for (const auto & elem : answer) + r1.add(elem); + return answer.size(); + } + else + { + UInt64 count = 0; + UInt64 offset_count = 0; + for (auto it = rb->begin(); it != rb->end(); ++it) + { + offset_count++; + if (offset_count <= offset) + continue; + + if (count < limit) + { + r1.add(*it); + ++count; + } + else + break; + } + return count; + } + } + UInt64 rb_min() const { if (isSmall()) diff --git a/src/Functions/FunctionsBitmap.cpp b/src/Functions/FunctionsBitmap.cpp index 72652288872..159d0ff6777 100644 --- a/src/Functions/FunctionsBitmap.cpp +++ b/src/Functions/FunctionsBitmap.cpp @@ -13,6 +13,7 @@ void registerFunctionsBitmap(FunctionFactory & factory) factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); + factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); diff --git a/src/Functions/FunctionsBitmap.h b/src/Functions/FunctionsBitmap.h index bbdc53c3006..b72ee06dc01 100644 --- a/src/Functions/FunctionsBitmap.h +++ b/src/Functions/FunctionsBitmap.h @@ -460,9 +460,24 @@ public: } }; +struct BitmapSubsetOffsetLimitImpl +{ +public: + static constexpr auto name = "bitmapSubsetOffsetLimit"; + template + static void apply( + const AggregateFunctionGroupBitmapData & bitmap_data_0, + UInt64 range_start, + UInt64 range_end, + AggregateFunctionGroupBitmapData & bitmap_data_2) + { + bitmap_data_0.rbs.rb_offset_limit(range_start, range_end, bitmap_data_2.rbs); + } +}; + using FunctionBitmapSubsetInRange = FunctionBitmapSubset; using FunctionBitmapSubsetLimit = FunctionBitmapSubset; - +using FunctionBitmapSubsetOffsetLimit = FunctionBitmapSubset; class FunctionBitmapTransform : public IFunction { diff --git a/tests/queries/0_stateless/00829_bitmap_function.reference b/tests/queries/0_stateless/00829_bitmap_function.reference index 02633af568d..f403907bce4 100644 --- a/tests/queries/0_stateless/00829_bitmap_function.reference +++ b/tests/queries/0_stateless/00829_bitmap_function.reference @@ -91,6 +91,14 @@ tag4 [0,1,2,3,4,5,6,7,8,9] [5,999,2] [2,888,20] [0,1,3,4,6,7,8,9,20] [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,100,200,500] [30,31,32,33,100,200,500] [100,200,500] +[] +[] +[1,5,7,9] +[5,7,9] +[5,7] +[0,1,2,3,4,5,6,7,8,9] +[30,31,32,33,100,200,500] +[100,200,500] 0 0 0 diff --git a/tests/queries/0_stateless/00829_bitmap_function.sql b/tests/queries/0_stateless/00829_bitmap_function.sql index b9e9664a56e..0854f788a50 100644 --- a/tests/queries/0_stateless/00829_bitmap_function.sql +++ b/tests/queries/0_stateless/00829_bitmap_function.sql @@ -286,6 +286,25 @@ select bitmapToArray(bitmapSubsetLimit(bitmapBuild([ 0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33, 100,200,500]), toUInt32(100), toUInt16(200))); +-- bitmapSubsetOffsetLimit: +---- Empty +SELECT bitmapToArray(bitmapSubsetOffsetLimit(bitmapBuild(emptyArrayUInt32()), toUInt8(0), toUInt32(10))); +SELECT bitmapToArray(bitmapSubsetOffsetLimit(bitmapBuild(emptyArrayUInt16()), toUInt32(0), toUInt64(10))); +---- Small +select bitmapToArray(bitmapSubsetOffsetLimit(bitmapBuild([1,5,7,9]), toUInt8(0), toUInt32(4))); +select bitmapToArray(bitmapSubsetOffsetLimit(bitmapBuild([1,5,7,9]), toUInt32(1), toUInt64(4))); +select bitmapToArray(bitmapSubsetOffsetLimit(bitmapBuild([1,5,7,9]), toUInt16(1), toUInt32(2))); +---- Large +select bitmapToArray(bitmapSubsetOffsetLimit(bitmapBuild([ + 0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33, + 100,200,500]), toUInt32(0), toUInt32(10))); +select bitmapToArray(bitmapSubsetOffsetLimit(bitmapBuild([ + 0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33, + 100,200,500]), toUInt32(30), toUInt32(200))); +select bitmapToArray(bitmapSubsetOffsetLimit(bitmapBuild([ + 0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33, + 100,200,500]), toUInt32(34), toUInt16(3))); + -- bitmapMin: ---- Empty SELECT bitmapMin(bitmapBuild(emptyArrayUInt8())); From fbcf0962eebec993694b247fee27e8111418d234 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 5 Aug 2021 21:01:35 +0300 Subject: [PATCH 166/599] Mark some tests as long --- ...=> 00062_replicated_merge_tree_alter_zookeeper_long.reference} | 0 ...r.sql => 00062_replicated_merge_tree_alter_zookeeper_long.sql} | 0 ...reference => 00083_create_merge_tree_zookeeper_long.reference} | 0 ...e_zookeeper.sql => 00083_create_merge_tree_zookeeper_long.sql} | 0 ...reference => 00215_primary_key_order_zookeeper_long.reference} | 0 ...r_zookeeper.sql => 00215_primary_key_order_zookeeper_long.sql} | 0 ...00502_custom_partitioning_replicated_zookeeper_long.reference} | 0 ...ql => 00502_custom_partitioning_replicated_zookeeper_long.sql} | 0 ...st_zookeeper.reference => 00643_cast_zookeeper_long.reference} | 0 .../{00643_cast_zookeeper.sql => 00643_cast_zookeeper_long.sql} | 0 ...ect_with_old_data_and_without_quorum_zookeeper_long.reference} | 0 ...rt_select_with_old_data_and_without_quorum_zookeeper_long.sql} | 0 ...ce => 00836_indices_alter_replicated_zookeeper_long.reference} | 0 ...eper.sql => 00836_indices_alter_replicated_zookeeper_long.sql} | 0 ...zookeeper_custom_compression_codecs_replicated_long.reference} | 0 ...00910_zookeeper_custom_compression_codecs_replicated_long.sql} | 0 ...aptive_index_granularity_replicated_merge_tree_long.reference} | 0 ...per_adaptive_index_granularity_replicated_merge_tree_long.sql} | 0 ...=> 01090_zookeeper_mutations_and_insert_quorum_long.reference} | 0 ...m.sql => 01090_zookeeper_mutations_and_insert_quorum_long.sql} | 0 ..._alter_enum_partition_key_replicated_zookeeper_long.reference} | 0 ... 01346_alter_enum_partition_key_replicated_zookeeper_long.sql} | 0 ....reference => 01430_modify_sample_by_zookeeper_long.reference} | 0 ...by_zookeeper.sql => 01430_modify_sample_by_zookeeper_long.sql} | 0 ...ence => 01532_execute_merges_on_single_replica_long.reference} | 0 ...eplica.sql => 01532_execute_merges_on_single_replica_long.sql} | 0 ...nce => 01581_deduplicate_by_columns_replicated_long.reference} | 0 ...cated.sql => 01581_deduplicate_by_columns_replicated_long.sql} | 0 28 files changed, 0 insertions(+), 0 deletions(-) rename tests/queries/0_stateless/{00062_replicated_merge_tree_alter_zookeeper.reference => 00062_replicated_merge_tree_alter_zookeeper_long.reference} (100%) rename tests/queries/0_stateless/{00062_replicated_merge_tree_alter_zookeeper.sql => 00062_replicated_merge_tree_alter_zookeeper_long.sql} (100%) rename tests/queries/0_stateless/{00083_create_merge_tree_zookeeper.reference => 00083_create_merge_tree_zookeeper_long.reference} (100%) rename tests/queries/0_stateless/{00083_create_merge_tree_zookeeper.sql => 00083_create_merge_tree_zookeeper_long.sql} (100%) rename tests/queries/0_stateless/{00215_primary_key_order_zookeeper.reference => 00215_primary_key_order_zookeeper_long.reference} (100%) rename tests/queries/0_stateless/{00215_primary_key_order_zookeeper.sql => 00215_primary_key_order_zookeeper_long.sql} (100%) rename tests/queries/0_stateless/{00502_custom_partitioning_replicated_zookeeper.reference => 00502_custom_partitioning_replicated_zookeeper_long.reference} (100%) rename tests/queries/0_stateless/{00502_custom_partitioning_replicated_zookeeper.sql => 00502_custom_partitioning_replicated_zookeeper_long.sql} (100%) rename tests/queries/0_stateless/{00643_cast_zookeeper.reference => 00643_cast_zookeeper_long.reference} (100%) rename tests/queries/0_stateless/{00643_cast_zookeeper.sql => 00643_cast_zookeeper_long.sql} (100%) rename tests/queries/0_stateless/{00732_quorum_insert_select_with_old_data_and_without_quorum_zookeeper.reference => 00732_quorum_insert_select_with_old_data_and_without_quorum_zookeeper_long.reference} (100%) rename tests/queries/0_stateless/{00732_quorum_insert_select_with_old_data_and_without_quorum_zookeeper.sql => 00732_quorum_insert_select_with_old_data_and_without_quorum_zookeeper_long.sql} (100%) rename tests/queries/0_stateless/{00836_indices_alter_replicated_zookeeper.reference => 00836_indices_alter_replicated_zookeeper_long.reference} (100%) rename tests/queries/0_stateless/{00836_indices_alter_replicated_zookeeper.sql => 00836_indices_alter_replicated_zookeeper_long.sql} (100%) rename tests/queries/0_stateless/{00910_zookeeper_custom_compression_codecs_replicated.reference => 00910_zookeeper_custom_compression_codecs_replicated_long.reference} (100%) rename tests/queries/0_stateless/{00910_zookeeper_custom_compression_codecs_replicated.sql => 00910_zookeeper_custom_compression_codecs_replicated_long.sql} (100%) rename tests/queries/0_stateless/{00926_zookeeper_adaptive_index_granularity_replicated_merge_tree.reference => 00926_zookeeper_adaptive_index_granularity_replicated_merge_tree_long.reference} (100%) rename tests/queries/0_stateless/{00926_zookeeper_adaptive_index_granularity_replicated_merge_tree.sql => 00926_zookeeper_adaptive_index_granularity_replicated_merge_tree_long.sql} (100%) rename tests/queries/0_stateless/{01090_zookeeper_mutations_and_insert_quorum.reference => 01090_zookeeper_mutations_and_insert_quorum_long.reference} (100%) rename tests/queries/0_stateless/{01090_zookeeper_mutations_and_insert_quorum.sql => 01090_zookeeper_mutations_and_insert_quorum_long.sql} (100%) rename tests/queries/0_stateless/{01346_alter_enum_partition_key_replicated_zookeeper.reference => 01346_alter_enum_partition_key_replicated_zookeeper_long.reference} (100%) rename tests/queries/0_stateless/{01346_alter_enum_partition_key_replicated_zookeeper.sql => 01346_alter_enum_partition_key_replicated_zookeeper_long.sql} (100%) rename tests/queries/0_stateless/{01430_modify_sample_by_zookeeper.reference => 01430_modify_sample_by_zookeeper_long.reference} (100%) rename tests/queries/0_stateless/{01430_modify_sample_by_zookeeper.sql => 01430_modify_sample_by_zookeeper_long.sql} (100%) rename tests/queries/0_stateless/{01532_execute_merges_on_single_replica.reference => 01532_execute_merges_on_single_replica_long.reference} (100%) rename tests/queries/0_stateless/{01532_execute_merges_on_single_replica.sql => 01532_execute_merges_on_single_replica_long.sql} (100%) rename tests/queries/0_stateless/{01581_deduplicate_by_columns_replicated.reference => 01581_deduplicate_by_columns_replicated_long.reference} (100%) rename tests/queries/0_stateless/{01581_deduplicate_by_columns_replicated.sql => 01581_deduplicate_by_columns_replicated_long.sql} (100%) diff --git a/tests/queries/0_stateless/00062_replicated_merge_tree_alter_zookeeper.reference b/tests/queries/0_stateless/00062_replicated_merge_tree_alter_zookeeper_long.reference similarity index 100% rename from tests/queries/0_stateless/00062_replicated_merge_tree_alter_zookeeper.reference rename to tests/queries/0_stateless/00062_replicated_merge_tree_alter_zookeeper_long.reference diff --git a/tests/queries/0_stateless/00062_replicated_merge_tree_alter_zookeeper.sql b/tests/queries/0_stateless/00062_replicated_merge_tree_alter_zookeeper_long.sql similarity index 100% rename from tests/queries/0_stateless/00062_replicated_merge_tree_alter_zookeeper.sql rename to tests/queries/0_stateless/00062_replicated_merge_tree_alter_zookeeper_long.sql diff --git a/tests/queries/0_stateless/00083_create_merge_tree_zookeeper.reference b/tests/queries/0_stateless/00083_create_merge_tree_zookeeper_long.reference similarity index 100% rename from tests/queries/0_stateless/00083_create_merge_tree_zookeeper.reference rename to tests/queries/0_stateless/00083_create_merge_tree_zookeeper_long.reference diff --git a/tests/queries/0_stateless/00083_create_merge_tree_zookeeper.sql b/tests/queries/0_stateless/00083_create_merge_tree_zookeeper_long.sql similarity index 100% rename from tests/queries/0_stateless/00083_create_merge_tree_zookeeper.sql rename to tests/queries/0_stateless/00083_create_merge_tree_zookeeper_long.sql diff --git a/tests/queries/0_stateless/00215_primary_key_order_zookeeper.reference b/tests/queries/0_stateless/00215_primary_key_order_zookeeper_long.reference similarity index 100% rename from tests/queries/0_stateless/00215_primary_key_order_zookeeper.reference rename to tests/queries/0_stateless/00215_primary_key_order_zookeeper_long.reference diff --git a/tests/queries/0_stateless/00215_primary_key_order_zookeeper.sql b/tests/queries/0_stateless/00215_primary_key_order_zookeeper_long.sql similarity index 100% rename from tests/queries/0_stateless/00215_primary_key_order_zookeeper.sql rename to tests/queries/0_stateless/00215_primary_key_order_zookeeper_long.sql diff --git a/tests/queries/0_stateless/00502_custom_partitioning_replicated_zookeeper.reference b/tests/queries/0_stateless/00502_custom_partitioning_replicated_zookeeper_long.reference similarity index 100% rename from tests/queries/0_stateless/00502_custom_partitioning_replicated_zookeeper.reference rename to tests/queries/0_stateless/00502_custom_partitioning_replicated_zookeeper_long.reference diff --git a/tests/queries/0_stateless/00502_custom_partitioning_replicated_zookeeper.sql b/tests/queries/0_stateless/00502_custom_partitioning_replicated_zookeeper_long.sql similarity index 100% rename from tests/queries/0_stateless/00502_custom_partitioning_replicated_zookeeper.sql rename to tests/queries/0_stateless/00502_custom_partitioning_replicated_zookeeper_long.sql diff --git a/tests/queries/0_stateless/00643_cast_zookeeper.reference b/tests/queries/0_stateless/00643_cast_zookeeper_long.reference similarity index 100% rename from tests/queries/0_stateless/00643_cast_zookeeper.reference rename to tests/queries/0_stateless/00643_cast_zookeeper_long.reference diff --git a/tests/queries/0_stateless/00643_cast_zookeeper.sql b/tests/queries/0_stateless/00643_cast_zookeeper_long.sql similarity index 100% rename from tests/queries/0_stateless/00643_cast_zookeeper.sql rename to tests/queries/0_stateless/00643_cast_zookeeper_long.sql diff --git a/tests/queries/0_stateless/00732_quorum_insert_select_with_old_data_and_without_quorum_zookeeper.reference b/tests/queries/0_stateless/00732_quorum_insert_select_with_old_data_and_without_quorum_zookeeper_long.reference similarity index 100% rename from tests/queries/0_stateless/00732_quorum_insert_select_with_old_data_and_without_quorum_zookeeper.reference rename to tests/queries/0_stateless/00732_quorum_insert_select_with_old_data_and_without_quorum_zookeeper_long.reference diff --git a/tests/queries/0_stateless/00732_quorum_insert_select_with_old_data_and_without_quorum_zookeeper.sql b/tests/queries/0_stateless/00732_quorum_insert_select_with_old_data_and_without_quorum_zookeeper_long.sql similarity index 100% rename from tests/queries/0_stateless/00732_quorum_insert_select_with_old_data_and_without_quorum_zookeeper.sql rename to tests/queries/0_stateless/00732_quorum_insert_select_with_old_data_and_without_quorum_zookeeper_long.sql diff --git a/tests/queries/0_stateless/00836_indices_alter_replicated_zookeeper.reference b/tests/queries/0_stateless/00836_indices_alter_replicated_zookeeper_long.reference similarity index 100% rename from tests/queries/0_stateless/00836_indices_alter_replicated_zookeeper.reference rename to tests/queries/0_stateless/00836_indices_alter_replicated_zookeeper_long.reference diff --git a/tests/queries/0_stateless/00836_indices_alter_replicated_zookeeper.sql b/tests/queries/0_stateless/00836_indices_alter_replicated_zookeeper_long.sql similarity index 100% rename from tests/queries/0_stateless/00836_indices_alter_replicated_zookeeper.sql rename to tests/queries/0_stateless/00836_indices_alter_replicated_zookeeper_long.sql diff --git a/tests/queries/0_stateless/00910_zookeeper_custom_compression_codecs_replicated.reference b/tests/queries/0_stateless/00910_zookeeper_custom_compression_codecs_replicated_long.reference similarity index 100% rename from tests/queries/0_stateless/00910_zookeeper_custom_compression_codecs_replicated.reference rename to tests/queries/0_stateless/00910_zookeeper_custom_compression_codecs_replicated_long.reference diff --git a/tests/queries/0_stateless/00910_zookeeper_custom_compression_codecs_replicated.sql b/tests/queries/0_stateless/00910_zookeeper_custom_compression_codecs_replicated_long.sql similarity index 100% rename from tests/queries/0_stateless/00910_zookeeper_custom_compression_codecs_replicated.sql rename to tests/queries/0_stateless/00910_zookeeper_custom_compression_codecs_replicated_long.sql diff --git a/tests/queries/0_stateless/00926_zookeeper_adaptive_index_granularity_replicated_merge_tree.reference b/tests/queries/0_stateless/00926_zookeeper_adaptive_index_granularity_replicated_merge_tree_long.reference similarity index 100% rename from tests/queries/0_stateless/00926_zookeeper_adaptive_index_granularity_replicated_merge_tree.reference rename to tests/queries/0_stateless/00926_zookeeper_adaptive_index_granularity_replicated_merge_tree_long.reference diff --git a/tests/queries/0_stateless/00926_zookeeper_adaptive_index_granularity_replicated_merge_tree.sql b/tests/queries/0_stateless/00926_zookeeper_adaptive_index_granularity_replicated_merge_tree_long.sql similarity index 100% rename from tests/queries/0_stateless/00926_zookeeper_adaptive_index_granularity_replicated_merge_tree.sql rename to tests/queries/0_stateless/00926_zookeeper_adaptive_index_granularity_replicated_merge_tree_long.sql diff --git a/tests/queries/0_stateless/01090_zookeeper_mutations_and_insert_quorum.reference b/tests/queries/0_stateless/01090_zookeeper_mutations_and_insert_quorum_long.reference similarity index 100% rename from tests/queries/0_stateless/01090_zookeeper_mutations_and_insert_quorum.reference rename to tests/queries/0_stateless/01090_zookeeper_mutations_and_insert_quorum_long.reference diff --git a/tests/queries/0_stateless/01090_zookeeper_mutations_and_insert_quorum.sql b/tests/queries/0_stateless/01090_zookeeper_mutations_and_insert_quorum_long.sql similarity index 100% rename from tests/queries/0_stateless/01090_zookeeper_mutations_and_insert_quorum.sql rename to tests/queries/0_stateless/01090_zookeeper_mutations_and_insert_quorum_long.sql diff --git a/tests/queries/0_stateless/01346_alter_enum_partition_key_replicated_zookeeper.reference b/tests/queries/0_stateless/01346_alter_enum_partition_key_replicated_zookeeper_long.reference similarity index 100% rename from tests/queries/0_stateless/01346_alter_enum_partition_key_replicated_zookeeper.reference rename to tests/queries/0_stateless/01346_alter_enum_partition_key_replicated_zookeeper_long.reference diff --git a/tests/queries/0_stateless/01346_alter_enum_partition_key_replicated_zookeeper.sql b/tests/queries/0_stateless/01346_alter_enum_partition_key_replicated_zookeeper_long.sql similarity index 100% rename from tests/queries/0_stateless/01346_alter_enum_partition_key_replicated_zookeeper.sql rename to tests/queries/0_stateless/01346_alter_enum_partition_key_replicated_zookeeper_long.sql diff --git a/tests/queries/0_stateless/01430_modify_sample_by_zookeeper.reference b/tests/queries/0_stateless/01430_modify_sample_by_zookeeper_long.reference similarity index 100% rename from tests/queries/0_stateless/01430_modify_sample_by_zookeeper.reference rename to tests/queries/0_stateless/01430_modify_sample_by_zookeeper_long.reference diff --git a/tests/queries/0_stateless/01430_modify_sample_by_zookeeper.sql b/tests/queries/0_stateless/01430_modify_sample_by_zookeeper_long.sql similarity index 100% rename from tests/queries/0_stateless/01430_modify_sample_by_zookeeper.sql rename to tests/queries/0_stateless/01430_modify_sample_by_zookeeper_long.sql diff --git a/tests/queries/0_stateless/01532_execute_merges_on_single_replica.reference b/tests/queries/0_stateless/01532_execute_merges_on_single_replica_long.reference similarity index 100% rename from tests/queries/0_stateless/01532_execute_merges_on_single_replica.reference rename to tests/queries/0_stateless/01532_execute_merges_on_single_replica_long.reference diff --git a/tests/queries/0_stateless/01532_execute_merges_on_single_replica.sql b/tests/queries/0_stateless/01532_execute_merges_on_single_replica_long.sql similarity index 100% rename from tests/queries/0_stateless/01532_execute_merges_on_single_replica.sql rename to tests/queries/0_stateless/01532_execute_merges_on_single_replica_long.sql diff --git a/tests/queries/0_stateless/01581_deduplicate_by_columns_replicated.reference b/tests/queries/0_stateless/01581_deduplicate_by_columns_replicated_long.reference similarity index 100% rename from tests/queries/0_stateless/01581_deduplicate_by_columns_replicated.reference rename to tests/queries/0_stateless/01581_deduplicate_by_columns_replicated_long.reference diff --git a/tests/queries/0_stateless/01581_deduplicate_by_columns_replicated.sql b/tests/queries/0_stateless/01581_deduplicate_by_columns_replicated_long.sql similarity index 100% rename from tests/queries/0_stateless/01581_deduplicate_by_columns_replicated.sql rename to tests/queries/0_stateless/01581_deduplicate_by_columns_replicated_long.sql From d5987b7a0695862371946b7e796fc279419c70b0 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 5 Aug 2021 21:01:36 +0300 Subject: [PATCH 167/599] Fix 00754_alter_modify_order_by_replicated_zookeeper --- ...00754_alter_modify_order_by_replicated_zookeeper.reference | 4 ++-- .../00754_alter_modify_order_by_replicated_zookeeper.sql | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/00754_alter_modify_order_by_replicated_zookeeper.reference b/tests/queries/0_stateless/00754_alter_modify_order_by_replicated_zookeeper.reference index 48fe2d30bf3..a1fecd72e30 100644 --- a/tests/queries/0_stateless/00754_alter_modify_order_by_replicated_zookeeper.reference +++ b/tests/queries/0_stateless/00754_alter_modify_order_by_replicated_zookeeper.reference @@ -9,6 +9,6 @@ 1 2 1 30 1 2 4 90 *** Check SHOW CREATE TABLE *** -CREATE TABLE default.summing_r2\n(\n `x` UInt32,\n `y` UInt32,\n `z` UInt32,\n `val` UInt32\n)\nENGINE = ReplicatedSummingMergeTree(\'/clickhouse/tables/test_00754/summing\', \'r2\')\nPRIMARY KEY (x, y)\nORDER BY (x, y, -z)\nSETTINGS index_granularity = 8192 +CREATE TABLE default.summing_r2\n(\n `x` UInt32,\n `y` UInt32,\n `z` UInt32,\n `val` UInt32\n)\nENGINE = ReplicatedSummingMergeTree(\'/clickhouse/tables/default/test_00754/summing\', \'r2\')\nPRIMARY KEY (x, y)\nORDER BY (x, y, -z)\nSETTINGS index_granularity = 8192 *** Check SHOW CREATE TABLE after offline ALTER *** -CREATE TABLE default.summing_r2\n(\n `x` UInt32,\n `y` UInt32,\n `z` UInt32,\n `t` UInt32,\n `val` UInt32\n)\nENGINE = ReplicatedSummingMergeTree(\'/clickhouse/tables/test_00754/summing\', \'r2\')\nPRIMARY KEY (x, y)\nORDER BY (x, y, t * t)\nSETTINGS index_granularity = 8192 +CREATE TABLE default.summing_r2\n(\n `x` UInt32,\n `y` UInt32,\n `z` UInt32,\n `t` UInt32,\n `val` UInt32\n)\nENGINE = ReplicatedSummingMergeTree(\'/clickhouse/tables/default/test_00754/summing\', \'r2\')\nPRIMARY KEY (x, y)\nORDER BY (x, y, t * t)\nSETTINGS index_granularity = 8192 diff --git a/tests/queries/0_stateless/00754_alter_modify_order_by_replicated_zookeeper.sql b/tests/queries/0_stateless/00754_alter_modify_order_by_replicated_zookeeper.sql index 9e08406202e..78986338cd9 100644 --- a/tests/queries/0_stateless/00754_alter_modify_order_by_replicated_zookeeper.sql +++ b/tests/queries/0_stateless/00754_alter_modify_order_by_replicated_zookeeper.sql @@ -9,8 +9,8 @@ DROP TABLE old_style; DROP TABLE IF EXISTS summing_r1; DROP TABLE IF EXISTS summing_r2; -CREATE TABLE summing_r1(x UInt32, y UInt32, val UInt32) ENGINE ReplicatedSummingMergeTree('/clickhouse/tables/test_00754/summing', 'r1') ORDER BY (x, y); -CREATE TABLE summing_r2(x UInt32, y UInt32, val UInt32) ENGINE ReplicatedSummingMergeTree('/clickhouse/tables/test_00754/summing', 'r2') ORDER BY (x, y); +CREATE TABLE summing_r1(x UInt32, y UInt32, val UInt32) ENGINE ReplicatedSummingMergeTree('/clickhouse/tables/{database}/test_00754/summing', 'r1') ORDER BY (x, y); +CREATE TABLE summing_r2(x UInt32, y UInt32, val UInt32) ENGINE ReplicatedSummingMergeTree('/clickhouse/tables/{database}/test_00754/summing', 'r2') ORDER BY (x, y); /* Can't add an expression with existing column to ORDER BY. */ ALTER TABLE summing_r1 MODIFY ORDER BY (x, y, -val); -- { serverError 36 } From 685c7b8df29f3e57ed61600237beafcb349ccd67 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 5 Aug 2021 21:01:36 +0300 Subject: [PATCH 168/599] Add some tests to parallel list excludes --- tests/queries/skip_list.json | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/tests/queries/skip_list.json b/tests/queries/skip_list.json index 5a4c982e13d..24acb86299f 100644 --- a/tests/queries/skip_list.json +++ b/tests/queries/skip_list.json @@ -498,6 +498,11 @@ "01902_table_function_merge_db_repr", "01946_test_zstd_decompression_with_escape_sequence_at_the_end_of_buffer", "01946_test_wrong_host_name_access", + "00510_materizlized_view_and_deduplication_zookeeper", /// static UUID + "00509_extended_storage_definition_syntax_zookeeper", /// leftovers + "00083_create_merge_tree_zookeeper", /// leftovers + "01532_execute_merges_on_single_replica", /// static zk path + "01530_drop_database_atomic_sync", /// creates database "02001_add_default_database_to_system_users" ///create user ] } From 8546df13c2ccaf19f3c8337cc5d1d64f7f0ed3c7 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 5 Aug 2021 21:08:52 +0300 Subject: [PATCH 169/599] Streams -> Processors for dicts, part 2. --- src/Bridge/LibraryBridgeHelper.cpp | 19 +++--- src/Bridge/LibraryBridgeHelper.h | 12 ++-- src/DataStreams/MongoDBBlockInputStream.cpp | 13 ++-- src/DataStreams/MongoDBBlockInputStream.h | 14 ++--- .../PostgreSQLBlockInputStream.cpp | 43 +++++++++---- src/DataStreams/PostgreSQLBlockInputStream.h | 22 ++++--- src/Dictionaries/HTTPDictionarySource.cpp | 19 +++--- src/Dictionaries/HTTPDictionarySource.h | 13 ++-- src/Dictionaries/HashedDictionary.cpp | 29 +++++---- src/Dictionaries/IPAddressDictionary.cpp | 24 +++---- src/Dictionaries/IPAddressDictionary.h | 4 +- src/Dictionaries/LibraryDictionarySource.cpp | 8 +-- src/Dictionaries/LibraryDictionarySource.h | 10 +-- src/Dictionaries/MongoDBDictionarySource.cpp | 14 ++--- src/Dictionaries/MongoDBDictionarySource.h | 10 +-- src/Dictionaries/MySQLDictionarySource.cpp | 17 ++--- src/Dictionaries/MySQLDictionarySource.h | 14 ++--- src/Dictionaries/PolygonDictionary.cpp | 16 ++--- src/Dictionaries/PolygonDictionary.h | 5 +- .../PostgreSQLDictionarySource.cpp | 12 ++-- src/Dictionaries/PostgreSQLDictionarySource.h | 12 ++-- .../RangeDictionaryBlockInputStream.h | 63 +++++++++++++------ src/Dictionaries/RangeHashedDictionary.cpp | 49 ++++++++------- src/Dictionaries/RangeHashedDictionary.h | 8 +-- src/Dictionaries/RedisBlockInputStream.cpp | 14 +++-- src/Dictionaries/RedisBlockInputStream.h | 14 ++--- src/Dictionaries/RedisDictionarySource.cpp | 16 ++--- src/Dictionaries/RedisDictionarySource.h | 10 +-- src/Dictionaries/XDBCDictionarySource.cpp | 61 ++++-------------- src/Dictionaries/XDBCDictionarySource.h | 12 ++-- src/Formats/MySQLBlockInputStream.cpp | 42 ++++++++----- src/Formats/MySQLBlockInputStream.h | 23 +++---- 32 files changed, 336 insertions(+), 306 deletions(-) diff --git a/src/Bridge/LibraryBridgeHelper.cpp b/src/Bridge/LibraryBridgeHelper.cpp index b13be0aba29..81484854367 100644 --- a/src/Bridge/LibraryBridgeHelper.cpp +++ b/src/Bridge/LibraryBridgeHelper.cpp @@ -1,11 +1,10 @@ #include "LibraryBridgeHelper.h" #include -#include -#include #include #include -#include +#include +#include #include #include #include @@ -117,7 +116,7 @@ bool LibraryBridgeHelper::supportsSelectiveLoad() } -BlockInputStreamPtr LibraryBridgeHelper::loadAll() +Pipe LibraryBridgeHelper::loadAll() { startBridgeSync(); auto uri = createRequestURI(LOAD_ALL_METHOD); @@ -125,7 +124,7 @@ BlockInputStreamPtr LibraryBridgeHelper::loadAll() } -BlockInputStreamPtr LibraryBridgeHelper::loadIds(const std::string ids_string) +Pipe LibraryBridgeHelper::loadIds(const std::string ids_string) { startBridgeSync(); auto uri = createRequestURI(LOAD_IDS_METHOD); @@ -133,7 +132,7 @@ BlockInputStreamPtr LibraryBridgeHelper::loadIds(const std::string ids_string) } -BlockInputStreamPtr LibraryBridgeHelper::loadKeys(const Block & requested_block) +Pipe LibraryBridgeHelper::loadKeys(const Block & requested_block) { startBridgeSync(); auto uri = createRequestURI(LOAD_KEYS_METHOD); @@ -163,7 +162,7 @@ bool LibraryBridgeHelper::executeRequest(const Poco::URI & uri, ReadWriteBufferF } -BlockInputStreamPtr LibraryBridgeHelper::loadBase(const Poco::URI & uri, ReadWriteBufferFromHTTP::OutStreamCallback out_stream_callback) +Pipe LibraryBridgeHelper::loadBase(const Poco::URI & uri, ReadWriteBufferFromHTTP::OutStreamCallback out_stream_callback) { auto read_buf_ptr = std::make_unique( uri, @@ -176,7 +175,9 @@ BlockInputStreamPtr LibraryBridgeHelper::loadBase(const Poco::URI & uri, ReadWri ReadWriteBufferFromHTTP::HTTPHeaderEntries{}); auto input_stream = getContext()->getInputFormat(LibraryBridgeHelper::DEFAULT_FORMAT, *read_buf_ptr, sample_block, DEFAULT_BLOCK_SIZE); - return std::make_shared>(input_stream, std::move(read_buf_ptr)); + auto source = FormatFactory::instance().getInput(LibraryBridgeHelper::DEFAULT_FORMAT, *read_buf_ptr, sample_block, getContext(), DEFAULT_BLOCK_SIZE); + source->addBuffer(std::move(read_buf_ptr)); + return Pipe(std::move(source)); } -} +} \ No newline at end of file diff --git a/src/Bridge/LibraryBridgeHelper.h b/src/Bridge/LibraryBridgeHelper.h index 12fe0c33363..18a9c18f2ec 100644 --- a/src/Bridge/LibraryBridgeHelper.h +++ b/src/Bridge/LibraryBridgeHelper.h @@ -11,6 +11,8 @@ namespace DB { +class Pipe; + class LibraryBridgeHelper : public IBridgeHelper { @@ -29,13 +31,13 @@ public: bool supportsSelectiveLoad(); - BlockInputStreamPtr loadAll(); + Pipe loadAll(); - BlockInputStreamPtr loadIds(std::string ids_string); + Pipe loadIds(std::string ids_string); - BlockInputStreamPtr loadKeys(const Block & requested_block); + Pipe loadKeys(const Block & requested_block); - BlockInputStreamPtr loadBase(const Poco::URI & uri, ReadWriteBufferFromHTTP::OutStreamCallback out_stream_callback = {}); + Pipe loadBase(const Poco::URI & uri, ReadWriteBufferFromHTTP::OutStreamCallback out_stream_callback = {}); bool executeRequest(const Poco::URI & uri, ReadWriteBufferFromHTTP::OutStreamCallback out_stream_callback = {}); @@ -83,4 +85,4 @@ private: size_t bridge_port; }; -} +} \ No newline at end of file diff --git a/src/DataStreams/MongoDBBlockInputStream.cpp b/src/DataStreams/MongoDBBlockInputStream.cpp index d583cb0d5b4..1bc77532eb9 100644 --- a/src/DataStreams/MongoDBBlockInputStream.cpp +++ b/src/DataStreams/MongoDBBlockInputStream.cpp @@ -150,13 +150,14 @@ std::unique_ptr createCursor(const std::string & database return cursor; } -MongoDBBlockInputStream::MongoDBBlockInputStream( +MongoDBSource::MongoDBSource( std::shared_ptr & connection_, std::unique_ptr cursor_, const Block & sample_block, UInt64 max_block_size_, bool strict_check_names_) - : connection(connection_) + : SourceWithProgress(sample_block.cloneEmpty()) + , connection(connection_) , cursor{std::move(cursor_)} , max_block_size{max_block_size_} , strict_check_names{strict_check_names_} @@ -165,7 +166,7 @@ MongoDBBlockInputStream::MongoDBBlockInputStream( } -MongoDBBlockInputStream::~MongoDBBlockInputStream() = default; +MongoDBSource::~MongoDBSource() = default; namespace @@ -307,7 +308,7 @@ namespace } -Block MongoDBBlockInputStream::readImpl() +Chunk MongoDBSource::generate() { if (all_read) return {}; @@ -362,7 +363,7 @@ Block MongoDBBlockInputStream::readImpl() if (num_rows == 0) return {}; - return description.sample_block.cloneWithColumns(std::move(columns)); + return Chunk(std::move(columns), num_rows); } -} +} \ No newline at end of file diff --git a/src/DataStreams/MongoDBBlockInputStream.h b/src/DataStreams/MongoDBBlockInputStream.h index e1cbf9d570c..9100ba3240b 100644 --- a/src/DataStreams/MongoDBBlockInputStream.h +++ b/src/DataStreams/MongoDBBlockInputStream.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include #include @@ -22,24 +22,22 @@ void authenticate(Poco::MongoDB::Connection & connection, const std::string & da std::unique_ptr createCursor(const std::string & database, const std::string & collection, const Block & sample_block_to_select); /// Converts MongoDB Cursor to a stream of Blocks -class MongoDBBlockInputStream final : public IBlockInputStream +class MongoDBSource final : public SourceWithProgress { public: - MongoDBBlockInputStream( + MongoDBSource( std::shared_ptr & connection_, std::unique_ptr cursor_, const Block & sample_block, UInt64 max_block_size_, bool strict_check_names_ = false); - ~MongoDBBlockInputStream() override; + ~MongoDBSource() override; String getName() const override { return "MongoDB"; } - Block getHeader() const override { return description.sample_block.cloneEmpty(); } - private: - Block readImpl() override; + Chunk generate() override; std::shared_ptr connection; std::unique_ptr cursor; @@ -52,4 +50,4 @@ private: bool strict_check_names; }; -} +} \ No newline at end of file diff --git a/src/DataStreams/PostgreSQLBlockInputStream.cpp b/src/DataStreams/PostgreSQLBlockInputStream.cpp index 3956ce16afa..477ede47258 100644 --- a/src/DataStreams/PostgreSQLBlockInputStream.cpp +++ b/src/DataStreams/PostgreSQLBlockInputStream.cpp @@ -24,12 +24,13 @@ namespace DB template -PostgreSQLBlockInputStream::PostgreSQLBlockInputStream( +PostgreSQLSource::PostgreSQLSource( postgres::ConnectionHolderPtr connection_holder_, const std::string & query_str_, const Block & sample_block, const UInt64 max_block_size_) - : query_str(query_str_) + : SourceWithProgress(sample_block.cloneEmpty()) + , query_str(query_str_) , max_block_size(max_block_size_) , connection_holder(std::move(connection_holder_)) { @@ -38,13 +39,14 @@ PostgreSQLBlockInputStream::PostgreSQLBlockInputStream( template -PostgreSQLBlockInputStream::PostgreSQLBlockInputStream( +PostgreSQLSource::PostgreSQLSource( std::shared_ptr tx_, const std::string & query_str_, const Block & sample_block, const UInt64 max_block_size_, bool auto_commit_) - : query_str(query_str_) + : SourceWithProgress(sample_block.cloneEmpty()) + , query_str(query_str_) , tx(std::move(tx_)) , max_block_size(max_block_size_) , auto_commit(auto_commit_) @@ -54,7 +56,7 @@ PostgreSQLBlockInputStream::PostgreSQLBlockInputStream( template -void PostgreSQLBlockInputStream::init(const Block & sample_block) +void PostgreSQLSource::init(const Block & sample_block) { description.init(sample_block); @@ -69,19 +71,34 @@ void PostgreSQLBlockInputStream::init(const Block & sample_block) template -void PostgreSQLBlockInputStream::readPrefix() +void PostgreSQLSource::onStart() { tx = std::make_shared(connection_holder->get()); stream = std::make_unique(*tx, pqxx::from_query, std::string_view(query_str)); } +template +IProcessor::Status PostgreSQLSource::prepare() +{ + if (!started) + { + onStart(); + started = true; + } + + auto status = SourceWithProgress::prepare(); + if (status == Status::Finished) + onFinish(); + + return status; +} template -Block PostgreSQLBlockInputStream::readImpl() +Chunk PostgreSQLSource::generate() { /// Check if pqxx::stream_from is finished if (!stream || !(*stream)) - return Block(); + return {}; MutableColumns columns = description.sample_block.cloneEmptyColumns(); size_t num_rows = 0; @@ -129,12 +146,12 @@ Block PostgreSQLBlockInputStream::readImpl() break; } - return description.sample_block.cloneWithColumns(std::move(columns)); + return Chunk(std::move(columns), num_rows); } template -void PostgreSQLBlockInputStream::readSuffix() +void PostgreSQLSource::onFinish() { if (stream) { @@ -146,11 +163,11 @@ void PostgreSQLBlockInputStream::readSuffix() } template -class PostgreSQLBlockInputStream; +class PostgreSQLSource; template -class PostgreSQLBlockInputStream; +class PostgreSQLSource; } -#endif +#endif \ No newline at end of file diff --git a/src/DataStreams/PostgreSQLBlockInputStream.h b/src/DataStreams/PostgreSQLBlockInputStream.h index 31bff0789bb..8b2ea3b62ef 100644 --- a/src/DataStreams/PostgreSQLBlockInputStream.h +++ b/src/DataStreams/PostgreSQLBlockInputStream.h @@ -6,7 +6,7 @@ #if USE_LIBPQXX #include -#include +#include #include #include #include @@ -18,23 +18,20 @@ namespace DB { template -class PostgreSQLBlockInputStream : public IBlockInputStream +class PostgreSQLSource : public SourceWithProgress { public: - PostgreSQLBlockInputStream( + PostgreSQLSource( postgres::ConnectionHolderPtr connection_holder_, const String & query_str_, const Block & sample_block, const UInt64 max_block_size_); String getName() const override { return "PostgreSQL"; } - Block getHeader() const override { return description.sample_block.cloneEmpty(); } - - void readPrefix() override; protected: - PostgreSQLBlockInputStream( + PostgreSQLSource( std::shared_ptr tx_, const std::string & query_str_, const Block & sample_block, @@ -45,9 +42,12 @@ protected: std::shared_ptr tx; std::unique_ptr stream; + Status prepare() override; + private: - Block readImpl() override; - void readSuffix() override; + void onStart(); + Chunk generate() override; + void onFinish(); void init(const Block & sample_block); @@ -55,6 +55,8 @@ private: bool auto_commit = true; ExternalResultDescription description; + bool started = false; + postgres::ConnectionHolderPtr connection_holder; std::unordered_map array_info; @@ -83,4 +85,4 @@ public: } -#endif +#endif \ No newline at end of file diff --git a/src/Dictionaries/HTTPDictionarySource.cpp b/src/Dictionaries/HTTPDictionarySource.cpp index ea26e9b7a2a..88aedd51395 100644 --- a/src/Dictionaries/HTTPDictionarySource.cpp +++ b/src/Dictionaries/HTTPDictionarySource.cpp @@ -8,6 +8,8 @@ #include #include #include +#include +#include #include #include #include @@ -62,14 +64,15 @@ HTTPDictionarySource::HTTPDictionarySource(const HTTPDictionarySource & other) credentials.setPassword(other.credentials.getPassword()); } -BlockInputStreamPtr HTTPDictionarySource::createWrappedBuffer(std::unique_ptr http_buffer_ptr) +Pipe HTTPDictionarySource::createWrappedBuffer(std::unique_ptr http_buffer_ptr) { Poco::URI uri(configuration.url); String http_request_compression_method_str = http_buffer_ptr->getCompressionMethod(); auto in_ptr_wrapped = wrapReadBufferWithCompressionMethod(std::move(http_buffer_ptr), chooseCompressionMethod(uri.getPath(), http_request_compression_method_str)); - auto input_stream = context->getInputFormat(configuration.format, *in_ptr_wrapped, sample_block, max_block_size); - return std::make_shared>(input_stream, std::move(in_ptr_wrapped)); + auto source = FormatFactory::instance().getInput(configuration.format, *in_ptr_wrapped, sample_block, context, max_block_size); + source->addBuffer(std::move(in_ptr_wrapped)); + return Pipe(std::move(source)); } void HTTPDictionarySource::getUpdateFieldAndDate(Poco::URI & uri) @@ -89,7 +92,7 @@ void HTTPDictionarySource::getUpdateFieldAndDate(Poco::URI & uri) } } -BlockInputStreamPtr HTTPDictionarySource::loadAll() +Pipe HTTPDictionarySource::loadAll() { LOG_TRACE(log, "loadAll {}", toString()); Poco::URI uri(configuration.url); @@ -106,7 +109,7 @@ BlockInputStreamPtr HTTPDictionarySource::loadAll() return createWrappedBuffer(std::move(in_ptr)); } -BlockInputStreamPtr HTTPDictionarySource::loadUpdatedAll() +Pipe HTTPDictionarySource::loadUpdatedAll() { Poco::URI uri(configuration.url); getUpdateFieldAndDate(uri); @@ -124,7 +127,7 @@ BlockInputStreamPtr HTTPDictionarySource::loadUpdatedAll() return createWrappedBuffer(std::move(in_ptr)); } -BlockInputStreamPtr HTTPDictionarySource::loadIds(const std::vector & ids) +Pipe HTTPDictionarySource::loadIds(const std::vector & ids) { LOG_TRACE(log, "loadIds {} size = {}", toString(), ids.size()); @@ -151,7 +154,7 @@ BlockInputStreamPtr HTTPDictionarySource::loadIds(const std::vector & id return createWrappedBuffer(std::move(in_ptr)); } -BlockInputStreamPtr HTTPDictionarySource::loadKeys(const Columns & key_columns, const std::vector & requested_rows) +Pipe HTTPDictionarySource::loadKeys(const Columns & key_columns, const std::vector & requested_rows) { LOG_TRACE(log, "loadKeys {} size = {}", toString(), requested_rows.size()); @@ -260,4 +263,4 @@ void registerDictionarySourceHTTP(DictionarySourceFactory & factory) factory.registerSource("http", create_table_source); } -} +} \ No newline at end of file diff --git a/src/Dictionaries/HTTPDictionarySource.h b/src/Dictionaries/HTTPDictionarySource.h index 638fcc876ed..1c7d67d3c23 100644 --- a/src/Dictionaries/HTTPDictionarySource.h +++ b/src/Dictionaries/HTTPDictionarySource.h @@ -43,13 +43,13 @@ public: HTTPDictionarySource(const HTTPDictionarySource & other); HTTPDictionarySource & operator=(const HTTPDictionarySource &) = delete; - BlockInputStreamPtr loadAll() override; + Pipe loadAll() override; - BlockInputStreamPtr loadUpdatedAll() override; + Pipe loadUpdatedAll() override; - BlockInputStreamPtr loadIds(const std::vector & ids) override; + Pipe loadIds(const std::vector & ids) override; - BlockInputStreamPtr loadKeys(const Columns & key_columns, const std::vector & requested_rows) override; + Pipe loadKeys(const Columns & key_columns, const std::vector & requested_rows) override; bool isModified() const override; @@ -65,7 +65,7 @@ private: void getUpdateFieldAndDate(Poco::URI & uri); // wrap buffer using encoding from made request - BlockInputStreamPtr createWrappedBuffer(std::unique_ptr http_buffer); + Pipe createWrappedBuffer(std::unique_ptr http_buffer); Poco::Logger * log; @@ -80,5 +80,4 @@ private: ConnectionTimeouts timeouts; }; -} - +} \ No newline at end of file diff --git a/src/Dictionaries/HashedDictionary.cpp b/src/Dictionaries/HashedDictionary.cpp index 33e06de23bf..22a7ec87b07 100644 --- a/src/Dictionaries/HashedDictionary.cpp +++ b/src/Dictionaries/HashedDictionary.cpp @@ -367,10 +367,12 @@ void HashedDictionary::updateData() if (!update_field_loaded_block || update_field_loaded_block->rows() == 0) { - auto stream = source_ptr->loadUpdatedAll(); - stream->readPrefix(); + QueryPipeline pipeline; + pipeline.init(source_ptr->loadUpdatedAll()); - while (const auto block = stream->read()) + PullingPipelineExecutor executor(pipeline); + Block block; + while (executor.pull(block)) { /// We are using this to keep saved data if input stream consists of multiple blocks if (!update_field_loaded_block) @@ -383,15 +385,14 @@ void HashedDictionary::updateData() saved_column->insertRangeFrom(update_column, 0, update_column.size()); } } - stream->readSuffix(); } else { - auto stream = source_ptr->loadUpdatedAll(); - mergeBlockWithStream( + auto pipe = source_ptr->loadUpdatedAll(); + mergeBlockWithPipe( dict_struct.getKeysSize(), *update_field_loaded_block, - stream); + std::move(pipe)); } if (update_field_loaded_block) @@ -560,15 +561,15 @@ void HashedDictionary::loadData() { std::atomic new_size = 0; - BlockInputStreamPtr stream; + QueryPipeline pipeline; if (configuration.preallocate) - stream = source_ptr->loadAllWithSizeHint(&new_size); + pipeline.init(source_ptr->loadAllWithSizeHint(&new_size)); else - stream = source_ptr->loadAll(); + pipeline.init(source_ptr->loadAll()); - stream->readPrefix(); - - while (const auto block = stream->read()) + PullingPipelineExecutor executor(pipeline); + Block block; + while (executor.pull(block)) { if (configuration.preallocate && new_size) { @@ -584,8 +585,6 @@ void HashedDictionary::loadData() blockToAttributes(block); } - - stream->readSuffix(); } else updateData(); diff --git a/src/Dictionaries/IPAddressDictionary.cpp b/src/Dictionaries/IPAddressDictionary.cpp index a36aeb59f01..53912f294b2 100644 --- a/src/Dictionaries/IPAddressDictionary.cpp +++ b/src/Dictionaries/IPAddressDictionary.cpp @@ -352,14 +352,16 @@ void IPAddressDictionary::createAttributes() void IPAddressDictionary::loadData() { - auto stream = source_ptr->loadAll(); - stream->readPrefix(); + QueryPipeline pipeline; + pipeline.init(source_ptr->loadAll()); std::vector ip_records; bool has_ipv6 = false; - while (const auto block = stream->read()) + PullingPipelineExecutor executor(pipeline); + Block block; + while (executor.pull(block)) { const auto rows = block.rows(); element_count += rows; @@ -387,8 +389,6 @@ void IPAddressDictionary::loadData() } } - stream->readSuffix(); - if (access_to_key_from_attributes) { /// We format key attribute values here instead of filling with data from key_column @@ -835,7 +835,7 @@ static auto keyViewGetter() }; } -BlockInputStreamPtr IPAddressDictionary::getBlockInputStream(const Names & column_names, size_t max_block_size) const +Pipe IPAddressDictionary::read(const Names & column_names, size_t max_block_size) const { const bool is_ipv4 = std::get_if(&ip_column) != nullptr; @@ -857,13 +857,15 @@ BlockInputStreamPtr IPAddressDictionary::getBlockInputStream(const Names & colum if (is_ipv4) { auto get_view = keyViewGetter, true>(); - return std::make_shared( - shared_from_this(), max_block_size, getKeyColumns(), column_names, std::move(get_keys), std::move(get_view)); + return Pipe(std::make_shared( + DictionarySourceData(shared_from_this(), getKeyColumns(), column_names, std::move(get_keys), std::move(get_view)), + max_block_size)); } auto get_view = keyViewGetter(); - return std::make_shared( - shared_from_this(), max_block_size, getKeyColumns(), column_names, std::move(get_keys), std::move(get_view)); + return Pipe(std::make_shared( + DictionarySourceData(shared_from_this(), getKeyColumns(), column_names, std::move(get_keys), std::move(get_view)), + max_block_size)); } IPAddressDictionary::RowIdxConstIter IPAddressDictionary::ipNotFound() const @@ -968,4 +970,4 @@ void registerDictionaryTrie(DictionaryFactory & factory) factory.registerLayout("ip_trie", create_layout, true); } -} +} \ No newline at end of file diff --git a/src/Dictionaries/IPAddressDictionary.h b/src/Dictionaries/IPAddressDictionary.h index f44291cec6a..06eb480f825 100644 --- a/src/Dictionaries/IPAddressDictionary.h +++ b/src/Dictionaries/IPAddressDictionary.h @@ -78,7 +78,7 @@ public: ColumnUInt8::Ptr hasKeys(const Columns & key_columns, const DataTypes & key_types) const override; - BlockInputStreamPtr getBlockInputStream(const Names & column_names, size_t max_block_size) const override; + Pipe read(const Names & column_names, size_t max_block_size) const override; private: @@ -224,4 +224,4 @@ private: Poco::Logger * logger; }; -} +} \ No newline at end of file diff --git a/src/Dictionaries/LibraryDictionarySource.cpp b/src/Dictionaries/LibraryDictionarySource.cpp index 0b8b52a2d67..f75ca845f01 100644 --- a/src/Dictionaries/LibraryDictionarySource.cpp +++ b/src/Dictionaries/LibraryDictionarySource.cpp @@ -89,21 +89,21 @@ bool LibraryDictionarySource::supportsSelectiveLoad() const } -BlockInputStreamPtr LibraryDictionarySource::loadAll() +Pipe LibraryDictionarySource::loadAll() { LOG_TRACE(log, "loadAll {}", toString()); return bridge_helper->loadAll(); } -BlockInputStreamPtr LibraryDictionarySource::loadIds(const std::vector & ids) +Pipe LibraryDictionarySource::loadIds(const std::vector & ids) { LOG_TRACE(log, "loadIds {} size = {}", toString(), ids.size()); return bridge_helper->loadIds(getDictIdsString(ids)); } -BlockInputStreamPtr LibraryDictionarySource::loadKeys(const Columns & key_columns, const std::vector & requested_rows) +Pipe LibraryDictionarySource::loadKeys(const Columns & key_columns, const std::vector & requested_rows) { LOG_TRACE(log, "loadKeys {} size = {}", toString(), requested_rows.size()); auto block = blockForKeys(dict_struct, key_columns, requested_rows); @@ -183,4 +183,4 @@ void registerDictionarySourceLibrary(DictionarySourceFactory & factory) } -} +} \ No newline at end of file diff --git a/src/Dictionaries/LibraryDictionarySource.h b/src/Dictionaries/LibraryDictionarySource.h index 88e133666e6..11b9245f769 100644 --- a/src/Dictionaries/LibraryDictionarySource.h +++ b/src/Dictionaries/LibraryDictionarySource.h @@ -47,16 +47,16 @@ public: ~LibraryDictionarySource() override; - BlockInputStreamPtr loadAll() override; + Pipe loadAll() override; - BlockInputStreamPtr loadUpdatedAll() override + Pipe loadUpdatedAll() override { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method loadUpdatedAll is unsupported for LibraryDictionarySource"); } - BlockInputStreamPtr loadIds(const std::vector & ids) override; + Pipe loadIds(const std::vector & ids) override; - BlockInputStreamPtr loadKeys(const Columns & key_columns, const std::vector & requested_rows) override; + Pipe loadKeys(const Columns & key_columns, const std::vector & requested_rows) override; bool isModified() const override; @@ -92,4 +92,4 @@ private: ExternalResultDescription description; }; -} +} \ No newline at end of file diff --git a/src/Dictionaries/MongoDBDictionarySource.cpp b/src/Dictionaries/MongoDBDictionarySource.cpp index 4a5207dc5d8..3afce312de1 100644 --- a/src/Dictionaries/MongoDBDictionarySource.cpp +++ b/src/Dictionaries/MongoDBDictionarySource.cpp @@ -142,12 +142,12 @@ MongoDBDictionarySource::MongoDBDictionarySource(const MongoDBDictionarySource & MongoDBDictionarySource::~MongoDBDictionarySource() = default; -BlockInputStreamPtr MongoDBDictionarySource::loadAll() +Pipe MongoDBDictionarySource::loadAll() { - return std::make_shared(connection, createCursor(db, collection, sample_block), sample_block, max_block_size); + return Pipe(std::make_shared(connection, createCursor(db, collection, sample_block), sample_block, max_block_size)); } -BlockInputStreamPtr MongoDBDictionarySource::loadIds(const std::vector & ids) +Pipe MongoDBDictionarySource::loadIds(const std::vector & ids) { if (!dict_struct.id) throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "'id' is required for selective loading"); @@ -164,11 +164,11 @@ BlockInputStreamPtr MongoDBDictionarySource::loadIds(const std::vector & cursor->query().selector().addNewDocument(dict_struct.id->name).add("$in", ids_array); - return std::make_shared(connection, std::move(cursor), sample_block, max_block_size); + return Pipe(std::make_shared(connection, std::move(cursor), sample_block, max_block_size)); } -BlockInputStreamPtr MongoDBDictionarySource::loadKeys(const Columns & key_columns, const std::vector & requested_rows) +Pipe MongoDBDictionarySource::loadKeys(const Columns & key_columns, const std::vector & requested_rows) { if (!dict_struct.key) throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "'key' is required for selective loading"); @@ -230,7 +230,7 @@ BlockInputStreamPtr MongoDBDictionarySource::loadKeys(const Columns & key_column /// If more than one key we should use $or cursor->query().selector().add("$or", keys_array); - return std::make_shared(connection, std::move(cursor), sample_block, max_block_size); + return Pipe(std::make_shared(connection, std::move(cursor), sample_block, max_block_size)); } std::string MongoDBDictionarySource::toString() const @@ -238,4 +238,4 @@ std::string MongoDBDictionarySource::toString() const return "MongoDB: " + db + '.' + collection + ',' + (user.empty() ? " " : " " + user + '@') + host + ':' + DB::toString(port); } -} +} \ No newline at end of file diff --git a/src/Dictionaries/MongoDBDictionarySource.h b/src/Dictionaries/MongoDBDictionarySource.h index fef5749190f..811bec28fe7 100644 --- a/src/Dictionaries/MongoDBDictionarySource.h +++ b/src/Dictionaries/MongoDBDictionarySource.h @@ -46,18 +46,18 @@ public: ~MongoDBDictionarySource() override; - BlockInputStreamPtr loadAll() override; + Pipe loadAll() override; - BlockInputStreamPtr loadUpdatedAll() override + Pipe loadUpdatedAll() override { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method loadUpdatedAll is unsupported for MongoDBDictionarySource"); } bool supportsSelectiveLoad() const override { return true; } - BlockInputStreamPtr loadIds(const std::vector & ids) override; + Pipe loadIds(const std::vector & ids) override; - BlockInputStreamPtr loadKeys(const Columns & key_columns, const std::vector & requested_rows) override; + Pipe loadKeys(const Columns & key_columns, const std::vector & requested_rows) override; /// @todo: for MongoDB, modification date can somehow be determined from the `_id` object field bool isModified() const override { return true; } @@ -84,4 +84,4 @@ private: std::shared_ptr connection; }; -} +} \ No newline at end of file diff --git a/src/Dictionaries/MySQLDictionarySource.cpp b/src/Dictionaries/MySQLDictionarySource.cpp index 77ad1e4bf29..6462158e882 100644 --- a/src/Dictionaries/MySQLDictionarySource.cpp +++ b/src/Dictionaries/MySQLDictionarySource.cpp @@ -11,6 +11,7 @@ #include "registerDictionaries.h" #include #include +#include namespace DB { @@ -131,13 +132,13 @@ std::string MySQLDictionarySource::getUpdateFieldAndDate() } } -BlockInputStreamPtr MySQLDictionarySource::loadFromQuery(const String & query) +Pipe MySQLDictionarySource::loadFromQuery(const String & query) { - return std::make_shared( - pool, query, sample_block, settings); + return Pipe(std::make_shared( + pool, query, sample_block, settings)); } -BlockInputStreamPtr MySQLDictionarySource::loadAll() +Pipe MySQLDictionarySource::loadAll() { auto connection = pool->get(); last_modification = getLastModification(connection, false); @@ -146,7 +147,7 @@ BlockInputStreamPtr MySQLDictionarySource::loadAll() return loadFromQuery(load_all_query); } -BlockInputStreamPtr MySQLDictionarySource::loadUpdatedAll() +Pipe MySQLDictionarySource::loadUpdatedAll() { auto connection = pool->get(); last_modification = getLastModification(connection, false); @@ -156,14 +157,14 @@ BlockInputStreamPtr MySQLDictionarySource::loadUpdatedAll() return loadFromQuery(load_update_query); } -BlockInputStreamPtr MySQLDictionarySource::loadIds(const std::vector & ids) +Pipe MySQLDictionarySource::loadIds(const std::vector & ids) { /// We do not log in here and do not update the modification time, as the request can be large, and often called. const auto query = query_builder.composeLoadIdsQuery(ids); return loadFromQuery(query); } -BlockInputStreamPtr MySQLDictionarySource::loadKeys(const Columns & key_columns, const std::vector & requested_rows) +Pipe MySQLDictionarySource::loadKeys(const Columns & key_columns, const std::vector & requested_rows) { /// We do not log in here and do not update the modification time, as the request can be large, and often called. const auto query = query_builder.composeLoadKeysQuery(key_columns, requested_rows, ExternalQueryBuilder::AND_OR_CHAIN); @@ -290,4 +291,4 @@ std::string MySQLDictionarySource::doInvalidateQuery(const std::string & request } -#endif +#endif \ No newline at end of file diff --git a/src/Dictionaries/MySQLDictionarySource.h b/src/Dictionaries/MySQLDictionarySource.h index afe0f0cfcc7..f3dbb89757d 100644 --- a/src/Dictionaries/MySQLDictionarySource.h +++ b/src/Dictionaries/MySQLDictionarySource.h @@ -6,7 +6,7 @@ # include "config_core.h" #endif -#if USE_MYSQL +//#if USE_MYSQL # include # include # include "DictionaryStructure.h" @@ -53,13 +53,13 @@ public: MySQLDictionarySource(const MySQLDictionarySource & other); MySQLDictionarySource & operator=(const MySQLDictionarySource &) = delete; - BlockInputStreamPtr loadAll() override; + Pipe loadAll() override; - BlockInputStreamPtr loadUpdatedAll() override; + Pipe loadUpdatedAll() override; - BlockInputStreamPtr loadIds(const std::vector & ids) override; + Pipe loadIds(const std::vector & ids) override; - BlockInputStreamPtr loadKeys(const Columns & key_columns, const std::vector & requested_rows) override; + Pipe loadKeys(const Columns & key_columns, const std::vector & requested_rows) override; bool isModified() const override; @@ -72,7 +72,7 @@ public: std::string toString() const override; private: - BlockInputStreamPtr loadFromQuery(const String & query); + Pipe loadFromQuery(const String & query); std::string getUpdateFieldAndDate(); @@ -99,4 +99,4 @@ private: } -#endif +#endif \ No newline at end of file diff --git a/src/Dictionaries/PolygonDictionary.cpp b/src/Dictionaries/PolygonDictionary.cpp index 73810d4b057..f1f337e8fcc 100644 --- a/src/Dictionaries/PolygonDictionary.cpp +++ b/src/Dictionaries/PolygonDictionary.cpp @@ -119,7 +119,7 @@ ColumnPtr IPolygonDictionary::getColumn( return result; } -BlockInputStreamPtr IPolygonDictionary::getBlockInputStream(const Names &, size_t) const +Pipe IPolygonDictionary::read(const Names &, size_t) const { // TODO: In order for this to work one would first have to support retrieving arrays from dictionaries. // I believe this is a separate task done by some other people. @@ -165,12 +165,13 @@ void IPolygonDictionary::blockToAttributes(const DB::Block & block) void IPolygonDictionary::loadData() { - auto stream = source_ptr->loadAll(); - stream->readPrefix(); - while (const auto block = stream->read()) - blockToAttributes(block); - stream->readSuffix(); + QueryPipeline pipeline; + pipeline.init(source_ptr->loadAll()); + PullingPipelineExecutor executor(pipeline); + Block block; + while (executor.pull(block)) + blockToAttributes(block); /// Correct and sort polygons by area and update polygon_index_to_attribute_value_index after sort PaddedPODArray areas; @@ -515,5 +516,4 @@ void IPolygonDictionary::extractPolygons(const ColumnPtr & column) } } -} - +} \ No newline at end of file diff --git a/src/Dictionaries/PolygonDictionary.h b/src/Dictionaries/PolygonDictionary.h index 14492df6469..c119b5a5cfe 100644 --- a/src/Dictionaries/PolygonDictionary.h +++ b/src/Dictionaries/PolygonDictionary.h @@ -97,7 +97,7 @@ public: ColumnUInt8::Ptr hasKeys(const Columns & key_columns, const DataTypes & key_types) const override; - BlockInputStreamPtr getBlockInputStream(const Names & column_names, size_t max_block_size) const override; + Pipe read(const Names & column_names, size_t max_block_size) const override; /** Single coordinate type. */ using Coord = Float32; @@ -166,5 +166,4 @@ private: static std::vector extractPoints(const Columns &key_columns); }; -} - +} \ No newline at end of file diff --git a/src/Dictionaries/PostgreSQLDictionarySource.cpp b/src/Dictionaries/PostgreSQLDictionarySource.cpp index f1ca3d4855b..1f256f3957f 100644 --- a/src/Dictionaries/PostgreSQLDictionarySource.cpp +++ b/src/Dictionaries/PostgreSQLDictionarySource.cpp @@ -77,35 +77,35 @@ PostgreSQLDictionarySource::PostgreSQLDictionarySource(const PostgreSQLDictionar } -BlockInputStreamPtr PostgreSQLDictionarySource::loadAll() +Pipe PostgreSQLDictionarySource::loadAll() { LOG_TRACE(log, load_all_query); return loadBase(load_all_query); } -BlockInputStreamPtr PostgreSQLDictionarySource::loadUpdatedAll() +Pipe PostgreSQLDictionarySource::loadUpdatedAll() { auto load_update_query = getUpdateFieldAndDate(); LOG_TRACE(log, load_update_query); return loadBase(load_update_query); } -BlockInputStreamPtr PostgreSQLDictionarySource::loadIds(const std::vector & ids) +Pipe PostgreSQLDictionarySource::loadIds(const std::vector & ids) { const auto query = query_builder.composeLoadIdsQuery(ids); return loadBase(query); } -BlockInputStreamPtr PostgreSQLDictionarySource::loadKeys(const Columns & key_columns, const std::vector & requested_rows) +Pipe PostgreSQLDictionarySource::loadKeys(const Columns & key_columns, const std::vector & requested_rows) { const auto query = query_builder.composeLoadKeysQuery(key_columns, requested_rows, ExternalQueryBuilder::AND_OR_CHAIN); return loadBase(query); } -BlockInputStreamPtr PostgreSQLDictionarySource::loadBase(const String & query) +Pipe PostgreSQLDictionarySource::loadBase(const String & query) { return std::make_shared>(pool->get(), query, sample_block, max_block_size); } @@ -220,4 +220,4 @@ void registerDictionarySourcePostgreSQL(DictionarySourceFactory & factory) factory.registerSource("postgresql", create_table_source); } -} +} \ No newline at end of file diff --git a/src/Dictionaries/PostgreSQLDictionarySource.h b/src/Dictionaries/PostgreSQLDictionarySource.h index 789f0d13966..e5a72f0f720 100644 --- a/src/Dictionaries/PostgreSQLDictionarySource.h +++ b/src/Dictionaries/PostgreSQLDictionarySource.h @@ -42,10 +42,10 @@ public: PostgreSQLDictionarySource(const PostgreSQLDictionarySource & other); PostgreSQLDictionarySource & operator=(const PostgreSQLDictionarySource &) = delete; - BlockInputStreamPtr loadAll() override; - BlockInputStreamPtr loadUpdatedAll() override; - BlockInputStreamPtr loadIds(const std::vector & ids) override; - BlockInputStreamPtr loadKeys(const Columns & key_columns, const std::vector & requested_rows) override; + Pipe loadAll() override; + Pipe loadUpdatedAll() override; + Pipe loadIds(const std::vector & ids) override; + Pipe loadKeys(const Columns & key_columns, const std::vector & requested_rows) override; bool isModified() const override; bool supportsSelectiveLoad() const override; @@ -57,7 +57,7 @@ public: private: String getUpdateFieldAndDate(); String doInvalidateQuery(const std::string & request) const; - BlockInputStreamPtr loadBase(const String & query); + Pipe loadBase(const String & query); const DictionaryStructure dict_struct; const Configuration configuration; @@ -72,4 +72,4 @@ private: }; } -#endif +#endif \ No newline at end of file diff --git a/src/Dictionaries/RangeDictionaryBlockInputStream.h b/src/Dictionaries/RangeDictionaryBlockInputStream.h index 7d40531cfa5..1164683521c 100644 --- a/src/Dictionaries/RangeDictionaryBlockInputStream.h +++ b/src/Dictionaries/RangeDictionaryBlockInputStream.h @@ -13,28 +13,22 @@ namespace DB { -/* - * BlockInputStream implementation for external dictionaries - * read() returns single block consisting of the in-memory contents of the dictionaries - */ + template -class RangeDictionaryBlockInputStream : public DictionaryBlockInputStreamBase +class RangeDictionarySourceData { public: using Key = UInt64; - RangeDictionaryBlockInputStream( + RangeDictionarySourceData( std::shared_ptr dictionary, - size_t max_block_size, const Names & column_names, PaddedPODArray && ids_to_fill, PaddedPODArray && start_dates, PaddedPODArray && end_dates); - String getName() const override { return "RangeDictionary"; } - -protected: - Block getBlock(size_t start, size_t length) const override; + Block getBlock(size_t start, size_t length) const; + size_t getNumRows() const { return ids.size(); } private: template @@ -58,15 +52,13 @@ private: template -RangeDictionaryBlockInputStream::RangeDictionaryBlockInputStream( +RangeDictionarySourceData::RangeDictionarySourceData( std::shared_ptr dictionary_, - size_t max_block_size_, const Names & column_names_, PaddedPODArray && ids_, PaddedPODArray && block_start_dates, PaddedPODArray && block_end_dates) - : DictionaryBlockInputStreamBase(ids_.size(), max_block_size_) - , dictionary(dictionary_) + : dictionary(dictionary_) , column_names(column_names_.begin(), column_names_.end()) , ids(std::move(ids_)) , start_dates(std::move(block_start_dates)) @@ -75,7 +67,7 @@ RangeDictionaryBlockInputStream::RangeDictionaryBlockInputStream( } template -Block RangeDictionaryBlockInputStream::getBlock(size_t start, size_t length) const +Block RangeDictionarySourceData::getBlock(size_t start, size_t length) const { PaddedPODArray block_ids; PaddedPODArray block_start_dates; @@ -96,7 +88,7 @@ Block RangeDictionaryBlockInputStream::getBlock(size_t start, size_t template template -ColumnPtr RangeDictionaryBlockInputStream::getColumnFromPODArray(const PaddedPODArray & array) const +ColumnPtr RangeDictionarySourceData::getColumnFromPODArray(const PaddedPODArray & array) const { auto column_vector = ColumnVector::create(); column_vector->getData().reserve(array.size()); @@ -106,7 +98,7 @@ ColumnPtr RangeDictionaryBlockInputStream::getColumnFromPODArray(cons } template -PaddedPODArray RangeDictionaryBlockInputStream::makeDateKey( +PaddedPODArray RangeDictionarySourceData::makeDateKey( const PaddedPODArray & block_start_dates, const PaddedPODArray & block_end_dates) const { PaddedPODArray key(block_start_dates.size()); @@ -123,7 +115,7 @@ PaddedPODArray RangeDictionaryBlockInputStream::makeDateKey( template -Block RangeDictionaryBlockInputStream::fillBlock( +Block RangeDictionarySourceData::fillBlock( const PaddedPODArray & ids_to_fill, const PaddedPODArray & block_start_dates, const PaddedPODArray & block_end_dates) const @@ -170,4 +162,37 @@ Block RangeDictionaryBlockInputStream::fillBlock( return Block(columns); } +/* + * BlockInputStream implementation for external dictionaries + * read() returns single block consisting of the in-memory contents of the dictionaries + */ +template +class RangeDictionarySource : public DictionarySourceBase +{ +public: + using Key = UInt64; + + RangeDictionarySource(RangeDictionarySourceData data_, size_t max_block_size); + + String getName() const override { return "RangeDictionarySource"; } + +protected: + Block getBlock(size_t start, size_t length) const override; + + RangeDictionarySourceData data; +}; + +template +RangeDictionarySource::RangeDictionarySource(RangeDictionarySourceData data_, size_t max_block_size) + : DictionarySourceBase(data_.getBlock(0, 0), data_.getNumRows(), max_block_size) + , data(std::move(data_)) +{ } + +template +Block RangeDictionarySource::getBlock(size_t start, size_t length) const +{ + return data.getBlock(start, length); +} + +} \ No newline at end of file diff --git a/src/Dictionaries/RangeHashedDictionary.cpp b/src/Dictionaries/RangeHashedDictionary.cpp index eadfc03ef33..bf930a7a526 100644 --- a/src/Dictionaries/RangeHashedDictionary.cpp +++ b/src/Dictionaries/RangeHashedDictionary.cpp @@ -298,10 +298,12 @@ void RangeHashedDictionary::createAttributes() void RangeHashedDictionary::loadData() { - auto stream = source_ptr->loadAll(); - stream->readPrefix(); + QueryPipeline pipeline; + pipeline.init(source_ptr->loadAll()); - while (const auto block = stream->read()) + PullingPipelineExecutor executor(pipeline); + Block block; + while (executor.pull(block)) { const auto & id_column = *block.safeGetByPosition(0).column; @@ -339,8 +341,6 @@ void RangeHashedDictionary::loadData() } } - stream->readSuffix(); - if (require_nonempty && 0 == element_count) throw Exception(ErrorCodes::DICTIONARY_IS_EMPTY, "{}: dictionary source is empty and 'require_nonempty' property is set."); @@ -594,29 +594,30 @@ void RangeHashedDictionary::getIdsAndDates( template -BlockInputStreamPtr RangeHashedDictionary::getBlockInputStreamImpl(const Names & column_names, size_t max_block_size) const +Pipe RangeHashedDictionary::readImpl(const Names & column_names, size_t max_block_size) const { PaddedPODArray ids; PaddedPODArray start_dates; PaddedPODArray end_dates; getIdsAndDates(ids, start_dates, end_dates); - using BlockInputStreamType = RangeDictionaryBlockInputStream; + using RangeDictionarySourceType = RangeDictionarySource; - auto stream = std::make_shared( - shared_from_this(), - max_block_size, - column_names, - std::move(ids), - std::move(start_dates), - std::move(end_dates)); + auto source = std::make_shared( + RangeDictionarySourceData( + shared_from_this(), + column_names, + std::move(ids), + std::move(start_dates), + std::move(end_dates)), + max_block_size); - return stream; + return Pipe(source); } -struct RangeHashedDictionaryCallGetBlockInputStreamImpl +struct RangeHashedDictionaryCallGetSourceImpl { - BlockInputStreamPtr stream; + Pipe pipe; const RangeHashedDictionary * dict; const Names * column_names; size_t max_block_size; @@ -625,28 +626,28 @@ struct RangeHashedDictionaryCallGetBlockInputStreamImpl void operator()() { const auto & type = dict->dict_struct.range_min->type; - if (!stream && dynamic_cast *>(type.get())) - stream = dict->getBlockInputStreamImpl(*column_names, max_block_size); + if (pipe.empty() && dynamic_cast *>(type.get())) + pipe = dict->readImpl(*column_names, max_block_size); } }; -BlockInputStreamPtr RangeHashedDictionary::getBlockInputStream(const Names & column_names, size_t max_block_size) const +Pipe RangeHashedDictionary::read(const Names & column_names, size_t max_block_size) const { using ListType = TypeList; - RangeHashedDictionaryCallGetBlockInputStreamImpl callable; + RangeHashedDictionaryCallGetSourceImpl callable; callable.dict = this; callable.column_names = &column_names; callable.max_block_size = max_block_size; ListType::forEach(callable); - if (!callable.stream) + if (callable.pipe.empty()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected range type for RangeHashed dictionary: {}", dict_struct.range_min->type->getName()); - return callable.stream; + return std::move(callable.pipe); } @@ -676,4 +677,4 @@ void registerDictionaryRangeHashed(DictionaryFactory & factory) factory.registerLayout("range_hashed", create_layout, false); } -} +} \ No newline at end of file diff --git a/src/Dictionaries/RangeHashedDictionary.h b/src/Dictionaries/RangeHashedDictionary.h index 0dd9c41fd71..b0ce1e41bad 100644 --- a/src/Dictionaries/RangeHashedDictionary.h +++ b/src/Dictionaries/RangeHashedDictionary.h @@ -75,7 +75,7 @@ public: using RangeStorageType = Int64; - BlockInputStreamPtr getBlockInputStream(const Names & column_names, size_t max_block_size) const override; + Pipe read(const Names & column_names, size_t max_block_size) const override; struct Range { @@ -178,9 +178,9 @@ private: PaddedPODArray & end_dates) const; template - BlockInputStreamPtr getBlockInputStreamImpl(const Names & column_names, size_t max_block_size) const; + Pipe readImpl(const Names & column_names, size_t max_block_size) const; - friend struct RangeHashedDictionaryCallGetBlockInputStreamImpl; + friend struct RangeHashedDictionaryCallGetSourceImpl; const DictionaryStructure dict_struct; const DictionarySourcePtr source_ptr; @@ -197,4 +197,4 @@ private: mutable std::atomic found_count{0}; }; -} +} \ No newline at end of file diff --git a/src/Dictionaries/RedisBlockInputStream.cpp b/src/Dictionaries/RedisBlockInputStream.cpp index 71d055e1d68..2d30e7fe167 100644 --- a/src/Dictionaries/RedisBlockInputStream.cpp +++ b/src/Dictionaries/RedisBlockInputStream.cpp @@ -29,18 +29,19 @@ namespace DB } - RedisBlockInputStream::RedisBlockInputStream( + RedisSource::RedisSource( const std::shared_ptr & client_, const RedisArray & keys_, const RedisStorageType & storage_type_, const DB::Block & sample_block, const size_t max_block_size_) - : client(client_), keys(keys_), storage_type(storage_type_), max_block_size{max_block_size_} + : SourceWithProgress(sample_block) + , client(client_), keys(keys_), storage_type(storage_type_), max_block_size{max_block_size_} { description.init(sample_block); } - RedisBlockInputStream::~RedisBlockInputStream() = default; + RedisSource::~RedisSource() = default; namespace @@ -121,7 +122,7 @@ namespace DB } - Block RedisBlockInputStream::readImpl() + Chunk RedisSource::generate() { if (keys.isNull() || description.sample_block.rows() == 0 || cursor >= keys.size()) all_read = true; @@ -218,6 +219,7 @@ namespace DB cursor += need_values; } - return description.sample_block.cloneWithColumns(std::move(columns)); + size_t num_rows = columns.at(0)->size(); + return Chunk(std::move(columns), num_rows); } -} +} \ No newline at end of file diff --git a/src/Dictionaries/RedisBlockInputStream.h b/src/Dictionaries/RedisBlockInputStream.h index 185cfedb09d..feef6e5423e 100644 --- a/src/Dictionaries/RedisBlockInputStream.h +++ b/src/Dictionaries/RedisBlockInputStream.h @@ -3,7 +3,7 @@ #include #include -#include +#include #include #include #include "RedisDictionarySource.h" @@ -19,27 +19,25 @@ namespace Poco namespace DB { - class RedisBlockInputStream final : public IBlockInputStream + class RedisSource final : public SourceWithProgress { public: using RedisArray = Poco::Redis::Array; using RedisBulkString = Poco::Redis::BulkString; - RedisBlockInputStream( + RedisSource( const std::shared_ptr & client_, const Poco::Redis::Array & keys_, const RedisStorageType & storage_type_, const Block & sample_block, const size_t max_block_size); - ~RedisBlockInputStream() override; + ~RedisSource() override; String getName() const override { return "Redis"; } - Block getHeader() const override { return description.sample_block.cloneEmpty(); } - private: - Block readImpl() override; + Chunk generate() override; std::shared_ptr client; Poco::Redis::Array keys; @@ -50,4 +48,4 @@ namespace DB bool all_read = false; }; -} +} \ No newline at end of file diff --git a/src/Dictionaries/RedisDictionarySource.cpp b/src/Dictionaries/RedisDictionarySource.cpp index 4e79a75c1d4..437c3fb0bf9 100644 --- a/src/Dictionaries/RedisDictionarySource.cpp +++ b/src/Dictionaries/RedisDictionarySource.cpp @@ -159,7 +159,7 @@ namespace DB __builtin_unreachable(); } - BlockInputStreamPtr RedisDictionarySource::loadAll() + Pipe RedisDictionarySource::loadAll() { if (!client->isConnected()) client->connect(host, port); @@ -170,7 +170,7 @@ namespace DB /// Get only keys for specified storage type. auto all_keys = client->execute(command_for_keys); if (all_keys.isNull()) - return std::make_shared(client, RedisArray{}, storage_type, sample_block, max_block_size); + return Pipe(std::make_shared(client, RedisArray{}, storage_type, sample_block, max_block_size)); RedisArray keys; auto key_type = storageTypeToKeyType(storage_type); @@ -209,11 +209,11 @@ namespace DB keys = std::move(hkeys); } - return std::make_shared(client, std::move(keys), storage_type, sample_block, max_block_size); + return Pipe(std::make_shared(client, std::move(keys), storage_type, sample_block, max_block_size)); } - BlockInputStreamPtr RedisDictionarySource::loadIds(const std::vector & ids) + Pipe RedisDictionarySource::loadIds(const std::vector & ids) { if (!client->isConnected()) client->connect(host, port); @@ -229,10 +229,10 @@ namespace DB for (UInt64 id : ids) keys << DB::toString(id); - return std::make_shared(client, std::move(keys), storage_type, sample_block, max_block_size); + return Pipe(std::make_shared(client, std::move(keys), storage_type, sample_block, max_block_size)); } - BlockInputStreamPtr RedisDictionarySource::loadKeys(const Columns & key_columns, const std::vector & requested_rows) + Pipe RedisDictionarySource::loadKeys(const Columns & key_columns, const std::vector & requested_rows) { if (!client->isConnected()) client->connect(host, port); @@ -258,7 +258,7 @@ namespace DB keys.add(key); } - return std::make_shared(client, std::move(keys), storage_type, sample_block, max_block_size); + return Pipe(std::make_shared(client, std::move(keys), storage_type, sample_block, max_block_size)); } @@ -276,4 +276,4 @@ namespace DB return RedisStorageType::SIMPLE; } -} +} \ No newline at end of file diff --git a/src/Dictionaries/RedisDictionarySource.h b/src/Dictionaries/RedisDictionarySource.h index d67403a7f1c..31c55fabef9 100644 --- a/src/Dictionaries/RedisDictionarySource.h +++ b/src/Dictionaries/RedisDictionarySource.h @@ -59,18 +59,18 @@ namespace ErrorCodes ~RedisDictionarySource() override; - BlockInputStreamPtr loadAll() override; + Pipe loadAll() override; - BlockInputStreamPtr loadUpdatedAll() override + Pipe loadUpdatedAll() override { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method loadUpdatedAll is unsupported for RedisDictionarySource"); } bool supportsSelectiveLoad() const override { return true; } - BlockInputStreamPtr loadIds(const std::vector & ids) override; + Pipe loadIds(const std::vector & ids) override; - BlockInputStreamPtr loadKeys(const Columns & key_columns, const std::vector & requested_rows) override; + Pipe loadKeys(const Columns & key_columns, const std::vector & requested_rows) override; bool isModified() const override { return true; } @@ -94,4 +94,4 @@ namespace ErrorCodes std::shared_ptr client; }; -} +} \ No newline at end of file diff --git a/src/Dictionaries/XDBCDictionarySource.cpp b/src/Dictionaries/XDBCDictionarySource.cpp index 51984e9103a..3839e269158 100644 --- a/src/Dictionaries/XDBCDictionarySource.cpp +++ b/src/Dictionaries/XDBCDictionarySource.cpp @@ -1,7 +1,7 @@ #include "XDBCDictionarySource.h" #include -#include +#include #include #include #include @@ -30,37 +30,6 @@ namespace ErrorCodes namespace { - class XDBCBridgeBlockInputStream : public IBlockInputStream - { - public: - XDBCBridgeBlockInputStream( - const Poco::URI & uri, - std::function callback, - const Block & sample_block, - ContextPtr context, - UInt64 max_block_size, - const ConnectionTimeouts & timeouts, - const String name_) - : name(name_) - { - read_buf = std::make_unique(uri, Poco::Net::HTTPRequest::HTTP_POST, callback, timeouts); - auto format = FormatFactory::instance().getInput(IXDBCBridgeHelper::DEFAULT_FORMAT, *read_buf, sample_block, context, max_block_size); - reader = std::make_shared(format); - } - - Block getHeader() const override { return reader->getHeader(); } - - String getName() const override { return name; } - - private: - Block readImpl() override { return reader->read(); } - - String name; - std::unique_ptr read_buf; - BlockInputStreamPtr reader; - }; - - ExternalQueryBuilder makeExternalQueryBuilder(const DictionaryStructure & dict_struct_, const std::string & db_, const std::string & schema_, @@ -155,14 +124,14 @@ std::string XDBCDictionarySource::getUpdateFieldAndDate() } -BlockInputStreamPtr XDBCDictionarySource::loadAll() +Pipe XDBCDictionarySource::loadAll() { LOG_TRACE(log, load_all_query); return loadFromQuery(bridge_url, sample_block, load_all_query); } -BlockInputStreamPtr XDBCDictionarySource::loadUpdatedAll() +Pipe XDBCDictionarySource::loadUpdatedAll() { std::string load_query_update = getUpdateFieldAndDate(); @@ -171,14 +140,14 @@ BlockInputStreamPtr XDBCDictionarySource::loadUpdatedAll() } -BlockInputStreamPtr XDBCDictionarySource::loadIds(const std::vector & ids) +Pipe XDBCDictionarySource::loadIds(const std::vector & ids) { const auto query = query_builder.composeLoadIdsQuery(ids); return loadFromQuery(bridge_url, sample_block, query); } -BlockInputStreamPtr XDBCDictionarySource::loadKeys(const Columns & key_columns, const std::vector & requested_rows) +Pipe XDBCDictionarySource::loadKeys(const Columns & key_columns, const std::vector & requested_rows) { const auto query = query_builder.composeLoadKeysQuery(key_columns, requested_rows, ExternalQueryBuilder::AND_OR_CHAIN); return loadFromQuery(bridge_url, sample_block, query); @@ -236,11 +205,11 @@ std::string XDBCDictionarySource::doInvalidateQuery(const std::string & request) for (const auto & [name, value] : url_params) invalidate_url.addQueryParameter(name, value); - return readInvalidateQuery(*loadFromQuery(invalidate_url, invalidate_sample_block, request)); + return readInvalidateQuery(loadFromQuery(invalidate_url, invalidate_sample_block, request)); } -BlockInputStreamPtr XDBCDictionarySource::loadFromQuery(const Poco::URI & url, const Block & required_sample_block, const std::string & query) const +Pipe XDBCDictionarySource::loadFromQuery(const Poco::URI & url, const Block & required_sample_block, const std::string & query) const { bridge_helper->startBridgeSync(); @@ -251,16 +220,12 @@ BlockInputStreamPtr XDBCDictionarySource::loadFromQuery(const Poco::URI & url, c os << "query=" << escapeForFileName(query); }; - return std::make_shared( - url, - write_body_callback, - required_sample_block, - getContext(), - max_block_size, - timeouts, - bridge_helper->getName() + "BlockInputStream"); -} + auto read_buf = std::make_unique(url, Poco::Net::HTTPRequest::HTTP_POST, write_body_callback, timeouts); + auto format = FormatFactory::instance().getInput(IXDBCBridgeHelper::DEFAULT_FORMAT, *read_buf, sample_block, getContext(), max_block_size); + format->addBuffer(std::move(read_buf)); + return Pipe(std::move(format)); +} void registerDictionarySourceXDBC(DictionarySourceFactory & factory) { @@ -320,4 +285,4 @@ void registerDictionarySourceJDBC(DictionarySourceFactory & factory) factory.registerSource("jdbc", create_table_source); } -} +} \ No newline at end of file diff --git a/src/Dictionaries/XDBCDictionarySource.h b/src/Dictionaries/XDBCDictionarySource.h index ab68ab05804..82da3f22d0a 100644 --- a/src/Dictionaries/XDBCDictionarySource.h +++ b/src/Dictionaries/XDBCDictionarySource.h @@ -49,13 +49,13 @@ public: XDBCDictionarySource(const XDBCDictionarySource & other); XDBCDictionarySource & operator=(const XDBCDictionarySource &) = delete; - BlockInputStreamPtr loadAll() override; + Pipe loadAll() override; - BlockInputStreamPtr loadUpdatedAll() override; + Pipe loadUpdatedAll() override; - BlockInputStreamPtr loadIds(const std::vector & ids) override; + Pipe loadIds(const std::vector & ids) override; - BlockInputStreamPtr loadKeys(const Columns & key_columns, const std::vector & requested_rows) override; + Pipe loadKeys(const Columns & key_columns, const std::vector & requested_rows) override; bool isModified() const override; @@ -73,7 +73,7 @@ private: // execute invalidate_query. expects single cell in result std::string doInvalidateQuery(const std::string & request) const; - BlockInputStreamPtr loadFromQuery(const Poco::URI & url, const Block & required_sample_block, const std::string & query) const; + Pipe loadFromQuery(const Poco::URI & url, const Block & required_sample_block, const std::string & query) const; Poco::Logger * log; @@ -90,4 +90,4 @@ private: ConnectionTimeouts timeouts; }; -} +} \ No newline at end of file diff --git a/src/Formats/MySQLBlockInputStream.cpp b/src/Formats/MySQLBlockInputStream.cpp index 79ebeacfad5..731afa5d1a6 100644 --- a/src/Formats/MySQLBlockInputStream.cpp +++ b/src/Formats/MySQLBlockInputStream.cpp @@ -40,7 +40,7 @@ StreamSettings::StreamSettings(const Settings & settings, bool auto_close_, bool { } -MySQLBlockInputStream::Connection::Connection( +MySQLSource::Connection::Connection( const mysqlxx::PoolWithFailover::Entry & entry_, const std::string & query_str) : entry(entry_) @@ -50,12 +50,13 @@ MySQLBlockInputStream::Connection::Connection( } /// Used in MaterializedMySQL and in doInvalidateQuery for dictionary source. -MySQLBlockInputStream::MySQLBlockInputStream( +MySQLSource::MySQLSource( const mysqlxx::PoolWithFailover::Entry & entry, const std::string & query_str, const Block & sample_block, const StreamSettings & settings_) - : log(&Poco::Logger::get("MySQLBlockInputStream")) + : SourceWithProgress(sample_block.cloneEmpty()) + , log(&Poco::Logger::get("MySQLBlockInputStream")) , connection{std::make_unique(entry, query_str)} , settings{std::make_unique(settings_)} { @@ -64,26 +65,27 @@ MySQLBlockInputStream::MySQLBlockInputStream( } /// For descendant MySQLWithFailoverBlockInputStream - MySQLBlockInputStream::MySQLBlockInputStream(const Block &sample_block_, const StreamSettings & settings_) - : log(&Poco::Logger::get("MySQLBlockInputStream")) +MySQLSource::MySQLSource(const Block &sample_block_, const StreamSettings & settings_) + : SourceWithProgress(sample_block_.cloneEmpty()) + , log(&Poco::Logger::get("MySQLBlockInputStream")) , settings(std::make_unique(settings_)) { description.init(sample_block_); } /// Used by MySQL storage / table function and dictionary source. -MySQLWithFailoverBlockInputStream::MySQLWithFailoverBlockInputStream( +MySQLWithFailoverSource::MySQLWithFailoverSource( mysqlxx::PoolWithFailoverPtr pool_, const std::string & query_str_, const Block & sample_block_, const StreamSettings & settings_) -: MySQLBlockInputStream(sample_block_, settings_) -, pool(pool_) -, query_str(query_str_) + : MySQLSource(sample_block_, settings_) + , pool(pool_) + , query_str(query_str_) { } -void MySQLWithFailoverBlockInputStream::readPrefix() +void MySQLWithFailoverSource::onStart() { size_t count_connect_attempts = 0; @@ -110,6 +112,18 @@ void MySQLWithFailoverBlockInputStream::readPrefix() initPositionMappingFromQueryResultStructure(); } +Chunk MySQLWithFailoverSource::generate() +{ + if (!is_initialized) + { + onStart(); + is_initialized = true; + } + + return MySQLSource::generate(); +} + + namespace { using ValueType = ExternalResultDescription::ValueType; @@ -213,7 +227,7 @@ namespace } -Block MySQLBlockInputStream::readImpl() +Chunk MySQLSource::generate() { auto row = connection->result.fetch(); if (!row) @@ -272,10 +286,10 @@ Block MySQLBlockInputStream::readImpl() row = connection->result.fetch(); } - return description.sample_block.cloneWithColumns(std::move(columns)); + return Chunk(std::move(columns), num_rows); } -void MySQLBlockInputStream::initPositionMappingFromQueryResultStructure() +void MySQLSource::initPositionMappingFromQueryResultStructure() { position_mapping.resize(description.sample_block.columns()); @@ -324,4 +338,4 @@ void MySQLBlockInputStream::initPositionMappingFromQueryResultStructure() } -#endif +#endif \ No newline at end of file diff --git a/src/Formats/MySQLBlockInputStream.h b/src/Formats/MySQLBlockInputStream.h index 12deb9c3146..a0ef9cfd4a8 100644 --- a/src/Formats/MySQLBlockInputStream.h +++ b/src/Formats/MySQLBlockInputStream.h @@ -2,7 +2,7 @@ #include #include -#include +#include #include #include #include @@ -25,10 +25,10 @@ struct StreamSettings }; /// Allows processing results of a MySQL query as a sequence of Blocks, simplifies chaining -class MySQLBlockInputStream : public IBlockInputStream +class MySQLSource : public SourceWithProgress { public: - MySQLBlockInputStream( + MySQLSource( const mysqlxx::PoolWithFailover::Entry & entry, const std::string & query_str, const Block & sample_block, @@ -36,11 +36,9 @@ public: String getName() const override { return "MySQL"; } - Block getHeader() const override { return description.sample_block.cloneEmpty(); } - protected: - MySQLBlockInputStream(const Block & sample_block_, const StreamSettings & settings); - Block readImpl() override; + MySQLSource(const Block & sample_block_, const StreamSettings & settings); + Chunk generate() override; void initPositionMappingFromQueryResultStructure(); struct Connection @@ -63,21 +61,24 @@ protected: /// Like MySQLBlockInputStream, but allocates connection only when reading is starting. /// It allows to create a lot of stream objects without occupation of all connection pool. /// Also makes attempts to reconnect in case of connection failures. -class MySQLWithFailoverBlockInputStream final : public MySQLBlockInputStream +class MySQLWithFailoverSource final : public MySQLSource { public: - MySQLWithFailoverBlockInputStream( + MySQLWithFailoverSource( mysqlxx::PoolWithFailoverPtr pool_, const std::string & query_str_, const Block & sample_block_, const StreamSettings & settings_); + Chunk generate() override; + private: - void readPrefix() override; + void onStart(); mysqlxx::PoolWithFailoverPtr pool; std::string query_str; + bool is_initialized = false; }; -} +} \ No newline at end of file From bd776e432f33907a557ad7e6d999712d774a5dd8 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 5 Aug 2021 23:36:46 +0300 Subject: [PATCH 170/599] Advancements --- contrib/croaring-cmake/CMakeLists.txt | 8 +++++- src/Common/clickhouse_malloc.cpp | 38 +++++++++++++++++++++++---- 2 files changed, 40 insertions(+), 6 deletions(-) diff --git a/contrib/croaring-cmake/CMakeLists.txt b/contrib/croaring-cmake/CMakeLists.txt index 522540e5e40..1c61eb39cc8 100644 --- a/contrib/croaring-cmake/CMakeLists.txt +++ b/contrib/croaring-cmake/CMakeLists.txt @@ -25,4 +25,10 @@ target_include_directories(roaring PRIVATE "${LIBRARY_DIR}/include/roaring") target_include_directories(roaring SYSTEM BEFORE PUBLIC "${LIBRARY_DIR}/include") target_include_directories(roaring SYSTEM BEFORE PUBLIC "${LIBRARY_DIR}/cpp") -target_compile_definitions(roaring PRIVATE -Dmalloc=clickhouse_malloc -Dfree=clickhouse_free -Dposix_memalign=clickhouse_posix_memalign) +target_compile_definitions(roaring PRIVATE + -Dmalloc=clickhouse_malloc + -Dcalloc=clickhouse_calloc + -Drealloc=clickhouse_realloc + -Dreallocarray=clickhouse_reallocarray + -Dfree=clickhouse_free + -Dposix_memalign=clickhouse_posix_memalign) diff --git a/src/Common/clickhouse_malloc.cpp b/src/Common/clickhouse_malloc.cpp index 5e37c8b3a2d..95b1d56974b 100644 --- a/src/Common/clickhouse_malloc.cpp +++ b/src/Common/clickhouse_malloc.cpp @@ -1,7 +1,5 @@ -#if 0 - #include -#include +#include extern "C" void * clickhouse_malloc(size_t size) @@ -12,6 +10,38 @@ extern "C" void * clickhouse_malloc(size_t size) return res; } +extern "C" void * clickhouse_calloc(size_t number_of_members, size_t size) +{ + void * res = calloc(number_of_members, size); + if (res) + Memory::trackMemory(number_of_members * size); + return res; +} + +extern "C" void * clickhouse_realloc(void * ptr, size_t size) +{ + void * res = realloc(ptr, size); + if (res) + { + if (ptr) + Memory::untrackMemory(ptr); + Memory::trackMemory(size); + } + return res; +} + +extern "C" void * clickhouse_reallocarray(void * ptr, size_t number_of_members, size_t size) +{ + void * res = reallocarray(ptr, number_of_members, size); + if (res) + { + if (ptr) + Memory::untrackMemory(ptr); + Memory::trackMemory(number_of_members * size); + } + return res; +} + extern "C" void clickhouse_free(void * ptr) { Memory::untrackMemory(ptr); @@ -25,5 +55,3 @@ extern "C" int clickhouse_posix_memalign(void ** memptr, size_t alignment, size_ Memory::trackMemory(size); return res; } - -#endif From 9a5533a0880c8d0e5e7639067ea2647874846625 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 5 Aug 2021 23:44:14 +0300 Subject: [PATCH 171/599] Improve performance --- src/AggregateFunctions/AggregateFunctionGroupBitmapData.h | 2 -- src/Interpreters/Aggregator.cpp | 3 +++ 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionGroupBitmapData.h b/src/AggregateFunctions/AggregateFunctionGroupBitmapData.h index d5484c4db79..a10cb7ebac5 100644 --- a/src/AggregateFunctions/AggregateFunctionGroupBitmapData.h +++ b/src/AggregateFunctions/AggregateFunctionGroupBitmapData.h @@ -68,14 +68,12 @@ public: { toLarge(); rb->add(static_cast(value)); - CurrentMemoryTracker::check(); } } } else { rb->add(static_cast(value)); - CurrentMemoryTracker::check(); } } diff --git a/src/Interpreters/Aggregator.cpp b/src/Interpreters/Aggregator.cpp index 7ffae761c0c..69ad1d56359 100644 --- a/src/Interpreters/Aggregator.cpp +++ b/src/Interpreters/Aggregator.cpp @@ -1193,6 +1193,9 @@ bool Aggregator::checkLimits(size_t result_size, bool & no_more_keys) const } } + /// Some aggregate functions cannot throw exceptions on allocations (e.g. from C malloc) + /// but still tracks memory. Check it here. + CurrentMemoryTracker::check(); return true; } From 638b229e275c837f600a32152533541eeecde90f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 5 Aug 2021 23:50:09 +0300 Subject: [PATCH 172/599] Remove debug headers --- src/AggregateFunctions/AggregateFunctionGroupBitmapData.h | 1 - src/Common/CurrentMemoryTracker.cpp | 1 - 2 files changed, 2 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionGroupBitmapData.h b/src/AggregateFunctions/AggregateFunctionGroupBitmapData.h index a10cb7ebac5..95c7e6075d7 100644 --- a/src/AggregateFunctions/AggregateFunctionGroupBitmapData.h +++ b/src/AggregateFunctions/AggregateFunctionGroupBitmapData.h @@ -7,7 +7,6 @@ #include #include #include -#include // Include this header last, because it is an auto-generated dump of questionable // garbage that breaks the build (e.g. it changes _POSIX_C_SOURCE). diff --git a/src/Common/CurrentMemoryTracker.cpp b/src/Common/CurrentMemoryTracker.cpp index 1156c6e56bc..bf0745e667e 100644 --- a/src/Common/CurrentMemoryTracker.cpp +++ b/src/Common/CurrentMemoryTracker.cpp @@ -2,7 +2,6 @@ #include #include -#include namespace From b1eb3fde6cae81c02ce3ae83434056386805bac5 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 5 Aug 2021 23:51:39 +0300 Subject: [PATCH 173/599] Add comment --- src/Common/clickhouse_malloc.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Common/clickhouse_malloc.cpp b/src/Common/clickhouse_malloc.cpp index 95b1d56974b..cb6419ae6da 100644 --- a/src/Common/clickhouse_malloc.cpp +++ b/src/Common/clickhouse_malloc.cpp @@ -2,6 +2,9 @@ #include +/** These functions can be substituted instead of regular ones when memory tracking is needed. + */ + extern "C" void * clickhouse_malloc(size_t size) { void * res = malloc(size); From c4e2342a065a3432338d40a9ddee0228a38dfacf Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 5 Aug 2021 21:05:46 +0300 Subject: [PATCH 174/599] Add verbose description for pre-LIMIT into EXPLAIN query --- src/Interpreters/InterpreterSelectQuery.cpp | 5 ++++- ...1562_optimize_monotonous_functions_in_order_by.reference | 6 +++--- .../0_stateless/01576_alias_column_rewrite.reference | 6 +++--- 3 files changed, 10 insertions(+), 7 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 33f9deaf805..051dff19bff 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -2429,7 +2429,10 @@ void InterpreterSelectQuery::executePreLimit(QueryPlan & query_plan, bool do_not } auto limit = std::make_unique(query_plan.getCurrentDataStream(), limit_length, limit_offset); - limit->setStepDescription("preliminary LIMIT"); + if (do_not_skip_offset) + limit->setStepDescription("preliminary LIMIT (with OFFSET)"); + else + limit->setStepDescription("preliminary LIMIT (without OFFSET)"); query_plan.addStep(std::move(limit)); } } diff --git a/tests/queries/0_stateless/01562_optimize_monotonous_functions_in_order_by.reference b/tests/queries/0_stateless/01562_optimize_monotonous_functions_in_order_by.reference index 0eb7e06f724..bf9bff06959 100644 --- a/tests/queries/0_stateless/01562_optimize_monotonous_functions_in_order_by.reference +++ b/tests/queries/0_stateless/01562_optimize_monotonous_functions_in_order_by.reference @@ -5,7 +5,7 @@ FROM test_order_by ORDER BY timestamp ASC LIMIT 10 Expression (Projection) - Limit (preliminary LIMIT) + Limit (preliminary LIMIT (without OFFSET)) MergingSorted (Merge sorted streams for ORDER BY) MergeSorting (Merge sorted blocks for ORDER BY) PartialSorting (Sort each block for ORDER BY) @@ -19,7 +19,7 @@ FROM test_order_by ORDER BY toDate(timestamp) ASC LIMIT 10 Expression (Projection) - Limit (preliminary LIMIT) + Limit (preliminary LIMIT (without OFFSET)) FinishSorting Expression (Before ORDER BY) SettingQuotaAndLimits (Set limits and quota after reading from storage) @@ -33,7 +33,7 @@ ORDER BY timestamp ASC LIMIT 10 Expression (Projection) - Limit (preliminary LIMIT) + Limit (preliminary LIMIT (without OFFSET)) FinishSorting Expression (Before ORDER BY) SettingQuotaAndLimits (Set limits and quota after reading from storage) diff --git a/tests/queries/0_stateless/01576_alias_column_rewrite.reference b/tests/queries/0_stateless/01576_alias_column_rewrite.reference index c9a4c04b352..2a824e62158 100644 --- a/tests/queries/0_stateless/01576_alias_column_rewrite.reference +++ b/tests/queries/0_stateless/01576_alias_column_rewrite.reference @@ -22,7 +22,7 @@ lambda 1 optimize_read_in_order Expression (Projection) - Limit (preliminary LIMIT) + Limit (preliminary LIMIT (without OFFSET)) MergingSorted (Merge sorted streams for ORDER BY) MergeSorting (Merge sorted blocks for ORDER BY) PartialSorting (Sort each block for ORDER BY) @@ -30,13 +30,13 @@ Expression (Projection) SettingQuotaAndLimits (Set limits and quota after reading from storage) ReadFromMergeTree Expression (Projection) - Limit (preliminary LIMIT) + Limit (preliminary LIMIT (without OFFSET)) FinishSorting Expression (Before ORDER BY) SettingQuotaAndLimits (Set limits and quota after reading from storage) ReadFromMergeTree Expression (Projection) - Limit (preliminary LIMIT) + Limit (preliminary LIMIT (without OFFSET)) FinishSorting Expression (Before ORDER BY) SettingQuotaAndLimits (Set limits and quota after reading from storage) From 1358a2db92b60339b5081825b3e2cafec0373089 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 5 Aug 2021 21:05:46 +0300 Subject: [PATCH 175/599] Fix WithMergeableStateAfterAggregationAndLimit with LIMIT BY and LIMIT OFFSET In case of LIMIT BY, pre LIMIT cannot be applied (that is done on the shard before), and hence shard applies regular LIMIT, but it goes with applying OFFSET, and also the initiator will do apply OFFSET too. --- src/Interpreters/InterpreterSelectQuery.cpp | 28 ++++++++++----- ...onAndLimit_LIMIT_BY_LIMIT_OFFSET.reference | 36 +++++++++++++++++++ ...regationAndLimit_LIMIT_BY_LIMIT_OFFSET.sql | 26 ++++++++++++++ 3 files changed, 81 insertions(+), 9 deletions(-) create mode 100644 tests/queries/0_stateless/02003_WithMergeableStateAfterAggregationAndLimit_LIMIT_BY_LIMIT_OFFSET.reference create mode 100644 tests/queries/0_stateless/02003_WithMergeableStateAfterAggregationAndLimit_LIMIT_BY_LIMIT_OFFSET.sql diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 051dff19bff..ed76074c496 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -1327,15 +1327,20 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, const BlockInpu } bool apply_limit = options.to_stage != QueryProcessingStage::WithMergeableStateAfterAggregation; + bool apply_prelimit = apply_limit && + query.limitLength() && !query.limit_with_ties && + !hasWithTotalsInAnySubqueryInFromClause(query) && + !query.arrayJoinExpressionList() && + !query.distinct && + !expressions.hasLimitBy() && + !settings.extremes && + !has_withfill; bool apply_offset = options.to_stage != QueryProcessingStage::WithMergeableStateAfterAggregationAndLimit; - bool has_prelimit = false; - if (apply_limit && - query.limitLength() && !query.limit_with_ties && !hasWithTotalsInAnySubqueryInFromClause(query) && - !query.arrayJoinExpressionList() && !query.distinct && !expressions.hasLimitBy() && !settings.extremes && - !has_withfill) + bool limit_applied = false; + if (apply_prelimit) { executePreLimit(query_plan, /* do_not_skip_offset= */!apply_offset); - has_prelimit = true; + limit_applied = true; } /** If there was more than one stream, @@ -1354,10 +1359,10 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, const BlockInpu /// If we have 'WITH TIES', we need execute limit before projection, /// because in that case columns from 'ORDER BY' are used. - if (query.limit_with_ties) + if (query.limit_with_ties && apply_offset) { executeLimit(query_plan); - has_prelimit = true; + limit_applied = true; } /// Projection not be done on the shards, since then initiator will not find column in blocks. @@ -1372,7 +1377,12 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, const BlockInpu executeExtremes(query_plan); /// Limit is no longer needed if there is prelimit. - if (apply_limit && !has_prelimit) + /// + /// NOTE: that LIMIT cannot be applied of OFFSET should not be applied, + /// since LIMIT will apply OFFSET too. + /// This is the case for various optimizations for distributed queries, + /// and when LIMIT cannot be applied it will be applied on the initiator anyway. + if (apply_limit && !limit_applied && apply_offset) executeLimit(query_plan); if (apply_offset) diff --git a/tests/queries/0_stateless/02003_WithMergeableStateAfterAggregationAndLimit_LIMIT_BY_LIMIT_OFFSET.reference b/tests/queries/0_stateless/02003_WithMergeableStateAfterAggregationAndLimit_LIMIT_BY_LIMIT_OFFSET.reference new file mode 100644 index 00000000000..08fe4251b69 --- /dev/null +++ b/tests/queries/0_stateless/02003_WithMergeableStateAfterAggregationAndLimit_LIMIT_BY_LIMIT_OFFSET.reference @@ -0,0 +1,36 @@ +-- { echo } +SELECT * +FROM remote('127.{1,2}', view( + SELECT * + FROM numbers(10) +), number) +GROUP BY number +ORDER BY number ASC +LIMIT 1 BY number +LIMIT 5, 5 +SETTINGS + optimize_skip_unused_shards = 1, + optimize_distributed_group_by_sharding_key = 1, + distributed_push_down_limit=1; +5 +6 +7 +8 +9 +SELECT * +FROM remote('127.{1,2}', view( + SELECT * + FROM numbers(10) +), number) +GROUP BY number +ORDER BY number ASC +LIMIT 1 BY number +LIMIT 5, 5 +SETTINGS + distributed_group_by_no_merge=2, + distributed_push_down_limit=1; +5 +6 +7 +8 +9 diff --git a/tests/queries/0_stateless/02003_WithMergeableStateAfterAggregationAndLimit_LIMIT_BY_LIMIT_OFFSET.sql b/tests/queries/0_stateless/02003_WithMergeableStateAfterAggregationAndLimit_LIMIT_BY_LIMIT_OFFSET.sql new file mode 100644 index 00000000000..75c06704ab9 --- /dev/null +++ b/tests/queries/0_stateless/02003_WithMergeableStateAfterAggregationAndLimit_LIMIT_BY_LIMIT_OFFSET.sql @@ -0,0 +1,26 @@ +-- { echo } +SELECT * +FROM remote('127.{1,2}', view( + SELECT * + FROM numbers(10) +), number) +GROUP BY number +ORDER BY number ASC +LIMIT 1 BY number +LIMIT 5, 5 +SETTINGS + optimize_skip_unused_shards = 1, + optimize_distributed_group_by_sharding_key = 1, + distributed_push_down_limit=1; +SELECT * +FROM remote('127.{1,2}', view( + SELECT * + FROM numbers(10) +), number) +GROUP BY number +ORDER BY number ASC +LIMIT 1 BY number +LIMIT 5, 5 +SETTINGS + distributed_group_by_no_merge=2, + distributed_push_down_limit=1; From d0b6b167f5d1cd2bca23237d616cef3a1ffe7d97 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 6 Aug 2021 00:19:04 +0300 Subject: [PATCH 176/599] Exclude 01493_alter_remove_properties_zookeeper from parallel run too --- tests/queries/skip_list.json | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/skip_list.json b/tests/queries/skip_list.json index 24acb86299f..aebe9a83e42 100644 --- a/tests/queries/skip_list.json +++ b/tests/queries/skip_list.json @@ -498,6 +498,7 @@ "01902_table_function_merge_db_repr", "01946_test_zstd_decompression_with_escape_sequence_at_the_end_of_buffer", "01946_test_wrong_host_name_access", + "01493_alter_remove_properties_zookeeper", "00510_materizlized_view_and_deduplication_zookeeper", /// static UUID "00509_extended_storage_definition_syntax_zookeeper", /// leftovers "00083_create_merge_tree_zookeeper", /// leftovers From 735b730a526169f6438a76ff4fe4634ba029193a Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 6 Aug 2021 00:20:48 +0300 Subject: [PATCH 177/599] Mark more tests as long --- ...6_zookeeper_deduplication_and_unexpected_parts_long.reference} | 0 ...> 00226_zookeeper_deduplication_and_unexpected_parts_long.sql} | 0 ... 00236_replicated_drop_on_non_leader_zookeeper_long.reference} | 0 ...sql => 00236_replicated_drop_on_non_leader_zookeeper_long.sql} | 0 ...e => 00446_clear_column_in_partition_zookeeper_long.reference} | 0 ...per.sql => 00446_clear_column_in_partition_zookeeper_long.sql} | 0 ...e => 00623_replicated_truncate_table_zookeeper_long.reference} | 0 ...per.sql => 00623_replicated_truncate_table_zookeeper_long.sql} | 0 ...orce_by_identical_result_after_merge_zookeeper_long.reference} | 0 ...0721_force_by_identical_result_after_merge_zookeeper_long.sql} | 0 ...ent_columns.reference => 00725_comment_columns_long.reference} | 0 .../{00725_comment_columns.sql => 00725_comment_columns_long.sql} | 0 ...uorum_insert_have_data_before_quorum_zookeeper_long.reference} | 0 ...0732_quorum_insert_have_data_before_quorum_zookeeper_long.sql} | 0 ...nce => 00732_quorum_insert_lost_part_zookeeper_long.reference} | 0 ...eeper.sql => 00732_quorum_insert_lost_part_zookeeper_long.sql} | 0 ...32_quorum_insert_simple_test_2_parts_zookeeper_long.reference} | 0 ...=> 00732_quorum_insert_simple_test_2_parts_zookeeper_long.sql} | 0 ...per_empty_replicated_merge_tree_optimize_final_long.reference} | 0 ...zookeeper_empty_replicated_merge_tree_optimize_final_long.sql} | 0 ...er.reference => 00933_ttl_replicated_zookeeper_long.reference} | 0 ...ated_zookeeper.sql => 00933_ttl_replicated_zookeeper_long.sql} | 0 ...eference => 01062_alter_on_mutataion_zookeeper_long.reference} | 0 ..._zookeeper.sql => 01062_alter_on_mutataion_zookeeper_long.sql} | 0 ...e => 01213_alter_rename_with_default_zookeeper_long.reference} | 0 ...per.sql => 01213_alter_rename_with_default_zookeeper_long.sql} | 0 ...e => 01267_alter_default_key_columns_zookeeper_long.reference} | 0 ...per.sql => 01267_alter_default_key_columns_zookeeper_long.sql} | 0 ...as.reference => 01319_manual_write_to_replicas_long.reference} | 0 ...te_to_replicas.sql => 01319_manual_write_to_replicas_long.sql} | 0 ...e.reference => 01669_columns_declaration_serde_long.reference} | 0 ...aration_serde.sql => 01669_columns_declaration_serde_long.sql} | 0 32 files changed, 0 insertions(+), 0 deletions(-) rename tests/queries/0_stateless/{00226_zookeeper_deduplication_and_unexpected_parts.reference => 00226_zookeeper_deduplication_and_unexpected_parts_long.reference} (100%) rename tests/queries/0_stateless/{00226_zookeeper_deduplication_and_unexpected_parts.sql => 00226_zookeeper_deduplication_and_unexpected_parts_long.sql} (100%) rename tests/queries/0_stateless/{00236_replicated_drop_on_non_leader_zookeeper.reference => 00236_replicated_drop_on_non_leader_zookeeper_long.reference} (100%) rename tests/queries/0_stateless/{00236_replicated_drop_on_non_leader_zookeeper.sql => 00236_replicated_drop_on_non_leader_zookeeper_long.sql} (100%) rename tests/queries/0_stateless/{00446_clear_column_in_partition_zookeeper.reference => 00446_clear_column_in_partition_zookeeper_long.reference} (100%) rename tests/queries/0_stateless/{00446_clear_column_in_partition_zookeeper.sql => 00446_clear_column_in_partition_zookeeper_long.sql} (100%) rename tests/queries/0_stateless/{00623_replicated_truncate_table_zookeeper.reference => 00623_replicated_truncate_table_zookeeper_long.reference} (100%) rename tests/queries/0_stateless/{00623_replicated_truncate_table_zookeeper.sql => 00623_replicated_truncate_table_zookeeper_long.sql} (100%) rename tests/queries/0_stateless/{00721_force_by_identical_result_after_merge_zookeeper.reference => 00721_force_by_identical_result_after_merge_zookeeper_long.reference} (100%) rename tests/queries/0_stateless/{00721_force_by_identical_result_after_merge_zookeeper.sql => 00721_force_by_identical_result_after_merge_zookeeper_long.sql} (100%) rename tests/queries/0_stateless/{00725_comment_columns.reference => 00725_comment_columns_long.reference} (100%) rename tests/queries/0_stateless/{00725_comment_columns.sql => 00725_comment_columns_long.sql} (100%) rename tests/queries/0_stateless/{00732_quorum_insert_have_data_before_quorum_zookeeper.reference => 00732_quorum_insert_have_data_before_quorum_zookeeper_long.reference} (100%) rename tests/queries/0_stateless/{00732_quorum_insert_have_data_before_quorum_zookeeper.sql => 00732_quorum_insert_have_data_before_quorum_zookeeper_long.sql} (100%) rename tests/queries/0_stateless/{00732_quorum_insert_lost_part_zookeeper.reference => 00732_quorum_insert_lost_part_zookeeper_long.reference} (100%) rename tests/queries/0_stateless/{00732_quorum_insert_lost_part_zookeeper.sql => 00732_quorum_insert_lost_part_zookeeper_long.sql} (100%) rename tests/queries/0_stateless/{00732_quorum_insert_simple_test_2_parts_zookeeper.reference => 00732_quorum_insert_simple_test_2_parts_zookeeper_long.reference} (100%) rename tests/queries/0_stateless/{00732_quorum_insert_simple_test_2_parts_zookeeper.sql => 00732_quorum_insert_simple_test_2_parts_zookeeper_long.sql} (100%) rename tests/queries/0_stateless/{00925_zookeeper_empty_replicated_merge_tree_optimize_final.reference => 00925_zookeeper_empty_replicated_merge_tree_optimize_final_long.reference} (100%) rename tests/queries/0_stateless/{00925_zookeeper_empty_replicated_merge_tree_optimize_final.sql => 00925_zookeeper_empty_replicated_merge_tree_optimize_final_long.sql} (100%) rename tests/queries/0_stateless/{00933_ttl_replicated_zookeeper.reference => 00933_ttl_replicated_zookeeper_long.reference} (100%) rename tests/queries/0_stateless/{00933_ttl_replicated_zookeeper.sql => 00933_ttl_replicated_zookeeper_long.sql} (100%) rename tests/queries/0_stateless/{01062_alter_on_mutataion_zookeeper.reference => 01062_alter_on_mutataion_zookeeper_long.reference} (100%) rename tests/queries/0_stateless/{01062_alter_on_mutataion_zookeeper.sql => 01062_alter_on_mutataion_zookeeper_long.sql} (100%) rename tests/queries/0_stateless/{01213_alter_rename_with_default_zookeeper.reference => 01213_alter_rename_with_default_zookeeper_long.reference} (100%) rename tests/queries/0_stateless/{01213_alter_rename_with_default_zookeeper.sql => 01213_alter_rename_with_default_zookeeper_long.sql} (100%) rename tests/queries/0_stateless/{01267_alter_default_key_columns_zookeeper.reference => 01267_alter_default_key_columns_zookeeper_long.reference} (100%) rename tests/queries/0_stateless/{01267_alter_default_key_columns_zookeeper.sql => 01267_alter_default_key_columns_zookeeper_long.sql} (100%) rename tests/queries/0_stateless/{01319_manual_write_to_replicas.reference => 01319_manual_write_to_replicas_long.reference} (100%) rename tests/queries/0_stateless/{01319_manual_write_to_replicas.sql => 01319_manual_write_to_replicas_long.sql} (100%) rename tests/queries/0_stateless/{01669_columns_declaration_serde.reference => 01669_columns_declaration_serde_long.reference} (100%) rename tests/queries/0_stateless/{01669_columns_declaration_serde.sql => 01669_columns_declaration_serde_long.sql} (100%) diff --git a/tests/queries/0_stateless/00226_zookeeper_deduplication_and_unexpected_parts.reference b/tests/queries/0_stateless/00226_zookeeper_deduplication_and_unexpected_parts_long.reference similarity index 100% rename from tests/queries/0_stateless/00226_zookeeper_deduplication_and_unexpected_parts.reference rename to tests/queries/0_stateless/00226_zookeeper_deduplication_and_unexpected_parts_long.reference diff --git a/tests/queries/0_stateless/00226_zookeeper_deduplication_and_unexpected_parts.sql b/tests/queries/0_stateless/00226_zookeeper_deduplication_and_unexpected_parts_long.sql similarity index 100% rename from tests/queries/0_stateless/00226_zookeeper_deduplication_and_unexpected_parts.sql rename to tests/queries/0_stateless/00226_zookeeper_deduplication_and_unexpected_parts_long.sql diff --git a/tests/queries/0_stateless/00236_replicated_drop_on_non_leader_zookeeper.reference b/tests/queries/0_stateless/00236_replicated_drop_on_non_leader_zookeeper_long.reference similarity index 100% rename from tests/queries/0_stateless/00236_replicated_drop_on_non_leader_zookeeper.reference rename to tests/queries/0_stateless/00236_replicated_drop_on_non_leader_zookeeper_long.reference diff --git a/tests/queries/0_stateless/00236_replicated_drop_on_non_leader_zookeeper.sql b/tests/queries/0_stateless/00236_replicated_drop_on_non_leader_zookeeper_long.sql similarity index 100% rename from tests/queries/0_stateless/00236_replicated_drop_on_non_leader_zookeeper.sql rename to tests/queries/0_stateless/00236_replicated_drop_on_non_leader_zookeeper_long.sql diff --git a/tests/queries/0_stateless/00446_clear_column_in_partition_zookeeper.reference b/tests/queries/0_stateless/00446_clear_column_in_partition_zookeeper_long.reference similarity index 100% rename from tests/queries/0_stateless/00446_clear_column_in_partition_zookeeper.reference rename to tests/queries/0_stateless/00446_clear_column_in_partition_zookeeper_long.reference diff --git a/tests/queries/0_stateless/00446_clear_column_in_partition_zookeeper.sql b/tests/queries/0_stateless/00446_clear_column_in_partition_zookeeper_long.sql similarity index 100% rename from tests/queries/0_stateless/00446_clear_column_in_partition_zookeeper.sql rename to tests/queries/0_stateless/00446_clear_column_in_partition_zookeeper_long.sql diff --git a/tests/queries/0_stateless/00623_replicated_truncate_table_zookeeper.reference b/tests/queries/0_stateless/00623_replicated_truncate_table_zookeeper_long.reference similarity index 100% rename from tests/queries/0_stateless/00623_replicated_truncate_table_zookeeper.reference rename to tests/queries/0_stateless/00623_replicated_truncate_table_zookeeper_long.reference diff --git a/tests/queries/0_stateless/00623_replicated_truncate_table_zookeeper.sql b/tests/queries/0_stateless/00623_replicated_truncate_table_zookeeper_long.sql similarity index 100% rename from tests/queries/0_stateless/00623_replicated_truncate_table_zookeeper.sql rename to tests/queries/0_stateless/00623_replicated_truncate_table_zookeeper_long.sql diff --git a/tests/queries/0_stateless/00721_force_by_identical_result_after_merge_zookeeper.reference b/tests/queries/0_stateless/00721_force_by_identical_result_after_merge_zookeeper_long.reference similarity index 100% rename from tests/queries/0_stateless/00721_force_by_identical_result_after_merge_zookeeper.reference rename to tests/queries/0_stateless/00721_force_by_identical_result_after_merge_zookeeper_long.reference diff --git a/tests/queries/0_stateless/00721_force_by_identical_result_after_merge_zookeeper.sql b/tests/queries/0_stateless/00721_force_by_identical_result_after_merge_zookeeper_long.sql similarity index 100% rename from tests/queries/0_stateless/00721_force_by_identical_result_after_merge_zookeeper.sql rename to tests/queries/0_stateless/00721_force_by_identical_result_after_merge_zookeeper_long.sql diff --git a/tests/queries/0_stateless/00725_comment_columns.reference b/tests/queries/0_stateless/00725_comment_columns_long.reference similarity index 100% rename from tests/queries/0_stateless/00725_comment_columns.reference rename to tests/queries/0_stateless/00725_comment_columns_long.reference diff --git a/tests/queries/0_stateless/00725_comment_columns.sql b/tests/queries/0_stateless/00725_comment_columns_long.sql similarity index 100% rename from tests/queries/0_stateless/00725_comment_columns.sql rename to tests/queries/0_stateless/00725_comment_columns_long.sql diff --git a/tests/queries/0_stateless/00732_quorum_insert_have_data_before_quorum_zookeeper.reference b/tests/queries/0_stateless/00732_quorum_insert_have_data_before_quorum_zookeeper_long.reference similarity index 100% rename from tests/queries/0_stateless/00732_quorum_insert_have_data_before_quorum_zookeeper.reference rename to tests/queries/0_stateless/00732_quorum_insert_have_data_before_quorum_zookeeper_long.reference diff --git a/tests/queries/0_stateless/00732_quorum_insert_have_data_before_quorum_zookeeper.sql b/tests/queries/0_stateless/00732_quorum_insert_have_data_before_quorum_zookeeper_long.sql similarity index 100% rename from tests/queries/0_stateless/00732_quorum_insert_have_data_before_quorum_zookeeper.sql rename to tests/queries/0_stateless/00732_quorum_insert_have_data_before_quorum_zookeeper_long.sql diff --git a/tests/queries/0_stateless/00732_quorum_insert_lost_part_zookeeper.reference b/tests/queries/0_stateless/00732_quorum_insert_lost_part_zookeeper_long.reference similarity index 100% rename from tests/queries/0_stateless/00732_quorum_insert_lost_part_zookeeper.reference rename to tests/queries/0_stateless/00732_quorum_insert_lost_part_zookeeper_long.reference diff --git a/tests/queries/0_stateless/00732_quorum_insert_lost_part_zookeeper.sql b/tests/queries/0_stateless/00732_quorum_insert_lost_part_zookeeper_long.sql similarity index 100% rename from tests/queries/0_stateless/00732_quorum_insert_lost_part_zookeeper.sql rename to tests/queries/0_stateless/00732_quorum_insert_lost_part_zookeeper_long.sql diff --git a/tests/queries/0_stateless/00732_quorum_insert_simple_test_2_parts_zookeeper.reference b/tests/queries/0_stateless/00732_quorum_insert_simple_test_2_parts_zookeeper_long.reference similarity index 100% rename from tests/queries/0_stateless/00732_quorum_insert_simple_test_2_parts_zookeeper.reference rename to tests/queries/0_stateless/00732_quorum_insert_simple_test_2_parts_zookeeper_long.reference diff --git a/tests/queries/0_stateless/00732_quorum_insert_simple_test_2_parts_zookeeper.sql b/tests/queries/0_stateless/00732_quorum_insert_simple_test_2_parts_zookeeper_long.sql similarity index 100% rename from tests/queries/0_stateless/00732_quorum_insert_simple_test_2_parts_zookeeper.sql rename to tests/queries/0_stateless/00732_quorum_insert_simple_test_2_parts_zookeeper_long.sql diff --git a/tests/queries/0_stateless/00925_zookeeper_empty_replicated_merge_tree_optimize_final.reference b/tests/queries/0_stateless/00925_zookeeper_empty_replicated_merge_tree_optimize_final_long.reference similarity index 100% rename from tests/queries/0_stateless/00925_zookeeper_empty_replicated_merge_tree_optimize_final.reference rename to tests/queries/0_stateless/00925_zookeeper_empty_replicated_merge_tree_optimize_final_long.reference diff --git a/tests/queries/0_stateless/00925_zookeeper_empty_replicated_merge_tree_optimize_final.sql b/tests/queries/0_stateless/00925_zookeeper_empty_replicated_merge_tree_optimize_final_long.sql similarity index 100% rename from tests/queries/0_stateless/00925_zookeeper_empty_replicated_merge_tree_optimize_final.sql rename to tests/queries/0_stateless/00925_zookeeper_empty_replicated_merge_tree_optimize_final_long.sql diff --git a/tests/queries/0_stateless/00933_ttl_replicated_zookeeper.reference b/tests/queries/0_stateless/00933_ttl_replicated_zookeeper_long.reference similarity index 100% rename from tests/queries/0_stateless/00933_ttl_replicated_zookeeper.reference rename to tests/queries/0_stateless/00933_ttl_replicated_zookeeper_long.reference diff --git a/tests/queries/0_stateless/00933_ttl_replicated_zookeeper.sql b/tests/queries/0_stateless/00933_ttl_replicated_zookeeper_long.sql similarity index 100% rename from tests/queries/0_stateless/00933_ttl_replicated_zookeeper.sql rename to tests/queries/0_stateless/00933_ttl_replicated_zookeeper_long.sql diff --git a/tests/queries/0_stateless/01062_alter_on_mutataion_zookeeper.reference b/tests/queries/0_stateless/01062_alter_on_mutataion_zookeeper_long.reference similarity index 100% rename from tests/queries/0_stateless/01062_alter_on_mutataion_zookeeper.reference rename to tests/queries/0_stateless/01062_alter_on_mutataion_zookeeper_long.reference diff --git a/tests/queries/0_stateless/01062_alter_on_mutataion_zookeeper.sql b/tests/queries/0_stateless/01062_alter_on_mutataion_zookeeper_long.sql similarity index 100% rename from tests/queries/0_stateless/01062_alter_on_mutataion_zookeeper.sql rename to tests/queries/0_stateless/01062_alter_on_mutataion_zookeeper_long.sql diff --git a/tests/queries/0_stateless/01213_alter_rename_with_default_zookeeper.reference b/tests/queries/0_stateless/01213_alter_rename_with_default_zookeeper_long.reference similarity index 100% rename from tests/queries/0_stateless/01213_alter_rename_with_default_zookeeper.reference rename to tests/queries/0_stateless/01213_alter_rename_with_default_zookeeper_long.reference diff --git a/tests/queries/0_stateless/01213_alter_rename_with_default_zookeeper.sql b/tests/queries/0_stateless/01213_alter_rename_with_default_zookeeper_long.sql similarity index 100% rename from tests/queries/0_stateless/01213_alter_rename_with_default_zookeeper.sql rename to tests/queries/0_stateless/01213_alter_rename_with_default_zookeeper_long.sql diff --git a/tests/queries/0_stateless/01267_alter_default_key_columns_zookeeper.reference b/tests/queries/0_stateless/01267_alter_default_key_columns_zookeeper_long.reference similarity index 100% rename from tests/queries/0_stateless/01267_alter_default_key_columns_zookeeper.reference rename to tests/queries/0_stateless/01267_alter_default_key_columns_zookeeper_long.reference diff --git a/tests/queries/0_stateless/01267_alter_default_key_columns_zookeeper.sql b/tests/queries/0_stateless/01267_alter_default_key_columns_zookeeper_long.sql similarity index 100% rename from tests/queries/0_stateless/01267_alter_default_key_columns_zookeeper.sql rename to tests/queries/0_stateless/01267_alter_default_key_columns_zookeeper_long.sql diff --git a/tests/queries/0_stateless/01319_manual_write_to_replicas.reference b/tests/queries/0_stateless/01319_manual_write_to_replicas_long.reference similarity index 100% rename from tests/queries/0_stateless/01319_manual_write_to_replicas.reference rename to tests/queries/0_stateless/01319_manual_write_to_replicas_long.reference diff --git a/tests/queries/0_stateless/01319_manual_write_to_replicas.sql b/tests/queries/0_stateless/01319_manual_write_to_replicas_long.sql similarity index 100% rename from tests/queries/0_stateless/01319_manual_write_to_replicas.sql rename to tests/queries/0_stateless/01319_manual_write_to_replicas_long.sql diff --git a/tests/queries/0_stateless/01669_columns_declaration_serde.reference b/tests/queries/0_stateless/01669_columns_declaration_serde_long.reference similarity index 100% rename from tests/queries/0_stateless/01669_columns_declaration_serde.reference rename to tests/queries/0_stateless/01669_columns_declaration_serde_long.reference diff --git a/tests/queries/0_stateless/01669_columns_declaration_serde.sql b/tests/queries/0_stateless/01669_columns_declaration_serde_long.sql similarity index 100% rename from tests/queries/0_stateless/01669_columns_declaration_serde.sql rename to tests/queries/0_stateless/01669_columns_declaration_serde_long.sql From a822e2d45361c90259671e66b7e3f8f48d5d2ff8 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 6 Aug 2021 02:06:26 +0300 Subject: [PATCH 178/599] Fix error --- src/Common/clickhouse_malloc.cpp | 12 ++++-------- 1 file changed, 4 insertions(+), 8 deletions(-) diff --git a/src/Common/clickhouse_malloc.cpp b/src/Common/clickhouse_malloc.cpp index cb6419ae6da..bb6ba7643b1 100644 --- a/src/Common/clickhouse_malloc.cpp +++ b/src/Common/clickhouse_malloc.cpp @@ -23,25 +23,21 @@ extern "C" void * clickhouse_calloc(size_t number_of_members, size_t size) extern "C" void * clickhouse_realloc(void * ptr, size_t size) { + if (ptr) + Memory::untrackMemory(ptr); void * res = realloc(ptr, size); if (res) - { - if (ptr) - Memory::untrackMemory(ptr); Memory::trackMemory(size); - } return res; } extern "C" void * clickhouse_reallocarray(void * ptr, size_t number_of_members, size_t size) { + if (ptr) + Memory::untrackMemory(ptr); void * res = reallocarray(ptr, number_of_members, size); if (res) - { - if (ptr) - Memory::untrackMemory(ptr); Memory::trackMemory(number_of_members * size); - } return res; } From 35062b2c1224bc4b17dbcdab17d7370283763a1b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 6 Aug 2021 02:10:55 +0300 Subject: [PATCH 179/599] Fix compatibility --- src/Common/clickhouse_malloc.cpp | 11 +++++------ 1 file changed, 5 insertions(+), 6 deletions(-) diff --git a/src/Common/clickhouse_malloc.cpp b/src/Common/clickhouse_malloc.cpp index bb6ba7643b1..3f69ebdf58d 100644 --- a/src/Common/clickhouse_malloc.cpp +++ b/src/Common/clickhouse_malloc.cpp @@ -33,12 +33,11 @@ extern "C" void * clickhouse_realloc(void * ptr, size_t size) extern "C" void * clickhouse_reallocarray(void * ptr, size_t number_of_members, size_t size) { - if (ptr) - Memory::untrackMemory(ptr); - void * res = reallocarray(ptr, number_of_members, size); - if (res) - Memory::trackMemory(number_of_members * size); - return res; + size_t real_size = 0; + if (__builtin_mul_overflow(number_of_members, size, &real_size)) + return nullptr; + + return clickhouse_realloc(ptr, real_size); } extern "C" void clickhouse_free(void * ptr) From c5beecd0aaaec9468836a7a54d5259d006cff456 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E8=91=A3=E6=B5=B7=E9=95=94?= Date: Fri, 6 Aug 2021 09:59:18 +0800 Subject: [PATCH 180/599] update code style --- src/AggregateFunctions/AggregateFunctionGroupBitmapData.h | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/AggregateFunctions/AggregateFunctionGroupBitmapData.h b/src/AggregateFunctions/AggregateFunctionGroupBitmapData.h index 8e5a1b34104..972d137042e 100644 --- a/src/AggregateFunctions/AggregateFunctionGroupBitmapData.h +++ b/src/AggregateFunctions/AggregateFunctionGroupBitmapData.h @@ -594,7 +594,9 @@ public: if (offset_count >= offset) { answer.push_back(val); - } else { + } + else + { offset_count++; } } From 6567aeda6c63a911a3bf2b3e515782ab9fbc9a60 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 6 Aug 2021 07:56:23 +0300 Subject: [PATCH 181/599] Mark more tests as long --- ...754_alter_modify_order_by_replicated_zookeeper_long.reference} | 0 ... => 00754_alter_modify_order_by_replicated_zookeeper_long.sql} | 0 ...rence => 01049_zookeeper_synchronous_mutations_long.reference} | 0 ...tations.sql => 01049_zookeeper_synchronous_mutations_long.sql} | 0 ..._without_select_sequence_consistency_zookeeper_long.reference} | 0 ..._count_without_select_sequence_consistency_zookeeper_long.sql} | 0 .../{01666_blns.reference => 01666_blns_long.reference} | 0 tests/queries/0_stateless/{01666_blns.sql => 01666_blns_long.sql} | 0 ... => 01715_background_checker_blather_zookeeper_long.reference} | 0 ...er.sql => 01715_background_checker_blather_zookeeper_long.sql} | 0 10 files changed, 0 insertions(+), 0 deletions(-) rename tests/queries/0_stateless/{00754_alter_modify_order_by_replicated_zookeeper.reference => 00754_alter_modify_order_by_replicated_zookeeper_long.reference} (100%) rename tests/queries/0_stateless/{00754_alter_modify_order_by_replicated_zookeeper.sql => 00754_alter_modify_order_by_replicated_zookeeper_long.sql} (100%) rename tests/queries/0_stateless/{01049_zookeeper_synchronous_mutations.reference => 01049_zookeeper_synchronous_mutations_long.reference} (100%) rename tests/queries/0_stateless/{01049_zookeeper_synchronous_mutations.sql => 01049_zookeeper_synchronous_mutations_long.sql} (100%) rename tests/queries/0_stateless/{01513_count_without_select_sequence_consistency_zookeeper.reference => 01513_count_without_select_sequence_consistency_zookeeper_long.reference} (100%) rename tests/queries/0_stateless/{01513_count_without_select_sequence_consistency_zookeeper.sql => 01513_count_without_select_sequence_consistency_zookeeper_long.sql} (100%) rename tests/queries/0_stateless/{01666_blns.reference => 01666_blns_long.reference} (100%) rename tests/queries/0_stateless/{01666_blns.sql => 01666_blns_long.sql} (100%) rename tests/queries/0_stateless/{01715_background_checker_blather_zookeeper.reference => 01715_background_checker_blather_zookeeper_long.reference} (100%) rename tests/queries/0_stateless/{01715_background_checker_blather_zookeeper.sql => 01715_background_checker_blather_zookeeper_long.sql} (100%) diff --git a/tests/queries/0_stateless/00754_alter_modify_order_by_replicated_zookeeper.reference b/tests/queries/0_stateless/00754_alter_modify_order_by_replicated_zookeeper_long.reference similarity index 100% rename from tests/queries/0_stateless/00754_alter_modify_order_by_replicated_zookeeper.reference rename to tests/queries/0_stateless/00754_alter_modify_order_by_replicated_zookeeper_long.reference diff --git a/tests/queries/0_stateless/00754_alter_modify_order_by_replicated_zookeeper.sql b/tests/queries/0_stateless/00754_alter_modify_order_by_replicated_zookeeper_long.sql similarity index 100% rename from tests/queries/0_stateless/00754_alter_modify_order_by_replicated_zookeeper.sql rename to tests/queries/0_stateless/00754_alter_modify_order_by_replicated_zookeeper_long.sql diff --git a/tests/queries/0_stateless/01049_zookeeper_synchronous_mutations.reference b/tests/queries/0_stateless/01049_zookeeper_synchronous_mutations_long.reference similarity index 100% rename from tests/queries/0_stateless/01049_zookeeper_synchronous_mutations.reference rename to tests/queries/0_stateless/01049_zookeeper_synchronous_mutations_long.reference diff --git a/tests/queries/0_stateless/01049_zookeeper_synchronous_mutations.sql b/tests/queries/0_stateless/01049_zookeeper_synchronous_mutations_long.sql similarity index 100% rename from tests/queries/0_stateless/01049_zookeeper_synchronous_mutations.sql rename to tests/queries/0_stateless/01049_zookeeper_synchronous_mutations_long.sql diff --git a/tests/queries/0_stateless/01513_count_without_select_sequence_consistency_zookeeper.reference b/tests/queries/0_stateless/01513_count_without_select_sequence_consistency_zookeeper_long.reference similarity index 100% rename from tests/queries/0_stateless/01513_count_without_select_sequence_consistency_zookeeper.reference rename to tests/queries/0_stateless/01513_count_without_select_sequence_consistency_zookeeper_long.reference diff --git a/tests/queries/0_stateless/01513_count_without_select_sequence_consistency_zookeeper.sql b/tests/queries/0_stateless/01513_count_without_select_sequence_consistency_zookeeper_long.sql similarity index 100% rename from tests/queries/0_stateless/01513_count_without_select_sequence_consistency_zookeeper.sql rename to tests/queries/0_stateless/01513_count_without_select_sequence_consistency_zookeeper_long.sql diff --git a/tests/queries/0_stateless/01666_blns.reference b/tests/queries/0_stateless/01666_blns_long.reference similarity index 100% rename from tests/queries/0_stateless/01666_blns.reference rename to tests/queries/0_stateless/01666_blns_long.reference diff --git a/tests/queries/0_stateless/01666_blns.sql b/tests/queries/0_stateless/01666_blns_long.sql similarity index 100% rename from tests/queries/0_stateless/01666_blns.sql rename to tests/queries/0_stateless/01666_blns_long.sql diff --git a/tests/queries/0_stateless/01715_background_checker_blather_zookeeper.reference b/tests/queries/0_stateless/01715_background_checker_blather_zookeeper_long.reference similarity index 100% rename from tests/queries/0_stateless/01715_background_checker_blather_zookeeper.reference rename to tests/queries/0_stateless/01715_background_checker_blather_zookeeper_long.reference diff --git a/tests/queries/0_stateless/01715_background_checker_blather_zookeeper.sql b/tests/queries/0_stateless/01715_background_checker_blather_zookeeper_long.sql similarity index 100% rename from tests/queries/0_stateless/01715_background_checker_blather_zookeeper.sql rename to tests/queries/0_stateless/01715_background_checker_blather_zookeeper_long.sql From c523904b746b70a941f802ff38031170880db489 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 6 Aug 2021 14:00:07 +0300 Subject: [PATCH 182/599] Mark more tests as long --- ...nce => 00837_minmax_index_replicated_zookeeper_long.reference} | 0 ...eeper.sql => 00837_minmax_index_replicated_zookeeper_long.sql} | 0 ...01277_alter_rename_column_constraint_zookeeper_long.reference} | 0 ...ql => 01277_alter_rename_column_constraint_zookeeper_long.sql} | 0 ...ce => 01509_parallel_quorum_insert_no_replicas_long.reference} | 0 ...icas.sql => 01509_parallel_quorum_insert_no_replicas_long.sql} | 0 6 files changed, 0 insertions(+), 0 deletions(-) rename tests/queries/0_stateless/{00837_minmax_index_replicated_zookeeper.reference => 00837_minmax_index_replicated_zookeeper_long.reference} (100%) rename tests/queries/0_stateless/{00837_minmax_index_replicated_zookeeper.sql => 00837_minmax_index_replicated_zookeeper_long.sql} (100%) rename tests/queries/0_stateless/{01277_alter_rename_column_constraint_zookeeper.reference => 01277_alter_rename_column_constraint_zookeeper_long.reference} (100%) rename tests/queries/0_stateless/{01277_alter_rename_column_constraint_zookeeper.sql => 01277_alter_rename_column_constraint_zookeeper_long.sql} (100%) rename tests/queries/0_stateless/{01509_parallel_quorum_insert_no_replicas.reference => 01509_parallel_quorum_insert_no_replicas_long.reference} (100%) rename tests/queries/0_stateless/{01509_parallel_quorum_insert_no_replicas.sql => 01509_parallel_quorum_insert_no_replicas_long.sql} (100%) diff --git a/tests/queries/0_stateless/00837_minmax_index_replicated_zookeeper.reference b/tests/queries/0_stateless/00837_minmax_index_replicated_zookeeper_long.reference similarity index 100% rename from tests/queries/0_stateless/00837_minmax_index_replicated_zookeeper.reference rename to tests/queries/0_stateless/00837_minmax_index_replicated_zookeeper_long.reference diff --git a/tests/queries/0_stateless/00837_minmax_index_replicated_zookeeper.sql b/tests/queries/0_stateless/00837_minmax_index_replicated_zookeeper_long.sql similarity index 100% rename from tests/queries/0_stateless/00837_minmax_index_replicated_zookeeper.sql rename to tests/queries/0_stateless/00837_minmax_index_replicated_zookeeper_long.sql diff --git a/tests/queries/0_stateless/01277_alter_rename_column_constraint_zookeeper.reference b/tests/queries/0_stateless/01277_alter_rename_column_constraint_zookeeper_long.reference similarity index 100% rename from tests/queries/0_stateless/01277_alter_rename_column_constraint_zookeeper.reference rename to tests/queries/0_stateless/01277_alter_rename_column_constraint_zookeeper_long.reference diff --git a/tests/queries/0_stateless/01277_alter_rename_column_constraint_zookeeper.sql b/tests/queries/0_stateless/01277_alter_rename_column_constraint_zookeeper_long.sql similarity index 100% rename from tests/queries/0_stateless/01277_alter_rename_column_constraint_zookeeper.sql rename to tests/queries/0_stateless/01277_alter_rename_column_constraint_zookeeper_long.sql diff --git a/tests/queries/0_stateless/01509_parallel_quorum_insert_no_replicas.reference b/tests/queries/0_stateless/01509_parallel_quorum_insert_no_replicas_long.reference similarity index 100% rename from tests/queries/0_stateless/01509_parallel_quorum_insert_no_replicas.reference rename to tests/queries/0_stateless/01509_parallel_quorum_insert_no_replicas_long.reference diff --git a/tests/queries/0_stateless/01509_parallel_quorum_insert_no_replicas.sql b/tests/queries/0_stateless/01509_parallel_quorum_insert_no_replicas_long.sql similarity index 100% rename from tests/queries/0_stateless/01509_parallel_quorum_insert_no_replicas.sql rename to tests/queries/0_stateless/01509_parallel_quorum_insert_no_replicas_long.sql From 6747b8a03bfc98982cc742ff50af3162de6fd261 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 6 Aug 2021 07:07:50 +0300 Subject: [PATCH 183/599] Mark more tests as long --- ...orum_insert_lost_part_and_alive_part_zookeeper_long.reference} | 0 ...732_quorum_insert_lost_part_and_alive_part_zookeeper_long.sql} | 0 ...nce => 00988_constraints_replication_zookeeper_long.reference} | 0 ...eeper.sql => 00988_constraints_replication_zookeeper_long.sql} | 0 ...per.reference => 01079_alter_default_zookeeper_long.reference} | 0 ...fault_zookeeper.sql => 01079_alter_default_zookeeper_long.sql} | 0 ..._drop_column_compact_part_replicated_zookeeper_long.reference} | 0 ... 01201_drop_column_compact_part_replicated_zookeeper_long.sql} | 0 8 files changed, 0 insertions(+), 0 deletions(-) rename tests/queries/0_stateless/{00732_quorum_insert_lost_part_and_alive_part_zookeeper.reference => 00732_quorum_insert_lost_part_and_alive_part_zookeeper_long.reference} (100%) rename tests/queries/0_stateless/{00732_quorum_insert_lost_part_and_alive_part_zookeeper.sql => 00732_quorum_insert_lost_part_and_alive_part_zookeeper_long.sql} (100%) rename tests/queries/0_stateless/{00988_constraints_replication_zookeeper.reference => 00988_constraints_replication_zookeeper_long.reference} (100%) rename tests/queries/0_stateless/{00988_constraints_replication_zookeeper.sql => 00988_constraints_replication_zookeeper_long.sql} (100%) rename tests/queries/0_stateless/{01079_alter_default_zookeeper.reference => 01079_alter_default_zookeeper_long.reference} (100%) rename tests/queries/0_stateless/{01079_alter_default_zookeeper.sql => 01079_alter_default_zookeeper_long.sql} (100%) rename tests/queries/0_stateless/{01201_drop_column_compact_part_replicated_zookeeper.reference => 01201_drop_column_compact_part_replicated_zookeeper_long.reference} (100%) rename tests/queries/0_stateless/{01201_drop_column_compact_part_replicated_zookeeper.sql => 01201_drop_column_compact_part_replicated_zookeeper_long.sql} (100%) diff --git a/tests/queries/0_stateless/00732_quorum_insert_lost_part_and_alive_part_zookeeper.reference b/tests/queries/0_stateless/00732_quorum_insert_lost_part_and_alive_part_zookeeper_long.reference similarity index 100% rename from tests/queries/0_stateless/00732_quorum_insert_lost_part_and_alive_part_zookeeper.reference rename to tests/queries/0_stateless/00732_quorum_insert_lost_part_and_alive_part_zookeeper_long.reference diff --git a/tests/queries/0_stateless/00732_quorum_insert_lost_part_and_alive_part_zookeeper.sql b/tests/queries/0_stateless/00732_quorum_insert_lost_part_and_alive_part_zookeeper_long.sql similarity index 100% rename from tests/queries/0_stateless/00732_quorum_insert_lost_part_and_alive_part_zookeeper.sql rename to tests/queries/0_stateless/00732_quorum_insert_lost_part_and_alive_part_zookeeper_long.sql diff --git a/tests/queries/0_stateless/00988_constraints_replication_zookeeper.reference b/tests/queries/0_stateless/00988_constraints_replication_zookeeper_long.reference similarity index 100% rename from tests/queries/0_stateless/00988_constraints_replication_zookeeper.reference rename to tests/queries/0_stateless/00988_constraints_replication_zookeeper_long.reference diff --git a/tests/queries/0_stateless/00988_constraints_replication_zookeeper.sql b/tests/queries/0_stateless/00988_constraints_replication_zookeeper_long.sql similarity index 100% rename from tests/queries/0_stateless/00988_constraints_replication_zookeeper.sql rename to tests/queries/0_stateless/00988_constraints_replication_zookeeper_long.sql diff --git a/tests/queries/0_stateless/01079_alter_default_zookeeper.reference b/tests/queries/0_stateless/01079_alter_default_zookeeper_long.reference similarity index 100% rename from tests/queries/0_stateless/01079_alter_default_zookeeper.reference rename to tests/queries/0_stateless/01079_alter_default_zookeeper_long.reference diff --git a/tests/queries/0_stateless/01079_alter_default_zookeeper.sql b/tests/queries/0_stateless/01079_alter_default_zookeeper_long.sql similarity index 100% rename from tests/queries/0_stateless/01079_alter_default_zookeeper.sql rename to tests/queries/0_stateless/01079_alter_default_zookeeper_long.sql diff --git a/tests/queries/0_stateless/01201_drop_column_compact_part_replicated_zookeeper.reference b/tests/queries/0_stateless/01201_drop_column_compact_part_replicated_zookeeper_long.reference similarity index 100% rename from tests/queries/0_stateless/01201_drop_column_compact_part_replicated_zookeeper.reference rename to tests/queries/0_stateless/01201_drop_column_compact_part_replicated_zookeeper_long.reference diff --git a/tests/queries/0_stateless/01201_drop_column_compact_part_replicated_zookeeper.sql b/tests/queries/0_stateless/01201_drop_column_compact_part_replicated_zookeeper_long.sql similarity index 100% rename from tests/queries/0_stateless/01201_drop_column_compact_part_replicated_zookeeper.sql rename to tests/queries/0_stateless/01201_drop_column_compact_part_replicated_zookeeper_long.sql From 83668d1bddc636eecae8669e9de0188d31bced37 Mon Sep 17 00:00:00 2001 From: Anna <42538400+adevyatova@users.noreply.github.com> Date: Fri, 6 Aug 2021 08:23:59 +0300 Subject: [PATCH 184/599] Update docs/en/sql-reference/functions/other-functions.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/en/sql-reference/functions/other-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index 446c7e6714c..b9172b088f8 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -2140,7 +2140,7 @@ Result: ## currentRoles {#current-roles} -Returns the names of the roles which are current for the current user. The command [SET ROLE](../../sql-reference/statements/set-role.md#set-role-statement) could be used to change the current roles. If the command `SET ROLE` hasn't been used the function `currentRoles` returns the same as `defaultRoles`. +Returns the names of the roles which are current for the current user. The current roles can be changed by the [SET ROLE](../../sql-reference/statements/set-role.md#set-role-statement) statement. If the `SET ROLE` statement was not used, the function `currentRoles` returns the same as `defaultRoles`. **Syntax** From 4a0b480294f21600cd17c189ab6b4a34721a91af Mon Sep 17 00:00:00 2001 From: Anna <42538400+adevyatova@users.noreply.github.com> Date: Fri, 6 Aug 2021 08:24:13 +0300 Subject: [PATCH 185/599] Update docs/en/sql-reference/functions/other-functions.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/en/sql-reference/functions/other-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index b9172b088f8..0ce13debca4 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -2156,7 +2156,7 @@ Type: [Array](../../sql-reference/data-types/array.md)([String](../../sql-refere ## enabledRoles {#enabled-roles} -Returns the names of the current roles merged with those roles which are granted to some of the current roles. +Returns the names of the current roles and the roles, granted to some of the current roles. **Syntax** From ea049f8bd4212e80a33952c2c63a284c24089672 Mon Sep 17 00:00:00 2001 From: Anna <42538400+adevyatova@users.noreply.github.com> Date: Fri, 6 Aug 2021 08:30:06 +0300 Subject: [PATCH 186/599] Update docs/ru/sql-reference/functions/other-functions.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/sql-reference/functions/other-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/other-functions.md b/docs/ru/sql-reference/functions/other-functions.md index f5e583e456c..1c6207ddb7c 100644 --- a/docs/ru/sql-reference/functions/other-functions.md +++ b/docs/ru/sql-reference/functions/other-functions.md @@ -2106,7 +2106,7 @@ currentRoles() ## enabledRoles {#enabled-roles} -Возвращает имена текущих ролей вместе с теми ролями, которые разрешено использовать текущему пользователю путем назначения привилегий. +Возвращает имена текущих ролей, а также ролей, которые разрешено использовать текущему пользователю путем назначения привилегий. **Синтаксис** From c7a5f528867064450a1f249cc36fafc44122bb1a Mon Sep 17 00:00:00 2001 From: Anna <42538400+adevyatova@users.noreply.github.com> Date: Fri, 6 Aug 2021 08:30:30 +0300 Subject: [PATCH 187/599] Update docs/ru/sql-reference/functions/other-functions.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/sql-reference/functions/other-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/other-functions.md b/docs/ru/sql-reference/functions/other-functions.md index 1c6207ddb7c..2cda1b6a6e8 100644 --- a/docs/ru/sql-reference/functions/other-functions.md +++ b/docs/ru/sql-reference/functions/other-functions.md @@ -2090,7 +2090,7 @@ SELECT tcpPort(); ## currentRoles {#current-roles} -Возвращает имена текущих ролей для текущего пользователя. Для изменения текущих ролей может использоваться команда [SET ROLE](../../sql-reference/statements/set-role.md#set-role-statement). Если команда `SET ROLE` не использовалась, функция возвращает тот же результат, что и функция [defaultRoles](#default-roles). +Возвращает список текущих ролей для текущего пользователя. Список ролей пользователя можно изменить с помощью выражения [SET ROLE](../../sql-reference/statements/set-role.md#set-role-statement). Если выражение `SET ROLE` не использовалось, данная функция возвращает тот же результат, что и функция [defaultRoles](#default-roles). **Синтаксис** From c1bbcf256ab3ea9ab37898f43413aeac89176b09 Mon Sep 17 00:00:00 2001 From: Anna <42538400+adevyatova@users.noreply.github.com> Date: Fri, 6 Aug 2021 09:54:19 +0300 Subject: [PATCH 188/599] Update docs/ru/sql-reference/functions/other-functions.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/sql-reference/functions/other-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/other-functions.md b/docs/ru/sql-reference/functions/other-functions.md index 2cda1b6a6e8..72f543f1cbb 100644 --- a/docs/ru/sql-reference/functions/other-functions.md +++ b/docs/ru/sql-reference/functions/other-functions.md @@ -2122,7 +2122,7 @@ enabledRoles() ## defaultRoles {#default-roles} -Возвращает имена ролей, которые задаются как текущие для данного пользователя при входе в систему. Они могут быть изменены при помощи команды [SET DEFAULT ROLE](../../sql-reference/statements/set-role.md#set-default-role-statement). По умолчанию это все роли, которые разрешено использовать текущему пользователю (см. команду [GRANT](../../sql-reference/statements/grant/#grant-select)). +Возвращает имена ролей, которые задаются по умолчанию для текущего пользователя при входе в систему. Изначально это все роли, которые разрешено использовать текущему пользователю (см. [GRANT](../../sql-reference/statements/grant/#grant-select)). Список ролей по умолчанию может быть изменен с помощью выражения [SET DEFAULT ROLE](../../sql-reference/statements/set-role.md#set-default-role-statement). **Синтаксис** From d0e678d72f2b2465ac616cb853ea561a97f1c3a7 Mon Sep 17 00:00:00 2001 From: Anna <42538400+adevyatova@users.noreply.github.com> Date: Fri, 6 Aug 2021 09:54:43 +0300 Subject: [PATCH 189/599] Update docs/en/sql-reference/functions/other-functions.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/en/sql-reference/functions/other-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index 0ce13debca4..47e92c253b1 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -2172,7 +2172,7 @@ Type: [Array](../../sql-reference/data-types/array.md)([String](../../sql-refere ## defaultRoles {#default-roles} -Returns the names of the roles which are set as current when the current user logins. The command [SET DEFAULT ROLE](../../sql-reference/statements/set-role.md#set-default-role-statement) could be used to change the default roles. By default the default roles are all the roles granted to the current user. +Returns the names of the roles which are enabled by default for the current user when he logins. Initially these are all roles granted to the current user (see [GRANT](../../sql-reference/statements/grant/#grant-select)). The list of the user roles can be changed lately with the [SET DEFAULT ROLE](../../sql-reference/statements/set-role.md#set-default-role-statement) statement. **Syntax** From a6ad811652a0757b972b96f572a97502c2f60b24 Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Fri, 6 Aug 2021 11:29:03 +0300 Subject: [PATCH 190/599] Update MySql settings. --- .../database-engines/materialized-mysql.md | 22 ++++++++------- .../database-engines/materialized-mysql.md | 27 ++++++++++++++++--- 2 files changed, 35 insertions(+), 14 deletions(-) diff --git a/docs/en/engines/database-engines/materialized-mysql.md b/docs/en/engines/database-engines/materialized-mysql.md index 57bc1fe44a2..1fad4414d57 100644 --- a/docs/en/engines/database-engines/materialized-mysql.md +++ b/docs/en/engines/database-engines/materialized-mysql.md @@ -3,7 +3,7 @@ toc_priority: 29 toc_title: MaterializedMySQL --- -# MaterializedMySQL {#materialized-mysql} +# [experimental] MaterializedMySQL {#materialized-mysql} **This is experimental feature that should not be used in production.** @@ -28,14 +28,16 @@ ENGINE = MaterializedMySQL('host:port', ['database' | database], 'user', 'passwo - `password` — User password. **Engine Settings** -- `max_rows_in_buffer` — Max rows that data is allowed to cache in memory(for single table and the cache data unable to query). when rows is exceeded, the data will be materialized. Default: `65505`. -- `max_bytes_in_buffer` — Max bytes that data is allowed to cache in memory(for single table and the cache data unable to query). when rows is exceeded, the data will be materialized. Default: `1048576`. -- `max_rows_in_buffers` — Max rows that data is allowed to cache in memory(for database and the cache data unable to query). when rows is exceeded, the data will be materialized. Default: `65505`. -- `max_bytes_in_buffers` — Max bytes that data is allowed to cache in memory(for database and the cache data unable to query). when rows is exceeded, the data will be materialized. Default: `1048576`. -- `max_flush_data_time` — Max milliseconds that data is allowed to cache in memory(for database and the cache data unable to query). when this time is exceeded, the data will be materialized. Default: `1000`. + +- `max_rows_in_buffer` — Max rows that data is allowed to cache in memory (for single table and the cache data unable to query). When rows is exceeded, the data will be materialized. Default: `65 505`. +- `max_bytes_in_buffer` — Max bytes that data is allowed to cache in memory (for single table and the cache data unable to query). when rows is exceeded, the data will be materialized. Default: `1 048 576`. +- `max_rows_in_buffers` — Max rows that data is allowed to cache in memory (for database and the cache data unable to query). when rows is exceeded, the data will be materialized. Default: `65505`. +- `max_bytes_in_buffers` — Max bytes that data is allowed to cache in memory (for database and the cache data unable to query). when rows is exceeded, the data will be materialized. Default: `1 048 576`. +- `max_flush_data_time` — Max milliseconds that data is allowed to cache in memory (for database and the cache data unable to query). when this time is exceeded, the data will be materialized. Default: `1000`. - `max_wait_time_when_mysql_unavailable` — Retry interval when MySQL is not available (milliseconds). Negative value disable retry. Default: `1000`. -- `allows_query_when_mysql_lost` — Allow query materialized table when mysql is lost. Default: `0` (`false`). -``` +- `allows_query_when_mysql_lost` — Allow query materialized table when MySql is lost. Default: `0` (`false`). + +```sql CREATE DATABASE mysql ENGINE = MaterializedMySQL('localhost:3306', 'db', 'user', '***') SETTINGS allows_query_when_mysql_lost=true, @@ -82,10 +84,10 @@ When working with the `MaterializedMySQL` database engine, [ReplacingMergeTree]( | BLOB | [String](../../sql-reference/data-types/string.md) | | BINARY | [FixedString](../../sql-reference/data-types/fixedstring.md) | -Other types are not supported. If MySQL table contains a column of such type, ClickHouse throws exception "Unhandled data type" and stops replication. - [Nullable](../../sql-reference/data-types/nullable.md) is supported. +Other types are not supported. If MySQL table contains a column of such type, ClickHouse throws exception "Unhandled data type" and stops replication. + ## Specifics and Recommendations {#specifics-and-recommendations} ### Compatibility restrictions {#compatibility-restrictions} diff --git a/docs/ru/engines/database-engines/materialized-mysql.md b/docs/ru/engines/database-engines/materialized-mysql.md index 761a5ef4aaa..b886dc9b4e7 100644 --- a/docs/ru/engines/database-engines/materialized-mysql.md +++ b/docs/ru/engines/database-engines/materialized-mysql.md @@ -1,10 +1,11 @@ - --- toc_priority: 29 toc_title: MaterializedMySQL --- -# MaterializedMySQL {#materialized-mysql} +# [experimental] MaterializedMySQL {#materialized-mysql} + +**Это экспериментальная функция, которую не следует использовать в продуктивной среде.** Создает базу данных ClickHouse со всеми таблицами, существующими в MySQL, и всеми данными в этих таблицах. @@ -26,6 +27,23 @@ ENGINE = MaterializedMySQL('host:port', ['database' | database], 'user', 'passwo - `user` — пользователь MySQL. - `password` — пароль пользователя. +**Настройки движка** + +- `max_rows_in_buffer` — максимальное количество строк, содержимое которых может кэшироваться в памяти (для одной таблицы и данных кэша, которые невозможно запросить). При превышении количества строк, данные будут материализованы. Значение по умолчанию: `65 505`. +- `max_bytes_in_buffer` — максимальное количество байтов, которое разрешено кэшировать в памяти (для одной таблицы и данных кэша, которые невозможно запросить). При превышении количества строк, данные будут материализованы. Значение по умолчанию: `1 048 576`. +- `max_rows_in_buffers` — максимальное количество строк, содержимое которых может кэшироваться в памяти (для базы данных и данных кэша, которые невозможно запросить). При превышении количества строк, данные будут материализованы. Значение по умолчанию: `65 505`. +- `max_bytes_in_buffers` — максимальное количество байтов, которое разрешено кэшировать данным в памяти (для базы данных и данных кэша, которые невозможно запросить). При превышении количества строк, данные будут материализованы. Значение по умолчанию: `1 048 576`. +- `max_flush_data_time` — Максимальное время в миллисекундах, в течение которого разрешено кэшировать данные в памяти (для базы данных и данных кэша, которые невозможно запросить). При превышении количества указанного периода, данные будут материализованы. Значение по умолчанию: `1000`. +- `max_wait_time_when_mysql_unavailable` — интервал повторных попыток, если MySQL недоступен. Указывается в миллисекундах. Отрицательное значение отключает повторные попытки. Значение по умолчанию: `1000`. +- `allows_query_when_mysql_lost` — разрешить запрос материализованной таблицы при потере MySQL. Значение по умолчанию: `0` (`false`). + +```sql +CREATE DATABASE mysql ENGINE = MaterializedMySQL('localhost:3306', 'db', 'user', '***') + SETTINGS + allows_query_when_mysql_lost=true, + max_wait_time_when_mysql_unavailable=10000; +``` + **Настройки на стороне MySQL-сервера** Для правильной работы `MaterializeMySQL` следует обязательно указать на MySQL сервере следующие параметры конфигурации: @@ -65,10 +83,10 @@ ENGINE = MaterializedMySQL('host:port', ['database' | database], 'user', 'passwo | BLOB | [String](../../sql-reference/data-types/string.md) | | BINARY | [FixedString](../../sql-reference/data-types/fixedstring.md) | -Другие типы не поддерживаются. Если таблица MySQL содержит столбец другого типа, ClickHouse выдаст исключение "Неподдерживаемый тип данных" ("Unhandled data type") и остановит репликацию. - Тип [Nullable](../../sql-reference/data-types/nullable.md) поддерживается. +Другие типы не поддерживаются. Если таблица MySQL содержит столбец другого типа, ClickHouse выдаст исключение "Неподдерживаемый тип данных" ("Unhandled data type") и остановит репликацию. + ## Особенности и рекомендации {#specifics-and-recommendations} ### Ограничения совместимости {#compatibility-restrictions} @@ -178,3 +196,4 @@ SELECT * FROM mysql.test; └───┴─────┴──────┘ ``` +[Оригинальная статья](https://clickhouse.tech/docs/ru/engines/database-engines/materialized-mysql/) From afd55921ea81ef03f38abd01dfb6b45cdaa4f14a Mon Sep 17 00:00:00 2001 From: filimonov <1549571+filimonov@users.noreply.github.com> Date: Fri, 6 Aug 2021 10:31:00 +0200 Subject: [PATCH 191/599] Changelog for 21.8 History of edits: https://gist.github.com/johnhummelAltinity/917a7bc9183d73558ada92930c2ed941/revisions --- CHANGELOG.md | 116 +++++++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 116 insertions(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 34d11c6a2cd..7b760f15746 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,3 +1,119 @@ +### ClickHouse release v21.8.2.19-prestable FIXME!! as compared to v21.7.5.29-stable + +#### Backward Incompatible Changes + +* Use Map data type for system logs tables (`system.query_log`, `system.query_thread_log`, `system.processes`, `system.OpenTelemetrySpanLog`). These tables will be auto-created with new datatypes; old queries may be incompatible. [#23934](https://github.com/ClickHouse/ClickHouse/pull/23934) ([hexiaoting](https://github.com/hexiaoting)). + +#### New Features + +* Add new functions `leftPad()`, `rightPad()`, `leftPadUTF8()`, `rightPadUTF8()`. [#26075](https://github.com/ClickHouse/ClickHouse/pull/26075) ([Vitaly Baranov](https://github.com/vitlibar)). +* Render pipelines as graphs in Web UI if `EXPLAIN PIPELINE graph = 1` query is submitted. [#26067](https://github.com/ClickHouse/ClickHouse/pull/26067) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Add the `FIRST` keyword to the `ADD INDEX` command to be able to add the index at the beginning of the indices list. [#25904](https://github.com/ClickHouse/ClickHouse/pull/25904) ([xjewer](https://github.com/xjewer)). +* Introduce `system.data_skipping_indices` table containing information about existing data skipping indices. This closes [#7659](https://github.com/ClickHouse/ClickHouse/issues/7659). [#25693](https://github.com/ClickHouse/ClickHouse/pull/25693) ([Dmitry Novik](https://github.com/novikd)). +* Add `bin`/`unbin` functions. [#25609](https://github.com/ClickHouse/ClickHouse/pull/25609) ([zhaoyu](https://github.com/zxc111)). +* Support Map type in `mapAdd` and `mapSubtract` functions, add support for `(U)Int128`, `U(Int256)` types in `mapAdd` and `mapSubtract` functions. [#25596](https://github.com/ClickHouse/ClickHouse/pull/25596) ([Ildus Kurbangaliev](https://github.com/ildus)). +* Add support for `DISTINCT ON (columns)` expression, close [#25404](https://github.com/ClickHouse/ClickHouse/issues/25404). [#25589](https://github.com/ClickHouse/ClickHouse/pull/25589) ([Zijie Lu](https://github.com/TszKitLo40)). +* Collect common system metrics (in `system.asynchronous_metrics` and `system.asynchronous_metric_log`) on CPU usage, disk usage, memory usage, IO, network, files, load average, CPU frequencies, thermal sensors, EDAC counters, system uptime; also added metrics about the scheduling jitter and the time spent collecting the metrics. It works similar to `atop` in ClickHouse and allows access to monitoring data even if you have no additional tools installed. This closes [#9430](https://github.com/ClickHouse/ClickHouse/issues/9430). [#24416](https://github.com/ClickHouse/ClickHouse/pull/24416) ([Yegor Levankov](https://github.com/elevankoff)). +* Add support for a part of SQLJSON standard. [#24148](https://github.com/ClickHouse/ClickHouse/pull/24148) ([l1tsolaiki](https://github.com/l1tsolaiki)). +* Add MaterializedPostgreSQL table engine and database engine. This database engine allows replicating a whole database or any subset of database tables. [#20470](https://github.com/ClickHouse/ClickHouse/pull/20470) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Add an ability to reset a custom setting to default and remove it from the table's metadata. It allows rolling back the change without knowing the system/config's default. Closes [#14449](https://github.com/ClickHouse/ClickHouse/issues/14449). [#17769](https://github.com/ClickHouse/ClickHouse/pull/17769) ([xjewer](https://github.com/xjewer)). + +#### Performance Improvements + +* Improve latency of short queries that require reading from tables with a large number of columns. [#26371](https://github.com/ClickHouse/ClickHouse/pull/26371) ([Anton Popov](https://github.com/CurtizJ)). +* Added option to compile aggregate functions if `compile_aggregate_expressions` setting is on. [#24789](https://github.com/ClickHouse/ClickHouse/pull/24789) ([Maksim Kita](https://github.com/kitaisreal)). + +#### Improvements + +* For a dictionary with a complex key containing only one attribute, allow not wrapping the key expression in tuple for functions `dictGet`, `dictHas`. [#26130](https://github.com/ClickHouse/ClickHouse/pull/26130) ([Maksim Kita](https://github.com/kitaisreal)). +* Support arguments of `AggregateFunction` type in `bin` and `hex` functions. [#26094](https://github.com/ClickHouse/ClickHouse/pull/26094) ([zhaoyu](https://github.com/zxc111)). +* Support arguments of `UUID` type for `empty` and `notEmpty` functions. `UUID` is empty if it is all zeros (nil UUID). Closes [#3446](https://github.com/ClickHouse/ClickHouse/issues/3446). [#25974](https://github.com/ClickHouse/ClickHouse/pull/25974) ([zhaoyu](https://github.com/zxc111)). +* Fix error with query `SET SQL_SELECT_LIMIT` in MySQL protocol. Closes [#17115](https://github.com/ClickHouse/ClickHouse/issues/17115). [#25972](https://github.com/ClickHouse/ClickHouse/pull/25972) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Web UI: if the value looks like an URL, automatically generate a link. [#25965](https://github.com/ClickHouse/ClickHouse/pull/25965) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* More instrumentation for network interaction: add counters for recv/send bytes; add gauges for recvs/sends. Added missing documentation. Closes [#5897](https://github.com/ClickHouse/ClickHouse/issues/5897). [#25962](https://github.com/ClickHouse/ClickHouse/pull/25962) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Add setting `optimize_move_to_prewhere_if_final`. If query has `FINAL`, the optimization `move_to_prewhere` will be enabled only if both `optimize_move_to_prewhere` and `optimize_move_to_prewhere_if_final` are enabled. Closes [#8684](https://github.com/ClickHouse/ClickHouse/issues/8684). [#25940](https://github.com/ClickHouse/ClickHouse/pull/25940) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Allow complex quoted identifiers of JOINed tables. This closes [#17861](https://github.com/ClickHouse/ClickHouse/issues/17861). [#25924](https://github.com/ClickHouse/ClickHouse/pull/25924) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Add support for Unicode (e.g. Chinese, Cyrillic) components in `Nested` data types. This closes [#25594](https://github.com/ClickHouse/ClickHouse/issues/25594). [#25923](https://github.com/ClickHouse/ClickHouse/pull/25923) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Make `sudo service clickhouse-server start` to work on systems with `systemd` like Centos 8. This closes [#14298](https://github.com/ClickHouse/ClickHouse/issues/14298). This closes [#17799](https://github.com/ClickHouse/ClickHouse/issues/17799). [#25921](https://github.com/ClickHouse/ClickHouse/pull/25921) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Allow `quantiles*` functions to work with `aggregate_functions_null_for_empty`. This closes [#25892](https://github.com/ClickHouse/ClickHouse/issues/25892). [#25919](https://github.com/ClickHouse/ClickHouse/pull/25919) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Allow parameters for parametric aggregate functions to be arbitrary constant expressions (e.g., `1 + 2`), not just literals. It also allows using the query parameters (in parameterized queries like `{param:UInt8}`) inside parametric aggregate functions. Closes [#11607](https://github.com/ClickHouse/ClickHouse/issues/11607). [#25910](https://github.com/ClickHouse/ClickHouse/pull/25910) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Correctly throw the exception on the attempt to parse an invalid Date. Closes [#6481](https://github.com/ClickHouse/ClickHouse/issues/6481). [#25909](https://github.com/ClickHouse/ClickHouse/pull/25909) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Add support for queries with a column named `"null"` (it must be specified in backticks or double quotes) and `ON CLUSTER`. Closes [#24035](https://github.com/ClickHouse/ClickHouse/issues/24035). [#25907](https://github.com/ClickHouse/ClickHouse/pull/25907) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Support LowCardinality, Decimal, and UUID for JSON extract. Closes [#24606](https://github.com/ClickHouse/ClickHouse/issues/24606). [#25900](https://github.com/ClickHouse/ClickHouse/pull/25900) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Convert history file from readline format to replxx format. [#25888](https://github.com/ClickHouse/ClickHouse/pull/25888) ([Azat Khuzhin](https://github.com/azat)). +* Fix an extremely rare bug which can lead to intersecting parts after `DROP PART` or background deletion of an empty part. [#25884](https://github.com/ClickHouse/ClickHouse/pull/25884) ([alesapin](https://github.com/alesapin)). +* Better handling of lost parts for ReplicatedMergeTree tables. Fixes rare inconsistencies in ReplicationQueue. Nothing should be visible to the user. Fixes [#10368](https://github.com/ClickHouse/ClickHouse/issues/10368). [#25820](https://github.com/ClickHouse/ClickHouse/pull/25820) ([alesapin](https://github.com/alesapin)). +* Allow to start clickhouse-client with unreadable working directory. [#25817](https://github.com/ClickHouse/ClickHouse/pull/25817) ([ianton-ru](https://github.com/ianton-ru)). +* Fix "No available columns" for Merge() storage. [#25801](https://github.com/ClickHouse/ClickHouse/pull/25801) ([Azat Khuzhin](https://github.com/azat)). +* MySQL Engine now supports the exchange of column comments between MySQL and ClickHouse. [#25795](https://github.com/ClickHouse/ClickHouse/pull/25795) ([Storozhuk Kostiantyn](https://github.com/sand6255)). +* Fix inconsistent behaviour of `GROUP BY` constant on empty set. Closes [#6842](https://github.com/ClickHouse/ClickHouse/issues/6842). [#25786](https://github.com/ClickHouse/ClickHouse/pull/25786) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Use `Map` data type for key-value dictionaries in system logs tables (`system.query_log`, `system.query_thread_log`, `system.processes`, `system.opentelemetry_span_log`). Virtual columns are created to support old queries. Closes [#18698](https://github.com/ClickHouse/ClickHouse/issues/18698). Authors @hexiaoting, @sundy-li. [#25773](https://github.com/ClickHouse/ClickHouse/pull/25773) ([Maksim Kita](https://github.com/kitaisreal)). +* Cancel already running merges in partition on `DROP PARTITION` and `TRUNCATE` for `ReplicatedMergeTree`. Resolves [#17151](https://github.com/ClickHouse/ClickHouse/issues/17151). [#25684](https://github.com/ClickHouse/ClickHouse/pull/25684) ([tavplubix](https://github.com/tavplubix)). +* MaterializeMySQL now supports `ENUM` data type. [#25676](https://github.com/ClickHouse/ClickHouse/pull/25676) ([Storozhuk Kostiantyn](https://github.com/sand6255)). +* Support materialized and aliased columns in joins, close [#13274](https://github.com/ClickHouse/ClickHouse/issues/13274). [#25634](https://github.com/ClickHouse/ClickHouse/pull/25634) ([Vladimir C](https://github.com/vdimir)). +* Fix possible logical race condition between `ALTER TABLE ... DETACH` and background merges. [#25605](https://github.com/ClickHouse/ClickHouse/pull/25605) ([Azat Khuzhin](https://github.com/azat)). +* Make `NetworkReceiveElapsedMicroseconds` metric to correctly include the time spent waiting for data from the client to `INSERT`. This closes [#9958](https://github.com/ClickHouse/ClickHouse/issues/9958). [#25602](https://github.com/ClickHouse/ClickHouse/pull/25602) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Support `TRUNCATE TABLE` for StorageS3 and StorageHDFS. This closes [#25530](https://github.com/ClickHouse/ClickHouse/issues/25530). [#25550](https://github.com/ClickHouse/ClickHouse/pull/25550) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Support for dynamic reloading of config to change number of threads in pool for background jobs execution (merges, mutations, fetches). [#25548](https://github.com/ClickHouse/ClickHouse/pull/25548) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Allow extract of non-string element as string using JSONExtract. This is for [#25414](https://github.com/ClickHouse/ClickHouse/issues/25414). [#25452](https://github.com/ClickHouse/ClickHouse/pull/25452) ([Amos Bird](https://github.com/amosbird)). +* `Database` argument for `StorageMerge` support regular expression. This closes [#776]](https://github.com/ClickHouse/ClickHouse/issues/776). [#25064](https://github.com/ClickHouse/ClickHouse/pull/25064) ([flynn](https://github.com/ucasfl)). +* Support for multiple includes in configuration. It is possible to include users configuration, remote servers configuration from multiple sources. Simply place `` element with `from_zk`, `from_env` or `incl` attribute and it will be replaced with the substitution. [#24404](https://github.com/ClickHouse/ClickHouse/pull/24404) ([nvartolomei](https://github.com/nvartolomei)). + +#### Bug Fixes + +* Sometimes `SET ROLE` could work incorrectly, this PR fixes that. [#26707](https://github.com/ClickHouse/ClickHouse/pull/26707) ([Vitaly Baranov](https://github.com/vitlibar)). +* Fix potential nullptr dereference in window functions. This fixes [#25276](https://github.com/ClickHouse/ClickHouse/issues/25276). [#26668](https://github.com/ClickHouse/ClickHouse/pull/26668) ([Alexander Kuzmenkov](https://github.com/akuzm)). +* Fix history file conversion if file is empty. [#26589](https://github.com/ClickHouse/ClickHouse/pull/26589) ([Azat Khuzhin](https://github.com/azat)). +* Fix incorrect function names of groupBitmapAnd/Or/Xor. This fixes. [#26557](https://github.com/ClickHouse/ClickHouse/pull/26557) ([Amos Bird](https://github.com/amosbird)). +* Update `chown` cmd check in clickhouse-server docker entrypoint. It fixes the bug 'cluster pod restart failed (or timeout) on kubernetes'. [#26545](https://github.com/ClickHouse/ClickHouse/pull/26545) ([Ky Li](https://github.com/Kylinrix)). +* Fix crash in rabbitmq shutdown in case rabbitmq setup was not started. Closes [#26504](https://github.com/ClickHouse/ClickHouse/issues/26504). [#26529](https://github.com/ClickHouse/ClickHouse/pull/26529) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix issues with `CREATE DICTIONARY` query if dictionary name or database name was quoted. Closes [#26491](https://github.com/ClickHouse/ClickHouse/issues/26491). [#26508](https://github.com/ClickHouse/ClickHouse/pull/26508) ([Maksim Kita](https://github.com/kitaisreal)). +* Fix broken name resolution after rewriting column aliases. This fixes [#26432](https://github.com/ClickHouse/ClickHouse/issues/26432). [#26475](https://github.com/ClickHouse/ClickHouse/pull/26475) ([Amos Bird](https://github.com/amosbird)). +* Fix some fuzzed msan crash. Fixes [#22517](https://github.com/ClickHouse/ClickHouse/issues/22517). [#26428](https://github.com/ClickHouse/ClickHouse/pull/26428) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix infinite non joined block stream in `partial_merge_join` close [#26325](https://github.com/ClickHouse/ClickHouse/issues/26325). [#26374](https://github.com/ClickHouse/ClickHouse/pull/26374) ([Vladimir C](https://github.com/vdimir)). +* Fix possible crash when login as dropped user. This PR fixes [#26073](https://github.com/ClickHouse/ClickHouse/issues/26073). [#26363](https://github.com/ClickHouse/ClickHouse/pull/26363) ([Vitaly Baranov](https://github.com/vitlibar)). +* Fix `optimize_distributed_group_by_sharding_key` for multiple columns (leads to incorrect result w/ `optimize_skip_unused_shards=1`/`allow_nondeterministic_optimize_skip_unused_shards=1` and multiple columns in sharding key expression). [#26353](https://github.com/ClickHouse/ClickHouse/pull/26353) ([Azat Khuzhin](https://github.com/azat)). +* Fixed rare bug in lost replica recovery that may cause replicas to diverge. [#26321](https://github.com/ClickHouse/ClickHouse/pull/26321) ([tavplubix](https://github.com/tavplubix)). +* Fix zstd decompression in case there are escape sequences at the end of internal buffer. Closes [#26013](https://github.com/ClickHouse/ClickHouse/issues/26013). [#26314](https://github.com/ClickHouse/ClickHouse/pull/26314) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix logical error on join with totals, closing [#26017](https://github.com/ClickHouse/ClickHouse/issues/26017). [#26250](https://github.com/ClickHouse/ClickHouse/pull/26250) ([Vladimir C](https://github.com/vdimir)). +* Remove excessive newline in `thread_name` column in `system.stack_trace` table. This fixes [#24124](https://github.com/ClickHouse/ClickHouse/issues/24124). [#26210](https://github.com/ClickHouse/ClickHouse/pull/26210) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix `joinGet` with LowCarinality columns, close [#25993](https://github.com/ClickHouse/ClickHouse/issues/25993). [#26118](https://github.com/ClickHouse/ClickHouse/pull/26118) ([Vladimir C](https://github.com/vdimir)). +* Fix possible crash in `pointInPolygon` if the setting `validate_polygons` is turned off. [#26113](https://github.com/ClickHouse/ClickHouse/pull/26113) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix throwing exception when iterate over non existing remote directory. [#26087](https://github.com/ClickHouse/ClickHouse/pull/26087) ([ianton-ru](https://github.com/ianton-ru)). +* Fix rare server crash because of `abort` in ZooKeeper client. Fixes [#25813](https://github.com/ClickHouse/ClickHouse/issues/25813). [#26079](https://github.com/ClickHouse/ClickHouse/pull/26079) ([alesapin](https://github.com/alesapin)). +* Fix wrong thread estimation for right subquery join in some cases. Close [#24075](https://github.com/ClickHouse/ClickHouse/issues/24075). [#26052](https://github.com/ClickHouse/ClickHouse/pull/26052) ([Vladimir C](https://github.com/vdimir)). +* Fixed incorrect `sequence_id` in MySQL protocol packets that ClickHouse sends on exception during query execution. It might cause MySQL client to reset connection to ClickHouse server. Fixes [#21184](https://github.com/ClickHouse/ClickHouse/issues/21184). [#26051](https://github.com/ClickHouse/ClickHouse/pull/26051) ([tavplubix](https://github.com/tavplubix)). +* Fix possible mismatched header when using normal projection with prewhere. This fixes [#26020](https://github.com/ClickHouse/ClickHouse/issues/26020). [#26038](https://github.com/ClickHouse/ClickHouse/pull/26038) ([Amos Bird](https://github.com/amosbird)). +* Fix formatting of type `Map` with integer keys to `JSON`. [#25982](https://github.com/ClickHouse/ClickHouse/pull/25982) ([Anton Popov](https://github.com/CurtizJ)). +* Fix possible deadlock during query profiler stack unwinding. Fixes [#25968](https://github.com/ClickHouse/ClickHouse/issues/25968). [#25970](https://github.com/ClickHouse/ClickHouse/pull/25970) ([Maksim Kita](https://github.com/kitaisreal)). +* Fix crash on call dictGet() with bad arguments. [#25913](https://github.com/ClickHouse/ClickHouse/pull/25913) ([Vitaly Baranov](https://github.com/vitlibar)). +* Fixed `scram-sha-256` authentication for PostgreSQL engines. Closes [#24516](https://github.com/ClickHouse/ClickHouse/issues/24516). [#25906](https://github.com/ClickHouse/ClickHouse/pull/25906) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix extremely long backoff for background tasks when the background pool is full. Fixes [#25836](https://github.com/ClickHouse/ClickHouse/issues/25836). [#25893](https://github.com/ClickHouse/ClickHouse/pull/25893) ([alesapin](https://github.com/alesapin)). +* Fix ARM exception handling with non default page size. Fixes [#25512](https://github.com/ClickHouse/ClickHouse/issues/25512). Fixes [#25044](https://github.com/ClickHouse/ClickHouse/issues/25044). Fixes [#24901](https://github.com/ClickHouse/ClickHouse/issues/24901). Fixes [#23183](https://github.com/ClickHouse/ClickHouse/issues/23183). Fixes [#20221](https://github.com/ClickHouse/ClickHouse/issues/20221). Fixes [#19703](https://github.com/ClickHouse/ClickHouse/issues/19703). Fixes [#19028](https://github.com/ClickHouse/ClickHouse/issues/19028). Fixes [#18391](https://github.com/ClickHouse/ClickHouse/issues/18391). Fixes [#18121](https://github.com/ClickHouse/ClickHouse/issues/18121). Fixes [#17994](https://github.com/ClickHouse/ClickHouse/issues/17994). Fixes [#12483](https://github.com/ClickHouse/ClickHouse/issues/12483). [#25854](https://github.com/ClickHouse/ClickHouse/pull/25854) ([Maksim Kita](https://github.com/kitaisreal)). +* Fix sharding_key from column w/o function for remote() (before `select * from remote('127.1', system.one, dummy)` leads to `Unknown column: dummy, there are only columns .` error). [#25824](https://github.com/ClickHouse/ClickHouse/pull/25824) ([Azat Khuzhin](https://github.com/azat)). +* Fixed `Not found column ...` and `Missing column ...` errors when selecting from `MaterializeMySQL`. Fixes [#23708](https://github.com/ClickHouse/ClickHouse/issues/23708), [#24830](https://github.com/ClickHouse/ClickHouse/issues/24830), [#25794](https://github.com/ClickHouse/ClickHouse/issues/25794). [#25822](https://github.com/ClickHouse/ClickHouse/pull/25822) ([tavplubix](https://github.com/tavplubix)). +* Fix `optimize_skip_unused_shards_rewrite_in` for non-UInt64 types (may select incorrect shards eventually or throw `Cannot infer type of an empty tuple` or `Function tuple requires at least one argument`). [#25798](https://github.com/ClickHouse/ClickHouse/pull/25798) ([Azat Khuzhin](https://github.com/azat)). +* Fix rare bug with `DROP PART` query for `ReplicatedMergeTree` tables which can lead to error message `Unexpected merged part intersecting drop range`. [#25783](https://github.com/ClickHouse/ClickHouse/pull/25783) ([alesapin](https://github.com/alesapin)). +* Fix bug in `TTL` with `GROUP BY` expression which refuses to execute `TTL` after first execution in part. [#25743](https://github.com/ClickHouse/ClickHouse/pull/25743) ([alesapin](https://github.com/alesapin)). +* Allow StorageMerge to access tables with aliases. Closes [#6051](https://github.com/ClickHouse/ClickHouse/issues/6051). [#25694](https://github.com/ClickHouse/ClickHouse/pull/25694) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix slow dict join in some cases, close [#24209](https://github.com/ClickHouse/ClickHouse/issues/24209). [#25618](https://github.com/ClickHouse/ClickHouse/pull/25618) ([Vladimir C](https://github.com/vdimir)). +* Fix `ALTER MODIFY COLUMN` of columns, which participates in TTL expressions. [#25554](https://github.com/ClickHouse/ClickHouse/pull/25554) ([Anton Popov](https://github.com/CurtizJ)). +* Fix assertion in PREWHERE with non-uint8 type, close [#19589](https://github.com/ClickHouse/ClickHouse/issues/19589). [#25484](https://github.com/ClickHouse/ClickHouse/pull/25484) ([Vladimir C](https://github.com/vdimir)). +* `CAST` from `Date` to `DateTime` (or `DateTime64`) was not using the timezone of the `DateTime` type. It can also affect the comparison between `Date` and `DateTime`. Inference of the common type for `Date` and `DateTime` also was not using the corresponding timezone. It affected the results of function `if` and array construction. Closes [#24128](https://github.com/ClickHouse/ClickHouse/issues/24128). [#24129](https://github.com/ClickHouse/ClickHouse/pull/24129) ([Maksim Kita](https://github.com/kitaisreal)). + +#### Build/Testing/Packaging Improvements + +* Disabling TestFlows LDAP module due to test fails. [#26065](https://github.com/ClickHouse/ClickHouse/pull/26065) ([vzakaznikov](https://github.com/vzakaznikov)). +* Enabling all TestFlows modules and fixing some tests. [#26011](https://github.com/ClickHouse/ClickHouse/pull/26011) ([vzakaznikov](https://github.com/vzakaznikov)). +* Add new tests for checking access rights for columns used in filters (`WHERE` / `PREWHERE` / row policy) of the `SELECT` statement after changes in [#24405](https://github.com/ClickHouse/ClickHouse/pull/24405). [#25619](https://github.com/ClickHouse/ClickHouse/pull/25619) ([Vitaly Baranov](https://github.com/vitlibar)). +* Syntax update: changing 'is' to '==' when necessary. [#25559](https://github.com/ClickHouse/ClickHouse/pull/25559) ([MyroTk](https://github.com/MyroTk)). + +#### Other + +* Add `clickhouse-keeper-converter` tool which allows converting zookeeper logs and snapshots into `clickhouse-keeper` snapshot format. [#25428](https://github.com/ClickHouse/ClickHouse/pull/25428) ([alesapin](https://github.com/alesapin)). + + + ### ClickHouse release v21.7, 2021-07-09 #### Backward Incompatible Change From d8db279c75f9671f3faf9380e6a74204507b562d Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Fri, 6 Aug 2021 11:39:47 +0300 Subject: [PATCH 192/599] Add experimental title. --- docs/en/engines/database-engines/materialized-mysql.md | 2 +- docs/ru/engines/database-engines/materialized-mysql.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/engines/database-engines/materialized-mysql.md b/docs/en/engines/database-engines/materialized-mysql.md index 1fad4414d57..1c69fd072d8 100644 --- a/docs/en/engines/database-engines/materialized-mysql.md +++ b/docs/en/engines/database-engines/materialized-mysql.md @@ -1,6 +1,6 @@ --- toc_priority: 29 -toc_title: MaterializedMySQL +toc_title: [experimental] MaterializedMySQL --- # [experimental] MaterializedMySQL {#materialized-mysql} diff --git a/docs/ru/engines/database-engines/materialized-mysql.md b/docs/ru/engines/database-engines/materialized-mysql.md index b886dc9b4e7..7b14867fa62 100644 --- a/docs/ru/engines/database-engines/materialized-mysql.md +++ b/docs/ru/engines/database-engines/materialized-mysql.md @@ -1,6 +1,6 @@ --- toc_priority: 29 -toc_title: MaterializedMySQL +toc_title: [experimental] MaterializedMySQL --- # [experimental] MaterializedMySQL {#materialized-mysql} From 13f95f3fdf7f71c0d9907384df4a86a41a027389 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 6 Aug 2021 11:41:45 +0300 Subject: [PATCH 193/599] Streams -> Processors for dicts, part 3. --- src/Bridge/LibraryBridgeHelper.cpp | 2 +- src/Bridge/LibraryBridgeHelper.h | 2 +- src/DataStreams/MongoDBBlockInputStream.cpp | 2 +- src/DataStreams/MongoDBBlockInputStream.h | 2 +- .../PostgreSQLBlockInputStream.cpp | 2 +- src/DataStreams/PostgreSQLBlockInputStream.h | 26 ++++++--- src/Databases/MySQL/DatabaseMySQL.cpp | 10 +++- .../MySQL/FetchTablesColumnsList.cpp | 11 +++- src/Databases/MySQL/MaterializeMetadata.cpp | 55 +++++++++++++++---- .../MySQL/MaterializedMySQLSyncThread.cpp | 25 +++++++-- src/Dictionaries/CacheDictionary.cpp | 2 +- src/Dictionaries/CassandraBlockInputStream.h | 3 - .../DictionaryBlockInputStreamBase.cpp | 4 +- src/Dictionaries/DictionaryHelpers.h | 4 +- src/Dictionaries/DictionarySourceHelpers.cpp | 2 +- src/Dictionaries/DictionarySourceHelpers.h | 2 +- src/Dictionaries/DirectDictionary.cpp | 2 +- src/Dictionaries/DirectDictionary.h | 2 +- .../ExecutableDictionarySource.cpp | 2 +- src/Dictionaries/ExecutableDictionarySource.h | 2 +- .../ExecutablePoolDictionarySource.cpp | 2 +- .../ExecutablePoolDictionarySource.h | 2 +- src/Dictionaries/FileDictionarySource.cpp | 2 +- src/Dictionaries/FileDictionarySource.h | 2 +- src/Dictionaries/FlatDictionary.cpp | 2 +- src/Dictionaries/FlatDictionary.h | 2 +- src/Dictionaries/HTTPDictionarySource.cpp | 2 +- src/Dictionaries/HTTPDictionarySource.h | 2 +- src/Dictionaries/HashedDictionary.cpp | 2 +- src/Dictionaries/HashedDictionary.h | 2 +- src/Dictionaries/IPAddressDictionary.cpp | 2 +- src/Dictionaries/IPAddressDictionary.h | 2 +- src/Dictionaries/LibraryDictionarySource.cpp | 2 +- src/Dictionaries/LibraryDictionarySource.h | 2 +- src/Dictionaries/MongoDBDictionarySource.cpp | 2 +- src/Dictionaries/MongoDBDictionarySource.h | 2 +- src/Dictionaries/MySQLDictionarySource.cpp | 5 +- src/Dictionaries/MySQLDictionarySource.h | 4 +- src/Dictionaries/PolygonDictionary.cpp | 2 +- src/Dictionaries/PolygonDictionary.h | 2 +- .../PostgreSQLDictionarySource.cpp | 7 +-- src/Dictionaries/PostgreSQLDictionarySource.h | 2 +- .../RangeDictionaryBlockInputStream.h | 2 +- src/Dictionaries/RangeHashedDictionary.cpp | 2 +- src/Dictionaries/RangeHashedDictionary.h | 2 +- src/Dictionaries/RedisBlockInputStream.cpp | 2 +- src/Dictionaries/RedisBlockInputStream.h | 3 +- src/Dictionaries/RedisDictionarySource.cpp | 2 +- src/Dictionaries/RedisDictionarySource.h | 2 +- src/Dictionaries/XDBCDictionarySource.cpp | 2 +- src/Dictionaries/XDBCDictionarySource.h | 2 +- src/Formats/MySQLBlockInputStream.cpp | 2 +- src/Formats/MySQLBlockInputStream.h | 2 +- .../PostgreSQLReplicationHandler.cpp | 16 +++++- src/Storages/StorageDictionary.cpp | 4 +- src/Storages/StorageMongoDB.cpp | 3 +- src/Storages/StorageMySQL.cpp | 3 +- src/Storages/StoragePostgreSQL.cpp | 3 +- 58 files changed, 170 insertions(+), 98 deletions(-) diff --git a/src/Bridge/LibraryBridgeHelper.cpp b/src/Bridge/LibraryBridgeHelper.cpp index 81484854367..66f5cb7a600 100644 --- a/src/Bridge/LibraryBridgeHelper.cpp +++ b/src/Bridge/LibraryBridgeHelper.cpp @@ -180,4 +180,4 @@ Pipe LibraryBridgeHelper::loadBase(const Poco::URI & uri, ReadWriteBufferFromHTT return Pipe(std::move(source)); } -} \ No newline at end of file +} diff --git a/src/Bridge/LibraryBridgeHelper.h b/src/Bridge/LibraryBridgeHelper.h index 18a9c18f2ec..677c258d45d 100644 --- a/src/Bridge/LibraryBridgeHelper.h +++ b/src/Bridge/LibraryBridgeHelper.h @@ -85,4 +85,4 @@ private: size_t bridge_port; }; -} \ No newline at end of file +} diff --git a/src/DataStreams/MongoDBBlockInputStream.cpp b/src/DataStreams/MongoDBBlockInputStream.cpp index 1bc77532eb9..a0a8e3e40a5 100644 --- a/src/DataStreams/MongoDBBlockInputStream.cpp +++ b/src/DataStreams/MongoDBBlockInputStream.cpp @@ -366,4 +366,4 @@ Chunk MongoDBSource::generate() return Chunk(std::move(columns), num_rows); } -} \ No newline at end of file +} diff --git a/src/DataStreams/MongoDBBlockInputStream.h b/src/DataStreams/MongoDBBlockInputStream.h index 9100ba3240b..2e8858932f4 100644 --- a/src/DataStreams/MongoDBBlockInputStream.h +++ b/src/DataStreams/MongoDBBlockInputStream.h @@ -50,4 +50,4 @@ private: bool strict_check_names; }; -} \ No newline at end of file +} diff --git a/src/DataStreams/PostgreSQLBlockInputStream.cpp b/src/DataStreams/PostgreSQLBlockInputStream.cpp index 477ede47258..f4734ae0e3d 100644 --- a/src/DataStreams/PostgreSQLBlockInputStream.cpp +++ b/src/DataStreams/PostgreSQLBlockInputStream.cpp @@ -170,4 +170,4 @@ class PostgreSQLSource; } -#endif \ No newline at end of file +#endif diff --git a/src/DataStreams/PostgreSQLBlockInputStream.h b/src/DataStreams/PostgreSQLBlockInputStream.h index 8b2ea3b62ef..008da976619 100644 --- a/src/DataStreams/PostgreSQLBlockInputStream.h +++ b/src/DataStreams/PostgreSQLBlockInputStream.h @@ -44,11 +44,11 @@ protected: Status prepare() override; -private: void onStart(); Chunk generate() override; void onFinish(); +private: void init(const Block & sample_block); const UInt64 max_block_size; @@ -63,26 +63,34 @@ private: }; -/// Passes transaction object into PostgreSQLBlockInputStream and does not close transaction after read is finished. +/// Passes transaction object into PostgreSQLSource and does not close transaction after read is finished. template -class PostgreSQLTransactionBlockInputStream : public PostgreSQLBlockInputStream +class PostgreSQLTransactionSource : public PostgreSQLSource { public: - using Base = PostgreSQLBlockInputStream; + using Base = PostgreSQLSource; - PostgreSQLTransactionBlockInputStream( + PostgreSQLTransactionSource( std::shared_ptr tx_, const std::string & query_str_, const Block & sample_block_, const UInt64 max_block_size_) - : PostgreSQLBlockInputStream(tx_, query_str_, sample_block_, max_block_size_, false) {} + : PostgreSQLSource(tx_, query_str_, sample_block_, max_block_size_, false) {} - void readPrefix() override + Chunk generate() override { - Base::stream = std::make_unique(*Base::tx, pqxx::from_query, std::string_view(Base::query_str)); + if (!is_initialized) + { + Base::stream = std::make_unique(*Base::tx, pqxx::from_query, std::string_view(Base::query_str)); + is_initialized = true; + } + + return Base::generate(); } + + bool is_initialized = false; }; } -#endif \ No newline at end of file +#endif diff --git a/src/Databases/MySQL/DatabaseMySQL.cpp b/src/Databases/MySQL/DatabaseMySQL.cpp index 5f356348829..d4acd2af85e 100644 --- a/src/Databases/MySQL/DatabaseMySQL.cpp +++ b/src/Databases/MySQL/DatabaseMySQL.cpp @@ -12,6 +12,8 @@ # include # include # include +# include +# include # include # include # include @@ -281,9 +283,13 @@ std::map DatabaseMySQL::fetchTablesWithModificationTime(ContextP std::map tables_with_modification_time; StreamSettings mysql_input_stream_settings(local_context->getSettingsRef()); - MySQLBlockInputStream result(mysql_pool.get(), query.str(), tables_status_sample_block, mysql_input_stream_settings); + auto result = std::make_unique(mysql_pool.get(), query.str(), tables_status_sample_block, mysql_input_stream_settings); + QueryPipeline pipeline; + pipeline.init(Pipe(std::move(result))); - while (Block block = result.read()) + Block block; + PullingPipelineExecutor executor(pipeline); + while (executor.pull(block)) { size_t rows = block.rows(); for (size_t index = 0; index < rows; ++index) diff --git a/src/Databases/MySQL/FetchTablesColumnsList.cpp b/src/Databases/MySQL/FetchTablesColumnsList.cpp index bbd187090df..353bcd877ee 100644 --- a/src/Databases/MySQL/FetchTablesColumnsList.cpp +++ b/src/Databases/MySQL/FetchTablesColumnsList.cpp @@ -8,6 +8,8 @@ #include #include #include +#include +#include #include #include #include @@ -85,8 +87,13 @@ std::map fetchTablesColumnsList( query << " TABLE_NAME IN " << toQueryStringWithQuote(tables_name) << " ORDER BY ORDINAL_POSITION"; StreamSettings mysql_input_stream_settings(settings); - MySQLBlockInputStream result(pool.get(), query.str(), tables_columns_sample_block, mysql_input_stream_settings); - while (Block block = result.read()) + auto result = std::make_unique(pool.get(), query.str(), tables_columns_sample_block, mysql_input_stream_settings); + QueryPipeline pipeline; + pipeline.init(Pipe(std::move(result))); + + Block block; + PullingPipelineExecutor executor(pipeline); + while (executor.pull(block)) { const auto & table_name_col = *block.getByPosition(0).column; const auto & column_name_col = *block.getByPosition(1).column; diff --git a/src/Databases/MySQL/MaterializeMetadata.cpp b/src/Databases/MySQL/MaterializeMetadata.cpp index 4fd99ca3438..9f5100991aa 100644 --- a/src/Databases/MySQL/MaterializeMetadata.cpp +++ b/src/Databases/MySQL/MaterializeMetadata.cpp @@ -6,6 +6,8 @@ #include #include #include +#include +#include #include #include #include @@ -38,11 +40,16 @@ static std::unordered_map fetchTablesCreateQuery( }; StreamSettings mysql_input_stream_settings(global_settings, false, true); - MySQLBlockInputStream show_create_table( + auto show_create_table = std::make_unique( connection, "SHOW CREATE TABLE " + backQuoteIfNeed(database_name) + "." + backQuoteIfNeed(fetch_table_name), show_create_table_header, mysql_input_stream_settings); - Block create_query_block = show_create_table.read(); + QueryPipeline pipeline; + pipeline.init(Pipe(std::move(show_create_table))); + + Block create_query_block; + PullingPipelineExecutor executor(pipeline); + executor.pull(create_query_block); if (!create_query_block || create_query_block.rows() != 1) throw Exception("LOGICAL ERROR mysql show create return more rows.", ErrorCodes::LOGICAL_ERROR); @@ -60,9 +67,14 @@ static std::vector fetchTablesInDB(const mysqlxx::PoolWithFailover::Entr std::vector tables_in_db; StreamSettings mysql_input_stream_settings(global_settings); - MySQLBlockInputStream input(connection, query, header, mysql_input_stream_settings); + auto input = std::make_unique(connection, query, header, mysql_input_stream_settings); - while (Block block = input.read()) + QueryPipeline pipeline; + pipeline.init(Pipe(std::move(input))); + + Block block; + PullingPipelineExecutor executor(pipeline); + while (executor.pull(block)) { tables_in_db.reserve(tables_in_db.size() + block.rows()); for (size_t index = 0; index < block.rows(); ++index) @@ -83,8 +95,14 @@ void MaterializeMetadata::fetchMasterStatus(mysqlxx::PoolWithFailover::Entry & c }; StreamSettings mysql_input_stream_settings(settings, false, true); - MySQLBlockInputStream input(connection, "SHOW MASTER STATUS;", header, mysql_input_stream_settings); - Block master_status = input.read(); + auto input = std::make_unique(connection, "SHOW MASTER STATUS;", header, mysql_input_stream_settings); + + QueryPipeline pipeline; + pipeline.init(Pipe(std::move(input))); + + Block master_status; + PullingPipelineExecutor executor(pipeline); + executor.pull(master_status); if (!master_status || master_status.rows() != 1) throw Exception("Unable to get master status from MySQL.", ErrorCodes::LOGICAL_ERROR); @@ -106,9 +124,13 @@ void MaterializeMetadata::fetchMasterVariablesValue(const mysqlxx::PoolWithFailo const String & fetch_query = "SHOW VARIABLES WHERE Variable_name = 'binlog_checksum'"; StreamSettings mysql_input_stream_settings(settings, false, true); - MySQLBlockInputStream variables_input(connection, fetch_query, variables_header, mysql_input_stream_settings); + auto variables_input = std::make_unique(connection, fetch_query, variables_header, mysql_input_stream_settings); + QueryPipeline pipeline; + pipeline.init(Pipe(std::move(variables_input))); - while (Block variables_block = variables_input.read()) + Block variables_block; + PullingPipelineExecutor executor(pipeline); + while (executor.pull(variables_block)) { ColumnPtr variables_name = variables_block.getByName("Variable_name").column; ColumnPtr variables_value = variables_block.getByName("Value").column; @@ -130,8 +152,13 @@ static bool checkSyncUserPrivImpl(const mysqlxx::PoolWithFailover::Entry & conne String grants_query, sub_privs; StreamSettings mysql_input_stream_settings(global_settings); - MySQLBlockInputStream input(connection, "SHOW GRANTS FOR CURRENT_USER();", sync_user_privs_header, mysql_input_stream_settings); - while (Block block = input.read()) + auto input = std::make_unique(connection, "SHOW GRANTS FOR CURRENT_USER();", sync_user_privs_header, mysql_input_stream_settings); + QueryPipeline pipeline; + pipeline.init(Pipe(std::move(input))); + + Block block; + PullingPipelineExecutor executor(pipeline); + while (executor.pull(block)) { for (size_t index = 0; index < block.rows(); ++index) { @@ -176,9 +203,13 @@ bool MaterializeMetadata::checkBinlogFileExists(const mysqlxx::PoolWithFailover: }; StreamSettings mysql_input_stream_settings(settings, false, true); - MySQLBlockInputStream input(connection, "SHOW MASTER LOGS", logs_header, mysql_input_stream_settings); + auto input = std::make_unique(connection, "SHOW MASTER LOGS", logs_header, mysql_input_stream_settings); + QueryPipeline pipeline; + pipeline.init(Pipe(std::move(input))); - while (Block block = input.read()) + Block block; + PullingPipelineExecutor executor(pipeline); + while (executor.pull(block)) { for (size_t index = 0; index < block.rows(); ++index) { diff --git a/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp b/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp index dcf77f56e18..5175e9d0467 100644 --- a/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp +++ b/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp @@ -9,6 +9,8 @@ # include # include # include +# include +# include # include # include # include @@ -100,7 +102,7 @@ static void checkMySQLVariables(const mysqlxx::Pool::Entry & connection, const S const String & check_query = "SHOW VARIABLES;"; StreamSettings mysql_input_stream_settings(settings, false, true); - MySQLBlockInputStream variables_input(connection, check_query, variables_header, mysql_input_stream_settings); + auto variables_input = std::make_unique(connection, check_query, variables_header, mysql_input_stream_settings); std::unordered_map variables_error_message{ {"log_bin", "ON"}, @@ -110,7 +112,12 @@ static void checkMySQLVariables(const mysqlxx::Pool::Entry & connection, const S {"log_bin_use_v1_row_events", "OFF"} }; - while (Block variables_block = variables_input.read()) + QueryPipeline pipeline; + pipeline.init(Pipe(std::move(variables_input))); + + PullingPipelineExecutor executor(pipeline); + Block variables_block; + while (executor.pull(variables_block)) { ColumnPtr variable_name_column = variables_block.getByName("Variable_name").column; ColumnPtr variable_value_column = variables_block.getByName("Value").column; @@ -327,12 +334,22 @@ static inline void dumpDataForTables( auto out = std::make_shared(getTableOutput(database_name, table_name, query_context)); StreamSettings mysql_input_stream_settings(context->getSettingsRef()); - MySQLBlockInputStream input( + auto input = std::make_unique( connection, "SELECT * FROM " + backQuoteIfNeed(mysql_database_name) + "." + backQuoteIfNeed(table_name), out->getHeader(), mysql_input_stream_settings); + QueryPipeline pipeline; + pipeline.init(Pipe(std::move(input))); + PullingPipelineExecutor executor(pipeline); + Stopwatch watch; - copyData(input, *out, is_cancelled); + + out->writePrefix(); + Block block; + while (executor.pull(block)) + out->write(block); + out->writeSuffix(); + const Progress & progress = out->getProgress(); LOG_INFO(&Poco::Logger::get("MaterializedMySQLSyncThread(" + database_name + ")"), "Materialize MySQL step 1: dump {}, {} rows, {} in {} sec., {} rows/sec., {}/sec." diff --git a/src/Dictionaries/CacheDictionary.cpp b/src/Dictionaries/CacheDictionary.cpp index a8754691425..39186f0d63f 100644 --- a/src/Dictionaries/CacheDictionary.cpp +++ b/src/Dictionaries/CacheDictionary.cpp @@ -691,4 +691,4 @@ void CacheDictionary::update(CacheDictionaryUpdateUnitPtr; template class CacheDictionary; -} \ No newline at end of file +} diff --git a/src/Dictionaries/CassandraBlockInputStream.h b/src/Dictionaries/CassandraBlockInputStream.h index 98adb19fee6..ba76e72c245 100644 --- a/src/Dictionaries/CassandraBlockInputStream.h +++ b/src/Dictionaries/CassandraBlockInputStream.h @@ -22,9 +22,6 @@ public: String getName() const override { return "Cassandra"; } - Block getHeader() const override { return description.sample_block.cloneEmpty(); } - - private: using ValueType = ExternalResultDescription::ValueType; diff --git a/src/Dictionaries/DictionaryBlockInputStreamBase.cpp b/src/Dictionaries/DictionaryBlockInputStreamBase.cpp index d5e6e8a1cda..0eac8edac3d 100644 --- a/src/Dictionaries/DictionaryBlockInputStreamBase.cpp +++ b/src/Dictionaries/DictionaryBlockInputStreamBase.cpp @@ -13,9 +13,9 @@ Chunk DictionarySourceBase::generate() return {}; size_t size = std::min(max_block_size, rows_count - next_row); - auto chunk = getChunk(next_row, size); + auto block = getBlock(next_row, size); next_row += size; - return chunk; + return Chunk(block.getColumns(), size); } } diff --git a/src/Dictionaries/DictionaryHelpers.h b/src/Dictionaries/DictionaryHelpers.h index 79459057822..5a050d68326 100644 --- a/src/Dictionaries/DictionaryHelpers.h +++ b/src/Dictionaries/DictionaryHelpers.h @@ -648,4 +648,6 @@ static const PaddedPODArray & getColumnVectorData( } } -} \ No newline at end of file +} + + diff --git a/src/Dictionaries/DictionarySourceHelpers.cpp b/src/Dictionaries/DictionarySourceHelpers.cpp index 79d4d2e5376..092e7187e8f 100644 --- a/src/Dictionaries/DictionarySourceHelpers.cpp +++ b/src/Dictionaries/DictionarySourceHelpers.cpp @@ -125,4 +125,4 @@ String TransformWithAdditionalColumns::getName() const { return "TransformWithAdditionalColumns"; } -} \ No newline at end of file +} diff --git a/src/Dictionaries/DictionarySourceHelpers.h b/src/Dictionaries/DictionarySourceHelpers.h index ba05cf9ebc9..b955b6ffb66 100644 --- a/src/Dictionaries/DictionarySourceHelpers.h +++ b/src/Dictionaries/DictionarySourceHelpers.h @@ -52,4 +52,4 @@ private: size_t current_range_index = 0; }; -} \ No newline at end of file +} diff --git a/src/Dictionaries/DirectDictionary.cpp b/src/Dictionaries/DirectDictionary.cpp index afdd9bee92c..eb06701ab7a 100644 --- a/src/Dictionaries/DirectDictionary.cpp +++ b/src/Dictionaries/DirectDictionary.cpp @@ -354,4 +354,4 @@ void registerDictionaryDirect(DictionaryFactory & factory) } -} \ No newline at end of file +} diff --git a/src/Dictionaries/DirectDictionary.h b/src/Dictionaries/DirectDictionary.h index 841590164f6..4700e71d94b 100644 --- a/src/Dictionaries/DirectDictionary.h +++ b/src/Dictionaries/DirectDictionary.h @@ -113,4 +113,4 @@ private: extern template class DirectDictionary; extern template class DirectDictionary; -} \ No newline at end of file +} diff --git a/src/Dictionaries/ExecutableDictionarySource.cpp b/src/Dictionaries/ExecutableDictionarySource.cpp index 82b74d8196e..8802d04ff30 100644 --- a/src/Dictionaries/ExecutableDictionarySource.cpp +++ b/src/Dictionaries/ExecutableDictionarySource.cpp @@ -307,4 +307,4 @@ void registerDictionarySourceExecutable(DictionarySourceFactory & factory) factory.registerSource("executable", create_table_source); } -} \ No newline at end of file +} diff --git a/src/Dictionaries/ExecutableDictionarySource.h b/src/Dictionaries/ExecutableDictionarySource.h index 084b8c13c25..f2d295b9b18 100644 --- a/src/Dictionaries/ExecutableDictionarySource.h +++ b/src/Dictionaries/ExecutableDictionarySource.h @@ -69,4 +69,4 @@ private: ContextPtr context; }; -} \ No newline at end of file +} diff --git a/src/Dictionaries/ExecutablePoolDictionarySource.cpp b/src/Dictionaries/ExecutablePoolDictionarySource.cpp index 2cd567bbc02..e97b4253407 100644 --- a/src/Dictionaries/ExecutablePoolDictionarySource.cpp +++ b/src/Dictionaries/ExecutablePoolDictionarySource.cpp @@ -325,4 +325,4 @@ void registerDictionarySourceExecutablePool(DictionarySourceFactory & factory) factory.registerSource("executable_pool", create_table_source); } -} \ No newline at end of file +} diff --git a/src/Dictionaries/ExecutablePoolDictionarySource.h b/src/Dictionaries/ExecutablePoolDictionarySource.h index 9c8730632f7..22d7b713a24 100644 --- a/src/Dictionaries/ExecutablePoolDictionarySource.h +++ b/src/Dictionaries/ExecutablePoolDictionarySource.h @@ -83,4 +83,4 @@ private: std::shared_ptr process_pool; }; -} \ No newline at end of file +} diff --git a/src/Dictionaries/FileDictionarySource.cpp b/src/Dictionaries/FileDictionarySource.cpp index 3766da0a28d..bea14d88d1e 100644 --- a/src/Dictionaries/FileDictionarySource.cpp +++ b/src/Dictionaries/FileDictionarySource.cpp @@ -95,4 +95,4 @@ void registerDictionarySourceFile(DictionarySourceFactory & factory) factory.registerSource("file", create_table_source); } -} \ No newline at end of file +} diff --git a/src/Dictionaries/FileDictionarySource.h b/src/Dictionaries/FileDictionarySource.h index efc86be8ba9..c8e37986b2f 100644 --- a/src/Dictionaries/FileDictionarySource.h +++ b/src/Dictionaries/FileDictionarySource.h @@ -65,4 +65,4 @@ private: Poco::Timestamp last_modification; }; -} \ No newline at end of file +} diff --git a/src/Dictionaries/FlatDictionary.cpp b/src/Dictionaries/FlatDictionary.cpp index 40afe70f504..58cb5048737 100644 --- a/src/Dictionaries/FlatDictionary.cpp +++ b/src/Dictionaries/FlatDictionary.cpp @@ -591,4 +591,4 @@ void registerDictionaryFlat(DictionaryFactory & factory) } -} \ No newline at end of file +} diff --git a/src/Dictionaries/FlatDictionary.h b/src/Dictionaries/FlatDictionary.h index c16547bd3b4..e6a07200c05 100644 --- a/src/Dictionaries/FlatDictionary.h +++ b/src/Dictionaries/FlatDictionary.h @@ -178,4 +178,4 @@ private: BlockPtr update_field_loaded_block; }; -} \ No newline at end of file +} diff --git a/src/Dictionaries/HTTPDictionarySource.cpp b/src/Dictionaries/HTTPDictionarySource.cpp index 88aedd51395..4a80ebdf975 100644 --- a/src/Dictionaries/HTTPDictionarySource.cpp +++ b/src/Dictionaries/HTTPDictionarySource.cpp @@ -263,4 +263,4 @@ void registerDictionarySourceHTTP(DictionarySourceFactory & factory) factory.registerSource("http", create_table_source); } -} \ No newline at end of file +} diff --git a/src/Dictionaries/HTTPDictionarySource.h b/src/Dictionaries/HTTPDictionarySource.h index 1c7d67d3c23..e4a76627d08 100644 --- a/src/Dictionaries/HTTPDictionarySource.h +++ b/src/Dictionaries/HTTPDictionarySource.h @@ -80,4 +80,4 @@ private: ConnectionTimeouts timeouts; }; -} \ No newline at end of file +} diff --git a/src/Dictionaries/HashedDictionary.cpp b/src/Dictionaries/HashedDictionary.cpp index 22a7ec87b07..b50b6a72707 100644 --- a/src/Dictionaries/HashedDictionary.cpp +++ b/src/Dictionaries/HashedDictionary.cpp @@ -766,4 +766,4 @@ void registerDictionaryHashed(DictionaryFactory & factory) } -} \ No newline at end of file +} diff --git a/src/Dictionaries/HashedDictionary.h b/src/Dictionaries/HashedDictionary.h index 82e8a91b603..33c5fbf98bf 100644 --- a/src/Dictionaries/HashedDictionary.h +++ b/src/Dictionaries/HashedDictionary.h @@ -225,4 +225,4 @@ extern template class HashedDictionary; extern template class HashedDictionary; extern template class HashedDictionary; -} \ No newline at end of file +} diff --git a/src/Dictionaries/IPAddressDictionary.cpp b/src/Dictionaries/IPAddressDictionary.cpp index 53912f294b2..380ad460cba 100644 --- a/src/Dictionaries/IPAddressDictionary.cpp +++ b/src/Dictionaries/IPAddressDictionary.cpp @@ -970,4 +970,4 @@ void registerDictionaryTrie(DictionaryFactory & factory) factory.registerLayout("ip_trie", create_layout, true); } -} \ No newline at end of file +} diff --git a/src/Dictionaries/IPAddressDictionary.h b/src/Dictionaries/IPAddressDictionary.h index 06eb480f825..94946e41ff8 100644 --- a/src/Dictionaries/IPAddressDictionary.h +++ b/src/Dictionaries/IPAddressDictionary.h @@ -224,4 +224,4 @@ private: Poco::Logger * logger; }; -} \ No newline at end of file +} diff --git a/src/Dictionaries/LibraryDictionarySource.cpp b/src/Dictionaries/LibraryDictionarySource.cpp index f75ca845f01..d923d114623 100644 --- a/src/Dictionaries/LibraryDictionarySource.cpp +++ b/src/Dictionaries/LibraryDictionarySource.cpp @@ -183,4 +183,4 @@ void registerDictionarySourceLibrary(DictionarySourceFactory & factory) } -} \ No newline at end of file +} diff --git a/src/Dictionaries/LibraryDictionarySource.h b/src/Dictionaries/LibraryDictionarySource.h index 11b9245f769..9c5de3df2e0 100644 --- a/src/Dictionaries/LibraryDictionarySource.h +++ b/src/Dictionaries/LibraryDictionarySource.h @@ -92,4 +92,4 @@ private: ExternalResultDescription description; }; -} \ No newline at end of file +} diff --git a/src/Dictionaries/MongoDBDictionarySource.cpp b/src/Dictionaries/MongoDBDictionarySource.cpp index 3afce312de1..a3c5119ade1 100644 --- a/src/Dictionaries/MongoDBDictionarySource.cpp +++ b/src/Dictionaries/MongoDBDictionarySource.cpp @@ -238,4 +238,4 @@ std::string MongoDBDictionarySource::toString() const return "MongoDB: " + db + '.' + collection + ',' + (user.empty() ? " " : " " + user + '@') + host + ':' + DB::toString(port); } -} \ No newline at end of file +} diff --git a/src/Dictionaries/MongoDBDictionarySource.h b/src/Dictionaries/MongoDBDictionarySource.h index 811bec28fe7..3625deca9c6 100644 --- a/src/Dictionaries/MongoDBDictionarySource.h +++ b/src/Dictionaries/MongoDBDictionarySource.h @@ -84,4 +84,4 @@ private: std::shared_ptr connection; }; -} \ No newline at end of file +} diff --git a/src/Dictionaries/MySQLDictionarySource.cpp b/src/Dictionaries/MySQLDictionarySource.cpp index 6462158e882..c7309ddb950 100644 --- a/src/Dictionaries/MySQLDictionarySource.cpp +++ b/src/Dictionaries/MySQLDictionarySource.cpp @@ -285,10 +285,9 @@ std::string MySQLDictionarySource::doInvalidateQuery(const std::string & request Block invalidate_sample_block; ColumnPtr column(ColumnString::create()); invalidate_sample_block.insert(ColumnWithTypeAndName(column, std::make_shared(), "Sample Block")); - MySQLBlockInputStream block_input_stream(pool->get(), request, invalidate_sample_block, settings); - return readInvalidateQuery(block_input_stream); + return readInvalidateQuery(Pipe(std::make_unique(pool->get(), request, invalidate_sample_block, settings))); } } -#endif \ No newline at end of file +#endif diff --git a/src/Dictionaries/MySQLDictionarySource.h b/src/Dictionaries/MySQLDictionarySource.h index f3dbb89757d..49ddc924a86 100644 --- a/src/Dictionaries/MySQLDictionarySource.h +++ b/src/Dictionaries/MySQLDictionarySource.h @@ -6,7 +6,7 @@ # include "config_core.h" #endif -//#if USE_MYSQL +#if USE_MYSQL # include # include # include "DictionaryStructure.h" @@ -99,4 +99,4 @@ private: } -#endif \ No newline at end of file +#endif diff --git a/src/Dictionaries/PolygonDictionary.cpp b/src/Dictionaries/PolygonDictionary.cpp index f1f337e8fcc..39152963ede 100644 --- a/src/Dictionaries/PolygonDictionary.cpp +++ b/src/Dictionaries/PolygonDictionary.cpp @@ -516,4 +516,4 @@ void IPolygonDictionary::extractPolygons(const ColumnPtr & column) } } -} \ No newline at end of file +} diff --git a/src/Dictionaries/PolygonDictionary.h b/src/Dictionaries/PolygonDictionary.h index c119b5a5cfe..32771be4b16 100644 --- a/src/Dictionaries/PolygonDictionary.h +++ b/src/Dictionaries/PolygonDictionary.h @@ -166,4 +166,4 @@ private: static std::vector extractPoints(const Columns &key_columns); }; -} \ No newline at end of file +} diff --git a/src/Dictionaries/PostgreSQLDictionarySource.cpp b/src/Dictionaries/PostgreSQLDictionarySource.cpp index 1f256f3957f..f226b7a9165 100644 --- a/src/Dictionaries/PostgreSQLDictionarySource.cpp +++ b/src/Dictionaries/PostgreSQLDictionarySource.cpp @@ -107,7 +107,7 @@ Pipe PostgreSQLDictionarySource::loadKeys(const Columns & key_columns, const std Pipe PostgreSQLDictionarySource::loadBase(const String & query) { - return std::make_shared>(pool->get(), query, sample_block, max_block_size); + return Pipe(std::make_shared>(pool->get(), query, sample_block, max_block_size)); } @@ -129,8 +129,7 @@ std::string PostgreSQLDictionarySource::doInvalidateQuery(const std::string & re Block invalidate_sample_block; ColumnPtr column(ColumnString::create()); invalidate_sample_block.insert(ColumnWithTypeAndName(column, std::make_shared(), "Sample Block")); - PostgreSQLBlockInputStream<> block_input_stream(pool->get(), request, invalidate_sample_block, 1); - return readInvalidateQuery(block_input_stream); + return readInvalidateQuery(Pipe(std::make_unique>(pool->get(), request, invalidate_sample_block, 1))); } @@ -220,4 +219,4 @@ void registerDictionarySourcePostgreSQL(DictionarySourceFactory & factory) factory.registerSource("postgresql", create_table_source); } -} \ No newline at end of file +} diff --git a/src/Dictionaries/PostgreSQLDictionarySource.h b/src/Dictionaries/PostgreSQLDictionarySource.h index e5a72f0f720..28ad28661ed 100644 --- a/src/Dictionaries/PostgreSQLDictionarySource.h +++ b/src/Dictionaries/PostgreSQLDictionarySource.h @@ -72,4 +72,4 @@ private: }; } -#endif \ No newline at end of file +#endif diff --git a/src/Dictionaries/RangeDictionaryBlockInputStream.h b/src/Dictionaries/RangeDictionaryBlockInputStream.h index 1164683521c..d17687b7164 100644 --- a/src/Dictionaries/RangeDictionaryBlockInputStream.h +++ b/src/Dictionaries/RangeDictionaryBlockInputStream.h @@ -195,4 +195,4 @@ Block RangeDictionarySource::getBlock(size_t start, size_t length) co return data.getBlock(start, length); } -} \ No newline at end of file +} diff --git a/src/Dictionaries/RangeHashedDictionary.cpp b/src/Dictionaries/RangeHashedDictionary.cpp index bf930a7a526..8b882b5a107 100644 --- a/src/Dictionaries/RangeHashedDictionary.cpp +++ b/src/Dictionaries/RangeHashedDictionary.cpp @@ -677,4 +677,4 @@ void registerDictionaryRangeHashed(DictionaryFactory & factory) factory.registerLayout("range_hashed", create_layout, false); } -} \ No newline at end of file +} diff --git a/src/Dictionaries/RangeHashedDictionary.h b/src/Dictionaries/RangeHashedDictionary.h index b0ce1e41bad..01ee2b3c773 100644 --- a/src/Dictionaries/RangeHashedDictionary.h +++ b/src/Dictionaries/RangeHashedDictionary.h @@ -197,4 +197,4 @@ private: mutable std::atomic found_count{0}; }; -} \ No newline at end of file +} diff --git a/src/Dictionaries/RedisBlockInputStream.cpp b/src/Dictionaries/RedisBlockInputStream.cpp index 2d30e7fe167..c6e2546cf1e 100644 --- a/src/Dictionaries/RedisBlockInputStream.cpp +++ b/src/Dictionaries/RedisBlockInputStream.cpp @@ -222,4 +222,4 @@ namespace DB size_t num_rows = columns.at(0)->size(); return Chunk(std::move(columns), num_rows); } -} \ No newline at end of file +} diff --git a/src/Dictionaries/RedisBlockInputStream.h b/src/Dictionaries/RedisBlockInputStream.h index feef6e5423e..db2e643eb4e 100644 --- a/src/Dictionaries/RedisBlockInputStream.h +++ b/src/Dictionaries/RedisBlockInputStream.h @@ -48,4 +48,5 @@ namespace DB bool all_read = false; }; -} \ No newline at end of file +} + diff --git a/src/Dictionaries/RedisDictionarySource.cpp b/src/Dictionaries/RedisDictionarySource.cpp index 437c3fb0bf9..bf309dd0e8a 100644 --- a/src/Dictionaries/RedisDictionarySource.cpp +++ b/src/Dictionaries/RedisDictionarySource.cpp @@ -276,4 +276,4 @@ namespace DB return RedisStorageType::SIMPLE; } -} \ No newline at end of file +} diff --git a/src/Dictionaries/RedisDictionarySource.h b/src/Dictionaries/RedisDictionarySource.h index 31c55fabef9..053094e2303 100644 --- a/src/Dictionaries/RedisDictionarySource.h +++ b/src/Dictionaries/RedisDictionarySource.h @@ -94,4 +94,4 @@ namespace ErrorCodes std::shared_ptr client; }; -} \ No newline at end of file +} diff --git a/src/Dictionaries/XDBCDictionarySource.cpp b/src/Dictionaries/XDBCDictionarySource.cpp index 3839e269158..26b6c24cd2d 100644 --- a/src/Dictionaries/XDBCDictionarySource.cpp +++ b/src/Dictionaries/XDBCDictionarySource.cpp @@ -285,4 +285,4 @@ void registerDictionarySourceJDBC(DictionarySourceFactory & factory) factory.registerSource("jdbc", create_table_source); } -} \ No newline at end of file +} diff --git a/src/Dictionaries/XDBCDictionarySource.h b/src/Dictionaries/XDBCDictionarySource.h index 82da3f22d0a..ebced022b62 100644 --- a/src/Dictionaries/XDBCDictionarySource.h +++ b/src/Dictionaries/XDBCDictionarySource.h @@ -90,4 +90,4 @@ private: ConnectionTimeouts timeouts; }; -} \ No newline at end of file +} diff --git a/src/Formats/MySQLBlockInputStream.cpp b/src/Formats/MySQLBlockInputStream.cpp index 731afa5d1a6..401d85f3d6b 100644 --- a/src/Formats/MySQLBlockInputStream.cpp +++ b/src/Formats/MySQLBlockInputStream.cpp @@ -338,4 +338,4 @@ void MySQLSource::initPositionMappingFromQueryResultStructure() } -#endif \ No newline at end of file +#endif diff --git a/src/Formats/MySQLBlockInputStream.h b/src/Formats/MySQLBlockInputStream.h index a0ef9cfd4a8..9c33b4404ae 100644 --- a/src/Formats/MySQLBlockInputStream.h +++ b/src/Formats/MySQLBlockInputStream.h @@ -81,4 +81,4 @@ private: bool is_initialized = false; }; -} \ No newline at end of file +} diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp index 4c614d8fd5a..3cc79756981 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp @@ -1,6 +1,8 @@ #include "PostgreSQLReplicationHandler.h" #include +#include +#include #include #include #include @@ -226,9 +228,17 @@ StoragePtr PostgreSQLReplicationHandler::loadFromSnapshot(String & snapshot_name const StorageInMemoryMetadata & storage_metadata = nested_storage->getInMemoryMetadata(); auto sample_block = storage_metadata.getSampleBlockNonMaterialized(); - PostgreSQLTransactionBlockInputStream input(tx, query_str, sample_block, DEFAULT_BLOCK_SIZE); - assertBlocksHaveEqualStructure(input.getHeader(), block_io.out->getHeader(), "postgresql replica load from snapshot"); - copyData(input, *block_io.out); + auto input = std::make_unique>(tx, query_str, sample_block, DEFAULT_BLOCK_SIZE); + QueryPipeline pipeline; + pipeline.init(Pipe(std::move(input))); + assertBlocksHaveEqualStructure(pipeline.getHeader(), block_io.out->getHeader(), "postgresql replica load from snapshot"); + + PullingPipelineExecutor executor(pipeline); + Block block; + block_io.out->writePrefix(); + while (executor.pull(block)) + block_io.out->write(block); + block_io.out->writeSuffix(); nested_storage = materialized_storage->prepare(); auto nested_table_id = nested_storage->getStorageID(); diff --git a/src/Storages/StorageDictionary.cpp b/src/Storages/StorageDictionary.cpp index 30a9dad8d91..c8bbb703999 100644 --- a/src/Storages/StorageDictionary.cpp +++ b/src/Storages/StorageDictionary.cpp @@ -169,9 +169,7 @@ Pipe StorageDictionary::read( { auto registered_dictionary_name = location == Location::SameDatabaseAndNameAsDictionary ? getStorageID().getInternalDictionaryName() : dictionary_name; auto dictionary = getContext()->getExternalDictionariesLoader().getDictionary(registered_dictionary_name, local_context); - auto stream = dictionary->getBlockInputStream(column_names, max_block_size); - /// TODO: update dictionary interface for processors. - return Pipe(std::make_shared(stream)); + return dictionary->read(column_names, max_block_size); } void StorageDictionary::shutdown() diff --git a/src/Storages/StorageMongoDB.cpp b/src/Storages/StorageMongoDB.cpp index 1fd58a293dc..a973efd7277 100644 --- a/src/Storages/StorageMongoDB.cpp +++ b/src/Storages/StorageMongoDB.cpp @@ -99,8 +99,7 @@ Pipe StorageMongoDB::read( sample_block.insert({ column_data.type, column_data.name }); } - return Pipe(std::make_shared( - std::make_shared(connection, createCursor(database_name, collection_name, sample_block), sample_block, max_block_size, true))); + return Pipe(std::make_shared(connection, createCursor(database_name, collection_name, sample_block), sample_block, max_block_size, true)); } void registerStorageMongoDB(StorageFactory & factory) diff --git a/src/Storages/StorageMySQL.cpp b/src/Storages/StorageMySQL.cpp index 99a930f37c4..431fda530f4 100644 --- a/src/Storages/StorageMySQL.cpp +++ b/src/Storages/StorageMySQL.cpp @@ -104,8 +104,7 @@ Pipe StorageMySQL::read( StreamSettings mysql_input_stream_settings(context_->getSettingsRef(), mysql_settings.connection_auto_close); - return Pipe(std::make_shared( - std::make_shared(pool, query, sample_block, mysql_input_stream_settings))); + return Pipe(std::make_shared(pool, query, sample_block, mysql_input_stream_settings)); } diff --git a/src/Storages/StoragePostgreSQL.cpp b/src/Storages/StoragePostgreSQL.cpp index 6072412af35..b71f2415fd8 100644 --- a/src/Storages/StoragePostgreSQL.cpp +++ b/src/Storages/StoragePostgreSQL.cpp @@ -90,8 +90,7 @@ Pipe StoragePostgreSQL::read( sample_block.insert({ column_data.type, column_data.name }); } - return Pipe(std::make_shared( - std::make_shared>(pool->get(), query, sample_block, max_block_size_))); + return Pipe(std::make_shared>(pool->get(), query, sample_block, max_block_size_)); } From f16a26ee6a110738655ac7a198394ade32c81ba3 Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Fri, 6 Aug 2021 11:43:46 +0300 Subject: [PATCH 194/599] Minor fix. --- docs/en/engines/database-engines/materialized-mysql.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/engines/database-engines/materialized-mysql.md b/docs/en/engines/database-engines/materialized-mysql.md index 1c69fd072d8..27af13a6976 100644 --- a/docs/en/engines/database-engines/materialized-mysql.md +++ b/docs/en/engines/database-engines/materialized-mysql.md @@ -54,7 +54,7 @@ For the correct work of `MaterializeMySQL`, there are few mandatory `MySQL`-side !!! attention "Attention" While turning on `gtid_mode` you should also specify `enforce_gtid_consistency = on`. -## Virtual columns {#virtual-columns} +## Virtual Columns {#virtual-columns} When working with the `MaterializedMySQL` database engine, [ReplacingMergeTree](../../engines/table-engines/mergetree-family/replacingmergetree.md) tables are used with virtual `_sign` and `_version` columns. @@ -90,7 +90,7 @@ Other types are not supported. If MySQL table contains a column of such type, Cl ## Specifics and Recommendations {#specifics-and-recommendations} -### Compatibility restrictions {#compatibility-restrictions} +### Compatibility Restrictions {#compatibility-restrictions} Apart of the data types limitations there are few restrictions comparing to `MySQL` databases, that should be resolved before replication will be possible: From 0f2c880876949cd8f024136887be7956a0701259 Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Fri, 6 Aug 2021 11:45:19 +0300 Subject: [PATCH 195/599] Minor fix. --- docs/ru/engines/database-engines/materialized-mysql.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/engines/database-engines/materialized-mysql.md b/docs/ru/engines/database-engines/materialized-mysql.md index 7b14867fa62..ef0d9be054e 100644 --- a/docs/ru/engines/database-engines/materialized-mysql.md +++ b/docs/ru/engines/database-engines/materialized-mysql.md @@ -33,7 +33,7 @@ ENGINE = MaterializedMySQL('host:port', ['database' | database], 'user', 'passwo - `max_bytes_in_buffer` — максимальное количество байтов, которое разрешено кэшировать в памяти (для одной таблицы и данных кэша, которые невозможно запросить). При превышении количества строк, данные будут материализованы. Значение по умолчанию: `1 048 576`. - `max_rows_in_buffers` — максимальное количество строк, содержимое которых может кэшироваться в памяти (для базы данных и данных кэша, которые невозможно запросить). При превышении количества строк, данные будут материализованы. Значение по умолчанию: `65 505`. - `max_bytes_in_buffers` — максимальное количество байтов, которое разрешено кэшировать данным в памяти (для базы данных и данных кэша, которые невозможно запросить). При превышении количества строк, данные будут материализованы. Значение по умолчанию: `1 048 576`. -- `max_flush_data_time` — Максимальное время в миллисекундах, в течение которого разрешено кэшировать данные в памяти (для базы данных и данных кэша, которые невозможно запросить). При превышении количества указанного периода, данные будут материализованы. Значение по умолчанию: `1000`. +- `max_flush_data_time` — максимальное время в миллисекундах, в течение которого разрешено кэшировать данные в памяти (для базы данных и данных кэша, которые невозможно запросить). При превышении количества указанного периода, данные будут материализованы. Значение по умолчанию: `1000`. - `max_wait_time_when_mysql_unavailable` — интервал повторных попыток, если MySQL недоступен. Указывается в миллисекундах. Отрицательное значение отключает повторные попытки. Значение по умолчанию: `1000`. - `allows_query_when_mysql_lost` — разрешить запрос материализованной таблицы при потере MySQL. Значение по умолчанию: `0` (`false`). From ae756dd2b0d8ba53cbf7e150cdfa98ec5e4f003f Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Fri, 6 Aug 2021 12:04:55 +0300 Subject: [PATCH 196/599] Add stings empty nonempty functions --- .../functions/string-functions.md | 38 +++++++++++++-- .../functions/string-functions.md | 46 ++++++++++++++++--- 2 files changed, 74 insertions(+), 10 deletions(-) diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index 8723b9d278a..0156b4331ba 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -48,9 +48,41 @@ Result: ## notEmpty {#notempty} -Returns 0 for an empty string or 1 for a non-empty string. -The result type is UInt8. -The function also works for arrays or UUID. +Checks whether the input string is not empty. + +**Syntax** + +``` sql +notempty(x) +``` + +A string is considered non-empty if it contains at least one byte, even if this is a space or a null byte. The UUID is empty if it contains all zeros (zero UUID). + +**Arguments** + +- `x` — Input value. [Array](../data-types/array.md), [String](../data-types/string.md), [UUID](../data-types/uuid.md). + +**Returned value** + +- Returns `1` for a non-empty string or `0` for an empty string string. + +Type: [UInt8](../data-types/int-uint.md). + +**Example** + +Query: + +```sql +SELECT notempty('text'); +``` + +Result: + +```text +┌─empty('')─┐ +│ 1 │ +└───────────┘ +``` ## length {#length} diff --git a/docs/ru/sql-reference/functions/string-functions.md b/docs/ru/sql-reference/functions/string-functions.md index e6c8e17bdca..097ff593ef7 100644 --- a/docs/ru/sql-reference/functions/string-functions.md +++ b/docs/ru/sql-reference/functions/string-functions.md @@ -23,7 +23,45 @@ empty(x) **Возвращаемое значение** -- Возвращает `1` для пустой строки, и `0` — для непустой строки. +- Возвращает `1` для пустой строки и `0` — для непустой строки. + +Тип: [UInt8](../data-types/int-uint.md). + +**Пример** + +Запрос: + +```sql +SELECT notempty('text'); +``` + +Результат: + +```text +┌─empty('')─┐ +│ 1 │ +└───────────┘ +``` + +## notEmpty {#notempty} + +Проверяет, является ли входная строка непустой. + +**Синтаксис** + +``` sql +empty(x) +``` + +Строка считается непустой, если содержит хотя бы один байт, пусть даже это пробел или нулевой байт. UUID считается пустой, если он содержит только нули (нулевой UUID). + +**Параметры** + +- `x` — Входная строка. [Array](../data-types/array.md), [String](../data-types/string.md), [UUID](../data-types/uuid.md). + +**Возвращаемое значение** + +- Возвращает `1` для непустой строки и `0` — для пустой строки. Тип: [UInt8](../data-types/int-uint.md). @@ -43,12 +81,6 @@ SELECT empty(''); └───────────┘ ``` -## notEmpty {#notempty} - -Возвращает 0 для пустой строки, и 1 для непустой строки. -Тип результата — UInt8. -Функция также работает для массивов. - ## length {#length} Возвращает длину строки в байтах (не символах, не кодовых точках). From afd3341b29192cf8232883dd44c9fe7aa0b4d1d0 Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Fri, 6 Aug 2021 12:11:08 +0300 Subject: [PATCH 197/599] Minor fix. --- docs/en/engines/database-engines/materialized-mysql.md | 2 +- docs/ru/engines/database-engines/materialized-mysql.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/engines/database-engines/materialized-mysql.md b/docs/en/engines/database-engines/materialized-mysql.md index 27af13a6976..163c86df84f 100644 --- a/docs/en/engines/database-engines/materialized-mysql.md +++ b/docs/en/engines/database-engines/materialized-mysql.md @@ -1,6 +1,6 @@ --- toc_priority: 29 -toc_title: [experimental] MaterializedMySQL +toc_title: "[experimental] MaterializedMySQL" --- # [experimental] MaterializedMySQL {#materialized-mysql} diff --git a/docs/ru/engines/database-engines/materialized-mysql.md b/docs/ru/engines/database-engines/materialized-mysql.md index ef0d9be054e..2f57b9f5a84 100644 --- a/docs/ru/engines/database-engines/materialized-mysql.md +++ b/docs/ru/engines/database-engines/materialized-mysql.md @@ -1,6 +1,6 @@ --- toc_priority: 29 -toc_title: [experimental] MaterializedMySQL +toc_title: "[experimental] MaterializedMySQL" --- # [experimental] MaterializedMySQL {#materialized-mysql} From 985a906c575b2e3eb04ad432303d6abd4e6ab6fa Mon Sep 17 00:00:00 2001 From: Artur <613623@mail.ru> Date: Fri, 6 Aug 2021 09:46:39 +0000 Subject: [PATCH 198/599] correcting tests --- .../02003_memory_limit_in_client.reference | 125 ++++++++++++++++++ .../02003_memory_limit_in_client.sh | 8 +- 2 files changed, 129 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/02003_memory_limit_in_client.reference b/tests/queries/0_stateless/02003_memory_limit_in_client.reference index 9f074d19d33..0b82ccc5544 100644 --- a/tests/queries/0_stateless/02003_memory_limit_in_client.reference +++ b/tests/queries/0_stateless/02003_memory_limit_in_client.reference @@ -59998,3 +59998,128 @@ 59997 59998 59999 +[] +[[]] +[[],[0]] +[[],[0],[0,1]] +[[],[0],[0,1],[0,1,2]] +[[],[0],[0,1],[0,1,2],[0,1,2,3]] +[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4]] +[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5]] +[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6]] +[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7]] +[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8]] +[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9]] +[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10]] +[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11]] +[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12]] +[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13]] +[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14]] +[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15]] +[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16]] +[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17]] +[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18]] +[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19]] +[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20]] +[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21]] +[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22]] +[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23]] +[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24]] +[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25]] +[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26]] +[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27]] +[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28]] +[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29]] +[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30]] +[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31]] +[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32]] +[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33]] +[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34]] +[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35]] +[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36]] +[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37]] +[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38]] +[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39]] +[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40]] +[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41]] +[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42]] +[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43]] +[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44]] +[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45]] +[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46]] +[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47]] +[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48]] +[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49]] +[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50]] +[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51]] +[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52]] +[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53]] +[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54]] +[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55]] +[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56]] +[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57]] +[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58]] +[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59]] +[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60]] +[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61]] +[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62]] +[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63]] +[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64]] +[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65]] +[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66]] +[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67]] +[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68]] +[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69]] +[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70]] +[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71]] +[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72]] +[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73]] +[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74]] +[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75]] +[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76]] +[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77]] +[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78]] +[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79]] +[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80]] +[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81]] +[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82]] +[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83]] +[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84]] +[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85]] +[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86]] +[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87]] +[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88]] +[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89]] +[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90]] +[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91]] +[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92]] +[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93]] +[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94]] +[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95]] +[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96]] +[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97]] +[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98]] +[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99]] +[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100]] +[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101]] +[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102]] +[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103]] +[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104]] +[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105]] +[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106]] +[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107]] +[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108]] +[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109]] +[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110]] +[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111]] +[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112]] +[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113]] +[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113,114]] +[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113,114],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113,114,115]] +[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113,114],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113,114,115],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113,114,115,116]] +[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113,114],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113,114,115],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113,114,115,116],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113,114,115,116,117]] +[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113,114],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113,114,115],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113,114,115,116],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113,114,115,116,117],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113,114,115,116,117,118]] +[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113,114],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113,114,115],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113,114,115,116],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113,114,115,116,117],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113,114,115,116,117,118],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113,114,115,116,117,118,119]] +[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113,114],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113,114,115],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113,114,115,116],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113,114,115,116,117],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113,114,115,116,117,118],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113,114,115,116,117,118,119],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113,114,115,116,117,118,119,120]] +[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113,114],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113,114,115],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113,114,115,116],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113,114,115,116,117],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113,114,115,116,117,118],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113,114,115,116,117,118,119],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113,114,115,116,117,118,119,120],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113,114,115,116,117,118,119,120,121]] +[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113,114],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113,114,115],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113,114,115,116],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113,114,115,116,117],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113,114,115,116,117,118],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113,114,115,116,117,118,119],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113,114,115,116,117,118,119,120],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113,114,115,116,117,118,119,120,121],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113,114,115,116,117,118,119,120,121,122]] diff --git a/tests/queries/0_stateless/02003_memory_limit_in_client.sh b/tests/queries/0_stateless/02003_memory_limit_in_client.sh index 1cb3014c38f..29e4f1ae283 100755 --- a/tests/queries/0_stateless/02003_memory_limit_in_client.sh +++ b/tests/queries/0_stateless/02003_memory_limit_in_client.sh @@ -5,10 +5,10 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CURDIR"/../shell_config.sh # Big quiery. Memory limit -${CLICKHOUSE_CLIENT} --max_memory_usage_in_client=1 --query "SELECT arrayMap(x -> range(x), range(number)) FROM numbers(1000) -- { serverError 241; }" +${CLICKHOUSE_CLIENT} --max_memory_usage_in_client=1 --testmode --query "SELECT arrayMap(x -> range(x), range(number)) FROM numbers(1000) -- { serverError 241; }" # small quiery. In max untracked memory range -${CLICKHOUSE_CLIENT} --max_memory_usage_in_client=1 --query "SELECT * FROM system.numbers LIMIT 60000" +${CLICKHOUSE_CLIENT} --max_memory_usage_in_client=1 --testmode --query "SELECT * FROM system.numbers LIMIT 60000" -# Have anough memory -${CLICKHOUSE_CLIENT} --max_memory_usage_in_client=20000000 --query "SELECT arrayMap(x -> range(x), range(number)) FROM numbers(1000)" +# Have anough memory (unlimited). Better not to set limit, because of adress sanitizer +${CLICKHOUSE_CLIENT} --testmode --query "SELECT arrayMap(x -> range(x), range(number)) FROM numbers(125)" From a0f9df4d8711083c3f0bf1b4f38665b8566c6fe9 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 6 Aug 2021 13:06:05 +0300 Subject: [PATCH 199/599] Fix 01682_cache_dictionary_complex_key --- src/Dictionaries/CacheDictionary.cpp | 12 ++++-------- 1 file changed, 4 insertions(+), 8 deletions(-) diff --git a/src/Dictionaries/CacheDictionary.cpp b/src/Dictionaries/CacheDictionary.cpp index 39186f0d63f..4dfe802dd2b 100644 --- a/src/Dictionaries/CacheDictionary.cpp +++ b/src/Dictionaries/CacheDictionary.cpp @@ -487,25 +487,21 @@ template Pipe CacheDictionary::read(const Names & column_names, size_t max_block_size) const { Pipe pipe; - + std::optional data; { /// Write lock on storage const ProfilingScopedWriteRWLock write_lock{rw_lock, ProfileEvents::DictCacheLockWriteNs}; if constexpr (dictionary_key_type == DictionaryKeyType::simple) - pipe = Pipe(std::make_shared( - DictionarySourceData(shared_from_this(), cache_storage_ptr->getCachedSimpleKeys(), column_names), - max_block_size)); + data.emplace(shared_from_this(), cache_storage_ptr->getCachedSimpleKeys(), column_names); else { auto keys = cache_storage_ptr->getCachedComplexKeys(); - pipe = Pipe(std::make_shared( - DictionarySourceData(shared_from_this(), keys, column_names), - max_block_size)); + data.emplace(shared_from_this(), keys, column_names); } } - return pipe; + return Pipe(std::make_shared(std::move(*data), max_block_size)); } template From b9ca3179dee154ea650899c80f4ffdfda0c2c636 Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 5 Aug 2021 13:31:40 +0300 Subject: [PATCH 200/599] Check if NotJoined::right_nullability_removes is used --- src/Interpreters/join_common.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Interpreters/join_common.cpp b/src/Interpreters/join_common.cpp index 9d6abda42ed..06c51cc3d53 100644 --- a/src/Interpreters/join_common.cpp +++ b/src/Interpreters/join_common.cpp @@ -552,6 +552,8 @@ void NotJoined::extractColumnChanges(size_t right_pos, size_t result_pos) void NotJoined::correctLowcardAndNullability(MutableColumns & columns_right) { + + assert(right_nullability_removes.size() == 0); for (size_t pos : right_nullability_removes) changeNullability(columns_right[pos]); From 9a40ce87e98a1c494dde0ddc19b6520b26dfe0b6 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Fri, 6 Aug 2021 13:55:49 +0300 Subject: [PATCH 201/599] Use bytes instead of strings in the GRPC protocol. --- src/Server/grpc_protos/clickhouse_grpc.proto | 12 ++-- tests/integration/test_grpc_protocol/test.py | 63 +++++++++++-------- .../test_grpc_protocol_ssl/test.py | 11 ++-- utils/grpc-client/clickhouse-grpc-client.py | 32 +++++----- 4 files changed, 65 insertions(+), 53 deletions(-) diff --git a/src/Server/grpc_protos/clickhouse_grpc.proto b/src/Server/grpc_protos/clickhouse_grpc.proto index 2f25973297c..da01432d76c 100644 --- a/src/Server/grpc_protos/clickhouse_grpc.proto +++ b/src/Server/grpc_protos/clickhouse_grpc.proto @@ -32,7 +32,7 @@ message ExternalTable { // Data to insert to the external table. // If a method with streaming input (i.e. ExecuteQueryWithStreamInput() or ExecuteQueryWithStreamIO()) is used, // then data for insertion to the same external table can be split between multiple QueryInfos. - string data = 3; + bytes data = 3; // Format of the data to insert to the external table. string format = 4; @@ -53,10 +53,10 @@ message QueryInfo { string database = 4; // Input data, used both as data for INSERT query and as data for the input() function. - string input_data = 5; + bytes input_data = 5; // Delimiter for input_data, inserted between input_data from adjacent QueryInfos. - string input_data_delimiter = 6; + bytes input_data_delimiter = 6; // Default output format. If not specified, 'TabSeparated' is used. string output_format = 7; @@ -128,9 +128,9 @@ message Exception { // Result of execution of a query which is sent back by the ClickHouse server to the client. message Result { // Output of the query, represented in the `output_format` or in a format specified in `query`. - string output = 1; - string totals = 2; - string extremes = 3; + bytes output = 1; + bytes totals = 2; + bytes extremes = 3; repeated LogEntry logs = 4; Progress progress = 5; diff --git a/tests/integration/test_grpc_protocol/test.py b/tests/integration/test_grpc_protocol/test.py index ee7e94bad1d..b0c1f8067b6 100644 --- a/tests/integration/test_grpc_protocol/test.py +++ b/tests/integration/test_grpc_protocol/test.py @@ -6,7 +6,9 @@ import grpc from helpers.cluster import ClickHouseCluster, run_and_check from threading import Thread +GRPC_PORT = 9100 SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) +DEFAULT_ENCODING = 'utf-8' # Use grpcio-tools to generate *pb2.py files from *.proto. @@ -28,11 +30,10 @@ import clickhouse_grpc_pb2_grpc config_dir = os.path.join(SCRIPT_DIR, './configs') cluster = ClickHouseCluster(__file__) node = cluster.add_instance('node', main_configs=['configs/grpc_config.xml']) -grpc_port = 9100 main_channel = None def create_channel(): - node_ip_with_grpc_port = cluster.get_instance_ip('node') + ':' + str(grpc_port) + node_ip_with_grpc_port = cluster.get_instance_ip('node') + ':' + str(GRPC_PORT) channel = grpc.insecure_channel(node_ip_with_grpc_port) grpc.channel_ready_future(channel).result(timeout=10) global main_channel @@ -42,20 +43,27 @@ def create_channel(): def query_common(query_text, settings={}, input_data=[], input_data_delimiter='', output_format='TabSeparated', external_tables=[], user_name='', password='', query_id='123', session_id='', stream_output=False, channel=None): - if type(input_data) == str: + if type(input_data) is not list: input_data = [input_data] + if type(input_data_delimiter) is str: + input_data_delimiter=input_data_delimiter.encode(DEFAULT_ENCODING) if not channel: channel = main_channel stub = clickhouse_grpc_pb2_grpc.ClickHouseStub(channel) def query_info(): - input_data_part = input_data.pop(0) if input_data else '' - return clickhouse_grpc_pb2.QueryInfo(query=query_text, settings=settings, input_data=input_data_part, input_data_delimiter=input_data_delimiter, - output_format=output_format, external_tables=external_tables, user_name=user_name, password=password, - query_id=query_id, session_id=session_id, next_query_info=bool(input_data)) + input_data_part = input_data.pop(0) if input_data else b'' + if type(input_data_part) is str: + input_data_part = input_data_part.encode(DEFAULT_ENCODING) + return clickhouse_grpc_pb2.QueryInfo(query=query_text, settings=settings, input_data=input_data_part, + input_data_delimiter=input_data_delimiter, output_format=output_format, + external_tables=external_tables, user_name=user_name, password=password, query_id=query_id, + session_id=session_id, next_query_info=bool(input_data)) def send_query_info(): yield query_info() while input_data: input_data_part = input_data.pop(0) + if type(input_data_part) is str: + input_data_part = input_data_part.encode(DEFAULT_ENCODING) yield clickhouse_grpc_pb2.QueryInfo(input_data=input_data_part, next_query_info=bool(input_data)) stream_input = len(input_data) > 1 if stream_input and stream_output: @@ -74,10 +82,10 @@ def query_no_errors(*args, **kwargs): return results def query(*args, **kwargs): - output = "" + output = b'' for result in query_no_errors(*args, **kwargs): output += result.output - return output + return output.decode(DEFAULT_ENCODING) def query_and_get_error(*args, **kwargs): results = query_common(*args, **kwargs) @@ -86,16 +94,16 @@ def query_and_get_error(*args, **kwargs): return results[-1].exception def query_and_get_totals(*args, **kwargs): - totals = "" + totals = b'' for result in query_no_errors(*args, **kwargs): totals += result.totals - return totals + return totals.decode(DEFAULT_ENCODING) def query_and_get_extremes(*args, **kwargs): - extremes = "" + extremes = b'' for result in query_no_errors(*args, **kwargs): extremes += result.extremes - return extremes + return extremes.decode(DEFAULT_ENCODING) def query_and_get_logs(*args, **kwargs): logs = "" @@ -135,6 +143,7 @@ def reset_after_test(): yield node.query_with_retry("DROP TABLE IF EXISTS t") + # Actual tests def test_select_one(): @@ -270,18 +279,18 @@ def test_input_function(): def test_external_table(): columns = [clickhouse_grpc_pb2.NameAndType(name='UserID', type='UInt64'), clickhouse_grpc_pb2.NameAndType(name='UserName', type='String')] - ext1 = clickhouse_grpc_pb2.ExternalTable(name='ext1', columns=columns, data='1\tAlex\n2\tBen\n3\tCarl\n', format='TabSeparated') + ext1 = clickhouse_grpc_pb2.ExternalTable(name='ext1', columns=columns, data=b'1\tAlex\n2\tBen\n3\tCarl\n', format='TabSeparated') assert query("SELECT * FROM ext1 ORDER BY UserID", external_tables=[ext1]) == "1\tAlex\n"\ "2\tBen\n"\ "3\tCarl\n" - ext2 = clickhouse_grpc_pb2.ExternalTable(name='ext2', columns=columns, data='4,Daniel\n5,Ethan\n', format='CSV') + ext2 = clickhouse_grpc_pb2.ExternalTable(name='ext2', columns=columns, data=b'4,Daniel\n5,Ethan\n', format='CSV') assert query("SELECT * FROM (SELECT * FROM ext1 UNION ALL SELECT * FROM ext2) ORDER BY UserID", external_tables=[ext1, ext2]) == "1\tAlex\n"\ "2\tBen\n"\ "3\tCarl\n"\ "4\tDaniel\n"\ "5\tEthan\n" unnamed_columns = [clickhouse_grpc_pb2.NameAndType(type='UInt64'), clickhouse_grpc_pb2.NameAndType(type='String')] - unnamed_table = clickhouse_grpc_pb2.ExternalTable(columns=unnamed_columns, data='6\tGeorge\n7\tFred\n') + unnamed_table = clickhouse_grpc_pb2.ExternalTable(columns=unnamed_columns, data=b'6\tGeorge\n7\tFred\n') assert query("SELECT * FROM _data ORDER BY _2", external_tables=[unnamed_table]) == "7\tFred\n"\ "6\tGeorge\n" @@ -289,16 +298,16 @@ def test_external_table_streaming(): columns = [clickhouse_grpc_pb2.NameAndType(name='UserID', type='UInt64'), clickhouse_grpc_pb2.NameAndType(name='UserName', type='String')] def send_query_info(): yield clickhouse_grpc_pb2.QueryInfo(query="SELECT * FROM exts ORDER BY UserID", - external_tables=[clickhouse_grpc_pb2.ExternalTable(name='exts', columns=columns, data='1\tAlex\n2\tBen\n3\tCarl\n')], + external_tables=[clickhouse_grpc_pb2.ExternalTable(name='exts', columns=columns, data=b'1\tAlex\n2\tBen\n3\tCarl\n')], next_query_info=True) - yield clickhouse_grpc_pb2.QueryInfo(external_tables=[clickhouse_grpc_pb2.ExternalTable(name='exts', data='4\tDaniel\n5\tEthan\n')]) + yield clickhouse_grpc_pb2.QueryInfo(external_tables=[clickhouse_grpc_pb2.ExternalTable(name='exts', data=b'4\tDaniel\n5\tEthan\n')]) stub = clickhouse_grpc_pb2_grpc.ClickHouseStub(main_channel) result = stub.ExecuteQueryWithStreamInput(send_query_info()) - assert result.output == "1\tAlex\n"\ - "2\tBen\n"\ - "3\tCarl\n"\ - "4\tDaniel\n"\ - "5\tEthan\n" + assert result.output == b'1\tAlex\n'\ + b'2\tBen\n'\ + b'3\tCarl\n'\ + b'4\tDaniel\n'\ + b'5\tEthan\n' def test_simultaneous_queries_same_channel(): threads=[] @@ -325,8 +334,8 @@ def test_simultaneous_queries_multiple_channels(): def test_cancel_while_processing_input(): query("CREATE TABLE t (a UInt8) ENGINE = Memory") def send_query_info(): - yield clickhouse_grpc_pb2.QueryInfo(query="INSERT INTO t FORMAT TabSeparated", input_data="1\n2\n3\n", next_query_info=True) - yield clickhouse_grpc_pb2.QueryInfo(input_data="4\n5\n6\n", next_query_info=True) + yield clickhouse_grpc_pb2.QueryInfo(query="INSERT INTO t FORMAT TabSeparated", input_data=b'1\n2\n3\n', next_query_info=True) + yield clickhouse_grpc_pb2.QueryInfo(input_data=b'4\n5\n6\n', next_query_info=True) yield clickhouse_grpc_pb2.QueryInfo(cancel=True) stub = clickhouse_grpc_pb2_grpc.ClickHouseStub(main_channel) result = stub.ExecuteQueryWithStreamInput(send_query_info()) @@ -343,7 +352,7 @@ def test_cancel_while_generating_output(): results = list(stub.ExecuteQueryWithStreamIO(send_query_info())) assert len(results) >= 1 assert results[-1].cancelled == True - output = '' + output = b'' for result in results: output += result.output - assert output == '0\t0\n1\t0\n2\t0\n3\t0\n' + assert output == b'0\t0\n1\t0\n2\t0\n3\t0\n' diff --git a/tests/integration/test_grpc_protocol_ssl/test.py b/tests/integration/test_grpc_protocol_ssl/test.py index c040ccd041f..1f21fbe5f8a 100644 --- a/tests/integration/test_grpc_protocol_ssl/test.py +++ b/tests/integration/test_grpc_protocol_ssl/test.py @@ -4,7 +4,10 @@ import sys import grpc from helpers.cluster import ClickHouseCluster, run_and_check +GRPC_PORT = 9100 +NODE_IP = '10.5.172.77' # It's important for the node to work at this IP because 'server-cert.pem' requires that (see server-ext.cnf). SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) +DEFAULT_ENCODING = 'utf-8' # Use grpcio-tools to generate *pb2.py files from *.proto. @@ -23,12 +26,10 @@ import clickhouse_grpc_pb2_grpc # Utilities -node_ip = '10.5.172.77' # It's important for the node to work at this IP because 'server-cert.pem' requires that (see server-ext.cnf). -grpc_port = 9100 -node_ip_with_grpc_port = node_ip + ':' + str(grpc_port) +node_ip_with_grpc_port = NODE_IP + ':' + str(GRPC_PORT) config_dir = os.path.join(SCRIPT_DIR, './configs') cluster = ClickHouseCluster(__file__) -node = cluster.add_instance('node', ipv4_address=node_ip, main_configs=['configs/grpc_config.xml', 'configs/server-key.pem', 'configs/server-cert.pem', 'configs/ca-cert.pem']) +node = cluster.add_instance('node', ipv4_address=NODE_IP, main_configs=['configs/grpc_config.xml', 'configs/server-key.pem', 'configs/server-cert.pem', 'configs/ca-cert.pem']) def create_secure_channel(): ca_cert = open(os.path.join(config_dir, 'ca-cert.pem'), 'rb').read() @@ -59,7 +60,7 @@ def query(query_text, channel): result = stub.ExecuteQuery(query_info) if result and result.HasField('exception'): raise Exception(result.exception.display_text) - return result.output + return result.output.decode(DEFAULT_ENCODING) @pytest.fixture(scope="module", autouse=True) def start_cluster(): diff --git a/utils/grpc-client/clickhouse-grpc-client.py b/utils/grpc-client/clickhouse-grpc-client.py index 19d213a8e3f..dfaa7ed4e01 100755 --- a/utils/grpc-client/clickhouse-grpc-client.py +++ b/utils/grpc-client/clickhouse-grpc-client.py @@ -14,13 +14,14 @@ import grpc # pip3 install grpcio import grpc_tools # pip3 install grpcio-tools import argparse, cmd, os, signal, subprocess, sys, threading, time, uuid -default_host = 'localhost' -default_port = 9100 -default_user_name = 'default' -default_output_format_for_interactive_mode = 'PrettyCompact' -history_filename = '~/.clickhouse_grpc_history' -history_size = 1000 -stdin_bufsize = 1048576 +DEFAULT_HOST = 'localhost' +DEFAULT_PORT = 9100 +DEFAULT_USER_NAME = 'default' +DEFAULT_OUTPUT_FORMAT_FOR_INTERACTIVE_MODE = 'PrettyCompact' +HISTORY_FILENAME = '~/.clickhouse_grpc_history' +HISTORY_SIZE = 1000 +STDIN_BUFFER_SIZE = 1048576 +DEFAULT_ENCODING = 'utf-8' class ClickHouseGRPCError(Exception): @@ -52,7 +53,7 @@ def error_print(*args, **kwargs): class ClickHouseGRPCClient(cmd.Cmd): prompt="grpc :) " - def __init__(self, host=default_host, port=default_port, user_name=default_user_name, password='', + def __init__(self, host=DEFAULT_HOST, port=DEFAULT_PORT, user_name=DEFAULT_USER_NAME, password='', database='', output_format='', settings='', verbatim=False, show_debug_info=False): super(ClickHouseGRPCClient, self).__init__(completekey=None) self.host = host @@ -85,7 +86,7 @@ class ClickHouseGRPCClient(cmd.Cmd): if self.show_debug_info: print('\nresult={}'.format(result)) ClickHouseGRPCClient.__check_no_errors(result) - return result.output + return result.output.decode(DEFAULT_ENCODING) # Executes a query using the stream IO and with ability to cancel it by pressing Ctrl+C. def run_query(self, query_text, raise_exceptions=True, allow_cancel=False): @@ -117,7 +118,7 @@ class ClickHouseGRPCClient(cmd.Cmd): # send input data if not sys.stdin.isatty(): while True: - info.input_data = sys.stdin.buffer.read(stdin_bufsize) + info.input_data = sys.stdin.buffer.read(STDIN_BUFFER_SIZE) if not info.input_data: break info.next_query_info = True @@ -134,7 +135,8 @@ class ClickHouseGRPCClient(cmd.Cmd): if self.show_debug_info: print('\nresult={}'.format(result)) ClickHouseGRPCClient.__check_no_errors(result) - print(result.output, end='') + sys.stdout.buffer.write(result.output) + sys.stdout.flush() if result.cancelled: cancelled = True self.verbatim_print("Query was cancelled.") @@ -244,7 +246,7 @@ class ClickHouseGRPCClient(cmd.Cmd): import readline except ImportError: readline = None - histfile = os.path.expanduser(history_filename) + histfile = os.path.expanduser(HISTORY_FILENAME) if readline and os.path.exists(histfile): readline.read_history_file(histfile) @@ -252,8 +254,8 @@ class ClickHouseGRPCClient(cmd.Cmd): def __write_history(): global readline if readline: - readline.set_history_length(history_size) - histfile = os.path.expanduser(history_filename) + readline.set_history_length(HISTORY_SIZE) + histfile = os.path.expanduser(HISTORY_FILENAME) readline.write_history_file(histfile) @@ -281,7 +283,7 @@ def main(args): output_format = args.output_format if not output_format and interactive_mode: - output_format = default_output_format_for_interactive_mode + output_format = DEFAULT_OUTPUT_FORMAT_FOR_INTERACTIVE_MODE try: with ClickHouseGRPCClient(host=args.host, port=args.port, user_name=args.user_name, password=args.password, From 226db3f6bead3b6839894e10f80189c66b94fb60 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 6 Aug 2021 13:38:30 +0200 Subject: [PATCH 202/599] Integration runner: Output longer errors --- tests/integration/runner | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/runner b/tests/integration/runner index 36cb4f22f9a..2143d7ebf29 100755 --- a/tests/integration/runner +++ b/tests/integration/runner @@ -277,7 +277,7 @@ if __name__ == "__main__": --volume={base_cfg}:/clickhouse-config --volume={cases_dir}:/ClickHouse/tests/integration \ --volume={src_dir}/Server/grpc_protos:/ClickHouse/src/Server/grpc_protos \ {dockerd_internal_volume} -e DOCKER_CLIENT_TIMEOUT=300 -e COMPOSE_HTTP_TIMEOUT=600 \ - {env_tags} {env_cleanup} -e PYTEST_OPTS='{parallel} {opts} {tests_list}' {img} {command}".format( + {env_tags} {env_cleanup} -e PYTEST_OPTS='{parallel} {opts} {tests_list} -vvv' {img} {command}".format( net=net, tty=tty, bin=args.binary, From 4f1b02426a5db73714f3a09e8ab83574418d8672 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 6 Aug 2021 13:38:56 +0200 Subject: [PATCH 203/599] test_storage_kafka: Remove duplicated code --- tests/integration/test_storage_kafka/test.py | 107 +------------------ 1 file changed, 2 insertions(+), 105 deletions(-) diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index b9fc0b2272f..3c91f26fde8 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -2,7 +2,6 @@ import json import os.path as p import random import socket -import subprocess import threading import time import logging @@ -181,32 +180,6 @@ def avro_confluent_message(schema_registry_client, value): }) return serializer.encode_record_with_schema('test_subject', schema, value) -# Since everything is async and shaky when receiving messages from Kafka, -# we may want to try and check results multiple times in a loop. -def kafka_check_result(result, check=False, ref_file='test_kafka_json.reference'): - fpath = p.join(p.dirname(__file__), ref_file) - with open(fpath) as reference: - if check: - assert TSV(result) == TSV(reference) - else: - return TSV(result) == TSV(reference) - -def describe_consumer_group(kafka_cluster, name): - admin_client = KafkaAdminClient(bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port)) - consumer_groups = admin_client.describe_consumer_groups([name]) - res = [] - for member in consumer_groups[0].members: - member_info = {} - member_info['member_id'] = member.member_id - member_info['client_id'] = member.client_id - member_info['client_host'] = member.client_host - member_topics_assignment = [] - for (topic, partitions) in member.member_assignment.assignment: - member_topics_assignment.append({'topic': topic, 'partitions': partitions}) - member_info['assignment'] = member_topics_assignment - res.append(member_info) - return res - # Fixtures @pytest.fixture(scope="module") @@ -262,7 +235,7 @@ kafka_topic_old old kafka_check_result(result, True) - members = describe_consumer_group('old') + members = describe_consumer_group(kafka_cluster, 'old') assert members[0]['client_id'] == 'ClickHouse-instance-test-kafka' # text_desc = kafka_cluster.exec_in_container(kafka_cluster.get_container_id('kafka1'),"kafka-consumer-groups --bootstrap-server localhost:9092 --describe --members --group old --verbose")) @@ -302,7 +275,7 @@ def test_kafka_settings_new_syntax(kafka_cluster): kafka_check_result(result, True) - members = describe_consumer_group('new') + members = describe_consumer_group(kafka_cluster, 'new') assert members[0]['client_id'] == 'instance test 1234' @@ -734,82 +707,6 @@ def kafka_setup_teardown(): # Tests - -def test_kafka_settings_old_syntax(kafka_cluster): - assert TSV(instance.query("SELECT * FROM system.macros WHERE macro like 'kafka%' ORDER BY macro", - ignore_error=True)) == TSV('''kafka_broker kafka1 -kafka_client_id instance -kafka_format_json_each_row JSONEachRow -kafka_group_name_new new -kafka_group_name_old old -kafka_topic_new new -kafka_topic_old old -''') - - instance.query(''' - CREATE TABLE test.kafka (key UInt64, value UInt64) - ENGINE = Kafka('{kafka_broker}:19092', '{kafka_topic_old}', '{kafka_group_name_old}', '{kafka_format_json_each_row}', '\\n'); - ''') - - # Don't insert malformed messages since old settings syntax - # doesn't support skipping of broken messages. - messages = [] - for i in range(50): - messages.append(json.dumps({'key': i, 'value': i})) - kafka_produce(kafka_cluster, 'old', messages) - - result = '' - while True: - result += instance.query('SELECT * FROM test.kafka', ignore_error=True) - if kafka_check_result(result): - break - - kafka_check_result(result, True) - - members = describe_consumer_group(kafka_cluster, 'old') - assert members[0]['client_id'] == 'ClickHouse-instance-test-kafka' - # text_desc = kafka_cluster.exec_in_container(kafka_cluster.get_container_id('kafka1'),"kafka-consumer-groups --bootstrap-server localhost:{} --describe --members --group old --verbose".format(cluster.kafka_port))) - - -def test_kafka_settings_new_syntax(kafka_cluster): - instance.query(''' - CREATE TABLE test.kafka (key UInt64, value UInt64) - ENGINE = Kafka - SETTINGS kafka_broker_list = '{kafka_broker}:19092', - kafka_topic_list = '{kafka_topic_new}', - kafka_group_name = '{kafka_group_name_new}', - kafka_format = '{kafka_format_json_each_row}', - kafka_row_delimiter = '\\n', - kafka_client_id = '{kafka_client_id} test 1234', - kafka_skip_broken_messages = 1; - ''') - - messages = [] - for i in range(25): - messages.append(json.dumps({'key': i, 'value': i})) - kafka_produce(kafka_cluster, 'new', messages) - - # Insert couple of malformed messages. - kafka_produce(kafka_cluster, 'new', ['}{very_broken_message,']) - kafka_produce(kafka_cluster, 'new', ['}another{very_broken_message,']) - - messages = [] - for i in range(25, 50): - messages.append(json.dumps({'key': i, 'value': i})) - kafka_produce(kafka_cluster, 'new', messages) - - result = '' - while True: - result += instance.query('SELECT * FROM test.kafka', ignore_error=True) - if kafka_check_result(result): - break - - kafka_check_result(result, True) - - members = describe_consumer_group(kafka_cluster, 'new') - assert members[0]['client_id'] == 'instance test 1234' - - def test_kafka_issue11308(kafka_cluster): # Check that matview does respect Kafka SETTINGS kafka_produce(kafka_cluster, 'issue11308', ['{"t": 123, "e": {"x": "woof"} }', '{"t": 123, "e": {"x": "woof"} }', From b4e417e507e9d4b77eba1cc13691536645767457 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 6 Aug 2021 13:39:33 +0200 Subject: [PATCH 204/599] test_kafka_virtual_columns_with_materialized_view: Limit iterations and show better error --- tests/integration/test_storage_kafka/test.py | 15 +++++++++------ 1 file changed, 9 insertions(+), 6 deletions(-) diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index 3c91f26fde8..947b71b5f96 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -1482,18 +1482,21 @@ def test_kafka_virtual_columns_with_materialized_view(kafka_cluster): messages.append(json.dumps({'key': i, 'value': i})) kafka_produce(kafka_cluster, 'virt2', messages, 0) - while True: - result = instance.query('SELECT kafka_key, key, topic, value, offset, partition, timestamp FROM test.view') - if kafka_check_result(result, False, 'test_kafka_virtual2.reference'): - break + sql = 'SELECT kafka_key, key, topic, value, offset, partition, timestamp FROM test.view ORDER BY kafka_key' + result = instance.query(sql) + iterations = 0 + while not kafka_check_result(result, False, 'test_kafka_virtual2.reference') and iterations < 10: + time.sleep(3) + iterations += 1 + result = instance.query(sql) + + kafka_check_result(result, True, 'test_kafka_virtual2.reference') instance.query(''' DROP TABLE test.consumer; DROP TABLE test.view; ''') - kafka_check_result(result, True, 'test_kafka_virtual2.reference') - def test_kafka_insert(kafka_cluster): instance.query(''' From d53ade572e557b086cb12bb74a61fe376f272350 Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 6 Aug 2021 14:54:04 +0300 Subject: [PATCH 205/599] Fix non joined stream structure --- src/Interpreters/HashJoin.cpp | 8 +- src/Interpreters/MergeJoin.cpp | 3 +- src/Interpreters/join_common.cpp | 125 +++++++++--------- src/Interpreters/join_common.h | 9 +- .../01049_join_low_card_bug_long.reference | 3 + .../01049_join_low_card_bug_long.sql | 5 + 6 files changed, 83 insertions(+), 70 deletions(-) diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index dd17fc1004c..e0aad706966 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -1516,12 +1516,16 @@ private: if (!rows_added) return {}; - correctLowcardAndNullability(columns_right); - Block res = result_sample_block.cloneEmpty(); addLeftColumns(res, rows_added); addRightColumns(res, columns_right); copySameKeys(res); + correctLowcardAndNullability(res); + +#ifndef NDEBUG + assertBlocksHaveEqualStructure(res, result_sample_block, getName()); +#endif + return res; } diff --git a/src/Interpreters/MergeJoin.cpp b/src/Interpreters/MergeJoin.cpp index b93a94b4215..a2c63a4693b 100644 --- a/src/Interpreters/MergeJoin.cpp +++ b/src/Interpreters/MergeJoin.cpp @@ -1075,12 +1075,11 @@ private: if (!rows_added) return {}; - correctLowcardAndNullability(columns_right); - Block res = result_sample_block.cloneEmpty(); addLeftColumns(res, rows_added); addRightColumns(res, columns_right); copySameKeys(res); + correctLowcardAndNullability(res); return res; } diff --git a/src/Interpreters/join_common.cpp b/src/Interpreters/join_common.cpp index 06c51cc3d53..2ae7cbba066 100644 --- a/src/Interpreters/join_common.cpp +++ b/src/Interpreters/join_common.cpp @@ -29,17 +29,6 @@ namespace ErrorCodes namespace { -void changeNullability(MutableColumnPtr & mutable_column) -{ - ColumnPtr column = std::move(mutable_column); - if (const auto * nullable = checkAndGetColumn(*column)) - column = nullable->getNestedColumnPtr(); - else - column = makeNullable(column); - - mutable_column = IColumn::mutate(std::move(column)); -} - ColumnPtr changeLowCardinality(const ColumnPtr & column, const ColumnPtr & dst_sample) { if (dst_sample->lowCardinality()) @@ -52,6 +41,24 @@ ColumnPtr changeLowCardinality(const ColumnPtr & column, const ColumnPtr & dst_s return column->convertToFullColumnIfLowCardinality(); } +struct LowcardAndNull +{ + bool is_lowcard; + bool is_nullable; +}; + +LowcardAndNull getLowcardAndNullability(const ColumnPtr & col) +{ + if (col->lowCardinality()) + { + /// Currently only `LowCardinality(Nullable(T))` is possible, but not `Nullable(LowCardinality(T))` + assert(!col->canBeInsideNullable()); + const auto * col_as_lc = assert_cast(col.get()); + return {true, col_as_lc->nestedIsNullable()}; + } + return {false, col->isNullable()}; +} + } namespace JoinCommon @@ -91,20 +98,15 @@ DataTypePtr convertTypeToNullable(const DataTypePtr & type) if (dict_type->canBeInsideNullable()) return std::make_shared(makeNullable(dict_type)); } - return makeNullable(type); + + if (type->canBeInsideNullable()) + return makeNullable(type); + + return type; } -void convertColumnToNullable(ColumnWithTypeAndName & column, bool remove_low_card) +void convertColumnToNullable(ColumnWithTypeAndName & column) { - if (remove_low_card && column.type->lowCardinality()) - { - column.column = recursiveRemoveLowCardinality(column.column); - column.type = recursiveRemoveLowCardinality(column.type); - } - - if (column.type->isNullable() || !canBecomeNullable(column.type)) - return; - column.type = convertTypeToNullable(column.type); if (column.column) @@ -146,20 +148,18 @@ void removeColumnNullability(ColumnWithTypeAndName & column) col_as_lc->nestedRemoveNullable(); column.column = std::move(mut_col); } - - return; } - - if (!column.type->isNullable()) - return; - - column.type = static_cast(*column.type).getNestedType(); - if (column.column) + else { - const auto * nullable_column = checkAndGetColumn(*column.column); - ColumnPtr nested_column = nullable_column->getNestedColumnPtr(); - MutableColumnPtr mutable_column = IColumn::mutate(std::move(nested_column)); - column.column = std::move(mutable_column); + column.type = removeNullable(column.type); + + if (column.column && column.column->isNullable()) + { + const auto * nullable_column = checkAndGetColumn(*column.column); + ColumnPtr nested_column = nullable_column->getNestedColumnPtr(); + MutableColumnPtr mutable_column = IColumn::mutate(std::move(nested_column)); + column.column = std::move(mutable_column); + } } } @@ -534,34 +534,42 @@ void NotJoined::setRightIndex(size_t right_pos, size_t result_position) void NotJoined::extractColumnChanges(size_t right_pos, size_t result_pos) { - const auto & src = saved_block_sample.getByPosition(right_pos).column; - const auto & dst = result_sample_block.getByPosition(result_pos).column; + auto src_props = getLowcardAndNullability(saved_block_sample.getByPosition(right_pos).column); + auto dst_props = getLowcardAndNullability(result_sample_block.getByPosition(result_pos).column); - if (!src->isNullable() && dst->isNullable()) - right_nullability_adds.push_back(right_pos); + if (src_props.is_nullable != dst_props.is_nullable) + right_nullability_changes.push_back({result_pos, dst_props.is_nullable}); - if (src->isNullable() && !dst->isNullable()) - right_nullability_removes.push_back(right_pos); - - ColumnPtr src_not_null = JoinCommon::emptyNotNullableClone(src); - ColumnPtr dst_not_null = JoinCommon::emptyNotNullableClone(dst); - - if (src_not_null->lowCardinality() != dst_not_null->lowCardinality()) - right_lowcard_changes.push_back({right_pos, dst_not_null}); + if (src_props.is_lowcard != dst_props.is_lowcard) + right_lowcard_changes.push_back({result_pos, dst_props.is_lowcard}); } -void NotJoined::correctLowcardAndNullability(MutableColumns & columns_right) +void NotJoined::correctLowcardAndNullability(Block & block) { + for (auto & [pos, added] : right_nullability_changes) + { + auto & col = block.getByPosition(pos); + if (added) + JoinCommon::convertColumnToNullable(col); + else + JoinCommon::removeColumnNullability(col); + } - assert(right_nullability_removes.size() == 0); - for (size_t pos : right_nullability_removes) - changeNullability(columns_right[pos]); - - for (auto & [pos, dst_sample] : right_lowcard_changes) - columns_right[pos] = changeLowCardinality(std::move(columns_right[pos]), dst_sample)->assumeMutable(); - - for (size_t pos : right_nullability_adds) - changeNullability(columns_right[pos]); + for (auto & [pos, added] : right_lowcard_changes) + { + auto & col = block.getByPosition(pos); + if (added) + { + if (!col.type->lowCardinality()) + col.type = std::make_shared(col.type); + col.column = changeLowCardinality(col.column, col.type->createColumn()); + } + else + { + col.column = recursiveRemoveLowCardinality(col.column); + col.type = recursiveRemoveLowCardinality(col.type); + } + } } void NotJoined::addLeftColumns(Block & block, size_t rows_added) const @@ -582,11 +590,6 @@ void NotJoined::addRightColumns(Block & block, MutableColumns & columns_right) c { auto & right_column = columns_right[pr.first]; auto & result_column = block.getByPosition(pr.second).column; -#ifndef NDEBUG - if (result_column->getName() != right_column->getName()) - throw Exception("Wrong columns assign in RIGHT|FULL JOIN: " + result_column->getName() + - " " + right_column->getName(), ErrorCodes::LOGICAL_ERROR); -#endif result_column = std::move(right_column); } } diff --git a/src/Interpreters/join_common.h b/src/Interpreters/join_common.h index 8862116d1fa..1f037899155 100644 --- a/src/Interpreters/join_common.h +++ b/src/Interpreters/join_common.h @@ -19,7 +19,7 @@ namespace JoinCommon { bool canBecomeNullable(const DataTypePtr & type); DataTypePtr convertTypeToNullable(const DataTypePtr & type); -void convertColumnToNullable(ColumnWithTypeAndName & column, bool remove_low_card = false); +void convertColumnToNullable(ColumnWithTypeAndName & column); void convertColumnsToNullable(Block & block, size_t starting_pos = 0); void removeColumnNullability(ColumnWithTypeAndName & column); void changeColumnRepresentation(const ColumnPtr & src_column, ColumnPtr & dst_column); @@ -70,7 +70,7 @@ public: NotJoined(const TableJoin & table_join, const Block & saved_block_sample_, const Block & right_sample_block, const Block & result_sample_block_, const Names & key_names_left_ = {}, const Names & key_names_right_ = {}); - void correctLowcardAndNullability(MutableColumns & columns_right); + void correctLowcardAndNullability(Block & block); void addLeftColumns(Block & block, size_t rows_added) const; void addRightColumns(Block & block, MutableColumns & columns_right) const; void copySameKeys(Block & block) const; @@ -92,10 +92,9 @@ private: /// std::unordered_map same_result_keys; /// Which right columns (saved in parent) need nullability change before placing them in result block - std::vector right_nullability_adds; - std::vector right_nullability_removes; + std::vector> right_nullability_changes; /// Which right columns (saved in parent) need LowCardinality change before placing them in result block - std::vector> right_lowcard_changes; + std::vector> right_lowcard_changes; void setRightIndex(size_t right_pos, size_t result_position); void extractColumnChanges(size_t right_pos, size_t result_pos); diff --git a/tests/queries/0_stateless/01049_join_low_card_bug_long.reference b/tests/queries/0_stateless/01049_join_low_card_bug_long.reference index 877b4370fce..52554622d95 100644 --- a/tests/queries/0_stateless/01049_join_low_card_bug_long.reference +++ b/tests/queries/0_stateless/01049_join_low_card_bug_long.reference @@ -405,3 +405,6 @@ str LowCardinality(String) str +str str LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) +str_r str_r LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) +\N \N LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) diff --git a/tests/queries/0_stateless/01049_join_low_card_bug_long.sql b/tests/queries/0_stateless/01049_join_low_card_bug_long.sql index e1732824428..1f3c48cb7ad 100644 --- a/tests/queries/0_stateless/01049_join_low_card_bug_long.sql +++ b/tests/queries/0_stateless/01049_join_low_card_bug_long.sql @@ -276,6 +276,11 @@ SELECT lc, toTypeName(lc) FROM l_lc AS l RIGHT JOIN r_lc AS r USING (x) ORDER B SELECT lowCardinalityKeys(lc.lc) FROM r FULL JOIN l_lc as lc USING (lc) ORDER BY lowCardinalityKeys(lc.lc); +SET join_algorithm = 'partial_merge'; +SET join_use_nulls = 1; + +SELECT r.lc, materialize(r.lc), toTypeName(r.lc), toTypeName(materialize(r.lc)) FROM l_lc AS l FULL OUTER JOIN r_lc AS r USING (x) ORDER BY r.lc; + DROP TABLE l; DROP TABLE r; DROP TABLE nl; From 3ca05653bbc797e9d08b71a7279a4e5e8f456447 Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 6 Aug 2021 15:00:52 +0300 Subject: [PATCH 206/599] Use template for 01049_join_low_card_bug_long.sql --- .../01049_join_low_card_bug_long.reference | 7 +- .../01049_join_low_card_bug_long.sql | 289 ------------------ .../01049_join_low_card_bug_long.sql.j2 | 109 +++++++ 3 files changed, 114 insertions(+), 291 deletions(-) delete mode 100644 tests/queries/0_stateless/01049_join_low_card_bug_long.sql create mode 100644 tests/queries/0_stateless/01049_join_low_card_bug_long.sql.j2 diff --git a/tests/queries/0_stateless/01049_join_low_card_bug_long.reference b/tests/queries/0_stateless/01049_join_low_card_bug_long.reference index 52554622d95..6d9f36df075 100644 --- a/tests/queries/0_stateless/01049_join_low_card_bug_long.reference +++ b/tests/queries/0_stateless/01049_join_low_card_bug_long.reference @@ -1,3 +1,4 @@ +-- join_algorithm = default, join_use_nulls = 0 -- str LowCardinality(String) LowCardinality(String) str_r LowCardinality(String) @@ -98,7 +99,7 @@ LowCardinality(String) LowCardinality(String) Nullable(String) Nullable(String LowCardinality(String) LowCardinality(String) str str Nullable(String) Nullable(String) str str LowCardinality(String) LowCardinality(String) Nullable(String) Nullable(String) str_l str_l LowCardinality(String) LowCardinality(String) str_r str_r Nullable(String) Nullable(String) \N \N --- join_use_nulls -- +-- join_algorithm = default, join_use_nulls = 1 -- str LowCardinality(Nullable(String)) \N LowCardinality(Nullable(String)) str LowCardinality(Nullable(String)) @@ -199,6 +200,7 @@ LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) \N \N Nullable LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str str Nullable(String) Nullable(String) str str LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) \N \N Nullable(String) Nullable(String) str_l str_l LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str_r str_r Nullable(String) Nullable(String) \N \N +-- join_algorithm = partial_merge, join_use_nulls = 0 -- str LowCardinality(String) LowCardinality(String) str_r LowCardinality(String) @@ -299,7 +301,7 @@ LowCardinality(String) LowCardinality(String) Nullable(String) Nullable(String LowCardinality(String) LowCardinality(String) str str Nullable(String) Nullable(String) str str LowCardinality(String) LowCardinality(String) Nullable(String) Nullable(String) str_l str_l LowCardinality(String) LowCardinality(String) str_r str_r Nullable(String) Nullable(String) \N \N --- join_use_nulls -- +-- join_algorithm = partial_merge, join_use_nulls = 1 -- str LowCardinality(Nullable(String)) \N LowCardinality(Nullable(String)) str LowCardinality(Nullable(String)) @@ -400,6 +402,7 @@ LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) \N \N Nullable LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str str Nullable(String) Nullable(String) str str LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) \N \N Nullable(String) Nullable(String) str_l str_l LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str_r str_r Nullable(String) Nullable(String) \N \N +-- LowCardinality(String) str LowCardinality(String) diff --git a/tests/queries/0_stateless/01049_join_low_card_bug_long.sql b/tests/queries/0_stateless/01049_join_low_card_bug_long.sql deleted file mode 100644 index 1f3c48cb7ad..00000000000 --- a/tests/queries/0_stateless/01049_join_low_card_bug_long.sql +++ /dev/null @@ -1,289 +0,0 @@ -DROP TABLE IF EXISTS l; -DROP TABLE IF EXISTS r; -DROP TABLE IF EXISTS nl; -DROP TABLE IF EXISTS nr; -DROP TABLE IF EXISTS l_lc; -DROP TABLE IF EXISTS r_lc; - -CREATE TABLE l (x UInt32, lc String) ENGINE = Memory; -CREATE TABLE r (x UInt32, lc String) ENGINE = Memory; -CREATE TABLE nl (x Nullable(UInt32), lc Nullable(String)) ENGINE = Memory; -CREATE TABLE nr (x Nullable(UInt32), lc Nullable(String)) ENGINE = Memory; -CREATE TABLE l_lc (x UInt32, lc LowCardinality(String)) ENGINE = Memory; -CREATE TABLE r_lc (x UInt32, lc LowCardinality(String)) ENGINE = Memory; - -INSERT INTO r VALUES (0, 'str'), (1, 'str_r'); -INSERT INTO nr VALUES (0, 'str'), (1, 'str_r'); -INSERT INTO r_lc VALUES (0, 'str'), (1, 'str_r'); - -INSERT INTO l VALUES (0, 'str'), (2, 'str_l'); -INSERT INTO nl VALUES (0, 'str'), (2, 'str_l'); -INSERT INTO l_lc VALUES (0, 'str'), (2, 'str_l'); - - -SET join_use_nulls = 0; - --- - -SELECT lc, toTypeName(lc) FROM l_lc AS l RIGHT JOIN r_lc AS r USING (x) ORDER BY x; -SELECT lc, toTypeName(lc) FROM l_lc AS l RIGHT JOIN r_lc AS r USING (lc) ORDER BY x; -SELECT lc, toTypeName(lc) FROM l_lc AS l FULL JOIN r_lc AS r USING (x) ORDER BY x; -SELECT lc, toTypeName(lc) FROM l_lc AS l FULL JOIN r_lc AS r USING (lc) ORDER BY x; - -SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l RIGHT JOIN r_lc AS r USING (x) ORDER BY x; -SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l RIGHT JOIN r_lc AS r USING (lc) ORDER BY x; -SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l FULL JOIN r_lc AS r USING (x) ORDER BY x; -SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l FULL JOIN r_lc AS r USING (lc) ORDER BY x; - --- - -SELECT lc, toTypeName(lc) FROM l_lc AS l RIGHT JOIN r USING (x) ORDER BY x; -SELECT lc, toTypeName(lc) FROM l_lc AS l RIGHT JOIN r USING (lc) ORDER BY x; -SELECT lc, toTypeName(lc) FROM l_lc AS l FULL JOIN r USING (x) ORDER BY x; -SELECT lc, toTypeName(lc) FROM l_lc AS l FULL JOIN r USING (lc) ORDER BY x; - -SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l RIGHT JOIN r USING (x) ORDER BY x; -SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l RIGHT JOIN r USING (lc) ORDER BY x; -SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l FULL JOIN r USING (x) ORDER BY x; -SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l FULL JOIN r USING (lc) ORDER BY x; - --- - -SELECT lc, toTypeName(lc) FROM l RIGHT JOIN r USING (x) ORDER BY x; -SELECT lc, toTypeName(lc) FROM l RIGHT JOIN r USING (lc) ORDER BY x; -SELECT lc, toTypeName(lc) FROM l FULL JOIN r USING (x) ORDER BY x; -SELECT lc, toTypeName(lc) FROM l FULL JOIN r USING (lc) ORDER BY x; - -SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l RIGHT JOIN r_lc AS r USING (x) ORDER BY x; -SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l RIGHT JOIN r_lc AS r USING (lc) ORDER BY x; -SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l FULL JOIN r_lc AS r USING (x) ORDER BY x; -SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l FULL JOIN r_lc AS r USING (lc) ORDER BY x; - --- - -SELECT lc, toTypeName(lc) FROM l_lc RIGHT JOIN nr USING (x) ORDER BY x; -SELECT lc, toTypeName(lc) FROM l_lc RIGHT JOIN nr USING (lc) ORDER BY x; -SELECT lc, toTypeName(lc) FROM l_lc FULL JOIN nr USING (x) ORDER BY x; -SELECT lc, toTypeName(lc) FROM l_lc FULL JOIN nr USING (lc) ORDER BY x; - -SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l RIGHT JOIN nr AS r USING (x) ORDER BY x; -SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l RIGHT JOIN nr AS r USING (lc) ORDER BY x; -SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l FULL JOIN nr AS r USING (x) ORDER BY x; -SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l FULL JOIN nr AS r USING (lc) ORDER BY x; - --- - -SELECT lc, toTypeName(lc) FROM nl RIGHT JOIN r_lc USING (x) ORDER BY x; -SELECT lc, toTypeName(lc) FROM nl RIGHT JOIN r_lc USING (lc) ORDER BY x; -SELECT lc, toTypeName(lc) FROM nl FULL JOIN r_lc USING (x) ORDER BY x; -SELECT lc, toTypeName(lc) FROM nl FULL JOIN r_lc USING (lc) ORDER BY x; - -SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM nl AS l RIGHT JOIN r_lc AS r USING (x) ORDER BY x; -SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM nl AS l RIGHT JOIN r_lc AS r USING (lc) ORDER BY x; -SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM nl AS l FULL JOIN r_lc AS r USING (x) ORDER BY x; -SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM nl AS l FULL JOIN r_lc AS r USING (lc) ORDER BY x; - -SELECT '-- join_use_nulls --'; - -SET join_use_nulls = 1; - -SELECT lc, toTypeName(lc) FROM l_lc AS l RIGHT JOIN r_lc AS r USING (x) ORDER BY x; -SELECT lc, toTypeName(lc) FROM l_lc AS l RIGHT JOIN r_lc AS r USING (lc) ORDER BY x; -SELECT lc, toTypeName(lc) FROM l_lc AS l FULL JOIN r_lc AS r USING (x) ORDER BY x; -SELECT lc, toTypeName(lc) FROM l_lc AS l FULL JOIN r_lc AS r USING (lc) ORDER BY x; - -SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l RIGHT JOIN r_lc AS r USING (x) ORDER BY x; -SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l RIGHT JOIN r_lc AS r USING (lc) ORDER BY x; -SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l FULL JOIN r_lc AS r USING (x) ORDER BY x; -SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l FULL JOIN r_lc AS r USING (lc) ORDER BY x; - --- - -SELECT lc, toTypeName(lc) FROM l_lc AS l RIGHT JOIN r USING (x) ORDER BY x; -SELECT lc, toTypeName(lc) FROM l_lc AS l RIGHT JOIN r USING (lc) ORDER BY x; -SELECT lc, toTypeName(lc) FROM l_lc AS l FULL JOIN r USING (x) ORDER BY x; -SELECT lc, toTypeName(lc) FROM l_lc AS l FULL JOIN r USING (lc) ORDER BY x; - -SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l RIGHT JOIN r USING (x) ORDER BY x; -SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l RIGHT JOIN r USING (lc) ORDER BY x; -SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l FULL JOIN r USING (x) ORDER BY x; -SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l FULL JOIN r USING (lc) ORDER BY x; - --- - -SELECT lc, toTypeName(lc) FROM l RIGHT JOIN r USING (x) ORDER BY x; -SELECT lc, toTypeName(lc) FROM l RIGHT JOIN r USING (lc) ORDER BY x; -SELECT lc, toTypeName(lc) FROM l FULL JOIN r USING (x) ORDER BY x; -SELECT lc, toTypeName(lc) FROM l FULL JOIN r USING (lc) ORDER BY x; - -SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l RIGHT JOIN r_lc AS r USING (x) ORDER BY x; -SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l RIGHT JOIN r_lc AS r USING (lc) ORDER BY x; -SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l FULL JOIN r_lc AS r USING (x) ORDER BY x; -SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l FULL JOIN r_lc AS r USING (lc) ORDER BY x; - --- - -SELECT lc, toTypeName(lc) FROM l_lc RIGHT JOIN nr USING (x) ORDER BY x; -SELECT lc, toTypeName(lc) FROM l_lc RIGHT JOIN nr USING (lc) ORDER BY x; -SELECT lc, toTypeName(lc) FROM l_lc FULL JOIN nr USING (x) ORDER BY x; -SELECT lc, toTypeName(lc) FROM l_lc FULL JOIN nr USING (lc) ORDER BY x; - -SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l RIGHT JOIN nr AS r USING (x) ORDER BY x; -SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l RIGHT JOIN nr AS r USING (lc) ORDER BY x; -SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l FULL JOIN nr AS r USING (x) ORDER BY x; -SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l FULL JOIN nr AS r USING (lc) ORDER BY x; - --- - -SELECT lc, toTypeName(lc) FROM nl RIGHT JOIN r_lc USING (x) ORDER BY x; -SELECT lc, toTypeName(lc) FROM nl RIGHT JOIN r_lc USING (lc) ORDER BY x; -SELECT lc, toTypeName(lc) FROM nl FULL JOIN r_lc USING (x) ORDER BY x; -SELECT lc, toTypeName(lc) FROM nl FULL JOIN r_lc USING (lc) ORDER BY x; - -SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM nl AS l RIGHT JOIN r_lc AS r USING (x) ORDER BY x; -SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM nl AS l RIGHT JOIN r_lc AS r USING (lc) ORDER BY x; -SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM nl AS l FULL JOIN r_lc AS r USING (x) ORDER BY x; -SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM nl AS l FULL JOIN r_lc AS r USING (lc) ORDER BY x; - -set join_algorithm = 'partial_merge'; - -SET join_use_nulls = 0; - --- - -SELECT lc, toTypeName(lc) FROM l_lc AS l RIGHT JOIN r_lc AS r USING (x) ORDER BY x; -SELECT lc, toTypeName(lc) FROM l_lc AS l RIGHT JOIN r_lc AS r USING (lc) ORDER BY x; -SELECT lc, toTypeName(lc) FROM l_lc AS l FULL JOIN r_lc AS r USING (x) ORDER BY x; -SELECT lc, toTypeName(lc) FROM l_lc AS l FULL JOIN r_lc AS r USING (lc) ORDER BY x; - -SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l RIGHT JOIN r_lc AS r USING (x) ORDER BY x; -SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l RIGHT JOIN r_lc AS r USING (lc) ORDER BY x; -SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l FULL JOIN r_lc AS r USING (x) ORDER BY x; -SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l FULL JOIN r_lc AS r USING (lc) ORDER BY x; - --- - -SELECT lc, toTypeName(lc) FROM l_lc AS l RIGHT JOIN r USING (x) ORDER BY x; -SELECT lc, toTypeName(lc) FROM l_lc AS l RIGHT JOIN r USING (lc) ORDER BY x; -SELECT lc, toTypeName(lc) FROM l_lc AS l FULL JOIN r USING (x) ORDER BY x; -SELECT lc, toTypeName(lc) FROM l_lc AS l FULL JOIN r USING (lc) ORDER BY x; - -SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l RIGHT JOIN r USING (x) ORDER BY x; -SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l RIGHT JOIN r USING (lc) ORDER BY x; -SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l FULL JOIN r USING (x) ORDER BY x; -SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l FULL JOIN r USING (lc) ORDER BY x; - --- - -SELECT lc, toTypeName(lc) FROM l RIGHT JOIN r USING (x) ORDER BY x; -SELECT lc, toTypeName(lc) FROM l RIGHT JOIN r USING (lc) ORDER BY x; -SELECT lc, toTypeName(lc) FROM l FULL JOIN r USING (x) ORDER BY x; -SELECT lc, toTypeName(lc) FROM l FULL JOIN r USING (lc) ORDER BY x; - -SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l RIGHT JOIN r_lc AS r USING (x) ORDER BY x; -SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l RIGHT JOIN r_lc AS r USING (lc) ORDER BY x; -SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l FULL JOIN r_lc AS r USING (x) ORDER BY x; -SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l FULL JOIN r_lc AS r USING (lc) ORDER BY x; - --- - -SELECT lc, toTypeName(lc) FROM l_lc RIGHT JOIN nr USING (x) ORDER BY x; -SELECT lc, toTypeName(lc) FROM l_lc RIGHT JOIN nr USING (lc) ORDER BY x; -SELECT lc, toTypeName(lc) FROM l_lc FULL JOIN nr USING (x) ORDER BY x; -SELECT lc, toTypeName(lc) FROM l_lc FULL JOIN nr USING (lc) ORDER BY x; - -SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l RIGHT JOIN nr AS r USING (x) ORDER BY x; -SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l RIGHT JOIN nr AS r USING (lc) ORDER BY x; -SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l FULL JOIN nr AS r USING (x) ORDER BY x; -SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l FULL JOIN nr AS r USING (lc) ORDER BY x; - --- - -SELECT lc, toTypeName(lc) FROM nl RIGHT JOIN r_lc USING (x) ORDER BY x; -SELECT lc, toTypeName(lc) FROM nl RIGHT JOIN r_lc USING (lc) ORDER BY x; -SELECT lc, toTypeName(lc) FROM nl FULL JOIN r_lc USING (x) ORDER BY x; -SELECT lc, toTypeName(lc) FROM nl FULL JOIN r_lc USING (lc) ORDER BY x; - -SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM nl AS l RIGHT JOIN r_lc AS r USING (x) ORDER BY x; -SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM nl AS l RIGHT JOIN r_lc AS r USING (lc) ORDER BY x; -SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM nl AS l FULL JOIN r_lc AS r USING (x) ORDER BY x; -SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM nl AS l FULL JOIN r_lc AS r USING (lc) ORDER BY x; - -SELECT '-- join_use_nulls --'; - -SET join_use_nulls = 1; - -SELECT lc, toTypeName(lc) FROM l_lc AS l RIGHT JOIN r_lc AS r USING (x) ORDER BY x; -SELECT lc, toTypeName(lc) FROM l_lc AS l RIGHT JOIN r_lc AS r USING (lc) ORDER BY x; -SELECT lc, toTypeName(lc) FROM l_lc AS l FULL JOIN r_lc AS r USING (x) ORDER BY x; -SELECT lc, toTypeName(lc) FROM l_lc AS l FULL JOIN r_lc AS r USING (lc) ORDER BY x; - -SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l RIGHT JOIN r_lc AS r USING (x) ORDER BY x; -SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l RIGHT JOIN r_lc AS r USING (lc) ORDER BY x; -SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l FULL JOIN r_lc AS r USING (x) ORDER BY x; -SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l FULL JOIN r_lc AS r USING (lc) ORDER BY x; - --- - -SELECT lc, toTypeName(lc) FROM l_lc AS l RIGHT JOIN r USING (x) ORDER BY x; -SELECT lc, toTypeName(lc) FROM l_lc AS l RIGHT JOIN r USING (lc) ORDER BY x; -SELECT lc, toTypeName(lc) FROM l_lc AS l FULL JOIN r USING (x) ORDER BY x; -SELECT lc, toTypeName(lc) FROM l_lc AS l FULL JOIN r USING (lc) ORDER BY x; - -SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l RIGHT JOIN r USING (x) ORDER BY x; -SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l RIGHT JOIN r USING (lc) ORDER BY x; -SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l FULL JOIN r USING (x) ORDER BY x; -SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l FULL JOIN r USING (lc) ORDER BY x; - --- - -SELECT lc, toTypeName(lc) FROM l RIGHT JOIN r USING (x) ORDER BY x; -SELECT lc, toTypeName(lc) FROM l RIGHT JOIN r USING (lc) ORDER BY x; -SELECT lc, toTypeName(lc) FROM l FULL JOIN r USING (x) ORDER BY x; -SELECT lc, toTypeName(lc) FROM l FULL JOIN r USING (lc) ORDER BY x; - -SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l RIGHT JOIN r_lc AS r USING (x) ORDER BY x; -SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l RIGHT JOIN r_lc AS r USING (lc) ORDER BY x; -SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l FULL JOIN r_lc AS r USING (x) ORDER BY x; -SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l FULL JOIN r_lc AS r USING (lc) ORDER BY x; - --- - -SELECT lc, toTypeName(lc) FROM l_lc RIGHT JOIN nr USING (x) ORDER BY x; -SELECT lc, toTypeName(lc) FROM l_lc RIGHT JOIN nr USING (lc) ORDER BY x; -SELECT lc, toTypeName(lc) FROM l_lc FULL JOIN nr USING (x) ORDER BY x; -SELECT lc, toTypeName(lc) FROM l_lc FULL JOIN nr USING (lc) ORDER BY x; - -SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l RIGHT JOIN nr AS r USING (x) ORDER BY x; -SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l RIGHT JOIN nr AS r USING (lc) ORDER BY x; -SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l FULL JOIN nr AS r USING (x) ORDER BY x; -SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l FULL JOIN nr AS r USING (lc) ORDER BY x; - --- - -SELECT lc, toTypeName(lc) FROM nl RIGHT JOIN r_lc USING (x) ORDER BY x; -SELECT lc, toTypeName(lc) FROM nl RIGHT JOIN r_lc USING (lc) ORDER BY x; -SELECT lc, toTypeName(lc) FROM nl FULL JOIN r_lc USING (x) ORDER BY x; -SELECT lc, toTypeName(lc) FROM nl FULL JOIN r_lc USING (lc) ORDER BY x; - -SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM nl AS l RIGHT JOIN r_lc AS r USING (x) ORDER BY x; -SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM nl AS l RIGHT JOIN r_lc AS r USING (lc) ORDER BY x; -SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM nl AS l FULL JOIN r_lc AS r USING (x) ORDER BY x; -SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM nl AS l FULL JOIN r_lc AS r USING (lc) ORDER BY x; - -SET join_use_nulls = 0; -SELECT lc, toTypeName(lc) FROM l_lc AS l RIGHT JOIN r_lc AS r USING (x) ORDER BY l.lc; - -SELECT lowCardinalityKeys(lc.lc) FROM r FULL JOIN l_lc as lc USING (lc) ORDER BY lowCardinalityKeys(lc.lc); - -SET join_algorithm = 'partial_merge'; -SET join_use_nulls = 1; - -SELECT r.lc, materialize(r.lc), toTypeName(r.lc), toTypeName(materialize(r.lc)) FROM l_lc AS l FULL OUTER JOIN r_lc AS r USING (x) ORDER BY r.lc; - -DROP TABLE l; -DROP TABLE r; -DROP TABLE nl; -DROP TABLE nr; -DROP TABLE l_lc; -DROP TABLE r_lc; diff --git a/tests/queries/0_stateless/01049_join_low_card_bug_long.sql.j2 b/tests/queries/0_stateless/01049_join_low_card_bug_long.sql.j2 new file mode 100644 index 00000000000..93019e7a0ec --- /dev/null +++ b/tests/queries/0_stateless/01049_join_low_card_bug_long.sql.j2 @@ -0,0 +1,109 @@ +DROP TABLE IF EXISTS l; +DROP TABLE IF EXISTS r; +DROP TABLE IF EXISTS nl; +DROP TABLE IF EXISTS nr; +DROP TABLE IF EXISTS l_lc; +DROP TABLE IF EXISTS r_lc; + +CREATE TABLE l (x UInt32, lc String) ENGINE = Memory; +CREATE TABLE r (x UInt32, lc String) ENGINE = Memory; +CREATE TABLE nl (x Nullable(UInt32), lc Nullable(String)) ENGINE = Memory; +CREATE TABLE nr (x Nullable(UInt32), lc Nullable(String)) ENGINE = Memory; +CREATE TABLE l_lc (x UInt32, lc LowCardinality(String)) ENGINE = Memory; +CREATE TABLE r_lc (x UInt32, lc LowCardinality(String)) ENGINE = Memory; + +INSERT INTO r VALUES (0, 'str'), (1, 'str_r'); +INSERT INTO nr VALUES (0, 'str'), (1, 'str_r'); +INSERT INTO r_lc VALUES (0, 'str'), (1, 'str_r'); + +INSERT INTO l VALUES (0, 'str'), (2, 'str_l'); +INSERT INTO nl VALUES (0, 'str'), (2, 'str_l'); +INSERT INTO l_lc VALUES (0, 'str'), (2, 'str_l'); + +{% for join_algorithm in [None, 'partial_merge'] -%} +{% for join_use_nulls in [0, 1] -%} + +SELECT '-- join_algorithm = {{ join_algorithm or 'default' }}, join_use_nulls = {{ join_use_nulls }} --'; + +{% if join_algorithm %}SET join_algorithm = '{{ join_algorithm }}';{% endif -%} +SET join_use_nulls = {{ join_use_nulls }}; + +SELECT lc, toTypeName(lc) FROM l_lc AS l RIGHT JOIN r_lc AS r USING (x) ORDER BY x; +SELECT lc, toTypeName(lc) FROM l_lc AS l RIGHT JOIN r_lc AS r USING (lc) ORDER BY x; +SELECT lc, toTypeName(lc) FROM l_lc AS l FULL JOIN r_lc AS r USING (x) ORDER BY x; +SELECT lc, toTypeName(lc) FROM l_lc AS l FULL JOIN r_lc AS r USING (lc) ORDER BY x; + +SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l RIGHT JOIN r_lc AS r USING (x) ORDER BY x; +SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l RIGHT JOIN r_lc AS r USING (lc) ORDER BY x; +SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l FULL JOIN r_lc AS r USING (x) ORDER BY x; +SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l FULL JOIN r_lc AS r USING (lc) ORDER BY x; + +-- + +SELECT lc, toTypeName(lc) FROM l_lc AS l RIGHT JOIN r USING (x) ORDER BY x; +SELECT lc, toTypeName(lc) FROM l_lc AS l RIGHT JOIN r USING (lc) ORDER BY x; +SELECT lc, toTypeName(lc) FROM l_lc AS l FULL JOIN r USING (x) ORDER BY x; +SELECT lc, toTypeName(lc) FROM l_lc AS l FULL JOIN r USING (lc) ORDER BY x; + +SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l RIGHT JOIN r USING (x) ORDER BY x; +SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l RIGHT JOIN r USING (lc) ORDER BY x; +SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l FULL JOIN r USING (x) ORDER BY x; +SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l FULL JOIN r USING (lc) ORDER BY x; + +-- + +SELECT lc, toTypeName(lc) FROM l RIGHT JOIN r USING (x) ORDER BY x; +SELECT lc, toTypeName(lc) FROM l RIGHT JOIN r USING (lc) ORDER BY x; +SELECT lc, toTypeName(lc) FROM l FULL JOIN r USING (x) ORDER BY x; +SELECT lc, toTypeName(lc) FROM l FULL JOIN r USING (lc) ORDER BY x; + +SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l RIGHT JOIN r_lc AS r USING (x) ORDER BY x; +SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l RIGHT JOIN r_lc AS r USING (lc) ORDER BY x; +SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l FULL JOIN r_lc AS r USING (x) ORDER BY x; +SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l FULL JOIN r_lc AS r USING (lc) ORDER BY x; + +-- + +SELECT lc, toTypeName(lc) FROM l_lc RIGHT JOIN nr USING (x) ORDER BY x; +SELECT lc, toTypeName(lc) FROM l_lc RIGHT JOIN nr USING (lc) ORDER BY x; +SELECT lc, toTypeName(lc) FROM l_lc FULL JOIN nr USING (x) ORDER BY x; +SELECT lc, toTypeName(lc) FROM l_lc FULL JOIN nr USING (lc) ORDER BY x; + +SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l RIGHT JOIN nr AS r USING (x) ORDER BY x; +SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l RIGHT JOIN nr AS r USING (lc) ORDER BY x; +SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l FULL JOIN nr AS r USING (x) ORDER BY x; +SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l FULL JOIN nr AS r USING (lc) ORDER BY x; + +-- + +SELECT lc, toTypeName(lc) FROM nl RIGHT JOIN r_lc USING (x) ORDER BY x; +SELECT lc, toTypeName(lc) FROM nl RIGHT JOIN r_lc USING (lc) ORDER BY x; +SELECT lc, toTypeName(lc) FROM nl FULL JOIN r_lc USING (x) ORDER BY x; +SELECT lc, toTypeName(lc) FROM nl FULL JOIN r_lc USING (lc) ORDER BY x; + +SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM nl AS l RIGHT JOIN r_lc AS r USING (x) ORDER BY x; +SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM nl AS l RIGHT JOIN r_lc AS r USING (lc) ORDER BY x; +SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM nl AS l FULL JOIN r_lc AS r USING (x) ORDER BY x; +SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM nl AS l FULL JOIN r_lc AS r USING (lc) ORDER BY x; + +{% endfor -%} +{% endfor -%} + +SELECT '--'; + +SET join_use_nulls = 0; +SELECT lc, toTypeName(lc) FROM l_lc AS l RIGHT JOIN r_lc AS r USING (x) ORDER BY l.lc; + +SELECT lowCardinalityKeys(lc.lc) FROM r FULL JOIN l_lc as lc USING (lc) ORDER BY lowCardinalityKeys(lc.lc); + +SET join_algorithm = 'partial_merge'; +SET join_use_nulls = 1; + +SELECT r.lc, materialize(r.lc), toTypeName(r.lc), toTypeName(materialize(r.lc)) FROM l_lc AS l FULL OUTER JOIN r_lc AS r USING (x) ORDER BY r.lc; + +DROP TABLE l; +DROP TABLE r; +DROP TABLE nl; +DROP TABLE nr; +DROP TABLE l_lc; +DROP TABLE r_lc; From d6f66ef30353c0dfbe18622940cb07099de985d9 Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 6 Aug 2021 16:13:30 +0300 Subject: [PATCH 207/599] Fix JoinCommon::convertColumnToNullable --- src/Interpreters/join_common.cpp | 26 ++++++++++++++------------ 1 file changed, 14 insertions(+), 12 deletions(-) diff --git a/src/Interpreters/join_common.cpp b/src/Interpreters/join_common.cpp index 2ae7cbba066..76bfd7f2899 100644 --- a/src/Interpreters/join_common.cpp +++ b/src/Interpreters/join_common.cpp @@ -109,19 +109,21 @@ void convertColumnToNullable(ColumnWithTypeAndName & column) { column.type = convertTypeToNullable(column.type); - if (column.column) + if (!column.column) + return; + + if (column.column->lowCardinality()) { - if (column.column->lowCardinality()) - { - /// Convert nested to nullable, not LowCardinality itself - auto mut_col = IColumn::mutate(std::move(column.column)); - ColumnLowCardinality * col_as_lc = assert_cast(mut_col.get()); - if (!col_as_lc->nestedIsNullable()) - col_as_lc->nestedToNullable(); - column.column = std::move(mut_col); - } - else - column.column = makeNullable(column.column); + /// Convert nested to nullable, not LowCardinality itself + auto mut_col = IColumn::mutate(std::move(column.column)); + ColumnLowCardinality * col_as_lc = assert_cast(mut_col.get()); + if (!col_as_lc->nestedIsNullable()) + col_as_lc->nestedToNullable(); + column.column = std::move(mut_col); + } + else if (column.column->canBeInsideNullable()) + { + column.column = makeNullable(column.column); } } From 7320411a7ff59b35cab2c3bb728580effcd9895d Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 6 Aug 2021 16:39:11 +0300 Subject: [PATCH 208/599] Fix bug from #23515. --- src/Storages/MergeTree/MergeTreeRangeReader.cpp | 7 ++++--- tests/queries/0_stateless/02003_bug_from_23515.reference | 1 + tests/queries/0_stateless/02003_bug_from_23515.sql | 6 ++++++ 3 files changed, 11 insertions(+), 3 deletions(-) create mode 100644 tests/queries/0_stateless/02003_bug_from_23515.reference create mode 100644 tests/queries/0_stateless/02003_bug_from_23515.sql diff --git a/src/Storages/MergeTree/MergeTreeRangeReader.cpp b/src/Storages/MergeTree/MergeTreeRangeReader.cpp index 2347280a4a0..464d253a12c 100644 --- a/src/Storages/MergeTree/MergeTreeRangeReader.cpp +++ b/src/Storages/MergeTree/MergeTreeRangeReader.cpp @@ -2,6 +2,7 @@ #include #include #include +#include #include #ifdef __SSE2__ @@ -1038,9 +1039,9 @@ void MergeTreeRangeReader::executePrewhereActionsAndFilterColumns(ReadResult & r /// Filter in WHERE instead else { - result.columns[prewhere_column_pos] = result.getFilterHolder()->convertToFullColumnIfConst(); - if (getSampleBlock().getByName(prewhere_info->prewhere_column_name).type->isNullable()) - result.columns[prewhere_column_pos] = makeNullable(std::move(result.columns[prewhere_column_pos])); + auto type = getSampleBlock().getByName(prewhere_info->prewhere_column_name).type; + ColumnWithTypeAndName col(result.getFilterHolder()->convertToFullColumnIfConst(), std::make_shared(), ""); + result.columns[prewhere_column_pos] = castColumn(col, type); result.clearFilter(); // Acting as a flag to not filter in PREWHERE } } diff --git a/tests/queries/0_stateless/02003_bug_from_23515.reference b/tests/queries/0_stateless/02003_bug_from_23515.reference new file mode 100644 index 00000000000..d00491fd7e5 --- /dev/null +++ b/tests/queries/0_stateless/02003_bug_from_23515.reference @@ -0,0 +1 @@ +1 diff --git a/tests/queries/0_stateless/02003_bug_from_23515.sql b/tests/queries/0_stateless/02003_bug_from_23515.sql new file mode 100644 index 00000000000..a1600c539c4 --- /dev/null +++ b/tests/queries/0_stateless/02003_bug_from_23515.sql @@ -0,0 +1,6 @@ +drop table if exists tab; +create table tab (a LowCardinality(String), b LowCardinality(String)) engine = MergeTree partition by a order by tuple() settings min_bytes_for_wide_part = 0, min_rows_for_wide_part = 0; + +insert into tab values ('1', 'a'), ('2', 'b'); +SELECT a = '1' FROM tab WHERE a = '1' and b='a'; +drop table if exists tab; From d33a634014e85bb87a88cf1464febae908b211c6 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 6 Aug 2021 16:12:59 +0200 Subject: [PATCH 209/599] Improve 00738_lock_for_inner_table stability A sufficiently fast machine would finish processing the MV before the drop --- tests/queries/0_stateless/00738_lock_for_inner_table.sh | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/00738_lock_for_inner_table.sh b/tests/queries/0_stateless/00738_lock_for_inner_table.sh index d19288f65d8..6cbed42baea 100755 --- a/tests/queries/0_stateless/00738_lock_for_inner_table.sh +++ b/tests/queries/0_stateless/00738_lock_for_inner_table.sh @@ -14,7 +14,8 @@ DROP TABLE IF EXISTS mv SYNC; -- create table with fsync and 20 partitions for slower INSERT -- (since increasing number of records will make it significantly slower in debug build, but not in release) CREATE TABLE tab_00738(a Int) ENGINE = MergeTree() ORDER BY a PARTITION BY a%20 SETTINGS fsync_after_insert=1; -CREATE MATERIALIZED VIEW mv UUID '$uuid' ENGINE = Log AS SELECT a FROM tab_00738;" | ${CLICKHOUSE_CLIENT} -n +-- The matview will take at least 2 seconds to be finished (10000000 * 0.0000002) +CREATE MATERIALIZED VIEW mv UUID '$uuid' ENGINE = Log AS SELECT sleepEachRow(0.0000002) FROM tab_00738;" | ${CLICKHOUSE_CLIENT} -n ${CLICKHOUSE_CLIENT} --query_id insert_$CLICKHOUSE_DATABASE --query "INSERT INTO tab_00738 SELECT number FROM numbers(10000000)" & From 48eb9954613eae87316243e012848dba4ff2ded3 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 6 Aug 2021 17:23:50 +0300 Subject: [PATCH 210/599] Fix unknown column bug in sampling. --- .../QueryPlan/ReadFromMergeTree.cpp | 8 ++++---- ..._sampling_and_unknown_column_bug.reference | 2 ++ .../02002_sampling_and_unknown_column_bug.sql | 20 +++++++++++++++++++ 3 files changed, 26 insertions(+), 4 deletions(-) create mode 100644 tests/queries/0_stateless/02002_sampling_and_unknown_column_bug.reference create mode 100644 tests/queries/0_stateless/02002_sampling_and_unknown_column_bug.sql diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index f8c12449c7e..379f44cdb5d 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -995,21 +995,18 @@ void ReadFromMergeTree::initializePipeline(QueryPipeline & pipeline, const Build Block cur_header = result_projection ? result_projection->getResultColumns() : pipe.getHeader(); - auto append_actions = [&result_projection, &cur_header](ActionsDAGPtr actions) + auto append_actions = [&result_projection](ActionsDAGPtr actions) { if (!result_projection) result_projection = std::move(actions); else result_projection = ActionsDAG::merge(std::move(*result_projection), std::move(*actions)); - - cur_header = result_projection->getResultColumns(); }; /// By the way, if a distributed query or query to a Merge table is made, then the `_sample_factor` column can have different values. if (sample_factor_column_queried) { ColumnWithTypeAndName column; - column.name = "_sample_factor"; column.type = std::make_shared(); column.column = column.type->createColumnConst(0, Field(result.sampling.used_sample_factor)); @@ -1017,6 +1014,9 @@ void ReadFromMergeTree::initializePipeline(QueryPipeline & pipeline, const Build append_actions(std::move(adding_column)); } + if (result_projection) + cur_header = result_projection->updateHeader(cur_header); + /// Extra columns may be returned (for example, if sampling is used). /// Convert pipe to step header structure. if (!isCompatibleHeader(cur_header, getOutputStream().header)) diff --git a/tests/queries/0_stateless/02002_sampling_and_unknown_column_bug.reference b/tests/queries/0_stateless/02002_sampling_and_unknown_column_bug.reference new file mode 100644 index 00000000000..9315e86b328 --- /dev/null +++ b/tests/queries/0_stateless/02002_sampling_and_unknown_column_bug.reference @@ -0,0 +1,2 @@ +1 +1 1 1 diff --git a/tests/queries/0_stateless/02002_sampling_and_unknown_column_bug.sql b/tests/queries/0_stateless/02002_sampling_and_unknown_column_bug.sql new file mode 100644 index 00000000000..838d7a5526b --- /dev/null +++ b/tests/queries/0_stateless/02002_sampling_and_unknown_column_bug.sql @@ -0,0 +1,20 @@ +drop table if exists sessions; +CREATE TABLE sessions +( + `user_id` UInt64 +) +ENGINE = MergeTree +ORDER BY user_id +SAMPLE BY user_id; + +insert into sessions values(1); + +SELECT + sum(user_id * _sample_factor) +FROM sessions +SAMPLE 10000000; + +SELECT + uniq(user_id) a, min(_sample_factor) x, a*x +FROM sessions +SAMPLE 10000000; From eb4d7415a285fa6d511879a0b3bc87105c37fba2 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 6 Aug 2021 17:28:56 +0300 Subject: [PATCH 211/599] Fix unknown column bug in sampling. --- src/Processors/QueryPlan/ReadFromMergeTree.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 379f44cdb5d..5fc32e2db69 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -1007,6 +1007,7 @@ void ReadFromMergeTree::initializePipeline(QueryPipeline & pipeline, const Build if (sample_factor_column_queried) { ColumnWithTypeAndName column; + column.name = "_sample_factor"; column.type = std::make_shared(); column.column = column.type->createColumnConst(0, Field(result.sampling.used_sample_factor)); From af536b1b5ee509f968b7cb77bccb3a7971f7fbd1 Mon Sep 17 00:00:00 2001 From: Mike Kot Date: Thu, 5 Aug 2021 17:15:51 +0300 Subject: [PATCH 212/599] Using formatted string literals, extracted sort funcs in tester --- tests/clickhouse-test | 316 ++++++++++++++++++++++++++---------------- 1 file changed, 194 insertions(+), 122 deletions(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index d83b3f08c42..212ccd79f00 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -11,6 +11,7 @@ import copy import traceback from argparse import ArgumentParser +from typing import Tuple, Union, Optional, TextIO import shlex import subprocess from subprocess import Popen @@ -20,10 +21,12 @@ from subprocess import TimeoutExpired from datetime import datetime from time import time, sleep from errno import ESRCH + try: import termcolor except ImportError: termcolor = None + import random import string import multiprocessing @@ -81,7 +84,7 @@ def stop_tests(): def json_minify(string): """ Removes all js-style comments from json string. Allows to have comments in skip_list.json. - The code taken from https://github.com/getify/JSON.minify/tree/python under the MIT license. + The code was taken from https://github.com/getify/JSON.minify/tree/python under the MIT license. """ tokenizer = re.compile(r'"|(/\*)|(\*/)|(//)|\n|\r') @@ -148,13 +151,17 @@ def remove_control_characters(s): s = re.sub(r"[\x00-\x08\x0b\x0e-\x1f\x7f]", "", s) return s + def get_db_engine(args, database_name): if args.replicated_database: - return " ON CLUSTER test_cluster_database_replicated ENGINE=Replicated('/test/clickhouse/db/{}', '{{shard}}', '{{replica}}')".format(database_name) + return f" ON CLUSTER test_cluster_database_replicated \ + ENGINE=Replicated('/test/clickhouse/db/{database_name}', \ + '{{shard}}', '{{replica}}')" if args.db_engine: return " ENGINE=" + args.db_engine return "" # Will use default engine + def configure_testcase_args(args, case_file, suite_tmp_dir, stderr_file): testcase_args = copy.deepcopy(args) @@ -166,7 +173,6 @@ def configure_testcase_args(args, case_file, suite_tmp_dir, stderr_file): database = testcase_args.database os.environ.setdefault("CLICKHOUSE_DATABASE", database) os.environ.setdefault("CLICKHOUSE_TMP", suite_tmp_dir) - else: # If --database is not specified, we will create temporary database with unique name # And we will recreate and drop it for each test @@ -176,8 +182,14 @@ def configure_testcase_args(args, case_file, suite_tmp_dir, stderr_file): database = 'test_{suffix}'.format(suffix=random_str()) with open(stderr_file, 'w') as stderr: - client_cmd = testcase_args.testcase_client + " " + get_additional_client_options(args) - clickhouse_proc_create = Popen(shlex.split(client_cmd), stdin=PIPE, stdout=PIPE, stderr=stderr, universal_newlines=True) + client_cmd = testcase_args.testcase_client + " " \ + + get_additional_client_options(args) + + clickhouse_proc_create = open_client_process( + universal_newlines=True, + client_args=client_cmd, + stderr_file=stderr) + try: clickhouse_proc_create.communicate(("CREATE DATABASE " + database + get_db_engine(testcase_args, database)), timeout=testcase_args.timeout) except TimeoutExpired: @@ -237,8 +249,10 @@ def run_single_test(args, ext, server_logs_level, client_options, case_file, std if need_drop_database: with open(stderr_file, 'a') as stderr: - clickhouse_proc_create = Popen(shlex.split(client), stdin=PIPE, stdout=PIPE, stderr=stderr, universal_newlines=True) + clickhouse_proc_create = open_client_process(client, universal_newlines=True, stderr_file=stderr) + seconds_left = max(args.timeout - (datetime.now() - start_time).total_seconds(), 20) + try: drop_database_query = "DROP DATABASE " + database if args.replicated_database: @@ -254,7 +268,7 @@ def run_single_test(args, ext, server_logs_level, client_options, case_file, std raise total_time = (datetime.now() - start_time).total_seconds() - return clickhouse_proc_create, "", "Timeout dropping database {} after test".format(database), total_time + return clickhouse_proc_create, "", f"Timeout dropping database {database} after test", total_time shutil.rmtree(args.test_tmp_dir) @@ -286,12 +300,16 @@ def need_retry(stdout, stderr): def get_processlist(args): try: query = b"SHOW PROCESSLIST FORMAT Vertical" + if args.replicated_database: query = b"SELECT materialize((hostName(), tcpPort())) as host, * " \ b"FROM clusterAllReplicas('test_cluster_database_replicated', system.processes) " \ b"WHERE query NOT LIKE '%system.processes%' FORMAT Vertical" - clickhouse_proc = Popen(shlex.split(args.client), stdin=PIPE, stdout=PIPE, stderr=PIPE) + + clickhouse_proc = open_client_process(args.client) + (stdout, _) = clickhouse_proc.communicate((query), timeout=20) + return False, stdout.decode('utf-8') except Exception as ex: print("Exception", ex) @@ -301,47 +319,90 @@ def get_processlist(args): # collect server stacktraces using gdb def get_stacktraces_from_gdb(server_pid): try: - cmd = "gdb -batch -ex 'thread apply all backtrace' -p {}".format(server_pid) + cmd = f"gdb -batch -ex 'thread apply all backtrace' -p {server_pid}" return subprocess.check_output(cmd, shell=True).decode('utf-8') - except Exception as ex: - print("Error occured while receiving stack traces from gdb: {}".format(str(ex))) + except Exception as e: + print(f"Error occurred while receiving stack traces from gdb: {e}") return None # collect server stacktraces from system.stack_trace table # it does not work in Sandbox def get_stacktraces_from_clickhouse(client, replicated_database=False): - try: - if replicated_database: - return subprocess.check_output("{} --allow_introspection_functions=1 --skip_unavailable_shards=1 --query " - "\"SELECT materialize((hostName(), tcpPort())) as host, thread_id, " - "arrayStringConcat(arrayMap(x, y -> concat(x, ': ', y), arrayMap(x -> addressToLine(x), trace), " - "arrayMap(x -> demangle(addressToSymbol(x)), trace)), '\n') as trace " - "FROM clusterAllReplicas('test_cluster_database_replicated', 'system.stack_trace') " - "ORDER BY host, thread_id format Vertical\"".format(client), shell=True, stderr=subprocess.STDOUT).decode('utf-8') + replicated_msg = \ + "{} --allow_introspection_functions=1 --skip_unavailable_shards=1 --query \ + \"SELECT materialize((hostName(), tcpPort())) as host, thread_id, \ + arrayStringConcat(arrayMap(x, y -> concat(x, ': ', y), \ + arrayMap(x -> addressToLine(x), trace), \ + arrayMap(x -> demangle(addressToSymbol(x)), trace)), '\n') as trace \ + FROM clusterAllReplicas('test_cluster_database_replicated', 'system.stack_trace') \ + ORDER BY host, thread_id FORMAT Vertical\"".format(client) - return subprocess.check_output("{} --allow_introspection_functions=1 --query " - "\"SELECT arrayStringConcat(arrayMap(x, y -> concat(x, ': ', y), arrayMap(x -> addressToLine(x), trace), " - "arrayMap(x -> demangle(addressToSymbol(x)), trace)), '\n') as trace " - "FROM system.stack_trace format Vertical\"".format(client), shell=True, stderr=subprocess.STDOUT).decode('utf-8') - except Exception as ex: - print("Error occured while receiving stack traces from client: {}".format(str(ex))) + msg = \ + "{} --allow_introspection_functions=1 --query \ + \"SELECT arrayStringConcat(arrayMap(x, y -> concat(x, ': ', y), \ + arrayMap(x -> addressToLine(x), trace), \ + arrayMap(x -> demangle(addressToSymbol(x)), trace)), '\n') as trace \ + FROM system.stack_trace FORMAT Vertical\"".format(client) + + try: + return subprocess.check_output( + replicated_msg if replicated_database else msg, + shell=True, stderr=subprocess.STDOUT).decode('utf-8') + except Exception as e: + print(f"Error occurred while receiving stack traces from client: {e}") return None -def get_server_pid(server_tcp_port): + +def print_stacktraces() -> None: + server_pid = get_server_pid() + + bt = None + + if server_pid and not args.replicated_database: + print("") + print(f"Located ClickHouse server process {server_pid} listening at TCP port {args.tcp_port}") + print("Collecting stacktraces from all running threads with gdb:") + + bt = get_stacktraces_from_gdb(server_pid) + + if len(bt) < 1000: + print("Got suspiciously small stacktraces: ", bt) + bt = None + + if bt is None: + print("\nCollecting stacktraces from system.stacktraces table:") + + bt = get_stacktraces_from_clickhouse( + args.client, args.replicated_database) + + if bt is not None: + print(bt) + return + + print(colored( + f"\nUnable to locate ClickHouse server process listening at TCP port {args.tcp_port}. " + "It must have crashed or exited prematurely!", + args, "red", attrs=["bold"])) + + +def get_server_pid(): # lsof does not work in stress tests for some reason - cmd_lsof = "lsof -i tcp:{port} -s tcp:LISTEN -Fp | awk '/^p[0-9]+$/{{print substr($0, 2)}}'".format(port=server_tcp_port) + cmd_lsof = f"lsof -i tcp:{args.tcp_port} -s tcp:LISTEN -Fp | awk '/^p[0-9]+$/{{print substr($0, 2)}}'" cmd_pidof = "pidof -s clickhouse-server" + commands = [cmd_lsof, cmd_pidof] output = None + for cmd in commands: try: output = subprocess.check_output(cmd, shell=True, stderr=subprocess.STDOUT, universal_newlines=True) if output: return int(output) except Exception as e: - print("Cannot get server pid with {}, got {}: {}".format(cmd, output, e)) - return None # most likely server dead + print(f"Cannot get server pid with {cmd}, got {output}: {e}") + + return None # most likely server is dead def colored(text, args, color=None, on_color=None, attrs=None): @@ -357,6 +418,14 @@ server_died = multiprocessing.Event() stop_tests_triggered_lock = multiprocessing.Lock() stop_tests_triggered = multiprocessing.Event() queue = multiprocessing.Queue(maxsize=1) + + +def print_test_time(test_time) -> str: + if args.print_time: + return " {0:.2f} sec.".format(test_time) + else: + return '' + restarted_tests = [] # (test, stderr) # def run_tests_array(all_tests, suite, suite_dir, suite_tmp_dir, run_total): @@ -385,15 +454,10 @@ def run_tests_array(all_tests_with_params): client_options = get_additional_client_options(args) - def print_test_time(test_time): - if args.print_time: - return " {0:.2f} sec.".format(test_time) - else: - return '' - if num_tests > 0: about = 'about ' if is_concurrent else '' - print(f"\nRunning {about}{num_tests} {suite} tests ({multiprocessing.current_process().name}).\n") + proc_name = multiprocessing.current_process().name + print(f"\nRunning {about}{num_tests} {suite} tests ({proc_name}).\n") while True: if is_concurrent: @@ -459,7 +523,6 @@ def run_tests_array(all_tests_with_params): message = open(disabled_file, 'r').read() status += MSG_SKIPPED + " - " + message + "\n" else: - if args.testname: clickhouse_proc = Popen(shlex.split(args.client), stdin=PIPE, stdout=PIPE, stderr=PIPE, universal_newlines=True) failed_to_check = False @@ -599,7 +662,12 @@ def run_tests_array(all_tests_with_params): except: exc_type, exc_value, tb = sys.exc_info() failures += 1 - print("{0} - Test internal error: {1}\n{2}\n{3}".format(MSG_FAIL, exc_type.__name__, exc_value, "\n".join(traceback.format_tb(tb, 10)))) + + exc_name = exc_type.__name__ + traceback_str = "\n".join(traceback.format_tb(tb, 10)) + + print(f"{MSG_FAIL} - Test internal error: {exc_name}") + print(f"{exc_value}\n{traceback_str}") if failures_chain >= 20: stop_tests() @@ -627,9 +695,11 @@ server_logs_level = "warning" def check_server_started(client, retry_count): print("Connecting to ClickHouse server...", end='') + sys.stdout.flush() + while retry_count > 0: - clickhouse_proc = Popen(shlex.split(client), stdin=PIPE, stdout=PIPE, stderr=PIPE) + clickhouse_proc = open_client_process(client) (stdout, stderr) = clickhouse_proc.communicate(b"SELECT 1") if clickhouse_proc.returncode == 0 and stdout.startswith(b"1"): @@ -679,7 +749,7 @@ class BuildFlags(): def collect_build_flags(client): - clickhouse_proc = Popen(shlex.split(client), stdin=PIPE, stdout=PIPE, stderr=PIPE) + clickhouse_proc = open_client_process(client) (stdout, stderr) = clickhouse_proc.communicate(b"SELECT value FROM system.build_options WHERE name = 'CXX_FLAGS'") result = [] @@ -695,7 +765,7 @@ def collect_build_flags(client): else: raise Exception("Cannot get information about build from server errorcode {}, stderr {}".format(clickhouse_proc.returncode, stderr)) - clickhouse_proc = Popen(shlex.split(client), stdin=PIPE, stdout=PIPE, stderr=PIPE) + clickhouse_proc = open_client_process(client) (stdout, stderr) = clickhouse_proc.communicate(b"SELECT value FROM system.build_options WHERE name = 'BUILD_TYPE'") if clickhouse_proc.returncode == 0: @@ -706,7 +776,7 @@ def collect_build_flags(client): else: raise Exception("Cannot get information about build from server errorcode {}, stderr {}".format(clickhouse_proc.returncode, stderr)) - clickhouse_proc = Popen(shlex.split(client), stdin=PIPE, stdout=PIPE, stderr=PIPE) + clickhouse_proc = open_client_process(client) (stdout, stderr) = clickhouse_proc.communicate(b"SELECT value FROM system.build_options WHERE name = 'UNBUNDLED'") if clickhouse_proc.returncode == 0: @@ -715,7 +785,7 @@ def collect_build_flags(client): else: raise Exception("Cannot get information about build from server errorcode {}, stderr {}".format(clickhouse_proc.returncode, stderr)) - clickhouse_proc = Popen(shlex.split(client), stdin=PIPE, stdout=PIPE, stderr=PIPE) + clickhouse_proc = open_client_process(client) (stdout, stderr) = clickhouse_proc.communicate(b"SELECT value FROM system.settings WHERE name = 'default_database_engine'") if clickhouse_proc.returncode == 0: @@ -724,7 +794,7 @@ def collect_build_flags(client): else: raise Exception("Cannot get information about build from server errorcode {}, stderr {}".format(clickhouse_proc.returncode, stderr)) - clickhouse_proc = Popen(shlex.split(client), stdin=PIPE, stdout=PIPE, stderr=PIPE) + clickhouse_proc = open_client_process(client) (stdout, stderr) = clickhouse_proc.communicate(b"SELECT value FROM system.merge_tree_settings WHERE name = 'min_bytes_for_wide_part'") if clickhouse_proc.returncode == 0: @@ -736,6 +806,56 @@ def collect_build_flags(client): return result +def suite_key_func(item: str) -> Union[int, Tuple[int, str]]: + if args.order == 'random': + return random.random() + + if -1 == item.find('_'): + return 99998, '' + + prefix, suffix = item.split('_', 1) + + try: + return int(prefix), suffix + except ValueError: + return 99997, '' + + +def tests_in_suite_key_func(item: str) -> int: + if args.order == 'random': + return random.random() + + reverse = 1 if args.order == 'asc' else -1 + + if -1 == item.find('_'): + return 99998 + + prefix, _ = item.split('_', 1) + + try: + return reverse * int(prefix) + except ValueError: + return 99997 + + +def extract_key(key: str) -> str: + return subprocess.getstatusoutput( + args.extract_from_config + + " --try --config " + + args.configserver + key)[1] + + +def open_client_process( + client_args: str, + universal_newlines: bool = False, + stderr_file: Optional[TextIO] = None): + return Popen( + shlex.split(client_args), stdin=PIPE, stdout=PIPE, + stderr=stderr_file if stderr_file is not None else PIPE, + universal_newlines=True if universal_newlines else None) + + + def do_run_tests(jobs, suite, suite_dir, suite_tmp_dir, all_tests, parallel_tests, sequential_tests, parallel): if jobs > 1 and len(parallel_tests) > 0: print("Found", len(parallel_tests), "parallel tests and", len(sequential_tests), "sequential tests") @@ -790,7 +910,7 @@ def removesuffix(text, *suffixes): Added in python 3.9 https://www.python.org/dev/peps/pep-0616/ - This version can work with severtal possible suffixes + This version can work with several possible suffixes """ for suffix in suffixes: if suffix and text.endswith(suffix): @@ -875,7 +995,7 @@ def main(args): global server_logs_level def is_data_present(): - clickhouse_proc = Popen(shlex.split(args.client), stdin=PIPE, stdout=PIPE, stderr=PIPE) + clickhouse_proc = open_client_process(args.client) (stdout, stderr) = clickhouse_proc.communicate(b"EXISTS TABLE test.hits") if clickhouse_proc.returncode != 0: raise CalledProcessError(clickhouse_proc.returncode, args.client, stderr) @@ -885,9 +1005,10 @@ def main(args): if not check_server_started(args.client, args.server_check_retries): raise Exception( "Server is not responding. Cannot execute 'SELECT 1' query. \ - Note: if you are using split build, you may have to specify -c option.") + If you are using split build, you have to specify -c option.") build_flags = collect_build_flags(args.client) + if args.replicated_database: build_flags.append(BuildFlags.DATABASE_REPLICATED) @@ -911,6 +1032,7 @@ def main(args): os.environ.setdefault("CLICKHOUSE_BINARY", args.binary) #os.environ.setdefault("CLICKHOUSE_CLIENT", args.client) os.environ.setdefault("CLICKHOUSE_CONFIG", args.configserver) + if args.configclient: os.environ.setdefault("CLICKHOUSE_CONFIG_CLIENT", args.configclient) @@ -923,52 +1045,35 @@ def main(args): stop_time = time() + args.global_time_limit if args.zookeeper is None: - _, out = subprocess.getstatusoutput(args.extract_from_config + " --try --config " + args.configserver + ' --key zookeeper | grep . | wc -l') try: - if int(out) > 0: - args.zookeeper = True - else: - args.zookeeper = False + args.zookeeper = int(extract_key(" --key zookeeper | grep . | wc -l")) > 0 except ValueError: args.zookeeper = False if args.shard is None: - _, out = subprocess.getstatusoutput(args.extract_from_config + " --try --config " + args.configserver + ' --key listen_host | grep -E "127.0.0.2|::"') - if out: - args.shard = True - else: - args.shard = False + args.shard = bool(extract_key(' --key listen_host | grep -E "127.0.0.2|::"')) def create_common_database(args, db_name): create_database_retries = 0 while create_database_retries < MAX_RETRIES: client_cmd = args.client + " " + get_additional_client_options(args) - clickhouse_proc_create = Popen(shlex.split(client_cmd), stdin=PIPE, stdout=PIPE, stderr=PIPE, universal_newlines=True) + + clickhouse_proc_create = open_client_process(client_cmd, universal_newlines=True) + (stdout, stderr) = clickhouse_proc_create.communicate(("CREATE DATABASE IF NOT EXISTS " + db_name + get_db_engine(args, db_name))) + if not need_retry(stdout, stderr): break create_database_retries += 1 if args.database and args.database != "test": create_common_database(args, args.database) + create_common_database(args, "test") - def sute_key_func(item): - if args.order == 'random': - return random.random() - - if -1 == item.find('_'): - return 99998, '' - - prefix, suffix = item.split('_', 1) - - try: - return int(prefix), suffix - except ValueError: - return 99997, '' - total_tests_run = 0 - for suite in sorted(os.listdir(base_dir), key=sute_key_func): + + for suite in sorted(os.listdir(base_dir), key=suite_key_func): if server_died.is_set(): break @@ -982,8 +1087,8 @@ def main(args): os.makedirs(suite_tmp_dir) suite = suite_re_obj.group(1) - if os.path.isdir(suite_dir): + if os.path.isdir(suite_dir): if 'stateful' in suite and not args.no_stateful and not is_data_present(): print("Won't run stateful tests because test data wasn't loaded.") continue @@ -994,29 +1099,14 @@ def main(args): print("Won't run stateful tests because they were manually disabled.") continue - # Reverse sort order: we want run newest test first. - # And not reverse subtests - def key_func(item): - if args.order == 'random': - return random.random() - - reverse = 1 if args.order == 'asc' else -1 - - if -1 == item.find('_'): - return 99998 - - prefix, _ = item.split('_', 1) - - try: - return reverse * int(prefix) - except ValueError: - return 99997 - - all_tests = get_tests_list(suite_dir, args.test, args.test_runs, key_func) + all_tests = get_tests_list( + suite_dir, args.test, args.test_runs, tests_in_suite_key_func) jobs = args.jobs + parallel_tests = [] sequential_tests = [] + for test in all_tests: if any(s in test for s in args.sequential): sequential_tests.append(test) @@ -1042,44 +1132,21 @@ def main(args): else: print(colored("Seems like server hung and cannot respond to queries", args, "red", attrs=["bold"])) - clickhouse_tcp_port = os.getenv("CLICKHOUSE_PORT_TCP", '9000') - server_pid = get_server_pid(clickhouse_tcp_port) - bt = None - if server_pid and not args.replicated_database: - print("\nLocated ClickHouse server process {} listening at TCP port {}".format(server_pid, clickhouse_tcp_port)) - print("\nCollecting stacktraces from all running threads with gdb:") - bt = get_stacktraces_from_gdb(server_pid) - if len(bt) < 1000: - print("Got suspiciously small stacktraces: ", bt) - bt = None - if bt is None: - print("\nCollecting stacktraces from system.stacktraces table:") - bt = get_stacktraces_from_clickhouse(args.client, args.replicated_database) - if bt is None: - print( - colored( - "\nUnable to locate ClickHouse server process listening at TCP port {}. " - "It must have crashed or exited prematurely!".format(clickhouse_tcp_port), - args, "red", attrs=["bold"])) - else: - print(bt) + print_stacktraces() exit_code.value = 1 else: print(colored("\nNo queries hung.", args, "green", attrs=["bold"])) if len(restarted_tests) > 0: print("\nSome tests were restarted:\n") + for (test_case, stderr) in restarted_tests: - print(test_case) - print(stderr) - print("\n") + print(test_case + "\n" + stderr + "\n") if total_tests_run == 0: print("No tests were run.") sys.exit(1) - else: - print("All tests have finished.") sys.exit(exit_code.value) @@ -1196,9 +1263,11 @@ if __name__ == '__main__': parser.add_argument('--no-long', action='store_false', dest='no_long', help='Do not run long tests') parser.add_argument('--client-option', nargs='+', help='Specify additional client argument') parser.add_argument('--print-time', action='store_true', dest='print_time', help='Print test time') + group=parser.add_mutually_exclusive_group(required=False) group.add_argument('--zookeeper', action='store_true', default=None, dest='zookeeper', help='Run zookeeper related tests') group.add_argument('--no-zookeeper', action='store_false', default=None, dest='zookeeper', help='Do not run zookeeper related tests') + group=parser.add_mutually_exclusive_group(required=False) group.add_argument('--shard', action='store_true', default=None, dest='shard', help='Run sharding related tests (required to clickhouse-server listen 127.0.0.2 127.0.0.3)') group.add_argument('--no-shard', action='store_false', default=None, dest='shard', help='Do not run shard related tests') @@ -1206,7 +1275,7 @@ if __name__ == '__main__': args = parser.parse_args() if args.queries and not os.path.isdir(args.queries): - print("Cannot access the specified directory with queries (" + args.queries + ")", file=sys.stderr) + print(f"Cannot access the specified directory with queries ({args.queries})", file=sys.stderr) sys.exit(1) # Autodetect the directory with queries if not specified @@ -1257,10 +1326,13 @@ if __name__ == '__main__': if args.configclient: args.client += ' --config-file=' + args.configclient + if os.getenv("CLICKHOUSE_HOST"): args.client += ' --host=' + os.getenv("CLICKHOUSE_HOST") - if os.getenv("CLICKHOUSE_PORT_TCP"): - args.client += ' --port=' + os.getenv("CLICKHOUSE_PORT_TCP") + + args.tcp_port = int(os.getenv("CLICKHOUSE_PORT_TCP", 9000)) + args.client += f" --port={args.tcp_port}" + if os.getenv("CLICKHOUSE_DATABASE"): args.client += ' --database=' + os.getenv("CLICKHOUSE_DATABASE") From a308c8f8427b3f58c5ca2b879fc564a5c6bfbc4a Mon Sep 17 00:00:00 2001 From: Mike Kot Date: Fri, 6 Aug 2021 17:38:28 +0300 Subject: [PATCH 213/599] Extracted test skipping functions --- tests/clickhouse-test | 357 ++++++++++++++++++++++++------------------ 1 file changed, 201 insertions(+), 156 deletions(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 212ccd79f00..b734af0bdea 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -54,7 +54,7 @@ MESSAGES_TO_RETRY = [ "DB::Exception: New table appeared in database being dropped or detached. Try again", "is already started to be removing by another replica right now", "Shutdown is called for table", # It happens in SYSTEM SYNC REPLICA query if session with ZooKeeper is being reinitialized. - DISTRIBUTED_DDL_TIMEOUT_MSG # FIXME + DISTRIBUTED_DDL_TIMEOUT_MSG # FIXME ] MAX_RETRIES = 3 @@ -426,6 +426,67 @@ def print_test_time(test_time) -> str: else: return '' + +def should_skip_test_by_name(name: str, test_ext: str) -> Tuple[bool, str]: + if args.skip and any(s in name for s in args.skip): + return True, "skip" + + if not args.zookeeper and ('zookeeper' in name or 'replica' in name): + return True, "no zookeeper" + + if not args.shard and \ + ('shard' in name or 'distributed' in name or 'global' in name): + return True, "no shard" + + # Tests for races and deadlocks usually are run in a loop for a significant + # amount of time + if args.no_long and \ + ('long' in name or 'deadlock' in name or 'race' in name): + return True, "no long" + + if not USE_JINJA and test_ext.endswith("j2"): + return True, "no jinja" + + return False, "" + + +def should_skip_disabled_test(name: str, suite_dir: str) -> Tuple[bool, str]: + disabled_file = os.path.join(suite_dir, name) + '.disabled' + + if os.path.exists(disabled_file) and not args.disabled: + return True, open(disabled_file, 'r').read() + + return False, "" + + +# should skip test, should increment skipped_total, skip reason +def should_skip_test(name: str, test_ext: str, suite_dir: str) -> Tuple[bool, bool, str]: + should_skip, skip_reason = should_skip_test_by_name(name, test_ext) + + if should_skip: + return True, True, skip_reason + + should_skip, skip_reason = should_skip_disabled_test(name, suite_dir) + + return should_skip, False, skip_reason + + +def send_test_name_failed(suite: str, case: str) -> bool: + clickhouse_proc = open_client_process(args.client, universal_newlines=True) + + failed_to_check = False + + pid = os.getpid() + query = f"SELECT 'Running test {suite}/{case} from pid={pid}';" + + try: + clickhouse_proc.communicate((query), timeout=20) + except: + failed_to_check = True + + return failed_to_check or clickhouse_proc.returncode != 0 + + restarted_tests = [] # (test, stderr) # def run_tests_array(all_tests, suite, suite_dir, suite_tmp_dir, run_total): @@ -494,161 +555,135 @@ def run_tests_array(all_tests_with_params): else: status = "{0:72}".format(removesuffix(name, ".gen", ".sql") + ": ") - if args.skip and any(s in name for s in args.skip): - status += MSG_SKIPPED + " - skip\n" - skipped_total += 1 - elif not args.zookeeper and ('zookeeper' in name - or 'replica' in name): - status += MSG_SKIPPED + " - no zookeeper\n" - skipped_total += 1 - elif not args.shard and ('shard' in name - or 'distributed' in name - or 'global' in name): - status += MSG_SKIPPED + " - no shard\n" - skipped_total += 1 - elif not args.no_long and ('long' in name - # Tests for races and deadlocks usually are runned in loop - # for significant amount of time - or 'deadlock' in name - or 'race' in name): - status += MSG_SKIPPED + " - no long\n" - skipped_total += 1 - elif not USE_JINJA and ext.endswith("j2"): - status += MSG_SKIPPED + " - no jinja\n" - skipped_total += 1 + skip_test, increment_skip_count, skip_reason = \ + should_skip_test(name, ext, suite_dir) + + if skip_test: + status += MSG_SKIPPED + f" - {skip_reason}\n" + + if increment_skip_count: + skipped_total += 1 else: - disabled_file = os.path.join(suite_dir, name) + '.disabled' + if args.testname and send_test_name_failed(suite, case): + failures += 1 + print("Server does not respond to health check") + server_died.set() + stop_tests() + break - if os.path.exists(disabled_file) and not args.disabled: - message = open(disabled_file, 'r').read() - status += MSG_SKIPPED + " - " + message + "\n" + file_suffix = ('.' + str(os.getpid())) if is_concurrent and args.test_runs > 1 else '' + reference_file = get_reference_file(suite_dir, name) + stdout_file = os.path.join(suite_tmp_dir, name) + file_suffix + '.stdout' + stderr_file = os.path.join(suite_tmp_dir, name) + file_suffix + '.stderr' + + testcase_args = configure_testcase_args(args, case_file, suite_tmp_dir, stderr_file) + proc, stdout, stderr, total_time = run_single_test(testcase_args, ext, server_logs_level, client_options, case_file, stdout_file, stderr_file) + + if proc.returncode is None: + try: + proc.kill() + except OSError as e: + if e.errno != ESRCH: + raise + + failures += 1 + status += MSG_FAIL + status += print_test_time(total_time) + status += " - Timeout!\n" + if stderr: + status += stderr + status += 'Database: ' + testcase_args.testcase_database else: - if args.testname: - clickhouse_proc = Popen(shlex.split(args.client), stdin=PIPE, stdout=PIPE, stderr=PIPE, universal_newlines=True) - failed_to_check = False - try: - clickhouse_proc.communicate(("SELECT 'Running test {suite}/{case} from pid={pid}';".format(pid = os.getpid(), case = case, suite = suite)), timeout=20) - except: - failed_to_check = True - - if failed_to_check or clickhouse_proc.returncode != 0: - failures += 1 - print("Server does not respond to health check") - server_died.set() - stop_tests() + counter = 1 + while need_retry(stdout, stderr): + restarted_tests.append((case_file, stderr)) + testcase_args = configure_testcase_args(args, case_file, suite_tmp_dir, stderr_file) + proc, stdout, stderr, total_time = run_single_test(testcase_args, ext, server_logs_level, client_options, case_file, stdout_file, stderr_file) + sleep(2**counter) + counter += 1 + if MAX_RETRIES < counter: + if args.replicated_database: + if DISTRIBUTED_DDL_TIMEOUT_MSG in stderr: + server_died.set() break - file_suffix = ('.' + str(os.getpid())) if is_concurrent and args.test_runs > 1 else '' - reference_file = get_reference_file(suite_dir, name) - stdout_file = os.path.join(suite_tmp_dir, name) + file_suffix + '.stdout' - stderr_file = os.path.join(suite_tmp_dir, name) + file_suffix + '.stderr' - - testcase_args = configure_testcase_args(args, case_file, suite_tmp_dir, stderr_file) - proc, stdout, stderr, total_time = run_single_test(testcase_args, ext, server_logs_level, client_options, case_file, stdout_file, stderr_file) - - if proc.returncode is None: - try: - proc.kill() - except OSError as e: - if e.errno != ESRCH: - raise - + if proc.returncode != 0: failures += 1 + failures_chain += 1 status += MSG_FAIL status += print_test_time(total_time) - status += " - Timeout!\n" + status += ' - return code {}\n'.format(proc.returncode) + if stderr: status += stderr + + # Stop on fatal errors like segmentation fault. They are sent to client via logs. + if ' ' in stderr: + server_died.set() + + if testcase_args.stop \ + and ('Connection refused' in stderr or 'Attempt to read after eof' in stderr) \ + and 'Received exception from server' not in stderr: + server_died.set() + + if os.path.isfile(stdout_file): + status += ", result:\n\n" + status += '\n'.join( + open(stdout_file).read().split('\n')[:100]) + status += '\n' + + status += 'Database: ' + testcase_args.testcase_database + + elif stderr: + failures += 1 + failures_chain += 1 + status += MSG_FAIL + status += print_test_time(total_time) + status += " - having stderror:\n{}\n".format( + '\n'.join(stderr.split('\n')[:100])) + status += 'Database: ' + testcase_args.testcase_database + elif 'Exception' in stdout: + failures += 1 + failures_chain += 1 + status += MSG_FAIL + status += print_test_time(total_time) + status += " - having exception:\n{}\n".format( + '\n'.join(stdout.split('\n')[:100])) + status += 'Database: ' + testcase_args.testcase_database + elif reference_file is None: + status += MSG_UNKNOWN + status += print_test_time(total_time) + status += " - no reference file\n" status += 'Database: ' + testcase_args.testcase_database else: - counter = 1 - while need_retry(stdout, stderr): - restarted_tests.append((case_file, stderr)) - testcase_args = configure_testcase_args(args, case_file, suite_tmp_dir, stderr_file) - proc, stdout, stderr, total_time = run_single_test(testcase_args, ext, server_logs_level, client_options, case_file, stdout_file, stderr_file) - sleep(2**counter) - counter += 1 - if MAX_RETRIES < counter: - if args.replicated_database: - if DISTRIBUTED_DDL_TIMEOUT_MSG in stderr: - server_died.set() - break + result_is_different = subprocess.call(['diff', '-q', reference_file, stdout_file], stdout=PIPE) - if proc.returncode != 0: + if result_is_different: + diff = Popen(['diff', '-U', str(testcase_args.unified), reference_file, stdout_file], stdout=PIPE, universal_newlines=True).communicate()[0] failures += 1 - failures_chain += 1 status += MSG_FAIL status += print_test_time(total_time) - status += ' - return code {}\n'.format(proc.returncode) - - if stderr: - status += stderr - - # Stop on fatal errors like segmentation fault. They are sent to client via logs. - if ' ' in stderr: - server_died.set() - - if testcase_args.stop and ('Connection refused' in stderr or 'Attempt to read after eof' in stderr) and not 'Received exception from server' in stderr: - server_died.set() - - if os.path.isfile(stdout_file): - status += ", result:\n\n" - status += '\n'.join( - open(stdout_file).read().split('\n')[:100]) - status += '\n' - - status += 'Database: ' + testcase_args.testcase_database - - elif stderr: - failures += 1 - failures_chain += 1 - status += MSG_FAIL - status += print_test_time(total_time) - status += " - having stderror:\n{}\n".format( - '\n'.join(stderr.split('\n')[:100])) - status += 'Database: ' + testcase_args.testcase_database - elif 'Exception' in stdout: - failures += 1 - failures_chain += 1 - status += MSG_FAIL - status += print_test_time(total_time) - status += " - having exception:\n{}\n".format( - '\n'.join(stdout.split('\n')[:100])) - status += 'Database: ' + testcase_args.testcase_database - elif reference_file is None: - status += MSG_UNKNOWN - status += print_test_time(total_time) - status += " - no reference file\n" + status += " - result differs with reference:\n{}\n".format(diff) status += 'Database: ' + testcase_args.testcase_database else: - result_is_different = subprocess.call(['diff', '-q', reference_file, stdout_file], stdout=PIPE) - - if result_is_different: - diff = Popen(['diff', '-U', str(testcase_args.unified), reference_file, stdout_file], stdout=PIPE, universal_newlines=True).communicate()[0] + if testcase_args.test_runs > 1 and total_time > 60 and 'long' not in name: + # We're in Flaky Check mode, check the run time as well while we're at it. failures += 1 + failures_chain += 1 status += MSG_FAIL status += print_test_time(total_time) - status += " - result differs with reference:\n{}\n".format(diff) + status += " - Test runs too long (> 60s). Make it faster.\n" status += 'Database: ' + testcase_args.testcase_database else: - if testcase_args.test_runs > 1 and total_time > 60 and 'long' not in name: - # We're in Flaky Check mode, check the run time as well while we're at it. - failures += 1 - failures_chain += 1 - status += MSG_FAIL - status += print_test_time(total_time) - status += " - Test runs too long (> 60s). Make it faster.\n" - status += 'Database: ' + testcase_args.testcase_database - else: - passed_total += 1 - failures_chain = 0 - status += MSG_OK - status += print_test_time(total_time) - status += "\n" - if os.path.exists(stdout_file): - os.remove(stdout_file) - if os.path.exists(stderr_file): - os.remove(stderr_file) + passed_total += 1 + failures_chain = 0 + status += MSG_OK + status += print_test_time(total_time) + status += "\n" + if os.path.exists(stdout_file): + os.remove(stdout_file) + if os.path.exists(stderr_file): + os.remove(stderr_file) if status and not status.endswith('\n'): status += '\n' @@ -709,27 +744,30 @@ def check_server_started(client, retry_count): if clickhouse_proc.returncode == 210: # Connection refused, retry - print('.', end = '') + print('.', end='') sys.stdout.flush() retry_count -= 1 sleep(0.5) continue - # Other kind of error, fail. - print('') - print("Client invocation failed with code ", clickhouse_proc.returncode, ": ") + # FIXME Some old comment, maybe now CH supports Python3 ? # We can't print this, because for some reason this is python 2, # and args appeared in 3.3. To hell with it. # print(''.join(clickhouse_proc.args)) - print("stdout: ") - print(stdout) - print("stderr: ") - print(stderr) + + # Other kind of error, fail. + + code: int = clickhouse_proc.returncode + + print(f"\nClient invocation failed with code {code}:\n\ + stdout: {stdout}\n\ + stderr: {stderr}") + sys.stdout.flush() + return False - print('') - print('All connection tries failed') + print('\nAll connection tries failed') sys.stdout.flush() return False @@ -928,7 +966,7 @@ def render_test_template(j2env, suite_dir, test_name): test_base_name = removesuffix(test_name, ".sql.j2", ".sql") - reference_file_name = test_base_name + ".reference.j2" + reference_file_name = test_base_name + ".reference.j2" reference_file_path = os.path.join(suite_dir, reference_file_name) if os.path.isfile(reference_file_path): tpl = j2env.get_template(reference_file_name) @@ -1015,7 +1053,7 @@ def main(args): if args.use_skip_list: tests_to_skip_from_list = collect_tests_to_skip(args.skip_list_path, build_flags) else: - tests_to_skip_from_list = set([]) + tests_to_skip_from_list = set() if args.skip: args.skip = set(args.skip) | tests_to_skip_from_list @@ -1030,7 +1068,7 @@ def main(args): # Keep same default values as in queries/shell_config.sh os.environ.setdefault("CLICKHOUSE_BINARY", args.binary) - #os.environ.setdefault("CLICKHOUSE_CLIENT", args.client) + # os.environ.setdefault("CLICKHOUSE_CLIENT", args.client) os.environ.setdefault("CLICKHOUSE_CONFIG", args.configserver) if args.configclient: @@ -1079,7 +1117,7 @@ def main(args): suite_dir = os.path.join(base_dir, suite) suite_re_obj = re.search('^[0-9]+_(.*)$', suite) - if not suite_re_obj: #skip .gitignore and so on + if not suite_re_obj: # skip .gitignore and so on continue suite_tmp_dir = os.path.join(tmp_dir, suite) @@ -1147,6 +1185,8 @@ def main(args): if total_tests_run == 0: print("No tests were run.") sys.exit(1) + else: + print("All tests have finished.") sys.exit(exit_code.value) @@ -1183,18 +1223,23 @@ def get_additional_client_options_url(args): def collect_tests_to_skip(skip_list_path, build_flags): result = set([]) + if not os.path.exists(skip_list_path): return result with open(skip_list_path, 'r') as skip_list_file: content = skip_list_file.read() + # allows to have comments in skip_list.json skip_dict = json.loads(json_minify(content)) + for build_flag in build_flags: result |= set(skip_dict[build_flag]) - if len(result) > 0: - print("Found file with skip-list {}, {} test will be skipped".format(skip_list_path, len(result))) + count = len(result) + + if count > 0: + print(f"Found file with skip-list {skip_list_path}, {count} test will be skipped") return result @@ -1221,7 +1266,7 @@ if __name__ == '__main__': signal.signal(signal.SIGINT, signal_handler) signal.signal(signal.SIGHUP, signal_handler) - parser=ArgumentParser(description='ClickHouse functional tests') + parser = ArgumentParser(description='ClickHouse functional tests') parser.add_argument('-q', '--queries', help='Path to queries dir') parser.add_argument('--tmp', help='Path to tmp dir') @@ -1233,7 +1278,7 @@ if __name__ == '__main__': parser.add_argument('--extract_from_config', help='extract-from-config program') parser.add_argument('--configclient', help='Client config (if you use not default ports)') - parser.add_argument('--configserver', default= '/etc/clickhouse-server/config.xml', help='Preprocessed server config') + parser.add_argument('--configserver', default='/etc/clickhouse-server/config.xml', help='Preprocessed server config') parser.add_argument('-o', '--output', help='Output xUnit compliant test report directory') parser.add_argument('-t', '--timeout', type=int, default=600, help='Timeout for each test case in seconds') parser.add_argument('--global_time_limit', type=int, help='Stop if executing more than specified time (after current test finished)') @@ -1260,15 +1305,15 @@ if __name__ == '__main__': parser.add_argument('--no-stateful', action='store_true', help='Disable all stateful tests') parser.add_argument('--skip', nargs='+', help="Skip these tests") parser.add_argument('--sequential', nargs='+', help="Run these tests sequentially even if --parallel specified") - parser.add_argument('--no-long', action='store_false', dest='no_long', help='Do not run long tests') + parser.add_argument('--no-long', action='store_true', dest='no_long', help='Do not run long tests') parser.add_argument('--client-option', nargs='+', help='Specify additional client argument') parser.add_argument('--print-time', action='store_true', dest='print_time', help='Print test time') - group=parser.add_mutually_exclusive_group(required=False) + group = parser.add_mutually_exclusive_group(required=False) group.add_argument('--zookeeper', action='store_true', default=None, dest='zookeeper', help='Run zookeeper related tests') group.add_argument('--no-zookeeper', action='store_false', default=None, dest='zookeeper', help='Do not run zookeeper related tests') - group=parser.add_mutually_exclusive_group(required=False) + group = parser.add_mutually_exclusive_group(required=False) group.add_argument('--shard', action='store_true', default=None, dest='shard', help='Run sharding related tests (required to clickhouse-server listen 127.0.0.2 127.0.0.3)') group.add_argument('--no-shard', action='store_false', default=None, dest='shard', help='Do not run shard related tests') @@ -1284,7 +1329,7 @@ if __name__ == '__main__': if not os.path.isdir(args.queries): # If we're running from the repo - args.queries = os.path.join(os.path.dirname(os.path.abspath( __file__ )), 'queries') + args.queries = os.path.join(os.path.dirname(os.path.abspath(__file__)), 'queries') if not os.path.isdir(args.queries): # Next we're going to try some system directories, don't write 'stdout' files into them. @@ -1330,7 +1375,7 @@ if __name__ == '__main__': if os.getenv("CLICKHOUSE_HOST"): args.client += ' --host=' + os.getenv("CLICKHOUSE_HOST") - args.tcp_port = int(os.getenv("CLICKHOUSE_PORT_TCP", 9000)) + args.tcp_port = int(os.getenv("CLICKHOUSE_PORT_TCP", "9000")) args.client += f" --port={args.tcp_port}" if os.getenv("CLICKHOUSE_DATABASE"): From e00536dede57e155e87ec05cb6028e728b88999b Mon Sep 17 00:00:00 2001 From: Artur <613623@mail.ru> Date: Fri, 6 Aug 2021 14:47:20 +0000 Subject: [PATCH 214/599] correcting test --- .../02003_memory_limit_in_client.reference | 60127 +--------------- .../02003_memory_limit_in_client.sh | 7 +- 2 files changed, 4 insertions(+), 60130 deletions(-) diff --git a/tests/queries/0_stateless/02003_memory_limit_in_client.reference b/tests/queries/0_stateless/02003_memory_limit_in_client.reference index 0b82ccc5544..52b628cf6d5 100644 --- a/tests/queries/0_stateless/02003_memory_limit_in_client.reference +++ b/tests/queries/0_stateless/02003_memory_limit_in_client.reference @@ -1,60125 +1,2 @@ -0 -1 -2 -3 -4 -5 -6 -7 -8 -9 -10 -11 -12 -13 -14 -15 -16 -17 -18 -19 -20 -21 -22 -23 -24 -25 -26 -27 -28 -29 -30 -31 -32 -33 -34 -35 -36 -37 -38 -39 -40 -41 -42 -43 -44 -45 -46 -47 -48 -49 -50 -51 -52 -53 -54 -55 -56 -57 -58 -59 -60 -61 -62 -63 -64 -65 -66 -67 -68 -69 -70 -71 -72 -73 -74 -75 -76 -77 -78 -79 -80 -81 -82 -83 -84 -85 -86 -87 -88 -89 -90 -91 -92 -93 -94 -95 -96 -97 -98 -99 -100 -101 -102 -103 -104 -105 -106 -107 -108 -109 -110 -111 -112 -113 -114 -115 -116 -117 -118 -119 -120 -121 -122 -123 -124 -125 -126 -127 -128 -129 -130 -131 -132 -133 -134 -135 -136 -137 -138 -139 -140 -141 -142 -143 -144 -145 -146 -147 -148 -149 -150 -151 -152 -153 -154 -155 -156 -157 -158 -159 -160 -161 -162 -163 -164 -165 -166 -167 -168 -169 -170 -171 -172 -173 -174 -175 -176 -177 -178 -179 -180 -181 -182 -183 -184 -185 -186 -187 -188 -189 -190 -191 -192 -193 -194 -195 -196 -197 -198 -199 -200 -201 -202 -203 -204 -205 -206 -207 -208 -209 -210 -211 -212 -213 -214 -215 -216 -217 -218 -219 -220 -221 -222 -223 -224 -225 -226 -227 -228 -229 -230 -231 -232 -233 -234 -235 -236 -237 -238 -239 -240 -241 -242 -243 -244 -245 -246 -247 -248 -249 -250 -251 -252 -253 -254 -255 -256 -257 -258 -259 -260 -261 -262 -263 -264 -265 -266 -267 -268 -269 -270 -271 -272 -273 -274 -275 -276 -277 -278 -279 -280 -281 -282 -283 -284 -285 -286 -287 -288 -289 -290 -291 -292 -293 -294 -295 -296 -297 -298 -299 -300 -301 -302 -303 -304 -305 -306 -307 -308 -309 -310 -311 -312 -313 -314 -315 -316 -317 -318 -319 -320 -321 -322 -323 -324 -325 -326 -327 -328 -329 -330 -331 -332 -333 -334 -335 -336 -337 -338 -339 -340 -341 -342 -343 -344 -345 -346 -347 -348 -349 -350 -351 -352 -353 -354 -355 -356 -357 -358 -359 -360 -361 -362 -363 -364 -365 -366 -367 -368 -369 -370 -371 -372 -373 -374 -375 -376 -377 -378 -379 -380 -381 -382 -383 -384 -385 -386 -387 -388 -389 -390 -391 -392 -393 -394 -395 -396 -397 -398 -399 -400 -401 -402 -403 -404 -405 -406 -407 -408 -409 -410 -411 -412 -413 -414 -415 -416 -417 -418 -419 -420 -421 -422 -423 -424 -425 -426 -427 -428 -429 -430 -431 -432 -433 -434 -435 -436 -437 -438 -439 -440 -441 -442 -443 -444 -445 -446 -447 -448 -449 -450 -451 -452 -453 -454 -455 -456 -457 -458 -459 -460 -461 -462 -463 -464 -465 -466 -467 -468 -469 -470 -471 -472 -473 -474 -475 -476 -477 -478 -479 -480 -481 -482 -483 -484 -485 -486 -487 -488 -489 -490 -491 -492 -493 -494 -495 -496 -497 -498 -499 -500 -501 -502 -503 -504 -505 -506 -507 -508 -509 -510 -511 -512 -513 -514 -515 -516 -517 -518 -519 -520 -521 -522 -523 -524 -525 -526 -527 -528 -529 -530 -531 -532 -533 -534 -535 -536 -537 -538 -539 -540 -541 -542 -543 -544 -545 -546 -547 -548 -549 -550 -551 -552 -553 -554 -555 -556 -557 -558 -559 -560 -561 -562 -563 -564 -565 -566 -567 -568 -569 -570 -571 -572 -573 -574 -575 -576 -577 -578 -579 -580 -581 -582 -583 -584 -585 -586 -587 -588 -589 -590 -591 -592 -593 -594 -595 -596 -597 -598 -599 -600 -601 -602 -603 -604 -605 -606 -607 -608 -609 -610 -611 -612 -613 -614 -615 -616 -617 -618 -619 -620 -621 -622 -623 -624 -625 -626 -627 -628 -629 -630 -631 -632 -633 -634 -635 -636 -637 -638 -639 -640 -641 -642 -643 -644 -645 -646 -647 -648 -649 -650 -651 -652 -653 -654 -655 -656 -657 -658 -659 -660 -661 -662 -663 -664 -665 -666 -667 -668 -669 -670 -671 -672 -673 -674 -675 -676 -677 -678 -679 -680 -681 -682 -683 -684 -685 -686 -687 -688 -689 -690 -691 -692 -693 -694 -695 -696 -697 -698 -699 -700 -701 -702 -703 -704 -705 -706 -707 -708 -709 -710 -711 -712 -713 -714 -715 -716 -717 -718 -719 -720 -721 -722 -723 -724 -725 -726 -727 -728 -729 -730 -731 -732 -733 -734 -735 -736 -737 -738 -739 -740 -741 -742 -743 -744 -745 -746 -747 -748 -749 -750 -751 -752 -753 -754 -755 -756 -757 -758 -759 -760 -761 -762 -763 -764 -765 -766 -767 -768 -769 -770 -771 -772 -773 -774 -775 -776 -777 -778 -779 -780 -781 -782 -783 -784 -785 -786 -787 -788 -789 -790 -791 -792 -793 -794 -795 -796 -797 -798 -799 -800 -801 -802 -803 -804 -805 -806 -807 -808 -809 -810 -811 -812 -813 -814 -815 -816 -817 -818 -819 -820 -821 -822 -823 -824 -825 -826 -827 -828 -829 -830 -831 -832 -833 -834 -835 -836 -837 -838 -839 -840 -841 -842 -843 -844 -845 -846 -847 -848 -849 -850 -851 -852 -853 -854 -855 -856 -857 -858 -859 -860 -861 -862 -863 -864 -865 -866 -867 -868 -869 -870 -871 -872 -873 -874 -875 -876 -877 -878 -879 -880 -881 -882 -883 -884 -885 -886 -887 -888 -889 -890 -891 -892 -893 -894 -895 -896 -897 -898 -899 -900 -901 -902 -903 -904 -905 -906 -907 -908 -909 -910 -911 -912 -913 -914 -915 -916 -917 -918 -919 -920 -921 -922 -923 -924 -925 -926 -927 -928 -929 -930 -931 -932 -933 -934 -935 -936 -937 -938 -939 -940 -941 -942 -943 -944 -945 -946 -947 -948 -949 -950 -951 -952 -953 -954 -955 -956 -957 -958 -959 -960 -961 -962 -963 -964 -965 -966 -967 -968 -969 -970 -971 -972 -973 -974 -975 -976 -977 -978 -979 -980 -981 -982 -983 -984 -985 -986 -987 -988 -989 -990 -991 -992 -993 -994 -995 -996 -997 -998 -999 -1000 -1001 -1002 -1003 -1004 -1005 -1006 -1007 -1008 -1009 -1010 -1011 -1012 -1013 -1014 -1015 -1016 -1017 -1018 -1019 -1020 -1021 -1022 -1023 -1024 -1025 -1026 -1027 -1028 -1029 -1030 -1031 -1032 -1033 -1034 -1035 -1036 -1037 -1038 -1039 -1040 -1041 -1042 -1043 -1044 -1045 -1046 -1047 -1048 -1049 -1050 -1051 -1052 -1053 -1054 -1055 -1056 -1057 -1058 -1059 -1060 -1061 -1062 -1063 -1064 -1065 -1066 -1067 -1068 -1069 -1070 -1071 -1072 -1073 -1074 -1075 -1076 -1077 -1078 -1079 -1080 -1081 -1082 -1083 -1084 -1085 -1086 -1087 -1088 -1089 -1090 -1091 -1092 -1093 -1094 -1095 -1096 -1097 -1098 -1099 -1100 -1101 -1102 -1103 -1104 -1105 -1106 -1107 -1108 -1109 -1110 -1111 -1112 -1113 -1114 -1115 -1116 -1117 -1118 -1119 -1120 -1121 -1122 -1123 -1124 -1125 -1126 -1127 -1128 -1129 -1130 -1131 -1132 -1133 -1134 -1135 -1136 -1137 -1138 -1139 -1140 -1141 -1142 -1143 -1144 -1145 -1146 -1147 -1148 -1149 -1150 -1151 -1152 -1153 -1154 -1155 -1156 -1157 -1158 -1159 -1160 -1161 -1162 -1163 -1164 -1165 -1166 -1167 -1168 -1169 -1170 -1171 -1172 -1173 -1174 -1175 -1176 -1177 -1178 -1179 -1180 -1181 -1182 -1183 -1184 -1185 -1186 -1187 -1188 -1189 -1190 -1191 -1192 -1193 -1194 -1195 -1196 -1197 -1198 -1199 -1200 -1201 -1202 -1203 -1204 -1205 -1206 -1207 -1208 -1209 -1210 -1211 -1212 -1213 -1214 -1215 -1216 -1217 -1218 -1219 -1220 -1221 -1222 -1223 -1224 -1225 -1226 -1227 -1228 -1229 -1230 -1231 -1232 -1233 -1234 -1235 -1236 -1237 -1238 -1239 -1240 -1241 -1242 -1243 -1244 -1245 -1246 -1247 -1248 -1249 -1250 -1251 -1252 -1253 -1254 -1255 -1256 -1257 -1258 -1259 -1260 -1261 -1262 -1263 -1264 -1265 -1266 -1267 -1268 -1269 -1270 -1271 -1272 -1273 -1274 -1275 -1276 -1277 -1278 -1279 -1280 -1281 -1282 -1283 -1284 -1285 -1286 -1287 -1288 -1289 -1290 -1291 -1292 -1293 -1294 -1295 -1296 -1297 -1298 -1299 -1300 -1301 -1302 -1303 -1304 -1305 -1306 -1307 -1308 -1309 -1310 -1311 -1312 -1313 -1314 -1315 -1316 -1317 -1318 -1319 -1320 -1321 -1322 -1323 -1324 -1325 -1326 -1327 -1328 -1329 -1330 -1331 -1332 -1333 -1334 -1335 -1336 -1337 -1338 -1339 -1340 -1341 -1342 -1343 -1344 -1345 -1346 -1347 -1348 -1349 -1350 -1351 -1352 -1353 -1354 -1355 -1356 -1357 -1358 -1359 -1360 -1361 -1362 -1363 -1364 -1365 -1366 -1367 -1368 -1369 -1370 -1371 -1372 -1373 -1374 -1375 -1376 -1377 -1378 -1379 -1380 -1381 -1382 -1383 -1384 -1385 -1386 -1387 -1388 -1389 -1390 -1391 -1392 -1393 -1394 -1395 -1396 -1397 -1398 -1399 -1400 -1401 -1402 -1403 -1404 -1405 -1406 -1407 -1408 -1409 -1410 -1411 -1412 -1413 -1414 -1415 -1416 -1417 -1418 -1419 -1420 -1421 -1422 -1423 -1424 -1425 -1426 -1427 -1428 -1429 -1430 -1431 -1432 -1433 -1434 -1435 -1436 -1437 -1438 -1439 -1440 -1441 -1442 -1443 -1444 -1445 -1446 -1447 -1448 -1449 -1450 -1451 -1452 -1453 -1454 -1455 -1456 -1457 -1458 -1459 -1460 -1461 -1462 -1463 -1464 -1465 -1466 -1467 -1468 -1469 -1470 -1471 -1472 -1473 -1474 -1475 -1476 -1477 -1478 -1479 -1480 -1481 -1482 -1483 -1484 -1485 -1486 -1487 -1488 -1489 -1490 -1491 -1492 -1493 -1494 -1495 -1496 -1497 -1498 -1499 -1500 -1501 -1502 -1503 -1504 -1505 -1506 -1507 -1508 -1509 -1510 -1511 -1512 -1513 -1514 -1515 -1516 -1517 -1518 -1519 -1520 -1521 -1522 -1523 -1524 -1525 -1526 -1527 -1528 -1529 -1530 -1531 -1532 -1533 -1534 -1535 -1536 -1537 -1538 -1539 -1540 -1541 -1542 -1543 -1544 -1545 -1546 -1547 -1548 -1549 -1550 -1551 -1552 -1553 -1554 -1555 -1556 -1557 -1558 -1559 -1560 -1561 -1562 -1563 -1564 -1565 -1566 -1567 -1568 -1569 -1570 -1571 -1572 -1573 -1574 -1575 -1576 -1577 -1578 -1579 -1580 -1581 -1582 -1583 -1584 -1585 -1586 -1587 -1588 -1589 -1590 -1591 -1592 -1593 -1594 -1595 -1596 -1597 -1598 -1599 -1600 -1601 -1602 -1603 -1604 -1605 -1606 -1607 -1608 -1609 -1610 -1611 -1612 -1613 -1614 -1615 -1616 -1617 -1618 -1619 -1620 -1621 -1622 -1623 -1624 -1625 -1626 -1627 -1628 -1629 -1630 -1631 -1632 -1633 -1634 -1635 -1636 -1637 -1638 -1639 -1640 -1641 -1642 -1643 -1644 -1645 -1646 -1647 -1648 -1649 -1650 -1651 -1652 -1653 -1654 -1655 -1656 -1657 -1658 -1659 -1660 -1661 -1662 -1663 -1664 -1665 -1666 -1667 -1668 -1669 -1670 -1671 -1672 -1673 -1674 -1675 -1676 -1677 -1678 -1679 -1680 -1681 -1682 -1683 -1684 -1685 -1686 -1687 -1688 -1689 -1690 -1691 -1692 -1693 -1694 -1695 -1696 -1697 -1698 -1699 -1700 -1701 -1702 -1703 -1704 -1705 -1706 -1707 -1708 -1709 -1710 -1711 -1712 -1713 -1714 -1715 -1716 -1717 -1718 -1719 -1720 -1721 -1722 -1723 -1724 -1725 -1726 -1727 -1728 -1729 -1730 -1731 -1732 -1733 -1734 -1735 -1736 -1737 -1738 -1739 -1740 -1741 -1742 -1743 -1744 -1745 -1746 -1747 -1748 -1749 -1750 -1751 -1752 -1753 -1754 -1755 -1756 -1757 -1758 -1759 -1760 -1761 -1762 -1763 -1764 -1765 -1766 -1767 -1768 -1769 -1770 -1771 -1772 -1773 -1774 -1775 -1776 -1777 -1778 -1779 -1780 -1781 -1782 -1783 -1784 -1785 -1786 -1787 -1788 -1789 -1790 -1791 -1792 -1793 -1794 -1795 -1796 -1797 -1798 -1799 -1800 -1801 -1802 -1803 -1804 -1805 -1806 -1807 -1808 -1809 -1810 -1811 -1812 -1813 -1814 -1815 -1816 -1817 -1818 -1819 -1820 -1821 -1822 -1823 -1824 -1825 -1826 -1827 -1828 -1829 -1830 -1831 -1832 -1833 -1834 -1835 -1836 -1837 -1838 -1839 -1840 -1841 -1842 -1843 -1844 -1845 -1846 -1847 -1848 -1849 -1850 -1851 -1852 -1853 -1854 -1855 -1856 -1857 -1858 -1859 -1860 -1861 -1862 -1863 -1864 -1865 -1866 -1867 -1868 -1869 -1870 -1871 -1872 -1873 -1874 -1875 -1876 -1877 -1878 -1879 -1880 -1881 -1882 -1883 -1884 -1885 -1886 -1887 -1888 -1889 -1890 -1891 -1892 -1893 -1894 -1895 -1896 -1897 -1898 -1899 -1900 -1901 -1902 -1903 -1904 -1905 -1906 -1907 -1908 -1909 -1910 -1911 -1912 -1913 -1914 -1915 -1916 -1917 -1918 -1919 -1920 -1921 -1922 -1923 -1924 -1925 -1926 -1927 -1928 -1929 -1930 -1931 -1932 -1933 -1934 -1935 -1936 -1937 -1938 -1939 -1940 -1941 -1942 -1943 -1944 -1945 -1946 -1947 -1948 -1949 -1950 -1951 -1952 -1953 -1954 -1955 -1956 -1957 -1958 -1959 -1960 -1961 -1962 -1963 -1964 -1965 -1966 -1967 -1968 -1969 -1970 -1971 -1972 -1973 -1974 -1975 -1976 -1977 -1978 -1979 -1980 -1981 -1982 -1983 -1984 -1985 -1986 -1987 -1988 -1989 -1990 -1991 -1992 -1993 -1994 -1995 -1996 -1997 -1998 -1999 -2000 -2001 -2002 -2003 -2004 -2005 -2006 -2007 -2008 -2009 -2010 -2011 -2012 -2013 -2014 -2015 -2016 -2017 -2018 -2019 -2020 -2021 -2022 -2023 -2024 -2025 -2026 -2027 -2028 -2029 -2030 -2031 -2032 -2033 -2034 -2035 -2036 -2037 -2038 -2039 -2040 -2041 -2042 -2043 -2044 -2045 -2046 -2047 -2048 -2049 -2050 -2051 -2052 -2053 -2054 -2055 -2056 -2057 -2058 -2059 -2060 -2061 -2062 -2063 -2064 -2065 -2066 -2067 -2068 -2069 -2070 -2071 -2072 -2073 -2074 -2075 -2076 -2077 -2078 -2079 -2080 -2081 -2082 -2083 -2084 -2085 -2086 -2087 -2088 -2089 -2090 -2091 -2092 -2093 -2094 -2095 -2096 -2097 -2098 -2099 -2100 -2101 -2102 -2103 -2104 -2105 -2106 -2107 -2108 -2109 -2110 -2111 -2112 -2113 -2114 -2115 -2116 -2117 -2118 -2119 -2120 -2121 -2122 -2123 -2124 -2125 -2126 -2127 -2128 -2129 -2130 -2131 -2132 -2133 -2134 -2135 -2136 -2137 -2138 -2139 -2140 -2141 -2142 -2143 -2144 -2145 -2146 -2147 -2148 -2149 -2150 -2151 -2152 -2153 -2154 -2155 -2156 -2157 -2158 -2159 -2160 -2161 -2162 -2163 -2164 -2165 -2166 -2167 -2168 -2169 -2170 -2171 -2172 -2173 -2174 -2175 -2176 -2177 -2178 -2179 -2180 -2181 -2182 -2183 -2184 -2185 -2186 -2187 -2188 -2189 -2190 -2191 -2192 -2193 -2194 -2195 -2196 -2197 -2198 -2199 -2200 -2201 -2202 -2203 -2204 -2205 -2206 -2207 -2208 -2209 -2210 -2211 -2212 -2213 -2214 -2215 -2216 -2217 -2218 -2219 -2220 -2221 -2222 -2223 -2224 -2225 -2226 -2227 -2228 -2229 -2230 -2231 -2232 -2233 -2234 -2235 -2236 -2237 -2238 -2239 -2240 -2241 -2242 -2243 -2244 -2245 -2246 -2247 -2248 -2249 -2250 -2251 -2252 -2253 -2254 -2255 -2256 -2257 -2258 -2259 -2260 -2261 -2262 -2263 -2264 -2265 -2266 -2267 -2268 -2269 -2270 -2271 -2272 -2273 -2274 -2275 -2276 -2277 -2278 -2279 -2280 -2281 -2282 -2283 -2284 -2285 -2286 -2287 -2288 -2289 -2290 -2291 -2292 -2293 -2294 -2295 -2296 -2297 -2298 -2299 -2300 -2301 -2302 -2303 -2304 -2305 -2306 -2307 -2308 -2309 -2310 -2311 -2312 -2313 -2314 -2315 -2316 -2317 -2318 -2319 -2320 -2321 -2322 -2323 -2324 -2325 -2326 -2327 -2328 -2329 -2330 -2331 -2332 -2333 -2334 -2335 -2336 -2337 -2338 -2339 -2340 -2341 -2342 -2343 -2344 -2345 -2346 -2347 -2348 -2349 -2350 -2351 -2352 -2353 -2354 -2355 -2356 -2357 -2358 -2359 -2360 -2361 -2362 -2363 -2364 -2365 -2366 -2367 -2368 -2369 -2370 -2371 -2372 -2373 -2374 -2375 -2376 -2377 -2378 -2379 -2380 -2381 -2382 -2383 -2384 -2385 -2386 -2387 -2388 -2389 -2390 -2391 -2392 -2393 -2394 -2395 -2396 -2397 -2398 -2399 -2400 -2401 -2402 -2403 -2404 -2405 -2406 -2407 -2408 -2409 -2410 -2411 -2412 -2413 -2414 -2415 -2416 -2417 -2418 -2419 -2420 -2421 -2422 -2423 -2424 -2425 -2426 -2427 -2428 -2429 -2430 -2431 -2432 -2433 -2434 -2435 -2436 -2437 -2438 -2439 -2440 -2441 -2442 -2443 -2444 -2445 -2446 -2447 -2448 -2449 -2450 -2451 -2452 -2453 -2454 -2455 -2456 -2457 -2458 -2459 -2460 -2461 -2462 -2463 -2464 -2465 -2466 -2467 -2468 -2469 -2470 -2471 -2472 -2473 -2474 -2475 -2476 -2477 -2478 -2479 -2480 -2481 -2482 -2483 -2484 -2485 -2486 -2487 -2488 -2489 -2490 -2491 -2492 -2493 -2494 -2495 -2496 -2497 -2498 -2499 -2500 -2501 -2502 -2503 -2504 -2505 -2506 -2507 -2508 -2509 -2510 -2511 -2512 -2513 -2514 -2515 -2516 -2517 -2518 -2519 -2520 -2521 -2522 -2523 -2524 -2525 -2526 -2527 -2528 -2529 -2530 -2531 -2532 -2533 -2534 -2535 -2536 -2537 -2538 -2539 -2540 -2541 -2542 -2543 -2544 -2545 -2546 -2547 -2548 -2549 -2550 -2551 -2552 -2553 -2554 -2555 -2556 -2557 -2558 -2559 -2560 -2561 -2562 -2563 -2564 -2565 -2566 -2567 -2568 -2569 -2570 -2571 -2572 -2573 -2574 -2575 -2576 -2577 -2578 -2579 -2580 -2581 -2582 -2583 -2584 -2585 -2586 -2587 -2588 -2589 -2590 -2591 -2592 -2593 -2594 -2595 -2596 -2597 -2598 -2599 -2600 -2601 -2602 -2603 -2604 -2605 -2606 -2607 -2608 -2609 -2610 -2611 -2612 -2613 -2614 -2615 -2616 -2617 -2618 -2619 -2620 -2621 -2622 -2623 -2624 -2625 -2626 -2627 -2628 -2629 -2630 -2631 -2632 -2633 -2634 -2635 -2636 -2637 -2638 -2639 -2640 -2641 -2642 -2643 -2644 -2645 -2646 -2647 -2648 -2649 -2650 -2651 -2652 -2653 -2654 -2655 -2656 -2657 -2658 -2659 -2660 -2661 -2662 -2663 -2664 -2665 -2666 -2667 -2668 -2669 -2670 -2671 -2672 -2673 -2674 -2675 -2676 -2677 -2678 -2679 -2680 -2681 -2682 -2683 -2684 -2685 -2686 -2687 -2688 -2689 -2690 -2691 -2692 -2693 -2694 -2695 -2696 -2697 -2698 -2699 -2700 -2701 -2702 -2703 -2704 -2705 -2706 -2707 -2708 -2709 -2710 -2711 -2712 -2713 -2714 -2715 -2716 -2717 -2718 -2719 -2720 -2721 -2722 -2723 -2724 -2725 -2726 -2727 -2728 -2729 -2730 -2731 -2732 -2733 -2734 -2735 -2736 -2737 -2738 -2739 -2740 -2741 -2742 -2743 -2744 -2745 -2746 -2747 -2748 -2749 -2750 -2751 -2752 -2753 -2754 -2755 -2756 -2757 -2758 -2759 -2760 -2761 -2762 -2763 -2764 -2765 -2766 -2767 -2768 -2769 -2770 -2771 -2772 -2773 -2774 -2775 -2776 -2777 -2778 -2779 -2780 -2781 -2782 -2783 -2784 -2785 -2786 -2787 -2788 -2789 -2790 -2791 -2792 -2793 -2794 -2795 -2796 -2797 -2798 -2799 -2800 -2801 -2802 -2803 -2804 -2805 -2806 -2807 -2808 -2809 -2810 -2811 -2812 -2813 -2814 -2815 -2816 -2817 -2818 -2819 -2820 -2821 -2822 -2823 -2824 -2825 -2826 -2827 -2828 -2829 -2830 -2831 -2832 -2833 -2834 -2835 -2836 -2837 -2838 -2839 -2840 -2841 -2842 -2843 -2844 -2845 -2846 -2847 -2848 -2849 -2850 -2851 -2852 -2853 -2854 -2855 -2856 -2857 -2858 -2859 -2860 -2861 -2862 -2863 -2864 -2865 -2866 -2867 -2868 -2869 -2870 -2871 -2872 -2873 -2874 -2875 -2876 -2877 -2878 -2879 -2880 -2881 -2882 -2883 -2884 -2885 -2886 -2887 -2888 -2889 -2890 -2891 -2892 -2893 -2894 -2895 -2896 -2897 -2898 -2899 -2900 -2901 -2902 -2903 -2904 -2905 -2906 -2907 -2908 -2909 -2910 -2911 -2912 -2913 -2914 -2915 -2916 -2917 -2918 -2919 -2920 -2921 -2922 -2923 -2924 -2925 -2926 -2927 -2928 -2929 -2930 -2931 -2932 -2933 -2934 -2935 -2936 -2937 -2938 -2939 -2940 -2941 -2942 -2943 -2944 -2945 -2946 -2947 -2948 -2949 -2950 -2951 -2952 -2953 -2954 -2955 -2956 -2957 -2958 -2959 -2960 -2961 -2962 -2963 -2964 -2965 -2966 -2967 -2968 -2969 -2970 -2971 -2972 -2973 -2974 -2975 -2976 -2977 -2978 -2979 -2980 -2981 -2982 -2983 -2984 -2985 -2986 -2987 -2988 -2989 -2990 -2991 -2992 -2993 -2994 -2995 -2996 -2997 -2998 -2999 -3000 -3001 -3002 -3003 -3004 -3005 -3006 -3007 -3008 -3009 -3010 -3011 -3012 -3013 -3014 -3015 -3016 -3017 -3018 -3019 -3020 -3021 -3022 -3023 -3024 -3025 -3026 -3027 -3028 -3029 -3030 -3031 -3032 -3033 -3034 -3035 -3036 -3037 -3038 -3039 -3040 -3041 -3042 -3043 -3044 -3045 -3046 -3047 -3048 -3049 -3050 -3051 -3052 -3053 -3054 -3055 -3056 -3057 -3058 -3059 -3060 -3061 -3062 -3063 -3064 -3065 -3066 -3067 -3068 -3069 -3070 -3071 -3072 -3073 -3074 -3075 -3076 -3077 -3078 -3079 -3080 -3081 -3082 -3083 -3084 -3085 -3086 -3087 -3088 -3089 -3090 -3091 -3092 -3093 -3094 -3095 -3096 -3097 -3098 -3099 -3100 -3101 -3102 -3103 -3104 -3105 -3106 -3107 -3108 -3109 -3110 -3111 -3112 -3113 -3114 -3115 -3116 -3117 -3118 -3119 -3120 -3121 -3122 -3123 -3124 -3125 -3126 -3127 -3128 -3129 -3130 -3131 -3132 -3133 -3134 -3135 -3136 -3137 -3138 -3139 -3140 -3141 -3142 -3143 -3144 -3145 -3146 -3147 -3148 -3149 -3150 -3151 -3152 -3153 -3154 -3155 -3156 -3157 -3158 -3159 -3160 -3161 -3162 -3163 -3164 -3165 -3166 -3167 -3168 -3169 -3170 -3171 -3172 -3173 -3174 -3175 -3176 -3177 -3178 -3179 -3180 -3181 -3182 -3183 -3184 -3185 -3186 -3187 -3188 -3189 -3190 -3191 -3192 -3193 -3194 -3195 -3196 -3197 -3198 -3199 -3200 -3201 -3202 -3203 -3204 -3205 -3206 -3207 -3208 -3209 -3210 -3211 -3212 -3213 -3214 -3215 -3216 -3217 -3218 -3219 -3220 -3221 -3222 -3223 -3224 -3225 -3226 -3227 -3228 -3229 -3230 -3231 -3232 -3233 -3234 -3235 -3236 -3237 -3238 -3239 -3240 -3241 -3242 -3243 -3244 -3245 -3246 -3247 -3248 -3249 -3250 -3251 -3252 -3253 -3254 -3255 -3256 -3257 -3258 -3259 -3260 -3261 -3262 -3263 -3264 -3265 -3266 -3267 -3268 -3269 -3270 -3271 -3272 -3273 -3274 -3275 -3276 -3277 -3278 -3279 -3280 -3281 -3282 -3283 -3284 -3285 -3286 -3287 -3288 -3289 -3290 -3291 -3292 -3293 -3294 -3295 -3296 -3297 -3298 -3299 -3300 -3301 -3302 -3303 -3304 -3305 -3306 -3307 -3308 -3309 -3310 -3311 -3312 -3313 -3314 -3315 -3316 -3317 -3318 -3319 -3320 -3321 -3322 -3323 -3324 -3325 -3326 -3327 -3328 -3329 -3330 -3331 -3332 -3333 -3334 -3335 -3336 -3337 -3338 -3339 -3340 -3341 -3342 -3343 -3344 -3345 -3346 -3347 -3348 -3349 -3350 -3351 -3352 -3353 -3354 -3355 -3356 -3357 -3358 -3359 -3360 -3361 -3362 -3363 -3364 -3365 -3366 -3367 -3368 -3369 -3370 -3371 -3372 -3373 -3374 -3375 -3376 -3377 -3378 -3379 -3380 -3381 -3382 -3383 -3384 -3385 -3386 -3387 -3388 -3389 -3390 -3391 -3392 -3393 -3394 -3395 -3396 -3397 -3398 -3399 -3400 -3401 -3402 -3403 -3404 -3405 -3406 -3407 -3408 -3409 -3410 -3411 -3412 -3413 -3414 -3415 -3416 -3417 -3418 -3419 -3420 -3421 -3422 -3423 -3424 -3425 -3426 -3427 -3428 -3429 -3430 -3431 -3432 -3433 -3434 -3435 -3436 -3437 -3438 -3439 -3440 -3441 -3442 -3443 -3444 -3445 -3446 -3447 -3448 -3449 -3450 -3451 -3452 -3453 -3454 -3455 -3456 -3457 -3458 -3459 -3460 -3461 -3462 -3463 -3464 -3465 -3466 -3467 -3468 -3469 -3470 -3471 -3472 -3473 -3474 -3475 -3476 -3477 -3478 -3479 -3480 -3481 -3482 -3483 -3484 -3485 -3486 -3487 -3488 -3489 -3490 -3491 -3492 -3493 -3494 -3495 -3496 -3497 -3498 -3499 -3500 -3501 -3502 -3503 -3504 -3505 -3506 -3507 -3508 -3509 -3510 -3511 -3512 -3513 -3514 -3515 -3516 -3517 -3518 -3519 -3520 -3521 -3522 -3523 -3524 -3525 -3526 -3527 -3528 -3529 -3530 -3531 -3532 -3533 -3534 -3535 -3536 -3537 -3538 -3539 -3540 -3541 -3542 -3543 -3544 -3545 -3546 -3547 -3548 -3549 -3550 -3551 -3552 -3553 -3554 -3555 -3556 -3557 -3558 -3559 -3560 -3561 -3562 -3563 -3564 -3565 -3566 -3567 -3568 -3569 -3570 -3571 -3572 -3573 -3574 -3575 -3576 -3577 -3578 -3579 -3580 -3581 -3582 -3583 -3584 -3585 -3586 -3587 -3588 -3589 -3590 -3591 -3592 -3593 -3594 -3595 -3596 -3597 -3598 -3599 -3600 -3601 -3602 -3603 -3604 -3605 -3606 -3607 -3608 -3609 -3610 -3611 -3612 -3613 -3614 -3615 -3616 -3617 -3618 -3619 -3620 -3621 -3622 -3623 -3624 -3625 -3626 -3627 -3628 -3629 -3630 -3631 -3632 -3633 -3634 -3635 -3636 -3637 -3638 -3639 -3640 -3641 -3642 -3643 -3644 -3645 -3646 -3647 -3648 -3649 -3650 -3651 -3652 -3653 -3654 -3655 -3656 -3657 -3658 -3659 -3660 -3661 -3662 -3663 -3664 -3665 -3666 -3667 -3668 -3669 -3670 -3671 -3672 -3673 -3674 -3675 -3676 -3677 -3678 -3679 -3680 -3681 -3682 -3683 -3684 -3685 -3686 -3687 -3688 -3689 -3690 -3691 -3692 -3693 -3694 -3695 -3696 -3697 -3698 -3699 -3700 -3701 -3702 -3703 -3704 -3705 -3706 -3707 -3708 -3709 -3710 -3711 -3712 -3713 -3714 -3715 -3716 -3717 -3718 -3719 -3720 -3721 -3722 -3723 -3724 -3725 -3726 -3727 -3728 -3729 -3730 -3731 -3732 -3733 -3734 -3735 -3736 -3737 -3738 -3739 -3740 -3741 -3742 -3743 -3744 -3745 -3746 -3747 -3748 -3749 -3750 -3751 -3752 -3753 -3754 -3755 -3756 -3757 -3758 -3759 -3760 -3761 -3762 -3763 -3764 -3765 -3766 -3767 -3768 -3769 -3770 -3771 -3772 -3773 -3774 -3775 -3776 -3777 -3778 -3779 -3780 -3781 -3782 -3783 -3784 -3785 -3786 -3787 -3788 -3789 -3790 -3791 -3792 -3793 -3794 -3795 -3796 -3797 -3798 -3799 -3800 -3801 -3802 -3803 -3804 -3805 -3806 -3807 -3808 -3809 -3810 -3811 -3812 -3813 -3814 -3815 -3816 -3817 -3818 -3819 -3820 -3821 -3822 -3823 -3824 -3825 -3826 -3827 -3828 -3829 -3830 -3831 -3832 -3833 -3834 -3835 -3836 -3837 -3838 -3839 -3840 -3841 -3842 -3843 -3844 -3845 -3846 -3847 -3848 -3849 -3850 -3851 -3852 -3853 -3854 -3855 -3856 -3857 -3858 -3859 -3860 -3861 -3862 -3863 -3864 -3865 -3866 -3867 -3868 -3869 -3870 -3871 -3872 -3873 -3874 -3875 -3876 -3877 -3878 -3879 -3880 -3881 -3882 -3883 -3884 -3885 -3886 -3887 -3888 -3889 -3890 -3891 -3892 -3893 -3894 -3895 -3896 -3897 -3898 -3899 -3900 -3901 -3902 -3903 -3904 -3905 -3906 -3907 -3908 -3909 -3910 -3911 -3912 -3913 -3914 -3915 -3916 -3917 -3918 -3919 -3920 -3921 -3922 -3923 -3924 -3925 -3926 -3927 -3928 -3929 -3930 -3931 -3932 -3933 -3934 -3935 -3936 -3937 -3938 -3939 -3940 -3941 -3942 -3943 -3944 -3945 -3946 -3947 -3948 -3949 -3950 -3951 -3952 -3953 -3954 -3955 -3956 -3957 -3958 -3959 -3960 -3961 -3962 -3963 -3964 -3965 -3966 -3967 -3968 -3969 -3970 -3971 -3972 -3973 -3974 -3975 -3976 -3977 -3978 -3979 -3980 -3981 -3982 -3983 -3984 -3985 -3986 -3987 -3988 -3989 -3990 -3991 -3992 -3993 -3994 -3995 -3996 -3997 -3998 -3999 -4000 -4001 -4002 -4003 -4004 -4005 -4006 -4007 -4008 -4009 -4010 -4011 -4012 -4013 -4014 -4015 -4016 -4017 -4018 -4019 -4020 -4021 -4022 -4023 -4024 -4025 -4026 -4027 -4028 -4029 -4030 -4031 -4032 -4033 -4034 -4035 -4036 -4037 -4038 -4039 -4040 -4041 -4042 -4043 -4044 -4045 -4046 -4047 -4048 -4049 -4050 -4051 -4052 -4053 -4054 -4055 -4056 -4057 -4058 -4059 -4060 -4061 -4062 -4063 -4064 -4065 -4066 -4067 -4068 -4069 -4070 -4071 -4072 -4073 -4074 -4075 -4076 -4077 -4078 -4079 -4080 -4081 -4082 -4083 -4084 -4085 -4086 -4087 -4088 -4089 -4090 -4091 -4092 -4093 -4094 -4095 -4096 -4097 -4098 -4099 -4100 -4101 -4102 -4103 -4104 -4105 -4106 -4107 -4108 -4109 -4110 -4111 -4112 -4113 -4114 -4115 -4116 -4117 -4118 -4119 -4120 -4121 -4122 -4123 -4124 -4125 -4126 -4127 -4128 -4129 -4130 -4131 -4132 -4133 -4134 -4135 -4136 -4137 -4138 -4139 -4140 -4141 -4142 -4143 -4144 -4145 -4146 -4147 -4148 -4149 -4150 -4151 -4152 -4153 -4154 -4155 -4156 -4157 -4158 -4159 -4160 -4161 -4162 -4163 -4164 -4165 -4166 -4167 -4168 -4169 -4170 -4171 -4172 -4173 -4174 -4175 -4176 -4177 -4178 -4179 -4180 -4181 -4182 -4183 -4184 -4185 -4186 -4187 -4188 -4189 -4190 -4191 -4192 -4193 -4194 -4195 -4196 -4197 -4198 -4199 -4200 -4201 -4202 -4203 -4204 -4205 -4206 -4207 -4208 -4209 -4210 -4211 -4212 -4213 -4214 -4215 -4216 -4217 -4218 -4219 -4220 -4221 -4222 -4223 -4224 -4225 -4226 -4227 -4228 -4229 -4230 -4231 -4232 -4233 -4234 -4235 -4236 -4237 -4238 -4239 -4240 -4241 -4242 -4243 -4244 -4245 -4246 -4247 -4248 -4249 -4250 -4251 -4252 -4253 -4254 -4255 -4256 -4257 -4258 -4259 -4260 -4261 -4262 -4263 -4264 -4265 -4266 -4267 -4268 -4269 -4270 -4271 -4272 -4273 -4274 -4275 -4276 -4277 -4278 -4279 -4280 -4281 -4282 -4283 -4284 -4285 -4286 -4287 -4288 -4289 -4290 -4291 -4292 -4293 -4294 -4295 -4296 -4297 -4298 -4299 -4300 -4301 -4302 -4303 -4304 -4305 -4306 -4307 -4308 -4309 -4310 -4311 -4312 -4313 -4314 -4315 -4316 -4317 -4318 -4319 -4320 -4321 -4322 -4323 -4324 -4325 -4326 -4327 -4328 -4329 -4330 -4331 -4332 -4333 -4334 -4335 -4336 -4337 -4338 -4339 -4340 -4341 -4342 -4343 -4344 -4345 -4346 -4347 -4348 -4349 -4350 -4351 -4352 -4353 -4354 -4355 -4356 -4357 -4358 -4359 -4360 -4361 -4362 -4363 -4364 -4365 -4366 -4367 -4368 -4369 -4370 -4371 -4372 -4373 -4374 -4375 -4376 -4377 -4378 -4379 -4380 -4381 -4382 -4383 -4384 -4385 -4386 -4387 -4388 -4389 -4390 -4391 -4392 -4393 -4394 -4395 -4396 -4397 -4398 -4399 -4400 -4401 -4402 -4403 -4404 -4405 -4406 -4407 -4408 -4409 -4410 -4411 -4412 -4413 -4414 -4415 -4416 -4417 -4418 -4419 -4420 -4421 -4422 -4423 -4424 -4425 -4426 -4427 -4428 -4429 -4430 -4431 -4432 -4433 -4434 -4435 -4436 -4437 -4438 -4439 -4440 -4441 -4442 -4443 -4444 -4445 -4446 -4447 -4448 -4449 -4450 -4451 -4452 -4453 -4454 -4455 -4456 -4457 -4458 -4459 -4460 -4461 -4462 -4463 -4464 -4465 -4466 -4467 -4468 -4469 -4470 -4471 -4472 -4473 -4474 -4475 -4476 -4477 -4478 -4479 -4480 -4481 -4482 -4483 -4484 -4485 -4486 -4487 -4488 -4489 -4490 -4491 -4492 -4493 -4494 -4495 -4496 -4497 -4498 -4499 -4500 -4501 -4502 -4503 -4504 -4505 -4506 -4507 -4508 -4509 -4510 -4511 -4512 -4513 -4514 -4515 -4516 -4517 -4518 -4519 -4520 -4521 -4522 -4523 -4524 -4525 -4526 -4527 -4528 -4529 -4530 -4531 -4532 -4533 -4534 -4535 -4536 -4537 -4538 -4539 -4540 -4541 -4542 -4543 -4544 -4545 -4546 -4547 -4548 -4549 -4550 -4551 -4552 -4553 -4554 -4555 -4556 -4557 -4558 -4559 -4560 -4561 -4562 -4563 -4564 -4565 -4566 -4567 -4568 -4569 -4570 -4571 -4572 -4573 -4574 -4575 -4576 -4577 -4578 -4579 -4580 -4581 -4582 -4583 -4584 -4585 -4586 -4587 -4588 -4589 -4590 -4591 -4592 -4593 -4594 -4595 -4596 -4597 -4598 -4599 -4600 -4601 -4602 -4603 -4604 -4605 -4606 -4607 -4608 -4609 -4610 -4611 -4612 -4613 -4614 -4615 -4616 -4617 -4618 -4619 -4620 -4621 -4622 -4623 -4624 -4625 -4626 -4627 -4628 -4629 -4630 -4631 -4632 -4633 -4634 -4635 -4636 -4637 -4638 -4639 -4640 -4641 -4642 -4643 -4644 -4645 -4646 -4647 -4648 -4649 -4650 -4651 -4652 -4653 -4654 -4655 -4656 -4657 -4658 -4659 -4660 -4661 -4662 -4663 -4664 -4665 -4666 -4667 -4668 -4669 -4670 -4671 -4672 -4673 -4674 -4675 -4676 -4677 -4678 -4679 -4680 -4681 -4682 -4683 -4684 -4685 -4686 -4687 -4688 -4689 -4690 -4691 -4692 -4693 -4694 -4695 -4696 -4697 -4698 -4699 -4700 -4701 -4702 -4703 -4704 -4705 -4706 -4707 -4708 -4709 -4710 -4711 -4712 -4713 -4714 -4715 -4716 -4717 -4718 -4719 -4720 -4721 -4722 -4723 -4724 -4725 -4726 -4727 -4728 -4729 -4730 -4731 -4732 -4733 -4734 -4735 -4736 -4737 -4738 -4739 -4740 -4741 -4742 -4743 -4744 -4745 -4746 -4747 -4748 -4749 -4750 -4751 -4752 -4753 -4754 -4755 -4756 -4757 -4758 -4759 -4760 -4761 -4762 -4763 -4764 -4765 -4766 -4767 -4768 -4769 -4770 -4771 -4772 -4773 -4774 -4775 -4776 -4777 -4778 -4779 -4780 -4781 -4782 -4783 -4784 -4785 -4786 -4787 -4788 -4789 -4790 -4791 -4792 -4793 -4794 -4795 -4796 -4797 -4798 -4799 -4800 -4801 -4802 -4803 -4804 -4805 -4806 -4807 -4808 -4809 -4810 -4811 -4812 -4813 -4814 -4815 -4816 -4817 -4818 -4819 -4820 -4821 -4822 -4823 -4824 -4825 -4826 -4827 -4828 -4829 -4830 -4831 -4832 -4833 -4834 -4835 -4836 -4837 -4838 -4839 -4840 -4841 -4842 -4843 -4844 -4845 -4846 -4847 -4848 -4849 -4850 -4851 -4852 -4853 -4854 -4855 -4856 -4857 -4858 -4859 -4860 -4861 -4862 -4863 -4864 -4865 -4866 -4867 -4868 -4869 -4870 -4871 -4872 -4873 -4874 -4875 -4876 -4877 -4878 -4879 -4880 -4881 -4882 -4883 -4884 -4885 -4886 -4887 -4888 -4889 -4890 -4891 -4892 -4893 -4894 -4895 -4896 -4897 -4898 -4899 -4900 -4901 -4902 -4903 -4904 -4905 -4906 -4907 -4908 -4909 -4910 -4911 -4912 -4913 -4914 -4915 -4916 -4917 -4918 -4919 -4920 -4921 -4922 -4923 -4924 -4925 -4926 -4927 -4928 -4929 -4930 -4931 -4932 -4933 -4934 -4935 -4936 -4937 -4938 -4939 -4940 -4941 -4942 -4943 -4944 -4945 -4946 -4947 -4948 -4949 -4950 -4951 -4952 -4953 -4954 -4955 -4956 -4957 -4958 -4959 -4960 -4961 -4962 -4963 -4964 -4965 -4966 -4967 -4968 -4969 -4970 -4971 -4972 -4973 -4974 -4975 -4976 -4977 -4978 -4979 -4980 -4981 -4982 -4983 -4984 -4985 -4986 -4987 -4988 -4989 -4990 -4991 -4992 -4993 -4994 -4995 -4996 -4997 -4998 -4999 -5000 -5001 -5002 -5003 -5004 -5005 -5006 -5007 -5008 -5009 -5010 -5011 -5012 -5013 -5014 -5015 -5016 -5017 -5018 -5019 -5020 -5021 -5022 -5023 -5024 -5025 -5026 -5027 -5028 -5029 -5030 -5031 -5032 -5033 -5034 -5035 -5036 -5037 -5038 -5039 -5040 -5041 -5042 -5043 -5044 -5045 -5046 -5047 -5048 -5049 -5050 -5051 -5052 -5053 -5054 -5055 -5056 -5057 -5058 -5059 -5060 -5061 -5062 -5063 -5064 -5065 -5066 -5067 -5068 -5069 -5070 -5071 -5072 -5073 -5074 -5075 -5076 -5077 -5078 -5079 -5080 -5081 -5082 -5083 -5084 -5085 -5086 -5087 -5088 -5089 -5090 -5091 -5092 -5093 -5094 -5095 -5096 -5097 -5098 -5099 -5100 -5101 -5102 -5103 -5104 -5105 -5106 -5107 -5108 -5109 -5110 -5111 -5112 -5113 -5114 -5115 -5116 -5117 -5118 -5119 -5120 -5121 -5122 -5123 -5124 -5125 -5126 -5127 -5128 -5129 -5130 -5131 -5132 -5133 -5134 -5135 -5136 -5137 -5138 -5139 -5140 -5141 -5142 -5143 -5144 -5145 -5146 -5147 -5148 -5149 -5150 -5151 -5152 -5153 -5154 -5155 -5156 -5157 -5158 -5159 -5160 -5161 -5162 -5163 -5164 -5165 -5166 -5167 -5168 -5169 -5170 -5171 -5172 -5173 -5174 -5175 -5176 -5177 -5178 -5179 -5180 -5181 -5182 -5183 -5184 -5185 -5186 -5187 -5188 -5189 -5190 -5191 -5192 -5193 -5194 -5195 -5196 -5197 -5198 -5199 -5200 -5201 -5202 -5203 -5204 -5205 -5206 -5207 -5208 -5209 -5210 -5211 -5212 -5213 -5214 -5215 -5216 -5217 -5218 -5219 -5220 -5221 -5222 -5223 -5224 -5225 -5226 -5227 -5228 -5229 -5230 -5231 -5232 -5233 -5234 -5235 -5236 -5237 -5238 -5239 -5240 -5241 -5242 -5243 -5244 -5245 -5246 -5247 -5248 -5249 -5250 -5251 -5252 -5253 -5254 -5255 -5256 -5257 -5258 -5259 -5260 -5261 -5262 -5263 -5264 -5265 -5266 -5267 -5268 -5269 -5270 -5271 -5272 -5273 -5274 -5275 -5276 -5277 -5278 -5279 -5280 -5281 -5282 -5283 -5284 -5285 -5286 -5287 -5288 -5289 -5290 -5291 -5292 -5293 -5294 -5295 -5296 -5297 -5298 -5299 -5300 -5301 -5302 -5303 -5304 -5305 -5306 -5307 -5308 -5309 -5310 -5311 -5312 -5313 -5314 -5315 -5316 -5317 -5318 -5319 -5320 -5321 -5322 -5323 -5324 -5325 -5326 -5327 -5328 -5329 -5330 -5331 -5332 -5333 -5334 -5335 -5336 -5337 -5338 -5339 -5340 -5341 -5342 -5343 -5344 -5345 -5346 -5347 -5348 -5349 -5350 -5351 -5352 -5353 -5354 -5355 -5356 -5357 -5358 -5359 -5360 -5361 -5362 -5363 -5364 -5365 -5366 -5367 -5368 -5369 -5370 -5371 -5372 -5373 -5374 -5375 -5376 -5377 -5378 -5379 -5380 -5381 -5382 -5383 -5384 -5385 -5386 -5387 -5388 -5389 -5390 -5391 -5392 -5393 -5394 -5395 -5396 -5397 -5398 -5399 -5400 -5401 -5402 -5403 -5404 -5405 -5406 -5407 -5408 -5409 -5410 -5411 -5412 -5413 -5414 -5415 -5416 -5417 -5418 -5419 -5420 -5421 -5422 -5423 -5424 -5425 -5426 -5427 -5428 -5429 -5430 -5431 -5432 -5433 -5434 -5435 -5436 -5437 -5438 -5439 -5440 -5441 -5442 -5443 -5444 -5445 -5446 -5447 -5448 -5449 -5450 -5451 -5452 -5453 -5454 -5455 -5456 -5457 -5458 -5459 -5460 -5461 -5462 -5463 -5464 -5465 -5466 -5467 -5468 -5469 -5470 -5471 -5472 -5473 -5474 -5475 -5476 -5477 -5478 -5479 -5480 -5481 -5482 -5483 -5484 -5485 -5486 -5487 -5488 -5489 -5490 -5491 -5492 -5493 -5494 -5495 -5496 -5497 -5498 -5499 -5500 -5501 -5502 -5503 -5504 -5505 -5506 -5507 -5508 -5509 -5510 -5511 -5512 -5513 -5514 -5515 -5516 -5517 -5518 -5519 -5520 -5521 -5522 -5523 -5524 -5525 -5526 -5527 -5528 -5529 -5530 -5531 -5532 -5533 -5534 -5535 -5536 -5537 -5538 -5539 -5540 -5541 -5542 -5543 -5544 -5545 -5546 -5547 -5548 -5549 -5550 -5551 -5552 -5553 -5554 -5555 -5556 -5557 -5558 -5559 -5560 -5561 -5562 -5563 -5564 -5565 -5566 -5567 -5568 -5569 -5570 -5571 -5572 -5573 -5574 -5575 -5576 -5577 -5578 -5579 -5580 -5581 -5582 -5583 -5584 -5585 -5586 -5587 -5588 -5589 -5590 -5591 -5592 -5593 -5594 -5595 -5596 -5597 -5598 -5599 -5600 -5601 -5602 -5603 -5604 -5605 -5606 -5607 -5608 -5609 -5610 -5611 -5612 -5613 -5614 -5615 -5616 -5617 -5618 -5619 -5620 -5621 -5622 -5623 -5624 -5625 -5626 -5627 -5628 -5629 -5630 -5631 -5632 -5633 -5634 -5635 -5636 -5637 -5638 -5639 -5640 -5641 -5642 -5643 -5644 -5645 -5646 -5647 -5648 -5649 -5650 -5651 -5652 -5653 -5654 -5655 -5656 -5657 -5658 -5659 -5660 -5661 -5662 -5663 -5664 -5665 -5666 -5667 -5668 -5669 -5670 -5671 -5672 -5673 -5674 -5675 -5676 -5677 -5678 -5679 -5680 -5681 -5682 -5683 -5684 -5685 -5686 -5687 -5688 -5689 -5690 -5691 -5692 -5693 -5694 -5695 -5696 -5697 -5698 -5699 -5700 -5701 -5702 -5703 -5704 -5705 -5706 -5707 -5708 -5709 -5710 -5711 -5712 -5713 -5714 -5715 -5716 -5717 -5718 -5719 -5720 -5721 -5722 -5723 -5724 -5725 -5726 -5727 -5728 -5729 -5730 -5731 -5732 -5733 -5734 -5735 -5736 -5737 -5738 -5739 -5740 -5741 -5742 -5743 -5744 -5745 -5746 -5747 -5748 -5749 -5750 -5751 -5752 -5753 -5754 -5755 -5756 -5757 -5758 -5759 -5760 -5761 -5762 -5763 -5764 -5765 -5766 -5767 -5768 -5769 -5770 -5771 -5772 -5773 -5774 -5775 -5776 -5777 -5778 -5779 -5780 -5781 -5782 -5783 -5784 -5785 -5786 -5787 -5788 -5789 -5790 -5791 -5792 -5793 -5794 -5795 -5796 -5797 -5798 -5799 -5800 -5801 -5802 -5803 -5804 -5805 -5806 -5807 -5808 -5809 -5810 -5811 -5812 -5813 -5814 -5815 -5816 -5817 -5818 -5819 -5820 -5821 -5822 -5823 -5824 -5825 -5826 -5827 -5828 -5829 -5830 -5831 -5832 -5833 -5834 -5835 -5836 -5837 -5838 -5839 -5840 -5841 -5842 -5843 -5844 -5845 -5846 -5847 -5848 -5849 -5850 -5851 -5852 -5853 -5854 -5855 -5856 -5857 -5858 -5859 -5860 -5861 -5862 -5863 -5864 -5865 -5866 -5867 -5868 -5869 -5870 -5871 -5872 -5873 -5874 -5875 -5876 -5877 -5878 -5879 -5880 -5881 -5882 -5883 -5884 -5885 -5886 -5887 -5888 -5889 -5890 -5891 -5892 -5893 -5894 -5895 -5896 -5897 -5898 -5899 -5900 -5901 -5902 -5903 -5904 -5905 -5906 -5907 -5908 -5909 -5910 -5911 -5912 -5913 -5914 -5915 -5916 -5917 -5918 -5919 -5920 -5921 -5922 -5923 -5924 -5925 -5926 -5927 -5928 -5929 -5930 -5931 -5932 -5933 -5934 -5935 -5936 -5937 -5938 -5939 -5940 -5941 -5942 -5943 -5944 -5945 -5946 -5947 -5948 -5949 -5950 -5951 -5952 -5953 -5954 -5955 -5956 -5957 -5958 -5959 -5960 -5961 -5962 -5963 -5964 -5965 -5966 -5967 -5968 -5969 -5970 -5971 -5972 -5973 -5974 -5975 -5976 -5977 -5978 -5979 -5980 -5981 -5982 -5983 -5984 -5985 -5986 -5987 -5988 -5989 -5990 -5991 -5992 -5993 -5994 -5995 -5996 -5997 -5998 -5999 -6000 -6001 -6002 -6003 -6004 -6005 -6006 -6007 -6008 -6009 -6010 -6011 -6012 -6013 -6014 -6015 -6016 -6017 -6018 -6019 -6020 -6021 -6022 -6023 -6024 -6025 -6026 -6027 -6028 -6029 -6030 -6031 -6032 -6033 -6034 -6035 -6036 -6037 -6038 -6039 -6040 -6041 -6042 -6043 -6044 -6045 -6046 -6047 -6048 -6049 -6050 -6051 -6052 -6053 -6054 -6055 -6056 -6057 -6058 -6059 -6060 -6061 -6062 -6063 -6064 -6065 -6066 -6067 -6068 -6069 -6070 -6071 -6072 -6073 -6074 -6075 -6076 -6077 -6078 -6079 -6080 -6081 -6082 -6083 -6084 -6085 -6086 -6087 -6088 -6089 -6090 -6091 -6092 -6093 -6094 -6095 -6096 -6097 -6098 -6099 -6100 -6101 -6102 -6103 -6104 -6105 -6106 -6107 -6108 -6109 -6110 -6111 -6112 -6113 -6114 -6115 -6116 -6117 -6118 -6119 -6120 -6121 -6122 -6123 -6124 -6125 -6126 -6127 -6128 -6129 -6130 -6131 -6132 -6133 -6134 -6135 -6136 -6137 -6138 -6139 -6140 -6141 -6142 -6143 -6144 -6145 -6146 -6147 -6148 -6149 -6150 -6151 -6152 -6153 -6154 -6155 -6156 -6157 -6158 -6159 -6160 -6161 -6162 -6163 -6164 -6165 -6166 -6167 -6168 -6169 -6170 -6171 -6172 -6173 -6174 -6175 -6176 -6177 -6178 -6179 -6180 -6181 -6182 -6183 -6184 -6185 -6186 -6187 -6188 -6189 -6190 -6191 -6192 -6193 -6194 -6195 -6196 -6197 -6198 -6199 -6200 -6201 -6202 -6203 -6204 -6205 -6206 -6207 -6208 -6209 -6210 -6211 -6212 -6213 -6214 -6215 -6216 -6217 -6218 -6219 -6220 -6221 -6222 -6223 -6224 -6225 -6226 -6227 -6228 -6229 -6230 -6231 -6232 -6233 -6234 -6235 -6236 -6237 -6238 -6239 -6240 -6241 -6242 -6243 -6244 -6245 -6246 -6247 -6248 -6249 -6250 -6251 -6252 -6253 -6254 -6255 -6256 -6257 -6258 -6259 -6260 -6261 -6262 -6263 -6264 -6265 -6266 -6267 -6268 -6269 -6270 -6271 -6272 -6273 -6274 -6275 -6276 -6277 -6278 -6279 -6280 -6281 -6282 -6283 -6284 -6285 -6286 -6287 -6288 -6289 -6290 -6291 -6292 -6293 -6294 -6295 -6296 -6297 -6298 -6299 -6300 -6301 -6302 -6303 -6304 -6305 -6306 -6307 -6308 -6309 -6310 -6311 -6312 -6313 -6314 -6315 -6316 -6317 -6318 -6319 -6320 -6321 -6322 -6323 -6324 -6325 -6326 -6327 -6328 -6329 -6330 -6331 -6332 -6333 -6334 -6335 -6336 -6337 -6338 -6339 -6340 -6341 -6342 -6343 -6344 -6345 -6346 -6347 -6348 -6349 -6350 -6351 -6352 -6353 -6354 -6355 -6356 -6357 -6358 -6359 -6360 -6361 -6362 -6363 -6364 -6365 -6366 -6367 -6368 -6369 -6370 -6371 -6372 -6373 -6374 -6375 -6376 -6377 -6378 -6379 -6380 -6381 -6382 -6383 -6384 -6385 -6386 -6387 -6388 -6389 -6390 -6391 -6392 -6393 -6394 -6395 -6396 -6397 -6398 -6399 -6400 -6401 -6402 -6403 -6404 -6405 -6406 -6407 -6408 -6409 -6410 -6411 -6412 -6413 -6414 -6415 -6416 -6417 -6418 -6419 -6420 -6421 -6422 -6423 -6424 -6425 -6426 -6427 -6428 -6429 -6430 -6431 -6432 -6433 -6434 -6435 -6436 -6437 -6438 -6439 -6440 -6441 -6442 -6443 -6444 -6445 -6446 -6447 -6448 -6449 -6450 -6451 -6452 -6453 -6454 -6455 -6456 -6457 -6458 -6459 -6460 -6461 -6462 -6463 -6464 -6465 -6466 -6467 -6468 -6469 -6470 -6471 -6472 -6473 -6474 -6475 -6476 -6477 -6478 -6479 -6480 -6481 -6482 -6483 -6484 -6485 -6486 -6487 -6488 -6489 -6490 -6491 -6492 -6493 -6494 -6495 -6496 -6497 -6498 -6499 -6500 -6501 -6502 -6503 -6504 -6505 -6506 -6507 -6508 -6509 -6510 -6511 -6512 -6513 -6514 -6515 -6516 -6517 -6518 -6519 -6520 -6521 -6522 -6523 -6524 -6525 -6526 -6527 -6528 -6529 -6530 -6531 -6532 -6533 -6534 -6535 -6536 -6537 -6538 -6539 -6540 -6541 -6542 -6543 -6544 -6545 -6546 -6547 -6548 -6549 -6550 -6551 -6552 -6553 -6554 -6555 -6556 -6557 -6558 -6559 -6560 -6561 -6562 -6563 -6564 -6565 -6566 -6567 -6568 -6569 -6570 -6571 -6572 -6573 -6574 -6575 -6576 -6577 -6578 -6579 -6580 -6581 -6582 -6583 -6584 -6585 -6586 -6587 -6588 -6589 -6590 -6591 -6592 -6593 -6594 -6595 -6596 -6597 -6598 -6599 -6600 -6601 -6602 -6603 -6604 -6605 -6606 -6607 -6608 -6609 -6610 -6611 -6612 -6613 -6614 -6615 -6616 -6617 -6618 -6619 -6620 -6621 -6622 -6623 -6624 -6625 -6626 -6627 -6628 -6629 -6630 -6631 -6632 -6633 -6634 -6635 -6636 -6637 -6638 -6639 -6640 -6641 -6642 -6643 -6644 -6645 -6646 -6647 -6648 -6649 -6650 -6651 -6652 -6653 -6654 -6655 -6656 -6657 -6658 -6659 -6660 -6661 -6662 -6663 -6664 -6665 -6666 -6667 -6668 -6669 -6670 -6671 -6672 -6673 -6674 -6675 -6676 -6677 -6678 -6679 -6680 -6681 -6682 -6683 -6684 -6685 -6686 -6687 -6688 -6689 -6690 -6691 -6692 -6693 -6694 -6695 -6696 -6697 -6698 -6699 -6700 -6701 -6702 -6703 -6704 -6705 -6706 -6707 -6708 -6709 -6710 -6711 -6712 -6713 -6714 -6715 -6716 -6717 -6718 -6719 -6720 -6721 -6722 -6723 -6724 -6725 -6726 -6727 -6728 -6729 -6730 -6731 -6732 -6733 -6734 -6735 -6736 -6737 -6738 -6739 -6740 -6741 -6742 -6743 -6744 -6745 -6746 -6747 -6748 -6749 -6750 -6751 -6752 -6753 -6754 -6755 -6756 -6757 -6758 -6759 -6760 -6761 -6762 -6763 -6764 -6765 -6766 -6767 -6768 -6769 -6770 -6771 -6772 -6773 -6774 -6775 -6776 -6777 -6778 -6779 -6780 -6781 -6782 -6783 -6784 -6785 -6786 -6787 -6788 -6789 -6790 -6791 -6792 -6793 -6794 -6795 -6796 -6797 -6798 -6799 -6800 -6801 -6802 -6803 -6804 -6805 -6806 -6807 -6808 -6809 -6810 -6811 -6812 -6813 -6814 -6815 -6816 -6817 -6818 -6819 -6820 -6821 -6822 -6823 -6824 -6825 -6826 -6827 -6828 -6829 -6830 -6831 -6832 -6833 -6834 -6835 -6836 -6837 -6838 -6839 -6840 -6841 -6842 -6843 -6844 -6845 -6846 -6847 -6848 -6849 -6850 -6851 -6852 -6853 -6854 -6855 -6856 -6857 -6858 -6859 -6860 -6861 -6862 -6863 -6864 -6865 -6866 -6867 -6868 -6869 -6870 -6871 -6872 -6873 -6874 -6875 -6876 -6877 -6878 -6879 -6880 -6881 -6882 -6883 -6884 -6885 -6886 -6887 -6888 -6889 -6890 -6891 -6892 -6893 -6894 -6895 -6896 -6897 -6898 -6899 -6900 -6901 -6902 -6903 -6904 -6905 -6906 -6907 -6908 -6909 -6910 -6911 -6912 -6913 -6914 -6915 -6916 -6917 -6918 -6919 -6920 -6921 -6922 -6923 -6924 -6925 -6926 -6927 -6928 -6929 -6930 -6931 -6932 -6933 -6934 -6935 -6936 -6937 -6938 -6939 -6940 -6941 -6942 -6943 -6944 -6945 -6946 -6947 -6948 -6949 -6950 -6951 -6952 -6953 -6954 -6955 -6956 -6957 -6958 -6959 -6960 -6961 -6962 -6963 -6964 -6965 -6966 -6967 -6968 -6969 -6970 -6971 -6972 -6973 -6974 -6975 -6976 -6977 -6978 -6979 -6980 -6981 -6982 -6983 -6984 -6985 -6986 -6987 -6988 -6989 -6990 -6991 -6992 -6993 -6994 -6995 -6996 -6997 -6998 -6999 -7000 -7001 -7002 -7003 -7004 -7005 -7006 -7007 -7008 -7009 -7010 -7011 -7012 -7013 -7014 -7015 -7016 -7017 -7018 -7019 -7020 -7021 -7022 -7023 -7024 -7025 -7026 -7027 -7028 -7029 -7030 -7031 -7032 -7033 -7034 -7035 -7036 -7037 -7038 -7039 -7040 -7041 -7042 -7043 -7044 -7045 -7046 -7047 -7048 -7049 -7050 -7051 -7052 -7053 -7054 -7055 -7056 -7057 -7058 -7059 -7060 -7061 -7062 -7063 -7064 -7065 -7066 -7067 -7068 -7069 -7070 -7071 -7072 -7073 -7074 -7075 -7076 -7077 -7078 -7079 -7080 -7081 -7082 -7083 -7084 -7085 -7086 -7087 -7088 -7089 -7090 -7091 -7092 -7093 -7094 -7095 -7096 -7097 -7098 -7099 -7100 -7101 -7102 -7103 -7104 -7105 -7106 -7107 -7108 -7109 -7110 -7111 -7112 -7113 -7114 -7115 -7116 -7117 -7118 -7119 -7120 -7121 -7122 -7123 -7124 -7125 -7126 -7127 -7128 -7129 -7130 -7131 -7132 -7133 -7134 -7135 -7136 -7137 -7138 -7139 -7140 -7141 -7142 -7143 -7144 -7145 -7146 -7147 -7148 -7149 -7150 -7151 -7152 -7153 -7154 -7155 -7156 -7157 -7158 -7159 -7160 -7161 -7162 -7163 -7164 -7165 -7166 -7167 -7168 -7169 -7170 -7171 -7172 -7173 -7174 -7175 -7176 -7177 -7178 -7179 -7180 -7181 -7182 -7183 -7184 -7185 -7186 -7187 -7188 -7189 -7190 -7191 -7192 -7193 -7194 -7195 -7196 -7197 -7198 -7199 -7200 -7201 -7202 -7203 -7204 -7205 -7206 -7207 -7208 -7209 -7210 -7211 -7212 -7213 -7214 -7215 -7216 -7217 -7218 -7219 -7220 -7221 -7222 -7223 -7224 -7225 -7226 -7227 -7228 -7229 -7230 -7231 -7232 -7233 -7234 -7235 -7236 -7237 -7238 -7239 -7240 -7241 -7242 -7243 -7244 -7245 -7246 -7247 -7248 -7249 -7250 -7251 -7252 -7253 -7254 -7255 -7256 -7257 -7258 -7259 -7260 -7261 -7262 -7263 -7264 -7265 -7266 -7267 -7268 -7269 -7270 -7271 -7272 -7273 -7274 -7275 -7276 -7277 -7278 -7279 -7280 -7281 -7282 -7283 -7284 -7285 -7286 -7287 -7288 -7289 -7290 -7291 -7292 -7293 -7294 -7295 -7296 -7297 -7298 -7299 -7300 -7301 -7302 -7303 -7304 -7305 -7306 -7307 -7308 -7309 -7310 -7311 -7312 -7313 -7314 -7315 -7316 -7317 -7318 -7319 -7320 -7321 -7322 -7323 -7324 -7325 -7326 -7327 -7328 -7329 -7330 -7331 -7332 -7333 -7334 -7335 -7336 -7337 -7338 -7339 -7340 -7341 -7342 -7343 -7344 -7345 -7346 -7347 -7348 -7349 -7350 -7351 -7352 -7353 -7354 -7355 -7356 -7357 -7358 -7359 -7360 -7361 -7362 -7363 -7364 -7365 -7366 -7367 -7368 -7369 -7370 -7371 -7372 -7373 -7374 -7375 -7376 -7377 -7378 -7379 -7380 -7381 -7382 -7383 -7384 -7385 -7386 -7387 -7388 -7389 -7390 -7391 -7392 -7393 -7394 -7395 -7396 -7397 -7398 -7399 -7400 -7401 -7402 -7403 -7404 -7405 -7406 -7407 -7408 -7409 -7410 -7411 -7412 -7413 -7414 -7415 -7416 -7417 -7418 -7419 -7420 -7421 -7422 -7423 -7424 -7425 -7426 -7427 -7428 -7429 -7430 -7431 -7432 -7433 -7434 -7435 -7436 -7437 -7438 -7439 -7440 -7441 -7442 -7443 -7444 -7445 -7446 -7447 -7448 -7449 -7450 -7451 -7452 -7453 -7454 -7455 -7456 -7457 -7458 -7459 -7460 -7461 -7462 -7463 -7464 -7465 -7466 -7467 -7468 -7469 -7470 -7471 -7472 -7473 -7474 -7475 -7476 -7477 -7478 -7479 -7480 -7481 -7482 -7483 -7484 -7485 -7486 -7487 -7488 -7489 -7490 -7491 -7492 -7493 -7494 -7495 -7496 -7497 -7498 -7499 -7500 -7501 -7502 -7503 -7504 -7505 -7506 -7507 -7508 -7509 -7510 -7511 -7512 -7513 -7514 -7515 -7516 -7517 -7518 -7519 -7520 -7521 -7522 -7523 -7524 -7525 -7526 -7527 -7528 -7529 -7530 -7531 -7532 -7533 -7534 -7535 -7536 -7537 -7538 -7539 -7540 -7541 -7542 -7543 -7544 -7545 -7546 -7547 -7548 -7549 -7550 -7551 -7552 -7553 -7554 -7555 -7556 -7557 -7558 -7559 -7560 -7561 -7562 -7563 -7564 -7565 -7566 -7567 -7568 -7569 -7570 -7571 -7572 -7573 -7574 -7575 -7576 -7577 -7578 -7579 -7580 -7581 -7582 -7583 -7584 -7585 -7586 -7587 -7588 -7589 -7590 -7591 -7592 -7593 -7594 -7595 -7596 -7597 -7598 -7599 -7600 -7601 -7602 -7603 -7604 -7605 -7606 -7607 -7608 -7609 -7610 -7611 -7612 -7613 -7614 -7615 -7616 -7617 -7618 -7619 -7620 -7621 -7622 -7623 -7624 -7625 -7626 -7627 -7628 -7629 -7630 -7631 -7632 -7633 -7634 -7635 -7636 -7637 -7638 -7639 -7640 -7641 -7642 -7643 -7644 -7645 -7646 -7647 -7648 -7649 -7650 -7651 -7652 -7653 -7654 -7655 -7656 -7657 -7658 -7659 -7660 -7661 -7662 -7663 -7664 -7665 -7666 -7667 -7668 -7669 -7670 -7671 -7672 -7673 -7674 -7675 -7676 -7677 -7678 -7679 -7680 -7681 -7682 -7683 -7684 -7685 -7686 -7687 -7688 -7689 -7690 -7691 -7692 -7693 -7694 -7695 -7696 -7697 -7698 -7699 -7700 -7701 -7702 -7703 -7704 -7705 -7706 -7707 -7708 -7709 -7710 -7711 -7712 -7713 -7714 -7715 -7716 -7717 -7718 -7719 -7720 -7721 -7722 -7723 -7724 -7725 -7726 -7727 -7728 -7729 -7730 -7731 -7732 -7733 -7734 -7735 -7736 -7737 -7738 -7739 -7740 -7741 -7742 -7743 -7744 -7745 -7746 -7747 -7748 -7749 -7750 -7751 -7752 -7753 -7754 -7755 -7756 -7757 -7758 -7759 -7760 -7761 -7762 -7763 -7764 -7765 -7766 -7767 -7768 -7769 -7770 -7771 -7772 -7773 -7774 -7775 -7776 -7777 -7778 -7779 -7780 -7781 -7782 -7783 -7784 -7785 -7786 -7787 -7788 -7789 -7790 -7791 -7792 -7793 -7794 -7795 -7796 -7797 -7798 -7799 -7800 -7801 -7802 -7803 -7804 -7805 -7806 -7807 -7808 -7809 -7810 -7811 -7812 -7813 -7814 -7815 -7816 -7817 -7818 -7819 -7820 -7821 -7822 -7823 -7824 -7825 -7826 -7827 -7828 -7829 -7830 -7831 -7832 -7833 -7834 -7835 -7836 -7837 -7838 -7839 -7840 -7841 -7842 -7843 -7844 -7845 -7846 -7847 -7848 -7849 -7850 -7851 -7852 -7853 -7854 -7855 -7856 -7857 -7858 -7859 -7860 -7861 -7862 -7863 -7864 -7865 -7866 -7867 -7868 -7869 -7870 -7871 -7872 -7873 -7874 -7875 -7876 -7877 -7878 -7879 -7880 -7881 -7882 -7883 -7884 -7885 -7886 -7887 -7888 -7889 -7890 -7891 -7892 -7893 -7894 -7895 -7896 -7897 -7898 -7899 -7900 -7901 -7902 -7903 -7904 -7905 -7906 -7907 -7908 -7909 -7910 -7911 -7912 -7913 -7914 -7915 -7916 -7917 -7918 -7919 -7920 -7921 -7922 -7923 -7924 -7925 -7926 -7927 -7928 -7929 -7930 -7931 -7932 -7933 -7934 -7935 -7936 -7937 -7938 -7939 -7940 -7941 -7942 -7943 -7944 -7945 -7946 -7947 -7948 -7949 -7950 -7951 -7952 -7953 -7954 -7955 -7956 -7957 -7958 -7959 -7960 -7961 -7962 -7963 -7964 -7965 -7966 -7967 -7968 -7969 -7970 -7971 -7972 -7973 -7974 -7975 -7976 -7977 -7978 -7979 -7980 -7981 -7982 -7983 -7984 -7985 -7986 -7987 -7988 -7989 -7990 -7991 -7992 -7993 -7994 -7995 -7996 -7997 -7998 -7999 -8000 -8001 -8002 -8003 -8004 -8005 -8006 -8007 -8008 -8009 -8010 -8011 -8012 -8013 -8014 -8015 -8016 -8017 -8018 -8019 -8020 -8021 -8022 -8023 -8024 -8025 -8026 -8027 -8028 -8029 -8030 -8031 -8032 -8033 -8034 -8035 -8036 -8037 -8038 -8039 -8040 -8041 -8042 -8043 -8044 -8045 -8046 -8047 -8048 -8049 -8050 -8051 -8052 -8053 -8054 -8055 -8056 -8057 -8058 -8059 -8060 -8061 -8062 -8063 -8064 -8065 -8066 -8067 -8068 -8069 -8070 -8071 -8072 -8073 -8074 -8075 -8076 -8077 -8078 -8079 -8080 -8081 -8082 -8083 -8084 -8085 -8086 -8087 -8088 -8089 -8090 -8091 -8092 -8093 -8094 -8095 -8096 -8097 -8098 -8099 -8100 -8101 -8102 -8103 -8104 -8105 -8106 -8107 -8108 -8109 -8110 -8111 -8112 -8113 -8114 -8115 -8116 -8117 -8118 -8119 -8120 -8121 -8122 -8123 -8124 -8125 -8126 -8127 -8128 -8129 -8130 -8131 -8132 -8133 -8134 -8135 -8136 -8137 -8138 -8139 -8140 -8141 -8142 -8143 -8144 -8145 -8146 -8147 -8148 -8149 -8150 -8151 -8152 -8153 -8154 -8155 -8156 -8157 -8158 -8159 -8160 -8161 -8162 -8163 -8164 -8165 -8166 -8167 -8168 -8169 -8170 -8171 -8172 -8173 -8174 -8175 -8176 -8177 -8178 -8179 -8180 -8181 -8182 -8183 -8184 -8185 -8186 -8187 -8188 -8189 -8190 -8191 -8192 -8193 -8194 -8195 -8196 -8197 -8198 -8199 -8200 -8201 -8202 -8203 -8204 -8205 -8206 -8207 -8208 -8209 -8210 -8211 -8212 -8213 -8214 -8215 -8216 -8217 -8218 -8219 -8220 -8221 -8222 -8223 -8224 -8225 -8226 -8227 -8228 -8229 -8230 -8231 -8232 -8233 -8234 -8235 -8236 -8237 -8238 -8239 -8240 -8241 -8242 -8243 -8244 -8245 -8246 -8247 -8248 -8249 -8250 -8251 -8252 -8253 -8254 -8255 -8256 -8257 -8258 -8259 -8260 -8261 -8262 -8263 -8264 -8265 -8266 -8267 -8268 -8269 -8270 -8271 -8272 -8273 -8274 -8275 -8276 -8277 -8278 -8279 -8280 -8281 -8282 -8283 -8284 -8285 -8286 -8287 -8288 -8289 -8290 -8291 -8292 -8293 -8294 -8295 -8296 -8297 -8298 -8299 -8300 -8301 -8302 -8303 -8304 -8305 -8306 -8307 -8308 -8309 -8310 -8311 -8312 -8313 -8314 -8315 -8316 -8317 -8318 -8319 -8320 -8321 -8322 -8323 -8324 -8325 -8326 -8327 -8328 -8329 -8330 -8331 -8332 -8333 -8334 -8335 -8336 -8337 -8338 -8339 -8340 -8341 -8342 -8343 -8344 -8345 -8346 -8347 -8348 -8349 -8350 -8351 -8352 -8353 -8354 -8355 -8356 -8357 -8358 -8359 -8360 -8361 -8362 -8363 -8364 -8365 -8366 -8367 -8368 -8369 -8370 -8371 -8372 -8373 -8374 -8375 -8376 -8377 -8378 -8379 -8380 -8381 -8382 -8383 -8384 -8385 -8386 -8387 -8388 -8389 -8390 -8391 -8392 -8393 -8394 -8395 -8396 -8397 -8398 -8399 -8400 -8401 -8402 -8403 -8404 -8405 -8406 -8407 -8408 -8409 -8410 -8411 -8412 -8413 -8414 -8415 -8416 -8417 -8418 -8419 -8420 -8421 -8422 -8423 -8424 -8425 -8426 -8427 -8428 -8429 -8430 -8431 -8432 -8433 -8434 -8435 -8436 -8437 -8438 -8439 -8440 -8441 -8442 -8443 -8444 -8445 -8446 -8447 -8448 -8449 -8450 -8451 -8452 -8453 -8454 -8455 -8456 -8457 -8458 -8459 -8460 -8461 -8462 -8463 -8464 -8465 -8466 -8467 -8468 -8469 -8470 -8471 -8472 -8473 -8474 -8475 -8476 -8477 -8478 -8479 -8480 -8481 -8482 -8483 -8484 -8485 -8486 -8487 -8488 -8489 -8490 -8491 -8492 -8493 -8494 -8495 -8496 -8497 -8498 -8499 -8500 -8501 -8502 -8503 -8504 -8505 -8506 -8507 -8508 -8509 -8510 -8511 -8512 -8513 -8514 -8515 -8516 -8517 -8518 -8519 -8520 -8521 -8522 -8523 -8524 -8525 -8526 -8527 -8528 -8529 -8530 -8531 -8532 -8533 -8534 -8535 -8536 -8537 -8538 -8539 -8540 -8541 -8542 -8543 -8544 -8545 -8546 -8547 -8548 -8549 -8550 -8551 -8552 -8553 -8554 -8555 -8556 -8557 -8558 -8559 -8560 -8561 -8562 -8563 -8564 -8565 -8566 -8567 -8568 -8569 -8570 -8571 -8572 -8573 -8574 -8575 -8576 -8577 -8578 -8579 -8580 -8581 -8582 -8583 -8584 -8585 -8586 -8587 -8588 -8589 -8590 -8591 -8592 -8593 -8594 -8595 -8596 -8597 -8598 -8599 -8600 -8601 -8602 -8603 -8604 -8605 -8606 -8607 -8608 -8609 -8610 -8611 -8612 -8613 -8614 -8615 -8616 -8617 -8618 -8619 -8620 -8621 -8622 -8623 -8624 -8625 -8626 -8627 -8628 -8629 -8630 -8631 -8632 -8633 -8634 -8635 -8636 -8637 -8638 -8639 -8640 -8641 -8642 -8643 -8644 -8645 -8646 -8647 -8648 -8649 -8650 -8651 -8652 -8653 -8654 -8655 -8656 -8657 -8658 -8659 -8660 -8661 -8662 -8663 -8664 -8665 -8666 -8667 -8668 -8669 -8670 -8671 -8672 -8673 -8674 -8675 -8676 -8677 -8678 -8679 -8680 -8681 -8682 -8683 -8684 -8685 -8686 -8687 -8688 -8689 -8690 -8691 -8692 -8693 -8694 -8695 -8696 -8697 -8698 -8699 -8700 -8701 -8702 -8703 -8704 -8705 -8706 -8707 -8708 -8709 -8710 -8711 -8712 -8713 -8714 -8715 -8716 -8717 -8718 -8719 -8720 -8721 -8722 -8723 -8724 -8725 -8726 -8727 -8728 -8729 -8730 -8731 -8732 -8733 -8734 -8735 -8736 -8737 -8738 -8739 -8740 -8741 -8742 -8743 -8744 -8745 -8746 -8747 -8748 -8749 -8750 -8751 -8752 -8753 -8754 -8755 -8756 -8757 -8758 -8759 -8760 -8761 -8762 -8763 -8764 -8765 -8766 -8767 -8768 -8769 -8770 -8771 -8772 -8773 -8774 -8775 -8776 -8777 -8778 -8779 -8780 -8781 -8782 -8783 -8784 -8785 -8786 -8787 -8788 -8789 -8790 -8791 -8792 -8793 -8794 -8795 -8796 -8797 -8798 -8799 -8800 -8801 -8802 -8803 -8804 -8805 -8806 -8807 -8808 -8809 -8810 -8811 -8812 -8813 -8814 -8815 -8816 -8817 -8818 -8819 -8820 -8821 -8822 -8823 -8824 -8825 -8826 -8827 -8828 -8829 -8830 -8831 -8832 -8833 -8834 -8835 -8836 -8837 -8838 -8839 -8840 -8841 -8842 -8843 -8844 -8845 -8846 -8847 -8848 -8849 -8850 -8851 -8852 -8853 -8854 -8855 -8856 -8857 -8858 -8859 -8860 -8861 -8862 -8863 -8864 -8865 -8866 -8867 -8868 -8869 -8870 -8871 -8872 -8873 -8874 -8875 -8876 -8877 -8878 -8879 -8880 -8881 -8882 -8883 -8884 -8885 -8886 -8887 -8888 -8889 -8890 -8891 -8892 -8893 -8894 -8895 -8896 -8897 -8898 -8899 -8900 -8901 -8902 -8903 -8904 -8905 -8906 -8907 -8908 -8909 -8910 -8911 -8912 -8913 -8914 -8915 -8916 -8917 -8918 -8919 -8920 -8921 -8922 -8923 -8924 -8925 -8926 -8927 -8928 -8929 -8930 -8931 -8932 -8933 -8934 -8935 -8936 -8937 -8938 -8939 -8940 -8941 -8942 -8943 -8944 -8945 -8946 -8947 -8948 -8949 -8950 -8951 -8952 -8953 -8954 -8955 -8956 -8957 -8958 -8959 -8960 -8961 -8962 -8963 -8964 -8965 -8966 -8967 -8968 -8969 -8970 -8971 -8972 -8973 -8974 -8975 -8976 -8977 -8978 -8979 -8980 -8981 -8982 -8983 -8984 -8985 -8986 -8987 -8988 -8989 -8990 -8991 -8992 -8993 -8994 -8995 -8996 -8997 -8998 -8999 -9000 -9001 -9002 -9003 -9004 -9005 -9006 -9007 -9008 -9009 -9010 -9011 -9012 -9013 -9014 -9015 -9016 -9017 -9018 -9019 -9020 -9021 -9022 -9023 -9024 -9025 -9026 -9027 -9028 -9029 -9030 -9031 -9032 -9033 -9034 -9035 -9036 -9037 -9038 -9039 -9040 -9041 -9042 -9043 -9044 -9045 -9046 -9047 -9048 -9049 -9050 -9051 -9052 -9053 -9054 -9055 -9056 -9057 -9058 -9059 -9060 -9061 -9062 -9063 -9064 -9065 -9066 -9067 -9068 -9069 -9070 -9071 -9072 -9073 -9074 -9075 -9076 -9077 -9078 -9079 -9080 -9081 -9082 -9083 -9084 -9085 -9086 -9087 -9088 -9089 -9090 -9091 -9092 -9093 -9094 -9095 -9096 -9097 -9098 -9099 -9100 -9101 -9102 -9103 -9104 -9105 -9106 -9107 -9108 -9109 -9110 -9111 -9112 -9113 -9114 -9115 -9116 -9117 -9118 -9119 -9120 -9121 -9122 -9123 -9124 -9125 -9126 -9127 -9128 -9129 -9130 -9131 -9132 -9133 -9134 -9135 -9136 -9137 -9138 -9139 -9140 -9141 -9142 -9143 -9144 -9145 -9146 -9147 -9148 -9149 -9150 -9151 -9152 -9153 -9154 -9155 -9156 -9157 -9158 -9159 -9160 -9161 -9162 -9163 -9164 -9165 -9166 -9167 -9168 -9169 -9170 -9171 -9172 -9173 -9174 -9175 -9176 -9177 -9178 -9179 -9180 -9181 -9182 -9183 -9184 -9185 -9186 -9187 -9188 -9189 -9190 -9191 -9192 -9193 -9194 -9195 -9196 -9197 -9198 -9199 -9200 -9201 -9202 -9203 -9204 -9205 -9206 -9207 -9208 -9209 -9210 -9211 -9212 -9213 -9214 -9215 -9216 -9217 -9218 -9219 -9220 -9221 -9222 -9223 -9224 -9225 -9226 -9227 -9228 -9229 -9230 -9231 -9232 -9233 -9234 -9235 -9236 -9237 -9238 -9239 -9240 -9241 -9242 -9243 -9244 -9245 -9246 -9247 -9248 -9249 -9250 -9251 -9252 -9253 -9254 -9255 -9256 -9257 -9258 -9259 -9260 -9261 -9262 -9263 -9264 -9265 -9266 -9267 -9268 -9269 -9270 -9271 -9272 -9273 -9274 -9275 -9276 -9277 -9278 -9279 -9280 -9281 -9282 -9283 -9284 -9285 -9286 -9287 -9288 -9289 -9290 -9291 -9292 -9293 -9294 -9295 -9296 -9297 -9298 -9299 -9300 -9301 -9302 -9303 -9304 -9305 -9306 -9307 -9308 -9309 -9310 -9311 -9312 -9313 -9314 -9315 -9316 -9317 -9318 -9319 -9320 -9321 -9322 -9323 -9324 -9325 -9326 -9327 -9328 -9329 -9330 -9331 -9332 -9333 -9334 -9335 -9336 -9337 -9338 -9339 -9340 -9341 -9342 -9343 -9344 -9345 -9346 -9347 -9348 -9349 -9350 -9351 -9352 -9353 -9354 -9355 -9356 -9357 -9358 -9359 -9360 -9361 -9362 -9363 -9364 -9365 -9366 -9367 -9368 -9369 -9370 -9371 -9372 -9373 -9374 -9375 -9376 -9377 -9378 -9379 -9380 -9381 -9382 -9383 -9384 -9385 -9386 -9387 -9388 -9389 -9390 -9391 -9392 -9393 -9394 -9395 -9396 -9397 -9398 -9399 -9400 -9401 -9402 -9403 -9404 -9405 -9406 -9407 -9408 -9409 -9410 -9411 -9412 -9413 -9414 -9415 -9416 -9417 -9418 -9419 -9420 -9421 -9422 -9423 -9424 -9425 -9426 -9427 -9428 -9429 -9430 -9431 -9432 -9433 -9434 -9435 -9436 -9437 -9438 -9439 -9440 -9441 -9442 -9443 -9444 -9445 -9446 -9447 -9448 -9449 -9450 -9451 -9452 -9453 -9454 -9455 -9456 -9457 -9458 -9459 -9460 -9461 -9462 -9463 -9464 -9465 -9466 -9467 -9468 -9469 -9470 -9471 -9472 -9473 -9474 -9475 -9476 -9477 -9478 -9479 -9480 -9481 -9482 -9483 -9484 -9485 -9486 -9487 -9488 -9489 -9490 -9491 -9492 -9493 -9494 -9495 -9496 -9497 -9498 -9499 -9500 -9501 -9502 -9503 -9504 -9505 -9506 -9507 -9508 -9509 -9510 -9511 -9512 -9513 -9514 -9515 -9516 -9517 -9518 -9519 -9520 -9521 -9522 -9523 -9524 -9525 -9526 -9527 -9528 -9529 -9530 -9531 -9532 -9533 -9534 -9535 -9536 -9537 -9538 -9539 -9540 -9541 -9542 -9543 -9544 -9545 -9546 -9547 -9548 -9549 -9550 -9551 -9552 -9553 -9554 -9555 -9556 -9557 -9558 -9559 -9560 -9561 -9562 -9563 -9564 -9565 -9566 -9567 -9568 -9569 -9570 -9571 -9572 -9573 -9574 -9575 -9576 -9577 -9578 -9579 -9580 -9581 -9582 -9583 -9584 -9585 -9586 -9587 -9588 -9589 -9590 -9591 -9592 -9593 -9594 -9595 -9596 -9597 -9598 -9599 -9600 -9601 -9602 -9603 -9604 -9605 -9606 -9607 -9608 -9609 -9610 -9611 -9612 -9613 -9614 -9615 -9616 -9617 -9618 -9619 -9620 -9621 -9622 -9623 -9624 -9625 -9626 -9627 -9628 -9629 -9630 -9631 -9632 -9633 -9634 -9635 -9636 -9637 -9638 -9639 -9640 -9641 -9642 -9643 -9644 -9645 -9646 -9647 -9648 -9649 -9650 -9651 -9652 -9653 -9654 -9655 -9656 -9657 -9658 -9659 -9660 -9661 -9662 -9663 -9664 -9665 -9666 -9667 -9668 -9669 -9670 -9671 -9672 -9673 -9674 -9675 -9676 -9677 -9678 -9679 -9680 -9681 -9682 -9683 -9684 -9685 -9686 -9687 -9688 -9689 -9690 -9691 -9692 -9693 -9694 -9695 -9696 -9697 -9698 -9699 -9700 -9701 -9702 -9703 -9704 -9705 -9706 -9707 -9708 -9709 -9710 -9711 -9712 -9713 -9714 -9715 -9716 -9717 -9718 -9719 -9720 -9721 -9722 -9723 -9724 -9725 -9726 -9727 -9728 -9729 -9730 -9731 -9732 -9733 -9734 -9735 -9736 -9737 -9738 -9739 -9740 -9741 -9742 -9743 -9744 -9745 -9746 -9747 -9748 -9749 -9750 -9751 -9752 -9753 -9754 -9755 -9756 -9757 -9758 -9759 -9760 -9761 -9762 -9763 -9764 -9765 -9766 -9767 -9768 -9769 -9770 -9771 -9772 -9773 -9774 -9775 -9776 -9777 -9778 -9779 -9780 -9781 -9782 -9783 -9784 -9785 -9786 -9787 -9788 -9789 -9790 -9791 -9792 -9793 -9794 -9795 -9796 -9797 -9798 -9799 -9800 -9801 -9802 -9803 -9804 -9805 -9806 -9807 -9808 -9809 -9810 -9811 -9812 -9813 -9814 -9815 -9816 -9817 -9818 -9819 -9820 -9821 -9822 -9823 -9824 -9825 -9826 -9827 -9828 -9829 -9830 -9831 -9832 -9833 -9834 -9835 -9836 -9837 -9838 -9839 -9840 -9841 -9842 -9843 -9844 -9845 -9846 -9847 -9848 -9849 -9850 -9851 -9852 -9853 -9854 -9855 -9856 -9857 -9858 -9859 -9860 -9861 -9862 -9863 -9864 -9865 -9866 -9867 -9868 -9869 -9870 -9871 -9872 -9873 -9874 -9875 -9876 -9877 -9878 -9879 -9880 -9881 -9882 -9883 -9884 -9885 -9886 -9887 -9888 -9889 -9890 -9891 -9892 -9893 -9894 -9895 -9896 -9897 -9898 -9899 -9900 -9901 -9902 -9903 -9904 -9905 -9906 -9907 -9908 -9909 -9910 -9911 -9912 -9913 -9914 -9915 -9916 -9917 -9918 -9919 -9920 -9921 -9922 -9923 -9924 -9925 -9926 -9927 -9928 -9929 -9930 -9931 -9932 -9933 -9934 -9935 -9936 -9937 -9938 -9939 -9940 -9941 -9942 -9943 -9944 -9945 -9946 -9947 -9948 -9949 -9950 -9951 -9952 -9953 -9954 -9955 -9956 -9957 -9958 -9959 -9960 -9961 -9962 -9963 -9964 -9965 -9966 -9967 -9968 -9969 -9970 -9971 -9972 -9973 -9974 -9975 -9976 -9977 -9978 -9979 -9980 -9981 -9982 -9983 -9984 -9985 -9986 -9987 -9988 -9989 -9990 -9991 -9992 -9993 -9994 -9995 -9996 -9997 -9998 -9999 -10000 -10001 -10002 -10003 -10004 -10005 -10006 -10007 -10008 -10009 -10010 -10011 -10012 -10013 -10014 -10015 -10016 -10017 -10018 -10019 -10020 -10021 -10022 -10023 -10024 -10025 -10026 -10027 -10028 -10029 -10030 -10031 -10032 -10033 -10034 -10035 -10036 -10037 -10038 -10039 -10040 -10041 -10042 -10043 -10044 -10045 -10046 -10047 -10048 -10049 -10050 -10051 -10052 -10053 -10054 -10055 -10056 -10057 -10058 -10059 -10060 -10061 -10062 -10063 -10064 -10065 -10066 -10067 -10068 -10069 -10070 -10071 -10072 -10073 -10074 -10075 -10076 -10077 -10078 -10079 -10080 -10081 -10082 -10083 -10084 -10085 -10086 -10087 -10088 -10089 -10090 -10091 -10092 -10093 -10094 -10095 -10096 -10097 -10098 -10099 -10100 -10101 -10102 -10103 -10104 -10105 -10106 -10107 -10108 -10109 -10110 -10111 -10112 -10113 -10114 -10115 -10116 -10117 -10118 -10119 -10120 -10121 -10122 -10123 -10124 -10125 -10126 -10127 -10128 -10129 -10130 -10131 -10132 -10133 -10134 -10135 -10136 -10137 -10138 -10139 -10140 -10141 -10142 -10143 -10144 -10145 -10146 -10147 -10148 -10149 -10150 -10151 -10152 -10153 -10154 -10155 -10156 -10157 -10158 -10159 -10160 -10161 -10162 -10163 -10164 -10165 -10166 -10167 -10168 -10169 -10170 -10171 -10172 -10173 -10174 -10175 -10176 -10177 -10178 -10179 -10180 -10181 -10182 -10183 -10184 -10185 -10186 -10187 -10188 -10189 -10190 -10191 -10192 -10193 -10194 -10195 -10196 -10197 -10198 -10199 -10200 -10201 -10202 -10203 -10204 -10205 -10206 -10207 -10208 -10209 -10210 -10211 -10212 -10213 -10214 -10215 -10216 -10217 -10218 -10219 -10220 -10221 -10222 -10223 -10224 -10225 -10226 -10227 -10228 -10229 -10230 -10231 -10232 -10233 -10234 -10235 -10236 -10237 -10238 -10239 -10240 -10241 -10242 -10243 -10244 -10245 -10246 -10247 -10248 -10249 -10250 -10251 -10252 -10253 -10254 -10255 -10256 -10257 -10258 -10259 -10260 -10261 -10262 -10263 -10264 -10265 -10266 -10267 -10268 -10269 -10270 -10271 -10272 -10273 -10274 -10275 -10276 -10277 -10278 -10279 -10280 -10281 -10282 -10283 -10284 -10285 -10286 -10287 -10288 -10289 -10290 -10291 -10292 -10293 -10294 -10295 -10296 -10297 -10298 -10299 -10300 -10301 -10302 -10303 -10304 -10305 -10306 -10307 -10308 -10309 -10310 -10311 -10312 -10313 -10314 -10315 -10316 -10317 -10318 -10319 -10320 -10321 -10322 -10323 -10324 -10325 -10326 -10327 -10328 -10329 -10330 -10331 -10332 -10333 -10334 -10335 -10336 -10337 -10338 -10339 -10340 -10341 -10342 -10343 -10344 -10345 -10346 -10347 -10348 -10349 -10350 -10351 -10352 -10353 -10354 -10355 -10356 -10357 -10358 -10359 -10360 -10361 -10362 -10363 -10364 -10365 -10366 -10367 -10368 -10369 -10370 -10371 -10372 -10373 -10374 -10375 -10376 -10377 -10378 -10379 -10380 -10381 -10382 -10383 -10384 -10385 -10386 -10387 -10388 -10389 -10390 -10391 -10392 -10393 -10394 -10395 -10396 -10397 -10398 -10399 -10400 -10401 -10402 -10403 -10404 -10405 -10406 -10407 -10408 -10409 -10410 -10411 -10412 -10413 -10414 -10415 -10416 -10417 -10418 -10419 -10420 -10421 -10422 -10423 -10424 -10425 -10426 -10427 -10428 -10429 -10430 -10431 -10432 -10433 -10434 -10435 -10436 -10437 -10438 -10439 -10440 -10441 -10442 -10443 -10444 -10445 -10446 -10447 -10448 -10449 -10450 -10451 -10452 -10453 -10454 -10455 -10456 -10457 -10458 -10459 -10460 -10461 -10462 -10463 -10464 -10465 -10466 -10467 -10468 -10469 -10470 -10471 -10472 -10473 -10474 -10475 -10476 -10477 -10478 -10479 -10480 -10481 -10482 -10483 -10484 -10485 -10486 -10487 -10488 -10489 -10490 -10491 -10492 -10493 -10494 -10495 -10496 -10497 -10498 -10499 -10500 -10501 -10502 -10503 -10504 -10505 -10506 -10507 -10508 -10509 -10510 -10511 -10512 -10513 -10514 -10515 -10516 -10517 -10518 -10519 -10520 -10521 -10522 -10523 -10524 -10525 -10526 -10527 -10528 -10529 -10530 -10531 -10532 -10533 -10534 -10535 -10536 -10537 -10538 -10539 -10540 -10541 -10542 -10543 -10544 -10545 -10546 -10547 -10548 -10549 -10550 -10551 -10552 -10553 -10554 -10555 -10556 -10557 -10558 -10559 -10560 -10561 -10562 -10563 -10564 -10565 -10566 -10567 -10568 -10569 -10570 -10571 -10572 -10573 -10574 -10575 -10576 -10577 -10578 -10579 -10580 -10581 -10582 -10583 -10584 -10585 -10586 -10587 -10588 -10589 -10590 -10591 -10592 -10593 -10594 -10595 -10596 -10597 -10598 -10599 -10600 -10601 -10602 -10603 -10604 -10605 -10606 -10607 -10608 -10609 -10610 -10611 -10612 -10613 -10614 -10615 -10616 -10617 -10618 -10619 -10620 -10621 -10622 -10623 -10624 -10625 -10626 -10627 -10628 -10629 -10630 -10631 -10632 -10633 -10634 -10635 -10636 -10637 -10638 -10639 -10640 -10641 -10642 -10643 -10644 -10645 -10646 -10647 -10648 -10649 -10650 -10651 -10652 -10653 -10654 -10655 -10656 -10657 -10658 -10659 -10660 -10661 -10662 -10663 -10664 -10665 -10666 -10667 -10668 -10669 -10670 -10671 -10672 -10673 -10674 -10675 -10676 -10677 -10678 -10679 -10680 -10681 -10682 -10683 -10684 -10685 -10686 -10687 -10688 -10689 -10690 -10691 -10692 -10693 -10694 -10695 -10696 -10697 -10698 -10699 -10700 -10701 -10702 -10703 -10704 -10705 -10706 -10707 -10708 -10709 -10710 -10711 -10712 -10713 -10714 -10715 -10716 -10717 -10718 -10719 -10720 -10721 -10722 -10723 -10724 -10725 -10726 -10727 -10728 -10729 -10730 -10731 -10732 -10733 -10734 -10735 -10736 -10737 -10738 -10739 -10740 -10741 -10742 -10743 -10744 -10745 -10746 -10747 -10748 -10749 -10750 -10751 -10752 -10753 -10754 -10755 -10756 -10757 -10758 -10759 -10760 -10761 -10762 -10763 -10764 -10765 -10766 -10767 -10768 -10769 -10770 -10771 -10772 -10773 -10774 -10775 -10776 -10777 -10778 -10779 -10780 -10781 -10782 -10783 -10784 -10785 -10786 -10787 -10788 -10789 -10790 -10791 -10792 -10793 -10794 -10795 -10796 -10797 -10798 -10799 -10800 -10801 -10802 -10803 -10804 -10805 -10806 -10807 -10808 -10809 -10810 -10811 -10812 -10813 -10814 -10815 -10816 -10817 -10818 -10819 -10820 -10821 -10822 -10823 -10824 -10825 -10826 -10827 -10828 -10829 -10830 -10831 -10832 -10833 -10834 -10835 -10836 -10837 -10838 -10839 -10840 -10841 -10842 -10843 -10844 -10845 -10846 -10847 -10848 -10849 -10850 -10851 -10852 -10853 -10854 -10855 -10856 -10857 -10858 -10859 -10860 -10861 -10862 -10863 -10864 -10865 -10866 -10867 -10868 -10869 -10870 -10871 -10872 -10873 -10874 -10875 -10876 -10877 -10878 -10879 -10880 -10881 -10882 -10883 -10884 -10885 -10886 -10887 -10888 -10889 -10890 -10891 -10892 -10893 -10894 -10895 -10896 -10897 -10898 -10899 -10900 -10901 -10902 -10903 -10904 -10905 -10906 -10907 -10908 -10909 -10910 -10911 -10912 -10913 -10914 -10915 -10916 -10917 -10918 -10919 -10920 -10921 -10922 -10923 -10924 -10925 -10926 -10927 -10928 -10929 -10930 -10931 -10932 -10933 -10934 -10935 -10936 -10937 -10938 -10939 -10940 -10941 -10942 -10943 -10944 -10945 -10946 -10947 -10948 -10949 -10950 -10951 -10952 -10953 -10954 -10955 -10956 -10957 -10958 -10959 -10960 -10961 -10962 -10963 -10964 -10965 -10966 -10967 -10968 -10969 -10970 -10971 -10972 -10973 -10974 -10975 -10976 -10977 -10978 -10979 -10980 -10981 -10982 -10983 -10984 -10985 -10986 -10987 -10988 -10989 -10990 -10991 -10992 -10993 -10994 -10995 -10996 -10997 -10998 -10999 -11000 -11001 -11002 -11003 -11004 -11005 -11006 -11007 -11008 -11009 -11010 -11011 -11012 -11013 -11014 -11015 -11016 -11017 -11018 -11019 -11020 -11021 -11022 -11023 -11024 -11025 -11026 -11027 -11028 -11029 -11030 -11031 -11032 -11033 -11034 -11035 -11036 -11037 -11038 -11039 -11040 -11041 -11042 -11043 -11044 -11045 -11046 -11047 -11048 -11049 -11050 -11051 -11052 -11053 -11054 -11055 -11056 -11057 -11058 -11059 -11060 -11061 -11062 -11063 -11064 -11065 -11066 -11067 -11068 -11069 -11070 -11071 -11072 -11073 -11074 -11075 -11076 -11077 -11078 -11079 -11080 -11081 -11082 -11083 -11084 -11085 -11086 -11087 -11088 -11089 -11090 -11091 -11092 -11093 -11094 -11095 -11096 -11097 -11098 -11099 -11100 -11101 -11102 -11103 -11104 -11105 -11106 -11107 -11108 -11109 -11110 -11111 -11112 -11113 -11114 -11115 -11116 -11117 -11118 -11119 -11120 -11121 -11122 -11123 -11124 -11125 -11126 -11127 -11128 -11129 -11130 -11131 -11132 -11133 -11134 -11135 -11136 -11137 -11138 -11139 -11140 -11141 -11142 -11143 -11144 -11145 -11146 -11147 -11148 -11149 -11150 -11151 -11152 -11153 -11154 -11155 -11156 -11157 -11158 -11159 -11160 -11161 -11162 -11163 -11164 -11165 -11166 -11167 -11168 -11169 -11170 -11171 -11172 -11173 -11174 -11175 -11176 -11177 -11178 -11179 -11180 -11181 -11182 -11183 -11184 -11185 -11186 -11187 -11188 -11189 -11190 -11191 -11192 -11193 -11194 -11195 -11196 -11197 -11198 -11199 -11200 -11201 -11202 -11203 -11204 -11205 -11206 -11207 -11208 -11209 -11210 -11211 -11212 -11213 -11214 -11215 -11216 -11217 -11218 -11219 -11220 -11221 -11222 -11223 -11224 -11225 -11226 -11227 -11228 -11229 -11230 -11231 -11232 -11233 -11234 -11235 -11236 -11237 -11238 -11239 -11240 -11241 -11242 -11243 -11244 -11245 -11246 -11247 -11248 -11249 -11250 -11251 -11252 -11253 -11254 -11255 -11256 -11257 -11258 -11259 -11260 -11261 -11262 -11263 -11264 -11265 -11266 -11267 -11268 -11269 -11270 -11271 -11272 -11273 -11274 -11275 -11276 -11277 -11278 -11279 -11280 -11281 -11282 -11283 -11284 -11285 -11286 -11287 -11288 -11289 -11290 -11291 -11292 -11293 -11294 -11295 -11296 -11297 -11298 -11299 -11300 -11301 -11302 -11303 -11304 -11305 -11306 -11307 -11308 -11309 -11310 -11311 -11312 -11313 -11314 -11315 -11316 -11317 -11318 -11319 -11320 -11321 -11322 -11323 -11324 -11325 -11326 -11327 -11328 -11329 -11330 -11331 -11332 -11333 -11334 -11335 -11336 -11337 -11338 -11339 -11340 -11341 -11342 -11343 -11344 -11345 -11346 -11347 -11348 -11349 -11350 -11351 -11352 -11353 -11354 -11355 -11356 -11357 -11358 -11359 -11360 -11361 -11362 -11363 -11364 -11365 -11366 -11367 -11368 -11369 -11370 -11371 -11372 -11373 -11374 -11375 -11376 -11377 -11378 -11379 -11380 -11381 -11382 -11383 -11384 -11385 -11386 -11387 -11388 -11389 -11390 -11391 -11392 -11393 -11394 -11395 -11396 -11397 -11398 -11399 -11400 -11401 -11402 -11403 -11404 -11405 -11406 -11407 -11408 -11409 -11410 -11411 -11412 -11413 -11414 -11415 -11416 -11417 -11418 -11419 -11420 -11421 -11422 -11423 -11424 -11425 -11426 -11427 -11428 -11429 -11430 -11431 -11432 -11433 -11434 -11435 -11436 -11437 -11438 -11439 -11440 -11441 -11442 -11443 -11444 -11445 -11446 -11447 -11448 -11449 -11450 -11451 -11452 -11453 -11454 -11455 -11456 -11457 -11458 -11459 -11460 -11461 -11462 -11463 -11464 -11465 -11466 -11467 -11468 -11469 -11470 -11471 -11472 -11473 -11474 -11475 -11476 -11477 -11478 -11479 -11480 -11481 -11482 -11483 -11484 -11485 -11486 -11487 -11488 -11489 -11490 -11491 -11492 -11493 -11494 -11495 -11496 -11497 -11498 -11499 -11500 -11501 -11502 -11503 -11504 -11505 -11506 -11507 -11508 -11509 -11510 -11511 -11512 -11513 -11514 -11515 -11516 -11517 -11518 -11519 -11520 -11521 -11522 -11523 -11524 -11525 -11526 -11527 -11528 -11529 -11530 -11531 -11532 -11533 -11534 -11535 -11536 -11537 -11538 -11539 -11540 -11541 -11542 -11543 -11544 -11545 -11546 -11547 -11548 -11549 -11550 -11551 -11552 -11553 -11554 -11555 -11556 -11557 -11558 -11559 -11560 -11561 -11562 -11563 -11564 -11565 -11566 -11567 -11568 -11569 -11570 -11571 -11572 -11573 -11574 -11575 -11576 -11577 -11578 -11579 -11580 -11581 -11582 -11583 -11584 -11585 -11586 -11587 -11588 -11589 -11590 -11591 -11592 -11593 -11594 -11595 -11596 -11597 -11598 -11599 -11600 -11601 -11602 -11603 -11604 -11605 -11606 -11607 -11608 -11609 -11610 -11611 -11612 -11613 -11614 -11615 -11616 -11617 -11618 -11619 -11620 -11621 -11622 -11623 -11624 -11625 -11626 -11627 -11628 -11629 -11630 -11631 -11632 -11633 -11634 -11635 -11636 -11637 -11638 -11639 -11640 -11641 -11642 -11643 -11644 -11645 -11646 -11647 -11648 -11649 -11650 -11651 -11652 -11653 -11654 -11655 -11656 -11657 -11658 -11659 -11660 -11661 -11662 -11663 -11664 -11665 -11666 -11667 -11668 -11669 -11670 -11671 -11672 -11673 -11674 -11675 -11676 -11677 -11678 -11679 -11680 -11681 -11682 -11683 -11684 -11685 -11686 -11687 -11688 -11689 -11690 -11691 -11692 -11693 -11694 -11695 -11696 -11697 -11698 -11699 -11700 -11701 -11702 -11703 -11704 -11705 -11706 -11707 -11708 -11709 -11710 -11711 -11712 -11713 -11714 -11715 -11716 -11717 -11718 -11719 -11720 -11721 -11722 -11723 -11724 -11725 -11726 -11727 -11728 -11729 -11730 -11731 -11732 -11733 -11734 -11735 -11736 -11737 -11738 -11739 -11740 -11741 -11742 -11743 -11744 -11745 -11746 -11747 -11748 -11749 -11750 -11751 -11752 -11753 -11754 -11755 -11756 -11757 -11758 -11759 -11760 -11761 -11762 -11763 -11764 -11765 -11766 -11767 -11768 -11769 -11770 -11771 -11772 -11773 -11774 -11775 -11776 -11777 -11778 -11779 -11780 -11781 -11782 -11783 -11784 -11785 -11786 -11787 -11788 -11789 -11790 -11791 -11792 -11793 -11794 -11795 -11796 -11797 -11798 -11799 -11800 -11801 -11802 -11803 -11804 -11805 -11806 -11807 -11808 -11809 -11810 -11811 -11812 -11813 -11814 -11815 -11816 -11817 -11818 -11819 -11820 -11821 -11822 -11823 -11824 -11825 -11826 -11827 -11828 -11829 -11830 -11831 -11832 -11833 -11834 -11835 -11836 -11837 -11838 -11839 -11840 -11841 -11842 -11843 -11844 -11845 -11846 -11847 -11848 -11849 -11850 -11851 -11852 -11853 -11854 -11855 -11856 -11857 -11858 -11859 -11860 -11861 -11862 -11863 -11864 -11865 -11866 -11867 -11868 -11869 -11870 -11871 -11872 -11873 -11874 -11875 -11876 -11877 -11878 -11879 -11880 -11881 -11882 -11883 -11884 -11885 -11886 -11887 -11888 -11889 -11890 -11891 -11892 -11893 -11894 -11895 -11896 -11897 -11898 -11899 -11900 -11901 -11902 -11903 -11904 -11905 -11906 -11907 -11908 -11909 -11910 -11911 -11912 -11913 -11914 -11915 -11916 -11917 -11918 -11919 -11920 -11921 -11922 -11923 -11924 -11925 -11926 -11927 -11928 -11929 -11930 -11931 -11932 -11933 -11934 -11935 -11936 -11937 -11938 -11939 -11940 -11941 -11942 -11943 -11944 -11945 -11946 -11947 -11948 -11949 -11950 -11951 -11952 -11953 -11954 -11955 -11956 -11957 -11958 -11959 -11960 -11961 -11962 -11963 -11964 -11965 -11966 -11967 -11968 -11969 -11970 -11971 -11972 -11973 -11974 -11975 -11976 -11977 -11978 -11979 -11980 -11981 -11982 -11983 -11984 -11985 -11986 -11987 -11988 -11989 -11990 -11991 -11992 -11993 -11994 -11995 -11996 -11997 -11998 -11999 -12000 -12001 -12002 -12003 -12004 -12005 -12006 -12007 -12008 -12009 -12010 -12011 -12012 -12013 -12014 -12015 -12016 -12017 -12018 -12019 -12020 -12021 -12022 -12023 -12024 -12025 -12026 -12027 -12028 -12029 -12030 -12031 -12032 -12033 -12034 -12035 -12036 -12037 -12038 -12039 -12040 -12041 -12042 -12043 -12044 -12045 -12046 -12047 -12048 -12049 -12050 -12051 -12052 -12053 -12054 -12055 -12056 -12057 -12058 -12059 -12060 -12061 -12062 -12063 -12064 -12065 -12066 -12067 -12068 -12069 -12070 -12071 -12072 -12073 -12074 -12075 -12076 -12077 -12078 -12079 -12080 -12081 -12082 -12083 -12084 -12085 -12086 -12087 -12088 -12089 -12090 -12091 -12092 -12093 -12094 -12095 -12096 -12097 -12098 -12099 -12100 -12101 -12102 -12103 -12104 -12105 -12106 -12107 -12108 -12109 -12110 -12111 -12112 -12113 -12114 -12115 -12116 -12117 -12118 -12119 -12120 -12121 -12122 -12123 -12124 -12125 -12126 -12127 -12128 -12129 -12130 -12131 -12132 -12133 -12134 -12135 -12136 -12137 -12138 -12139 -12140 -12141 -12142 -12143 -12144 -12145 -12146 -12147 -12148 -12149 -12150 -12151 -12152 -12153 -12154 -12155 -12156 -12157 -12158 -12159 -12160 -12161 -12162 -12163 -12164 -12165 -12166 -12167 -12168 -12169 -12170 -12171 -12172 -12173 -12174 -12175 -12176 -12177 -12178 -12179 -12180 -12181 -12182 -12183 -12184 -12185 -12186 -12187 -12188 -12189 -12190 -12191 -12192 -12193 -12194 -12195 -12196 -12197 -12198 -12199 -12200 -12201 -12202 -12203 -12204 -12205 -12206 -12207 -12208 -12209 -12210 -12211 -12212 -12213 -12214 -12215 -12216 -12217 -12218 -12219 -12220 -12221 -12222 -12223 -12224 -12225 -12226 -12227 -12228 -12229 -12230 -12231 -12232 -12233 -12234 -12235 -12236 -12237 -12238 -12239 -12240 -12241 -12242 -12243 -12244 -12245 -12246 -12247 -12248 -12249 -12250 -12251 -12252 -12253 -12254 -12255 -12256 -12257 -12258 -12259 -12260 -12261 -12262 -12263 -12264 -12265 -12266 -12267 -12268 -12269 -12270 -12271 -12272 -12273 -12274 -12275 -12276 -12277 -12278 -12279 -12280 -12281 -12282 -12283 -12284 -12285 -12286 -12287 -12288 -12289 -12290 -12291 -12292 -12293 -12294 -12295 -12296 -12297 -12298 -12299 -12300 -12301 -12302 -12303 -12304 -12305 -12306 -12307 -12308 -12309 -12310 -12311 -12312 -12313 -12314 -12315 -12316 -12317 -12318 -12319 -12320 -12321 -12322 -12323 -12324 -12325 -12326 -12327 -12328 -12329 -12330 -12331 -12332 -12333 -12334 -12335 -12336 -12337 -12338 -12339 -12340 -12341 -12342 -12343 -12344 -12345 -12346 -12347 -12348 -12349 -12350 -12351 -12352 -12353 -12354 -12355 -12356 -12357 -12358 -12359 -12360 -12361 -12362 -12363 -12364 -12365 -12366 -12367 -12368 -12369 -12370 -12371 -12372 -12373 -12374 -12375 -12376 -12377 -12378 -12379 -12380 -12381 -12382 -12383 -12384 -12385 -12386 -12387 -12388 -12389 -12390 -12391 -12392 -12393 -12394 -12395 -12396 -12397 -12398 -12399 -12400 -12401 -12402 -12403 -12404 -12405 -12406 -12407 -12408 -12409 -12410 -12411 -12412 -12413 -12414 -12415 -12416 -12417 -12418 -12419 -12420 -12421 -12422 -12423 -12424 -12425 -12426 -12427 -12428 -12429 -12430 -12431 -12432 -12433 -12434 -12435 -12436 -12437 -12438 -12439 -12440 -12441 -12442 -12443 -12444 -12445 -12446 -12447 -12448 -12449 -12450 -12451 -12452 -12453 -12454 -12455 -12456 -12457 -12458 -12459 -12460 -12461 -12462 -12463 -12464 -12465 -12466 -12467 -12468 -12469 -12470 -12471 -12472 -12473 -12474 -12475 -12476 -12477 -12478 -12479 -12480 -12481 -12482 -12483 -12484 -12485 -12486 -12487 -12488 -12489 -12490 -12491 -12492 -12493 -12494 -12495 -12496 -12497 -12498 -12499 -12500 -12501 -12502 -12503 -12504 -12505 -12506 -12507 -12508 -12509 -12510 -12511 -12512 -12513 -12514 -12515 -12516 -12517 -12518 -12519 -12520 -12521 -12522 -12523 -12524 -12525 -12526 -12527 -12528 -12529 -12530 -12531 -12532 -12533 -12534 -12535 -12536 -12537 -12538 -12539 -12540 -12541 -12542 -12543 -12544 -12545 -12546 -12547 -12548 -12549 -12550 -12551 -12552 -12553 -12554 -12555 -12556 -12557 -12558 -12559 -12560 -12561 -12562 -12563 -12564 -12565 -12566 -12567 -12568 -12569 -12570 -12571 -12572 -12573 -12574 -12575 -12576 -12577 -12578 -12579 -12580 -12581 -12582 -12583 -12584 -12585 -12586 -12587 -12588 -12589 -12590 -12591 -12592 -12593 -12594 -12595 -12596 -12597 -12598 -12599 -12600 -12601 -12602 -12603 -12604 -12605 -12606 -12607 -12608 -12609 -12610 -12611 -12612 -12613 -12614 -12615 -12616 -12617 -12618 -12619 -12620 -12621 -12622 -12623 -12624 -12625 -12626 -12627 -12628 -12629 -12630 -12631 -12632 -12633 -12634 -12635 -12636 -12637 -12638 -12639 -12640 -12641 -12642 -12643 -12644 -12645 -12646 -12647 -12648 -12649 -12650 -12651 -12652 -12653 -12654 -12655 -12656 -12657 -12658 -12659 -12660 -12661 -12662 -12663 -12664 -12665 -12666 -12667 -12668 -12669 -12670 -12671 -12672 -12673 -12674 -12675 -12676 -12677 -12678 -12679 -12680 -12681 -12682 -12683 -12684 -12685 -12686 -12687 -12688 -12689 -12690 -12691 -12692 -12693 -12694 -12695 -12696 -12697 -12698 -12699 -12700 -12701 -12702 -12703 -12704 -12705 -12706 -12707 -12708 -12709 -12710 -12711 -12712 -12713 -12714 -12715 -12716 -12717 -12718 -12719 -12720 -12721 -12722 -12723 -12724 -12725 -12726 -12727 -12728 -12729 -12730 -12731 -12732 -12733 -12734 -12735 -12736 -12737 -12738 -12739 -12740 -12741 -12742 -12743 -12744 -12745 -12746 -12747 -12748 -12749 -12750 -12751 -12752 -12753 -12754 -12755 -12756 -12757 -12758 -12759 -12760 -12761 -12762 -12763 -12764 -12765 -12766 -12767 -12768 -12769 -12770 -12771 -12772 -12773 -12774 -12775 -12776 -12777 -12778 -12779 -12780 -12781 -12782 -12783 -12784 -12785 -12786 -12787 -12788 -12789 -12790 -12791 -12792 -12793 -12794 -12795 -12796 -12797 -12798 -12799 -12800 -12801 -12802 -12803 -12804 -12805 -12806 -12807 -12808 -12809 -12810 -12811 -12812 -12813 -12814 -12815 -12816 -12817 -12818 -12819 -12820 -12821 -12822 -12823 -12824 -12825 -12826 -12827 -12828 -12829 -12830 -12831 -12832 -12833 -12834 -12835 -12836 -12837 -12838 -12839 -12840 -12841 -12842 -12843 -12844 -12845 -12846 -12847 -12848 -12849 -12850 -12851 -12852 -12853 -12854 -12855 -12856 -12857 -12858 -12859 -12860 -12861 -12862 -12863 -12864 -12865 -12866 -12867 -12868 -12869 -12870 -12871 -12872 -12873 -12874 -12875 -12876 -12877 -12878 -12879 -12880 -12881 -12882 -12883 -12884 -12885 -12886 -12887 -12888 -12889 -12890 -12891 -12892 -12893 -12894 -12895 -12896 -12897 -12898 -12899 -12900 -12901 -12902 -12903 -12904 -12905 -12906 -12907 -12908 -12909 -12910 -12911 -12912 -12913 -12914 -12915 -12916 -12917 -12918 -12919 -12920 -12921 -12922 -12923 -12924 -12925 -12926 -12927 -12928 -12929 -12930 -12931 -12932 -12933 -12934 -12935 -12936 -12937 -12938 -12939 -12940 -12941 -12942 -12943 -12944 -12945 -12946 -12947 -12948 -12949 -12950 -12951 -12952 -12953 -12954 -12955 -12956 -12957 -12958 -12959 -12960 -12961 -12962 -12963 -12964 -12965 -12966 -12967 -12968 -12969 -12970 -12971 -12972 -12973 -12974 -12975 -12976 -12977 -12978 -12979 -12980 -12981 -12982 -12983 -12984 -12985 -12986 -12987 -12988 -12989 -12990 -12991 -12992 -12993 -12994 -12995 -12996 -12997 -12998 -12999 -13000 -13001 -13002 -13003 -13004 -13005 -13006 -13007 -13008 -13009 -13010 -13011 -13012 -13013 -13014 -13015 -13016 -13017 -13018 -13019 -13020 -13021 -13022 -13023 -13024 -13025 -13026 -13027 -13028 -13029 -13030 -13031 -13032 -13033 -13034 -13035 -13036 -13037 -13038 -13039 -13040 -13041 -13042 -13043 -13044 -13045 -13046 -13047 -13048 -13049 -13050 -13051 -13052 -13053 -13054 -13055 -13056 -13057 -13058 -13059 -13060 -13061 -13062 -13063 -13064 -13065 -13066 -13067 -13068 -13069 -13070 -13071 -13072 -13073 -13074 -13075 -13076 -13077 -13078 -13079 -13080 -13081 -13082 -13083 -13084 -13085 -13086 -13087 -13088 -13089 -13090 -13091 -13092 -13093 -13094 -13095 -13096 -13097 -13098 -13099 -13100 -13101 -13102 -13103 -13104 -13105 -13106 -13107 -13108 -13109 -13110 -13111 -13112 -13113 -13114 -13115 -13116 -13117 -13118 -13119 -13120 -13121 -13122 -13123 -13124 -13125 -13126 -13127 -13128 -13129 -13130 -13131 -13132 -13133 -13134 -13135 -13136 -13137 -13138 -13139 -13140 -13141 -13142 -13143 -13144 -13145 -13146 -13147 -13148 -13149 -13150 -13151 -13152 -13153 -13154 -13155 -13156 -13157 -13158 -13159 -13160 -13161 -13162 -13163 -13164 -13165 -13166 -13167 -13168 -13169 -13170 -13171 -13172 -13173 -13174 -13175 -13176 -13177 -13178 -13179 -13180 -13181 -13182 -13183 -13184 -13185 -13186 -13187 -13188 -13189 -13190 -13191 -13192 -13193 -13194 -13195 -13196 -13197 -13198 -13199 -13200 -13201 -13202 -13203 -13204 -13205 -13206 -13207 -13208 -13209 -13210 -13211 -13212 -13213 -13214 -13215 -13216 -13217 -13218 -13219 -13220 -13221 -13222 -13223 -13224 -13225 -13226 -13227 -13228 -13229 -13230 -13231 -13232 -13233 -13234 -13235 -13236 -13237 -13238 -13239 -13240 -13241 -13242 -13243 -13244 -13245 -13246 -13247 -13248 -13249 -13250 -13251 -13252 -13253 -13254 -13255 -13256 -13257 -13258 -13259 -13260 -13261 -13262 -13263 -13264 -13265 -13266 -13267 -13268 -13269 -13270 -13271 -13272 -13273 -13274 -13275 -13276 -13277 -13278 -13279 -13280 -13281 -13282 -13283 -13284 -13285 -13286 -13287 -13288 -13289 -13290 -13291 -13292 -13293 -13294 -13295 -13296 -13297 -13298 -13299 -13300 -13301 -13302 -13303 -13304 -13305 -13306 -13307 -13308 -13309 -13310 -13311 -13312 -13313 -13314 -13315 -13316 -13317 -13318 -13319 -13320 -13321 -13322 -13323 -13324 -13325 -13326 -13327 -13328 -13329 -13330 -13331 -13332 -13333 -13334 -13335 -13336 -13337 -13338 -13339 -13340 -13341 -13342 -13343 -13344 -13345 -13346 -13347 -13348 -13349 -13350 -13351 -13352 -13353 -13354 -13355 -13356 -13357 -13358 -13359 -13360 -13361 -13362 -13363 -13364 -13365 -13366 -13367 -13368 -13369 -13370 -13371 -13372 -13373 -13374 -13375 -13376 -13377 -13378 -13379 -13380 -13381 -13382 -13383 -13384 -13385 -13386 -13387 -13388 -13389 -13390 -13391 -13392 -13393 -13394 -13395 -13396 -13397 -13398 -13399 -13400 -13401 -13402 -13403 -13404 -13405 -13406 -13407 -13408 -13409 -13410 -13411 -13412 -13413 -13414 -13415 -13416 -13417 -13418 -13419 -13420 -13421 -13422 -13423 -13424 -13425 -13426 -13427 -13428 -13429 -13430 -13431 -13432 -13433 -13434 -13435 -13436 -13437 -13438 -13439 -13440 -13441 -13442 -13443 -13444 -13445 -13446 -13447 -13448 -13449 -13450 -13451 -13452 -13453 -13454 -13455 -13456 -13457 -13458 -13459 -13460 -13461 -13462 -13463 -13464 -13465 -13466 -13467 -13468 -13469 -13470 -13471 -13472 -13473 -13474 -13475 -13476 -13477 -13478 -13479 -13480 -13481 -13482 -13483 -13484 -13485 -13486 -13487 -13488 -13489 -13490 -13491 -13492 -13493 -13494 -13495 -13496 -13497 -13498 -13499 -13500 -13501 -13502 -13503 -13504 -13505 -13506 -13507 -13508 -13509 -13510 -13511 -13512 -13513 -13514 -13515 -13516 -13517 -13518 -13519 -13520 -13521 -13522 -13523 -13524 -13525 -13526 -13527 -13528 -13529 -13530 -13531 -13532 -13533 -13534 -13535 -13536 -13537 -13538 -13539 -13540 -13541 -13542 -13543 -13544 -13545 -13546 -13547 -13548 -13549 -13550 -13551 -13552 -13553 -13554 -13555 -13556 -13557 -13558 -13559 -13560 -13561 -13562 -13563 -13564 -13565 -13566 -13567 -13568 -13569 -13570 -13571 -13572 -13573 -13574 -13575 -13576 -13577 -13578 -13579 -13580 -13581 -13582 -13583 -13584 -13585 -13586 -13587 -13588 -13589 -13590 -13591 -13592 -13593 -13594 -13595 -13596 -13597 -13598 -13599 -13600 -13601 -13602 -13603 -13604 -13605 -13606 -13607 -13608 -13609 -13610 -13611 -13612 -13613 -13614 -13615 -13616 -13617 -13618 -13619 -13620 -13621 -13622 -13623 -13624 -13625 -13626 -13627 -13628 -13629 -13630 -13631 -13632 -13633 -13634 -13635 -13636 -13637 -13638 -13639 -13640 -13641 -13642 -13643 -13644 -13645 -13646 -13647 -13648 -13649 -13650 -13651 -13652 -13653 -13654 -13655 -13656 -13657 -13658 -13659 -13660 -13661 -13662 -13663 -13664 -13665 -13666 -13667 -13668 -13669 -13670 -13671 -13672 -13673 -13674 -13675 -13676 -13677 -13678 -13679 -13680 -13681 -13682 -13683 -13684 -13685 -13686 -13687 -13688 -13689 -13690 -13691 -13692 -13693 -13694 -13695 -13696 -13697 -13698 -13699 -13700 -13701 -13702 -13703 -13704 -13705 -13706 -13707 -13708 -13709 -13710 -13711 -13712 -13713 -13714 -13715 -13716 -13717 -13718 -13719 -13720 -13721 -13722 -13723 -13724 -13725 -13726 -13727 -13728 -13729 -13730 -13731 -13732 -13733 -13734 -13735 -13736 -13737 -13738 -13739 -13740 -13741 -13742 -13743 -13744 -13745 -13746 -13747 -13748 -13749 -13750 -13751 -13752 -13753 -13754 -13755 -13756 -13757 -13758 -13759 -13760 -13761 -13762 -13763 -13764 -13765 -13766 -13767 -13768 -13769 -13770 -13771 -13772 -13773 -13774 -13775 -13776 -13777 -13778 -13779 -13780 -13781 -13782 -13783 -13784 -13785 -13786 -13787 -13788 -13789 -13790 -13791 -13792 -13793 -13794 -13795 -13796 -13797 -13798 -13799 -13800 -13801 -13802 -13803 -13804 -13805 -13806 -13807 -13808 -13809 -13810 -13811 -13812 -13813 -13814 -13815 -13816 -13817 -13818 -13819 -13820 -13821 -13822 -13823 -13824 -13825 -13826 -13827 -13828 -13829 -13830 -13831 -13832 -13833 -13834 -13835 -13836 -13837 -13838 -13839 -13840 -13841 -13842 -13843 -13844 -13845 -13846 -13847 -13848 -13849 -13850 -13851 -13852 -13853 -13854 -13855 -13856 -13857 -13858 -13859 -13860 -13861 -13862 -13863 -13864 -13865 -13866 -13867 -13868 -13869 -13870 -13871 -13872 -13873 -13874 -13875 -13876 -13877 -13878 -13879 -13880 -13881 -13882 -13883 -13884 -13885 -13886 -13887 -13888 -13889 -13890 -13891 -13892 -13893 -13894 -13895 -13896 -13897 -13898 -13899 -13900 -13901 -13902 -13903 -13904 -13905 -13906 -13907 -13908 -13909 -13910 -13911 -13912 -13913 -13914 -13915 -13916 -13917 -13918 -13919 -13920 -13921 -13922 -13923 -13924 -13925 -13926 -13927 -13928 -13929 -13930 -13931 -13932 -13933 -13934 -13935 -13936 -13937 -13938 -13939 -13940 -13941 -13942 -13943 -13944 -13945 -13946 -13947 -13948 -13949 -13950 -13951 -13952 -13953 -13954 -13955 -13956 -13957 -13958 -13959 -13960 -13961 -13962 -13963 -13964 -13965 -13966 -13967 -13968 -13969 -13970 -13971 -13972 -13973 -13974 -13975 -13976 -13977 -13978 -13979 -13980 -13981 -13982 -13983 -13984 -13985 -13986 -13987 -13988 -13989 -13990 -13991 -13992 -13993 -13994 -13995 -13996 -13997 -13998 -13999 -14000 -14001 -14002 -14003 -14004 -14005 -14006 -14007 -14008 -14009 -14010 -14011 -14012 -14013 -14014 -14015 -14016 -14017 -14018 -14019 -14020 -14021 -14022 -14023 -14024 -14025 -14026 -14027 -14028 -14029 -14030 -14031 -14032 -14033 -14034 -14035 -14036 -14037 -14038 -14039 -14040 -14041 -14042 -14043 -14044 -14045 -14046 -14047 -14048 -14049 -14050 -14051 -14052 -14053 -14054 -14055 -14056 -14057 -14058 -14059 -14060 -14061 -14062 -14063 -14064 -14065 -14066 -14067 -14068 -14069 -14070 -14071 -14072 -14073 -14074 -14075 -14076 -14077 -14078 -14079 -14080 -14081 -14082 -14083 -14084 -14085 -14086 -14087 -14088 -14089 -14090 -14091 -14092 -14093 -14094 -14095 -14096 -14097 -14098 -14099 -14100 -14101 -14102 -14103 -14104 -14105 -14106 -14107 -14108 -14109 -14110 -14111 -14112 -14113 -14114 -14115 -14116 -14117 -14118 -14119 -14120 -14121 -14122 -14123 -14124 -14125 -14126 -14127 -14128 -14129 -14130 -14131 -14132 -14133 -14134 -14135 -14136 -14137 -14138 -14139 -14140 -14141 -14142 -14143 -14144 -14145 -14146 -14147 -14148 -14149 -14150 -14151 -14152 -14153 -14154 -14155 -14156 -14157 -14158 -14159 -14160 -14161 -14162 -14163 -14164 -14165 -14166 -14167 -14168 -14169 -14170 -14171 -14172 -14173 -14174 -14175 -14176 -14177 -14178 -14179 -14180 -14181 -14182 -14183 -14184 -14185 -14186 -14187 -14188 -14189 -14190 -14191 -14192 -14193 -14194 -14195 -14196 -14197 -14198 -14199 -14200 -14201 -14202 -14203 -14204 -14205 -14206 -14207 -14208 -14209 -14210 -14211 -14212 -14213 -14214 -14215 -14216 -14217 -14218 -14219 -14220 -14221 -14222 -14223 -14224 -14225 -14226 -14227 -14228 -14229 -14230 -14231 -14232 -14233 -14234 -14235 -14236 -14237 -14238 -14239 -14240 -14241 -14242 -14243 -14244 -14245 -14246 -14247 -14248 -14249 -14250 -14251 -14252 -14253 -14254 -14255 -14256 -14257 -14258 -14259 -14260 -14261 -14262 -14263 -14264 -14265 -14266 -14267 -14268 -14269 -14270 -14271 -14272 -14273 -14274 -14275 -14276 -14277 -14278 -14279 -14280 -14281 -14282 -14283 -14284 -14285 -14286 -14287 -14288 -14289 -14290 -14291 -14292 -14293 -14294 -14295 -14296 -14297 -14298 -14299 -14300 -14301 -14302 -14303 -14304 -14305 -14306 -14307 -14308 -14309 -14310 -14311 -14312 -14313 -14314 -14315 -14316 -14317 -14318 -14319 -14320 -14321 -14322 -14323 -14324 -14325 -14326 -14327 -14328 -14329 -14330 -14331 -14332 -14333 -14334 -14335 -14336 -14337 -14338 -14339 -14340 -14341 -14342 -14343 -14344 -14345 -14346 -14347 -14348 -14349 -14350 -14351 -14352 -14353 -14354 -14355 -14356 -14357 -14358 -14359 -14360 -14361 -14362 -14363 -14364 -14365 -14366 -14367 -14368 -14369 -14370 -14371 -14372 -14373 -14374 -14375 -14376 -14377 -14378 -14379 -14380 -14381 -14382 -14383 -14384 -14385 -14386 -14387 -14388 -14389 -14390 -14391 -14392 -14393 -14394 -14395 -14396 -14397 -14398 -14399 -14400 -14401 -14402 -14403 -14404 -14405 -14406 -14407 -14408 -14409 -14410 -14411 -14412 -14413 -14414 -14415 -14416 -14417 -14418 -14419 -14420 -14421 -14422 -14423 -14424 -14425 -14426 -14427 -14428 -14429 -14430 -14431 -14432 -14433 -14434 -14435 -14436 -14437 -14438 -14439 -14440 -14441 -14442 -14443 -14444 -14445 -14446 -14447 -14448 -14449 -14450 -14451 -14452 -14453 -14454 -14455 -14456 -14457 -14458 -14459 -14460 -14461 -14462 -14463 -14464 -14465 -14466 -14467 -14468 -14469 -14470 -14471 -14472 -14473 -14474 -14475 -14476 -14477 -14478 -14479 -14480 -14481 -14482 -14483 -14484 -14485 -14486 -14487 -14488 -14489 -14490 -14491 -14492 -14493 -14494 -14495 -14496 -14497 -14498 -14499 -14500 -14501 -14502 -14503 -14504 -14505 -14506 -14507 -14508 -14509 -14510 -14511 -14512 -14513 -14514 -14515 -14516 -14517 -14518 -14519 -14520 -14521 -14522 -14523 -14524 -14525 -14526 -14527 -14528 -14529 -14530 -14531 -14532 -14533 -14534 -14535 -14536 -14537 -14538 -14539 -14540 -14541 -14542 -14543 -14544 -14545 -14546 -14547 -14548 -14549 -14550 -14551 -14552 -14553 -14554 -14555 -14556 -14557 -14558 -14559 -14560 -14561 -14562 -14563 -14564 -14565 -14566 -14567 -14568 -14569 -14570 -14571 -14572 -14573 -14574 -14575 -14576 -14577 -14578 -14579 -14580 -14581 -14582 -14583 -14584 -14585 -14586 -14587 -14588 -14589 -14590 -14591 -14592 -14593 -14594 -14595 -14596 -14597 -14598 -14599 -14600 -14601 -14602 -14603 -14604 -14605 -14606 -14607 -14608 -14609 -14610 -14611 -14612 -14613 -14614 -14615 -14616 -14617 -14618 -14619 -14620 -14621 -14622 -14623 -14624 -14625 -14626 -14627 -14628 -14629 -14630 -14631 -14632 -14633 -14634 -14635 -14636 -14637 -14638 -14639 -14640 -14641 -14642 -14643 -14644 -14645 -14646 -14647 -14648 -14649 -14650 -14651 -14652 -14653 -14654 -14655 -14656 -14657 -14658 -14659 -14660 -14661 -14662 -14663 -14664 -14665 -14666 -14667 -14668 -14669 -14670 -14671 -14672 -14673 -14674 -14675 -14676 -14677 -14678 -14679 -14680 -14681 -14682 -14683 -14684 -14685 -14686 -14687 -14688 -14689 -14690 -14691 -14692 -14693 -14694 -14695 -14696 -14697 -14698 -14699 -14700 -14701 -14702 -14703 -14704 -14705 -14706 -14707 -14708 -14709 -14710 -14711 -14712 -14713 -14714 -14715 -14716 -14717 -14718 -14719 -14720 -14721 -14722 -14723 -14724 -14725 -14726 -14727 -14728 -14729 -14730 -14731 -14732 -14733 -14734 -14735 -14736 -14737 -14738 -14739 -14740 -14741 -14742 -14743 -14744 -14745 -14746 -14747 -14748 -14749 -14750 -14751 -14752 -14753 -14754 -14755 -14756 -14757 -14758 -14759 -14760 -14761 -14762 -14763 -14764 -14765 -14766 -14767 -14768 -14769 -14770 -14771 -14772 -14773 -14774 -14775 -14776 -14777 -14778 -14779 -14780 -14781 -14782 -14783 -14784 -14785 -14786 -14787 -14788 -14789 -14790 -14791 -14792 -14793 -14794 -14795 -14796 -14797 -14798 -14799 -14800 -14801 -14802 -14803 -14804 -14805 -14806 -14807 -14808 -14809 -14810 -14811 -14812 -14813 -14814 -14815 -14816 -14817 -14818 -14819 -14820 -14821 -14822 -14823 -14824 -14825 -14826 -14827 -14828 -14829 -14830 -14831 -14832 -14833 -14834 -14835 -14836 -14837 -14838 -14839 -14840 -14841 -14842 -14843 -14844 -14845 -14846 -14847 -14848 -14849 -14850 -14851 -14852 -14853 -14854 -14855 -14856 -14857 -14858 -14859 -14860 -14861 -14862 -14863 -14864 -14865 -14866 -14867 -14868 -14869 -14870 -14871 -14872 -14873 -14874 -14875 -14876 -14877 -14878 -14879 -14880 -14881 -14882 -14883 -14884 -14885 -14886 -14887 -14888 -14889 -14890 -14891 -14892 -14893 -14894 -14895 -14896 -14897 -14898 -14899 -14900 -14901 -14902 -14903 -14904 -14905 -14906 -14907 -14908 -14909 -14910 -14911 -14912 -14913 -14914 -14915 -14916 -14917 -14918 -14919 -14920 -14921 -14922 -14923 -14924 -14925 -14926 -14927 -14928 -14929 -14930 -14931 -14932 -14933 -14934 -14935 -14936 -14937 -14938 -14939 -14940 -14941 -14942 -14943 -14944 -14945 -14946 -14947 -14948 -14949 -14950 -14951 -14952 -14953 -14954 -14955 -14956 -14957 -14958 -14959 -14960 -14961 -14962 -14963 -14964 -14965 -14966 -14967 -14968 -14969 -14970 -14971 -14972 -14973 -14974 -14975 -14976 -14977 -14978 -14979 -14980 -14981 -14982 -14983 -14984 -14985 -14986 -14987 -14988 -14989 -14990 -14991 -14992 -14993 -14994 -14995 -14996 -14997 -14998 -14999 -15000 -15001 -15002 -15003 -15004 -15005 -15006 -15007 -15008 -15009 -15010 -15011 -15012 -15013 -15014 -15015 -15016 -15017 -15018 -15019 -15020 -15021 -15022 -15023 -15024 -15025 -15026 -15027 -15028 -15029 -15030 -15031 -15032 -15033 -15034 -15035 -15036 -15037 -15038 -15039 -15040 -15041 -15042 -15043 -15044 -15045 -15046 -15047 -15048 -15049 -15050 -15051 -15052 -15053 -15054 -15055 -15056 -15057 -15058 -15059 -15060 -15061 -15062 -15063 -15064 -15065 -15066 -15067 -15068 -15069 -15070 -15071 -15072 -15073 -15074 -15075 -15076 -15077 -15078 -15079 -15080 -15081 -15082 -15083 -15084 -15085 -15086 -15087 -15088 -15089 -15090 -15091 -15092 -15093 -15094 -15095 -15096 -15097 -15098 -15099 -15100 -15101 -15102 -15103 -15104 -15105 -15106 -15107 -15108 -15109 -15110 -15111 -15112 -15113 -15114 -15115 -15116 -15117 -15118 -15119 -15120 -15121 -15122 -15123 -15124 -15125 -15126 -15127 -15128 -15129 -15130 -15131 -15132 -15133 -15134 -15135 -15136 -15137 -15138 -15139 -15140 -15141 -15142 -15143 -15144 -15145 -15146 -15147 -15148 -15149 -15150 -15151 -15152 -15153 -15154 -15155 -15156 -15157 -15158 -15159 -15160 -15161 -15162 -15163 -15164 -15165 -15166 -15167 -15168 -15169 -15170 -15171 -15172 -15173 -15174 -15175 -15176 -15177 -15178 -15179 -15180 -15181 -15182 -15183 -15184 -15185 -15186 -15187 -15188 -15189 -15190 -15191 -15192 -15193 -15194 -15195 -15196 -15197 -15198 -15199 -15200 -15201 -15202 -15203 -15204 -15205 -15206 -15207 -15208 -15209 -15210 -15211 -15212 -15213 -15214 -15215 -15216 -15217 -15218 -15219 -15220 -15221 -15222 -15223 -15224 -15225 -15226 -15227 -15228 -15229 -15230 -15231 -15232 -15233 -15234 -15235 -15236 -15237 -15238 -15239 -15240 -15241 -15242 -15243 -15244 -15245 -15246 -15247 -15248 -15249 -15250 -15251 -15252 -15253 -15254 -15255 -15256 -15257 -15258 -15259 -15260 -15261 -15262 -15263 -15264 -15265 -15266 -15267 -15268 -15269 -15270 -15271 -15272 -15273 -15274 -15275 -15276 -15277 -15278 -15279 -15280 -15281 -15282 -15283 -15284 -15285 -15286 -15287 -15288 -15289 -15290 -15291 -15292 -15293 -15294 -15295 -15296 -15297 -15298 -15299 -15300 -15301 -15302 -15303 -15304 -15305 -15306 -15307 -15308 -15309 -15310 -15311 -15312 -15313 -15314 -15315 -15316 -15317 -15318 -15319 -15320 -15321 -15322 -15323 -15324 -15325 -15326 -15327 -15328 -15329 -15330 -15331 -15332 -15333 -15334 -15335 -15336 -15337 -15338 -15339 -15340 -15341 -15342 -15343 -15344 -15345 -15346 -15347 -15348 -15349 -15350 -15351 -15352 -15353 -15354 -15355 -15356 -15357 -15358 -15359 -15360 -15361 -15362 -15363 -15364 -15365 -15366 -15367 -15368 -15369 -15370 -15371 -15372 -15373 -15374 -15375 -15376 -15377 -15378 -15379 -15380 -15381 -15382 -15383 -15384 -15385 -15386 -15387 -15388 -15389 -15390 -15391 -15392 -15393 -15394 -15395 -15396 -15397 -15398 -15399 -15400 -15401 -15402 -15403 -15404 -15405 -15406 -15407 -15408 -15409 -15410 -15411 -15412 -15413 -15414 -15415 -15416 -15417 -15418 -15419 -15420 -15421 -15422 -15423 -15424 -15425 -15426 -15427 -15428 -15429 -15430 -15431 -15432 -15433 -15434 -15435 -15436 -15437 -15438 -15439 -15440 -15441 -15442 -15443 -15444 -15445 -15446 -15447 -15448 -15449 -15450 -15451 -15452 -15453 -15454 -15455 -15456 -15457 -15458 -15459 -15460 -15461 -15462 -15463 -15464 -15465 -15466 -15467 -15468 -15469 -15470 -15471 -15472 -15473 -15474 -15475 -15476 -15477 -15478 -15479 -15480 -15481 -15482 -15483 -15484 -15485 -15486 -15487 -15488 -15489 -15490 -15491 -15492 -15493 -15494 -15495 -15496 -15497 -15498 -15499 -15500 -15501 -15502 -15503 -15504 -15505 -15506 -15507 -15508 -15509 -15510 -15511 -15512 -15513 -15514 -15515 -15516 -15517 -15518 -15519 -15520 -15521 -15522 -15523 -15524 -15525 -15526 -15527 -15528 -15529 -15530 -15531 -15532 -15533 -15534 -15535 -15536 -15537 -15538 -15539 -15540 -15541 -15542 -15543 -15544 -15545 -15546 -15547 -15548 -15549 -15550 -15551 -15552 -15553 -15554 -15555 -15556 -15557 -15558 -15559 -15560 -15561 -15562 -15563 -15564 -15565 -15566 -15567 -15568 -15569 -15570 -15571 -15572 -15573 -15574 -15575 -15576 -15577 -15578 -15579 -15580 -15581 -15582 -15583 -15584 -15585 -15586 -15587 -15588 -15589 -15590 -15591 -15592 -15593 -15594 -15595 -15596 -15597 -15598 -15599 -15600 -15601 -15602 -15603 -15604 -15605 -15606 -15607 -15608 -15609 -15610 -15611 -15612 -15613 -15614 -15615 -15616 -15617 -15618 -15619 -15620 -15621 -15622 -15623 -15624 -15625 -15626 -15627 -15628 -15629 -15630 -15631 -15632 -15633 -15634 -15635 -15636 -15637 -15638 -15639 -15640 -15641 -15642 -15643 -15644 -15645 -15646 -15647 -15648 -15649 -15650 -15651 -15652 -15653 -15654 -15655 -15656 -15657 -15658 -15659 -15660 -15661 -15662 -15663 -15664 -15665 -15666 -15667 -15668 -15669 -15670 -15671 -15672 -15673 -15674 -15675 -15676 -15677 -15678 -15679 -15680 -15681 -15682 -15683 -15684 -15685 -15686 -15687 -15688 -15689 -15690 -15691 -15692 -15693 -15694 -15695 -15696 -15697 -15698 -15699 -15700 -15701 -15702 -15703 -15704 -15705 -15706 -15707 -15708 -15709 -15710 -15711 -15712 -15713 -15714 -15715 -15716 -15717 -15718 -15719 -15720 -15721 -15722 -15723 -15724 -15725 -15726 -15727 -15728 -15729 -15730 -15731 -15732 -15733 -15734 -15735 -15736 -15737 -15738 -15739 -15740 -15741 -15742 -15743 -15744 -15745 -15746 -15747 -15748 -15749 -15750 -15751 -15752 -15753 -15754 -15755 -15756 -15757 -15758 -15759 -15760 -15761 -15762 -15763 -15764 -15765 -15766 -15767 -15768 -15769 -15770 -15771 -15772 -15773 -15774 -15775 -15776 -15777 -15778 -15779 -15780 -15781 -15782 -15783 -15784 -15785 -15786 -15787 -15788 -15789 -15790 -15791 -15792 -15793 -15794 -15795 -15796 -15797 -15798 -15799 -15800 -15801 -15802 -15803 -15804 -15805 -15806 -15807 -15808 -15809 -15810 -15811 -15812 -15813 -15814 -15815 -15816 -15817 -15818 -15819 -15820 -15821 -15822 -15823 -15824 -15825 -15826 -15827 -15828 -15829 -15830 -15831 -15832 -15833 -15834 -15835 -15836 -15837 -15838 -15839 -15840 -15841 -15842 -15843 -15844 -15845 -15846 -15847 -15848 -15849 -15850 -15851 -15852 -15853 -15854 -15855 -15856 -15857 -15858 -15859 -15860 -15861 -15862 -15863 -15864 -15865 -15866 -15867 -15868 -15869 -15870 -15871 -15872 -15873 -15874 -15875 -15876 -15877 -15878 -15879 -15880 -15881 -15882 -15883 -15884 -15885 -15886 -15887 -15888 -15889 -15890 -15891 -15892 -15893 -15894 -15895 -15896 -15897 -15898 -15899 -15900 -15901 -15902 -15903 -15904 -15905 -15906 -15907 -15908 -15909 -15910 -15911 -15912 -15913 -15914 -15915 -15916 -15917 -15918 -15919 -15920 -15921 -15922 -15923 -15924 -15925 -15926 -15927 -15928 -15929 -15930 -15931 -15932 -15933 -15934 -15935 -15936 -15937 -15938 -15939 -15940 -15941 -15942 -15943 -15944 -15945 -15946 -15947 -15948 -15949 -15950 -15951 -15952 -15953 -15954 -15955 -15956 -15957 -15958 -15959 -15960 -15961 -15962 -15963 -15964 -15965 -15966 -15967 -15968 -15969 -15970 -15971 -15972 -15973 -15974 -15975 -15976 -15977 -15978 -15979 -15980 -15981 -15982 -15983 -15984 -15985 -15986 -15987 -15988 -15989 -15990 -15991 -15992 -15993 -15994 -15995 -15996 -15997 -15998 -15999 -16000 -16001 -16002 -16003 -16004 -16005 -16006 -16007 -16008 -16009 -16010 -16011 -16012 -16013 -16014 -16015 -16016 -16017 -16018 -16019 -16020 -16021 -16022 -16023 -16024 -16025 -16026 -16027 -16028 -16029 -16030 -16031 -16032 -16033 -16034 -16035 -16036 -16037 -16038 -16039 -16040 -16041 -16042 -16043 -16044 -16045 -16046 -16047 -16048 -16049 -16050 -16051 -16052 -16053 -16054 -16055 -16056 -16057 -16058 -16059 -16060 -16061 -16062 -16063 -16064 -16065 -16066 -16067 -16068 -16069 -16070 -16071 -16072 -16073 -16074 -16075 -16076 -16077 -16078 -16079 -16080 -16081 -16082 -16083 -16084 -16085 -16086 -16087 -16088 -16089 -16090 -16091 -16092 -16093 -16094 -16095 -16096 -16097 -16098 -16099 -16100 -16101 -16102 -16103 -16104 -16105 -16106 -16107 -16108 -16109 -16110 -16111 -16112 -16113 -16114 -16115 -16116 -16117 -16118 -16119 -16120 -16121 -16122 -16123 -16124 -16125 -16126 -16127 -16128 -16129 -16130 -16131 -16132 -16133 -16134 -16135 -16136 -16137 -16138 -16139 -16140 -16141 -16142 -16143 -16144 -16145 -16146 -16147 -16148 -16149 -16150 -16151 -16152 -16153 -16154 -16155 -16156 -16157 -16158 -16159 -16160 -16161 -16162 -16163 -16164 -16165 -16166 -16167 -16168 -16169 -16170 -16171 -16172 -16173 -16174 -16175 -16176 -16177 -16178 -16179 -16180 -16181 -16182 -16183 -16184 -16185 -16186 -16187 -16188 -16189 -16190 -16191 -16192 -16193 -16194 -16195 -16196 -16197 -16198 -16199 -16200 -16201 -16202 -16203 -16204 -16205 -16206 -16207 -16208 -16209 -16210 -16211 -16212 -16213 -16214 -16215 -16216 -16217 -16218 -16219 -16220 -16221 -16222 -16223 -16224 -16225 -16226 -16227 -16228 -16229 -16230 -16231 -16232 -16233 -16234 -16235 -16236 -16237 -16238 -16239 -16240 -16241 -16242 -16243 -16244 -16245 -16246 -16247 -16248 -16249 -16250 -16251 -16252 -16253 -16254 -16255 -16256 -16257 -16258 -16259 -16260 -16261 -16262 -16263 -16264 -16265 -16266 -16267 -16268 -16269 -16270 -16271 -16272 -16273 -16274 -16275 -16276 -16277 -16278 -16279 -16280 -16281 -16282 -16283 -16284 -16285 -16286 -16287 -16288 -16289 -16290 -16291 -16292 -16293 -16294 -16295 -16296 -16297 -16298 -16299 -16300 -16301 -16302 -16303 -16304 -16305 -16306 -16307 -16308 -16309 -16310 -16311 -16312 -16313 -16314 -16315 -16316 -16317 -16318 -16319 -16320 -16321 -16322 -16323 -16324 -16325 -16326 -16327 -16328 -16329 -16330 -16331 -16332 -16333 -16334 -16335 -16336 -16337 -16338 -16339 -16340 -16341 -16342 -16343 -16344 -16345 -16346 -16347 -16348 -16349 -16350 -16351 -16352 -16353 -16354 -16355 -16356 -16357 -16358 -16359 -16360 -16361 -16362 -16363 -16364 -16365 -16366 -16367 -16368 -16369 -16370 -16371 -16372 -16373 -16374 -16375 -16376 -16377 -16378 -16379 -16380 -16381 -16382 -16383 -16384 -16385 -16386 -16387 -16388 -16389 -16390 -16391 -16392 -16393 -16394 -16395 -16396 -16397 -16398 -16399 -16400 -16401 -16402 -16403 -16404 -16405 -16406 -16407 -16408 -16409 -16410 -16411 -16412 -16413 -16414 -16415 -16416 -16417 -16418 -16419 -16420 -16421 -16422 -16423 -16424 -16425 -16426 -16427 -16428 -16429 -16430 -16431 -16432 -16433 -16434 -16435 -16436 -16437 -16438 -16439 -16440 -16441 -16442 -16443 -16444 -16445 -16446 -16447 -16448 -16449 -16450 -16451 -16452 -16453 -16454 -16455 -16456 -16457 -16458 -16459 -16460 -16461 -16462 -16463 -16464 -16465 -16466 -16467 -16468 -16469 -16470 -16471 -16472 -16473 -16474 -16475 -16476 -16477 -16478 -16479 -16480 -16481 -16482 -16483 -16484 -16485 -16486 -16487 -16488 -16489 -16490 -16491 -16492 -16493 -16494 -16495 -16496 -16497 -16498 -16499 -16500 -16501 -16502 -16503 -16504 -16505 -16506 -16507 -16508 -16509 -16510 -16511 -16512 -16513 -16514 -16515 -16516 -16517 -16518 -16519 -16520 -16521 -16522 -16523 -16524 -16525 -16526 -16527 -16528 -16529 -16530 -16531 -16532 -16533 -16534 -16535 -16536 -16537 -16538 -16539 -16540 -16541 -16542 -16543 -16544 -16545 -16546 -16547 -16548 -16549 -16550 -16551 -16552 -16553 -16554 -16555 -16556 -16557 -16558 -16559 -16560 -16561 -16562 -16563 -16564 -16565 -16566 -16567 -16568 -16569 -16570 -16571 -16572 -16573 -16574 -16575 -16576 -16577 -16578 -16579 -16580 -16581 -16582 -16583 -16584 -16585 -16586 -16587 -16588 -16589 -16590 -16591 -16592 -16593 -16594 -16595 -16596 -16597 -16598 -16599 -16600 -16601 -16602 -16603 -16604 -16605 -16606 -16607 -16608 -16609 -16610 -16611 -16612 -16613 -16614 -16615 -16616 -16617 -16618 -16619 -16620 -16621 -16622 -16623 -16624 -16625 -16626 -16627 -16628 -16629 -16630 -16631 -16632 -16633 -16634 -16635 -16636 -16637 -16638 -16639 -16640 -16641 -16642 -16643 -16644 -16645 -16646 -16647 -16648 -16649 -16650 -16651 -16652 -16653 -16654 -16655 -16656 -16657 -16658 -16659 -16660 -16661 -16662 -16663 -16664 -16665 -16666 -16667 -16668 -16669 -16670 -16671 -16672 -16673 -16674 -16675 -16676 -16677 -16678 -16679 -16680 -16681 -16682 -16683 -16684 -16685 -16686 -16687 -16688 -16689 -16690 -16691 -16692 -16693 -16694 -16695 -16696 -16697 -16698 -16699 -16700 -16701 -16702 -16703 -16704 -16705 -16706 -16707 -16708 -16709 -16710 -16711 -16712 -16713 -16714 -16715 -16716 -16717 -16718 -16719 -16720 -16721 -16722 -16723 -16724 -16725 -16726 -16727 -16728 -16729 -16730 -16731 -16732 -16733 -16734 -16735 -16736 -16737 -16738 -16739 -16740 -16741 -16742 -16743 -16744 -16745 -16746 -16747 -16748 -16749 -16750 -16751 -16752 -16753 -16754 -16755 -16756 -16757 -16758 -16759 -16760 -16761 -16762 -16763 -16764 -16765 -16766 -16767 -16768 -16769 -16770 -16771 -16772 -16773 -16774 -16775 -16776 -16777 -16778 -16779 -16780 -16781 -16782 -16783 -16784 -16785 -16786 -16787 -16788 -16789 -16790 -16791 -16792 -16793 -16794 -16795 -16796 -16797 -16798 -16799 -16800 -16801 -16802 -16803 -16804 -16805 -16806 -16807 -16808 -16809 -16810 -16811 -16812 -16813 -16814 -16815 -16816 -16817 -16818 -16819 -16820 -16821 -16822 -16823 -16824 -16825 -16826 -16827 -16828 -16829 -16830 -16831 -16832 -16833 -16834 -16835 -16836 -16837 -16838 -16839 -16840 -16841 -16842 -16843 -16844 -16845 -16846 -16847 -16848 -16849 -16850 -16851 -16852 -16853 -16854 -16855 -16856 -16857 -16858 -16859 -16860 -16861 -16862 -16863 -16864 -16865 -16866 -16867 -16868 -16869 -16870 -16871 -16872 -16873 -16874 -16875 -16876 -16877 -16878 -16879 -16880 -16881 -16882 -16883 -16884 -16885 -16886 -16887 -16888 -16889 -16890 -16891 -16892 -16893 -16894 -16895 -16896 -16897 -16898 -16899 -16900 -16901 -16902 -16903 -16904 -16905 -16906 -16907 -16908 -16909 -16910 -16911 -16912 -16913 -16914 -16915 -16916 -16917 -16918 -16919 -16920 -16921 -16922 -16923 -16924 -16925 -16926 -16927 -16928 -16929 -16930 -16931 -16932 -16933 -16934 -16935 -16936 -16937 -16938 -16939 -16940 -16941 -16942 -16943 -16944 -16945 -16946 -16947 -16948 -16949 -16950 -16951 -16952 -16953 -16954 -16955 -16956 -16957 -16958 -16959 -16960 -16961 -16962 -16963 -16964 -16965 -16966 -16967 -16968 -16969 -16970 -16971 -16972 -16973 -16974 -16975 -16976 -16977 -16978 -16979 -16980 -16981 -16982 -16983 -16984 -16985 -16986 -16987 -16988 -16989 -16990 -16991 -16992 -16993 -16994 -16995 -16996 -16997 -16998 -16999 -17000 -17001 -17002 -17003 -17004 -17005 -17006 -17007 -17008 -17009 -17010 -17011 -17012 -17013 -17014 -17015 -17016 -17017 -17018 -17019 -17020 -17021 -17022 -17023 -17024 -17025 -17026 -17027 -17028 -17029 -17030 -17031 -17032 -17033 -17034 -17035 -17036 -17037 -17038 -17039 -17040 -17041 -17042 -17043 -17044 -17045 -17046 -17047 -17048 -17049 -17050 -17051 -17052 -17053 -17054 -17055 -17056 -17057 -17058 -17059 -17060 -17061 -17062 -17063 -17064 -17065 -17066 -17067 -17068 -17069 -17070 -17071 -17072 -17073 -17074 -17075 -17076 -17077 -17078 -17079 -17080 -17081 -17082 -17083 -17084 -17085 -17086 -17087 -17088 -17089 -17090 -17091 -17092 -17093 -17094 -17095 -17096 -17097 -17098 -17099 -17100 -17101 -17102 -17103 -17104 -17105 -17106 -17107 -17108 -17109 -17110 -17111 -17112 -17113 -17114 -17115 -17116 -17117 -17118 -17119 -17120 -17121 -17122 -17123 -17124 -17125 -17126 -17127 -17128 -17129 -17130 -17131 -17132 -17133 -17134 -17135 -17136 -17137 -17138 -17139 -17140 -17141 -17142 -17143 -17144 -17145 -17146 -17147 -17148 -17149 -17150 -17151 -17152 -17153 -17154 -17155 -17156 -17157 -17158 -17159 -17160 -17161 -17162 -17163 -17164 -17165 -17166 -17167 -17168 -17169 -17170 -17171 -17172 -17173 -17174 -17175 -17176 -17177 -17178 -17179 -17180 -17181 -17182 -17183 -17184 -17185 -17186 -17187 -17188 -17189 -17190 -17191 -17192 -17193 -17194 -17195 -17196 -17197 -17198 -17199 -17200 -17201 -17202 -17203 -17204 -17205 -17206 -17207 -17208 -17209 -17210 -17211 -17212 -17213 -17214 -17215 -17216 -17217 -17218 -17219 -17220 -17221 -17222 -17223 -17224 -17225 -17226 -17227 -17228 -17229 -17230 -17231 -17232 -17233 -17234 -17235 -17236 -17237 -17238 -17239 -17240 -17241 -17242 -17243 -17244 -17245 -17246 -17247 -17248 -17249 -17250 -17251 -17252 -17253 -17254 -17255 -17256 -17257 -17258 -17259 -17260 -17261 -17262 -17263 -17264 -17265 -17266 -17267 -17268 -17269 -17270 -17271 -17272 -17273 -17274 -17275 -17276 -17277 -17278 -17279 -17280 -17281 -17282 -17283 -17284 -17285 -17286 -17287 -17288 -17289 -17290 -17291 -17292 -17293 -17294 -17295 -17296 -17297 -17298 -17299 -17300 -17301 -17302 -17303 -17304 -17305 -17306 -17307 -17308 -17309 -17310 -17311 -17312 -17313 -17314 -17315 -17316 -17317 -17318 -17319 -17320 -17321 -17322 -17323 -17324 -17325 -17326 -17327 -17328 -17329 -17330 -17331 -17332 -17333 -17334 -17335 -17336 -17337 -17338 -17339 -17340 -17341 -17342 -17343 -17344 -17345 -17346 -17347 -17348 -17349 -17350 -17351 -17352 -17353 -17354 -17355 -17356 -17357 -17358 -17359 -17360 -17361 -17362 -17363 -17364 -17365 -17366 -17367 -17368 -17369 -17370 -17371 -17372 -17373 -17374 -17375 -17376 -17377 -17378 -17379 -17380 -17381 -17382 -17383 -17384 -17385 -17386 -17387 -17388 -17389 -17390 -17391 -17392 -17393 -17394 -17395 -17396 -17397 -17398 -17399 -17400 -17401 -17402 -17403 -17404 -17405 -17406 -17407 -17408 -17409 -17410 -17411 -17412 -17413 -17414 -17415 -17416 -17417 -17418 -17419 -17420 -17421 -17422 -17423 -17424 -17425 -17426 -17427 -17428 -17429 -17430 -17431 -17432 -17433 -17434 -17435 -17436 -17437 -17438 -17439 -17440 -17441 -17442 -17443 -17444 -17445 -17446 -17447 -17448 -17449 -17450 -17451 -17452 -17453 -17454 -17455 -17456 -17457 -17458 -17459 -17460 -17461 -17462 -17463 -17464 -17465 -17466 -17467 -17468 -17469 -17470 -17471 -17472 -17473 -17474 -17475 -17476 -17477 -17478 -17479 -17480 -17481 -17482 -17483 -17484 -17485 -17486 -17487 -17488 -17489 -17490 -17491 -17492 -17493 -17494 -17495 -17496 -17497 -17498 -17499 -17500 -17501 -17502 -17503 -17504 -17505 -17506 -17507 -17508 -17509 -17510 -17511 -17512 -17513 -17514 -17515 -17516 -17517 -17518 -17519 -17520 -17521 -17522 -17523 -17524 -17525 -17526 -17527 -17528 -17529 -17530 -17531 -17532 -17533 -17534 -17535 -17536 -17537 -17538 -17539 -17540 -17541 -17542 -17543 -17544 -17545 -17546 -17547 -17548 -17549 -17550 -17551 -17552 -17553 -17554 -17555 -17556 -17557 -17558 -17559 -17560 -17561 -17562 -17563 -17564 -17565 -17566 -17567 -17568 -17569 -17570 -17571 -17572 -17573 -17574 -17575 -17576 -17577 -17578 -17579 -17580 -17581 -17582 -17583 -17584 -17585 -17586 -17587 -17588 -17589 -17590 -17591 -17592 -17593 -17594 -17595 -17596 -17597 -17598 -17599 -17600 -17601 -17602 -17603 -17604 -17605 -17606 -17607 -17608 -17609 -17610 -17611 -17612 -17613 -17614 -17615 -17616 -17617 -17618 -17619 -17620 -17621 -17622 -17623 -17624 -17625 -17626 -17627 -17628 -17629 -17630 -17631 -17632 -17633 -17634 -17635 -17636 -17637 -17638 -17639 -17640 -17641 -17642 -17643 -17644 -17645 -17646 -17647 -17648 -17649 -17650 -17651 -17652 -17653 -17654 -17655 -17656 -17657 -17658 -17659 -17660 -17661 -17662 -17663 -17664 -17665 -17666 -17667 -17668 -17669 -17670 -17671 -17672 -17673 -17674 -17675 -17676 -17677 -17678 -17679 -17680 -17681 -17682 -17683 -17684 -17685 -17686 -17687 -17688 -17689 -17690 -17691 -17692 -17693 -17694 -17695 -17696 -17697 -17698 -17699 -17700 -17701 -17702 -17703 -17704 -17705 -17706 -17707 -17708 -17709 -17710 -17711 -17712 -17713 -17714 -17715 -17716 -17717 -17718 -17719 -17720 -17721 -17722 -17723 -17724 -17725 -17726 -17727 -17728 -17729 -17730 -17731 -17732 -17733 -17734 -17735 -17736 -17737 -17738 -17739 -17740 -17741 -17742 -17743 -17744 -17745 -17746 -17747 -17748 -17749 -17750 -17751 -17752 -17753 -17754 -17755 -17756 -17757 -17758 -17759 -17760 -17761 -17762 -17763 -17764 -17765 -17766 -17767 -17768 -17769 -17770 -17771 -17772 -17773 -17774 -17775 -17776 -17777 -17778 -17779 -17780 -17781 -17782 -17783 -17784 -17785 -17786 -17787 -17788 -17789 -17790 -17791 -17792 -17793 -17794 -17795 -17796 -17797 -17798 -17799 -17800 -17801 -17802 -17803 -17804 -17805 -17806 -17807 -17808 -17809 -17810 -17811 -17812 -17813 -17814 -17815 -17816 -17817 -17818 -17819 -17820 -17821 -17822 -17823 -17824 -17825 -17826 -17827 -17828 -17829 -17830 -17831 -17832 -17833 -17834 -17835 -17836 -17837 -17838 -17839 -17840 -17841 -17842 -17843 -17844 -17845 -17846 -17847 -17848 -17849 -17850 -17851 -17852 -17853 -17854 -17855 -17856 -17857 -17858 -17859 -17860 -17861 -17862 -17863 -17864 -17865 -17866 -17867 -17868 -17869 -17870 -17871 -17872 -17873 -17874 -17875 -17876 -17877 -17878 -17879 -17880 -17881 -17882 -17883 -17884 -17885 -17886 -17887 -17888 -17889 -17890 -17891 -17892 -17893 -17894 -17895 -17896 -17897 -17898 -17899 -17900 -17901 -17902 -17903 -17904 -17905 -17906 -17907 -17908 -17909 -17910 -17911 -17912 -17913 -17914 -17915 -17916 -17917 -17918 -17919 -17920 -17921 -17922 -17923 -17924 -17925 -17926 -17927 -17928 -17929 -17930 -17931 -17932 -17933 -17934 -17935 -17936 -17937 -17938 -17939 -17940 -17941 -17942 -17943 -17944 -17945 -17946 -17947 -17948 -17949 -17950 -17951 -17952 -17953 -17954 -17955 -17956 -17957 -17958 -17959 -17960 -17961 -17962 -17963 -17964 -17965 -17966 -17967 -17968 -17969 -17970 -17971 -17972 -17973 -17974 -17975 -17976 -17977 -17978 -17979 -17980 -17981 -17982 -17983 -17984 -17985 -17986 -17987 -17988 -17989 -17990 -17991 -17992 -17993 -17994 -17995 -17996 -17997 -17998 -17999 -18000 -18001 -18002 -18003 -18004 -18005 -18006 -18007 -18008 -18009 -18010 -18011 -18012 -18013 -18014 -18015 -18016 -18017 -18018 -18019 -18020 -18021 -18022 -18023 -18024 -18025 -18026 -18027 -18028 -18029 -18030 -18031 -18032 -18033 -18034 -18035 -18036 -18037 -18038 -18039 -18040 -18041 -18042 -18043 -18044 -18045 -18046 -18047 -18048 -18049 -18050 -18051 -18052 -18053 -18054 -18055 -18056 -18057 -18058 -18059 -18060 -18061 -18062 -18063 -18064 -18065 -18066 -18067 -18068 -18069 -18070 -18071 -18072 -18073 -18074 -18075 -18076 -18077 -18078 -18079 -18080 -18081 -18082 -18083 -18084 -18085 -18086 -18087 -18088 -18089 -18090 -18091 -18092 -18093 -18094 -18095 -18096 -18097 -18098 -18099 -18100 -18101 -18102 -18103 -18104 -18105 -18106 -18107 -18108 -18109 -18110 -18111 -18112 -18113 -18114 -18115 -18116 -18117 -18118 -18119 -18120 -18121 -18122 -18123 -18124 -18125 -18126 -18127 -18128 -18129 -18130 -18131 -18132 -18133 -18134 -18135 -18136 -18137 -18138 -18139 -18140 -18141 -18142 -18143 -18144 -18145 -18146 -18147 -18148 -18149 -18150 -18151 -18152 -18153 -18154 -18155 -18156 -18157 -18158 -18159 -18160 -18161 -18162 -18163 -18164 -18165 -18166 -18167 -18168 -18169 -18170 -18171 -18172 -18173 -18174 -18175 -18176 -18177 -18178 -18179 -18180 -18181 -18182 -18183 -18184 -18185 -18186 -18187 -18188 -18189 -18190 -18191 -18192 -18193 -18194 -18195 -18196 -18197 -18198 -18199 -18200 -18201 -18202 -18203 -18204 -18205 -18206 -18207 -18208 -18209 -18210 -18211 -18212 -18213 -18214 -18215 -18216 -18217 -18218 -18219 -18220 -18221 -18222 -18223 -18224 -18225 -18226 -18227 -18228 -18229 -18230 -18231 -18232 -18233 -18234 -18235 -18236 -18237 -18238 -18239 -18240 -18241 -18242 -18243 -18244 -18245 -18246 -18247 -18248 -18249 -18250 -18251 -18252 -18253 -18254 -18255 -18256 -18257 -18258 -18259 -18260 -18261 -18262 -18263 -18264 -18265 -18266 -18267 -18268 -18269 -18270 -18271 -18272 -18273 -18274 -18275 -18276 -18277 -18278 -18279 -18280 -18281 -18282 -18283 -18284 -18285 -18286 -18287 -18288 -18289 -18290 -18291 -18292 -18293 -18294 -18295 -18296 -18297 -18298 -18299 -18300 -18301 -18302 -18303 -18304 -18305 -18306 -18307 -18308 -18309 -18310 -18311 -18312 -18313 -18314 -18315 -18316 -18317 -18318 -18319 -18320 -18321 -18322 -18323 -18324 -18325 -18326 -18327 -18328 -18329 -18330 -18331 -18332 -18333 -18334 -18335 -18336 -18337 -18338 -18339 -18340 -18341 -18342 -18343 -18344 -18345 -18346 -18347 -18348 -18349 -18350 -18351 -18352 -18353 -18354 -18355 -18356 -18357 -18358 -18359 -18360 -18361 -18362 -18363 -18364 -18365 -18366 -18367 -18368 -18369 -18370 -18371 -18372 -18373 -18374 -18375 -18376 -18377 -18378 -18379 -18380 -18381 -18382 -18383 -18384 -18385 -18386 -18387 -18388 -18389 -18390 -18391 -18392 -18393 -18394 -18395 -18396 -18397 -18398 -18399 -18400 -18401 -18402 -18403 -18404 -18405 -18406 -18407 -18408 -18409 -18410 -18411 -18412 -18413 -18414 -18415 -18416 -18417 -18418 -18419 -18420 -18421 -18422 -18423 -18424 -18425 -18426 -18427 -18428 -18429 -18430 -18431 -18432 -18433 -18434 -18435 -18436 -18437 -18438 -18439 -18440 -18441 -18442 -18443 -18444 -18445 -18446 -18447 -18448 -18449 -18450 -18451 -18452 -18453 -18454 -18455 -18456 -18457 -18458 -18459 -18460 -18461 -18462 -18463 -18464 -18465 -18466 -18467 -18468 -18469 -18470 -18471 -18472 -18473 -18474 -18475 -18476 -18477 -18478 -18479 -18480 -18481 -18482 -18483 -18484 -18485 -18486 -18487 -18488 -18489 -18490 -18491 -18492 -18493 -18494 -18495 -18496 -18497 -18498 -18499 -18500 -18501 -18502 -18503 -18504 -18505 -18506 -18507 -18508 -18509 -18510 -18511 -18512 -18513 -18514 -18515 -18516 -18517 -18518 -18519 -18520 -18521 -18522 -18523 -18524 -18525 -18526 -18527 -18528 -18529 -18530 -18531 -18532 -18533 -18534 -18535 -18536 -18537 -18538 -18539 -18540 -18541 -18542 -18543 -18544 -18545 -18546 -18547 -18548 -18549 -18550 -18551 -18552 -18553 -18554 -18555 -18556 -18557 -18558 -18559 -18560 -18561 -18562 -18563 -18564 -18565 -18566 -18567 -18568 -18569 -18570 -18571 -18572 -18573 -18574 -18575 -18576 -18577 -18578 -18579 -18580 -18581 -18582 -18583 -18584 -18585 -18586 -18587 -18588 -18589 -18590 -18591 -18592 -18593 -18594 -18595 -18596 -18597 -18598 -18599 -18600 -18601 -18602 -18603 -18604 -18605 -18606 -18607 -18608 -18609 -18610 -18611 -18612 -18613 -18614 -18615 -18616 -18617 -18618 -18619 -18620 -18621 -18622 -18623 -18624 -18625 -18626 -18627 -18628 -18629 -18630 -18631 -18632 -18633 -18634 -18635 -18636 -18637 -18638 -18639 -18640 -18641 -18642 -18643 -18644 -18645 -18646 -18647 -18648 -18649 -18650 -18651 -18652 -18653 -18654 -18655 -18656 -18657 -18658 -18659 -18660 -18661 -18662 -18663 -18664 -18665 -18666 -18667 -18668 -18669 -18670 -18671 -18672 -18673 -18674 -18675 -18676 -18677 -18678 -18679 -18680 -18681 -18682 -18683 -18684 -18685 -18686 -18687 -18688 -18689 -18690 -18691 -18692 -18693 -18694 -18695 -18696 -18697 -18698 -18699 -18700 -18701 -18702 -18703 -18704 -18705 -18706 -18707 -18708 -18709 -18710 -18711 -18712 -18713 -18714 -18715 -18716 -18717 -18718 -18719 -18720 -18721 -18722 -18723 -18724 -18725 -18726 -18727 -18728 -18729 -18730 -18731 -18732 -18733 -18734 -18735 -18736 -18737 -18738 -18739 -18740 -18741 -18742 -18743 -18744 -18745 -18746 -18747 -18748 -18749 -18750 -18751 -18752 -18753 -18754 -18755 -18756 -18757 -18758 -18759 -18760 -18761 -18762 -18763 -18764 -18765 -18766 -18767 -18768 -18769 -18770 -18771 -18772 -18773 -18774 -18775 -18776 -18777 -18778 -18779 -18780 -18781 -18782 -18783 -18784 -18785 -18786 -18787 -18788 -18789 -18790 -18791 -18792 -18793 -18794 -18795 -18796 -18797 -18798 -18799 -18800 -18801 -18802 -18803 -18804 -18805 -18806 -18807 -18808 -18809 -18810 -18811 -18812 -18813 -18814 -18815 -18816 -18817 -18818 -18819 -18820 -18821 -18822 -18823 -18824 -18825 -18826 -18827 -18828 -18829 -18830 -18831 -18832 -18833 -18834 -18835 -18836 -18837 -18838 -18839 -18840 -18841 -18842 -18843 -18844 -18845 -18846 -18847 -18848 -18849 -18850 -18851 -18852 -18853 -18854 -18855 -18856 -18857 -18858 -18859 -18860 -18861 -18862 -18863 -18864 -18865 -18866 -18867 -18868 -18869 -18870 -18871 -18872 -18873 -18874 -18875 -18876 -18877 -18878 -18879 -18880 -18881 -18882 -18883 -18884 -18885 -18886 -18887 -18888 -18889 -18890 -18891 -18892 -18893 -18894 -18895 -18896 -18897 -18898 -18899 -18900 -18901 -18902 -18903 -18904 -18905 -18906 -18907 -18908 -18909 -18910 -18911 -18912 -18913 -18914 -18915 -18916 -18917 -18918 -18919 -18920 -18921 -18922 -18923 -18924 -18925 -18926 -18927 -18928 -18929 -18930 -18931 -18932 -18933 -18934 -18935 -18936 -18937 -18938 -18939 -18940 -18941 -18942 -18943 -18944 -18945 -18946 -18947 -18948 -18949 -18950 -18951 -18952 -18953 -18954 -18955 -18956 -18957 -18958 -18959 -18960 -18961 -18962 -18963 -18964 -18965 -18966 -18967 -18968 -18969 -18970 -18971 -18972 -18973 -18974 -18975 -18976 -18977 -18978 -18979 -18980 -18981 -18982 -18983 -18984 -18985 -18986 -18987 -18988 -18989 -18990 -18991 -18992 -18993 -18994 -18995 -18996 -18997 -18998 -18999 -19000 -19001 -19002 -19003 -19004 -19005 -19006 -19007 -19008 -19009 -19010 -19011 -19012 -19013 -19014 -19015 -19016 -19017 -19018 -19019 -19020 -19021 -19022 -19023 -19024 -19025 -19026 -19027 -19028 -19029 -19030 -19031 -19032 -19033 -19034 -19035 -19036 -19037 -19038 -19039 -19040 -19041 -19042 -19043 -19044 -19045 -19046 -19047 -19048 -19049 -19050 -19051 -19052 -19053 -19054 -19055 -19056 -19057 -19058 -19059 -19060 -19061 -19062 -19063 -19064 -19065 -19066 -19067 -19068 -19069 -19070 -19071 -19072 -19073 -19074 -19075 -19076 -19077 -19078 -19079 -19080 -19081 -19082 -19083 -19084 -19085 -19086 -19087 -19088 -19089 -19090 -19091 -19092 -19093 -19094 -19095 -19096 -19097 -19098 -19099 -19100 -19101 -19102 -19103 -19104 -19105 -19106 -19107 -19108 -19109 -19110 -19111 -19112 -19113 -19114 -19115 -19116 -19117 -19118 -19119 -19120 -19121 -19122 -19123 -19124 -19125 -19126 -19127 -19128 -19129 -19130 -19131 -19132 -19133 -19134 -19135 -19136 -19137 -19138 -19139 -19140 -19141 -19142 -19143 -19144 -19145 -19146 -19147 -19148 -19149 -19150 -19151 -19152 -19153 -19154 -19155 -19156 -19157 -19158 -19159 -19160 -19161 -19162 -19163 -19164 -19165 -19166 -19167 -19168 -19169 -19170 -19171 -19172 -19173 -19174 -19175 -19176 -19177 -19178 -19179 -19180 -19181 -19182 -19183 -19184 -19185 -19186 -19187 -19188 -19189 -19190 -19191 -19192 -19193 -19194 -19195 -19196 -19197 -19198 -19199 -19200 -19201 -19202 -19203 -19204 -19205 -19206 -19207 -19208 -19209 -19210 -19211 -19212 -19213 -19214 -19215 -19216 -19217 -19218 -19219 -19220 -19221 -19222 -19223 -19224 -19225 -19226 -19227 -19228 -19229 -19230 -19231 -19232 -19233 -19234 -19235 -19236 -19237 -19238 -19239 -19240 -19241 -19242 -19243 -19244 -19245 -19246 -19247 -19248 -19249 -19250 -19251 -19252 -19253 -19254 -19255 -19256 -19257 -19258 -19259 -19260 -19261 -19262 -19263 -19264 -19265 -19266 -19267 -19268 -19269 -19270 -19271 -19272 -19273 -19274 -19275 -19276 -19277 -19278 -19279 -19280 -19281 -19282 -19283 -19284 -19285 -19286 -19287 -19288 -19289 -19290 -19291 -19292 -19293 -19294 -19295 -19296 -19297 -19298 -19299 -19300 -19301 -19302 -19303 -19304 -19305 -19306 -19307 -19308 -19309 -19310 -19311 -19312 -19313 -19314 -19315 -19316 -19317 -19318 -19319 -19320 -19321 -19322 -19323 -19324 -19325 -19326 -19327 -19328 -19329 -19330 -19331 -19332 -19333 -19334 -19335 -19336 -19337 -19338 -19339 -19340 -19341 -19342 -19343 -19344 -19345 -19346 -19347 -19348 -19349 -19350 -19351 -19352 -19353 -19354 -19355 -19356 -19357 -19358 -19359 -19360 -19361 -19362 -19363 -19364 -19365 -19366 -19367 -19368 -19369 -19370 -19371 -19372 -19373 -19374 -19375 -19376 -19377 -19378 -19379 -19380 -19381 -19382 -19383 -19384 -19385 -19386 -19387 -19388 -19389 -19390 -19391 -19392 -19393 -19394 -19395 -19396 -19397 -19398 -19399 -19400 -19401 -19402 -19403 -19404 -19405 -19406 -19407 -19408 -19409 -19410 -19411 -19412 -19413 -19414 -19415 -19416 -19417 -19418 -19419 -19420 -19421 -19422 -19423 -19424 -19425 -19426 -19427 -19428 -19429 -19430 -19431 -19432 -19433 -19434 -19435 -19436 -19437 -19438 -19439 -19440 -19441 -19442 -19443 -19444 -19445 -19446 -19447 -19448 -19449 -19450 -19451 -19452 -19453 -19454 -19455 -19456 -19457 -19458 -19459 -19460 -19461 -19462 -19463 -19464 -19465 -19466 -19467 -19468 -19469 -19470 -19471 -19472 -19473 -19474 -19475 -19476 -19477 -19478 -19479 -19480 -19481 -19482 -19483 -19484 -19485 -19486 -19487 -19488 -19489 -19490 -19491 -19492 -19493 -19494 -19495 -19496 -19497 -19498 -19499 -19500 -19501 -19502 -19503 -19504 -19505 -19506 -19507 -19508 -19509 -19510 -19511 -19512 -19513 -19514 -19515 -19516 -19517 -19518 -19519 -19520 -19521 -19522 -19523 -19524 -19525 -19526 -19527 -19528 -19529 -19530 -19531 -19532 -19533 -19534 -19535 -19536 -19537 -19538 -19539 -19540 -19541 -19542 -19543 -19544 -19545 -19546 -19547 -19548 -19549 -19550 -19551 -19552 -19553 -19554 -19555 -19556 -19557 -19558 -19559 -19560 -19561 -19562 -19563 -19564 -19565 -19566 -19567 -19568 -19569 -19570 -19571 -19572 -19573 -19574 -19575 -19576 -19577 -19578 -19579 -19580 -19581 -19582 -19583 -19584 -19585 -19586 -19587 -19588 -19589 -19590 -19591 -19592 -19593 -19594 -19595 -19596 -19597 -19598 -19599 -19600 -19601 -19602 -19603 -19604 -19605 -19606 -19607 -19608 -19609 -19610 -19611 -19612 -19613 -19614 -19615 -19616 -19617 -19618 -19619 -19620 -19621 -19622 -19623 -19624 -19625 -19626 -19627 -19628 -19629 -19630 -19631 -19632 -19633 -19634 -19635 -19636 -19637 -19638 -19639 -19640 -19641 -19642 -19643 -19644 -19645 -19646 -19647 -19648 -19649 -19650 -19651 -19652 -19653 -19654 -19655 -19656 -19657 -19658 -19659 -19660 -19661 -19662 -19663 -19664 -19665 -19666 -19667 -19668 -19669 -19670 -19671 -19672 -19673 -19674 -19675 -19676 -19677 -19678 -19679 -19680 -19681 -19682 -19683 -19684 -19685 -19686 -19687 -19688 -19689 -19690 -19691 -19692 -19693 -19694 -19695 -19696 -19697 -19698 -19699 -19700 -19701 -19702 -19703 -19704 -19705 -19706 -19707 -19708 -19709 -19710 -19711 -19712 -19713 -19714 -19715 -19716 -19717 -19718 -19719 -19720 -19721 -19722 -19723 -19724 -19725 -19726 -19727 -19728 -19729 -19730 -19731 -19732 -19733 -19734 -19735 -19736 -19737 -19738 -19739 -19740 -19741 -19742 -19743 -19744 -19745 -19746 -19747 -19748 -19749 -19750 -19751 -19752 -19753 -19754 -19755 -19756 -19757 -19758 -19759 -19760 -19761 -19762 -19763 -19764 -19765 -19766 -19767 -19768 -19769 -19770 -19771 -19772 -19773 -19774 -19775 -19776 -19777 -19778 -19779 -19780 -19781 -19782 -19783 -19784 -19785 -19786 -19787 -19788 -19789 -19790 -19791 -19792 -19793 -19794 -19795 -19796 -19797 -19798 -19799 -19800 -19801 -19802 -19803 -19804 -19805 -19806 -19807 -19808 -19809 -19810 -19811 -19812 -19813 -19814 -19815 -19816 -19817 -19818 -19819 -19820 -19821 -19822 -19823 -19824 -19825 -19826 -19827 -19828 -19829 -19830 -19831 -19832 -19833 -19834 -19835 -19836 -19837 -19838 -19839 -19840 -19841 -19842 -19843 -19844 -19845 -19846 -19847 -19848 -19849 -19850 -19851 -19852 -19853 -19854 -19855 -19856 -19857 -19858 -19859 -19860 -19861 -19862 -19863 -19864 -19865 -19866 -19867 -19868 -19869 -19870 -19871 -19872 -19873 -19874 -19875 -19876 -19877 -19878 -19879 -19880 -19881 -19882 -19883 -19884 -19885 -19886 -19887 -19888 -19889 -19890 -19891 -19892 -19893 -19894 -19895 -19896 -19897 -19898 -19899 -19900 -19901 -19902 -19903 -19904 -19905 -19906 -19907 -19908 -19909 -19910 -19911 -19912 -19913 -19914 -19915 -19916 -19917 -19918 -19919 -19920 -19921 -19922 -19923 -19924 -19925 -19926 -19927 -19928 -19929 -19930 -19931 -19932 -19933 -19934 -19935 -19936 -19937 -19938 -19939 -19940 -19941 -19942 -19943 -19944 -19945 -19946 -19947 -19948 -19949 -19950 -19951 -19952 -19953 -19954 -19955 -19956 -19957 -19958 -19959 -19960 -19961 -19962 -19963 -19964 -19965 -19966 -19967 -19968 -19969 -19970 -19971 -19972 -19973 -19974 -19975 -19976 -19977 -19978 -19979 -19980 -19981 -19982 -19983 -19984 -19985 -19986 -19987 -19988 -19989 -19990 -19991 -19992 -19993 -19994 -19995 -19996 -19997 -19998 -19999 -20000 -20001 -20002 -20003 -20004 -20005 -20006 -20007 -20008 -20009 -20010 -20011 -20012 -20013 -20014 -20015 -20016 -20017 -20018 -20019 -20020 -20021 -20022 -20023 -20024 -20025 -20026 -20027 -20028 -20029 -20030 -20031 -20032 -20033 -20034 -20035 -20036 -20037 -20038 -20039 -20040 -20041 -20042 -20043 -20044 -20045 -20046 -20047 -20048 -20049 -20050 -20051 -20052 -20053 -20054 -20055 -20056 -20057 -20058 -20059 -20060 -20061 -20062 -20063 -20064 -20065 -20066 -20067 -20068 -20069 -20070 -20071 -20072 -20073 -20074 -20075 -20076 -20077 -20078 -20079 -20080 -20081 -20082 -20083 -20084 -20085 -20086 -20087 -20088 -20089 -20090 -20091 -20092 -20093 -20094 -20095 -20096 -20097 -20098 -20099 -20100 -20101 -20102 -20103 -20104 -20105 -20106 -20107 -20108 -20109 -20110 -20111 -20112 -20113 -20114 -20115 -20116 -20117 -20118 -20119 -20120 -20121 -20122 -20123 -20124 -20125 -20126 -20127 -20128 -20129 -20130 -20131 -20132 -20133 -20134 -20135 -20136 -20137 -20138 -20139 -20140 -20141 -20142 -20143 -20144 -20145 -20146 -20147 -20148 -20149 -20150 -20151 -20152 -20153 -20154 -20155 -20156 -20157 -20158 -20159 -20160 -20161 -20162 -20163 -20164 -20165 -20166 -20167 -20168 -20169 -20170 -20171 -20172 -20173 -20174 -20175 -20176 -20177 -20178 -20179 -20180 -20181 -20182 -20183 -20184 -20185 -20186 -20187 -20188 -20189 -20190 -20191 -20192 -20193 -20194 -20195 -20196 -20197 -20198 -20199 -20200 -20201 -20202 -20203 -20204 -20205 -20206 -20207 -20208 -20209 -20210 -20211 -20212 -20213 -20214 -20215 -20216 -20217 -20218 -20219 -20220 -20221 -20222 -20223 -20224 -20225 -20226 -20227 -20228 -20229 -20230 -20231 -20232 -20233 -20234 -20235 -20236 -20237 -20238 -20239 -20240 -20241 -20242 -20243 -20244 -20245 -20246 -20247 -20248 -20249 -20250 -20251 -20252 -20253 -20254 -20255 -20256 -20257 -20258 -20259 -20260 -20261 -20262 -20263 -20264 -20265 -20266 -20267 -20268 -20269 -20270 -20271 -20272 -20273 -20274 -20275 -20276 -20277 -20278 -20279 -20280 -20281 -20282 -20283 -20284 -20285 -20286 -20287 -20288 -20289 -20290 -20291 -20292 -20293 -20294 -20295 -20296 -20297 -20298 -20299 -20300 -20301 -20302 -20303 -20304 -20305 -20306 -20307 -20308 -20309 -20310 -20311 -20312 -20313 -20314 -20315 -20316 -20317 -20318 -20319 -20320 -20321 -20322 -20323 -20324 -20325 -20326 -20327 -20328 -20329 -20330 -20331 -20332 -20333 -20334 -20335 -20336 -20337 -20338 -20339 -20340 -20341 -20342 -20343 -20344 -20345 -20346 -20347 -20348 -20349 -20350 -20351 -20352 -20353 -20354 -20355 -20356 -20357 -20358 -20359 -20360 -20361 -20362 -20363 -20364 -20365 -20366 -20367 -20368 -20369 -20370 -20371 -20372 -20373 -20374 -20375 -20376 -20377 -20378 -20379 -20380 -20381 -20382 -20383 -20384 -20385 -20386 -20387 -20388 -20389 -20390 -20391 -20392 -20393 -20394 -20395 -20396 -20397 -20398 -20399 -20400 -20401 -20402 -20403 -20404 -20405 -20406 -20407 -20408 -20409 -20410 -20411 -20412 -20413 -20414 -20415 -20416 -20417 -20418 -20419 -20420 -20421 -20422 -20423 -20424 -20425 -20426 -20427 -20428 -20429 -20430 -20431 -20432 -20433 -20434 -20435 -20436 -20437 -20438 -20439 -20440 -20441 -20442 -20443 -20444 -20445 -20446 -20447 -20448 -20449 -20450 -20451 -20452 -20453 -20454 -20455 -20456 -20457 -20458 -20459 -20460 -20461 -20462 -20463 -20464 -20465 -20466 -20467 -20468 -20469 -20470 -20471 -20472 -20473 -20474 -20475 -20476 -20477 -20478 -20479 -20480 -20481 -20482 -20483 -20484 -20485 -20486 -20487 -20488 -20489 -20490 -20491 -20492 -20493 -20494 -20495 -20496 -20497 -20498 -20499 -20500 -20501 -20502 -20503 -20504 -20505 -20506 -20507 -20508 -20509 -20510 -20511 -20512 -20513 -20514 -20515 -20516 -20517 -20518 -20519 -20520 -20521 -20522 -20523 -20524 -20525 -20526 -20527 -20528 -20529 -20530 -20531 -20532 -20533 -20534 -20535 -20536 -20537 -20538 -20539 -20540 -20541 -20542 -20543 -20544 -20545 -20546 -20547 -20548 -20549 -20550 -20551 -20552 -20553 -20554 -20555 -20556 -20557 -20558 -20559 -20560 -20561 -20562 -20563 -20564 -20565 -20566 -20567 -20568 -20569 -20570 -20571 -20572 -20573 -20574 -20575 -20576 -20577 -20578 -20579 -20580 -20581 -20582 -20583 -20584 -20585 -20586 -20587 -20588 -20589 -20590 -20591 -20592 -20593 -20594 -20595 -20596 -20597 -20598 -20599 -20600 -20601 -20602 -20603 -20604 -20605 -20606 -20607 -20608 -20609 -20610 -20611 -20612 -20613 -20614 -20615 -20616 -20617 -20618 -20619 -20620 -20621 -20622 -20623 -20624 -20625 -20626 -20627 -20628 -20629 -20630 -20631 -20632 -20633 -20634 -20635 -20636 -20637 -20638 -20639 -20640 -20641 -20642 -20643 -20644 -20645 -20646 -20647 -20648 -20649 -20650 -20651 -20652 -20653 -20654 -20655 -20656 -20657 -20658 -20659 -20660 -20661 -20662 -20663 -20664 -20665 -20666 -20667 -20668 -20669 -20670 -20671 -20672 -20673 -20674 -20675 -20676 -20677 -20678 -20679 -20680 -20681 -20682 -20683 -20684 -20685 -20686 -20687 -20688 -20689 -20690 -20691 -20692 -20693 -20694 -20695 -20696 -20697 -20698 -20699 -20700 -20701 -20702 -20703 -20704 -20705 -20706 -20707 -20708 -20709 -20710 -20711 -20712 -20713 -20714 -20715 -20716 -20717 -20718 -20719 -20720 -20721 -20722 -20723 -20724 -20725 -20726 -20727 -20728 -20729 -20730 -20731 -20732 -20733 -20734 -20735 -20736 -20737 -20738 -20739 -20740 -20741 -20742 -20743 -20744 -20745 -20746 -20747 -20748 -20749 -20750 -20751 -20752 -20753 -20754 -20755 -20756 -20757 -20758 -20759 -20760 -20761 -20762 -20763 -20764 -20765 -20766 -20767 -20768 -20769 -20770 -20771 -20772 -20773 -20774 -20775 -20776 -20777 -20778 -20779 -20780 -20781 -20782 -20783 -20784 -20785 -20786 -20787 -20788 -20789 -20790 -20791 -20792 -20793 -20794 -20795 -20796 -20797 -20798 -20799 -20800 -20801 -20802 -20803 -20804 -20805 -20806 -20807 -20808 -20809 -20810 -20811 -20812 -20813 -20814 -20815 -20816 -20817 -20818 -20819 -20820 -20821 -20822 -20823 -20824 -20825 -20826 -20827 -20828 -20829 -20830 -20831 -20832 -20833 -20834 -20835 -20836 -20837 -20838 -20839 -20840 -20841 -20842 -20843 -20844 -20845 -20846 -20847 -20848 -20849 -20850 -20851 -20852 -20853 -20854 -20855 -20856 -20857 -20858 -20859 -20860 -20861 -20862 -20863 -20864 -20865 -20866 -20867 -20868 -20869 -20870 -20871 -20872 -20873 -20874 -20875 -20876 -20877 -20878 -20879 -20880 -20881 -20882 -20883 -20884 -20885 -20886 -20887 -20888 -20889 -20890 -20891 -20892 -20893 -20894 -20895 -20896 -20897 -20898 -20899 -20900 -20901 -20902 -20903 -20904 -20905 -20906 -20907 -20908 -20909 -20910 -20911 -20912 -20913 -20914 -20915 -20916 -20917 -20918 -20919 -20920 -20921 -20922 -20923 -20924 -20925 -20926 -20927 -20928 -20929 -20930 -20931 -20932 -20933 -20934 -20935 -20936 -20937 -20938 -20939 -20940 -20941 -20942 -20943 -20944 -20945 -20946 -20947 -20948 -20949 -20950 -20951 -20952 -20953 -20954 -20955 -20956 -20957 -20958 -20959 -20960 -20961 -20962 -20963 -20964 -20965 -20966 -20967 -20968 -20969 -20970 -20971 -20972 -20973 -20974 -20975 -20976 -20977 -20978 -20979 -20980 -20981 -20982 -20983 -20984 -20985 -20986 -20987 -20988 -20989 -20990 -20991 -20992 -20993 -20994 -20995 -20996 -20997 -20998 -20999 -21000 -21001 -21002 -21003 -21004 -21005 -21006 -21007 -21008 -21009 -21010 -21011 -21012 -21013 -21014 -21015 -21016 -21017 -21018 -21019 -21020 -21021 -21022 -21023 -21024 -21025 -21026 -21027 -21028 -21029 -21030 -21031 -21032 -21033 -21034 -21035 -21036 -21037 -21038 -21039 -21040 -21041 -21042 -21043 -21044 -21045 -21046 -21047 -21048 -21049 -21050 -21051 -21052 -21053 -21054 -21055 -21056 -21057 -21058 -21059 -21060 -21061 -21062 -21063 -21064 -21065 -21066 -21067 -21068 -21069 -21070 -21071 -21072 -21073 -21074 -21075 -21076 -21077 -21078 -21079 -21080 -21081 -21082 -21083 -21084 -21085 -21086 -21087 -21088 -21089 -21090 -21091 -21092 -21093 -21094 -21095 -21096 -21097 -21098 -21099 -21100 -21101 -21102 -21103 -21104 -21105 -21106 -21107 -21108 -21109 -21110 -21111 -21112 -21113 -21114 -21115 -21116 -21117 -21118 -21119 -21120 -21121 -21122 -21123 -21124 -21125 -21126 -21127 -21128 -21129 -21130 -21131 -21132 -21133 -21134 -21135 -21136 -21137 -21138 -21139 -21140 -21141 -21142 -21143 -21144 -21145 -21146 -21147 -21148 -21149 -21150 -21151 -21152 -21153 -21154 -21155 -21156 -21157 -21158 -21159 -21160 -21161 -21162 -21163 -21164 -21165 -21166 -21167 -21168 -21169 -21170 -21171 -21172 -21173 -21174 -21175 -21176 -21177 -21178 -21179 -21180 -21181 -21182 -21183 -21184 -21185 -21186 -21187 -21188 -21189 -21190 -21191 -21192 -21193 -21194 -21195 -21196 -21197 -21198 -21199 -21200 -21201 -21202 -21203 -21204 -21205 -21206 -21207 -21208 -21209 -21210 -21211 -21212 -21213 -21214 -21215 -21216 -21217 -21218 -21219 -21220 -21221 -21222 -21223 -21224 -21225 -21226 -21227 -21228 -21229 -21230 -21231 -21232 -21233 -21234 -21235 -21236 -21237 -21238 -21239 -21240 -21241 -21242 -21243 -21244 -21245 -21246 -21247 -21248 -21249 -21250 -21251 -21252 -21253 -21254 -21255 -21256 -21257 -21258 -21259 -21260 -21261 -21262 -21263 -21264 -21265 -21266 -21267 -21268 -21269 -21270 -21271 -21272 -21273 -21274 -21275 -21276 -21277 -21278 -21279 -21280 -21281 -21282 -21283 -21284 -21285 -21286 -21287 -21288 -21289 -21290 -21291 -21292 -21293 -21294 -21295 -21296 -21297 -21298 -21299 -21300 -21301 -21302 -21303 -21304 -21305 -21306 -21307 -21308 -21309 -21310 -21311 -21312 -21313 -21314 -21315 -21316 -21317 -21318 -21319 -21320 -21321 -21322 -21323 -21324 -21325 -21326 -21327 -21328 -21329 -21330 -21331 -21332 -21333 -21334 -21335 -21336 -21337 -21338 -21339 -21340 -21341 -21342 -21343 -21344 -21345 -21346 -21347 -21348 -21349 -21350 -21351 -21352 -21353 -21354 -21355 -21356 -21357 -21358 -21359 -21360 -21361 -21362 -21363 -21364 -21365 -21366 -21367 -21368 -21369 -21370 -21371 -21372 -21373 -21374 -21375 -21376 -21377 -21378 -21379 -21380 -21381 -21382 -21383 -21384 -21385 -21386 -21387 -21388 -21389 -21390 -21391 -21392 -21393 -21394 -21395 -21396 -21397 -21398 -21399 -21400 -21401 -21402 -21403 -21404 -21405 -21406 -21407 -21408 -21409 -21410 -21411 -21412 -21413 -21414 -21415 -21416 -21417 -21418 -21419 -21420 -21421 -21422 -21423 -21424 -21425 -21426 -21427 -21428 -21429 -21430 -21431 -21432 -21433 -21434 -21435 -21436 -21437 -21438 -21439 -21440 -21441 -21442 -21443 -21444 -21445 -21446 -21447 -21448 -21449 -21450 -21451 -21452 -21453 -21454 -21455 -21456 -21457 -21458 -21459 -21460 -21461 -21462 -21463 -21464 -21465 -21466 -21467 -21468 -21469 -21470 -21471 -21472 -21473 -21474 -21475 -21476 -21477 -21478 -21479 -21480 -21481 -21482 -21483 -21484 -21485 -21486 -21487 -21488 -21489 -21490 -21491 -21492 -21493 -21494 -21495 -21496 -21497 -21498 -21499 -21500 -21501 -21502 -21503 -21504 -21505 -21506 -21507 -21508 -21509 -21510 -21511 -21512 -21513 -21514 -21515 -21516 -21517 -21518 -21519 -21520 -21521 -21522 -21523 -21524 -21525 -21526 -21527 -21528 -21529 -21530 -21531 -21532 -21533 -21534 -21535 -21536 -21537 -21538 -21539 -21540 -21541 -21542 -21543 -21544 -21545 -21546 -21547 -21548 -21549 -21550 -21551 -21552 -21553 -21554 -21555 -21556 -21557 -21558 -21559 -21560 -21561 -21562 -21563 -21564 -21565 -21566 -21567 -21568 -21569 -21570 -21571 -21572 -21573 -21574 -21575 -21576 -21577 -21578 -21579 -21580 -21581 -21582 -21583 -21584 -21585 -21586 -21587 -21588 -21589 -21590 -21591 -21592 -21593 -21594 -21595 -21596 -21597 -21598 -21599 -21600 -21601 -21602 -21603 -21604 -21605 -21606 -21607 -21608 -21609 -21610 -21611 -21612 -21613 -21614 -21615 -21616 -21617 -21618 -21619 -21620 -21621 -21622 -21623 -21624 -21625 -21626 -21627 -21628 -21629 -21630 -21631 -21632 -21633 -21634 -21635 -21636 -21637 -21638 -21639 -21640 -21641 -21642 -21643 -21644 -21645 -21646 -21647 -21648 -21649 -21650 -21651 -21652 -21653 -21654 -21655 -21656 -21657 -21658 -21659 -21660 -21661 -21662 -21663 -21664 -21665 -21666 -21667 -21668 -21669 -21670 -21671 -21672 -21673 -21674 -21675 -21676 -21677 -21678 -21679 -21680 -21681 -21682 -21683 -21684 -21685 -21686 -21687 -21688 -21689 -21690 -21691 -21692 -21693 -21694 -21695 -21696 -21697 -21698 -21699 -21700 -21701 -21702 -21703 -21704 -21705 -21706 -21707 -21708 -21709 -21710 -21711 -21712 -21713 -21714 -21715 -21716 -21717 -21718 -21719 -21720 -21721 -21722 -21723 -21724 -21725 -21726 -21727 -21728 -21729 -21730 -21731 -21732 -21733 -21734 -21735 -21736 -21737 -21738 -21739 -21740 -21741 -21742 -21743 -21744 -21745 -21746 -21747 -21748 -21749 -21750 -21751 -21752 -21753 -21754 -21755 -21756 -21757 -21758 -21759 -21760 -21761 -21762 -21763 -21764 -21765 -21766 -21767 -21768 -21769 -21770 -21771 -21772 -21773 -21774 -21775 -21776 -21777 -21778 -21779 -21780 -21781 -21782 -21783 -21784 -21785 -21786 -21787 -21788 -21789 -21790 -21791 -21792 -21793 -21794 -21795 -21796 -21797 -21798 -21799 -21800 -21801 -21802 -21803 -21804 -21805 -21806 -21807 -21808 -21809 -21810 -21811 -21812 -21813 -21814 -21815 -21816 -21817 -21818 -21819 -21820 -21821 -21822 -21823 -21824 -21825 -21826 -21827 -21828 -21829 -21830 -21831 -21832 -21833 -21834 -21835 -21836 -21837 -21838 -21839 -21840 -21841 -21842 -21843 -21844 -21845 -21846 -21847 -21848 -21849 -21850 -21851 -21852 -21853 -21854 -21855 -21856 -21857 -21858 -21859 -21860 -21861 -21862 -21863 -21864 -21865 -21866 -21867 -21868 -21869 -21870 -21871 -21872 -21873 -21874 -21875 -21876 -21877 -21878 -21879 -21880 -21881 -21882 -21883 -21884 -21885 -21886 -21887 -21888 -21889 -21890 -21891 -21892 -21893 -21894 -21895 -21896 -21897 -21898 -21899 -21900 -21901 -21902 -21903 -21904 -21905 -21906 -21907 -21908 -21909 -21910 -21911 -21912 -21913 -21914 -21915 -21916 -21917 -21918 -21919 -21920 -21921 -21922 -21923 -21924 -21925 -21926 -21927 -21928 -21929 -21930 -21931 -21932 -21933 -21934 -21935 -21936 -21937 -21938 -21939 -21940 -21941 -21942 -21943 -21944 -21945 -21946 -21947 -21948 -21949 -21950 -21951 -21952 -21953 -21954 -21955 -21956 -21957 -21958 -21959 -21960 -21961 -21962 -21963 -21964 -21965 -21966 -21967 -21968 -21969 -21970 -21971 -21972 -21973 -21974 -21975 -21976 -21977 -21978 -21979 -21980 -21981 -21982 -21983 -21984 -21985 -21986 -21987 -21988 -21989 -21990 -21991 -21992 -21993 -21994 -21995 -21996 -21997 -21998 -21999 -22000 -22001 -22002 -22003 -22004 -22005 -22006 -22007 -22008 -22009 -22010 -22011 -22012 -22013 -22014 -22015 -22016 -22017 -22018 -22019 -22020 -22021 -22022 -22023 -22024 -22025 -22026 -22027 -22028 -22029 -22030 -22031 -22032 -22033 -22034 -22035 -22036 -22037 -22038 -22039 -22040 -22041 -22042 -22043 -22044 -22045 -22046 -22047 -22048 -22049 -22050 -22051 -22052 -22053 -22054 -22055 -22056 -22057 -22058 -22059 -22060 -22061 -22062 -22063 -22064 -22065 -22066 -22067 -22068 -22069 -22070 -22071 -22072 -22073 -22074 -22075 -22076 -22077 -22078 -22079 -22080 -22081 -22082 -22083 -22084 -22085 -22086 -22087 -22088 -22089 -22090 -22091 -22092 -22093 -22094 -22095 -22096 -22097 -22098 -22099 -22100 -22101 -22102 -22103 -22104 -22105 -22106 -22107 -22108 -22109 -22110 -22111 -22112 -22113 -22114 -22115 -22116 -22117 -22118 -22119 -22120 -22121 -22122 -22123 -22124 -22125 -22126 -22127 -22128 -22129 -22130 -22131 -22132 -22133 -22134 -22135 -22136 -22137 -22138 -22139 -22140 -22141 -22142 -22143 -22144 -22145 -22146 -22147 -22148 -22149 -22150 -22151 -22152 -22153 -22154 -22155 -22156 -22157 -22158 -22159 -22160 -22161 -22162 -22163 -22164 -22165 -22166 -22167 -22168 -22169 -22170 -22171 -22172 -22173 -22174 -22175 -22176 -22177 -22178 -22179 -22180 -22181 -22182 -22183 -22184 -22185 -22186 -22187 -22188 -22189 -22190 -22191 -22192 -22193 -22194 -22195 -22196 -22197 -22198 -22199 -22200 -22201 -22202 -22203 -22204 -22205 -22206 -22207 -22208 -22209 -22210 -22211 -22212 -22213 -22214 -22215 -22216 -22217 -22218 -22219 -22220 -22221 -22222 -22223 -22224 -22225 -22226 -22227 -22228 -22229 -22230 -22231 -22232 -22233 -22234 -22235 -22236 -22237 -22238 -22239 -22240 -22241 -22242 -22243 -22244 -22245 -22246 -22247 -22248 -22249 -22250 -22251 -22252 -22253 -22254 -22255 -22256 -22257 -22258 -22259 -22260 -22261 -22262 -22263 -22264 -22265 -22266 -22267 -22268 -22269 -22270 -22271 -22272 -22273 -22274 -22275 -22276 -22277 -22278 -22279 -22280 -22281 -22282 -22283 -22284 -22285 -22286 -22287 -22288 -22289 -22290 -22291 -22292 -22293 -22294 -22295 -22296 -22297 -22298 -22299 -22300 -22301 -22302 -22303 -22304 -22305 -22306 -22307 -22308 -22309 -22310 -22311 -22312 -22313 -22314 -22315 -22316 -22317 -22318 -22319 -22320 -22321 -22322 -22323 -22324 -22325 -22326 -22327 -22328 -22329 -22330 -22331 -22332 -22333 -22334 -22335 -22336 -22337 -22338 -22339 -22340 -22341 -22342 -22343 -22344 -22345 -22346 -22347 -22348 -22349 -22350 -22351 -22352 -22353 -22354 -22355 -22356 -22357 -22358 -22359 -22360 -22361 -22362 -22363 -22364 -22365 -22366 -22367 -22368 -22369 -22370 -22371 -22372 -22373 -22374 -22375 -22376 -22377 -22378 -22379 -22380 -22381 -22382 -22383 -22384 -22385 -22386 -22387 -22388 -22389 -22390 -22391 -22392 -22393 -22394 -22395 -22396 -22397 -22398 -22399 -22400 -22401 -22402 -22403 -22404 -22405 -22406 -22407 -22408 -22409 -22410 -22411 -22412 -22413 -22414 -22415 -22416 -22417 -22418 -22419 -22420 -22421 -22422 -22423 -22424 -22425 -22426 -22427 -22428 -22429 -22430 -22431 -22432 -22433 -22434 -22435 -22436 -22437 -22438 -22439 -22440 -22441 -22442 -22443 -22444 -22445 -22446 -22447 -22448 -22449 -22450 -22451 -22452 -22453 -22454 -22455 -22456 -22457 -22458 -22459 -22460 -22461 -22462 -22463 -22464 -22465 -22466 -22467 -22468 -22469 -22470 -22471 -22472 -22473 -22474 -22475 -22476 -22477 -22478 -22479 -22480 -22481 -22482 -22483 -22484 -22485 -22486 -22487 -22488 -22489 -22490 -22491 -22492 -22493 -22494 -22495 -22496 -22497 -22498 -22499 -22500 -22501 -22502 -22503 -22504 -22505 -22506 -22507 -22508 -22509 -22510 -22511 -22512 -22513 -22514 -22515 -22516 -22517 -22518 -22519 -22520 -22521 -22522 -22523 -22524 -22525 -22526 -22527 -22528 -22529 -22530 -22531 -22532 -22533 -22534 -22535 -22536 -22537 -22538 -22539 -22540 -22541 -22542 -22543 -22544 -22545 -22546 -22547 -22548 -22549 -22550 -22551 -22552 -22553 -22554 -22555 -22556 -22557 -22558 -22559 -22560 -22561 -22562 -22563 -22564 -22565 -22566 -22567 -22568 -22569 -22570 -22571 -22572 -22573 -22574 -22575 -22576 -22577 -22578 -22579 -22580 -22581 -22582 -22583 -22584 -22585 -22586 -22587 -22588 -22589 -22590 -22591 -22592 -22593 -22594 -22595 -22596 -22597 -22598 -22599 -22600 -22601 -22602 -22603 -22604 -22605 -22606 -22607 -22608 -22609 -22610 -22611 -22612 -22613 -22614 -22615 -22616 -22617 -22618 -22619 -22620 -22621 -22622 -22623 -22624 -22625 -22626 -22627 -22628 -22629 -22630 -22631 -22632 -22633 -22634 -22635 -22636 -22637 -22638 -22639 -22640 -22641 -22642 -22643 -22644 -22645 -22646 -22647 -22648 -22649 -22650 -22651 -22652 -22653 -22654 -22655 -22656 -22657 -22658 -22659 -22660 -22661 -22662 -22663 -22664 -22665 -22666 -22667 -22668 -22669 -22670 -22671 -22672 -22673 -22674 -22675 -22676 -22677 -22678 -22679 -22680 -22681 -22682 -22683 -22684 -22685 -22686 -22687 -22688 -22689 -22690 -22691 -22692 -22693 -22694 -22695 -22696 -22697 -22698 -22699 -22700 -22701 -22702 -22703 -22704 -22705 -22706 -22707 -22708 -22709 -22710 -22711 -22712 -22713 -22714 -22715 -22716 -22717 -22718 -22719 -22720 -22721 -22722 -22723 -22724 -22725 -22726 -22727 -22728 -22729 -22730 -22731 -22732 -22733 -22734 -22735 -22736 -22737 -22738 -22739 -22740 -22741 -22742 -22743 -22744 -22745 -22746 -22747 -22748 -22749 -22750 -22751 -22752 -22753 -22754 -22755 -22756 -22757 -22758 -22759 -22760 -22761 -22762 -22763 -22764 -22765 -22766 -22767 -22768 -22769 -22770 -22771 -22772 -22773 -22774 -22775 -22776 -22777 -22778 -22779 -22780 -22781 -22782 -22783 -22784 -22785 -22786 -22787 -22788 -22789 -22790 -22791 -22792 -22793 -22794 -22795 -22796 -22797 -22798 -22799 -22800 -22801 -22802 -22803 -22804 -22805 -22806 -22807 -22808 -22809 -22810 -22811 -22812 -22813 -22814 -22815 -22816 -22817 -22818 -22819 -22820 -22821 -22822 -22823 -22824 -22825 -22826 -22827 -22828 -22829 -22830 -22831 -22832 -22833 -22834 -22835 -22836 -22837 -22838 -22839 -22840 -22841 -22842 -22843 -22844 -22845 -22846 -22847 -22848 -22849 -22850 -22851 -22852 -22853 -22854 -22855 -22856 -22857 -22858 -22859 -22860 -22861 -22862 -22863 -22864 -22865 -22866 -22867 -22868 -22869 -22870 -22871 -22872 -22873 -22874 -22875 -22876 -22877 -22878 -22879 -22880 -22881 -22882 -22883 -22884 -22885 -22886 -22887 -22888 -22889 -22890 -22891 -22892 -22893 -22894 -22895 -22896 -22897 -22898 -22899 -22900 -22901 -22902 -22903 -22904 -22905 -22906 -22907 -22908 -22909 -22910 -22911 -22912 -22913 -22914 -22915 -22916 -22917 -22918 -22919 -22920 -22921 -22922 -22923 -22924 -22925 -22926 -22927 -22928 -22929 -22930 -22931 -22932 -22933 -22934 -22935 -22936 -22937 -22938 -22939 -22940 -22941 -22942 -22943 -22944 -22945 -22946 -22947 -22948 -22949 -22950 -22951 -22952 -22953 -22954 -22955 -22956 -22957 -22958 -22959 -22960 -22961 -22962 -22963 -22964 -22965 -22966 -22967 -22968 -22969 -22970 -22971 -22972 -22973 -22974 -22975 -22976 -22977 -22978 -22979 -22980 -22981 -22982 -22983 -22984 -22985 -22986 -22987 -22988 -22989 -22990 -22991 -22992 -22993 -22994 -22995 -22996 -22997 -22998 -22999 -23000 -23001 -23002 -23003 -23004 -23005 -23006 -23007 -23008 -23009 -23010 -23011 -23012 -23013 -23014 -23015 -23016 -23017 -23018 -23019 -23020 -23021 -23022 -23023 -23024 -23025 -23026 -23027 -23028 -23029 -23030 -23031 -23032 -23033 -23034 -23035 -23036 -23037 -23038 -23039 -23040 -23041 -23042 -23043 -23044 -23045 -23046 -23047 -23048 -23049 -23050 -23051 -23052 -23053 -23054 -23055 -23056 -23057 -23058 -23059 -23060 -23061 -23062 -23063 -23064 -23065 -23066 -23067 -23068 -23069 -23070 -23071 -23072 -23073 -23074 -23075 -23076 -23077 -23078 -23079 -23080 -23081 -23082 -23083 -23084 -23085 -23086 -23087 -23088 -23089 -23090 -23091 -23092 -23093 -23094 -23095 -23096 -23097 -23098 -23099 -23100 -23101 -23102 -23103 -23104 -23105 -23106 -23107 -23108 -23109 -23110 -23111 -23112 -23113 -23114 -23115 -23116 -23117 -23118 -23119 -23120 -23121 -23122 -23123 -23124 -23125 -23126 -23127 -23128 -23129 -23130 -23131 -23132 -23133 -23134 -23135 -23136 -23137 -23138 -23139 -23140 -23141 -23142 -23143 -23144 -23145 -23146 -23147 -23148 -23149 -23150 -23151 -23152 -23153 -23154 -23155 -23156 -23157 -23158 -23159 -23160 -23161 -23162 -23163 -23164 -23165 -23166 -23167 -23168 -23169 -23170 -23171 -23172 -23173 -23174 -23175 -23176 -23177 -23178 -23179 -23180 -23181 -23182 -23183 -23184 -23185 -23186 -23187 -23188 -23189 -23190 -23191 -23192 -23193 -23194 -23195 -23196 -23197 -23198 -23199 -23200 -23201 -23202 -23203 -23204 -23205 -23206 -23207 -23208 -23209 -23210 -23211 -23212 -23213 -23214 -23215 -23216 -23217 -23218 -23219 -23220 -23221 -23222 -23223 -23224 -23225 -23226 -23227 -23228 -23229 -23230 -23231 -23232 -23233 -23234 -23235 -23236 -23237 -23238 -23239 -23240 -23241 -23242 -23243 -23244 -23245 -23246 -23247 -23248 -23249 -23250 -23251 -23252 -23253 -23254 -23255 -23256 -23257 -23258 -23259 -23260 -23261 -23262 -23263 -23264 -23265 -23266 -23267 -23268 -23269 -23270 -23271 -23272 -23273 -23274 -23275 -23276 -23277 -23278 -23279 -23280 -23281 -23282 -23283 -23284 -23285 -23286 -23287 -23288 -23289 -23290 -23291 -23292 -23293 -23294 -23295 -23296 -23297 -23298 -23299 -23300 -23301 -23302 -23303 -23304 -23305 -23306 -23307 -23308 -23309 -23310 -23311 -23312 -23313 -23314 -23315 -23316 -23317 -23318 -23319 -23320 -23321 -23322 -23323 -23324 -23325 -23326 -23327 -23328 -23329 -23330 -23331 -23332 -23333 -23334 -23335 -23336 -23337 -23338 -23339 -23340 -23341 -23342 -23343 -23344 -23345 -23346 -23347 -23348 -23349 -23350 -23351 -23352 -23353 -23354 -23355 -23356 -23357 -23358 -23359 -23360 -23361 -23362 -23363 -23364 -23365 -23366 -23367 -23368 -23369 -23370 -23371 -23372 -23373 -23374 -23375 -23376 -23377 -23378 -23379 -23380 -23381 -23382 -23383 -23384 -23385 -23386 -23387 -23388 -23389 -23390 -23391 -23392 -23393 -23394 -23395 -23396 -23397 -23398 -23399 -23400 -23401 -23402 -23403 -23404 -23405 -23406 -23407 -23408 -23409 -23410 -23411 -23412 -23413 -23414 -23415 -23416 -23417 -23418 -23419 -23420 -23421 -23422 -23423 -23424 -23425 -23426 -23427 -23428 -23429 -23430 -23431 -23432 -23433 -23434 -23435 -23436 -23437 -23438 -23439 -23440 -23441 -23442 -23443 -23444 -23445 -23446 -23447 -23448 -23449 -23450 -23451 -23452 -23453 -23454 -23455 -23456 -23457 -23458 -23459 -23460 -23461 -23462 -23463 -23464 -23465 -23466 -23467 -23468 -23469 -23470 -23471 -23472 -23473 -23474 -23475 -23476 -23477 -23478 -23479 -23480 -23481 -23482 -23483 -23484 -23485 -23486 -23487 -23488 -23489 -23490 -23491 -23492 -23493 -23494 -23495 -23496 -23497 -23498 -23499 -23500 -23501 -23502 -23503 -23504 -23505 -23506 -23507 -23508 -23509 -23510 -23511 -23512 -23513 -23514 -23515 -23516 -23517 -23518 -23519 -23520 -23521 -23522 -23523 -23524 -23525 -23526 -23527 -23528 -23529 -23530 -23531 -23532 -23533 -23534 -23535 -23536 -23537 -23538 -23539 -23540 -23541 -23542 -23543 -23544 -23545 -23546 -23547 -23548 -23549 -23550 -23551 -23552 -23553 -23554 -23555 -23556 -23557 -23558 -23559 -23560 -23561 -23562 -23563 -23564 -23565 -23566 -23567 -23568 -23569 -23570 -23571 -23572 -23573 -23574 -23575 -23576 -23577 -23578 -23579 -23580 -23581 -23582 -23583 -23584 -23585 -23586 -23587 -23588 -23589 -23590 -23591 -23592 -23593 -23594 -23595 -23596 -23597 -23598 -23599 -23600 -23601 -23602 -23603 -23604 -23605 -23606 -23607 -23608 -23609 -23610 -23611 -23612 -23613 -23614 -23615 -23616 -23617 -23618 -23619 -23620 -23621 -23622 -23623 -23624 -23625 -23626 -23627 -23628 -23629 -23630 -23631 -23632 -23633 -23634 -23635 -23636 -23637 -23638 -23639 -23640 -23641 -23642 -23643 -23644 -23645 -23646 -23647 -23648 -23649 -23650 -23651 -23652 -23653 -23654 -23655 -23656 -23657 -23658 -23659 -23660 -23661 -23662 -23663 -23664 -23665 -23666 -23667 -23668 -23669 -23670 -23671 -23672 -23673 -23674 -23675 -23676 -23677 -23678 -23679 -23680 -23681 -23682 -23683 -23684 -23685 -23686 -23687 -23688 -23689 -23690 -23691 -23692 -23693 -23694 -23695 -23696 -23697 -23698 -23699 -23700 -23701 -23702 -23703 -23704 -23705 -23706 -23707 -23708 -23709 -23710 -23711 -23712 -23713 -23714 -23715 -23716 -23717 -23718 -23719 -23720 -23721 -23722 -23723 -23724 -23725 -23726 -23727 -23728 -23729 -23730 -23731 -23732 -23733 -23734 -23735 -23736 -23737 -23738 -23739 -23740 -23741 -23742 -23743 -23744 -23745 -23746 -23747 -23748 -23749 -23750 -23751 -23752 -23753 -23754 -23755 -23756 -23757 -23758 -23759 -23760 -23761 -23762 -23763 -23764 -23765 -23766 -23767 -23768 -23769 -23770 -23771 -23772 -23773 -23774 -23775 -23776 -23777 -23778 -23779 -23780 -23781 -23782 -23783 -23784 -23785 -23786 -23787 -23788 -23789 -23790 -23791 -23792 -23793 -23794 -23795 -23796 -23797 -23798 -23799 -23800 -23801 -23802 -23803 -23804 -23805 -23806 -23807 -23808 -23809 -23810 -23811 -23812 -23813 -23814 -23815 -23816 -23817 -23818 -23819 -23820 -23821 -23822 -23823 -23824 -23825 -23826 -23827 -23828 -23829 -23830 -23831 -23832 -23833 -23834 -23835 -23836 -23837 -23838 -23839 -23840 -23841 -23842 -23843 -23844 -23845 -23846 -23847 -23848 -23849 -23850 -23851 -23852 -23853 -23854 -23855 -23856 -23857 -23858 -23859 -23860 -23861 -23862 -23863 -23864 -23865 -23866 -23867 -23868 -23869 -23870 -23871 -23872 -23873 -23874 -23875 -23876 -23877 -23878 -23879 -23880 -23881 -23882 -23883 -23884 -23885 -23886 -23887 -23888 -23889 -23890 -23891 -23892 -23893 -23894 -23895 -23896 -23897 -23898 -23899 -23900 -23901 -23902 -23903 -23904 -23905 -23906 -23907 -23908 -23909 -23910 -23911 -23912 -23913 -23914 -23915 -23916 -23917 -23918 -23919 -23920 -23921 -23922 -23923 -23924 -23925 -23926 -23927 -23928 -23929 -23930 -23931 -23932 -23933 -23934 -23935 -23936 -23937 -23938 -23939 -23940 -23941 -23942 -23943 -23944 -23945 -23946 -23947 -23948 -23949 -23950 -23951 -23952 -23953 -23954 -23955 -23956 -23957 -23958 -23959 -23960 -23961 -23962 -23963 -23964 -23965 -23966 -23967 -23968 -23969 -23970 -23971 -23972 -23973 -23974 -23975 -23976 -23977 -23978 -23979 -23980 -23981 -23982 -23983 -23984 -23985 -23986 -23987 -23988 -23989 -23990 -23991 -23992 -23993 -23994 -23995 -23996 -23997 -23998 -23999 -24000 -24001 -24002 -24003 -24004 -24005 -24006 -24007 -24008 -24009 -24010 -24011 -24012 -24013 -24014 -24015 -24016 -24017 -24018 -24019 -24020 -24021 -24022 -24023 -24024 -24025 -24026 -24027 -24028 -24029 -24030 -24031 -24032 -24033 -24034 -24035 -24036 -24037 -24038 -24039 -24040 -24041 -24042 -24043 -24044 -24045 -24046 -24047 -24048 -24049 -24050 -24051 -24052 -24053 -24054 -24055 -24056 -24057 -24058 -24059 -24060 -24061 -24062 -24063 -24064 -24065 -24066 -24067 -24068 -24069 -24070 -24071 -24072 -24073 -24074 -24075 -24076 -24077 -24078 -24079 -24080 -24081 -24082 -24083 -24084 -24085 -24086 -24087 -24088 -24089 -24090 -24091 -24092 -24093 -24094 -24095 -24096 -24097 -24098 -24099 -24100 -24101 -24102 -24103 -24104 -24105 -24106 -24107 -24108 -24109 -24110 -24111 -24112 -24113 -24114 -24115 -24116 -24117 -24118 -24119 -24120 -24121 -24122 -24123 -24124 -24125 -24126 -24127 -24128 -24129 -24130 -24131 -24132 -24133 -24134 -24135 -24136 -24137 -24138 -24139 -24140 -24141 -24142 -24143 -24144 -24145 -24146 -24147 -24148 -24149 -24150 -24151 -24152 -24153 -24154 -24155 -24156 -24157 -24158 -24159 -24160 -24161 -24162 -24163 -24164 -24165 -24166 -24167 -24168 -24169 -24170 -24171 -24172 -24173 -24174 -24175 -24176 -24177 -24178 -24179 -24180 -24181 -24182 -24183 -24184 -24185 -24186 -24187 -24188 -24189 -24190 -24191 -24192 -24193 -24194 -24195 -24196 -24197 -24198 -24199 -24200 -24201 -24202 -24203 -24204 -24205 -24206 -24207 -24208 -24209 -24210 -24211 -24212 -24213 -24214 -24215 -24216 -24217 -24218 -24219 -24220 -24221 -24222 -24223 -24224 -24225 -24226 -24227 -24228 -24229 -24230 -24231 -24232 -24233 -24234 -24235 -24236 -24237 -24238 -24239 -24240 -24241 -24242 -24243 -24244 -24245 -24246 -24247 -24248 -24249 -24250 -24251 -24252 -24253 -24254 -24255 -24256 -24257 -24258 -24259 -24260 -24261 -24262 -24263 -24264 -24265 -24266 -24267 -24268 -24269 -24270 -24271 -24272 -24273 -24274 -24275 -24276 -24277 -24278 -24279 -24280 -24281 -24282 -24283 -24284 -24285 -24286 -24287 -24288 -24289 -24290 -24291 -24292 -24293 -24294 -24295 -24296 -24297 -24298 -24299 -24300 -24301 -24302 -24303 -24304 -24305 -24306 -24307 -24308 -24309 -24310 -24311 -24312 -24313 -24314 -24315 -24316 -24317 -24318 -24319 -24320 -24321 -24322 -24323 -24324 -24325 -24326 -24327 -24328 -24329 -24330 -24331 -24332 -24333 -24334 -24335 -24336 -24337 -24338 -24339 -24340 -24341 -24342 -24343 -24344 -24345 -24346 -24347 -24348 -24349 -24350 -24351 -24352 -24353 -24354 -24355 -24356 -24357 -24358 -24359 -24360 -24361 -24362 -24363 -24364 -24365 -24366 -24367 -24368 -24369 -24370 -24371 -24372 -24373 -24374 -24375 -24376 -24377 -24378 -24379 -24380 -24381 -24382 -24383 -24384 -24385 -24386 -24387 -24388 -24389 -24390 -24391 -24392 -24393 -24394 -24395 -24396 -24397 -24398 -24399 -24400 -24401 -24402 -24403 -24404 -24405 -24406 -24407 -24408 -24409 -24410 -24411 -24412 -24413 -24414 -24415 -24416 -24417 -24418 -24419 -24420 -24421 -24422 -24423 -24424 -24425 -24426 -24427 -24428 -24429 -24430 -24431 -24432 -24433 -24434 -24435 -24436 -24437 -24438 -24439 -24440 -24441 -24442 -24443 -24444 -24445 -24446 -24447 -24448 -24449 -24450 -24451 -24452 -24453 -24454 -24455 -24456 -24457 -24458 -24459 -24460 -24461 -24462 -24463 -24464 -24465 -24466 -24467 -24468 -24469 -24470 -24471 -24472 -24473 -24474 -24475 -24476 -24477 -24478 -24479 -24480 -24481 -24482 -24483 -24484 -24485 -24486 -24487 -24488 -24489 -24490 -24491 -24492 -24493 -24494 -24495 -24496 -24497 -24498 -24499 -24500 -24501 -24502 -24503 -24504 -24505 -24506 -24507 -24508 -24509 -24510 -24511 -24512 -24513 -24514 -24515 -24516 -24517 -24518 -24519 -24520 -24521 -24522 -24523 -24524 -24525 -24526 -24527 -24528 -24529 -24530 -24531 -24532 -24533 -24534 -24535 -24536 -24537 -24538 -24539 -24540 -24541 -24542 -24543 -24544 -24545 -24546 -24547 -24548 -24549 -24550 -24551 -24552 -24553 -24554 -24555 -24556 -24557 -24558 -24559 -24560 -24561 -24562 -24563 -24564 -24565 -24566 -24567 -24568 -24569 -24570 -24571 -24572 -24573 -24574 -24575 -24576 -24577 -24578 -24579 -24580 -24581 -24582 -24583 -24584 -24585 -24586 -24587 -24588 -24589 -24590 -24591 -24592 -24593 -24594 -24595 -24596 -24597 -24598 -24599 -24600 -24601 -24602 -24603 -24604 -24605 -24606 -24607 -24608 -24609 -24610 -24611 -24612 -24613 -24614 -24615 -24616 -24617 -24618 -24619 -24620 -24621 -24622 -24623 -24624 -24625 -24626 -24627 -24628 -24629 -24630 -24631 -24632 -24633 -24634 -24635 -24636 -24637 -24638 -24639 -24640 -24641 -24642 -24643 -24644 -24645 -24646 -24647 -24648 -24649 -24650 -24651 -24652 -24653 -24654 -24655 -24656 -24657 -24658 -24659 -24660 -24661 -24662 -24663 -24664 -24665 -24666 -24667 -24668 -24669 -24670 -24671 -24672 -24673 -24674 -24675 -24676 -24677 -24678 -24679 -24680 -24681 -24682 -24683 -24684 -24685 -24686 -24687 -24688 -24689 -24690 -24691 -24692 -24693 -24694 -24695 -24696 -24697 -24698 -24699 -24700 -24701 -24702 -24703 -24704 -24705 -24706 -24707 -24708 -24709 -24710 -24711 -24712 -24713 -24714 -24715 -24716 -24717 -24718 -24719 -24720 -24721 -24722 -24723 -24724 -24725 -24726 -24727 -24728 -24729 -24730 -24731 -24732 -24733 -24734 -24735 -24736 -24737 -24738 -24739 -24740 -24741 -24742 -24743 -24744 -24745 -24746 -24747 -24748 -24749 -24750 -24751 -24752 -24753 -24754 -24755 -24756 -24757 -24758 -24759 -24760 -24761 -24762 -24763 -24764 -24765 -24766 -24767 -24768 -24769 -24770 -24771 -24772 -24773 -24774 -24775 -24776 -24777 -24778 -24779 -24780 -24781 -24782 -24783 -24784 -24785 -24786 -24787 -24788 -24789 -24790 -24791 -24792 -24793 -24794 -24795 -24796 -24797 -24798 -24799 -24800 -24801 -24802 -24803 -24804 -24805 -24806 -24807 -24808 -24809 -24810 -24811 -24812 -24813 -24814 -24815 -24816 -24817 -24818 -24819 -24820 -24821 -24822 -24823 -24824 -24825 -24826 -24827 -24828 -24829 -24830 -24831 -24832 -24833 -24834 -24835 -24836 -24837 -24838 -24839 -24840 -24841 -24842 -24843 -24844 -24845 -24846 -24847 -24848 -24849 -24850 -24851 -24852 -24853 -24854 -24855 -24856 -24857 -24858 -24859 -24860 -24861 -24862 -24863 -24864 -24865 -24866 -24867 -24868 -24869 -24870 -24871 -24872 -24873 -24874 -24875 -24876 -24877 -24878 -24879 -24880 -24881 -24882 -24883 -24884 -24885 -24886 -24887 -24888 -24889 -24890 -24891 -24892 -24893 -24894 -24895 -24896 -24897 -24898 -24899 -24900 -24901 -24902 -24903 -24904 -24905 -24906 -24907 -24908 -24909 -24910 -24911 -24912 -24913 -24914 -24915 -24916 -24917 -24918 -24919 -24920 -24921 -24922 -24923 -24924 -24925 -24926 -24927 -24928 -24929 -24930 -24931 -24932 -24933 -24934 -24935 -24936 -24937 -24938 -24939 -24940 -24941 -24942 -24943 -24944 -24945 -24946 -24947 -24948 -24949 -24950 -24951 -24952 -24953 -24954 -24955 -24956 -24957 -24958 -24959 -24960 -24961 -24962 -24963 -24964 -24965 -24966 -24967 -24968 -24969 -24970 -24971 -24972 -24973 -24974 -24975 -24976 -24977 -24978 -24979 -24980 -24981 -24982 -24983 -24984 -24985 -24986 -24987 -24988 -24989 -24990 -24991 -24992 -24993 -24994 -24995 -24996 -24997 -24998 -24999 -25000 -25001 -25002 -25003 -25004 -25005 -25006 -25007 -25008 -25009 -25010 -25011 -25012 -25013 -25014 -25015 -25016 -25017 -25018 -25019 -25020 -25021 -25022 -25023 -25024 -25025 -25026 -25027 -25028 -25029 -25030 -25031 -25032 -25033 -25034 -25035 -25036 -25037 -25038 -25039 -25040 -25041 -25042 -25043 -25044 -25045 -25046 -25047 -25048 -25049 -25050 -25051 -25052 -25053 -25054 -25055 -25056 -25057 -25058 -25059 -25060 -25061 -25062 -25063 -25064 -25065 -25066 -25067 -25068 -25069 -25070 -25071 -25072 -25073 -25074 -25075 -25076 -25077 -25078 -25079 -25080 -25081 -25082 -25083 -25084 -25085 -25086 -25087 -25088 -25089 -25090 -25091 -25092 -25093 -25094 -25095 -25096 -25097 -25098 -25099 -25100 -25101 -25102 -25103 -25104 -25105 -25106 -25107 -25108 -25109 -25110 -25111 -25112 -25113 -25114 -25115 -25116 -25117 -25118 -25119 -25120 -25121 -25122 -25123 -25124 -25125 -25126 -25127 -25128 -25129 -25130 -25131 -25132 -25133 -25134 -25135 -25136 -25137 -25138 -25139 -25140 -25141 -25142 -25143 -25144 -25145 -25146 -25147 -25148 -25149 -25150 -25151 -25152 -25153 -25154 -25155 -25156 -25157 -25158 -25159 -25160 -25161 -25162 -25163 -25164 -25165 -25166 -25167 -25168 -25169 -25170 -25171 -25172 -25173 -25174 -25175 -25176 -25177 -25178 -25179 -25180 -25181 -25182 -25183 -25184 -25185 -25186 -25187 -25188 -25189 -25190 -25191 -25192 -25193 -25194 -25195 -25196 -25197 -25198 -25199 -25200 -25201 -25202 -25203 -25204 -25205 -25206 -25207 -25208 -25209 -25210 -25211 -25212 -25213 -25214 -25215 -25216 -25217 -25218 -25219 -25220 -25221 -25222 -25223 -25224 -25225 -25226 -25227 -25228 -25229 -25230 -25231 -25232 -25233 -25234 -25235 -25236 -25237 -25238 -25239 -25240 -25241 -25242 -25243 -25244 -25245 -25246 -25247 -25248 -25249 -25250 -25251 -25252 -25253 -25254 -25255 -25256 -25257 -25258 -25259 -25260 -25261 -25262 -25263 -25264 -25265 -25266 -25267 -25268 -25269 -25270 -25271 -25272 -25273 -25274 -25275 -25276 -25277 -25278 -25279 -25280 -25281 -25282 -25283 -25284 -25285 -25286 -25287 -25288 -25289 -25290 -25291 -25292 -25293 -25294 -25295 -25296 -25297 -25298 -25299 -25300 -25301 -25302 -25303 -25304 -25305 -25306 -25307 -25308 -25309 -25310 -25311 -25312 -25313 -25314 -25315 -25316 -25317 -25318 -25319 -25320 -25321 -25322 -25323 -25324 -25325 -25326 -25327 -25328 -25329 -25330 -25331 -25332 -25333 -25334 -25335 -25336 -25337 -25338 -25339 -25340 -25341 -25342 -25343 -25344 -25345 -25346 -25347 -25348 -25349 -25350 -25351 -25352 -25353 -25354 -25355 -25356 -25357 -25358 -25359 -25360 -25361 -25362 -25363 -25364 -25365 -25366 -25367 -25368 -25369 -25370 -25371 -25372 -25373 -25374 -25375 -25376 -25377 -25378 -25379 -25380 -25381 -25382 -25383 -25384 -25385 -25386 -25387 -25388 -25389 -25390 -25391 -25392 -25393 -25394 -25395 -25396 -25397 -25398 -25399 -25400 -25401 -25402 -25403 -25404 -25405 -25406 -25407 -25408 -25409 -25410 -25411 -25412 -25413 -25414 -25415 -25416 -25417 -25418 -25419 -25420 -25421 -25422 -25423 -25424 -25425 -25426 -25427 -25428 -25429 -25430 -25431 -25432 -25433 -25434 -25435 -25436 -25437 -25438 -25439 -25440 -25441 -25442 -25443 -25444 -25445 -25446 -25447 -25448 -25449 -25450 -25451 -25452 -25453 -25454 -25455 -25456 -25457 -25458 -25459 -25460 -25461 -25462 -25463 -25464 -25465 -25466 -25467 -25468 -25469 -25470 -25471 -25472 -25473 -25474 -25475 -25476 -25477 -25478 -25479 -25480 -25481 -25482 -25483 -25484 -25485 -25486 -25487 -25488 -25489 -25490 -25491 -25492 -25493 -25494 -25495 -25496 -25497 -25498 -25499 -25500 -25501 -25502 -25503 -25504 -25505 -25506 -25507 -25508 -25509 -25510 -25511 -25512 -25513 -25514 -25515 -25516 -25517 -25518 -25519 -25520 -25521 -25522 -25523 -25524 -25525 -25526 -25527 -25528 -25529 -25530 -25531 -25532 -25533 -25534 -25535 -25536 -25537 -25538 -25539 -25540 -25541 -25542 -25543 -25544 -25545 -25546 -25547 -25548 -25549 -25550 -25551 -25552 -25553 -25554 -25555 -25556 -25557 -25558 -25559 -25560 -25561 -25562 -25563 -25564 -25565 -25566 -25567 -25568 -25569 -25570 -25571 -25572 -25573 -25574 -25575 -25576 -25577 -25578 -25579 -25580 -25581 -25582 -25583 -25584 -25585 -25586 -25587 -25588 -25589 -25590 -25591 -25592 -25593 -25594 -25595 -25596 -25597 -25598 -25599 -25600 -25601 -25602 -25603 -25604 -25605 -25606 -25607 -25608 -25609 -25610 -25611 -25612 -25613 -25614 -25615 -25616 -25617 -25618 -25619 -25620 -25621 -25622 -25623 -25624 -25625 -25626 -25627 -25628 -25629 -25630 -25631 -25632 -25633 -25634 -25635 -25636 -25637 -25638 -25639 -25640 -25641 -25642 -25643 -25644 -25645 -25646 -25647 -25648 -25649 -25650 -25651 -25652 -25653 -25654 -25655 -25656 -25657 -25658 -25659 -25660 -25661 -25662 -25663 -25664 -25665 -25666 -25667 -25668 -25669 -25670 -25671 -25672 -25673 -25674 -25675 -25676 -25677 -25678 -25679 -25680 -25681 -25682 -25683 -25684 -25685 -25686 -25687 -25688 -25689 -25690 -25691 -25692 -25693 -25694 -25695 -25696 -25697 -25698 -25699 -25700 -25701 -25702 -25703 -25704 -25705 -25706 -25707 -25708 -25709 -25710 -25711 -25712 -25713 -25714 -25715 -25716 -25717 -25718 -25719 -25720 -25721 -25722 -25723 -25724 -25725 -25726 -25727 -25728 -25729 -25730 -25731 -25732 -25733 -25734 -25735 -25736 -25737 -25738 -25739 -25740 -25741 -25742 -25743 -25744 -25745 -25746 -25747 -25748 -25749 -25750 -25751 -25752 -25753 -25754 -25755 -25756 -25757 -25758 -25759 -25760 -25761 -25762 -25763 -25764 -25765 -25766 -25767 -25768 -25769 -25770 -25771 -25772 -25773 -25774 -25775 -25776 -25777 -25778 -25779 -25780 -25781 -25782 -25783 -25784 -25785 -25786 -25787 -25788 -25789 -25790 -25791 -25792 -25793 -25794 -25795 -25796 -25797 -25798 -25799 -25800 -25801 -25802 -25803 -25804 -25805 -25806 -25807 -25808 -25809 -25810 -25811 -25812 -25813 -25814 -25815 -25816 -25817 -25818 -25819 -25820 -25821 -25822 -25823 -25824 -25825 -25826 -25827 -25828 -25829 -25830 -25831 -25832 -25833 -25834 -25835 -25836 -25837 -25838 -25839 -25840 -25841 -25842 -25843 -25844 -25845 -25846 -25847 -25848 -25849 -25850 -25851 -25852 -25853 -25854 -25855 -25856 -25857 -25858 -25859 -25860 -25861 -25862 -25863 -25864 -25865 -25866 -25867 -25868 -25869 -25870 -25871 -25872 -25873 -25874 -25875 -25876 -25877 -25878 -25879 -25880 -25881 -25882 -25883 -25884 -25885 -25886 -25887 -25888 -25889 -25890 -25891 -25892 -25893 -25894 -25895 -25896 -25897 -25898 -25899 -25900 -25901 -25902 -25903 -25904 -25905 -25906 -25907 -25908 -25909 -25910 -25911 -25912 -25913 -25914 -25915 -25916 -25917 -25918 -25919 -25920 -25921 -25922 -25923 -25924 -25925 -25926 -25927 -25928 -25929 -25930 -25931 -25932 -25933 -25934 -25935 -25936 -25937 -25938 -25939 -25940 -25941 -25942 -25943 -25944 -25945 -25946 -25947 -25948 -25949 -25950 -25951 -25952 -25953 -25954 -25955 -25956 -25957 -25958 -25959 -25960 -25961 -25962 -25963 -25964 -25965 -25966 -25967 -25968 -25969 -25970 -25971 -25972 -25973 -25974 -25975 -25976 -25977 -25978 -25979 -25980 -25981 -25982 -25983 -25984 -25985 -25986 -25987 -25988 -25989 -25990 -25991 -25992 -25993 -25994 -25995 -25996 -25997 -25998 -25999 -26000 -26001 -26002 -26003 -26004 -26005 -26006 -26007 -26008 -26009 -26010 -26011 -26012 -26013 -26014 -26015 -26016 -26017 -26018 -26019 -26020 -26021 -26022 -26023 -26024 -26025 -26026 -26027 -26028 -26029 -26030 -26031 -26032 -26033 -26034 -26035 -26036 -26037 -26038 -26039 -26040 -26041 -26042 -26043 -26044 -26045 -26046 -26047 -26048 -26049 -26050 -26051 -26052 -26053 -26054 -26055 -26056 -26057 -26058 -26059 -26060 -26061 -26062 -26063 -26064 -26065 -26066 -26067 -26068 -26069 -26070 -26071 -26072 -26073 -26074 -26075 -26076 -26077 -26078 -26079 -26080 -26081 -26082 -26083 -26084 -26085 -26086 -26087 -26088 -26089 -26090 -26091 -26092 -26093 -26094 -26095 -26096 -26097 -26098 -26099 -26100 -26101 -26102 -26103 -26104 -26105 -26106 -26107 -26108 -26109 -26110 -26111 -26112 -26113 -26114 -26115 -26116 -26117 -26118 -26119 -26120 -26121 -26122 -26123 -26124 -26125 -26126 -26127 -26128 -26129 -26130 -26131 -26132 -26133 -26134 -26135 -26136 -26137 -26138 -26139 -26140 -26141 -26142 -26143 -26144 -26145 -26146 -26147 -26148 -26149 -26150 -26151 -26152 -26153 -26154 -26155 -26156 -26157 -26158 -26159 -26160 -26161 -26162 -26163 -26164 -26165 -26166 -26167 -26168 -26169 -26170 -26171 -26172 -26173 -26174 -26175 -26176 -26177 -26178 -26179 -26180 -26181 -26182 -26183 -26184 -26185 -26186 -26187 -26188 -26189 -26190 -26191 -26192 -26193 -26194 -26195 -26196 -26197 -26198 -26199 -26200 -26201 -26202 -26203 -26204 -26205 -26206 -26207 -26208 -26209 -26210 -26211 -26212 -26213 -26214 -26215 -26216 -26217 -26218 -26219 -26220 -26221 -26222 -26223 -26224 -26225 -26226 -26227 -26228 -26229 -26230 -26231 -26232 -26233 -26234 -26235 -26236 -26237 -26238 -26239 -26240 -26241 -26242 -26243 -26244 -26245 -26246 -26247 -26248 -26249 -26250 -26251 -26252 -26253 -26254 -26255 -26256 -26257 -26258 -26259 -26260 -26261 -26262 -26263 -26264 -26265 -26266 -26267 -26268 -26269 -26270 -26271 -26272 -26273 -26274 -26275 -26276 -26277 -26278 -26279 -26280 -26281 -26282 -26283 -26284 -26285 -26286 -26287 -26288 -26289 -26290 -26291 -26292 -26293 -26294 -26295 -26296 -26297 -26298 -26299 -26300 -26301 -26302 -26303 -26304 -26305 -26306 -26307 -26308 -26309 -26310 -26311 -26312 -26313 -26314 -26315 -26316 -26317 -26318 -26319 -26320 -26321 -26322 -26323 -26324 -26325 -26326 -26327 -26328 -26329 -26330 -26331 -26332 -26333 -26334 -26335 -26336 -26337 -26338 -26339 -26340 -26341 -26342 -26343 -26344 -26345 -26346 -26347 -26348 -26349 -26350 -26351 -26352 -26353 -26354 -26355 -26356 -26357 -26358 -26359 -26360 -26361 -26362 -26363 -26364 -26365 -26366 -26367 -26368 -26369 -26370 -26371 -26372 -26373 -26374 -26375 -26376 -26377 -26378 -26379 -26380 -26381 -26382 -26383 -26384 -26385 -26386 -26387 -26388 -26389 -26390 -26391 -26392 -26393 -26394 -26395 -26396 -26397 -26398 -26399 -26400 -26401 -26402 -26403 -26404 -26405 -26406 -26407 -26408 -26409 -26410 -26411 -26412 -26413 -26414 -26415 -26416 -26417 -26418 -26419 -26420 -26421 -26422 -26423 -26424 -26425 -26426 -26427 -26428 -26429 -26430 -26431 -26432 -26433 -26434 -26435 -26436 -26437 -26438 -26439 -26440 -26441 -26442 -26443 -26444 -26445 -26446 -26447 -26448 -26449 -26450 -26451 -26452 -26453 -26454 -26455 -26456 -26457 -26458 -26459 -26460 -26461 -26462 -26463 -26464 -26465 -26466 -26467 -26468 -26469 -26470 -26471 -26472 -26473 -26474 -26475 -26476 -26477 -26478 -26479 -26480 -26481 -26482 -26483 -26484 -26485 -26486 -26487 -26488 -26489 -26490 -26491 -26492 -26493 -26494 -26495 -26496 -26497 -26498 -26499 -26500 -26501 -26502 -26503 -26504 -26505 -26506 -26507 -26508 -26509 -26510 -26511 -26512 -26513 -26514 -26515 -26516 -26517 -26518 -26519 -26520 -26521 -26522 -26523 -26524 -26525 -26526 -26527 -26528 -26529 -26530 -26531 -26532 -26533 -26534 -26535 -26536 -26537 -26538 -26539 -26540 -26541 -26542 -26543 -26544 -26545 -26546 -26547 -26548 -26549 -26550 -26551 -26552 -26553 -26554 -26555 -26556 -26557 -26558 -26559 -26560 -26561 -26562 -26563 -26564 -26565 -26566 -26567 -26568 -26569 -26570 -26571 -26572 -26573 -26574 -26575 -26576 -26577 -26578 -26579 -26580 -26581 -26582 -26583 -26584 -26585 -26586 -26587 -26588 -26589 -26590 -26591 -26592 -26593 -26594 -26595 -26596 -26597 -26598 -26599 -26600 -26601 -26602 -26603 -26604 -26605 -26606 -26607 -26608 -26609 -26610 -26611 -26612 -26613 -26614 -26615 -26616 -26617 -26618 -26619 -26620 -26621 -26622 -26623 -26624 -26625 -26626 -26627 -26628 -26629 -26630 -26631 -26632 -26633 -26634 -26635 -26636 -26637 -26638 -26639 -26640 -26641 -26642 -26643 -26644 -26645 -26646 -26647 -26648 -26649 -26650 -26651 -26652 -26653 -26654 -26655 -26656 -26657 -26658 -26659 -26660 -26661 -26662 -26663 -26664 -26665 -26666 -26667 -26668 -26669 -26670 -26671 -26672 -26673 -26674 -26675 -26676 -26677 -26678 -26679 -26680 -26681 -26682 -26683 -26684 -26685 -26686 -26687 -26688 -26689 -26690 -26691 -26692 -26693 -26694 -26695 -26696 -26697 -26698 -26699 -26700 -26701 -26702 -26703 -26704 -26705 -26706 -26707 -26708 -26709 -26710 -26711 -26712 -26713 -26714 -26715 -26716 -26717 -26718 -26719 -26720 -26721 -26722 -26723 -26724 -26725 -26726 -26727 -26728 -26729 -26730 -26731 -26732 -26733 -26734 -26735 -26736 -26737 -26738 -26739 -26740 -26741 -26742 -26743 -26744 -26745 -26746 -26747 -26748 -26749 -26750 -26751 -26752 -26753 -26754 -26755 -26756 -26757 -26758 -26759 -26760 -26761 -26762 -26763 -26764 -26765 -26766 -26767 -26768 -26769 -26770 -26771 -26772 -26773 -26774 -26775 -26776 -26777 -26778 -26779 -26780 -26781 -26782 -26783 -26784 -26785 -26786 -26787 -26788 -26789 -26790 -26791 -26792 -26793 -26794 -26795 -26796 -26797 -26798 -26799 -26800 -26801 -26802 -26803 -26804 -26805 -26806 -26807 -26808 -26809 -26810 -26811 -26812 -26813 -26814 -26815 -26816 -26817 -26818 -26819 -26820 -26821 -26822 -26823 -26824 -26825 -26826 -26827 -26828 -26829 -26830 -26831 -26832 -26833 -26834 -26835 -26836 -26837 -26838 -26839 -26840 -26841 -26842 -26843 -26844 -26845 -26846 -26847 -26848 -26849 -26850 -26851 -26852 -26853 -26854 -26855 -26856 -26857 -26858 -26859 -26860 -26861 -26862 -26863 -26864 -26865 -26866 -26867 -26868 -26869 -26870 -26871 -26872 -26873 -26874 -26875 -26876 -26877 -26878 -26879 -26880 -26881 -26882 -26883 -26884 -26885 -26886 -26887 -26888 -26889 -26890 -26891 -26892 -26893 -26894 -26895 -26896 -26897 -26898 -26899 -26900 -26901 -26902 -26903 -26904 -26905 -26906 -26907 -26908 -26909 -26910 -26911 -26912 -26913 -26914 -26915 -26916 -26917 -26918 -26919 -26920 -26921 -26922 -26923 -26924 -26925 -26926 -26927 -26928 -26929 -26930 -26931 -26932 -26933 -26934 -26935 -26936 -26937 -26938 -26939 -26940 -26941 -26942 -26943 -26944 -26945 -26946 -26947 -26948 -26949 -26950 -26951 -26952 -26953 -26954 -26955 -26956 -26957 -26958 -26959 -26960 -26961 -26962 -26963 -26964 -26965 -26966 -26967 -26968 -26969 -26970 -26971 -26972 -26973 -26974 -26975 -26976 -26977 -26978 -26979 -26980 -26981 -26982 -26983 -26984 -26985 -26986 -26987 -26988 -26989 -26990 -26991 -26992 -26993 -26994 -26995 -26996 -26997 -26998 -26999 -27000 -27001 -27002 -27003 -27004 -27005 -27006 -27007 -27008 -27009 -27010 -27011 -27012 -27013 -27014 -27015 -27016 -27017 -27018 -27019 -27020 -27021 -27022 -27023 -27024 -27025 -27026 -27027 -27028 -27029 -27030 -27031 -27032 -27033 -27034 -27035 -27036 -27037 -27038 -27039 -27040 -27041 -27042 -27043 -27044 -27045 -27046 -27047 -27048 -27049 -27050 -27051 -27052 -27053 -27054 -27055 -27056 -27057 -27058 -27059 -27060 -27061 -27062 -27063 -27064 -27065 -27066 -27067 -27068 -27069 -27070 -27071 -27072 -27073 -27074 -27075 -27076 -27077 -27078 -27079 -27080 -27081 -27082 -27083 -27084 -27085 -27086 -27087 -27088 -27089 -27090 -27091 -27092 -27093 -27094 -27095 -27096 -27097 -27098 -27099 -27100 -27101 -27102 -27103 -27104 -27105 -27106 -27107 -27108 -27109 -27110 -27111 -27112 -27113 -27114 -27115 -27116 -27117 -27118 -27119 -27120 -27121 -27122 -27123 -27124 -27125 -27126 -27127 -27128 -27129 -27130 -27131 -27132 -27133 -27134 -27135 -27136 -27137 -27138 -27139 -27140 -27141 -27142 -27143 -27144 -27145 -27146 -27147 -27148 -27149 -27150 -27151 -27152 -27153 -27154 -27155 -27156 -27157 -27158 -27159 -27160 -27161 -27162 -27163 -27164 -27165 -27166 -27167 -27168 -27169 -27170 -27171 -27172 -27173 -27174 -27175 -27176 -27177 -27178 -27179 -27180 -27181 -27182 -27183 -27184 -27185 -27186 -27187 -27188 -27189 -27190 -27191 -27192 -27193 -27194 -27195 -27196 -27197 -27198 -27199 -27200 -27201 -27202 -27203 -27204 -27205 -27206 -27207 -27208 -27209 -27210 -27211 -27212 -27213 -27214 -27215 -27216 -27217 -27218 -27219 -27220 -27221 -27222 -27223 -27224 -27225 -27226 -27227 -27228 -27229 -27230 -27231 -27232 -27233 -27234 -27235 -27236 -27237 -27238 -27239 -27240 -27241 -27242 -27243 -27244 -27245 -27246 -27247 -27248 -27249 -27250 -27251 -27252 -27253 -27254 -27255 -27256 -27257 -27258 -27259 -27260 -27261 -27262 -27263 -27264 -27265 -27266 -27267 -27268 -27269 -27270 -27271 -27272 -27273 -27274 -27275 -27276 -27277 -27278 -27279 -27280 -27281 -27282 -27283 -27284 -27285 -27286 -27287 -27288 -27289 -27290 -27291 -27292 -27293 -27294 -27295 -27296 -27297 -27298 -27299 -27300 -27301 -27302 -27303 -27304 -27305 -27306 -27307 -27308 -27309 -27310 -27311 -27312 -27313 -27314 -27315 -27316 -27317 -27318 -27319 -27320 -27321 -27322 -27323 -27324 -27325 -27326 -27327 -27328 -27329 -27330 -27331 -27332 -27333 -27334 -27335 -27336 -27337 -27338 -27339 -27340 -27341 -27342 -27343 -27344 -27345 -27346 -27347 -27348 -27349 -27350 -27351 -27352 -27353 -27354 -27355 -27356 -27357 -27358 -27359 -27360 -27361 -27362 -27363 -27364 -27365 -27366 -27367 -27368 -27369 -27370 -27371 -27372 -27373 -27374 -27375 -27376 -27377 -27378 -27379 -27380 -27381 -27382 -27383 -27384 -27385 -27386 -27387 -27388 -27389 -27390 -27391 -27392 -27393 -27394 -27395 -27396 -27397 -27398 -27399 -27400 -27401 -27402 -27403 -27404 -27405 -27406 -27407 -27408 -27409 -27410 -27411 -27412 -27413 -27414 -27415 -27416 -27417 -27418 -27419 -27420 -27421 -27422 -27423 -27424 -27425 -27426 -27427 -27428 -27429 -27430 -27431 -27432 -27433 -27434 -27435 -27436 -27437 -27438 -27439 -27440 -27441 -27442 -27443 -27444 -27445 -27446 -27447 -27448 -27449 -27450 -27451 -27452 -27453 -27454 -27455 -27456 -27457 -27458 -27459 -27460 -27461 -27462 -27463 -27464 -27465 -27466 -27467 -27468 -27469 -27470 -27471 -27472 -27473 -27474 -27475 -27476 -27477 -27478 -27479 -27480 -27481 -27482 -27483 -27484 -27485 -27486 -27487 -27488 -27489 -27490 -27491 -27492 -27493 -27494 -27495 -27496 -27497 -27498 -27499 -27500 -27501 -27502 -27503 -27504 -27505 -27506 -27507 -27508 -27509 -27510 -27511 -27512 -27513 -27514 -27515 -27516 -27517 -27518 -27519 -27520 -27521 -27522 -27523 -27524 -27525 -27526 -27527 -27528 -27529 -27530 -27531 -27532 -27533 -27534 -27535 -27536 -27537 -27538 -27539 -27540 -27541 -27542 -27543 -27544 -27545 -27546 -27547 -27548 -27549 -27550 -27551 -27552 -27553 -27554 -27555 -27556 -27557 -27558 -27559 -27560 -27561 -27562 -27563 -27564 -27565 -27566 -27567 -27568 -27569 -27570 -27571 -27572 -27573 -27574 -27575 -27576 -27577 -27578 -27579 -27580 -27581 -27582 -27583 -27584 -27585 -27586 -27587 -27588 -27589 -27590 -27591 -27592 -27593 -27594 -27595 -27596 -27597 -27598 -27599 -27600 -27601 -27602 -27603 -27604 -27605 -27606 -27607 -27608 -27609 -27610 -27611 -27612 -27613 -27614 -27615 -27616 -27617 -27618 -27619 -27620 -27621 -27622 -27623 -27624 -27625 -27626 -27627 -27628 -27629 -27630 -27631 -27632 -27633 -27634 -27635 -27636 -27637 -27638 -27639 -27640 -27641 -27642 -27643 -27644 -27645 -27646 -27647 -27648 -27649 -27650 -27651 -27652 -27653 -27654 -27655 -27656 -27657 -27658 -27659 -27660 -27661 -27662 -27663 -27664 -27665 -27666 -27667 -27668 -27669 -27670 -27671 -27672 -27673 -27674 -27675 -27676 -27677 -27678 -27679 -27680 -27681 -27682 -27683 -27684 -27685 -27686 -27687 -27688 -27689 -27690 -27691 -27692 -27693 -27694 -27695 -27696 -27697 -27698 -27699 -27700 -27701 -27702 -27703 -27704 -27705 -27706 -27707 -27708 -27709 -27710 -27711 -27712 -27713 -27714 -27715 -27716 -27717 -27718 -27719 -27720 -27721 -27722 -27723 -27724 -27725 -27726 -27727 -27728 -27729 -27730 -27731 -27732 -27733 -27734 -27735 -27736 -27737 -27738 -27739 -27740 -27741 -27742 -27743 -27744 -27745 -27746 -27747 -27748 -27749 -27750 -27751 -27752 -27753 -27754 -27755 -27756 -27757 -27758 -27759 -27760 -27761 -27762 -27763 -27764 -27765 -27766 -27767 -27768 -27769 -27770 -27771 -27772 -27773 -27774 -27775 -27776 -27777 -27778 -27779 -27780 -27781 -27782 -27783 -27784 -27785 -27786 -27787 -27788 -27789 -27790 -27791 -27792 -27793 -27794 -27795 -27796 -27797 -27798 -27799 -27800 -27801 -27802 -27803 -27804 -27805 -27806 -27807 -27808 -27809 -27810 -27811 -27812 -27813 -27814 -27815 -27816 -27817 -27818 -27819 -27820 -27821 -27822 -27823 -27824 -27825 -27826 -27827 -27828 -27829 -27830 -27831 -27832 -27833 -27834 -27835 -27836 -27837 -27838 -27839 -27840 -27841 -27842 -27843 -27844 -27845 -27846 -27847 -27848 -27849 -27850 -27851 -27852 -27853 -27854 -27855 -27856 -27857 -27858 -27859 -27860 -27861 -27862 -27863 -27864 -27865 -27866 -27867 -27868 -27869 -27870 -27871 -27872 -27873 -27874 -27875 -27876 -27877 -27878 -27879 -27880 -27881 -27882 -27883 -27884 -27885 -27886 -27887 -27888 -27889 -27890 -27891 -27892 -27893 -27894 -27895 -27896 -27897 -27898 -27899 -27900 -27901 -27902 -27903 -27904 -27905 -27906 -27907 -27908 -27909 -27910 -27911 -27912 -27913 -27914 -27915 -27916 -27917 -27918 -27919 -27920 -27921 -27922 -27923 -27924 -27925 -27926 -27927 -27928 -27929 -27930 -27931 -27932 -27933 -27934 -27935 -27936 -27937 -27938 -27939 -27940 -27941 -27942 -27943 -27944 -27945 -27946 -27947 -27948 -27949 -27950 -27951 -27952 -27953 -27954 -27955 -27956 -27957 -27958 -27959 -27960 -27961 -27962 -27963 -27964 -27965 -27966 -27967 -27968 -27969 -27970 -27971 -27972 -27973 -27974 -27975 -27976 -27977 -27978 -27979 -27980 -27981 -27982 -27983 -27984 -27985 -27986 -27987 -27988 -27989 -27990 -27991 -27992 -27993 -27994 -27995 -27996 -27997 -27998 -27999 -28000 -28001 -28002 -28003 -28004 -28005 -28006 -28007 -28008 -28009 -28010 -28011 -28012 -28013 -28014 -28015 -28016 -28017 -28018 -28019 -28020 -28021 -28022 -28023 -28024 -28025 -28026 -28027 -28028 -28029 -28030 -28031 -28032 -28033 -28034 -28035 -28036 -28037 -28038 -28039 -28040 -28041 -28042 -28043 -28044 -28045 -28046 -28047 -28048 -28049 -28050 -28051 -28052 -28053 -28054 -28055 -28056 -28057 -28058 -28059 -28060 -28061 -28062 -28063 -28064 -28065 -28066 -28067 -28068 -28069 -28070 -28071 -28072 -28073 -28074 -28075 -28076 -28077 -28078 -28079 -28080 -28081 -28082 -28083 -28084 -28085 -28086 -28087 -28088 -28089 -28090 -28091 -28092 -28093 -28094 -28095 -28096 -28097 -28098 -28099 -28100 -28101 -28102 -28103 -28104 -28105 -28106 -28107 -28108 -28109 -28110 -28111 -28112 -28113 -28114 -28115 -28116 -28117 -28118 -28119 -28120 -28121 -28122 -28123 -28124 -28125 -28126 -28127 -28128 -28129 -28130 -28131 -28132 -28133 -28134 -28135 -28136 -28137 -28138 -28139 -28140 -28141 -28142 -28143 -28144 -28145 -28146 -28147 -28148 -28149 -28150 -28151 -28152 -28153 -28154 -28155 -28156 -28157 -28158 -28159 -28160 -28161 -28162 -28163 -28164 -28165 -28166 -28167 -28168 -28169 -28170 -28171 -28172 -28173 -28174 -28175 -28176 -28177 -28178 -28179 -28180 -28181 -28182 -28183 -28184 -28185 -28186 -28187 -28188 -28189 -28190 -28191 -28192 -28193 -28194 -28195 -28196 -28197 -28198 -28199 -28200 -28201 -28202 -28203 -28204 -28205 -28206 -28207 -28208 -28209 -28210 -28211 -28212 -28213 -28214 -28215 -28216 -28217 -28218 -28219 -28220 -28221 -28222 -28223 -28224 -28225 -28226 -28227 -28228 -28229 -28230 -28231 -28232 -28233 -28234 -28235 -28236 -28237 -28238 -28239 -28240 -28241 -28242 -28243 -28244 -28245 -28246 -28247 -28248 -28249 -28250 -28251 -28252 -28253 -28254 -28255 -28256 -28257 -28258 -28259 -28260 -28261 -28262 -28263 -28264 -28265 -28266 -28267 -28268 -28269 -28270 -28271 -28272 -28273 -28274 -28275 -28276 -28277 -28278 -28279 -28280 -28281 -28282 -28283 -28284 -28285 -28286 -28287 -28288 -28289 -28290 -28291 -28292 -28293 -28294 -28295 -28296 -28297 -28298 -28299 -28300 -28301 -28302 -28303 -28304 -28305 -28306 -28307 -28308 -28309 -28310 -28311 -28312 -28313 -28314 -28315 -28316 -28317 -28318 -28319 -28320 -28321 -28322 -28323 -28324 -28325 -28326 -28327 -28328 -28329 -28330 -28331 -28332 -28333 -28334 -28335 -28336 -28337 -28338 -28339 -28340 -28341 -28342 -28343 -28344 -28345 -28346 -28347 -28348 -28349 -28350 -28351 -28352 -28353 -28354 -28355 -28356 -28357 -28358 -28359 -28360 -28361 -28362 -28363 -28364 -28365 -28366 -28367 -28368 -28369 -28370 -28371 -28372 -28373 -28374 -28375 -28376 -28377 -28378 -28379 -28380 -28381 -28382 -28383 -28384 -28385 -28386 -28387 -28388 -28389 -28390 -28391 -28392 -28393 -28394 -28395 -28396 -28397 -28398 -28399 -28400 -28401 -28402 -28403 -28404 -28405 -28406 -28407 -28408 -28409 -28410 -28411 -28412 -28413 -28414 -28415 -28416 -28417 -28418 -28419 -28420 -28421 -28422 -28423 -28424 -28425 -28426 -28427 -28428 -28429 -28430 -28431 -28432 -28433 -28434 -28435 -28436 -28437 -28438 -28439 -28440 -28441 -28442 -28443 -28444 -28445 -28446 -28447 -28448 -28449 -28450 -28451 -28452 -28453 -28454 -28455 -28456 -28457 -28458 -28459 -28460 -28461 -28462 -28463 -28464 -28465 -28466 -28467 -28468 -28469 -28470 -28471 -28472 -28473 -28474 -28475 -28476 -28477 -28478 -28479 -28480 -28481 -28482 -28483 -28484 -28485 -28486 -28487 -28488 -28489 -28490 -28491 -28492 -28493 -28494 -28495 -28496 -28497 -28498 -28499 -28500 -28501 -28502 -28503 -28504 -28505 -28506 -28507 -28508 -28509 -28510 -28511 -28512 -28513 -28514 -28515 -28516 -28517 -28518 -28519 -28520 -28521 -28522 -28523 -28524 -28525 -28526 -28527 -28528 -28529 -28530 -28531 -28532 -28533 -28534 -28535 -28536 -28537 -28538 -28539 -28540 -28541 -28542 -28543 -28544 -28545 -28546 -28547 -28548 -28549 -28550 -28551 -28552 -28553 -28554 -28555 -28556 -28557 -28558 -28559 -28560 -28561 -28562 -28563 -28564 -28565 -28566 -28567 -28568 -28569 -28570 -28571 -28572 -28573 -28574 -28575 -28576 -28577 -28578 -28579 -28580 -28581 -28582 -28583 -28584 -28585 -28586 -28587 -28588 -28589 -28590 -28591 -28592 -28593 -28594 -28595 -28596 -28597 -28598 -28599 -28600 -28601 -28602 -28603 -28604 -28605 -28606 -28607 -28608 -28609 -28610 -28611 -28612 -28613 -28614 -28615 -28616 -28617 -28618 -28619 -28620 -28621 -28622 -28623 -28624 -28625 -28626 -28627 -28628 -28629 -28630 -28631 -28632 -28633 -28634 -28635 -28636 -28637 -28638 -28639 -28640 -28641 -28642 -28643 -28644 -28645 -28646 -28647 -28648 -28649 -28650 -28651 -28652 -28653 -28654 -28655 -28656 -28657 -28658 -28659 -28660 -28661 -28662 -28663 -28664 -28665 -28666 -28667 -28668 -28669 -28670 -28671 -28672 -28673 -28674 -28675 -28676 -28677 -28678 -28679 -28680 -28681 -28682 -28683 -28684 -28685 -28686 -28687 -28688 -28689 -28690 -28691 -28692 -28693 -28694 -28695 -28696 -28697 -28698 -28699 -28700 -28701 -28702 -28703 -28704 -28705 -28706 -28707 -28708 -28709 -28710 -28711 -28712 -28713 -28714 -28715 -28716 -28717 -28718 -28719 -28720 -28721 -28722 -28723 -28724 -28725 -28726 -28727 -28728 -28729 -28730 -28731 -28732 -28733 -28734 -28735 -28736 -28737 -28738 -28739 -28740 -28741 -28742 -28743 -28744 -28745 -28746 -28747 -28748 -28749 -28750 -28751 -28752 -28753 -28754 -28755 -28756 -28757 -28758 -28759 -28760 -28761 -28762 -28763 -28764 -28765 -28766 -28767 -28768 -28769 -28770 -28771 -28772 -28773 -28774 -28775 -28776 -28777 -28778 -28779 -28780 -28781 -28782 -28783 -28784 -28785 -28786 -28787 -28788 -28789 -28790 -28791 -28792 -28793 -28794 -28795 -28796 -28797 -28798 -28799 -28800 -28801 -28802 -28803 -28804 -28805 -28806 -28807 -28808 -28809 -28810 -28811 -28812 -28813 -28814 -28815 -28816 -28817 -28818 -28819 -28820 -28821 -28822 -28823 -28824 -28825 -28826 -28827 -28828 -28829 -28830 -28831 -28832 -28833 -28834 -28835 -28836 -28837 -28838 -28839 -28840 -28841 -28842 -28843 -28844 -28845 -28846 -28847 -28848 -28849 -28850 -28851 -28852 -28853 -28854 -28855 -28856 -28857 -28858 -28859 -28860 -28861 -28862 -28863 -28864 -28865 -28866 -28867 -28868 -28869 -28870 -28871 -28872 -28873 -28874 -28875 -28876 -28877 -28878 -28879 -28880 -28881 -28882 -28883 -28884 -28885 -28886 -28887 -28888 -28889 -28890 -28891 -28892 -28893 -28894 -28895 -28896 -28897 -28898 -28899 -28900 -28901 -28902 -28903 -28904 -28905 -28906 -28907 -28908 -28909 -28910 -28911 -28912 -28913 -28914 -28915 -28916 -28917 -28918 -28919 -28920 -28921 -28922 -28923 -28924 -28925 -28926 -28927 -28928 -28929 -28930 -28931 -28932 -28933 -28934 -28935 -28936 -28937 -28938 -28939 -28940 -28941 -28942 -28943 -28944 -28945 -28946 -28947 -28948 -28949 -28950 -28951 -28952 -28953 -28954 -28955 -28956 -28957 -28958 -28959 -28960 -28961 -28962 -28963 -28964 -28965 -28966 -28967 -28968 -28969 -28970 -28971 -28972 -28973 -28974 -28975 -28976 -28977 -28978 -28979 -28980 -28981 -28982 -28983 -28984 -28985 -28986 -28987 -28988 -28989 -28990 -28991 -28992 -28993 -28994 -28995 -28996 -28997 -28998 -28999 -29000 -29001 -29002 -29003 -29004 -29005 -29006 -29007 -29008 -29009 -29010 -29011 -29012 -29013 -29014 -29015 -29016 -29017 -29018 -29019 -29020 -29021 -29022 -29023 -29024 -29025 -29026 -29027 -29028 -29029 -29030 -29031 -29032 -29033 -29034 -29035 -29036 -29037 -29038 -29039 -29040 -29041 -29042 -29043 -29044 -29045 -29046 -29047 -29048 -29049 -29050 -29051 -29052 -29053 -29054 -29055 -29056 -29057 -29058 -29059 -29060 -29061 -29062 -29063 -29064 -29065 -29066 -29067 -29068 -29069 -29070 -29071 -29072 -29073 -29074 -29075 -29076 -29077 -29078 -29079 -29080 -29081 -29082 -29083 -29084 -29085 -29086 -29087 -29088 -29089 -29090 -29091 -29092 -29093 -29094 -29095 -29096 -29097 -29098 -29099 -29100 -29101 -29102 -29103 -29104 -29105 -29106 -29107 -29108 -29109 -29110 -29111 -29112 -29113 -29114 -29115 -29116 -29117 -29118 -29119 -29120 -29121 -29122 -29123 -29124 -29125 -29126 -29127 -29128 -29129 -29130 -29131 -29132 -29133 -29134 -29135 -29136 -29137 -29138 -29139 -29140 -29141 -29142 -29143 -29144 -29145 -29146 -29147 -29148 -29149 -29150 -29151 -29152 -29153 -29154 -29155 -29156 -29157 -29158 -29159 -29160 -29161 -29162 -29163 -29164 -29165 -29166 -29167 -29168 -29169 -29170 -29171 -29172 -29173 -29174 -29175 -29176 -29177 -29178 -29179 -29180 -29181 -29182 -29183 -29184 -29185 -29186 -29187 -29188 -29189 -29190 -29191 -29192 -29193 -29194 -29195 -29196 -29197 -29198 -29199 -29200 -29201 -29202 -29203 -29204 -29205 -29206 -29207 -29208 -29209 -29210 -29211 -29212 -29213 -29214 -29215 -29216 -29217 -29218 -29219 -29220 -29221 -29222 -29223 -29224 -29225 -29226 -29227 -29228 -29229 -29230 -29231 -29232 -29233 -29234 -29235 -29236 -29237 -29238 -29239 -29240 -29241 -29242 -29243 -29244 -29245 -29246 -29247 -29248 -29249 -29250 -29251 -29252 -29253 -29254 -29255 -29256 -29257 -29258 -29259 -29260 -29261 -29262 -29263 -29264 -29265 -29266 -29267 -29268 -29269 -29270 -29271 -29272 -29273 -29274 -29275 -29276 -29277 -29278 -29279 -29280 -29281 -29282 -29283 -29284 -29285 -29286 -29287 -29288 -29289 -29290 -29291 -29292 -29293 -29294 -29295 -29296 -29297 -29298 -29299 -29300 -29301 -29302 -29303 -29304 -29305 -29306 -29307 -29308 -29309 -29310 -29311 -29312 -29313 -29314 -29315 -29316 -29317 -29318 -29319 -29320 -29321 -29322 -29323 -29324 -29325 -29326 -29327 -29328 -29329 -29330 -29331 -29332 -29333 -29334 -29335 -29336 -29337 -29338 -29339 -29340 -29341 -29342 -29343 -29344 -29345 -29346 -29347 -29348 -29349 -29350 -29351 -29352 -29353 -29354 -29355 -29356 -29357 -29358 -29359 -29360 -29361 -29362 -29363 -29364 -29365 -29366 -29367 -29368 -29369 -29370 -29371 -29372 -29373 -29374 -29375 -29376 -29377 -29378 -29379 -29380 -29381 -29382 -29383 -29384 -29385 -29386 -29387 -29388 -29389 -29390 -29391 -29392 -29393 -29394 -29395 -29396 -29397 -29398 -29399 -29400 -29401 -29402 -29403 -29404 -29405 -29406 -29407 -29408 -29409 -29410 -29411 -29412 -29413 -29414 -29415 -29416 -29417 -29418 -29419 -29420 -29421 -29422 -29423 -29424 -29425 -29426 -29427 -29428 -29429 -29430 -29431 -29432 -29433 -29434 -29435 -29436 -29437 -29438 -29439 -29440 -29441 -29442 -29443 -29444 -29445 -29446 -29447 -29448 -29449 -29450 -29451 -29452 -29453 -29454 -29455 -29456 -29457 -29458 -29459 -29460 -29461 -29462 -29463 -29464 -29465 -29466 -29467 -29468 -29469 -29470 -29471 -29472 -29473 -29474 -29475 -29476 -29477 -29478 -29479 -29480 -29481 -29482 -29483 -29484 -29485 -29486 -29487 -29488 -29489 -29490 -29491 -29492 -29493 -29494 -29495 -29496 -29497 -29498 -29499 -29500 -29501 -29502 -29503 -29504 -29505 -29506 -29507 -29508 -29509 -29510 -29511 -29512 -29513 -29514 -29515 -29516 -29517 -29518 -29519 -29520 -29521 -29522 -29523 -29524 -29525 -29526 -29527 -29528 -29529 -29530 -29531 -29532 -29533 -29534 -29535 -29536 -29537 -29538 -29539 -29540 -29541 -29542 -29543 -29544 -29545 -29546 -29547 -29548 -29549 -29550 -29551 -29552 -29553 -29554 -29555 -29556 -29557 -29558 -29559 -29560 -29561 -29562 -29563 -29564 -29565 -29566 -29567 -29568 -29569 -29570 -29571 -29572 -29573 -29574 -29575 -29576 -29577 -29578 -29579 -29580 -29581 -29582 -29583 -29584 -29585 -29586 -29587 -29588 -29589 -29590 -29591 -29592 -29593 -29594 -29595 -29596 -29597 -29598 -29599 -29600 -29601 -29602 -29603 -29604 -29605 -29606 -29607 -29608 -29609 -29610 -29611 -29612 -29613 -29614 -29615 -29616 -29617 -29618 -29619 -29620 -29621 -29622 -29623 -29624 -29625 -29626 -29627 -29628 -29629 -29630 -29631 -29632 -29633 -29634 -29635 -29636 -29637 -29638 -29639 -29640 -29641 -29642 -29643 -29644 -29645 -29646 -29647 -29648 -29649 -29650 -29651 -29652 -29653 -29654 -29655 -29656 -29657 -29658 -29659 -29660 -29661 -29662 -29663 -29664 -29665 -29666 -29667 -29668 -29669 -29670 -29671 -29672 -29673 -29674 -29675 -29676 -29677 -29678 -29679 -29680 -29681 -29682 -29683 -29684 -29685 -29686 -29687 -29688 -29689 -29690 -29691 -29692 -29693 -29694 -29695 -29696 -29697 -29698 -29699 -29700 -29701 -29702 -29703 -29704 -29705 -29706 -29707 -29708 -29709 -29710 -29711 -29712 -29713 -29714 -29715 -29716 -29717 -29718 -29719 -29720 -29721 -29722 -29723 -29724 -29725 -29726 -29727 -29728 -29729 -29730 -29731 -29732 -29733 -29734 -29735 -29736 -29737 -29738 -29739 -29740 -29741 -29742 -29743 -29744 -29745 -29746 -29747 -29748 -29749 -29750 -29751 -29752 -29753 -29754 -29755 -29756 -29757 -29758 -29759 -29760 -29761 -29762 -29763 -29764 -29765 -29766 -29767 -29768 -29769 -29770 -29771 -29772 -29773 -29774 -29775 -29776 -29777 -29778 -29779 -29780 -29781 -29782 -29783 -29784 -29785 -29786 -29787 -29788 -29789 -29790 -29791 -29792 -29793 -29794 -29795 -29796 -29797 -29798 -29799 -29800 -29801 -29802 -29803 -29804 -29805 -29806 -29807 -29808 -29809 -29810 -29811 -29812 -29813 -29814 -29815 -29816 -29817 -29818 -29819 -29820 -29821 -29822 -29823 -29824 -29825 -29826 -29827 -29828 -29829 -29830 -29831 -29832 -29833 -29834 -29835 -29836 -29837 -29838 -29839 -29840 -29841 -29842 -29843 -29844 -29845 -29846 -29847 -29848 -29849 -29850 -29851 -29852 -29853 -29854 -29855 -29856 -29857 -29858 -29859 -29860 -29861 -29862 -29863 -29864 -29865 -29866 -29867 -29868 -29869 -29870 -29871 -29872 -29873 -29874 -29875 -29876 -29877 -29878 -29879 -29880 -29881 -29882 -29883 -29884 -29885 -29886 -29887 -29888 -29889 -29890 -29891 -29892 -29893 -29894 -29895 -29896 -29897 -29898 -29899 -29900 -29901 -29902 -29903 -29904 -29905 -29906 -29907 -29908 -29909 -29910 -29911 -29912 -29913 -29914 -29915 -29916 -29917 -29918 -29919 -29920 -29921 -29922 -29923 -29924 -29925 -29926 -29927 -29928 -29929 -29930 -29931 -29932 -29933 -29934 -29935 -29936 -29937 -29938 -29939 -29940 -29941 -29942 -29943 -29944 -29945 -29946 -29947 -29948 -29949 -29950 -29951 -29952 -29953 -29954 -29955 -29956 -29957 -29958 -29959 -29960 -29961 -29962 -29963 -29964 -29965 -29966 -29967 -29968 -29969 -29970 -29971 -29972 -29973 -29974 -29975 -29976 -29977 -29978 -29979 -29980 -29981 -29982 -29983 -29984 -29985 -29986 -29987 -29988 -29989 -29990 -29991 -29992 -29993 -29994 -29995 -29996 -29997 -29998 -29999 -30000 -30001 -30002 -30003 -30004 -30005 -30006 -30007 -30008 -30009 -30010 -30011 -30012 -30013 -30014 -30015 -30016 -30017 -30018 -30019 -30020 -30021 -30022 -30023 -30024 -30025 -30026 -30027 -30028 -30029 -30030 -30031 -30032 -30033 -30034 -30035 -30036 -30037 -30038 -30039 -30040 -30041 -30042 -30043 -30044 -30045 -30046 -30047 -30048 -30049 -30050 -30051 -30052 -30053 -30054 -30055 -30056 -30057 -30058 -30059 -30060 -30061 -30062 -30063 -30064 -30065 -30066 -30067 -30068 -30069 -30070 -30071 -30072 -30073 -30074 -30075 -30076 -30077 -30078 -30079 -30080 -30081 -30082 -30083 -30084 -30085 -30086 -30087 -30088 -30089 -30090 -30091 -30092 -30093 -30094 -30095 -30096 -30097 -30098 -30099 -30100 -30101 -30102 -30103 -30104 -30105 -30106 -30107 -30108 -30109 -30110 -30111 -30112 -30113 -30114 -30115 -30116 -30117 -30118 -30119 -30120 -30121 -30122 -30123 -30124 -30125 -30126 -30127 -30128 -30129 -30130 -30131 -30132 -30133 -30134 -30135 -30136 -30137 -30138 -30139 -30140 -30141 -30142 -30143 -30144 -30145 -30146 -30147 -30148 -30149 -30150 -30151 -30152 -30153 -30154 -30155 -30156 -30157 -30158 -30159 -30160 -30161 -30162 -30163 -30164 -30165 -30166 -30167 -30168 -30169 -30170 -30171 -30172 -30173 -30174 -30175 -30176 -30177 -30178 -30179 -30180 -30181 -30182 -30183 -30184 -30185 -30186 -30187 -30188 -30189 -30190 -30191 -30192 -30193 -30194 -30195 -30196 -30197 -30198 -30199 -30200 -30201 -30202 -30203 -30204 -30205 -30206 -30207 -30208 -30209 -30210 -30211 -30212 -30213 -30214 -30215 -30216 -30217 -30218 -30219 -30220 -30221 -30222 -30223 -30224 -30225 -30226 -30227 -30228 -30229 -30230 -30231 -30232 -30233 -30234 -30235 -30236 -30237 -30238 -30239 -30240 -30241 -30242 -30243 -30244 -30245 -30246 -30247 -30248 -30249 -30250 -30251 -30252 -30253 -30254 -30255 -30256 -30257 -30258 -30259 -30260 -30261 -30262 -30263 -30264 -30265 -30266 -30267 -30268 -30269 -30270 -30271 -30272 -30273 -30274 -30275 -30276 -30277 -30278 -30279 -30280 -30281 -30282 -30283 -30284 -30285 -30286 -30287 -30288 -30289 -30290 -30291 -30292 -30293 -30294 -30295 -30296 -30297 -30298 -30299 -30300 -30301 -30302 -30303 -30304 -30305 -30306 -30307 -30308 -30309 -30310 -30311 -30312 -30313 -30314 -30315 -30316 -30317 -30318 -30319 -30320 -30321 -30322 -30323 -30324 -30325 -30326 -30327 -30328 -30329 -30330 -30331 -30332 -30333 -30334 -30335 -30336 -30337 -30338 -30339 -30340 -30341 -30342 -30343 -30344 -30345 -30346 -30347 -30348 -30349 -30350 -30351 -30352 -30353 -30354 -30355 -30356 -30357 -30358 -30359 -30360 -30361 -30362 -30363 -30364 -30365 -30366 -30367 -30368 -30369 -30370 -30371 -30372 -30373 -30374 -30375 -30376 -30377 -30378 -30379 -30380 -30381 -30382 -30383 -30384 -30385 -30386 -30387 -30388 -30389 -30390 -30391 -30392 -30393 -30394 -30395 -30396 -30397 -30398 -30399 -30400 -30401 -30402 -30403 -30404 -30405 -30406 -30407 -30408 -30409 -30410 -30411 -30412 -30413 -30414 -30415 -30416 -30417 -30418 -30419 -30420 -30421 -30422 -30423 -30424 -30425 -30426 -30427 -30428 -30429 -30430 -30431 -30432 -30433 -30434 -30435 -30436 -30437 -30438 -30439 -30440 -30441 -30442 -30443 -30444 -30445 -30446 -30447 -30448 -30449 -30450 -30451 -30452 -30453 -30454 -30455 -30456 -30457 -30458 -30459 -30460 -30461 -30462 -30463 -30464 -30465 -30466 -30467 -30468 -30469 -30470 -30471 -30472 -30473 -30474 -30475 -30476 -30477 -30478 -30479 -30480 -30481 -30482 -30483 -30484 -30485 -30486 -30487 -30488 -30489 -30490 -30491 -30492 -30493 -30494 -30495 -30496 -30497 -30498 -30499 -30500 -30501 -30502 -30503 -30504 -30505 -30506 -30507 -30508 -30509 -30510 -30511 -30512 -30513 -30514 -30515 -30516 -30517 -30518 -30519 -30520 -30521 -30522 -30523 -30524 -30525 -30526 -30527 -30528 -30529 -30530 -30531 -30532 -30533 -30534 -30535 -30536 -30537 -30538 -30539 -30540 -30541 -30542 -30543 -30544 -30545 -30546 -30547 -30548 -30549 -30550 -30551 -30552 -30553 -30554 -30555 -30556 -30557 -30558 -30559 -30560 -30561 -30562 -30563 -30564 -30565 -30566 -30567 -30568 -30569 -30570 -30571 -30572 -30573 -30574 -30575 -30576 -30577 -30578 -30579 -30580 -30581 -30582 -30583 -30584 -30585 -30586 -30587 -30588 -30589 -30590 -30591 -30592 -30593 -30594 -30595 -30596 -30597 -30598 -30599 -30600 -30601 -30602 -30603 -30604 -30605 -30606 -30607 -30608 -30609 -30610 -30611 -30612 -30613 -30614 -30615 -30616 -30617 -30618 -30619 -30620 -30621 -30622 -30623 -30624 -30625 -30626 -30627 -30628 -30629 -30630 -30631 -30632 -30633 -30634 -30635 -30636 -30637 -30638 -30639 -30640 -30641 -30642 -30643 -30644 -30645 -30646 -30647 -30648 -30649 -30650 -30651 -30652 -30653 -30654 -30655 -30656 -30657 -30658 -30659 -30660 -30661 -30662 -30663 -30664 -30665 -30666 -30667 -30668 -30669 -30670 -30671 -30672 -30673 -30674 -30675 -30676 -30677 -30678 -30679 -30680 -30681 -30682 -30683 -30684 -30685 -30686 -30687 -30688 -30689 -30690 -30691 -30692 -30693 -30694 -30695 -30696 -30697 -30698 -30699 -30700 -30701 -30702 -30703 -30704 -30705 -30706 -30707 -30708 -30709 -30710 -30711 -30712 -30713 -30714 -30715 -30716 -30717 -30718 -30719 -30720 -30721 -30722 -30723 -30724 -30725 -30726 -30727 -30728 -30729 -30730 -30731 -30732 -30733 -30734 -30735 -30736 -30737 -30738 -30739 -30740 -30741 -30742 -30743 -30744 -30745 -30746 -30747 -30748 -30749 -30750 -30751 -30752 -30753 -30754 -30755 -30756 -30757 -30758 -30759 -30760 -30761 -30762 -30763 -30764 -30765 -30766 -30767 -30768 -30769 -30770 -30771 -30772 -30773 -30774 -30775 -30776 -30777 -30778 -30779 -30780 -30781 -30782 -30783 -30784 -30785 -30786 -30787 -30788 -30789 -30790 -30791 -30792 -30793 -30794 -30795 -30796 -30797 -30798 -30799 -30800 -30801 -30802 -30803 -30804 -30805 -30806 -30807 -30808 -30809 -30810 -30811 -30812 -30813 -30814 -30815 -30816 -30817 -30818 -30819 -30820 -30821 -30822 -30823 -30824 -30825 -30826 -30827 -30828 -30829 -30830 -30831 -30832 -30833 -30834 -30835 -30836 -30837 -30838 -30839 -30840 -30841 -30842 -30843 -30844 -30845 -30846 -30847 -30848 -30849 -30850 -30851 -30852 -30853 -30854 -30855 -30856 -30857 -30858 -30859 -30860 -30861 -30862 -30863 -30864 -30865 -30866 -30867 -30868 -30869 -30870 -30871 -30872 -30873 -30874 -30875 -30876 -30877 -30878 -30879 -30880 -30881 -30882 -30883 -30884 -30885 -30886 -30887 -30888 -30889 -30890 -30891 -30892 -30893 -30894 -30895 -30896 -30897 -30898 -30899 -30900 -30901 -30902 -30903 -30904 -30905 -30906 -30907 -30908 -30909 -30910 -30911 -30912 -30913 -30914 -30915 -30916 -30917 -30918 -30919 -30920 -30921 -30922 -30923 -30924 -30925 -30926 -30927 -30928 -30929 -30930 -30931 -30932 -30933 -30934 -30935 -30936 -30937 -30938 -30939 -30940 -30941 -30942 -30943 -30944 -30945 -30946 -30947 -30948 -30949 -30950 -30951 -30952 -30953 -30954 -30955 -30956 -30957 -30958 -30959 -30960 -30961 -30962 -30963 -30964 -30965 -30966 -30967 -30968 -30969 -30970 -30971 -30972 -30973 -30974 -30975 -30976 -30977 -30978 -30979 -30980 -30981 -30982 -30983 -30984 -30985 -30986 -30987 -30988 -30989 -30990 -30991 -30992 -30993 -30994 -30995 -30996 -30997 -30998 -30999 -31000 -31001 -31002 -31003 -31004 -31005 -31006 -31007 -31008 -31009 -31010 -31011 -31012 -31013 -31014 -31015 -31016 -31017 -31018 -31019 -31020 -31021 -31022 -31023 -31024 -31025 -31026 -31027 -31028 -31029 -31030 -31031 -31032 -31033 -31034 -31035 -31036 -31037 -31038 -31039 -31040 -31041 -31042 -31043 -31044 -31045 -31046 -31047 -31048 -31049 -31050 -31051 -31052 -31053 -31054 -31055 -31056 -31057 -31058 -31059 -31060 -31061 -31062 -31063 -31064 -31065 -31066 -31067 -31068 -31069 -31070 -31071 -31072 -31073 -31074 -31075 -31076 -31077 -31078 -31079 -31080 -31081 -31082 -31083 -31084 -31085 -31086 -31087 -31088 -31089 -31090 -31091 -31092 -31093 -31094 -31095 -31096 -31097 -31098 -31099 -31100 -31101 -31102 -31103 -31104 -31105 -31106 -31107 -31108 -31109 -31110 -31111 -31112 -31113 -31114 -31115 -31116 -31117 -31118 -31119 -31120 -31121 -31122 -31123 -31124 -31125 -31126 -31127 -31128 -31129 -31130 -31131 -31132 -31133 -31134 -31135 -31136 -31137 -31138 -31139 -31140 -31141 -31142 -31143 -31144 -31145 -31146 -31147 -31148 -31149 -31150 -31151 -31152 -31153 -31154 -31155 -31156 -31157 -31158 -31159 -31160 -31161 -31162 -31163 -31164 -31165 -31166 -31167 -31168 -31169 -31170 -31171 -31172 -31173 -31174 -31175 -31176 -31177 -31178 -31179 -31180 -31181 -31182 -31183 -31184 -31185 -31186 -31187 -31188 -31189 -31190 -31191 -31192 -31193 -31194 -31195 -31196 -31197 -31198 -31199 -31200 -31201 -31202 -31203 -31204 -31205 -31206 -31207 -31208 -31209 -31210 -31211 -31212 -31213 -31214 -31215 -31216 -31217 -31218 -31219 -31220 -31221 -31222 -31223 -31224 -31225 -31226 -31227 -31228 -31229 -31230 -31231 -31232 -31233 -31234 -31235 -31236 -31237 -31238 -31239 -31240 -31241 -31242 -31243 -31244 -31245 -31246 -31247 -31248 -31249 -31250 -31251 -31252 -31253 -31254 -31255 -31256 -31257 -31258 -31259 -31260 -31261 -31262 -31263 -31264 -31265 -31266 -31267 -31268 -31269 -31270 -31271 -31272 -31273 -31274 -31275 -31276 -31277 -31278 -31279 -31280 -31281 -31282 -31283 -31284 -31285 -31286 -31287 -31288 -31289 -31290 -31291 -31292 -31293 -31294 -31295 -31296 -31297 -31298 -31299 -31300 -31301 -31302 -31303 -31304 -31305 -31306 -31307 -31308 -31309 -31310 -31311 -31312 -31313 -31314 -31315 -31316 -31317 -31318 -31319 -31320 -31321 -31322 -31323 -31324 -31325 -31326 -31327 -31328 -31329 -31330 -31331 -31332 -31333 -31334 -31335 -31336 -31337 -31338 -31339 -31340 -31341 -31342 -31343 -31344 -31345 -31346 -31347 -31348 -31349 -31350 -31351 -31352 -31353 -31354 -31355 -31356 -31357 -31358 -31359 -31360 -31361 -31362 -31363 -31364 -31365 -31366 -31367 -31368 -31369 -31370 -31371 -31372 -31373 -31374 -31375 -31376 -31377 -31378 -31379 -31380 -31381 -31382 -31383 -31384 -31385 -31386 -31387 -31388 -31389 -31390 -31391 -31392 -31393 -31394 -31395 -31396 -31397 -31398 -31399 -31400 -31401 -31402 -31403 -31404 -31405 -31406 -31407 -31408 -31409 -31410 -31411 -31412 -31413 -31414 -31415 -31416 -31417 -31418 -31419 -31420 -31421 -31422 -31423 -31424 -31425 -31426 -31427 -31428 -31429 -31430 -31431 -31432 -31433 -31434 -31435 -31436 -31437 -31438 -31439 -31440 -31441 -31442 -31443 -31444 -31445 -31446 -31447 -31448 -31449 -31450 -31451 -31452 -31453 -31454 -31455 -31456 -31457 -31458 -31459 -31460 -31461 -31462 -31463 -31464 -31465 -31466 -31467 -31468 -31469 -31470 -31471 -31472 -31473 -31474 -31475 -31476 -31477 -31478 -31479 -31480 -31481 -31482 -31483 -31484 -31485 -31486 -31487 -31488 -31489 -31490 -31491 -31492 -31493 -31494 -31495 -31496 -31497 -31498 -31499 -31500 -31501 -31502 -31503 -31504 -31505 -31506 -31507 -31508 -31509 -31510 -31511 -31512 -31513 -31514 -31515 -31516 -31517 -31518 -31519 -31520 -31521 -31522 -31523 -31524 -31525 -31526 -31527 -31528 -31529 -31530 -31531 -31532 -31533 -31534 -31535 -31536 -31537 -31538 -31539 -31540 -31541 -31542 -31543 -31544 -31545 -31546 -31547 -31548 -31549 -31550 -31551 -31552 -31553 -31554 -31555 -31556 -31557 -31558 -31559 -31560 -31561 -31562 -31563 -31564 -31565 -31566 -31567 -31568 -31569 -31570 -31571 -31572 -31573 -31574 -31575 -31576 -31577 -31578 -31579 -31580 -31581 -31582 -31583 -31584 -31585 -31586 -31587 -31588 -31589 -31590 -31591 -31592 -31593 -31594 -31595 -31596 -31597 -31598 -31599 -31600 -31601 -31602 -31603 -31604 -31605 -31606 -31607 -31608 -31609 -31610 -31611 -31612 -31613 -31614 -31615 -31616 -31617 -31618 -31619 -31620 -31621 -31622 -31623 -31624 -31625 -31626 -31627 -31628 -31629 -31630 -31631 -31632 -31633 -31634 -31635 -31636 -31637 -31638 -31639 -31640 -31641 -31642 -31643 -31644 -31645 -31646 -31647 -31648 -31649 -31650 -31651 -31652 -31653 -31654 -31655 -31656 -31657 -31658 -31659 -31660 -31661 -31662 -31663 -31664 -31665 -31666 -31667 -31668 -31669 -31670 -31671 -31672 -31673 -31674 -31675 -31676 -31677 -31678 -31679 -31680 -31681 -31682 -31683 -31684 -31685 -31686 -31687 -31688 -31689 -31690 -31691 -31692 -31693 -31694 -31695 -31696 -31697 -31698 -31699 -31700 -31701 -31702 -31703 -31704 -31705 -31706 -31707 -31708 -31709 -31710 -31711 -31712 -31713 -31714 -31715 -31716 -31717 -31718 -31719 -31720 -31721 -31722 -31723 -31724 -31725 -31726 -31727 -31728 -31729 -31730 -31731 -31732 -31733 -31734 -31735 -31736 -31737 -31738 -31739 -31740 -31741 -31742 -31743 -31744 -31745 -31746 -31747 -31748 -31749 -31750 -31751 -31752 -31753 -31754 -31755 -31756 -31757 -31758 -31759 -31760 -31761 -31762 -31763 -31764 -31765 -31766 -31767 -31768 -31769 -31770 -31771 -31772 -31773 -31774 -31775 -31776 -31777 -31778 -31779 -31780 -31781 -31782 -31783 -31784 -31785 -31786 -31787 -31788 -31789 -31790 -31791 -31792 -31793 -31794 -31795 -31796 -31797 -31798 -31799 -31800 -31801 -31802 -31803 -31804 -31805 -31806 -31807 -31808 -31809 -31810 -31811 -31812 -31813 -31814 -31815 -31816 -31817 -31818 -31819 -31820 -31821 -31822 -31823 -31824 -31825 -31826 -31827 -31828 -31829 -31830 -31831 -31832 -31833 -31834 -31835 -31836 -31837 -31838 -31839 -31840 -31841 -31842 -31843 -31844 -31845 -31846 -31847 -31848 -31849 -31850 -31851 -31852 -31853 -31854 -31855 -31856 -31857 -31858 -31859 -31860 -31861 -31862 -31863 -31864 -31865 -31866 -31867 -31868 -31869 -31870 -31871 -31872 -31873 -31874 -31875 -31876 -31877 -31878 -31879 -31880 -31881 -31882 -31883 -31884 -31885 -31886 -31887 -31888 -31889 -31890 -31891 -31892 -31893 -31894 -31895 -31896 -31897 -31898 -31899 -31900 -31901 -31902 -31903 -31904 -31905 -31906 -31907 -31908 -31909 -31910 -31911 -31912 -31913 -31914 -31915 -31916 -31917 -31918 -31919 -31920 -31921 -31922 -31923 -31924 -31925 -31926 -31927 -31928 -31929 -31930 -31931 -31932 -31933 -31934 -31935 -31936 -31937 -31938 -31939 -31940 -31941 -31942 -31943 -31944 -31945 -31946 -31947 -31948 -31949 -31950 -31951 -31952 -31953 -31954 -31955 -31956 -31957 -31958 -31959 -31960 -31961 -31962 -31963 -31964 -31965 -31966 -31967 -31968 -31969 -31970 -31971 -31972 -31973 -31974 -31975 -31976 -31977 -31978 -31979 -31980 -31981 -31982 -31983 -31984 -31985 -31986 -31987 -31988 -31989 -31990 -31991 -31992 -31993 -31994 -31995 -31996 -31997 -31998 -31999 -32000 -32001 -32002 -32003 -32004 -32005 -32006 -32007 -32008 -32009 -32010 -32011 -32012 -32013 -32014 -32015 -32016 -32017 -32018 -32019 -32020 -32021 -32022 -32023 -32024 -32025 -32026 -32027 -32028 -32029 -32030 -32031 -32032 -32033 -32034 -32035 -32036 -32037 -32038 -32039 -32040 -32041 -32042 -32043 -32044 -32045 -32046 -32047 -32048 -32049 -32050 -32051 -32052 -32053 -32054 -32055 -32056 -32057 -32058 -32059 -32060 -32061 -32062 -32063 -32064 -32065 -32066 -32067 -32068 -32069 -32070 -32071 -32072 -32073 -32074 -32075 -32076 -32077 -32078 -32079 -32080 -32081 -32082 -32083 -32084 -32085 -32086 -32087 -32088 -32089 -32090 -32091 -32092 -32093 -32094 -32095 -32096 -32097 -32098 -32099 -32100 -32101 -32102 -32103 -32104 -32105 -32106 -32107 -32108 -32109 -32110 -32111 -32112 -32113 -32114 -32115 -32116 -32117 -32118 -32119 -32120 -32121 -32122 -32123 -32124 -32125 -32126 -32127 -32128 -32129 -32130 -32131 -32132 -32133 -32134 -32135 -32136 -32137 -32138 -32139 -32140 -32141 -32142 -32143 -32144 -32145 -32146 -32147 -32148 -32149 -32150 -32151 -32152 -32153 -32154 -32155 -32156 -32157 -32158 -32159 -32160 -32161 -32162 -32163 -32164 -32165 -32166 -32167 -32168 -32169 -32170 -32171 -32172 -32173 -32174 -32175 -32176 -32177 -32178 -32179 -32180 -32181 -32182 -32183 -32184 -32185 -32186 -32187 -32188 -32189 -32190 -32191 -32192 -32193 -32194 -32195 -32196 -32197 -32198 -32199 -32200 -32201 -32202 -32203 -32204 -32205 -32206 -32207 -32208 -32209 -32210 -32211 -32212 -32213 -32214 -32215 -32216 -32217 -32218 -32219 -32220 -32221 -32222 -32223 -32224 -32225 -32226 -32227 -32228 -32229 -32230 -32231 -32232 -32233 -32234 -32235 -32236 -32237 -32238 -32239 -32240 -32241 -32242 -32243 -32244 -32245 -32246 -32247 -32248 -32249 -32250 -32251 -32252 -32253 -32254 -32255 -32256 -32257 -32258 -32259 -32260 -32261 -32262 -32263 -32264 -32265 -32266 -32267 -32268 -32269 -32270 -32271 -32272 -32273 -32274 -32275 -32276 -32277 -32278 -32279 -32280 -32281 -32282 -32283 -32284 -32285 -32286 -32287 -32288 -32289 -32290 -32291 -32292 -32293 -32294 -32295 -32296 -32297 -32298 -32299 -32300 -32301 -32302 -32303 -32304 -32305 -32306 -32307 -32308 -32309 -32310 -32311 -32312 -32313 -32314 -32315 -32316 -32317 -32318 -32319 -32320 -32321 -32322 -32323 -32324 -32325 -32326 -32327 -32328 -32329 -32330 -32331 -32332 -32333 -32334 -32335 -32336 -32337 -32338 -32339 -32340 -32341 -32342 -32343 -32344 -32345 -32346 -32347 -32348 -32349 -32350 -32351 -32352 -32353 -32354 -32355 -32356 -32357 -32358 -32359 -32360 -32361 -32362 -32363 -32364 -32365 -32366 -32367 -32368 -32369 -32370 -32371 -32372 -32373 -32374 -32375 -32376 -32377 -32378 -32379 -32380 -32381 -32382 -32383 -32384 -32385 -32386 -32387 -32388 -32389 -32390 -32391 -32392 -32393 -32394 -32395 -32396 -32397 -32398 -32399 -32400 -32401 -32402 -32403 -32404 -32405 -32406 -32407 -32408 -32409 -32410 -32411 -32412 -32413 -32414 -32415 -32416 -32417 -32418 -32419 -32420 -32421 -32422 -32423 -32424 -32425 -32426 -32427 -32428 -32429 -32430 -32431 -32432 -32433 -32434 -32435 -32436 -32437 -32438 -32439 -32440 -32441 -32442 -32443 -32444 -32445 -32446 -32447 -32448 -32449 -32450 -32451 -32452 -32453 -32454 -32455 -32456 -32457 -32458 -32459 -32460 -32461 -32462 -32463 -32464 -32465 -32466 -32467 -32468 -32469 -32470 -32471 -32472 -32473 -32474 -32475 -32476 -32477 -32478 -32479 -32480 -32481 -32482 -32483 -32484 -32485 -32486 -32487 -32488 -32489 -32490 -32491 -32492 -32493 -32494 -32495 -32496 -32497 -32498 -32499 -32500 -32501 -32502 -32503 -32504 -32505 -32506 -32507 -32508 -32509 -32510 -32511 -32512 -32513 -32514 -32515 -32516 -32517 -32518 -32519 -32520 -32521 -32522 -32523 -32524 -32525 -32526 -32527 -32528 -32529 -32530 -32531 -32532 -32533 -32534 -32535 -32536 -32537 -32538 -32539 -32540 -32541 -32542 -32543 -32544 -32545 -32546 -32547 -32548 -32549 -32550 -32551 -32552 -32553 -32554 -32555 -32556 -32557 -32558 -32559 -32560 -32561 -32562 -32563 -32564 -32565 -32566 -32567 -32568 -32569 -32570 -32571 -32572 -32573 -32574 -32575 -32576 -32577 -32578 -32579 -32580 -32581 -32582 -32583 -32584 -32585 -32586 -32587 -32588 -32589 -32590 -32591 -32592 -32593 -32594 -32595 -32596 -32597 -32598 -32599 -32600 -32601 -32602 -32603 -32604 -32605 -32606 -32607 -32608 -32609 -32610 -32611 -32612 -32613 -32614 -32615 -32616 -32617 -32618 -32619 -32620 -32621 -32622 -32623 -32624 -32625 -32626 -32627 -32628 -32629 -32630 -32631 -32632 -32633 -32634 -32635 -32636 -32637 -32638 -32639 -32640 -32641 -32642 -32643 -32644 -32645 -32646 -32647 -32648 -32649 -32650 -32651 -32652 -32653 -32654 -32655 -32656 -32657 -32658 -32659 -32660 -32661 -32662 -32663 -32664 -32665 -32666 -32667 -32668 -32669 -32670 -32671 -32672 -32673 -32674 -32675 -32676 -32677 -32678 -32679 -32680 -32681 -32682 -32683 -32684 -32685 -32686 -32687 -32688 -32689 -32690 -32691 -32692 -32693 -32694 -32695 -32696 -32697 -32698 -32699 -32700 -32701 -32702 -32703 -32704 -32705 -32706 -32707 -32708 -32709 -32710 -32711 -32712 -32713 -32714 -32715 -32716 -32717 -32718 -32719 -32720 -32721 -32722 -32723 -32724 -32725 -32726 -32727 -32728 -32729 -32730 -32731 -32732 -32733 -32734 -32735 -32736 -32737 -32738 -32739 -32740 -32741 -32742 -32743 -32744 -32745 -32746 -32747 -32748 -32749 -32750 -32751 -32752 -32753 -32754 -32755 -32756 -32757 -32758 -32759 -32760 -32761 -32762 -32763 -32764 -32765 -32766 -32767 -32768 -32769 -32770 -32771 -32772 -32773 -32774 -32775 -32776 -32777 -32778 -32779 -32780 -32781 -32782 -32783 -32784 -32785 -32786 -32787 -32788 -32789 -32790 -32791 -32792 -32793 -32794 -32795 -32796 -32797 -32798 -32799 -32800 -32801 -32802 -32803 -32804 -32805 -32806 -32807 -32808 -32809 -32810 -32811 -32812 -32813 -32814 -32815 -32816 -32817 -32818 -32819 -32820 -32821 -32822 -32823 -32824 -32825 -32826 -32827 -32828 -32829 -32830 -32831 -32832 -32833 -32834 -32835 -32836 -32837 -32838 -32839 -32840 -32841 -32842 -32843 -32844 -32845 -32846 -32847 -32848 -32849 -32850 -32851 -32852 -32853 -32854 -32855 -32856 -32857 -32858 -32859 -32860 -32861 -32862 -32863 -32864 -32865 -32866 -32867 -32868 -32869 -32870 -32871 -32872 -32873 -32874 -32875 -32876 -32877 -32878 -32879 -32880 -32881 -32882 -32883 -32884 -32885 -32886 -32887 -32888 -32889 -32890 -32891 -32892 -32893 -32894 -32895 -32896 -32897 -32898 -32899 -32900 -32901 -32902 -32903 -32904 -32905 -32906 -32907 -32908 -32909 -32910 -32911 -32912 -32913 -32914 -32915 -32916 -32917 -32918 -32919 -32920 -32921 -32922 -32923 -32924 -32925 -32926 -32927 -32928 -32929 -32930 -32931 -32932 -32933 -32934 -32935 -32936 -32937 -32938 -32939 -32940 -32941 -32942 -32943 -32944 -32945 -32946 -32947 -32948 -32949 -32950 -32951 -32952 -32953 -32954 -32955 -32956 -32957 -32958 -32959 -32960 -32961 -32962 -32963 -32964 -32965 -32966 -32967 -32968 -32969 -32970 -32971 -32972 -32973 -32974 -32975 -32976 -32977 -32978 -32979 -32980 -32981 -32982 -32983 -32984 -32985 -32986 -32987 -32988 -32989 -32990 -32991 -32992 -32993 -32994 -32995 -32996 -32997 -32998 -32999 -33000 -33001 -33002 -33003 -33004 -33005 -33006 -33007 -33008 -33009 -33010 -33011 -33012 -33013 -33014 -33015 -33016 -33017 -33018 -33019 -33020 -33021 -33022 -33023 -33024 -33025 -33026 -33027 -33028 -33029 -33030 -33031 -33032 -33033 -33034 -33035 -33036 -33037 -33038 -33039 -33040 -33041 -33042 -33043 -33044 -33045 -33046 -33047 -33048 -33049 -33050 -33051 -33052 -33053 -33054 -33055 -33056 -33057 -33058 -33059 -33060 -33061 -33062 -33063 -33064 -33065 -33066 -33067 -33068 -33069 -33070 -33071 -33072 -33073 -33074 -33075 -33076 -33077 -33078 -33079 -33080 -33081 -33082 -33083 -33084 -33085 -33086 -33087 -33088 -33089 -33090 -33091 -33092 -33093 -33094 -33095 -33096 -33097 -33098 -33099 -33100 -33101 -33102 -33103 -33104 -33105 -33106 -33107 -33108 -33109 -33110 -33111 -33112 -33113 -33114 -33115 -33116 -33117 -33118 -33119 -33120 -33121 -33122 -33123 -33124 -33125 -33126 -33127 -33128 -33129 -33130 -33131 -33132 -33133 -33134 -33135 -33136 -33137 -33138 -33139 -33140 -33141 -33142 -33143 -33144 -33145 -33146 -33147 -33148 -33149 -33150 -33151 -33152 -33153 -33154 -33155 -33156 -33157 -33158 -33159 -33160 -33161 -33162 -33163 -33164 -33165 -33166 -33167 -33168 -33169 -33170 -33171 -33172 -33173 -33174 -33175 -33176 -33177 -33178 -33179 -33180 -33181 -33182 -33183 -33184 -33185 -33186 -33187 -33188 -33189 -33190 -33191 -33192 -33193 -33194 -33195 -33196 -33197 -33198 -33199 -33200 -33201 -33202 -33203 -33204 -33205 -33206 -33207 -33208 -33209 -33210 -33211 -33212 -33213 -33214 -33215 -33216 -33217 -33218 -33219 -33220 -33221 -33222 -33223 -33224 -33225 -33226 -33227 -33228 -33229 -33230 -33231 -33232 -33233 -33234 -33235 -33236 -33237 -33238 -33239 -33240 -33241 -33242 -33243 -33244 -33245 -33246 -33247 -33248 -33249 -33250 -33251 -33252 -33253 -33254 -33255 -33256 -33257 -33258 -33259 -33260 -33261 -33262 -33263 -33264 -33265 -33266 -33267 -33268 -33269 -33270 -33271 -33272 -33273 -33274 -33275 -33276 -33277 -33278 -33279 -33280 -33281 -33282 -33283 -33284 -33285 -33286 -33287 -33288 -33289 -33290 -33291 -33292 -33293 -33294 -33295 -33296 -33297 -33298 -33299 -33300 -33301 -33302 -33303 -33304 -33305 -33306 -33307 -33308 -33309 -33310 -33311 -33312 -33313 -33314 -33315 -33316 -33317 -33318 -33319 -33320 -33321 -33322 -33323 -33324 -33325 -33326 -33327 -33328 -33329 -33330 -33331 -33332 -33333 -33334 -33335 -33336 -33337 -33338 -33339 -33340 -33341 -33342 -33343 -33344 -33345 -33346 -33347 -33348 -33349 -33350 -33351 -33352 -33353 -33354 -33355 -33356 -33357 -33358 -33359 -33360 -33361 -33362 -33363 -33364 -33365 -33366 -33367 -33368 -33369 -33370 -33371 -33372 -33373 -33374 -33375 -33376 -33377 -33378 -33379 -33380 -33381 -33382 -33383 -33384 -33385 -33386 -33387 -33388 -33389 -33390 -33391 -33392 -33393 -33394 -33395 -33396 -33397 -33398 -33399 -33400 -33401 -33402 -33403 -33404 -33405 -33406 -33407 -33408 -33409 -33410 -33411 -33412 -33413 -33414 -33415 -33416 -33417 -33418 -33419 -33420 -33421 -33422 -33423 -33424 -33425 -33426 -33427 -33428 -33429 -33430 -33431 -33432 -33433 -33434 -33435 -33436 -33437 -33438 -33439 -33440 -33441 -33442 -33443 -33444 -33445 -33446 -33447 -33448 -33449 -33450 -33451 -33452 -33453 -33454 -33455 -33456 -33457 -33458 -33459 -33460 -33461 -33462 -33463 -33464 -33465 -33466 -33467 -33468 -33469 -33470 -33471 -33472 -33473 -33474 -33475 -33476 -33477 -33478 -33479 -33480 -33481 -33482 -33483 -33484 -33485 -33486 -33487 -33488 -33489 -33490 -33491 -33492 -33493 -33494 -33495 -33496 -33497 -33498 -33499 -33500 -33501 -33502 -33503 -33504 -33505 -33506 -33507 -33508 -33509 -33510 -33511 -33512 -33513 -33514 -33515 -33516 -33517 -33518 -33519 -33520 -33521 -33522 -33523 -33524 -33525 -33526 -33527 -33528 -33529 -33530 -33531 -33532 -33533 -33534 -33535 -33536 -33537 -33538 -33539 -33540 -33541 -33542 -33543 -33544 -33545 -33546 -33547 -33548 -33549 -33550 -33551 -33552 -33553 -33554 -33555 -33556 -33557 -33558 -33559 -33560 -33561 -33562 -33563 -33564 -33565 -33566 -33567 -33568 -33569 -33570 -33571 -33572 -33573 -33574 -33575 -33576 -33577 -33578 -33579 -33580 -33581 -33582 -33583 -33584 -33585 -33586 -33587 -33588 -33589 -33590 -33591 -33592 -33593 -33594 -33595 -33596 -33597 -33598 -33599 -33600 -33601 -33602 -33603 -33604 -33605 -33606 -33607 -33608 -33609 -33610 -33611 -33612 -33613 -33614 -33615 -33616 -33617 -33618 -33619 -33620 -33621 -33622 -33623 -33624 -33625 -33626 -33627 -33628 -33629 -33630 -33631 -33632 -33633 -33634 -33635 -33636 -33637 -33638 -33639 -33640 -33641 -33642 -33643 -33644 -33645 -33646 -33647 -33648 -33649 -33650 -33651 -33652 -33653 -33654 -33655 -33656 -33657 -33658 -33659 -33660 -33661 -33662 -33663 -33664 -33665 -33666 -33667 -33668 -33669 -33670 -33671 -33672 -33673 -33674 -33675 -33676 -33677 -33678 -33679 -33680 -33681 -33682 -33683 -33684 -33685 -33686 -33687 -33688 -33689 -33690 -33691 -33692 -33693 -33694 -33695 -33696 -33697 -33698 -33699 -33700 -33701 -33702 -33703 -33704 -33705 -33706 -33707 -33708 -33709 -33710 -33711 -33712 -33713 -33714 -33715 -33716 -33717 -33718 -33719 -33720 -33721 -33722 -33723 -33724 -33725 -33726 -33727 -33728 -33729 -33730 -33731 -33732 -33733 -33734 -33735 -33736 -33737 -33738 -33739 -33740 -33741 -33742 -33743 -33744 -33745 -33746 -33747 -33748 -33749 -33750 -33751 -33752 -33753 -33754 -33755 -33756 -33757 -33758 -33759 -33760 -33761 -33762 -33763 -33764 -33765 -33766 -33767 -33768 -33769 -33770 -33771 -33772 -33773 -33774 -33775 -33776 -33777 -33778 -33779 -33780 -33781 -33782 -33783 -33784 -33785 -33786 -33787 -33788 -33789 -33790 -33791 -33792 -33793 -33794 -33795 -33796 -33797 -33798 -33799 -33800 -33801 -33802 -33803 -33804 -33805 -33806 -33807 -33808 -33809 -33810 -33811 -33812 -33813 -33814 -33815 -33816 -33817 -33818 -33819 -33820 -33821 -33822 -33823 -33824 -33825 -33826 -33827 -33828 -33829 -33830 -33831 -33832 -33833 -33834 -33835 -33836 -33837 -33838 -33839 -33840 -33841 -33842 -33843 -33844 -33845 -33846 -33847 -33848 -33849 -33850 -33851 -33852 -33853 -33854 -33855 -33856 -33857 -33858 -33859 -33860 -33861 -33862 -33863 -33864 -33865 -33866 -33867 -33868 -33869 -33870 -33871 -33872 -33873 -33874 -33875 -33876 -33877 -33878 -33879 -33880 -33881 -33882 -33883 -33884 -33885 -33886 -33887 -33888 -33889 -33890 -33891 -33892 -33893 -33894 -33895 -33896 -33897 -33898 -33899 -33900 -33901 -33902 -33903 -33904 -33905 -33906 -33907 -33908 -33909 -33910 -33911 -33912 -33913 -33914 -33915 -33916 -33917 -33918 -33919 -33920 -33921 -33922 -33923 -33924 -33925 -33926 -33927 -33928 -33929 -33930 -33931 -33932 -33933 -33934 -33935 -33936 -33937 -33938 -33939 -33940 -33941 -33942 -33943 -33944 -33945 -33946 -33947 -33948 -33949 -33950 -33951 -33952 -33953 -33954 -33955 -33956 -33957 -33958 -33959 -33960 -33961 -33962 -33963 -33964 -33965 -33966 -33967 -33968 -33969 -33970 -33971 -33972 -33973 -33974 -33975 -33976 -33977 -33978 -33979 -33980 -33981 -33982 -33983 -33984 -33985 -33986 -33987 -33988 -33989 -33990 -33991 -33992 -33993 -33994 -33995 -33996 -33997 -33998 -33999 -34000 -34001 -34002 -34003 -34004 -34005 -34006 -34007 -34008 -34009 -34010 -34011 -34012 -34013 -34014 -34015 -34016 -34017 -34018 -34019 -34020 -34021 -34022 -34023 -34024 -34025 -34026 -34027 -34028 -34029 -34030 -34031 -34032 -34033 -34034 -34035 -34036 -34037 -34038 -34039 -34040 -34041 -34042 -34043 -34044 -34045 -34046 -34047 -34048 -34049 -34050 -34051 -34052 -34053 -34054 -34055 -34056 -34057 -34058 -34059 -34060 -34061 -34062 -34063 -34064 -34065 -34066 -34067 -34068 -34069 -34070 -34071 -34072 -34073 -34074 -34075 -34076 -34077 -34078 -34079 -34080 -34081 -34082 -34083 -34084 -34085 -34086 -34087 -34088 -34089 -34090 -34091 -34092 -34093 -34094 -34095 -34096 -34097 -34098 -34099 -34100 -34101 -34102 -34103 -34104 -34105 -34106 -34107 -34108 -34109 -34110 -34111 -34112 -34113 -34114 -34115 -34116 -34117 -34118 -34119 -34120 -34121 -34122 -34123 -34124 -34125 -34126 -34127 -34128 -34129 -34130 -34131 -34132 -34133 -34134 -34135 -34136 -34137 -34138 -34139 -34140 -34141 -34142 -34143 -34144 -34145 -34146 -34147 -34148 -34149 -34150 -34151 -34152 -34153 -34154 -34155 -34156 -34157 -34158 -34159 -34160 -34161 -34162 -34163 -34164 -34165 -34166 -34167 -34168 -34169 -34170 -34171 -34172 -34173 -34174 -34175 -34176 -34177 -34178 -34179 -34180 -34181 -34182 -34183 -34184 -34185 -34186 -34187 -34188 -34189 -34190 -34191 -34192 -34193 -34194 -34195 -34196 -34197 -34198 -34199 -34200 -34201 -34202 -34203 -34204 -34205 -34206 -34207 -34208 -34209 -34210 -34211 -34212 -34213 -34214 -34215 -34216 -34217 -34218 -34219 -34220 -34221 -34222 -34223 -34224 -34225 -34226 -34227 -34228 -34229 -34230 -34231 -34232 -34233 -34234 -34235 -34236 -34237 -34238 -34239 -34240 -34241 -34242 -34243 -34244 -34245 -34246 -34247 -34248 -34249 -34250 -34251 -34252 -34253 -34254 -34255 -34256 -34257 -34258 -34259 -34260 -34261 -34262 -34263 -34264 -34265 -34266 -34267 -34268 -34269 -34270 -34271 -34272 -34273 -34274 -34275 -34276 -34277 -34278 -34279 -34280 -34281 -34282 -34283 -34284 -34285 -34286 -34287 -34288 -34289 -34290 -34291 -34292 -34293 -34294 -34295 -34296 -34297 -34298 -34299 -34300 -34301 -34302 -34303 -34304 -34305 -34306 -34307 -34308 -34309 -34310 -34311 -34312 -34313 -34314 -34315 -34316 -34317 -34318 -34319 -34320 -34321 -34322 -34323 -34324 -34325 -34326 -34327 -34328 -34329 -34330 -34331 -34332 -34333 -34334 -34335 -34336 -34337 -34338 -34339 -34340 -34341 -34342 -34343 -34344 -34345 -34346 -34347 -34348 -34349 -34350 -34351 -34352 -34353 -34354 -34355 -34356 -34357 -34358 -34359 -34360 -34361 -34362 -34363 -34364 -34365 -34366 -34367 -34368 -34369 -34370 -34371 -34372 -34373 -34374 -34375 -34376 -34377 -34378 -34379 -34380 -34381 -34382 -34383 -34384 -34385 -34386 -34387 -34388 -34389 -34390 -34391 -34392 -34393 -34394 -34395 -34396 -34397 -34398 -34399 -34400 -34401 -34402 -34403 -34404 -34405 -34406 -34407 -34408 -34409 -34410 -34411 -34412 -34413 -34414 -34415 -34416 -34417 -34418 -34419 -34420 -34421 -34422 -34423 -34424 -34425 -34426 -34427 -34428 -34429 -34430 -34431 -34432 -34433 -34434 -34435 -34436 -34437 -34438 -34439 -34440 -34441 -34442 -34443 -34444 -34445 -34446 -34447 -34448 -34449 -34450 -34451 -34452 -34453 -34454 -34455 -34456 -34457 -34458 -34459 -34460 -34461 -34462 -34463 -34464 -34465 -34466 -34467 -34468 -34469 -34470 -34471 -34472 -34473 -34474 -34475 -34476 -34477 -34478 -34479 -34480 -34481 -34482 -34483 -34484 -34485 -34486 -34487 -34488 -34489 -34490 -34491 -34492 -34493 -34494 -34495 -34496 -34497 -34498 -34499 -34500 -34501 -34502 -34503 -34504 -34505 -34506 -34507 -34508 -34509 -34510 -34511 -34512 -34513 -34514 -34515 -34516 -34517 -34518 -34519 -34520 -34521 -34522 -34523 -34524 -34525 -34526 -34527 -34528 -34529 -34530 -34531 -34532 -34533 -34534 -34535 -34536 -34537 -34538 -34539 -34540 -34541 -34542 -34543 -34544 -34545 -34546 -34547 -34548 -34549 -34550 -34551 -34552 -34553 -34554 -34555 -34556 -34557 -34558 -34559 -34560 -34561 -34562 -34563 -34564 -34565 -34566 -34567 -34568 -34569 -34570 -34571 -34572 -34573 -34574 -34575 -34576 -34577 -34578 -34579 -34580 -34581 -34582 -34583 -34584 -34585 -34586 -34587 -34588 -34589 -34590 -34591 -34592 -34593 -34594 -34595 -34596 -34597 -34598 -34599 -34600 -34601 -34602 -34603 -34604 -34605 -34606 -34607 -34608 -34609 -34610 -34611 -34612 -34613 -34614 -34615 -34616 -34617 -34618 -34619 -34620 -34621 -34622 -34623 -34624 -34625 -34626 -34627 -34628 -34629 -34630 -34631 -34632 -34633 -34634 -34635 -34636 -34637 -34638 -34639 -34640 -34641 -34642 -34643 -34644 -34645 -34646 -34647 -34648 -34649 -34650 -34651 -34652 -34653 -34654 -34655 -34656 -34657 -34658 -34659 -34660 -34661 -34662 -34663 -34664 -34665 -34666 -34667 -34668 -34669 -34670 -34671 -34672 -34673 -34674 -34675 -34676 -34677 -34678 -34679 -34680 -34681 -34682 -34683 -34684 -34685 -34686 -34687 -34688 -34689 -34690 -34691 -34692 -34693 -34694 -34695 -34696 -34697 -34698 -34699 -34700 -34701 -34702 -34703 -34704 -34705 -34706 -34707 -34708 -34709 -34710 -34711 -34712 -34713 -34714 -34715 -34716 -34717 -34718 -34719 -34720 -34721 -34722 -34723 -34724 -34725 -34726 -34727 -34728 -34729 -34730 -34731 -34732 -34733 -34734 -34735 -34736 -34737 -34738 -34739 -34740 -34741 -34742 -34743 -34744 -34745 -34746 -34747 -34748 -34749 -34750 -34751 -34752 -34753 -34754 -34755 -34756 -34757 -34758 -34759 -34760 -34761 -34762 -34763 -34764 -34765 -34766 -34767 -34768 -34769 -34770 -34771 -34772 -34773 -34774 -34775 -34776 -34777 -34778 -34779 -34780 -34781 -34782 -34783 -34784 -34785 -34786 -34787 -34788 -34789 -34790 -34791 -34792 -34793 -34794 -34795 -34796 -34797 -34798 -34799 -34800 -34801 -34802 -34803 -34804 -34805 -34806 -34807 -34808 -34809 -34810 -34811 -34812 -34813 -34814 -34815 -34816 -34817 -34818 -34819 -34820 -34821 -34822 -34823 -34824 -34825 -34826 -34827 -34828 -34829 -34830 -34831 -34832 -34833 -34834 -34835 -34836 -34837 -34838 -34839 -34840 -34841 -34842 -34843 -34844 -34845 -34846 -34847 -34848 -34849 -34850 -34851 -34852 -34853 -34854 -34855 -34856 -34857 -34858 -34859 -34860 -34861 -34862 -34863 -34864 -34865 -34866 -34867 -34868 -34869 -34870 -34871 -34872 -34873 -34874 -34875 -34876 -34877 -34878 -34879 -34880 -34881 -34882 -34883 -34884 -34885 -34886 -34887 -34888 -34889 -34890 -34891 -34892 -34893 -34894 -34895 -34896 -34897 -34898 -34899 -34900 -34901 -34902 -34903 -34904 -34905 -34906 -34907 -34908 -34909 -34910 -34911 -34912 -34913 -34914 -34915 -34916 -34917 -34918 -34919 -34920 -34921 -34922 -34923 -34924 -34925 -34926 -34927 -34928 -34929 -34930 -34931 -34932 -34933 -34934 -34935 -34936 -34937 -34938 -34939 -34940 -34941 -34942 -34943 -34944 -34945 -34946 -34947 -34948 -34949 -34950 -34951 -34952 -34953 -34954 -34955 -34956 -34957 -34958 -34959 -34960 -34961 -34962 -34963 -34964 -34965 -34966 -34967 -34968 -34969 -34970 -34971 -34972 -34973 -34974 -34975 -34976 -34977 -34978 -34979 -34980 -34981 -34982 -34983 -34984 -34985 -34986 -34987 -34988 -34989 -34990 -34991 -34992 -34993 -34994 -34995 -34996 -34997 -34998 -34999 -35000 -35001 -35002 -35003 -35004 -35005 -35006 -35007 -35008 -35009 -35010 -35011 -35012 -35013 -35014 -35015 -35016 -35017 -35018 -35019 -35020 -35021 -35022 -35023 -35024 -35025 -35026 -35027 -35028 -35029 -35030 -35031 -35032 -35033 -35034 -35035 -35036 -35037 -35038 -35039 -35040 -35041 -35042 -35043 -35044 -35045 -35046 -35047 -35048 -35049 -35050 -35051 -35052 -35053 -35054 -35055 -35056 -35057 -35058 -35059 -35060 -35061 -35062 -35063 -35064 -35065 -35066 -35067 -35068 -35069 -35070 -35071 -35072 -35073 -35074 -35075 -35076 -35077 -35078 -35079 -35080 -35081 -35082 -35083 -35084 -35085 -35086 -35087 -35088 -35089 -35090 -35091 -35092 -35093 -35094 -35095 -35096 -35097 -35098 -35099 -35100 -35101 -35102 -35103 -35104 -35105 -35106 -35107 -35108 -35109 -35110 -35111 -35112 -35113 -35114 -35115 -35116 -35117 -35118 -35119 -35120 -35121 -35122 -35123 -35124 -35125 -35126 -35127 -35128 -35129 -35130 -35131 -35132 -35133 -35134 -35135 -35136 -35137 -35138 -35139 -35140 -35141 -35142 -35143 -35144 -35145 -35146 -35147 -35148 -35149 -35150 -35151 -35152 -35153 -35154 -35155 -35156 -35157 -35158 -35159 -35160 -35161 -35162 -35163 -35164 -35165 -35166 -35167 -35168 -35169 -35170 -35171 -35172 -35173 -35174 -35175 -35176 -35177 -35178 -35179 -35180 -35181 -35182 -35183 -35184 -35185 -35186 -35187 -35188 -35189 -35190 -35191 -35192 -35193 -35194 -35195 -35196 -35197 -35198 -35199 -35200 -35201 -35202 -35203 -35204 -35205 -35206 -35207 -35208 -35209 -35210 -35211 -35212 -35213 -35214 -35215 -35216 -35217 -35218 -35219 -35220 -35221 -35222 -35223 -35224 -35225 -35226 -35227 -35228 -35229 -35230 -35231 -35232 -35233 -35234 -35235 -35236 -35237 -35238 -35239 -35240 -35241 -35242 -35243 -35244 -35245 -35246 -35247 -35248 -35249 -35250 -35251 -35252 -35253 -35254 -35255 -35256 -35257 -35258 -35259 -35260 -35261 -35262 -35263 -35264 -35265 -35266 -35267 -35268 -35269 -35270 -35271 -35272 -35273 -35274 -35275 -35276 -35277 -35278 -35279 -35280 -35281 -35282 -35283 -35284 -35285 -35286 -35287 -35288 -35289 -35290 -35291 -35292 -35293 -35294 -35295 -35296 -35297 -35298 -35299 -35300 -35301 -35302 -35303 -35304 -35305 -35306 -35307 -35308 -35309 -35310 -35311 -35312 -35313 -35314 -35315 -35316 -35317 -35318 -35319 -35320 -35321 -35322 -35323 -35324 -35325 -35326 -35327 -35328 -35329 -35330 -35331 -35332 -35333 -35334 -35335 -35336 -35337 -35338 -35339 -35340 -35341 -35342 -35343 -35344 -35345 -35346 -35347 -35348 -35349 -35350 -35351 -35352 -35353 -35354 -35355 -35356 -35357 -35358 -35359 -35360 -35361 -35362 -35363 -35364 -35365 -35366 -35367 -35368 -35369 -35370 -35371 -35372 -35373 -35374 -35375 -35376 -35377 -35378 -35379 -35380 -35381 -35382 -35383 -35384 -35385 -35386 -35387 -35388 -35389 -35390 -35391 -35392 -35393 -35394 -35395 -35396 -35397 -35398 -35399 -35400 -35401 -35402 -35403 -35404 -35405 -35406 -35407 -35408 -35409 -35410 -35411 -35412 -35413 -35414 -35415 -35416 -35417 -35418 -35419 -35420 -35421 -35422 -35423 -35424 -35425 -35426 -35427 -35428 -35429 -35430 -35431 -35432 -35433 -35434 -35435 -35436 -35437 -35438 -35439 -35440 -35441 -35442 -35443 -35444 -35445 -35446 -35447 -35448 -35449 -35450 -35451 -35452 -35453 -35454 -35455 -35456 -35457 -35458 -35459 -35460 -35461 -35462 -35463 -35464 -35465 -35466 -35467 -35468 -35469 -35470 -35471 -35472 -35473 -35474 -35475 -35476 -35477 -35478 -35479 -35480 -35481 -35482 -35483 -35484 -35485 -35486 -35487 -35488 -35489 -35490 -35491 -35492 -35493 -35494 -35495 -35496 -35497 -35498 -35499 -35500 -35501 -35502 -35503 -35504 -35505 -35506 -35507 -35508 -35509 -35510 -35511 -35512 -35513 -35514 -35515 -35516 -35517 -35518 -35519 -35520 -35521 -35522 -35523 -35524 -35525 -35526 -35527 -35528 -35529 -35530 -35531 -35532 -35533 -35534 -35535 -35536 -35537 -35538 -35539 -35540 -35541 -35542 -35543 -35544 -35545 -35546 -35547 -35548 -35549 -35550 -35551 -35552 -35553 -35554 -35555 -35556 -35557 -35558 -35559 -35560 -35561 -35562 -35563 -35564 -35565 -35566 -35567 -35568 -35569 -35570 -35571 -35572 -35573 -35574 -35575 -35576 -35577 -35578 -35579 -35580 -35581 -35582 -35583 -35584 -35585 -35586 -35587 -35588 -35589 -35590 -35591 -35592 -35593 -35594 -35595 -35596 -35597 -35598 -35599 -35600 -35601 -35602 -35603 -35604 -35605 -35606 -35607 -35608 -35609 -35610 -35611 -35612 -35613 -35614 -35615 -35616 -35617 -35618 -35619 -35620 -35621 -35622 -35623 -35624 -35625 -35626 -35627 -35628 -35629 -35630 -35631 -35632 -35633 -35634 -35635 -35636 -35637 -35638 -35639 -35640 -35641 -35642 -35643 -35644 -35645 -35646 -35647 -35648 -35649 -35650 -35651 -35652 -35653 -35654 -35655 -35656 -35657 -35658 -35659 -35660 -35661 -35662 -35663 -35664 -35665 -35666 -35667 -35668 -35669 -35670 -35671 -35672 -35673 -35674 -35675 -35676 -35677 -35678 -35679 -35680 -35681 -35682 -35683 -35684 -35685 -35686 -35687 -35688 -35689 -35690 -35691 -35692 -35693 -35694 -35695 -35696 -35697 -35698 -35699 -35700 -35701 -35702 -35703 -35704 -35705 -35706 -35707 -35708 -35709 -35710 -35711 -35712 -35713 -35714 -35715 -35716 -35717 -35718 -35719 -35720 -35721 -35722 -35723 -35724 -35725 -35726 -35727 -35728 -35729 -35730 -35731 -35732 -35733 -35734 -35735 -35736 -35737 -35738 -35739 -35740 -35741 -35742 -35743 -35744 -35745 -35746 -35747 -35748 -35749 -35750 -35751 -35752 -35753 -35754 -35755 -35756 -35757 -35758 -35759 -35760 -35761 -35762 -35763 -35764 -35765 -35766 -35767 -35768 -35769 -35770 -35771 -35772 -35773 -35774 -35775 -35776 -35777 -35778 -35779 -35780 -35781 -35782 -35783 -35784 -35785 -35786 -35787 -35788 -35789 -35790 -35791 -35792 -35793 -35794 -35795 -35796 -35797 -35798 -35799 -35800 -35801 -35802 -35803 -35804 -35805 -35806 -35807 -35808 -35809 -35810 -35811 -35812 -35813 -35814 -35815 -35816 -35817 -35818 -35819 -35820 -35821 -35822 -35823 -35824 -35825 -35826 -35827 -35828 -35829 -35830 -35831 -35832 -35833 -35834 -35835 -35836 -35837 -35838 -35839 -35840 -35841 -35842 -35843 -35844 -35845 -35846 -35847 -35848 -35849 -35850 -35851 -35852 -35853 -35854 -35855 -35856 -35857 -35858 -35859 -35860 -35861 -35862 -35863 -35864 -35865 -35866 -35867 -35868 -35869 -35870 -35871 -35872 -35873 -35874 -35875 -35876 -35877 -35878 -35879 -35880 -35881 -35882 -35883 -35884 -35885 -35886 -35887 -35888 -35889 -35890 -35891 -35892 -35893 -35894 -35895 -35896 -35897 -35898 -35899 -35900 -35901 -35902 -35903 -35904 -35905 -35906 -35907 -35908 -35909 -35910 -35911 -35912 -35913 -35914 -35915 -35916 -35917 -35918 -35919 -35920 -35921 -35922 -35923 -35924 -35925 -35926 -35927 -35928 -35929 -35930 -35931 -35932 -35933 -35934 -35935 -35936 -35937 -35938 -35939 -35940 -35941 -35942 -35943 -35944 -35945 -35946 -35947 -35948 -35949 -35950 -35951 -35952 -35953 -35954 -35955 -35956 -35957 -35958 -35959 -35960 -35961 -35962 -35963 -35964 -35965 -35966 -35967 -35968 -35969 -35970 -35971 -35972 -35973 -35974 -35975 -35976 -35977 -35978 -35979 -35980 -35981 -35982 -35983 -35984 -35985 -35986 -35987 -35988 -35989 -35990 -35991 -35992 -35993 -35994 -35995 -35996 -35997 -35998 -35999 -36000 -36001 -36002 -36003 -36004 -36005 -36006 -36007 -36008 -36009 -36010 -36011 -36012 -36013 -36014 -36015 -36016 -36017 -36018 -36019 -36020 -36021 -36022 -36023 -36024 -36025 -36026 -36027 -36028 -36029 -36030 -36031 -36032 -36033 -36034 -36035 -36036 -36037 -36038 -36039 -36040 -36041 -36042 -36043 -36044 -36045 -36046 -36047 -36048 -36049 -36050 -36051 -36052 -36053 -36054 -36055 -36056 -36057 -36058 -36059 -36060 -36061 -36062 -36063 -36064 -36065 -36066 -36067 -36068 -36069 -36070 -36071 -36072 -36073 -36074 -36075 -36076 -36077 -36078 -36079 -36080 -36081 -36082 -36083 -36084 -36085 -36086 -36087 -36088 -36089 -36090 -36091 -36092 -36093 -36094 -36095 -36096 -36097 -36098 -36099 -36100 -36101 -36102 -36103 -36104 -36105 -36106 -36107 -36108 -36109 -36110 -36111 -36112 -36113 -36114 -36115 -36116 -36117 -36118 -36119 -36120 -36121 -36122 -36123 -36124 -36125 -36126 -36127 -36128 -36129 -36130 -36131 -36132 -36133 -36134 -36135 -36136 -36137 -36138 -36139 -36140 -36141 -36142 -36143 -36144 -36145 -36146 -36147 -36148 -36149 -36150 -36151 -36152 -36153 -36154 -36155 -36156 -36157 -36158 -36159 -36160 -36161 -36162 -36163 -36164 -36165 -36166 -36167 -36168 -36169 -36170 -36171 -36172 -36173 -36174 -36175 -36176 -36177 -36178 -36179 -36180 -36181 -36182 -36183 -36184 -36185 -36186 -36187 -36188 -36189 -36190 -36191 -36192 -36193 -36194 -36195 -36196 -36197 -36198 -36199 -36200 -36201 -36202 -36203 -36204 -36205 -36206 -36207 -36208 -36209 -36210 -36211 -36212 -36213 -36214 -36215 -36216 -36217 -36218 -36219 -36220 -36221 -36222 -36223 -36224 -36225 -36226 -36227 -36228 -36229 -36230 -36231 -36232 -36233 -36234 -36235 -36236 -36237 -36238 -36239 -36240 -36241 -36242 -36243 -36244 -36245 -36246 -36247 -36248 -36249 -36250 -36251 -36252 -36253 -36254 -36255 -36256 -36257 -36258 -36259 -36260 -36261 -36262 -36263 -36264 -36265 -36266 -36267 -36268 -36269 -36270 -36271 -36272 -36273 -36274 -36275 -36276 -36277 -36278 -36279 -36280 -36281 -36282 -36283 -36284 -36285 -36286 -36287 -36288 -36289 -36290 -36291 -36292 -36293 -36294 -36295 -36296 -36297 -36298 -36299 -36300 -36301 -36302 -36303 -36304 -36305 -36306 -36307 -36308 -36309 -36310 -36311 -36312 -36313 -36314 -36315 -36316 -36317 -36318 -36319 -36320 -36321 -36322 -36323 -36324 -36325 -36326 -36327 -36328 -36329 -36330 -36331 -36332 -36333 -36334 -36335 -36336 -36337 -36338 -36339 -36340 -36341 -36342 -36343 -36344 -36345 -36346 -36347 -36348 -36349 -36350 -36351 -36352 -36353 -36354 -36355 -36356 -36357 -36358 -36359 -36360 -36361 -36362 -36363 -36364 -36365 -36366 -36367 -36368 -36369 -36370 -36371 -36372 -36373 -36374 -36375 -36376 -36377 -36378 -36379 -36380 -36381 -36382 -36383 -36384 -36385 -36386 -36387 -36388 -36389 -36390 -36391 -36392 -36393 -36394 -36395 -36396 -36397 -36398 -36399 -36400 -36401 -36402 -36403 -36404 -36405 -36406 -36407 -36408 -36409 -36410 -36411 -36412 -36413 -36414 -36415 -36416 -36417 -36418 -36419 -36420 -36421 -36422 -36423 -36424 -36425 -36426 -36427 -36428 -36429 -36430 -36431 -36432 -36433 -36434 -36435 -36436 -36437 -36438 -36439 -36440 -36441 -36442 -36443 -36444 -36445 -36446 -36447 -36448 -36449 -36450 -36451 -36452 -36453 -36454 -36455 -36456 -36457 -36458 -36459 -36460 -36461 -36462 -36463 -36464 -36465 -36466 -36467 -36468 -36469 -36470 -36471 -36472 -36473 -36474 -36475 -36476 -36477 -36478 -36479 -36480 -36481 -36482 -36483 -36484 -36485 -36486 -36487 -36488 -36489 -36490 -36491 -36492 -36493 -36494 -36495 -36496 -36497 -36498 -36499 -36500 -36501 -36502 -36503 -36504 -36505 -36506 -36507 -36508 -36509 -36510 -36511 -36512 -36513 -36514 -36515 -36516 -36517 -36518 -36519 -36520 -36521 -36522 -36523 -36524 -36525 -36526 -36527 -36528 -36529 -36530 -36531 -36532 -36533 -36534 -36535 -36536 -36537 -36538 -36539 -36540 -36541 -36542 -36543 -36544 -36545 -36546 -36547 -36548 -36549 -36550 -36551 -36552 -36553 -36554 -36555 -36556 -36557 -36558 -36559 -36560 -36561 -36562 -36563 -36564 -36565 -36566 -36567 -36568 -36569 -36570 -36571 -36572 -36573 -36574 -36575 -36576 -36577 -36578 -36579 -36580 -36581 -36582 -36583 -36584 -36585 -36586 -36587 -36588 -36589 -36590 -36591 -36592 -36593 -36594 -36595 -36596 -36597 -36598 -36599 -36600 -36601 -36602 -36603 -36604 -36605 -36606 -36607 -36608 -36609 -36610 -36611 -36612 -36613 -36614 -36615 -36616 -36617 -36618 -36619 -36620 -36621 -36622 -36623 -36624 -36625 -36626 -36627 -36628 -36629 -36630 -36631 -36632 -36633 -36634 -36635 -36636 -36637 -36638 -36639 -36640 -36641 -36642 -36643 -36644 -36645 -36646 -36647 -36648 -36649 -36650 -36651 -36652 -36653 -36654 -36655 -36656 -36657 -36658 -36659 -36660 -36661 -36662 -36663 -36664 -36665 -36666 -36667 -36668 -36669 -36670 -36671 -36672 -36673 -36674 -36675 -36676 -36677 -36678 -36679 -36680 -36681 -36682 -36683 -36684 -36685 -36686 -36687 -36688 -36689 -36690 -36691 -36692 -36693 -36694 -36695 -36696 -36697 -36698 -36699 -36700 -36701 -36702 -36703 -36704 -36705 -36706 -36707 -36708 -36709 -36710 -36711 -36712 -36713 -36714 -36715 -36716 -36717 -36718 -36719 -36720 -36721 -36722 -36723 -36724 -36725 -36726 -36727 -36728 -36729 -36730 -36731 -36732 -36733 -36734 -36735 -36736 -36737 -36738 -36739 -36740 -36741 -36742 -36743 -36744 -36745 -36746 -36747 -36748 -36749 -36750 -36751 -36752 -36753 -36754 -36755 -36756 -36757 -36758 -36759 -36760 -36761 -36762 -36763 -36764 -36765 -36766 -36767 -36768 -36769 -36770 -36771 -36772 -36773 -36774 -36775 -36776 -36777 -36778 -36779 -36780 -36781 -36782 -36783 -36784 -36785 -36786 -36787 -36788 -36789 -36790 -36791 -36792 -36793 -36794 -36795 -36796 -36797 -36798 -36799 -36800 -36801 -36802 -36803 -36804 -36805 -36806 -36807 -36808 -36809 -36810 -36811 -36812 -36813 -36814 -36815 -36816 -36817 -36818 -36819 -36820 -36821 -36822 -36823 -36824 -36825 -36826 -36827 -36828 -36829 -36830 -36831 -36832 -36833 -36834 -36835 -36836 -36837 -36838 -36839 -36840 -36841 -36842 -36843 -36844 -36845 -36846 -36847 -36848 -36849 -36850 -36851 -36852 -36853 -36854 -36855 -36856 -36857 -36858 -36859 -36860 -36861 -36862 -36863 -36864 -36865 -36866 -36867 -36868 -36869 -36870 -36871 -36872 -36873 -36874 -36875 -36876 -36877 -36878 -36879 -36880 -36881 -36882 -36883 -36884 -36885 -36886 -36887 -36888 -36889 -36890 -36891 -36892 -36893 -36894 -36895 -36896 -36897 -36898 -36899 -36900 -36901 -36902 -36903 -36904 -36905 -36906 -36907 -36908 -36909 -36910 -36911 -36912 -36913 -36914 -36915 -36916 -36917 -36918 -36919 -36920 -36921 -36922 -36923 -36924 -36925 -36926 -36927 -36928 -36929 -36930 -36931 -36932 -36933 -36934 -36935 -36936 -36937 -36938 -36939 -36940 -36941 -36942 -36943 -36944 -36945 -36946 -36947 -36948 -36949 -36950 -36951 -36952 -36953 -36954 -36955 -36956 -36957 -36958 -36959 -36960 -36961 -36962 -36963 -36964 -36965 -36966 -36967 -36968 -36969 -36970 -36971 -36972 -36973 -36974 -36975 -36976 -36977 -36978 -36979 -36980 -36981 -36982 -36983 -36984 -36985 -36986 -36987 -36988 -36989 -36990 -36991 -36992 -36993 -36994 -36995 -36996 -36997 -36998 -36999 -37000 -37001 -37002 -37003 -37004 -37005 -37006 -37007 -37008 -37009 -37010 -37011 -37012 -37013 -37014 -37015 -37016 -37017 -37018 -37019 -37020 -37021 -37022 -37023 -37024 -37025 -37026 -37027 -37028 -37029 -37030 -37031 -37032 -37033 -37034 -37035 -37036 -37037 -37038 -37039 -37040 -37041 -37042 -37043 -37044 -37045 -37046 -37047 -37048 -37049 -37050 -37051 -37052 -37053 -37054 -37055 -37056 -37057 -37058 -37059 -37060 -37061 -37062 -37063 -37064 -37065 -37066 -37067 -37068 -37069 -37070 -37071 -37072 -37073 -37074 -37075 -37076 -37077 -37078 -37079 -37080 -37081 -37082 -37083 -37084 -37085 -37086 -37087 -37088 -37089 -37090 -37091 -37092 -37093 -37094 -37095 -37096 -37097 -37098 -37099 -37100 -37101 -37102 -37103 -37104 -37105 -37106 -37107 -37108 -37109 -37110 -37111 -37112 -37113 -37114 -37115 -37116 -37117 -37118 -37119 -37120 -37121 -37122 -37123 -37124 -37125 -37126 -37127 -37128 -37129 -37130 -37131 -37132 -37133 -37134 -37135 -37136 -37137 -37138 -37139 -37140 -37141 -37142 -37143 -37144 -37145 -37146 -37147 -37148 -37149 -37150 -37151 -37152 -37153 -37154 -37155 -37156 -37157 -37158 -37159 -37160 -37161 -37162 -37163 -37164 -37165 -37166 -37167 -37168 -37169 -37170 -37171 -37172 -37173 -37174 -37175 -37176 -37177 -37178 -37179 -37180 -37181 -37182 -37183 -37184 -37185 -37186 -37187 -37188 -37189 -37190 -37191 -37192 -37193 -37194 -37195 -37196 -37197 -37198 -37199 -37200 -37201 -37202 -37203 -37204 -37205 -37206 -37207 -37208 -37209 -37210 -37211 -37212 -37213 -37214 -37215 -37216 -37217 -37218 -37219 -37220 -37221 -37222 -37223 -37224 -37225 -37226 -37227 -37228 -37229 -37230 -37231 -37232 -37233 -37234 -37235 -37236 -37237 -37238 -37239 -37240 -37241 -37242 -37243 -37244 -37245 -37246 -37247 -37248 -37249 -37250 -37251 -37252 -37253 -37254 -37255 -37256 -37257 -37258 -37259 -37260 -37261 -37262 -37263 -37264 -37265 -37266 -37267 -37268 -37269 -37270 -37271 -37272 -37273 -37274 -37275 -37276 -37277 -37278 -37279 -37280 -37281 -37282 -37283 -37284 -37285 -37286 -37287 -37288 -37289 -37290 -37291 -37292 -37293 -37294 -37295 -37296 -37297 -37298 -37299 -37300 -37301 -37302 -37303 -37304 -37305 -37306 -37307 -37308 -37309 -37310 -37311 -37312 -37313 -37314 -37315 -37316 -37317 -37318 -37319 -37320 -37321 -37322 -37323 -37324 -37325 -37326 -37327 -37328 -37329 -37330 -37331 -37332 -37333 -37334 -37335 -37336 -37337 -37338 -37339 -37340 -37341 -37342 -37343 -37344 -37345 -37346 -37347 -37348 -37349 -37350 -37351 -37352 -37353 -37354 -37355 -37356 -37357 -37358 -37359 -37360 -37361 -37362 -37363 -37364 -37365 -37366 -37367 -37368 -37369 -37370 -37371 -37372 -37373 -37374 -37375 -37376 -37377 -37378 -37379 -37380 -37381 -37382 -37383 -37384 -37385 -37386 -37387 -37388 -37389 -37390 -37391 -37392 -37393 -37394 -37395 -37396 -37397 -37398 -37399 -37400 -37401 -37402 -37403 -37404 -37405 -37406 -37407 -37408 -37409 -37410 -37411 -37412 -37413 -37414 -37415 -37416 -37417 -37418 -37419 -37420 -37421 -37422 -37423 -37424 -37425 -37426 -37427 -37428 -37429 -37430 -37431 -37432 -37433 -37434 -37435 -37436 -37437 -37438 -37439 -37440 -37441 -37442 -37443 -37444 -37445 -37446 -37447 -37448 -37449 -37450 -37451 -37452 -37453 -37454 -37455 -37456 -37457 -37458 -37459 -37460 -37461 -37462 -37463 -37464 -37465 -37466 -37467 -37468 -37469 -37470 -37471 -37472 -37473 -37474 -37475 -37476 -37477 -37478 -37479 -37480 -37481 -37482 -37483 -37484 -37485 -37486 -37487 -37488 -37489 -37490 -37491 -37492 -37493 -37494 -37495 -37496 -37497 -37498 -37499 -37500 -37501 -37502 -37503 -37504 -37505 -37506 -37507 -37508 -37509 -37510 -37511 -37512 -37513 -37514 -37515 -37516 -37517 -37518 -37519 -37520 -37521 -37522 -37523 -37524 -37525 -37526 -37527 -37528 -37529 -37530 -37531 -37532 -37533 -37534 -37535 -37536 -37537 -37538 -37539 -37540 -37541 -37542 -37543 -37544 -37545 -37546 -37547 -37548 -37549 -37550 -37551 -37552 -37553 -37554 -37555 -37556 -37557 -37558 -37559 -37560 -37561 -37562 -37563 -37564 -37565 -37566 -37567 -37568 -37569 -37570 -37571 -37572 -37573 -37574 -37575 -37576 -37577 -37578 -37579 -37580 -37581 -37582 -37583 -37584 -37585 -37586 -37587 -37588 -37589 -37590 -37591 -37592 -37593 -37594 -37595 -37596 -37597 -37598 -37599 -37600 -37601 -37602 -37603 -37604 -37605 -37606 -37607 -37608 -37609 -37610 -37611 -37612 -37613 -37614 -37615 -37616 -37617 -37618 -37619 -37620 -37621 -37622 -37623 -37624 -37625 -37626 -37627 -37628 -37629 -37630 -37631 -37632 -37633 -37634 -37635 -37636 -37637 -37638 -37639 -37640 -37641 -37642 -37643 -37644 -37645 -37646 -37647 -37648 -37649 -37650 -37651 -37652 -37653 -37654 -37655 -37656 -37657 -37658 -37659 -37660 -37661 -37662 -37663 -37664 -37665 -37666 -37667 -37668 -37669 -37670 -37671 -37672 -37673 -37674 -37675 -37676 -37677 -37678 -37679 -37680 -37681 -37682 -37683 -37684 -37685 -37686 -37687 -37688 -37689 -37690 -37691 -37692 -37693 -37694 -37695 -37696 -37697 -37698 -37699 -37700 -37701 -37702 -37703 -37704 -37705 -37706 -37707 -37708 -37709 -37710 -37711 -37712 -37713 -37714 -37715 -37716 -37717 -37718 -37719 -37720 -37721 -37722 -37723 -37724 -37725 -37726 -37727 -37728 -37729 -37730 -37731 -37732 -37733 -37734 -37735 -37736 -37737 -37738 -37739 -37740 -37741 -37742 -37743 -37744 -37745 -37746 -37747 -37748 -37749 -37750 -37751 -37752 -37753 -37754 -37755 -37756 -37757 -37758 -37759 -37760 -37761 -37762 -37763 -37764 -37765 -37766 -37767 -37768 -37769 -37770 -37771 -37772 -37773 -37774 -37775 -37776 -37777 -37778 -37779 -37780 -37781 -37782 -37783 -37784 -37785 -37786 -37787 -37788 -37789 -37790 -37791 -37792 -37793 -37794 -37795 -37796 -37797 -37798 -37799 -37800 -37801 -37802 -37803 -37804 -37805 -37806 -37807 -37808 -37809 -37810 -37811 -37812 -37813 -37814 -37815 -37816 -37817 -37818 -37819 -37820 -37821 -37822 -37823 -37824 -37825 -37826 -37827 -37828 -37829 -37830 -37831 -37832 -37833 -37834 -37835 -37836 -37837 -37838 -37839 -37840 -37841 -37842 -37843 -37844 -37845 -37846 -37847 -37848 -37849 -37850 -37851 -37852 -37853 -37854 -37855 -37856 -37857 -37858 -37859 -37860 -37861 -37862 -37863 -37864 -37865 -37866 -37867 -37868 -37869 -37870 -37871 -37872 -37873 -37874 -37875 -37876 -37877 -37878 -37879 -37880 -37881 -37882 -37883 -37884 -37885 -37886 -37887 -37888 -37889 -37890 -37891 -37892 -37893 -37894 -37895 -37896 -37897 -37898 -37899 -37900 -37901 -37902 -37903 -37904 -37905 -37906 -37907 -37908 -37909 -37910 -37911 -37912 -37913 -37914 -37915 -37916 -37917 -37918 -37919 -37920 -37921 -37922 -37923 -37924 -37925 -37926 -37927 -37928 -37929 -37930 -37931 -37932 -37933 -37934 -37935 -37936 -37937 -37938 -37939 -37940 -37941 -37942 -37943 -37944 -37945 -37946 -37947 -37948 -37949 -37950 -37951 -37952 -37953 -37954 -37955 -37956 -37957 -37958 -37959 -37960 -37961 -37962 -37963 -37964 -37965 -37966 -37967 -37968 -37969 -37970 -37971 -37972 -37973 -37974 -37975 -37976 -37977 -37978 -37979 -37980 -37981 -37982 -37983 -37984 -37985 -37986 -37987 -37988 -37989 -37990 -37991 -37992 -37993 -37994 -37995 -37996 -37997 -37998 -37999 -38000 -38001 -38002 -38003 -38004 -38005 -38006 -38007 -38008 -38009 -38010 -38011 -38012 -38013 -38014 -38015 -38016 -38017 -38018 -38019 -38020 -38021 -38022 -38023 -38024 -38025 -38026 -38027 -38028 -38029 -38030 -38031 -38032 -38033 -38034 -38035 -38036 -38037 -38038 -38039 -38040 -38041 -38042 -38043 -38044 -38045 -38046 -38047 -38048 -38049 -38050 -38051 -38052 -38053 -38054 -38055 -38056 -38057 -38058 -38059 -38060 -38061 -38062 -38063 -38064 -38065 -38066 -38067 -38068 -38069 -38070 -38071 -38072 -38073 -38074 -38075 -38076 -38077 -38078 -38079 -38080 -38081 -38082 -38083 -38084 -38085 -38086 -38087 -38088 -38089 -38090 -38091 -38092 -38093 -38094 -38095 -38096 -38097 -38098 -38099 -38100 -38101 -38102 -38103 -38104 -38105 -38106 -38107 -38108 -38109 -38110 -38111 -38112 -38113 -38114 -38115 -38116 -38117 -38118 -38119 -38120 -38121 -38122 -38123 -38124 -38125 -38126 -38127 -38128 -38129 -38130 -38131 -38132 -38133 -38134 -38135 -38136 -38137 -38138 -38139 -38140 -38141 -38142 -38143 -38144 -38145 -38146 -38147 -38148 -38149 -38150 -38151 -38152 -38153 -38154 -38155 -38156 -38157 -38158 -38159 -38160 -38161 -38162 -38163 -38164 -38165 -38166 -38167 -38168 -38169 -38170 -38171 -38172 -38173 -38174 -38175 -38176 -38177 -38178 -38179 -38180 -38181 -38182 -38183 -38184 -38185 -38186 -38187 -38188 -38189 -38190 -38191 -38192 -38193 -38194 -38195 -38196 -38197 -38198 -38199 -38200 -38201 -38202 -38203 -38204 -38205 -38206 -38207 -38208 -38209 -38210 -38211 -38212 -38213 -38214 -38215 -38216 -38217 -38218 -38219 -38220 -38221 -38222 -38223 -38224 -38225 -38226 -38227 -38228 -38229 -38230 -38231 -38232 -38233 -38234 -38235 -38236 -38237 -38238 -38239 -38240 -38241 -38242 -38243 -38244 -38245 -38246 -38247 -38248 -38249 -38250 -38251 -38252 -38253 -38254 -38255 -38256 -38257 -38258 -38259 -38260 -38261 -38262 -38263 -38264 -38265 -38266 -38267 -38268 -38269 -38270 -38271 -38272 -38273 -38274 -38275 -38276 -38277 -38278 -38279 -38280 -38281 -38282 -38283 -38284 -38285 -38286 -38287 -38288 -38289 -38290 -38291 -38292 -38293 -38294 -38295 -38296 -38297 -38298 -38299 -38300 -38301 -38302 -38303 -38304 -38305 -38306 -38307 -38308 -38309 -38310 -38311 -38312 -38313 -38314 -38315 -38316 -38317 -38318 -38319 -38320 -38321 -38322 -38323 -38324 -38325 -38326 -38327 -38328 -38329 -38330 -38331 -38332 -38333 -38334 -38335 -38336 -38337 -38338 -38339 -38340 -38341 -38342 -38343 -38344 -38345 -38346 -38347 -38348 -38349 -38350 -38351 -38352 -38353 -38354 -38355 -38356 -38357 -38358 -38359 -38360 -38361 -38362 -38363 -38364 -38365 -38366 -38367 -38368 -38369 -38370 -38371 -38372 -38373 -38374 -38375 -38376 -38377 -38378 -38379 -38380 -38381 -38382 -38383 -38384 -38385 -38386 -38387 -38388 -38389 -38390 -38391 -38392 -38393 -38394 -38395 -38396 -38397 -38398 -38399 -38400 -38401 -38402 -38403 -38404 -38405 -38406 -38407 -38408 -38409 -38410 -38411 -38412 -38413 -38414 -38415 -38416 -38417 -38418 -38419 -38420 -38421 -38422 -38423 -38424 -38425 -38426 -38427 -38428 -38429 -38430 -38431 -38432 -38433 -38434 -38435 -38436 -38437 -38438 -38439 -38440 -38441 -38442 -38443 -38444 -38445 -38446 -38447 -38448 -38449 -38450 -38451 -38452 -38453 -38454 -38455 -38456 -38457 -38458 -38459 -38460 -38461 -38462 -38463 -38464 -38465 -38466 -38467 -38468 -38469 -38470 -38471 -38472 -38473 -38474 -38475 -38476 -38477 -38478 -38479 -38480 -38481 -38482 -38483 -38484 -38485 -38486 -38487 -38488 -38489 -38490 -38491 -38492 -38493 -38494 -38495 -38496 -38497 -38498 -38499 -38500 -38501 -38502 -38503 -38504 -38505 -38506 -38507 -38508 -38509 -38510 -38511 -38512 -38513 -38514 -38515 -38516 -38517 -38518 -38519 -38520 -38521 -38522 -38523 -38524 -38525 -38526 -38527 -38528 -38529 -38530 -38531 -38532 -38533 -38534 -38535 -38536 -38537 -38538 -38539 -38540 -38541 -38542 -38543 -38544 -38545 -38546 -38547 -38548 -38549 -38550 -38551 -38552 -38553 -38554 -38555 -38556 -38557 -38558 -38559 -38560 -38561 -38562 -38563 -38564 -38565 -38566 -38567 -38568 -38569 -38570 -38571 -38572 -38573 -38574 -38575 -38576 -38577 -38578 -38579 -38580 -38581 -38582 -38583 -38584 -38585 -38586 -38587 -38588 -38589 -38590 -38591 -38592 -38593 -38594 -38595 -38596 -38597 -38598 -38599 -38600 -38601 -38602 -38603 -38604 -38605 -38606 -38607 -38608 -38609 -38610 -38611 -38612 -38613 -38614 -38615 -38616 -38617 -38618 -38619 -38620 -38621 -38622 -38623 -38624 -38625 -38626 -38627 -38628 -38629 -38630 -38631 -38632 -38633 -38634 -38635 -38636 -38637 -38638 -38639 -38640 -38641 -38642 -38643 -38644 -38645 -38646 -38647 -38648 -38649 -38650 -38651 -38652 -38653 -38654 -38655 -38656 -38657 -38658 -38659 -38660 -38661 -38662 -38663 -38664 -38665 -38666 -38667 -38668 -38669 -38670 -38671 -38672 -38673 -38674 -38675 -38676 -38677 -38678 -38679 -38680 -38681 -38682 -38683 -38684 -38685 -38686 -38687 -38688 -38689 -38690 -38691 -38692 -38693 -38694 -38695 -38696 -38697 -38698 -38699 -38700 -38701 -38702 -38703 -38704 -38705 -38706 -38707 -38708 -38709 -38710 -38711 -38712 -38713 -38714 -38715 -38716 -38717 -38718 -38719 -38720 -38721 -38722 -38723 -38724 -38725 -38726 -38727 -38728 -38729 -38730 -38731 -38732 -38733 -38734 -38735 -38736 -38737 -38738 -38739 -38740 -38741 -38742 -38743 -38744 -38745 -38746 -38747 -38748 -38749 -38750 -38751 -38752 -38753 -38754 -38755 -38756 -38757 -38758 -38759 -38760 -38761 -38762 -38763 -38764 -38765 -38766 -38767 -38768 -38769 -38770 -38771 -38772 -38773 -38774 -38775 -38776 -38777 -38778 -38779 -38780 -38781 -38782 -38783 -38784 -38785 -38786 -38787 -38788 -38789 -38790 -38791 -38792 -38793 -38794 -38795 -38796 -38797 -38798 -38799 -38800 -38801 -38802 -38803 -38804 -38805 -38806 -38807 -38808 -38809 -38810 -38811 -38812 -38813 -38814 -38815 -38816 -38817 -38818 -38819 -38820 -38821 -38822 -38823 -38824 -38825 -38826 -38827 -38828 -38829 -38830 -38831 -38832 -38833 -38834 -38835 -38836 -38837 -38838 -38839 -38840 -38841 -38842 -38843 -38844 -38845 -38846 -38847 -38848 -38849 -38850 -38851 -38852 -38853 -38854 -38855 -38856 -38857 -38858 -38859 -38860 -38861 -38862 -38863 -38864 -38865 -38866 -38867 -38868 -38869 -38870 -38871 -38872 -38873 -38874 -38875 -38876 -38877 -38878 -38879 -38880 -38881 -38882 -38883 -38884 -38885 -38886 -38887 -38888 -38889 -38890 -38891 -38892 -38893 -38894 -38895 -38896 -38897 -38898 -38899 -38900 -38901 -38902 -38903 -38904 -38905 -38906 -38907 -38908 -38909 -38910 -38911 -38912 -38913 -38914 -38915 -38916 -38917 -38918 -38919 -38920 -38921 -38922 -38923 -38924 -38925 -38926 -38927 -38928 -38929 -38930 -38931 -38932 -38933 -38934 -38935 -38936 -38937 -38938 -38939 -38940 -38941 -38942 -38943 -38944 -38945 -38946 -38947 -38948 -38949 -38950 -38951 -38952 -38953 -38954 -38955 -38956 -38957 -38958 -38959 -38960 -38961 -38962 -38963 -38964 -38965 -38966 -38967 -38968 -38969 -38970 -38971 -38972 -38973 -38974 -38975 -38976 -38977 -38978 -38979 -38980 -38981 -38982 -38983 -38984 -38985 -38986 -38987 -38988 -38989 -38990 -38991 -38992 -38993 -38994 -38995 -38996 -38997 -38998 -38999 -39000 -39001 -39002 -39003 -39004 -39005 -39006 -39007 -39008 -39009 -39010 -39011 -39012 -39013 -39014 -39015 -39016 -39017 -39018 -39019 -39020 -39021 -39022 -39023 -39024 -39025 -39026 -39027 -39028 -39029 -39030 -39031 -39032 -39033 -39034 -39035 -39036 -39037 -39038 -39039 -39040 -39041 -39042 -39043 -39044 -39045 -39046 -39047 -39048 -39049 -39050 -39051 -39052 -39053 -39054 -39055 -39056 -39057 -39058 -39059 -39060 -39061 -39062 -39063 -39064 -39065 -39066 -39067 -39068 -39069 -39070 -39071 -39072 -39073 -39074 -39075 -39076 -39077 -39078 -39079 -39080 -39081 -39082 -39083 -39084 -39085 -39086 -39087 -39088 -39089 -39090 -39091 -39092 -39093 -39094 -39095 -39096 -39097 -39098 -39099 -39100 -39101 -39102 -39103 -39104 -39105 -39106 -39107 -39108 -39109 -39110 -39111 -39112 -39113 -39114 -39115 -39116 -39117 -39118 -39119 -39120 -39121 -39122 -39123 -39124 -39125 -39126 -39127 -39128 -39129 -39130 -39131 -39132 -39133 -39134 -39135 -39136 -39137 -39138 -39139 -39140 -39141 -39142 -39143 -39144 -39145 -39146 -39147 -39148 -39149 -39150 -39151 -39152 -39153 -39154 -39155 -39156 -39157 -39158 -39159 -39160 -39161 -39162 -39163 -39164 -39165 -39166 -39167 -39168 -39169 -39170 -39171 -39172 -39173 -39174 -39175 -39176 -39177 -39178 -39179 -39180 -39181 -39182 -39183 -39184 -39185 -39186 -39187 -39188 -39189 -39190 -39191 -39192 -39193 -39194 -39195 -39196 -39197 -39198 -39199 -39200 -39201 -39202 -39203 -39204 -39205 -39206 -39207 -39208 -39209 -39210 -39211 -39212 -39213 -39214 -39215 -39216 -39217 -39218 -39219 -39220 -39221 -39222 -39223 -39224 -39225 -39226 -39227 -39228 -39229 -39230 -39231 -39232 -39233 -39234 -39235 -39236 -39237 -39238 -39239 -39240 -39241 -39242 -39243 -39244 -39245 -39246 -39247 -39248 -39249 -39250 -39251 -39252 -39253 -39254 -39255 -39256 -39257 -39258 -39259 -39260 -39261 -39262 -39263 -39264 -39265 -39266 -39267 -39268 -39269 -39270 -39271 -39272 -39273 -39274 -39275 -39276 -39277 -39278 -39279 -39280 -39281 -39282 -39283 -39284 -39285 -39286 -39287 -39288 -39289 -39290 -39291 -39292 -39293 -39294 -39295 -39296 -39297 -39298 -39299 -39300 -39301 -39302 -39303 -39304 -39305 -39306 -39307 -39308 -39309 -39310 -39311 -39312 -39313 -39314 -39315 -39316 -39317 -39318 -39319 -39320 -39321 -39322 -39323 -39324 -39325 -39326 -39327 -39328 -39329 -39330 -39331 -39332 -39333 -39334 -39335 -39336 -39337 -39338 -39339 -39340 -39341 -39342 -39343 -39344 -39345 -39346 -39347 -39348 -39349 -39350 -39351 -39352 -39353 -39354 -39355 -39356 -39357 -39358 -39359 -39360 -39361 -39362 -39363 -39364 -39365 -39366 -39367 -39368 -39369 -39370 -39371 -39372 -39373 -39374 -39375 -39376 -39377 -39378 -39379 -39380 -39381 -39382 -39383 -39384 -39385 -39386 -39387 -39388 -39389 -39390 -39391 -39392 -39393 -39394 -39395 -39396 -39397 -39398 -39399 -39400 -39401 -39402 -39403 -39404 -39405 -39406 -39407 -39408 -39409 -39410 -39411 -39412 -39413 -39414 -39415 -39416 -39417 -39418 -39419 -39420 -39421 -39422 -39423 -39424 -39425 -39426 -39427 -39428 -39429 -39430 -39431 -39432 -39433 -39434 -39435 -39436 -39437 -39438 -39439 -39440 -39441 -39442 -39443 -39444 -39445 -39446 -39447 -39448 -39449 -39450 -39451 -39452 -39453 -39454 -39455 -39456 -39457 -39458 -39459 -39460 -39461 -39462 -39463 -39464 -39465 -39466 -39467 -39468 -39469 -39470 -39471 -39472 -39473 -39474 -39475 -39476 -39477 -39478 -39479 -39480 -39481 -39482 -39483 -39484 -39485 -39486 -39487 -39488 -39489 -39490 -39491 -39492 -39493 -39494 -39495 -39496 -39497 -39498 -39499 -39500 -39501 -39502 -39503 -39504 -39505 -39506 -39507 -39508 -39509 -39510 -39511 -39512 -39513 -39514 -39515 -39516 -39517 -39518 -39519 -39520 -39521 -39522 -39523 -39524 -39525 -39526 -39527 -39528 -39529 -39530 -39531 -39532 -39533 -39534 -39535 -39536 -39537 -39538 -39539 -39540 -39541 -39542 -39543 -39544 -39545 -39546 -39547 -39548 -39549 -39550 -39551 -39552 -39553 -39554 -39555 -39556 -39557 -39558 -39559 -39560 -39561 -39562 -39563 -39564 -39565 -39566 -39567 -39568 -39569 -39570 -39571 -39572 -39573 -39574 -39575 -39576 -39577 -39578 -39579 -39580 -39581 -39582 -39583 -39584 -39585 -39586 -39587 -39588 -39589 -39590 -39591 -39592 -39593 -39594 -39595 -39596 -39597 -39598 -39599 -39600 -39601 -39602 -39603 -39604 -39605 -39606 -39607 -39608 -39609 -39610 -39611 -39612 -39613 -39614 -39615 -39616 -39617 -39618 -39619 -39620 -39621 -39622 -39623 -39624 -39625 -39626 -39627 -39628 -39629 -39630 -39631 -39632 -39633 -39634 -39635 -39636 -39637 -39638 -39639 -39640 -39641 -39642 -39643 -39644 -39645 -39646 -39647 -39648 -39649 -39650 -39651 -39652 -39653 -39654 -39655 -39656 -39657 -39658 -39659 -39660 -39661 -39662 -39663 -39664 -39665 -39666 -39667 -39668 -39669 -39670 -39671 -39672 -39673 -39674 -39675 -39676 -39677 -39678 -39679 -39680 -39681 -39682 -39683 -39684 -39685 -39686 -39687 -39688 -39689 -39690 -39691 -39692 -39693 -39694 -39695 -39696 -39697 -39698 -39699 -39700 -39701 -39702 -39703 -39704 -39705 -39706 -39707 -39708 -39709 -39710 -39711 -39712 -39713 -39714 -39715 -39716 -39717 -39718 -39719 -39720 -39721 -39722 -39723 -39724 -39725 -39726 -39727 -39728 -39729 -39730 -39731 -39732 -39733 -39734 -39735 -39736 -39737 -39738 -39739 -39740 -39741 -39742 -39743 -39744 -39745 -39746 -39747 -39748 -39749 -39750 -39751 -39752 -39753 -39754 -39755 -39756 -39757 -39758 -39759 -39760 -39761 -39762 -39763 -39764 -39765 -39766 -39767 -39768 -39769 -39770 -39771 -39772 -39773 -39774 -39775 -39776 -39777 -39778 -39779 -39780 -39781 -39782 -39783 -39784 -39785 -39786 -39787 -39788 -39789 -39790 -39791 -39792 -39793 -39794 -39795 -39796 -39797 -39798 -39799 -39800 -39801 -39802 -39803 -39804 -39805 -39806 -39807 -39808 -39809 -39810 -39811 -39812 -39813 -39814 -39815 -39816 -39817 -39818 -39819 -39820 -39821 -39822 -39823 -39824 -39825 -39826 -39827 -39828 -39829 -39830 -39831 -39832 -39833 -39834 -39835 -39836 -39837 -39838 -39839 -39840 -39841 -39842 -39843 -39844 -39845 -39846 -39847 -39848 -39849 -39850 -39851 -39852 -39853 -39854 -39855 -39856 -39857 -39858 -39859 -39860 -39861 -39862 -39863 -39864 -39865 -39866 -39867 -39868 -39869 -39870 -39871 -39872 -39873 -39874 -39875 -39876 -39877 -39878 -39879 -39880 -39881 -39882 -39883 -39884 -39885 -39886 -39887 -39888 -39889 -39890 -39891 -39892 -39893 -39894 -39895 -39896 -39897 -39898 -39899 -39900 -39901 -39902 -39903 -39904 -39905 -39906 -39907 -39908 -39909 -39910 -39911 -39912 -39913 -39914 -39915 -39916 -39917 -39918 -39919 -39920 -39921 -39922 -39923 -39924 -39925 -39926 -39927 -39928 -39929 -39930 -39931 -39932 -39933 -39934 -39935 -39936 -39937 -39938 -39939 -39940 -39941 -39942 -39943 -39944 -39945 -39946 -39947 -39948 -39949 -39950 -39951 -39952 -39953 -39954 -39955 -39956 -39957 -39958 -39959 -39960 -39961 -39962 -39963 -39964 -39965 -39966 -39967 -39968 -39969 -39970 -39971 -39972 -39973 -39974 -39975 -39976 -39977 -39978 -39979 -39980 -39981 -39982 -39983 -39984 -39985 -39986 -39987 -39988 -39989 -39990 -39991 -39992 -39993 -39994 -39995 -39996 -39997 -39998 -39999 -40000 -40001 -40002 -40003 -40004 -40005 -40006 -40007 -40008 -40009 -40010 -40011 -40012 -40013 -40014 -40015 -40016 -40017 -40018 -40019 -40020 -40021 -40022 -40023 -40024 -40025 -40026 -40027 -40028 -40029 -40030 -40031 -40032 -40033 -40034 -40035 -40036 -40037 -40038 -40039 -40040 -40041 -40042 -40043 -40044 -40045 -40046 -40047 -40048 -40049 -40050 -40051 -40052 -40053 -40054 -40055 -40056 -40057 -40058 -40059 -40060 -40061 -40062 -40063 -40064 -40065 -40066 -40067 -40068 -40069 -40070 -40071 -40072 -40073 -40074 -40075 -40076 -40077 -40078 -40079 -40080 -40081 -40082 -40083 -40084 -40085 -40086 -40087 -40088 -40089 -40090 -40091 -40092 -40093 -40094 -40095 -40096 -40097 -40098 -40099 -40100 -40101 -40102 -40103 -40104 -40105 -40106 -40107 -40108 -40109 -40110 -40111 -40112 -40113 -40114 -40115 -40116 -40117 -40118 -40119 -40120 -40121 -40122 -40123 -40124 -40125 -40126 -40127 -40128 -40129 -40130 -40131 -40132 -40133 -40134 -40135 -40136 -40137 -40138 -40139 -40140 -40141 -40142 -40143 -40144 -40145 -40146 -40147 -40148 -40149 -40150 -40151 -40152 -40153 -40154 -40155 -40156 -40157 -40158 -40159 -40160 -40161 -40162 -40163 -40164 -40165 -40166 -40167 -40168 -40169 -40170 -40171 -40172 -40173 -40174 -40175 -40176 -40177 -40178 -40179 -40180 -40181 -40182 -40183 -40184 -40185 -40186 -40187 -40188 -40189 -40190 -40191 -40192 -40193 -40194 -40195 -40196 -40197 -40198 -40199 -40200 -40201 -40202 -40203 -40204 -40205 -40206 -40207 -40208 -40209 -40210 -40211 -40212 -40213 -40214 -40215 -40216 -40217 -40218 -40219 -40220 -40221 -40222 -40223 -40224 -40225 -40226 -40227 -40228 -40229 -40230 -40231 -40232 -40233 -40234 -40235 -40236 -40237 -40238 -40239 -40240 -40241 -40242 -40243 -40244 -40245 -40246 -40247 -40248 -40249 -40250 -40251 -40252 -40253 -40254 -40255 -40256 -40257 -40258 -40259 -40260 -40261 -40262 -40263 -40264 -40265 -40266 -40267 -40268 -40269 -40270 -40271 -40272 -40273 -40274 -40275 -40276 -40277 -40278 -40279 -40280 -40281 -40282 -40283 -40284 -40285 -40286 -40287 -40288 -40289 -40290 -40291 -40292 -40293 -40294 -40295 -40296 -40297 -40298 -40299 -40300 -40301 -40302 -40303 -40304 -40305 -40306 -40307 -40308 -40309 -40310 -40311 -40312 -40313 -40314 -40315 -40316 -40317 -40318 -40319 -40320 -40321 -40322 -40323 -40324 -40325 -40326 -40327 -40328 -40329 -40330 -40331 -40332 -40333 -40334 -40335 -40336 -40337 -40338 -40339 -40340 -40341 -40342 -40343 -40344 -40345 -40346 -40347 -40348 -40349 -40350 -40351 -40352 -40353 -40354 -40355 -40356 -40357 -40358 -40359 -40360 -40361 -40362 -40363 -40364 -40365 -40366 -40367 -40368 -40369 -40370 -40371 -40372 -40373 -40374 -40375 -40376 -40377 -40378 -40379 -40380 -40381 -40382 -40383 -40384 -40385 -40386 -40387 -40388 -40389 -40390 -40391 -40392 -40393 -40394 -40395 -40396 -40397 -40398 -40399 -40400 -40401 -40402 -40403 -40404 -40405 -40406 -40407 -40408 -40409 -40410 -40411 -40412 -40413 -40414 -40415 -40416 -40417 -40418 -40419 -40420 -40421 -40422 -40423 -40424 -40425 -40426 -40427 -40428 -40429 -40430 -40431 -40432 -40433 -40434 -40435 -40436 -40437 -40438 -40439 -40440 -40441 -40442 -40443 -40444 -40445 -40446 -40447 -40448 -40449 -40450 -40451 -40452 -40453 -40454 -40455 -40456 -40457 -40458 -40459 -40460 -40461 -40462 -40463 -40464 -40465 -40466 -40467 -40468 -40469 -40470 -40471 -40472 -40473 -40474 -40475 -40476 -40477 -40478 -40479 -40480 -40481 -40482 -40483 -40484 -40485 -40486 -40487 -40488 -40489 -40490 -40491 -40492 -40493 -40494 -40495 -40496 -40497 -40498 -40499 -40500 -40501 -40502 -40503 -40504 -40505 -40506 -40507 -40508 -40509 -40510 -40511 -40512 -40513 -40514 -40515 -40516 -40517 -40518 -40519 -40520 -40521 -40522 -40523 -40524 -40525 -40526 -40527 -40528 -40529 -40530 -40531 -40532 -40533 -40534 -40535 -40536 -40537 -40538 -40539 -40540 -40541 -40542 -40543 -40544 -40545 -40546 -40547 -40548 -40549 -40550 -40551 -40552 -40553 -40554 -40555 -40556 -40557 -40558 -40559 -40560 -40561 -40562 -40563 -40564 -40565 -40566 -40567 -40568 -40569 -40570 -40571 -40572 -40573 -40574 -40575 -40576 -40577 -40578 -40579 -40580 -40581 -40582 -40583 -40584 -40585 -40586 -40587 -40588 -40589 -40590 -40591 -40592 -40593 -40594 -40595 -40596 -40597 -40598 -40599 -40600 -40601 -40602 -40603 -40604 -40605 -40606 -40607 -40608 -40609 -40610 -40611 -40612 -40613 -40614 -40615 -40616 -40617 -40618 -40619 -40620 -40621 -40622 -40623 -40624 -40625 -40626 -40627 -40628 -40629 -40630 -40631 -40632 -40633 -40634 -40635 -40636 -40637 -40638 -40639 -40640 -40641 -40642 -40643 -40644 -40645 -40646 -40647 -40648 -40649 -40650 -40651 -40652 -40653 -40654 -40655 -40656 -40657 -40658 -40659 -40660 -40661 -40662 -40663 -40664 -40665 -40666 -40667 -40668 -40669 -40670 -40671 -40672 -40673 -40674 -40675 -40676 -40677 -40678 -40679 -40680 -40681 -40682 -40683 -40684 -40685 -40686 -40687 -40688 -40689 -40690 -40691 -40692 -40693 -40694 -40695 -40696 -40697 -40698 -40699 -40700 -40701 -40702 -40703 -40704 -40705 -40706 -40707 -40708 -40709 -40710 -40711 -40712 -40713 -40714 -40715 -40716 -40717 -40718 -40719 -40720 -40721 -40722 -40723 -40724 -40725 -40726 -40727 -40728 -40729 -40730 -40731 -40732 -40733 -40734 -40735 -40736 -40737 -40738 -40739 -40740 -40741 -40742 -40743 -40744 -40745 -40746 -40747 -40748 -40749 -40750 -40751 -40752 -40753 -40754 -40755 -40756 -40757 -40758 -40759 -40760 -40761 -40762 -40763 -40764 -40765 -40766 -40767 -40768 -40769 -40770 -40771 -40772 -40773 -40774 -40775 -40776 -40777 -40778 -40779 -40780 -40781 -40782 -40783 -40784 -40785 -40786 -40787 -40788 -40789 -40790 -40791 -40792 -40793 -40794 -40795 -40796 -40797 -40798 -40799 -40800 -40801 -40802 -40803 -40804 -40805 -40806 -40807 -40808 -40809 -40810 -40811 -40812 -40813 -40814 -40815 -40816 -40817 -40818 -40819 -40820 -40821 -40822 -40823 -40824 -40825 -40826 -40827 -40828 -40829 -40830 -40831 -40832 -40833 -40834 -40835 -40836 -40837 -40838 -40839 -40840 -40841 -40842 -40843 -40844 -40845 -40846 -40847 -40848 -40849 -40850 -40851 -40852 -40853 -40854 -40855 -40856 -40857 -40858 -40859 -40860 -40861 -40862 -40863 -40864 -40865 -40866 -40867 -40868 -40869 -40870 -40871 -40872 -40873 -40874 -40875 -40876 -40877 -40878 -40879 -40880 -40881 -40882 -40883 -40884 -40885 -40886 -40887 -40888 -40889 -40890 -40891 -40892 -40893 -40894 -40895 -40896 -40897 -40898 -40899 -40900 -40901 -40902 -40903 -40904 -40905 -40906 -40907 -40908 -40909 -40910 -40911 -40912 -40913 -40914 -40915 -40916 -40917 -40918 -40919 -40920 -40921 -40922 -40923 -40924 -40925 -40926 -40927 -40928 -40929 -40930 -40931 -40932 -40933 -40934 -40935 -40936 -40937 -40938 -40939 -40940 -40941 -40942 -40943 -40944 -40945 -40946 -40947 -40948 -40949 -40950 -40951 -40952 -40953 -40954 -40955 -40956 -40957 -40958 -40959 -40960 -40961 -40962 -40963 -40964 -40965 -40966 -40967 -40968 -40969 -40970 -40971 -40972 -40973 -40974 -40975 -40976 -40977 -40978 -40979 -40980 -40981 -40982 -40983 -40984 -40985 -40986 -40987 -40988 -40989 -40990 -40991 -40992 -40993 -40994 -40995 -40996 -40997 -40998 -40999 -41000 -41001 -41002 -41003 -41004 -41005 -41006 -41007 -41008 -41009 -41010 -41011 -41012 -41013 -41014 -41015 -41016 -41017 -41018 -41019 -41020 -41021 -41022 -41023 -41024 -41025 -41026 -41027 -41028 -41029 -41030 -41031 -41032 -41033 -41034 -41035 -41036 -41037 -41038 -41039 -41040 -41041 -41042 -41043 -41044 -41045 -41046 -41047 -41048 -41049 -41050 -41051 -41052 -41053 -41054 -41055 -41056 -41057 -41058 -41059 -41060 -41061 -41062 -41063 -41064 -41065 -41066 -41067 -41068 -41069 -41070 -41071 -41072 -41073 -41074 -41075 -41076 -41077 -41078 -41079 -41080 -41081 -41082 -41083 -41084 -41085 -41086 -41087 -41088 -41089 -41090 -41091 -41092 -41093 -41094 -41095 -41096 -41097 -41098 -41099 -41100 -41101 -41102 -41103 -41104 -41105 -41106 -41107 -41108 -41109 -41110 -41111 -41112 -41113 -41114 -41115 -41116 -41117 -41118 -41119 -41120 -41121 -41122 -41123 -41124 -41125 -41126 -41127 -41128 -41129 -41130 -41131 -41132 -41133 -41134 -41135 -41136 -41137 -41138 -41139 -41140 -41141 -41142 -41143 -41144 -41145 -41146 -41147 -41148 -41149 -41150 -41151 -41152 -41153 -41154 -41155 -41156 -41157 -41158 -41159 -41160 -41161 -41162 -41163 -41164 -41165 -41166 -41167 -41168 -41169 -41170 -41171 -41172 -41173 -41174 -41175 -41176 -41177 -41178 -41179 -41180 -41181 -41182 -41183 -41184 -41185 -41186 -41187 -41188 -41189 -41190 -41191 -41192 -41193 -41194 -41195 -41196 -41197 -41198 -41199 -41200 -41201 -41202 -41203 -41204 -41205 -41206 -41207 -41208 -41209 -41210 -41211 -41212 -41213 -41214 -41215 -41216 -41217 -41218 -41219 -41220 -41221 -41222 -41223 -41224 -41225 -41226 -41227 -41228 -41229 -41230 -41231 -41232 -41233 -41234 -41235 -41236 -41237 -41238 -41239 -41240 -41241 -41242 -41243 -41244 -41245 -41246 -41247 -41248 -41249 -41250 -41251 -41252 -41253 -41254 -41255 -41256 -41257 -41258 -41259 -41260 -41261 -41262 -41263 -41264 -41265 -41266 -41267 -41268 -41269 -41270 -41271 -41272 -41273 -41274 -41275 -41276 -41277 -41278 -41279 -41280 -41281 -41282 -41283 -41284 -41285 -41286 -41287 -41288 -41289 -41290 -41291 -41292 -41293 -41294 -41295 -41296 -41297 -41298 -41299 -41300 -41301 -41302 -41303 -41304 -41305 -41306 -41307 -41308 -41309 -41310 -41311 -41312 -41313 -41314 -41315 -41316 -41317 -41318 -41319 -41320 -41321 -41322 -41323 -41324 -41325 -41326 -41327 -41328 -41329 -41330 -41331 -41332 -41333 -41334 -41335 -41336 -41337 -41338 -41339 -41340 -41341 -41342 -41343 -41344 -41345 -41346 -41347 -41348 -41349 -41350 -41351 -41352 -41353 -41354 -41355 -41356 -41357 -41358 -41359 -41360 -41361 -41362 -41363 -41364 -41365 -41366 -41367 -41368 -41369 -41370 -41371 -41372 -41373 -41374 -41375 -41376 -41377 -41378 -41379 -41380 -41381 -41382 -41383 -41384 -41385 -41386 -41387 -41388 -41389 -41390 -41391 -41392 -41393 -41394 -41395 -41396 -41397 -41398 -41399 -41400 -41401 -41402 -41403 -41404 -41405 -41406 -41407 -41408 -41409 -41410 -41411 -41412 -41413 -41414 -41415 -41416 -41417 -41418 -41419 -41420 -41421 -41422 -41423 -41424 -41425 -41426 -41427 -41428 -41429 -41430 -41431 -41432 -41433 -41434 -41435 -41436 -41437 -41438 -41439 -41440 -41441 -41442 -41443 -41444 -41445 -41446 -41447 -41448 -41449 -41450 -41451 -41452 -41453 -41454 -41455 -41456 -41457 -41458 -41459 -41460 -41461 -41462 -41463 -41464 -41465 -41466 -41467 -41468 -41469 -41470 -41471 -41472 -41473 -41474 -41475 -41476 -41477 -41478 -41479 -41480 -41481 -41482 -41483 -41484 -41485 -41486 -41487 -41488 -41489 -41490 -41491 -41492 -41493 -41494 -41495 -41496 -41497 -41498 -41499 -41500 -41501 -41502 -41503 -41504 -41505 -41506 -41507 -41508 -41509 -41510 -41511 -41512 -41513 -41514 -41515 -41516 -41517 -41518 -41519 -41520 -41521 -41522 -41523 -41524 -41525 -41526 -41527 -41528 -41529 -41530 -41531 -41532 -41533 -41534 -41535 -41536 -41537 -41538 -41539 -41540 -41541 -41542 -41543 -41544 -41545 -41546 -41547 -41548 -41549 -41550 -41551 -41552 -41553 -41554 -41555 -41556 -41557 -41558 -41559 -41560 -41561 -41562 -41563 -41564 -41565 -41566 -41567 -41568 -41569 -41570 -41571 -41572 -41573 -41574 -41575 -41576 -41577 -41578 -41579 -41580 -41581 -41582 -41583 -41584 -41585 -41586 -41587 -41588 -41589 -41590 -41591 -41592 -41593 -41594 -41595 -41596 -41597 -41598 -41599 -41600 -41601 -41602 -41603 -41604 -41605 -41606 -41607 -41608 -41609 -41610 -41611 -41612 -41613 -41614 -41615 -41616 -41617 -41618 -41619 -41620 -41621 -41622 -41623 -41624 -41625 -41626 -41627 -41628 -41629 -41630 -41631 -41632 -41633 -41634 -41635 -41636 -41637 -41638 -41639 -41640 -41641 -41642 -41643 -41644 -41645 -41646 -41647 -41648 -41649 -41650 -41651 -41652 -41653 -41654 -41655 -41656 -41657 -41658 -41659 -41660 -41661 -41662 -41663 -41664 -41665 -41666 -41667 -41668 -41669 -41670 -41671 -41672 -41673 -41674 -41675 -41676 -41677 -41678 -41679 -41680 -41681 -41682 -41683 -41684 -41685 -41686 -41687 -41688 -41689 -41690 -41691 -41692 -41693 -41694 -41695 -41696 -41697 -41698 -41699 -41700 -41701 -41702 -41703 -41704 -41705 -41706 -41707 -41708 -41709 -41710 -41711 -41712 -41713 -41714 -41715 -41716 -41717 -41718 -41719 -41720 -41721 -41722 -41723 -41724 -41725 -41726 -41727 -41728 -41729 -41730 -41731 -41732 -41733 -41734 -41735 -41736 -41737 -41738 -41739 -41740 -41741 -41742 -41743 -41744 -41745 -41746 -41747 -41748 -41749 -41750 -41751 -41752 -41753 -41754 -41755 -41756 -41757 -41758 -41759 -41760 -41761 -41762 -41763 -41764 -41765 -41766 -41767 -41768 -41769 -41770 -41771 -41772 -41773 -41774 -41775 -41776 -41777 -41778 -41779 -41780 -41781 -41782 -41783 -41784 -41785 -41786 -41787 -41788 -41789 -41790 -41791 -41792 -41793 -41794 -41795 -41796 -41797 -41798 -41799 -41800 -41801 -41802 -41803 -41804 -41805 -41806 -41807 -41808 -41809 -41810 -41811 -41812 -41813 -41814 -41815 -41816 -41817 -41818 -41819 -41820 -41821 -41822 -41823 -41824 -41825 -41826 -41827 -41828 -41829 -41830 -41831 -41832 -41833 -41834 -41835 -41836 -41837 -41838 -41839 -41840 -41841 -41842 -41843 -41844 -41845 -41846 -41847 -41848 -41849 -41850 -41851 -41852 -41853 -41854 -41855 -41856 -41857 -41858 -41859 -41860 -41861 -41862 -41863 -41864 -41865 -41866 -41867 -41868 -41869 -41870 -41871 -41872 -41873 -41874 -41875 -41876 -41877 -41878 -41879 -41880 -41881 -41882 -41883 -41884 -41885 -41886 -41887 -41888 -41889 -41890 -41891 -41892 -41893 -41894 -41895 -41896 -41897 -41898 -41899 -41900 -41901 -41902 -41903 -41904 -41905 -41906 -41907 -41908 -41909 -41910 -41911 -41912 -41913 -41914 -41915 -41916 -41917 -41918 -41919 -41920 -41921 -41922 -41923 -41924 -41925 -41926 -41927 -41928 -41929 -41930 -41931 -41932 -41933 -41934 -41935 -41936 -41937 -41938 -41939 -41940 -41941 -41942 -41943 -41944 -41945 -41946 -41947 -41948 -41949 -41950 -41951 -41952 -41953 -41954 -41955 -41956 -41957 -41958 -41959 -41960 -41961 -41962 -41963 -41964 -41965 -41966 -41967 -41968 -41969 -41970 -41971 -41972 -41973 -41974 -41975 -41976 -41977 -41978 -41979 -41980 -41981 -41982 -41983 -41984 -41985 -41986 -41987 -41988 -41989 -41990 -41991 -41992 -41993 -41994 -41995 -41996 -41997 -41998 -41999 -42000 -42001 -42002 -42003 -42004 -42005 -42006 -42007 -42008 -42009 -42010 -42011 -42012 -42013 -42014 -42015 -42016 -42017 -42018 -42019 -42020 -42021 -42022 -42023 -42024 -42025 -42026 -42027 -42028 -42029 -42030 -42031 -42032 -42033 -42034 -42035 -42036 -42037 -42038 -42039 -42040 -42041 -42042 -42043 -42044 -42045 -42046 -42047 -42048 -42049 -42050 -42051 -42052 -42053 -42054 -42055 -42056 -42057 -42058 -42059 -42060 -42061 -42062 -42063 -42064 -42065 -42066 -42067 -42068 -42069 -42070 -42071 -42072 -42073 -42074 -42075 -42076 -42077 -42078 -42079 -42080 -42081 -42082 -42083 -42084 -42085 -42086 -42087 -42088 -42089 -42090 -42091 -42092 -42093 -42094 -42095 -42096 -42097 -42098 -42099 -42100 -42101 -42102 -42103 -42104 -42105 -42106 -42107 -42108 -42109 -42110 -42111 -42112 -42113 -42114 -42115 -42116 -42117 -42118 -42119 -42120 -42121 -42122 -42123 -42124 -42125 -42126 -42127 -42128 -42129 -42130 -42131 -42132 -42133 -42134 -42135 -42136 -42137 -42138 -42139 -42140 -42141 -42142 -42143 -42144 -42145 -42146 -42147 -42148 -42149 -42150 -42151 -42152 -42153 -42154 -42155 -42156 -42157 -42158 -42159 -42160 -42161 -42162 -42163 -42164 -42165 -42166 -42167 -42168 -42169 -42170 -42171 -42172 -42173 -42174 -42175 -42176 -42177 -42178 -42179 -42180 -42181 -42182 -42183 -42184 -42185 -42186 -42187 -42188 -42189 -42190 -42191 -42192 -42193 -42194 -42195 -42196 -42197 -42198 -42199 -42200 -42201 -42202 -42203 -42204 -42205 -42206 -42207 -42208 -42209 -42210 -42211 -42212 -42213 -42214 -42215 -42216 -42217 -42218 -42219 -42220 -42221 -42222 -42223 -42224 -42225 -42226 -42227 -42228 -42229 -42230 -42231 -42232 -42233 -42234 -42235 -42236 -42237 -42238 -42239 -42240 -42241 -42242 -42243 -42244 -42245 -42246 -42247 -42248 -42249 -42250 -42251 -42252 -42253 -42254 -42255 -42256 -42257 -42258 -42259 -42260 -42261 -42262 -42263 -42264 -42265 -42266 -42267 -42268 -42269 -42270 -42271 -42272 -42273 -42274 -42275 -42276 -42277 -42278 -42279 -42280 -42281 -42282 -42283 -42284 -42285 -42286 -42287 -42288 -42289 -42290 -42291 -42292 -42293 -42294 -42295 -42296 -42297 -42298 -42299 -42300 -42301 -42302 -42303 -42304 -42305 -42306 -42307 -42308 -42309 -42310 -42311 -42312 -42313 -42314 -42315 -42316 -42317 -42318 -42319 -42320 -42321 -42322 -42323 -42324 -42325 -42326 -42327 -42328 -42329 -42330 -42331 -42332 -42333 -42334 -42335 -42336 -42337 -42338 -42339 -42340 -42341 -42342 -42343 -42344 -42345 -42346 -42347 -42348 -42349 -42350 -42351 -42352 -42353 -42354 -42355 -42356 -42357 -42358 -42359 -42360 -42361 -42362 -42363 -42364 -42365 -42366 -42367 -42368 -42369 -42370 -42371 -42372 -42373 -42374 -42375 -42376 -42377 -42378 -42379 -42380 -42381 -42382 -42383 -42384 -42385 -42386 -42387 -42388 -42389 -42390 -42391 -42392 -42393 -42394 -42395 -42396 -42397 -42398 -42399 -42400 -42401 -42402 -42403 -42404 -42405 -42406 -42407 -42408 -42409 -42410 -42411 -42412 -42413 -42414 -42415 -42416 -42417 -42418 -42419 -42420 -42421 -42422 -42423 -42424 -42425 -42426 -42427 -42428 -42429 -42430 -42431 -42432 -42433 -42434 -42435 -42436 -42437 -42438 -42439 -42440 -42441 -42442 -42443 -42444 -42445 -42446 -42447 -42448 -42449 -42450 -42451 -42452 -42453 -42454 -42455 -42456 -42457 -42458 -42459 -42460 -42461 -42462 -42463 -42464 -42465 -42466 -42467 -42468 -42469 -42470 -42471 -42472 -42473 -42474 -42475 -42476 -42477 -42478 -42479 -42480 -42481 -42482 -42483 -42484 -42485 -42486 -42487 -42488 -42489 -42490 -42491 -42492 -42493 -42494 -42495 -42496 -42497 -42498 -42499 -42500 -42501 -42502 -42503 -42504 -42505 -42506 -42507 -42508 -42509 -42510 -42511 -42512 -42513 -42514 -42515 -42516 -42517 -42518 -42519 -42520 -42521 -42522 -42523 -42524 -42525 -42526 -42527 -42528 -42529 -42530 -42531 -42532 -42533 -42534 -42535 -42536 -42537 -42538 -42539 -42540 -42541 -42542 -42543 -42544 -42545 -42546 -42547 -42548 -42549 -42550 -42551 -42552 -42553 -42554 -42555 -42556 -42557 -42558 -42559 -42560 -42561 -42562 -42563 -42564 -42565 -42566 -42567 -42568 -42569 -42570 -42571 -42572 -42573 -42574 -42575 -42576 -42577 -42578 -42579 -42580 -42581 -42582 -42583 -42584 -42585 -42586 -42587 -42588 -42589 -42590 -42591 -42592 -42593 -42594 -42595 -42596 -42597 -42598 -42599 -42600 -42601 -42602 -42603 -42604 -42605 -42606 -42607 -42608 -42609 -42610 -42611 -42612 -42613 -42614 -42615 -42616 -42617 -42618 -42619 -42620 -42621 -42622 -42623 -42624 -42625 -42626 -42627 -42628 -42629 -42630 -42631 -42632 -42633 -42634 -42635 -42636 -42637 -42638 -42639 -42640 -42641 -42642 -42643 -42644 -42645 -42646 -42647 -42648 -42649 -42650 -42651 -42652 -42653 -42654 -42655 -42656 -42657 -42658 -42659 -42660 -42661 -42662 -42663 -42664 -42665 -42666 -42667 -42668 -42669 -42670 -42671 -42672 -42673 -42674 -42675 -42676 -42677 -42678 -42679 -42680 -42681 -42682 -42683 -42684 -42685 -42686 -42687 -42688 -42689 -42690 -42691 -42692 -42693 -42694 -42695 -42696 -42697 -42698 -42699 -42700 -42701 -42702 -42703 -42704 -42705 -42706 -42707 -42708 -42709 -42710 -42711 -42712 -42713 -42714 -42715 -42716 -42717 -42718 -42719 -42720 -42721 -42722 -42723 -42724 -42725 -42726 -42727 -42728 -42729 -42730 -42731 -42732 -42733 -42734 -42735 -42736 -42737 -42738 -42739 -42740 -42741 -42742 -42743 -42744 -42745 -42746 -42747 -42748 -42749 -42750 -42751 -42752 -42753 -42754 -42755 -42756 -42757 -42758 -42759 -42760 -42761 -42762 -42763 -42764 -42765 -42766 -42767 -42768 -42769 -42770 -42771 -42772 -42773 -42774 -42775 -42776 -42777 -42778 -42779 -42780 -42781 -42782 -42783 -42784 -42785 -42786 -42787 -42788 -42789 -42790 -42791 -42792 -42793 -42794 -42795 -42796 -42797 -42798 -42799 -42800 -42801 -42802 -42803 -42804 -42805 -42806 -42807 -42808 -42809 -42810 -42811 -42812 -42813 -42814 -42815 -42816 -42817 -42818 -42819 -42820 -42821 -42822 -42823 -42824 -42825 -42826 -42827 -42828 -42829 -42830 -42831 -42832 -42833 -42834 -42835 -42836 -42837 -42838 -42839 -42840 -42841 -42842 -42843 -42844 -42845 -42846 -42847 -42848 -42849 -42850 -42851 -42852 -42853 -42854 -42855 -42856 -42857 -42858 -42859 -42860 -42861 -42862 -42863 -42864 -42865 -42866 -42867 -42868 -42869 -42870 -42871 -42872 -42873 -42874 -42875 -42876 -42877 -42878 -42879 -42880 -42881 -42882 -42883 -42884 -42885 -42886 -42887 -42888 -42889 -42890 -42891 -42892 -42893 -42894 -42895 -42896 -42897 -42898 -42899 -42900 -42901 -42902 -42903 -42904 -42905 -42906 -42907 -42908 -42909 -42910 -42911 -42912 -42913 -42914 -42915 -42916 -42917 -42918 -42919 -42920 -42921 -42922 -42923 -42924 -42925 -42926 -42927 -42928 -42929 -42930 -42931 -42932 -42933 -42934 -42935 -42936 -42937 -42938 -42939 -42940 -42941 -42942 -42943 -42944 -42945 -42946 -42947 -42948 -42949 -42950 -42951 -42952 -42953 -42954 -42955 -42956 -42957 -42958 -42959 -42960 -42961 -42962 -42963 -42964 -42965 -42966 -42967 -42968 -42969 -42970 -42971 -42972 -42973 -42974 -42975 -42976 -42977 -42978 -42979 -42980 -42981 -42982 -42983 -42984 -42985 -42986 -42987 -42988 -42989 -42990 -42991 -42992 -42993 -42994 -42995 -42996 -42997 -42998 -42999 -43000 -43001 -43002 -43003 -43004 -43005 -43006 -43007 -43008 -43009 -43010 -43011 -43012 -43013 -43014 -43015 -43016 -43017 -43018 -43019 -43020 -43021 -43022 -43023 -43024 -43025 -43026 -43027 -43028 -43029 -43030 -43031 -43032 -43033 -43034 -43035 -43036 -43037 -43038 -43039 -43040 -43041 -43042 -43043 -43044 -43045 -43046 -43047 -43048 -43049 -43050 -43051 -43052 -43053 -43054 -43055 -43056 -43057 -43058 -43059 -43060 -43061 -43062 -43063 -43064 -43065 -43066 -43067 -43068 -43069 -43070 -43071 -43072 -43073 -43074 -43075 -43076 -43077 -43078 -43079 -43080 -43081 -43082 -43083 -43084 -43085 -43086 -43087 -43088 -43089 -43090 -43091 -43092 -43093 -43094 -43095 -43096 -43097 -43098 -43099 -43100 -43101 -43102 -43103 -43104 -43105 -43106 -43107 -43108 -43109 -43110 -43111 -43112 -43113 -43114 -43115 -43116 -43117 -43118 -43119 -43120 -43121 -43122 -43123 -43124 -43125 -43126 -43127 -43128 -43129 -43130 -43131 -43132 -43133 -43134 -43135 -43136 -43137 -43138 -43139 -43140 -43141 -43142 -43143 -43144 -43145 -43146 -43147 -43148 -43149 -43150 -43151 -43152 -43153 -43154 -43155 -43156 -43157 -43158 -43159 -43160 -43161 -43162 -43163 -43164 -43165 -43166 -43167 -43168 -43169 -43170 -43171 -43172 -43173 -43174 -43175 -43176 -43177 -43178 -43179 -43180 -43181 -43182 -43183 -43184 -43185 -43186 -43187 -43188 -43189 -43190 -43191 -43192 -43193 -43194 -43195 -43196 -43197 -43198 -43199 -43200 -43201 -43202 -43203 -43204 -43205 -43206 -43207 -43208 -43209 -43210 -43211 -43212 -43213 -43214 -43215 -43216 -43217 -43218 -43219 -43220 -43221 -43222 -43223 -43224 -43225 -43226 -43227 -43228 -43229 -43230 -43231 -43232 -43233 -43234 -43235 -43236 -43237 -43238 -43239 -43240 -43241 -43242 -43243 -43244 -43245 -43246 -43247 -43248 -43249 -43250 -43251 -43252 -43253 -43254 -43255 -43256 -43257 -43258 -43259 -43260 -43261 -43262 -43263 -43264 -43265 -43266 -43267 -43268 -43269 -43270 -43271 -43272 -43273 -43274 -43275 -43276 -43277 -43278 -43279 -43280 -43281 -43282 -43283 -43284 -43285 -43286 -43287 -43288 -43289 -43290 -43291 -43292 -43293 -43294 -43295 -43296 -43297 -43298 -43299 -43300 -43301 -43302 -43303 -43304 -43305 -43306 -43307 -43308 -43309 -43310 -43311 -43312 -43313 -43314 -43315 -43316 -43317 -43318 -43319 -43320 -43321 -43322 -43323 -43324 -43325 -43326 -43327 -43328 -43329 -43330 -43331 -43332 -43333 -43334 -43335 -43336 -43337 -43338 -43339 -43340 -43341 -43342 -43343 -43344 -43345 -43346 -43347 -43348 -43349 -43350 -43351 -43352 -43353 -43354 -43355 -43356 -43357 -43358 -43359 -43360 -43361 -43362 -43363 -43364 -43365 -43366 -43367 -43368 -43369 -43370 -43371 -43372 -43373 -43374 -43375 -43376 -43377 -43378 -43379 -43380 -43381 -43382 -43383 -43384 -43385 -43386 -43387 -43388 -43389 -43390 -43391 -43392 -43393 -43394 -43395 -43396 -43397 -43398 -43399 -43400 -43401 -43402 -43403 -43404 -43405 -43406 -43407 -43408 -43409 -43410 -43411 -43412 -43413 -43414 -43415 -43416 -43417 -43418 -43419 -43420 -43421 -43422 -43423 -43424 -43425 -43426 -43427 -43428 -43429 -43430 -43431 -43432 -43433 -43434 -43435 -43436 -43437 -43438 -43439 -43440 -43441 -43442 -43443 -43444 -43445 -43446 -43447 -43448 -43449 -43450 -43451 -43452 -43453 -43454 -43455 -43456 -43457 -43458 -43459 -43460 -43461 -43462 -43463 -43464 -43465 -43466 -43467 -43468 -43469 -43470 -43471 -43472 -43473 -43474 -43475 -43476 -43477 -43478 -43479 -43480 -43481 -43482 -43483 -43484 -43485 -43486 -43487 -43488 -43489 -43490 -43491 -43492 -43493 -43494 -43495 -43496 -43497 -43498 -43499 -43500 -43501 -43502 -43503 -43504 -43505 -43506 -43507 -43508 -43509 -43510 -43511 -43512 -43513 -43514 -43515 -43516 -43517 -43518 -43519 -43520 -43521 -43522 -43523 -43524 -43525 -43526 -43527 -43528 -43529 -43530 -43531 -43532 -43533 -43534 -43535 -43536 -43537 -43538 -43539 -43540 -43541 -43542 -43543 -43544 -43545 -43546 -43547 -43548 -43549 -43550 -43551 -43552 -43553 -43554 -43555 -43556 -43557 -43558 -43559 -43560 -43561 -43562 -43563 -43564 -43565 -43566 -43567 -43568 -43569 -43570 -43571 -43572 -43573 -43574 -43575 -43576 -43577 -43578 -43579 -43580 -43581 -43582 -43583 -43584 -43585 -43586 -43587 -43588 -43589 -43590 -43591 -43592 -43593 -43594 -43595 -43596 -43597 -43598 -43599 -43600 -43601 -43602 -43603 -43604 -43605 -43606 -43607 -43608 -43609 -43610 -43611 -43612 -43613 -43614 -43615 -43616 -43617 -43618 -43619 -43620 -43621 -43622 -43623 -43624 -43625 -43626 -43627 -43628 -43629 -43630 -43631 -43632 -43633 -43634 -43635 -43636 -43637 -43638 -43639 -43640 -43641 -43642 -43643 -43644 -43645 -43646 -43647 -43648 -43649 -43650 -43651 -43652 -43653 -43654 -43655 -43656 -43657 -43658 -43659 -43660 -43661 -43662 -43663 -43664 -43665 -43666 -43667 -43668 -43669 -43670 -43671 -43672 -43673 -43674 -43675 -43676 -43677 -43678 -43679 -43680 -43681 -43682 -43683 -43684 -43685 -43686 -43687 -43688 -43689 -43690 -43691 -43692 -43693 -43694 -43695 -43696 -43697 -43698 -43699 -43700 -43701 -43702 -43703 -43704 -43705 -43706 -43707 -43708 -43709 -43710 -43711 -43712 -43713 -43714 -43715 -43716 -43717 -43718 -43719 -43720 -43721 -43722 -43723 -43724 -43725 -43726 -43727 -43728 -43729 -43730 -43731 -43732 -43733 -43734 -43735 -43736 -43737 -43738 -43739 -43740 -43741 -43742 -43743 -43744 -43745 -43746 -43747 -43748 -43749 -43750 -43751 -43752 -43753 -43754 -43755 -43756 -43757 -43758 -43759 -43760 -43761 -43762 -43763 -43764 -43765 -43766 -43767 -43768 -43769 -43770 -43771 -43772 -43773 -43774 -43775 -43776 -43777 -43778 -43779 -43780 -43781 -43782 -43783 -43784 -43785 -43786 -43787 -43788 -43789 -43790 -43791 -43792 -43793 -43794 -43795 -43796 -43797 -43798 -43799 -43800 -43801 -43802 -43803 -43804 -43805 -43806 -43807 -43808 -43809 -43810 -43811 -43812 -43813 -43814 -43815 -43816 -43817 -43818 -43819 -43820 -43821 -43822 -43823 -43824 -43825 -43826 -43827 -43828 -43829 -43830 -43831 -43832 -43833 -43834 -43835 -43836 -43837 -43838 -43839 -43840 -43841 -43842 -43843 -43844 -43845 -43846 -43847 -43848 -43849 -43850 -43851 -43852 -43853 -43854 -43855 -43856 -43857 -43858 -43859 -43860 -43861 -43862 -43863 -43864 -43865 -43866 -43867 -43868 -43869 -43870 -43871 -43872 -43873 -43874 -43875 -43876 -43877 -43878 -43879 -43880 -43881 -43882 -43883 -43884 -43885 -43886 -43887 -43888 -43889 -43890 -43891 -43892 -43893 -43894 -43895 -43896 -43897 -43898 -43899 -43900 -43901 -43902 -43903 -43904 -43905 -43906 -43907 -43908 -43909 -43910 -43911 -43912 -43913 -43914 -43915 -43916 -43917 -43918 -43919 -43920 -43921 -43922 -43923 -43924 -43925 -43926 -43927 -43928 -43929 -43930 -43931 -43932 -43933 -43934 -43935 -43936 -43937 -43938 -43939 -43940 -43941 -43942 -43943 -43944 -43945 -43946 -43947 -43948 -43949 -43950 -43951 -43952 -43953 -43954 -43955 -43956 -43957 -43958 -43959 -43960 -43961 -43962 -43963 -43964 -43965 -43966 -43967 -43968 -43969 -43970 -43971 -43972 -43973 -43974 -43975 -43976 -43977 -43978 -43979 -43980 -43981 -43982 -43983 -43984 -43985 -43986 -43987 -43988 -43989 -43990 -43991 -43992 -43993 -43994 -43995 -43996 -43997 -43998 -43999 -44000 -44001 -44002 -44003 -44004 -44005 -44006 -44007 -44008 -44009 -44010 -44011 -44012 -44013 -44014 -44015 -44016 -44017 -44018 -44019 -44020 -44021 -44022 -44023 -44024 -44025 -44026 -44027 -44028 -44029 -44030 -44031 -44032 -44033 -44034 -44035 -44036 -44037 -44038 -44039 -44040 -44041 -44042 -44043 -44044 -44045 -44046 -44047 -44048 -44049 -44050 -44051 -44052 -44053 -44054 -44055 -44056 -44057 -44058 -44059 -44060 -44061 -44062 -44063 -44064 -44065 -44066 -44067 -44068 -44069 -44070 -44071 -44072 -44073 -44074 -44075 -44076 -44077 -44078 -44079 -44080 -44081 -44082 -44083 -44084 -44085 -44086 -44087 -44088 -44089 -44090 -44091 -44092 -44093 -44094 -44095 -44096 -44097 -44098 -44099 -44100 -44101 -44102 -44103 -44104 -44105 -44106 -44107 -44108 -44109 -44110 -44111 -44112 -44113 -44114 -44115 -44116 -44117 -44118 -44119 -44120 -44121 -44122 -44123 -44124 -44125 -44126 -44127 -44128 -44129 -44130 -44131 -44132 -44133 -44134 -44135 -44136 -44137 -44138 -44139 -44140 -44141 -44142 -44143 -44144 -44145 -44146 -44147 -44148 -44149 -44150 -44151 -44152 -44153 -44154 -44155 -44156 -44157 -44158 -44159 -44160 -44161 -44162 -44163 -44164 -44165 -44166 -44167 -44168 -44169 -44170 -44171 -44172 -44173 -44174 -44175 -44176 -44177 -44178 -44179 -44180 -44181 -44182 -44183 -44184 -44185 -44186 -44187 -44188 -44189 -44190 -44191 -44192 -44193 -44194 -44195 -44196 -44197 -44198 -44199 -44200 -44201 -44202 -44203 -44204 -44205 -44206 -44207 -44208 -44209 -44210 -44211 -44212 -44213 -44214 -44215 -44216 -44217 -44218 -44219 -44220 -44221 -44222 -44223 -44224 -44225 -44226 -44227 -44228 -44229 -44230 -44231 -44232 -44233 -44234 -44235 -44236 -44237 -44238 -44239 -44240 -44241 -44242 -44243 -44244 -44245 -44246 -44247 -44248 -44249 -44250 -44251 -44252 -44253 -44254 -44255 -44256 -44257 -44258 -44259 -44260 -44261 -44262 -44263 -44264 -44265 -44266 -44267 -44268 -44269 -44270 -44271 -44272 -44273 -44274 -44275 -44276 -44277 -44278 -44279 -44280 -44281 -44282 -44283 -44284 -44285 -44286 -44287 -44288 -44289 -44290 -44291 -44292 -44293 -44294 -44295 -44296 -44297 -44298 -44299 -44300 -44301 -44302 -44303 -44304 -44305 -44306 -44307 -44308 -44309 -44310 -44311 -44312 -44313 -44314 -44315 -44316 -44317 -44318 -44319 -44320 -44321 -44322 -44323 -44324 -44325 -44326 -44327 -44328 -44329 -44330 -44331 -44332 -44333 -44334 -44335 -44336 -44337 -44338 -44339 -44340 -44341 -44342 -44343 -44344 -44345 -44346 -44347 -44348 -44349 -44350 -44351 -44352 -44353 -44354 -44355 -44356 -44357 -44358 -44359 -44360 -44361 -44362 -44363 -44364 -44365 -44366 -44367 -44368 -44369 -44370 -44371 -44372 -44373 -44374 -44375 -44376 -44377 -44378 -44379 -44380 -44381 -44382 -44383 -44384 -44385 -44386 -44387 -44388 -44389 -44390 -44391 -44392 -44393 -44394 -44395 -44396 -44397 -44398 -44399 -44400 -44401 -44402 -44403 -44404 -44405 -44406 -44407 -44408 -44409 -44410 -44411 -44412 -44413 -44414 -44415 -44416 -44417 -44418 -44419 -44420 -44421 -44422 -44423 -44424 -44425 -44426 -44427 -44428 -44429 -44430 -44431 -44432 -44433 -44434 -44435 -44436 -44437 -44438 -44439 -44440 -44441 -44442 -44443 -44444 -44445 -44446 -44447 -44448 -44449 -44450 -44451 -44452 -44453 -44454 -44455 -44456 -44457 -44458 -44459 -44460 -44461 -44462 -44463 -44464 -44465 -44466 -44467 -44468 -44469 -44470 -44471 -44472 -44473 -44474 -44475 -44476 -44477 -44478 -44479 -44480 -44481 -44482 -44483 -44484 -44485 -44486 -44487 -44488 -44489 -44490 -44491 -44492 -44493 -44494 -44495 -44496 -44497 -44498 -44499 -44500 -44501 -44502 -44503 -44504 -44505 -44506 -44507 -44508 -44509 -44510 -44511 -44512 -44513 -44514 -44515 -44516 -44517 -44518 -44519 -44520 -44521 -44522 -44523 -44524 -44525 -44526 -44527 -44528 -44529 -44530 -44531 -44532 -44533 -44534 -44535 -44536 -44537 -44538 -44539 -44540 -44541 -44542 -44543 -44544 -44545 -44546 -44547 -44548 -44549 -44550 -44551 -44552 -44553 -44554 -44555 -44556 -44557 -44558 -44559 -44560 -44561 -44562 -44563 -44564 -44565 -44566 -44567 -44568 -44569 -44570 -44571 -44572 -44573 -44574 -44575 -44576 -44577 -44578 -44579 -44580 -44581 -44582 -44583 -44584 -44585 -44586 -44587 -44588 -44589 -44590 -44591 -44592 -44593 -44594 -44595 -44596 -44597 -44598 -44599 -44600 -44601 -44602 -44603 -44604 -44605 -44606 -44607 -44608 -44609 -44610 -44611 -44612 -44613 -44614 -44615 -44616 -44617 -44618 -44619 -44620 -44621 -44622 -44623 -44624 -44625 -44626 -44627 -44628 -44629 -44630 -44631 -44632 -44633 -44634 -44635 -44636 -44637 -44638 -44639 -44640 -44641 -44642 -44643 -44644 -44645 -44646 -44647 -44648 -44649 -44650 -44651 -44652 -44653 -44654 -44655 -44656 -44657 -44658 -44659 -44660 -44661 -44662 -44663 -44664 -44665 -44666 -44667 -44668 -44669 -44670 -44671 -44672 -44673 -44674 -44675 -44676 -44677 -44678 -44679 -44680 -44681 -44682 -44683 -44684 -44685 -44686 -44687 -44688 -44689 -44690 -44691 -44692 -44693 -44694 -44695 -44696 -44697 -44698 -44699 -44700 -44701 -44702 -44703 -44704 -44705 -44706 -44707 -44708 -44709 -44710 -44711 -44712 -44713 -44714 -44715 -44716 -44717 -44718 -44719 -44720 -44721 -44722 -44723 -44724 -44725 -44726 -44727 -44728 -44729 -44730 -44731 -44732 -44733 -44734 -44735 -44736 -44737 -44738 -44739 -44740 -44741 -44742 -44743 -44744 -44745 -44746 -44747 -44748 -44749 -44750 -44751 -44752 -44753 -44754 -44755 -44756 -44757 -44758 -44759 -44760 -44761 -44762 -44763 -44764 -44765 -44766 -44767 -44768 -44769 -44770 -44771 -44772 -44773 -44774 -44775 -44776 -44777 -44778 -44779 -44780 -44781 -44782 -44783 -44784 -44785 -44786 -44787 -44788 -44789 -44790 -44791 -44792 -44793 -44794 -44795 -44796 -44797 -44798 -44799 -44800 -44801 -44802 -44803 -44804 -44805 -44806 -44807 -44808 -44809 -44810 -44811 -44812 -44813 -44814 -44815 -44816 -44817 -44818 -44819 -44820 -44821 -44822 -44823 -44824 -44825 -44826 -44827 -44828 -44829 -44830 -44831 -44832 -44833 -44834 -44835 -44836 -44837 -44838 -44839 -44840 -44841 -44842 -44843 -44844 -44845 -44846 -44847 -44848 -44849 -44850 -44851 -44852 -44853 -44854 -44855 -44856 -44857 -44858 -44859 -44860 -44861 -44862 -44863 -44864 -44865 -44866 -44867 -44868 -44869 -44870 -44871 -44872 -44873 -44874 -44875 -44876 -44877 -44878 -44879 -44880 -44881 -44882 -44883 -44884 -44885 -44886 -44887 -44888 -44889 -44890 -44891 -44892 -44893 -44894 -44895 -44896 -44897 -44898 -44899 -44900 -44901 -44902 -44903 -44904 -44905 -44906 -44907 -44908 -44909 -44910 -44911 -44912 -44913 -44914 -44915 -44916 -44917 -44918 -44919 -44920 -44921 -44922 -44923 -44924 -44925 -44926 -44927 -44928 -44929 -44930 -44931 -44932 -44933 -44934 -44935 -44936 -44937 -44938 -44939 -44940 -44941 -44942 -44943 -44944 -44945 -44946 -44947 -44948 -44949 -44950 -44951 -44952 -44953 -44954 -44955 -44956 -44957 -44958 -44959 -44960 -44961 -44962 -44963 -44964 -44965 -44966 -44967 -44968 -44969 -44970 -44971 -44972 -44973 -44974 -44975 -44976 -44977 -44978 -44979 -44980 -44981 -44982 -44983 -44984 -44985 -44986 -44987 -44988 -44989 -44990 -44991 -44992 -44993 -44994 -44995 -44996 -44997 -44998 -44999 -45000 -45001 -45002 -45003 -45004 -45005 -45006 -45007 -45008 -45009 -45010 -45011 -45012 -45013 -45014 -45015 -45016 -45017 -45018 -45019 -45020 -45021 -45022 -45023 -45024 -45025 -45026 -45027 -45028 -45029 -45030 -45031 -45032 -45033 -45034 -45035 -45036 -45037 -45038 -45039 -45040 -45041 -45042 -45043 -45044 -45045 -45046 -45047 -45048 -45049 -45050 -45051 -45052 -45053 -45054 -45055 -45056 -45057 -45058 -45059 -45060 -45061 -45062 -45063 -45064 -45065 -45066 -45067 -45068 -45069 -45070 -45071 -45072 -45073 -45074 -45075 -45076 -45077 -45078 -45079 -45080 -45081 -45082 -45083 -45084 -45085 -45086 -45087 -45088 -45089 -45090 -45091 -45092 -45093 -45094 -45095 -45096 -45097 -45098 -45099 -45100 -45101 -45102 -45103 -45104 -45105 -45106 -45107 -45108 -45109 -45110 -45111 -45112 -45113 -45114 -45115 -45116 -45117 -45118 -45119 -45120 -45121 -45122 -45123 -45124 -45125 -45126 -45127 -45128 -45129 -45130 -45131 -45132 -45133 -45134 -45135 -45136 -45137 -45138 -45139 -45140 -45141 -45142 -45143 -45144 -45145 -45146 -45147 -45148 -45149 -45150 -45151 -45152 -45153 -45154 -45155 -45156 -45157 -45158 -45159 -45160 -45161 -45162 -45163 -45164 -45165 -45166 -45167 -45168 -45169 -45170 -45171 -45172 -45173 -45174 -45175 -45176 -45177 -45178 -45179 -45180 -45181 -45182 -45183 -45184 -45185 -45186 -45187 -45188 -45189 -45190 -45191 -45192 -45193 -45194 -45195 -45196 -45197 -45198 -45199 -45200 -45201 -45202 -45203 -45204 -45205 -45206 -45207 -45208 -45209 -45210 -45211 -45212 -45213 -45214 -45215 -45216 -45217 -45218 -45219 -45220 -45221 -45222 -45223 -45224 -45225 -45226 -45227 -45228 -45229 -45230 -45231 -45232 -45233 -45234 -45235 -45236 -45237 -45238 -45239 -45240 -45241 -45242 -45243 -45244 -45245 -45246 -45247 -45248 -45249 -45250 -45251 -45252 -45253 -45254 -45255 -45256 -45257 -45258 -45259 -45260 -45261 -45262 -45263 -45264 -45265 -45266 -45267 -45268 -45269 -45270 -45271 -45272 -45273 -45274 -45275 -45276 -45277 -45278 -45279 -45280 -45281 -45282 -45283 -45284 -45285 -45286 -45287 -45288 -45289 -45290 -45291 -45292 -45293 -45294 -45295 -45296 -45297 -45298 -45299 -45300 -45301 -45302 -45303 -45304 -45305 -45306 -45307 -45308 -45309 -45310 -45311 -45312 -45313 -45314 -45315 -45316 -45317 -45318 -45319 -45320 -45321 -45322 -45323 -45324 -45325 -45326 -45327 -45328 -45329 -45330 -45331 -45332 -45333 -45334 -45335 -45336 -45337 -45338 -45339 -45340 -45341 -45342 -45343 -45344 -45345 -45346 -45347 -45348 -45349 -45350 -45351 -45352 -45353 -45354 -45355 -45356 -45357 -45358 -45359 -45360 -45361 -45362 -45363 -45364 -45365 -45366 -45367 -45368 -45369 -45370 -45371 -45372 -45373 -45374 -45375 -45376 -45377 -45378 -45379 -45380 -45381 -45382 -45383 -45384 -45385 -45386 -45387 -45388 -45389 -45390 -45391 -45392 -45393 -45394 -45395 -45396 -45397 -45398 -45399 -45400 -45401 -45402 -45403 -45404 -45405 -45406 -45407 -45408 -45409 -45410 -45411 -45412 -45413 -45414 -45415 -45416 -45417 -45418 -45419 -45420 -45421 -45422 -45423 -45424 -45425 -45426 -45427 -45428 -45429 -45430 -45431 -45432 -45433 -45434 -45435 -45436 -45437 -45438 -45439 -45440 -45441 -45442 -45443 -45444 -45445 -45446 -45447 -45448 -45449 -45450 -45451 -45452 -45453 -45454 -45455 -45456 -45457 -45458 -45459 -45460 -45461 -45462 -45463 -45464 -45465 -45466 -45467 -45468 -45469 -45470 -45471 -45472 -45473 -45474 -45475 -45476 -45477 -45478 -45479 -45480 -45481 -45482 -45483 -45484 -45485 -45486 -45487 -45488 -45489 -45490 -45491 -45492 -45493 -45494 -45495 -45496 -45497 -45498 -45499 -45500 -45501 -45502 -45503 -45504 -45505 -45506 -45507 -45508 -45509 -45510 -45511 -45512 -45513 -45514 -45515 -45516 -45517 -45518 -45519 -45520 -45521 -45522 -45523 -45524 -45525 -45526 -45527 -45528 -45529 -45530 -45531 -45532 -45533 -45534 -45535 -45536 -45537 -45538 -45539 -45540 -45541 -45542 -45543 -45544 -45545 -45546 -45547 -45548 -45549 -45550 -45551 -45552 -45553 -45554 -45555 -45556 -45557 -45558 -45559 -45560 -45561 -45562 -45563 -45564 -45565 -45566 -45567 -45568 -45569 -45570 -45571 -45572 -45573 -45574 -45575 -45576 -45577 -45578 -45579 -45580 -45581 -45582 -45583 -45584 -45585 -45586 -45587 -45588 -45589 -45590 -45591 -45592 -45593 -45594 -45595 -45596 -45597 -45598 -45599 -45600 -45601 -45602 -45603 -45604 -45605 -45606 -45607 -45608 -45609 -45610 -45611 -45612 -45613 -45614 -45615 -45616 -45617 -45618 -45619 -45620 -45621 -45622 -45623 -45624 -45625 -45626 -45627 -45628 -45629 -45630 -45631 -45632 -45633 -45634 -45635 -45636 -45637 -45638 -45639 -45640 -45641 -45642 -45643 -45644 -45645 -45646 -45647 -45648 -45649 -45650 -45651 -45652 -45653 -45654 -45655 -45656 -45657 -45658 -45659 -45660 -45661 -45662 -45663 -45664 -45665 -45666 -45667 -45668 -45669 -45670 -45671 -45672 -45673 -45674 -45675 -45676 -45677 -45678 -45679 -45680 -45681 -45682 -45683 -45684 -45685 -45686 -45687 -45688 -45689 -45690 -45691 -45692 -45693 -45694 -45695 -45696 -45697 -45698 -45699 -45700 -45701 -45702 -45703 -45704 -45705 -45706 -45707 -45708 -45709 -45710 -45711 -45712 -45713 -45714 -45715 -45716 -45717 -45718 -45719 -45720 -45721 -45722 -45723 -45724 -45725 -45726 -45727 -45728 -45729 -45730 -45731 -45732 -45733 -45734 -45735 -45736 -45737 -45738 -45739 -45740 -45741 -45742 -45743 -45744 -45745 -45746 -45747 -45748 -45749 -45750 -45751 -45752 -45753 -45754 -45755 -45756 -45757 -45758 -45759 -45760 -45761 -45762 -45763 -45764 -45765 -45766 -45767 -45768 -45769 -45770 -45771 -45772 -45773 -45774 -45775 -45776 -45777 -45778 -45779 -45780 -45781 -45782 -45783 -45784 -45785 -45786 -45787 -45788 -45789 -45790 -45791 -45792 -45793 -45794 -45795 -45796 -45797 -45798 -45799 -45800 -45801 -45802 -45803 -45804 -45805 -45806 -45807 -45808 -45809 -45810 -45811 -45812 -45813 -45814 -45815 -45816 -45817 -45818 -45819 -45820 -45821 -45822 -45823 -45824 -45825 -45826 -45827 -45828 -45829 -45830 -45831 -45832 -45833 -45834 -45835 -45836 -45837 -45838 -45839 -45840 -45841 -45842 -45843 -45844 -45845 -45846 -45847 -45848 -45849 -45850 -45851 -45852 -45853 -45854 -45855 -45856 -45857 -45858 -45859 -45860 -45861 -45862 -45863 -45864 -45865 -45866 -45867 -45868 -45869 -45870 -45871 -45872 -45873 -45874 -45875 -45876 -45877 -45878 -45879 -45880 -45881 -45882 -45883 -45884 -45885 -45886 -45887 -45888 -45889 -45890 -45891 -45892 -45893 -45894 -45895 -45896 -45897 -45898 -45899 -45900 -45901 -45902 -45903 -45904 -45905 -45906 -45907 -45908 -45909 -45910 -45911 -45912 -45913 -45914 -45915 -45916 -45917 -45918 -45919 -45920 -45921 -45922 -45923 -45924 -45925 -45926 -45927 -45928 -45929 -45930 -45931 -45932 -45933 -45934 -45935 -45936 -45937 -45938 -45939 -45940 -45941 -45942 -45943 -45944 -45945 -45946 -45947 -45948 -45949 -45950 -45951 -45952 -45953 -45954 -45955 -45956 -45957 -45958 -45959 -45960 -45961 -45962 -45963 -45964 -45965 -45966 -45967 -45968 -45969 -45970 -45971 -45972 -45973 -45974 -45975 -45976 -45977 -45978 -45979 -45980 -45981 -45982 -45983 -45984 -45985 -45986 -45987 -45988 -45989 -45990 -45991 -45992 -45993 -45994 -45995 -45996 -45997 -45998 -45999 -46000 -46001 -46002 -46003 -46004 -46005 -46006 -46007 -46008 -46009 -46010 -46011 -46012 -46013 -46014 -46015 -46016 -46017 -46018 -46019 -46020 -46021 -46022 -46023 -46024 -46025 -46026 -46027 -46028 -46029 -46030 -46031 -46032 -46033 -46034 -46035 -46036 -46037 -46038 -46039 -46040 -46041 -46042 -46043 -46044 -46045 -46046 -46047 -46048 -46049 -46050 -46051 -46052 -46053 -46054 -46055 -46056 -46057 -46058 -46059 -46060 -46061 -46062 -46063 -46064 -46065 -46066 -46067 -46068 -46069 -46070 -46071 -46072 -46073 -46074 -46075 -46076 -46077 -46078 -46079 -46080 -46081 -46082 -46083 -46084 -46085 -46086 -46087 -46088 -46089 -46090 -46091 -46092 -46093 -46094 -46095 -46096 -46097 -46098 -46099 -46100 -46101 -46102 -46103 -46104 -46105 -46106 -46107 -46108 -46109 -46110 -46111 -46112 -46113 -46114 -46115 -46116 -46117 -46118 -46119 -46120 -46121 -46122 -46123 -46124 -46125 -46126 -46127 -46128 -46129 -46130 -46131 -46132 -46133 -46134 -46135 -46136 -46137 -46138 -46139 -46140 -46141 -46142 -46143 -46144 -46145 -46146 -46147 -46148 -46149 -46150 -46151 -46152 -46153 -46154 -46155 -46156 -46157 -46158 -46159 -46160 -46161 -46162 -46163 -46164 -46165 -46166 -46167 -46168 -46169 -46170 -46171 -46172 -46173 -46174 -46175 -46176 -46177 -46178 -46179 -46180 -46181 -46182 -46183 -46184 -46185 -46186 -46187 -46188 -46189 -46190 -46191 -46192 -46193 -46194 -46195 -46196 -46197 -46198 -46199 -46200 -46201 -46202 -46203 -46204 -46205 -46206 -46207 -46208 -46209 -46210 -46211 -46212 -46213 -46214 -46215 -46216 -46217 -46218 -46219 -46220 -46221 -46222 -46223 -46224 -46225 -46226 -46227 -46228 -46229 -46230 -46231 -46232 -46233 -46234 -46235 -46236 -46237 -46238 -46239 -46240 -46241 -46242 -46243 -46244 -46245 -46246 -46247 -46248 -46249 -46250 -46251 -46252 -46253 -46254 -46255 -46256 -46257 -46258 -46259 -46260 -46261 -46262 -46263 -46264 -46265 -46266 -46267 -46268 -46269 -46270 -46271 -46272 -46273 -46274 -46275 -46276 -46277 -46278 -46279 -46280 -46281 -46282 -46283 -46284 -46285 -46286 -46287 -46288 -46289 -46290 -46291 -46292 -46293 -46294 -46295 -46296 -46297 -46298 -46299 -46300 -46301 -46302 -46303 -46304 -46305 -46306 -46307 -46308 -46309 -46310 -46311 -46312 -46313 -46314 -46315 -46316 -46317 -46318 -46319 -46320 -46321 -46322 -46323 -46324 -46325 -46326 -46327 -46328 -46329 -46330 -46331 -46332 -46333 -46334 -46335 -46336 -46337 -46338 -46339 -46340 -46341 -46342 -46343 -46344 -46345 -46346 -46347 -46348 -46349 -46350 -46351 -46352 -46353 -46354 -46355 -46356 -46357 -46358 -46359 -46360 -46361 -46362 -46363 -46364 -46365 -46366 -46367 -46368 -46369 -46370 -46371 -46372 -46373 -46374 -46375 -46376 -46377 -46378 -46379 -46380 -46381 -46382 -46383 -46384 -46385 -46386 -46387 -46388 -46389 -46390 -46391 -46392 -46393 -46394 -46395 -46396 -46397 -46398 -46399 -46400 -46401 -46402 -46403 -46404 -46405 -46406 -46407 -46408 -46409 -46410 -46411 -46412 -46413 -46414 -46415 -46416 -46417 -46418 -46419 -46420 -46421 -46422 -46423 -46424 -46425 -46426 -46427 -46428 -46429 -46430 -46431 -46432 -46433 -46434 -46435 -46436 -46437 -46438 -46439 -46440 -46441 -46442 -46443 -46444 -46445 -46446 -46447 -46448 -46449 -46450 -46451 -46452 -46453 -46454 -46455 -46456 -46457 -46458 -46459 -46460 -46461 -46462 -46463 -46464 -46465 -46466 -46467 -46468 -46469 -46470 -46471 -46472 -46473 -46474 -46475 -46476 -46477 -46478 -46479 -46480 -46481 -46482 -46483 -46484 -46485 -46486 -46487 -46488 -46489 -46490 -46491 -46492 -46493 -46494 -46495 -46496 -46497 -46498 -46499 -46500 -46501 -46502 -46503 -46504 -46505 -46506 -46507 -46508 -46509 -46510 -46511 -46512 -46513 -46514 -46515 -46516 -46517 -46518 -46519 -46520 -46521 -46522 -46523 -46524 -46525 -46526 -46527 -46528 -46529 -46530 -46531 -46532 -46533 -46534 -46535 -46536 -46537 -46538 -46539 -46540 -46541 -46542 -46543 -46544 -46545 -46546 -46547 -46548 -46549 -46550 -46551 -46552 -46553 -46554 -46555 -46556 -46557 -46558 -46559 -46560 -46561 -46562 -46563 -46564 -46565 -46566 -46567 -46568 -46569 -46570 -46571 -46572 -46573 -46574 -46575 -46576 -46577 -46578 -46579 -46580 -46581 -46582 -46583 -46584 -46585 -46586 -46587 -46588 -46589 -46590 -46591 -46592 -46593 -46594 -46595 -46596 -46597 -46598 -46599 -46600 -46601 -46602 -46603 -46604 -46605 -46606 -46607 -46608 -46609 -46610 -46611 -46612 -46613 -46614 -46615 -46616 -46617 -46618 -46619 -46620 -46621 -46622 -46623 -46624 -46625 -46626 -46627 -46628 -46629 -46630 -46631 -46632 -46633 -46634 -46635 -46636 -46637 -46638 -46639 -46640 -46641 -46642 -46643 -46644 -46645 -46646 -46647 -46648 -46649 -46650 -46651 -46652 -46653 -46654 -46655 -46656 -46657 -46658 -46659 -46660 -46661 -46662 -46663 -46664 -46665 -46666 -46667 -46668 -46669 -46670 -46671 -46672 -46673 -46674 -46675 -46676 -46677 -46678 -46679 -46680 -46681 -46682 -46683 -46684 -46685 -46686 -46687 -46688 -46689 -46690 -46691 -46692 -46693 -46694 -46695 -46696 -46697 -46698 -46699 -46700 -46701 -46702 -46703 -46704 -46705 -46706 -46707 -46708 -46709 -46710 -46711 -46712 -46713 -46714 -46715 -46716 -46717 -46718 -46719 -46720 -46721 -46722 -46723 -46724 -46725 -46726 -46727 -46728 -46729 -46730 -46731 -46732 -46733 -46734 -46735 -46736 -46737 -46738 -46739 -46740 -46741 -46742 -46743 -46744 -46745 -46746 -46747 -46748 -46749 -46750 -46751 -46752 -46753 -46754 -46755 -46756 -46757 -46758 -46759 -46760 -46761 -46762 -46763 -46764 -46765 -46766 -46767 -46768 -46769 -46770 -46771 -46772 -46773 -46774 -46775 -46776 -46777 -46778 -46779 -46780 -46781 -46782 -46783 -46784 -46785 -46786 -46787 -46788 -46789 -46790 -46791 -46792 -46793 -46794 -46795 -46796 -46797 -46798 -46799 -46800 -46801 -46802 -46803 -46804 -46805 -46806 -46807 -46808 -46809 -46810 -46811 -46812 -46813 -46814 -46815 -46816 -46817 -46818 -46819 -46820 -46821 -46822 -46823 -46824 -46825 -46826 -46827 -46828 -46829 -46830 -46831 -46832 -46833 -46834 -46835 -46836 -46837 -46838 -46839 -46840 -46841 -46842 -46843 -46844 -46845 -46846 -46847 -46848 -46849 -46850 -46851 -46852 -46853 -46854 -46855 -46856 -46857 -46858 -46859 -46860 -46861 -46862 -46863 -46864 -46865 -46866 -46867 -46868 -46869 -46870 -46871 -46872 -46873 -46874 -46875 -46876 -46877 -46878 -46879 -46880 -46881 -46882 -46883 -46884 -46885 -46886 -46887 -46888 -46889 -46890 -46891 -46892 -46893 -46894 -46895 -46896 -46897 -46898 -46899 -46900 -46901 -46902 -46903 -46904 -46905 -46906 -46907 -46908 -46909 -46910 -46911 -46912 -46913 -46914 -46915 -46916 -46917 -46918 -46919 -46920 -46921 -46922 -46923 -46924 -46925 -46926 -46927 -46928 -46929 -46930 -46931 -46932 -46933 -46934 -46935 -46936 -46937 -46938 -46939 -46940 -46941 -46942 -46943 -46944 -46945 -46946 -46947 -46948 -46949 -46950 -46951 -46952 -46953 -46954 -46955 -46956 -46957 -46958 -46959 -46960 -46961 -46962 -46963 -46964 -46965 -46966 -46967 -46968 -46969 -46970 -46971 -46972 -46973 -46974 -46975 -46976 -46977 -46978 -46979 -46980 -46981 -46982 -46983 -46984 -46985 -46986 -46987 -46988 -46989 -46990 -46991 -46992 -46993 -46994 -46995 -46996 -46997 -46998 -46999 -47000 -47001 -47002 -47003 -47004 -47005 -47006 -47007 -47008 -47009 -47010 -47011 -47012 -47013 -47014 -47015 -47016 -47017 -47018 -47019 -47020 -47021 -47022 -47023 -47024 -47025 -47026 -47027 -47028 -47029 -47030 -47031 -47032 -47033 -47034 -47035 -47036 -47037 -47038 -47039 -47040 -47041 -47042 -47043 -47044 -47045 -47046 -47047 -47048 -47049 -47050 -47051 -47052 -47053 -47054 -47055 -47056 -47057 -47058 -47059 -47060 -47061 -47062 -47063 -47064 -47065 -47066 -47067 -47068 -47069 -47070 -47071 -47072 -47073 -47074 -47075 -47076 -47077 -47078 -47079 -47080 -47081 -47082 -47083 -47084 -47085 -47086 -47087 -47088 -47089 -47090 -47091 -47092 -47093 -47094 -47095 -47096 -47097 -47098 -47099 -47100 -47101 -47102 -47103 -47104 -47105 -47106 -47107 -47108 -47109 -47110 -47111 -47112 -47113 -47114 -47115 -47116 -47117 -47118 -47119 -47120 -47121 -47122 -47123 -47124 -47125 -47126 -47127 -47128 -47129 -47130 -47131 -47132 -47133 -47134 -47135 -47136 -47137 -47138 -47139 -47140 -47141 -47142 -47143 -47144 -47145 -47146 -47147 -47148 -47149 -47150 -47151 -47152 -47153 -47154 -47155 -47156 -47157 -47158 -47159 -47160 -47161 -47162 -47163 -47164 -47165 -47166 -47167 -47168 -47169 -47170 -47171 -47172 -47173 -47174 -47175 -47176 -47177 -47178 -47179 -47180 -47181 -47182 -47183 -47184 -47185 -47186 -47187 -47188 -47189 -47190 -47191 -47192 -47193 -47194 -47195 -47196 -47197 -47198 -47199 -47200 -47201 -47202 -47203 -47204 -47205 -47206 -47207 -47208 -47209 -47210 -47211 -47212 -47213 -47214 -47215 -47216 -47217 -47218 -47219 -47220 -47221 -47222 -47223 -47224 -47225 -47226 -47227 -47228 -47229 -47230 -47231 -47232 -47233 -47234 -47235 -47236 -47237 -47238 -47239 -47240 -47241 -47242 -47243 -47244 -47245 -47246 -47247 -47248 -47249 -47250 -47251 -47252 -47253 -47254 -47255 -47256 -47257 -47258 -47259 -47260 -47261 -47262 -47263 -47264 -47265 -47266 -47267 -47268 -47269 -47270 -47271 -47272 -47273 -47274 -47275 -47276 -47277 -47278 -47279 -47280 -47281 -47282 -47283 -47284 -47285 -47286 -47287 -47288 -47289 -47290 -47291 -47292 -47293 -47294 -47295 -47296 -47297 -47298 -47299 -47300 -47301 -47302 -47303 -47304 -47305 -47306 -47307 -47308 -47309 -47310 -47311 -47312 -47313 -47314 -47315 -47316 -47317 -47318 -47319 -47320 -47321 -47322 -47323 -47324 -47325 -47326 -47327 -47328 -47329 -47330 -47331 -47332 -47333 -47334 -47335 -47336 -47337 -47338 -47339 -47340 -47341 -47342 -47343 -47344 -47345 -47346 -47347 -47348 -47349 -47350 -47351 -47352 -47353 -47354 -47355 -47356 -47357 -47358 -47359 -47360 -47361 -47362 -47363 -47364 -47365 -47366 -47367 -47368 -47369 -47370 -47371 -47372 -47373 -47374 -47375 -47376 -47377 -47378 -47379 -47380 -47381 -47382 -47383 -47384 -47385 -47386 -47387 -47388 -47389 -47390 -47391 -47392 -47393 -47394 -47395 -47396 -47397 -47398 -47399 -47400 -47401 -47402 -47403 -47404 -47405 -47406 -47407 -47408 -47409 -47410 -47411 -47412 -47413 -47414 -47415 -47416 -47417 -47418 -47419 -47420 -47421 -47422 -47423 -47424 -47425 -47426 -47427 -47428 -47429 -47430 -47431 -47432 -47433 -47434 -47435 -47436 -47437 -47438 -47439 -47440 -47441 -47442 -47443 -47444 -47445 -47446 -47447 -47448 -47449 -47450 -47451 -47452 -47453 -47454 -47455 -47456 -47457 -47458 -47459 -47460 -47461 -47462 -47463 -47464 -47465 -47466 -47467 -47468 -47469 -47470 -47471 -47472 -47473 -47474 -47475 -47476 -47477 -47478 -47479 -47480 -47481 -47482 -47483 -47484 -47485 -47486 -47487 -47488 -47489 -47490 -47491 -47492 -47493 -47494 -47495 -47496 -47497 -47498 -47499 -47500 -47501 -47502 -47503 -47504 -47505 -47506 -47507 -47508 -47509 -47510 -47511 -47512 -47513 -47514 -47515 -47516 -47517 -47518 -47519 -47520 -47521 -47522 -47523 -47524 -47525 -47526 -47527 -47528 -47529 -47530 -47531 -47532 -47533 -47534 -47535 -47536 -47537 -47538 -47539 -47540 -47541 -47542 -47543 -47544 -47545 -47546 -47547 -47548 -47549 -47550 -47551 -47552 -47553 -47554 -47555 -47556 -47557 -47558 -47559 -47560 -47561 -47562 -47563 -47564 -47565 -47566 -47567 -47568 -47569 -47570 -47571 -47572 -47573 -47574 -47575 -47576 -47577 -47578 -47579 -47580 -47581 -47582 -47583 -47584 -47585 -47586 -47587 -47588 -47589 -47590 -47591 -47592 -47593 -47594 -47595 -47596 -47597 -47598 -47599 -47600 -47601 -47602 -47603 -47604 -47605 -47606 -47607 -47608 -47609 -47610 -47611 -47612 -47613 -47614 -47615 -47616 -47617 -47618 -47619 -47620 -47621 -47622 -47623 -47624 -47625 -47626 -47627 -47628 -47629 -47630 -47631 -47632 -47633 -47634 -47635 -47636 -47637 -47638 -47639 -47640 -47641 -47642 -47643 -47644 -47645 -47646 -47647 -47648 -47649 -47650 -47651 -47652 -47653 -47654 -47655 -47656 -47657 -47658 -47659 -47660 -47661 -47662 -47663 -47664 -47665 -47666 -47667 -47668 -47669 -47670 -47671 -47672 -47673 -47674 -47675 -47676 -47677 -47678 -47679 -47680 -47681 -47682 -47683 -47684 -47685 -47686 -47687 -47688 -47689 -47690 -47691 -47692 -47693 -47694 -47695 -47696 -47697 -47698 -47699 -47700 -47701 -47702 -47703 -47704 -47705 -47706 -47707 -47708 -47709 -47710 -47711 -47712 -47713 -47714 -47715 -47716 -47717 -47718 -47719 -47720 -47721 -47722 -47723 -47724 -47725 -47726 -47727 -47728 -47729 -47730 -47731 -47732 -47733 -47734 -47735 -47736 -47737 -47738 -47739 -47740 -47741 -47742 -47743 -47744 -47745 -47746 -47747 -47748 -47749 -47750 -47751 -47752 -47753 -47754 -47755 -47756 -47757 -47758 -47759 -47760 -47761 -47762 -47763 -47764 -47765 -47766 -47767 -47768 -47769 -47770 -47771 -47772 -47773 -47774 -47775 -47776 -47777 -47778 -47779 -47780 -47781 -47782 -47783 -47784 -47785 -47786 -47787 -47788 -47789 -47790 -47791 -47792 -47793 -47794 -47795 -47796 -47797 -47798 -47799 -47800 -47801 -47802 -47803 -47804 -47805 -47806 -47807 -47808 -47809 -47810 -47811 -47812 -47813 -47814 -47815 -47816 -47817 -47818 -47819 -47820 -47821 -47822 -47823 -47824 -47825 -47826 -47827 -47828 -47829 -47830 -47831 -47832 -47833 -47834 -47835 -47836 -47837 -47838 -47839 -47840 -47841 -47842 -47843 -47844 -47845 -47846 -47847 -47848 -47849 -47850 -47851 -47852 -47853 -47854 -47855 -47856 -47857 -47858 -47859 -47860 -47861 -47862 -47863 -47864 -47865 -47866 -47867 -47868 -47869 -47870 -47871 -47872 -47873 -47874 -47875 -47876 -47877 -47878 -47879 -47880 -47881 -47882 -47883 -47884 -47885 -47886 -47887 -47888 -47889 -47890 -47891 -47892 -47893 -47894 -47895 -47896 -47897 -47898 -47899 -47900 -47901 -47902 -47903 -47904 -47905 -47906 -47907 -47908 -47909 -47910 -47911 -47912 -47913 -47914 -47915 -47916 -47917 -47918 -47919 -47920 -47921 -47922 -47923 -47924 -47925 -47926 -47927 -47928 -47929 -47930 -47931 -47932 -47933 -47934 -47935 -47936 -47937 -47938 -47939 -47940 -47941 -47942 -47943 -47944 -47945 -47946 -47947 -47948 -47949 -47950 -47951 -47952 -47953 -47954 -47955 -47956 -47957 -47958 -47959 -47960 -47961 -47962 -47963 -47964 -47965 -47966 -47967 -47968 -47969 -47970 -47971 -47972 -47973 -47974 -47975 -47976 -47977 -47978 -47979 -47980 -47981 -47982 -47983 -47984 -47985 -47986 -47987 -47988 -47989 -47990 -47991 -47992 -47993 -47994 -47995 -47996 -47997 -47998 -47999 -48000 -48001 -48002 -48003 -48004 -48005 -48006 -48007 -48008 -48009 -48010 -48011 -48012 -48013 -48014 -48015 -48016 -48017 -48018 -48019 -48020 -48021 -48022 -48023 -48024 -48025 -48026 -48027 -48028 -48029 -48030 -48031 -48032 -48033 -48034 -48035 -48036 -48037 -48038 -48039 -48040 -48041 -48042 -48043 -48044 -48045 -48046 -48047 -48048 -48049 -48050 -48051 -48052 -48053 -48054 -48055 -48056 -48057 -48058 -48059 -48060 -48061 -48062 -48063 -48064 -48065 -48066 -48067 -48068 -48069 -48070 -48071 -48072 -48073 -48074 -48075 -48076 -48077 -48078 -48079 -48080 -48081 -48082 -48083 -48084 -48085 -48086 -48087 -48088 -48089 -48090 -48091 -48092 -48093 -48094 -48095 -48096 -48097 -48098 -48099 -48100 -48101 -48102 -48103 -48104 -48105 -48106 -48107 -48108 -48109 -48110 -48111 -48112 -48113 -48114 -48115 -48116 -48117 -48118 -48119 -48120 -48121 -48122 -48123 -48124 -48125 -48126 -48127 -48128 -48129 -48130 -48131 -48132 -48133 -48134 -48135 -48136 -48137 -48138 -48139 -48140 -48141 -48142 -48143 -48144 -48145 -48146 -48147 -48148 -48149 -48150 -48151 -48152 -48153 -48154 -48155 -48156 -48157 -48158 -48159 -48160 -48161 -48162 -48163 -48164 -48165 -48166 -48167 -48168 -48169 -48170 -48171 -48172 -48173 -48174 -48175 -48176 -48177 -48178 -48179 -48180 -48181 -48182 -48183 -48184 -48185 -48186 -48187 -48188 -48189 -48190 -48191 -48192 -48193 -48194 -48195 -48196 -48197 -48198 -48199 -48200 -48201 -48202 -48203 -48204 -48205 -48206 -48207 -48208 -48209 -48210 -48211 -48212 -48213 -48214 -48215 -48216 -48217 -48218 -48219 -48220 -48221 -48222 -48223 -48224 -48225 -48226 -48227 -48228 -48229 -48230 -48231 -48232 -48233 -48234 -48235 -48236 -48237 -48238 -48239 -48240 -48241 -48242 -48243 -48244 -48245 -48246 -48247 -48248 -48249 -48250 -48251 -48252 -48253 -48254 -48255 -48256 -48257 -48258 -48259 -48260 -48261 -48262 -48263 -48264 -48265 -48266 -48267 -48268 -48269 -48270 -48271 -48272 -48273 -48274 -48275 -48276 -48277 -48278 -48279 -48280 -48281 -48282 -48283 -48284 -48285 -48286 -48287 -48288 -48289 -48290 -48291 -48292 -48293 -48294 -48295 -48296 -48297 -48298 -48299 -48300 -48301 -48302 -48303 -48304 -48305 -48306 -48307 -48308 -48309 -48310 -48311 -48312 -48313 -48314 -48315 -48316 -48317 -48318 -48319 -48320 -48321 -48322 -48323 -48324 -48325 -48326 -48327 -48328 -48329 -48330 -48331 -48332 -48333 -48334 -48335 -48336 -48337 -48338 -48339 -48340 -48341 -48342 -48343 -48344 -48345 -48346 -48347 -48348 -48349 -48350 -48351 -48352 -48353 -48354 -48355 -48356 -48357 -48358 -48359 -48360 -48361 -48362 -48363 -48364 -48365 -48366 -48367 -48368 -48369 -48370 -48371 -48372 -48373 -48374 -48375 -48376 -48377 -48378 -48379 -48380 -48381 -48382 -48383 -48384 -48385 -48386 -48387 -48388 -48389 -48390 -48391 -48392 -48393 -48394 -48395 -48396 -48397 -48398 -48399 -48400 -48401 -48402 -48403 -48404 -48405 -48406 -48407 -48408 -48409 -48410 -48411 -48412 -48413 -48414 -48415 -48416 -48417 -48418 -48419 -48420 -48421 -48422 -48423 -48424 -48425 -48426 -48427 -48428 -48429 -48430 -48431 -48432 -48433 -48434 -48435 -48436 -48437 -48438 -48439 -48440 -48441 -48442 -48443 -48444 -48445 -48446 -48447 -48448 -48449 -48450 -48451 -48452 -48453 -48454 -48455 -48456 -48457 -48458 -48459 -48460 -48461 -48462 -48463 -48464 -48465 -48466 -48467 -48468 -48469 -48470 -48471 -48472 -48473 -48474 -48475 -48476 -48477 -48478 -48479 -48480 -48481 -48482 -48483 -48484 -48485 -48486 -48487 -48488 -48489 -48490 -48491 -48492 -48493 -48494 -48495 -48496 -48497 -48498 -48499 -48500 -48501 -48502 -48503 -48504 -48505 -48506 -48507 -48508 -48509 -48510 -48511 -48512 -48513 -48514 -48515 -48516 -48517 -48518 -48519 -48520 -48521 -48522 -48523 -48524 -48525 -48526 -48527 -48528 -48529 -48530 -48531 -48532 -48533 -48534 -48535 -48536 -48537 -48538 -48539 -48540 -48541 -48542 -48543 -48544 -48545 -48546 -48547 -48548 -48549 -48550 -48551 -48552 -48553 -48554 -48555 -48556 -48557 -48558 -48559 -48560 -48561 -48562 -48563 -48564 -48565 -48566 -48567 -48568 -48569 -48570 -48571 -48572 -48573 -48574 -48575 -48576 -48577 -48578 -48579 -48580 -48581 -48582 -48583 -48584 -48585 -48586 -48587 -48588 -48589 -48590 -48591 -48592 -48593 -48594 -48595 -48596 -48597 -48598 -48599 -48600 -48601 -48602 -48603 -48604 -48605 -48606 -48607 -48608 -48609 -48610 -48611 -48612 -48613 -48614 -48615 -48616 -48617 -48618 -48619 -48620 -48621 -48622 -48623 -48624 -48625 -48626 -48627 -48628 -48629 -48630 -48631 -48632 -48633 -48634 -48635 -48636 -48637 -48638 -48639 -48640 -48641 -48642 -48643 -48644 -48645 -48646 -48647 -48648 -48649 -48650 -48651 -48652 -48653 -48654 -48655 -48656 -48657 -48658 -48659 -48660 -48661 -48662 -48663 -48664 -48665 -48666 -48667 -48668 -48669 -48670 -48671 -48672 -48673 -48674 -48675 -48676 -48677 -48678 -48679 -48680 -48681 -48682 -48683 -48684 -48685 -48686 -48687 -48688 -48689 -48690 -48691 -48692 -48693 -48694 -48695 -48696 -48697 -48698 -48699 -48700 -48701 -48702 -48703 -48704 -48705 -48706 -48707 -48708 -48709 -48710 -48711 -48712 -48713 -48714 -48715 -48716 -48717 -48718 -48719 -48720 -48721 -48722 -48723 -48724 -48725 -48726 -48727 -48728 -48729 -48730 -48731 -48732 -48733 -48734 -48735 -48736 -48737 -48738 -48739 -48740 -48741 -48742 -48743 -48744 -48745 -48746 -48747 -48748 -48749 -48750 -48751 -48752 -48753 -48754 -48755 -48756 -48757 -48758 -48759 -48760 -48761 -48762 -48763 -48764 -48765 -48766 -48767 -48768 -48769 -48770 -48771 -48772 -48773 -48774 -48775 -48776 -48777 -48778 -48779 -48780 -48781 -48782 -48783 -48784 -48785 -48786 -48787 -48788 -48789 -48790 -48791 -48792 -48793 -48794 -48795 -48796 -48797 -48798 -48799 -48800 -48801 -48802 -48803 -48804 -48805 -48806 -48807 -48808 -48809 -48810 -48811 -48812 -48813 -48814 -48815 -48816 -48817 -48818 -48819 -48820 -48821 -48822 -48823 -48824 -48825 -48826 -48827 -48828 -48829 -48830 -48831 -48832 -48833 -48834 -48835 -48836 -48837 -48838 -48839 -48840 -48841 -48842 -48843 -48844 -48845 -48846 -48847 -48848 -48849 -48850 -48851 -48852 -48853 -48854 -48855 -48856 -48857 -48858 -48859 -48860 -48861 -48862 -48863 -48864 -48865 -48866 -48867 -48868 -48869 -48870 -48871 -48872 -48873 -48874 -48875 -48876 -48877 -48878 -48879 -48880 -48881 -48882 -48883 -48884 -48885 -48886 -48887 -48888 -48889 -48890 -48891 -48892 -48893 -48894 -48895 -48896 -48897 -48898 -48899 -48900 -48901 -48902 -48903 -48904 -48905 -48906 -48907 -48908 -48909 -48910 -48911 -48912 -48913 -48914 -48915 -48916 -48917 -48918 -48919 -48920 -48921 -48922 -48923 -48924 -48925 -48926 -48927 -48928 -48929 -48930 -48931 -48932 -48933 -48934 -48935 -48936 -48937 -48938 -48939 -48940 -48941 -48942 -48943 -48944 -48945 -48946 -48947 -48948 -48949 -48950 -48951 -48952 -48953 -48954 -48955 -48956 -48957 -48958 -48959 -48960 -48961 -48962 -48963 -48964 -48965 -48966 -48967 -48968 -48969 -48970 -48971 -48972 -48973 -48974 -48975 -48976 -48977 -48978 -48979 -48980 -48981 -48982 -48983 -48984 -48985 -48986 -48987 -48988 -48989 -48990 -48991 -48992 -48993 -48994 -48995 -48996 -48997 -48998 -48999 -49000 -49001 -49002 -49003 -49004 -49005 -49006 -49007 -49008 -49009 -49010 -49011 -49012 -49013 -49014 -49015 -49016 -49017 -49018 -49019 -49020 -49021 -49022 -49023 -49024 -49025 -49026 -49027 -49028 -49029 -49030 -49031 -49032 -49033 -49034 -49035 -49036 -49037 -49038 -49039 -49040 -49041 -49042 -49043 -49044 -49045 -49046 -49047 -49048 -49049 -49050 -49051 -49052 -49053 -49054 -49055 -49056 -49057 -49058 -49059 -49060 -49061 -49062 -49063 -49064 -49065 -49066 -49067 -49068 -49069 -49070 -49071 -49072 -49073 -49074 -49075 -49076 -49077 -49078 -49079 -49080 -49081 -49082 -49083 -49084 -49085 -49086 -49087 -49088 -49089 -49090 -49091 -49092 -49093 -49094 -49095 -49096 -49097 -49098 -49099 -49100 -49101 -49102 -49103 -49104 -49105 -49106 -49107 -49108 -49109 -49110 -49111 -49112 -49113 -49114 -49115 -49116 -49117 -49118 -49119 -49120 -49121 -49122 -49123 -49124 -49125 -49126 -49127 -49128 -49129 -49130 -49131 -49132 -49133 -49134 -49135 -49136 -49137 -49138 -49139 -49140 -49141 -49142 -49143 -49144 -49145 -49146 -49147 -49148 -49149 -49150 -49151 -49152 -49153 -49154 -49155 -49156 -49157 -49158 -49159 -49160 -49161 -49162 -49163 -49164 -49165 -49166 -49167 -49168 -49169 -49170 -49171 -49172 -49173 -49174 -49175 -49176 -49177 -49178 -49179 -49180 -49181 -49182 -49183 -49184 -49185 -49186 -49187 -49188 -49189 -49190 -49191 -49192 -49193 -49194 -49195 -49196 -49197 -49198 -49199 -49200 -49201 -49202 -49203 -49204 -49205 -49206 -49207 -49208 -49209 -49210 -49211 -49212 -49213 -49214 -49215 -49216 -49217 -49218 -49219 -49220 -49221 -49222 -49223 -49224 -49225 -49226 -49227 -49228 -49229 -49230 -49231 -49232 -49233 -49234 -49235 -49236 -49237 -49238 -49239 -49240 -49241 -49242 -49243 -49244 -49245 -49246 -49247 -49248 -49249 -49250 -49251 -49252 -49253 -49254 -49255 -49256 -49257 -49258 -49259 -49260 -49261 -49262 -49263 -49264 -49265 -49266 -49267 -49268 -49269 -49270 -49271 -49272 -49273 -49274 -49275 -49276 -49277 -49278 -49279 -49280 -49281 -49282 -49283 -49284 -49285 -49286 -49287 -49288 -49289 -49290 -49291 -49292 -49293 -49294 -49295 -49296 -49297 -49298 -49299 -49300 -49301 -49302 -49303 -49304 -49305 -49306 -49307 -49308 -49309 -49310 -49311 -49312 -49313 -49314 -49315 -49316 -49317 -49318 -49319 -49320 -49321 -49322 -49323 -49324 -49325 -49326 -49327 -49328 -49329 -49330 -49331 -49332 -49333 -49334 -49335 -49336 -49337 -49338 -49339 -49340 -49341 -49342 -49343 -49344 -49345 -49346 -49347 -49348 -49349 -49350 -49351 -49352 -49353 -49354 -49355 -49356 -49357 -49358 -49359 -49360 -49361 -49362 -49363 -49364 -49365 -49366 -49367 -49368 -49369 -49370 -49371 -49372 -49373 -49374 -49375 -49376 -49377 -49378 -49379 -49380 -49381 -49382 -49383 -49384 -49385 -49386 -49387 -49388 -49389 -49390 -49391 -49392 -49393 -49394 -49395 -49396 -49397 -49398 -49399 -49400 -49401 -49402 -49403 -49404 -49405 -49406 -49407 -49408 -49409 -49410 -49411 -49412 -49413 -49414 -49415 -49416 -49417 -49418 -49419 -49420 -49421 -49422 -49423 -49424 -49425 -49426 -49427 -49428 -49429 -49430 -49431 -49432 -49433 -49434 -49435 -49436 -49437 -49438 -49439 -49440 -49441 -49442 -49443 -49444 -49445 -49446 -49447 -49448 -49449 -49450 -49451 -49452 -49453 -49454 -49455 -49456 -49457 -49458 -49459 -49460 -49461 -49462 -49463 -49464 -49465 -49466 -49467 -49468 -49469 -49470 -49471 -49472 -49473 -49474 -49475 -49476 -49477 -49478 -49479 -49480 -49481 -49482 -49483 -49484 -49485 -49486 -49487 -49488 -49489 -49490 -49491 -49492 -49493 -49494 -49495 -49496 -49497 -49498 -49499 -49500 -49501 -49502 -49503 -49504 -49505 -49506 -49507 -49508 -49509 -49510 -49511 -49512 -49513 -49514 -49515 -49516 -49517 -49518 -49519 -49520 -49521 -49522 -49523 -49524 -49525 -49526 -49527 -49528 -49529 -49530 -49531 -49532 -49533 -49534 -49535 -49536 -49537 -49538 -49539 -49540 -49541 -49542 -49543 -49544 -49545 -49546 -49547 -49548 -49549 -49550 -49551 -49552 -49553 -49554 -49555 -49556 -49557 -49558 -49559 -49560 -49561 -49562 -49563 -49564 -49565 -49566 -49567 -49568 -49569 -49570 -49571 -49572 -49573 -49574 -49575 -49576 -49577 -49578 -49579 -49580 -49581 -49582 -49583 -49584 -49585 -49586 -49587 -49588 -49589 -49590 -49591 -49592 -49593 -49594 -49595 -49596 -49597 -49598 -49599 -49600 -49601 -49602 -49603 -49604 -49605 -49606 -49607 -49608 -49609 -49610 -49611 -49612 -49613 -49614 -49615 -49616 -49617 -49618 -49619 -49620 -49621 -49622 -49623 -49624 -49625 -49626 -49627 -49628 -49629 -49630 -49631 -49632 -49633 -49634 -49635 -49636 -49637 -49638 -49639 -49640 -49641 -49642 -49643 -49644 -49645 -49646 -49647 -49648 -49649 -49650 -49651 -49652 -49653 -49654 -49655 -49656 -49657 -49658 -49659 -49660 -49661 -49662 -49663 -49664 -49665 -49666 -49667 -49668 -49669 -49670 -49671 -49672 -49673 -49674 -49675 -49676 -49677 -49678 -49679 -49680 -49681 -49682 -49683 -49684 -49685 -49686 -49687 -49688 -49689 -49690 -49691 -49692 -49693 -49694 -49695 -49696 -49697 -49698 -49699 -49700 -49701 -49702 -49703 -49704 -49705 -49706 -49707 -49708 -49709 -49710 -49711 -49712 -49713 -49714 -49715 -49716 -49717 -49718 -49719 -49720 -49721 -49722 -49723 -49724 -49725 -49726 -49727 -49728 -49729 -49730 -49731 -49732 -49733 -49734 -49735 -49736 -49737 -49738 -49739 -49740 -49741 -49742 -49743 -49744 -49745 -49746 -49747 -49748 -49749 -49750 -49751 -49752 -49753 -49754 -49755 -49756 -49757 -49758 -49759 -49760 -49761 -49762 -49763 -49764 -49765 -49766 -49767 -49768 -49769 -49770 -49771 -49772 -49773 -49774 -49775 -49776 -49777 -49778 -49779 -49780 -49781 -49782 -49783 -49784 -49785 -49786 -49787 -49788 -49789 -49790 -49791 -49792 -49793 -49794 -49795 -49796 -49797 -49798 -49799 -49800 -49801 -49802 -49803 -49804 -49805 -49806 -49807 -49808 -49809 -49810 -49811 -49812 -49813 -49814 -49815 -49816 -49817 -49818 -49819 -49820 -49821 -49822 -49823 -49824 -49825 -49826 -49827 -49828 -49829 -49830 -49831 -49832 -49833 -49834 -49835 -49836 -49837 -49838 -49839 -49840 -49841 -49842 -49843 -49844 -49845 -49846 -49847 -49848 -49849 -49850 -49851 -49852 -49853 -49854 -49855 -49856 -49857 -49858 -49859 -49860 -49861 -49862 -49863 -49864 -49865 -49866 -49867 -49868 -49869 -49870 -49871 -49872 -49873 -49874 -49875 -49876 -49877 -49878 -49879 -49880 -49881 -49882 -49883 -49884 -49885 -49886 -49887 -49888 -49889 -49890 -49891 -49892 -49893 -49894 -49895 -49896 -49897 -49898 -49899 -49900 -49901 -49902 -49903 -49904 -49905 -49906 -49907 -49908 -49909 -49910 -49911 -49912 -49913 -49914 -49915 -49916 -49917 -49918 -49919 -49920 -49921 -49922 -49923 -49924 -49925 -49926 -49927 -49928 -49929 -49930 -49931 -49932 -49933 -49934 -49935 -49936 -49937 -49938 -49939 -49940 -49941 -49942 -49943 -49944 -49945 -49946 -49947 -49948 -49949 -49950 -49951 -49952 -49953 -49954 -49955 -49956 -49957 -49958 -49959 -49960 -49961 -49962 -49963 -49964 -49965 -49966 -49967 -49968 -49969 -49970 -49971 -49972 -49973 -49974 -49975 -49976 -49977 -49978 -49979 -49980 -49981 -49982 -49983 -49984 -49985 -49986 -49987 -49988 -49989 -49990 -49991 -49992 -49993 -49994 -49995 -49996 -49997 -49998 -49999 -50000 -50001 -50002 -50003 -50004 -50005 -50006 -50007 -50008 -50009 -50010 -50011 -50012 -50013 -50014 -50015 -50016 -50017 -50018 -50019 -50020 -50021 -50022 -50023 -50024 -50025 -50026 -50027 -50028 -50029 -50030 -50031 -50032 -50033 -50034 -50035 -50036 -50037 -50038 -50039 -50040 -50041 -50042 -50043 -50044 -50045 -50046 -50047 -50048 -50049 -50050 -50051 -50052 -50053 -50054 -50055 -50056 -50057 -50058 -50059 -50060 -50061 -50062 -50063 -50064 -50065 -50066 -50067 -50068 -50069 -50070 -50071 -50072 -50073 -50074 -50075 -50076 -50077 -50078 -50079 -50080 -50081 -50082 -50083 -50084 -50085 -50086 -50087 -50088 -50089 -50090 -50091 -50092 -50093 -50094 -50095 -50096 -50097 -50098 -50099 -50100 -50101 -50102 -50103 -50104 -50105 -50106 -50107 -50108 -50109 -50110 -50111 -50112 -50113 -50114 -50115 -50116 -50117 -50118 -50119 -50120 -50121 -50122 -50123 -50124 -50125 -50126 -50127 -50128 -50129 -50130 -50131 -50132 -50133 -50134 -50135 -50136 -50137 -50138 -50139 -50140 -50141 -50142 -50143 -50144 -50145 -50146 -50147 -50148 -50149 -50150 -50151 -50152 -50153 -50154 -50155 -50156 -50157 -50158 -50159 -50160 -50161 -50162 -50163 -50164 -50165 -50166 -50167 -50168 -50169 -50170 -50171 -50172 -50173 -50174 -50175 -50176 -50177 -50178 -50179 -50180 -50181 -50182 -50183 -50184 -50185 -50186 -50187 -50188 -50189 -50190 -50191 -50192 -50193 -50194 -50195 -50196 -50197 -50198 -50199 -50200 -50201 -50202 -50203 -50204 -50205 -50206 -50207 -50208 -50209 -50210 -50211 -50212 -50213 -50214 -50215 -50216 -50217 -50218 -50219 -50220 -50221 -50222 -50223 -50224 -50225 -50226 -50227 -50228 -50229 -50230 -50231 -50232 -50233 -50234 -50235 -50236 -50237 -50238 -50239 -50240 -50241 -50242 -50243 -50244 -50245 -50246 -50247 -50248 -50249 -50250 -50251 -50252 -50253 -50254 -50255 -50256 -50257 -50258 -50259 -50260 -50261 -50262 -50263 -50264 -50265 -50266 -50267 -50268 -50269 -50270 -50271 -50272 -50273 -50274 -50275 -50276 -50277 -50278 -50279 -50280 -50281 -50282 -50283 -50284 -50285 -50286 -50287 -50288 -50289 -50290 -50291 -50292 -50293 -50294 -50295 -50296 -50297 -50298 -50299 -50300 -50301 -50302 -50303 -50304 -50305 -50306 -50307 -50308 -50309 -50310 -50311 -50312 -50313 -50314 -50315 -50316 -50317 -50318 -50319 -50320 -50321 -50322 -50323 -50324 -50325 -50326 -50327 -50328 -50329 -50330 -50331 -50332 -50333 -50334 -50335 -50336 -50337 -50338 -50339 -50340 -50341 -50342 -50343 -50344 -50345 -50346 -50347 -50348 -50349 -50350 -50351 -50352 -50353 -50354 -50355 -50356 -50357 -50358 -50359 -50360 -50361 -50362 -50363 -50364 -50365 -50366 -50367 -50368 -50369 -50370 -50371 -50372 -50373 -50374 -50375 -50376 -50377 -50378 -50379 -50380 -50381 -50382 -50383 -50384 -50385 -50386 -50387 -50388 -50389 -50390 -50391 -50392 -50393 -50394 -50395 -50396 -50397 -50398 -50399 -50400 -50401 -50402 -50403 -50404 -50405 -50406 -50407 -50408 -50409 -50410 -50411 -50412 -50413 -50414 -50415 -50416 -50417 -50418 -50419 -50420 -50421 -50422 -50423 -50424 -50425 -50426 -50427 -50428 -50429 -50430 -50431 -50432 -50433 -50434 -50435 -50436 -50437 -50438 -50439 -50440 -50441 -50442 -50443 -50444 -50445 -50446 -50447 -50448 -50449 -50450 -50451 -50452 -50453 -50454 -50455 -50456 -50457 -50458 -50459 -50460 -50461 -50462 -50463 -50464 -50465 -50466 -50467 -50468 -50469 -50470 -50471 -50472 -50473 -50474 -50475 -50476 -50477 -50478 -50479 -50480 -50481 -50482 -50483 -50484 -50485 -50486 -50487 -50488 -50489 -50490 -50491 -50492 -50493 -50494 -50495 -50496 -50497 -50498 -50499 -50500 -50501 -50502 -50503 -50504 -50505 -50506 -50507 -50508 -50509 -50510 -50511 -50512 -50513 -50514 -50515 -50516 -50517 -50518 -50519 -50520 -50521 -50522 -50523 -50524 -50525 -50526 -50527 -50528 -50529 -50530 -50531 -50532 -50533 -50534 -50535 -50536 -50537 -50538 -50539 -50540 -50541 -50542 -50543 -50544 -50545 -50546 -50547 -50548 -50549 -50550 -50551 -50552 -50553 -50554 -50555 -50556 -50557 -50558 -50559 -50560 -50561 -50562 -50563 -50564 -50565 -50566 -50567 -50568 -50569 -50570 -50571 -50572 -50573 -50574 -50575 -50576 -50577 -50578 -50579 -50580 -50581 -50582 -50583 -50584 -50585 -50586 -50587 -50588 -50589 -50590 -50591 -50592 -50593 -50594 -50595 -50596 -50597 -50598 -50599 -50600 -50601 -50602 -50603 -50604 -50605 -50606 -50607 -50608 -50609 -50610 -50611 -50612 -50613 -50614 -50615 -50616 -50617 -50618 -50619 -50620 -50621 -50622 -50623 -50624 -50625 -50626 -50627 -50628 -50629 -50630 -50631 -50632 -50633 -50634 -50635 -50636 -50637 -50638 -50639 -50640 -50641 -50642 -50643 -50644 -50645 -50646 -50647 -50648 -50649 -50650 -50651 -50652 -50653 -50654 -50655 -50656 -50657 -50658 -50659 -50660 -50661 -50662 -50663 -50664 -50665 -50666 -50667 -50668 -50669 -50670 -50671 -50672 -50673 -50674 -50675 -50676 -50677 -50678 -50679 -50680 -50681 -50682 -50683 -50684 -50685 -50686 -50687 -50688 -50689 -50690 -50691 -50692 -50693 -50694 -50695 -50696 -50697 -50698 -50699 -50700 -50701 -50702 -50703 -50704 -50705 -50706 -50707 -50708 -50709 -50710 -50711 -50712 -50713 -50714 -50715 -50716 -50717 -50718 -50719 -50720 -50721 -50722 -50723 -50724 -50725 -50726 -50727 -50728 -50729 -50730 -50731 -50732 -50733 -50734 -50735 -50736 -50737 -50738 -50739 -50740 -50741 -50742 -50743 -50744 -50745 -50746 -50747 -50748 -50749 -50750 -50751 -50752 -50753 -50754 -50755 -50756 -50757 -50758 -50759 -50760 -50761 -50762 -50763 -50764 -50765 -50766 -50767 -50768 -50769 -50770 -50771 -50772 -50773 -50774 -50775 -50776 -50777 -50778 -50779 -50780 -50781 -50782 -50783 -50784 -50785 -50786 -50787 -50788 -50789 -50790 -50791 -50792 -50793 -50794 -50795 -50796 -50797 -50798 -50799 -50800 -50801 -50802 -50803 -50804 -50805 -50806 -50807 -50808 -50809 -50810 -50811 -50812 -50813 -50814 -50815 -50816 -50817 -50818 -50819 -50820 -50821 -50822 -50823 -50824 -50825 -50826 -50827 -50828 -50829 -50830 -50831 -50832 -50833 -50834 -50835 -50836 -50837 -50838 -50839 -50840 -50841 -50842 -50843 -50844 -50845 -50846 -50847 -50848 -50849 -50850 -50851 -50852 -50853 -50854 -50855 -50856 -50857 -50858 -50859 -50860 -50861 -50862 -50863 -50864 -50865 -50866 -50867 -50868 -50869 -50870 -50871 -50872 -50873 -50874 -50875 -50876 -50877 -50878 -50879 -50880 -50881 -50882 -50883 -50884 -50885 -50886 -50887 -50888 -50889 -50890 -50891 -50892 -50893 -50894 -50895 -50896 -50897 -50898 -50899 -50900 -50901 -50902 -50903 -50904 -50905 -50906 -50907 -50908 -50909 -50910 -50911 -50912 -50913 -50914 -50915 -50916 -50917 -50918 -50919 -50920 -50921 -50922 -50923 -50924 -50925 -50926 -50927 -50928 -50929 -50930 -50931 -50932 -50933 -50934 -50935 -50936 -50937 -50938 -50939 -50940 -50941 -50942 -50943 -50944 -50945 -50946 -50947 -50948 -50949 -50950 -50951 -50952 -50953 -50954 -50955 -50956 -50957 -50958 -50959 -50960 -50961 -50962 -50963 -50964 -50965 -50966 -50967 -50968 -50969 -50970 -50971 -50972 -50973 -50974 -50975 -50976 -50977 -50978 -50979 -50980 -50981 -50982 -50983 -50984 -50985 -50986 -50987 -50988 -50989 -50990 -50991 -50992 -50993 -50994 -50995 -50996 -50997 -50998 -50999 -51000 -51001 -51002 -51003 -51004 -51005 -51006 -51007 -51008 -51009 -51010 -51011 -51012 -51013 -51014 -51015 -51016 -51017 -51018 -51019 -51020 -51021 -51022 -51023 -51024 -51025 -51026 -51027 -51028 -51029 -51030 -51031 -51032 -51033 -51034 -51035 -51036 -51037 -51038 -51039 -51040 -51041 -51042 -51043 -51044 -51045 -51046 -51047 -51048 -51049 -51050 -51051 -51052 -51053 -51054 -51055 -51056 -51057 -51058 -51059 -51060 -51061 -51062 -51063 -51064 -51065 -51066 -51067 -51068 -51069 -51070 -51071 -51072 -51073 -51074 -51075 -51076 -51077 -51078 -51079 -51080 -51081 -51082 -51083 -51084 -51085 -51086 -51087 -51088 -51089 -51090 -51091 -51092 -51093 -51094 -51095 -51096 -51097 -51098 -51099 -51100 -51101 -51102 -51103 -51104 -51105 -51106 -51107 -51108 -51109 -51110 -51111 -51112 -51113 -51114 -51115 -51116 -51117 -51118 -51119 -51120 -51121 -51122 -51123 -51124 -51125 -51126 -51127 -51128 -51129 -51130 -51131 -51132 -51133 -51134 -51135 -51136 -51137 -51138 -51139 -51140 -51141 -51142 -51143 -51144 -51145 -51146 -51147 -51148 -51149 -51150 -51151 -51152 -51153 -51154 -51155 -51156 -51157 -51158 -51159 -51160 -51161 -51162 -51163 -51164 -51165 -51166 -51167 -51168 -51169 -51170 -51171 -51172 -51173 -51174 -51175 -51176 -51177 -51178 -51179 -51180 -51181 -51182 -51183 -51184 -51185 -51186 -51187 -51188 -51189 -51190 -51191 -51192 -51193 -51194 -51195 -51196 -51197 -51198 -51199 -51200 -51201 -51202 -51203 -51204 -51205 -51206 -51207 -51208 -51209 -51210 -51211 -51212 -51213 -51214 -51215 -51216 -51217 -51218 -51219 -51220 -51221 -51222 -51223 -51224 -51225 -51226 -51227 -51228 -51229 -51230 -51231 -51232 -51233 -51234 -51235 -51236 -51237 -51238 -51239 -51240 -51241 -51242 -51243 -51244 -51245 -51246 -51247 -51248 -51249 -51250 -51251 -51252 -51253 -51254 -51255 -51256 -51257 -51258 -51259 -51260 -51261 -51262 -51263 -51264 -51265 -51266 -51267 -51268 -51269 -51270 -51271 -51272 -51273 -51274 -51275 -51276 -51277 -51278 -51279 -51280 -51281 -51282 -51283 -51284 -51285 -51286 -51287 -51288 -51289 -51290 -51291 -51292 -51293 -51294 -51295 -51296 -51297 -51298 -51299 -51300 -51301 -51302 -51303 -51304 -51305 -51306 -51307 -51308 -51309 -51310 -51311 -51312 -51313 -51314 -51315 -51316 -51317 -51318 -51319 -51320 -51321 -51322 -51323 -51324 -51325 -51326 -51327 -51328 -51329 -51330 -51331 -51332 -51333 -51334 -51335 -51336 -51337 -51338 -51339 -51340 -51341 -51342 -51343 -51344 -51345 -51346 -51347 -51348 -51349 -51350 -51351 -51352 -51353 -51354 -51355 -51356 -51357 -51358 -51359 -51360 -51361 -51362 -51363 -51364 -51365 -51366 -51367 -51368 -51369 -51370 -51371 -51372 -51373 -51374 -51375 -51376 -51377 -51378 -51379 -51380 -51381 -51382 -51383 -51384 -51385 -51386 -51387 -51388 -51389 -51390 -51391 -51392 -51393 -51394 -51395 -51396 -51397 -51398 -51399 -51400 -51401 -51402 -51403 -51404 -51405 -51406 -51407 -51408 -51409 -51410 -51411 -51412 -51413 -51414 -51415 -51416 -51417 -51418 -51419 -51420 -51421 -51422 -51423 -51424 -51425 -51426 -51427 -51428 -51429 -51430 -51431 -51432 -51433 -51434 -51435 -51436 -51437 -51438 -51439 -51440 -51441 -51442 -51443 -51444 -51445 -51446 -51447 -51448 -51449 -51450 -51451 -51452 -51453 -51454 -51455 -51456 -51457 -51458 -51459 -51460 -51461 -51462 -51463 -51464 -51465 -51466 -51467 -51468 -51469 -51470 -51471 -51472 -51473 -51474 -51475 -51476 -51477 -51478 -51479 -51480 -51481 -51482 -51483 -51484 -51485 -51486 -51487 -51488 -51489 -51490 -51491 -51492 -51493 -51494 -51495 -51496 -51497 -51498 -51499 -51500 -51501 -51502 -51503 -51504 -51505 -51506 -51507 -51508 -51509 -51510 -51511 -51512 -51513 -51514 -51515 -51516 -51517 -51518 -51519 -51520 -51521 -51522 -51523 -51524 -51525 -51526 -51527 -51528 -51529 -51530 -51531 -51532 -51533 -51534 -51535 -51536 -51537 -51538 -51539 -51540 -51541 -51542 -51543 -51544 -51545 -51546 -51547 -51548 -51549 -51550 -51551 -51552 -51553 -51554 -51555 -51556 -51557 -51558 -51559 -51560 -51561 -51562 -51563 -51564 -51565 -51566 -51567 -51568 -51569 -51570 -51571 -51572 -51573 -51574 -51575 -51576 -51577 -51578 -51579 -51580 -51581 -51582 -51583 -51584 -51585 -51586 -51587 -51588 -51589 -51590 -51591 -51592 -51593 -51594 -51595 -51596 -51597 -51598 -51599 -51600 -51601 -51602 -51603 -51604 -51605 -51606 -51607 -51608 -51609 -51610 -51611 -51612 -51613 -51614 -51615 -51616 -51617 -51618 -51619 -51620 -51621 -51622 -51623 -51624 -51625 -51626 -51627 -51628 -51629 -51630 -51631 -51632 -51633 -51634 -51635 -51636 -51637 -51638 -51639 -51640 -51641 -51642 -51643 -51644 -51645 -51646 -51647 -51648 -51649 -51650 -51651 -51652 -51653 -51654 -51655 -51656 -51657 -51658 -51659 -51660 -51661 -51662 -51663 -51664 -51665 -51666 -51667 -51668 -51669 -51670 -51671 -51672 -51673 -51674 -51675 -51676 -51677 -51678 -51679 -51680 -51681 -51682 -51683 -51684 -51685 -51686 -51687 -51688 -51689 -51690 -51691 -51692 -51693 -51694 -51695 -51696 -51697 -51698 -51699 -51700 -51701 -51702 -51703 -51704 -51705 -51706 -51707 -51708 -51709 -51710 -51711 -51712 -51713 -51714 -51715 -51716 -51717 -51718 -51719 -51720 -51721 -51722 -51723 -51724 -51725 -51726 -51727 -51728 -51729 -51730 -51731 -51732 -51733 -51734 -51735 -51736 -51737 -51738 -51739 -51740 -51741 -51742 -51743 -51744 -51745 -51746 -51747 -51748 -51749 -51750 -51751 -51752 -51753 -51754 -51755 -51756 -51757 -51758 -51759 -51760 -51761 -51762 -51763 -51764 -51765 -51766 -51767 -51768 -51769 -51770 -51771 -51772 -51773 -51774 -51775 -51776 -51777 -51778 -51779 -51780 -51781 -51782 -51783 -51784 -51785 -51786 -51787 -51788 -51789 -51790 -51791 -51792 -51793 -51794 -51795 -51796 -51797 -51798 -51799 -51800 -51801 -51802 -51803 -51804 -51805 -51806 -51807 -51808 -51809 -51810 -51811 -51812 -51813 -51814 -51815 -51816 -51817 -51818 -51819 -51820 -51821 -51822 -51823 -51824 -51825 -51826 -51827 -51828 -51829 -51830 -51831 -51832 -51833 -51834 -51835 -51836 -51837 -51838 -51839 -51840 -51841 -51842 -51843 -51844 -51845 -51846 -51847 -51848 -51849 -51850 -51851 -51852 -51853 -51854 -51855 -51856 -51857 -51858 -51859 -51860 -51861 -51862 -51863 -51864 -51865 -51866 -51867 -51868 -51869 -51870 -51871 -51872 -51873 -51874 -51875 -51876 -51877 -51878 -51879 -51880 -51881 -51882 -51883 -51884 -51885 -51886 -51887 -51888 -51889 -51890 -51891 -51892 -51893 -51894 -51895 -51896 -51897 -51898 -51899 -51900 -51901 -51902 -51903 -51904 -51905 -51906 -51907 -51908 -51909 -51910 -51911 -51912 -51913 -51914 -51915 -51916 -51917 -51918 -51919 -51920 -51921 -51922 -51923 -51924 -51925 -51926 -51927 -51928 -51929 -51930 -51931 -51932 -51933 -51934 -51935 -51936 -51937 -51938 -51939 -51940 -51941 -51942 -51943 -51944 -51945 -51946 -51947 -51948 -51949 -51950 -51951 -51952 -51953 -51954 -51955 -51956 -51957 -51958 -51959 -51960 -51961 -51962 -51963 -51964 -51965 -51966 -51967 -51968 -51969 -51970 -51971 -51972 -51973 -51974 -51975 -51976 -51977 -51978 -51979 -51980 -51981 -51982 -51983 -51984 -51985 -51986 -51987 -51988 -51989 -51990 -51991 -51992 -51993 -51994 -51995 -51996 -51997 -51998 -51999 -52000 -52001 -52002 -52003 -52004 -52005 -52006 -52007 -52008 -52009 -52010 -52011 -52012 -52013 -52014 -52015 -52016 -52017 -52018 -52019 -52020 -52021 -52022 -52023 -52024 -52025 -52026 -52027 -52028 -52029 -52030 -52031 -52032 -52033 -52034 -52035 -52036 -52037 -52038 -52039 -52040 -52041 -52042 -52043 -52044 -52045 -52046 -52047 -52048 -52049 -52050 -52051 -52052 -52053 -52054 -52055 -52056 -52057 -52058 -52059 -52060 -52061 -52062 -52063 -52064 -52065 -52066 -52067 -52068 -52069 -52070 -52071 -52072 -52073 -52074 -52075 -52076 -52077 -52078 -52079 -52080 -52081 -52082 -52083 -52084 -52085 -52086 -52087 -52088 -52089 -52090 -52091 -52092 -52093 -52094 -52095 -52096 -52097 -52098 -52099 -52100 -52101 -52102 -52103 -52104 -52105 -52106 -52107 -52108 -52109 -52110 -52111 -52112 -52113 -52114 -52115 -52116 -52117 -52118 -52119 -52120 -52121 -52122 -52123 -52124 -52125 -52126 -52127 -52128 -52129 -52130 -52131 -52132 -52133 -52134 -52135 -52136 -52137 -52138 -52139 -52140 -52141 -52142 -52143 -52144 -52145 -52146 -52147 -52148 -52149 -52150 -52151 -52152 -52153 -52154 -52155 -52156 -52157 -52158 -52159 -52160 -52161 -52162 -52163 -52164 -52165 -52166 -52167 -52168 -52169 -52170 -52171 -52172 -52173 -52174 -52175 -52176 -52177 -52178 -52179 -52180 -52181 -52182 -52183 -52184 -52185 -52186 -52187 -52188 -52189 -52190 -52191 -52192 -52193 -52194 -52195 -52196 -52197 -52198 -52199 -52200 -52201 -52202 -52203 -52204 -52205 -52206 -52207 -52208 -52209 -52210 -52211 -52212 -52213 -52214 -52215 -52216 -52217 -52218 -52219 -52220 -52221 -52222 -52223 -52224 -52225 -52226 -52227 -52228 -52229 -52230 -52231 -52232 -52233 -52234 -52235 -52236 -52237 -52238 -52239 -52240 -52241 -52242 -52243 -52244 -52245 -52246 -52247 -52248 -52249 -52250 -52251 -52252 -52253 -52254 -52255 -52256 -52257 -52258 -52259 -52260 -52261 -52262 -52263 -52264 -52265 -52266 -52267 -52268 -52269 -52270 -52271 -52272 -52273 -52274 -52275 -52276 -52277 -52278 -52279 -52280 -52281 -52282 -52283 -52284 -52285 -52286 -52287 -52288 -52289 -52290 -52291 -52292 -52293 -52294 -52295 -52296 -52297 -52298 -52299 -52300 -52301 -52302 -52303 -52304 -52305 -52306 -52307 -52308 -52309 -52310 -52311 -52312 -52313 -52314 -52315 -52316 -52317 -52318 -52319 -52320 -52321 -52322 -52323 -52324 -52325 -52326 -52327 -52328 -52329 -52330 -52331 -52332 -52333 -52334 -52335 -52336 -52337 -52338 -52339 -52340 -52341 -52342 -52343 -52344 -52345 -52346 -52347 -52348 -52349 -52350 -52351 -52352 -52353 -52354 -52355 -52356 -52357 -52358 -52359 -52360 -52361 -52362 -52363 -52364 -52365 -52366 -52367 -52368 -52369 -52370 -52371 -52372 -52373 -52374 -52375 -52376 -52377 -52378 -52379 -52380 -52381 -52382 -52383 -52384 -52385 -52386 -52387 -52388 -52389 -52390 -52391 -52392 -52393 -52394 -52395 -52396 -52397 -52398 -52399 -52400 -52401 -52402 -52403 -52404 -52405 -52406 -52407 -52408 -52409 -52410 -52411 -52412 -52413 -52414 -52415 -52416 -52417 -52418 -52419 -52420 -52421 -52422 -52423 -52424 -52425 -52426 -52427 -52428 -52429 -52430 -52431 -52432 -52433 -52434 -52435 -52436 -52437 -52438 -52439 -52440 -52441 -52442 -52443 -52444 -52445 -52446 -52447 -52448 -52449 -52450 -52451 -52452 -52453 -52454 -52455 -52456 -52457 -52458 -52459 -52460 -52461 -52462 -52463 -52464 -52465 -52466 -52467 -52468 -52469 -52470 -52471 -52472 -52473 -52474 -52475 -52476 -52477 -52478 -52479 -52480 -52481 -52482 -52483 -52484 -52485 -52486 -52487 -52488 -52489 -52490 -52491 -52492 -52493 -52494 -52495 -52496 -52497 -52498 -52499 -52500 -52501 -52502 -52503 -52504 -52505 -52506 -52507 -52508 -52509 -52510 -52511 -52512 -52513 -52514 -52515 -52516 -52517 -52518 -52519 -52520 -52521 -52522 -52523 -52524 -52525 -52526 -52527 -52528 -52529 -52530 -52531 -52532 -52533 -52534 -52535 -52536 -52537 -52538 -52539 -52540 -52541 -52542 -52543 -52544 -52545 -52546 -52547 -52548 -52549 -52550 -52551 -52552 -52553 -52554 -52555 -52556 -52557 -52558 -52559 -52560 -52561 -52562 -52563 -52564 -52565 -52566 -52567 -52568 -52569 -52570 -52571 -52572 -52573 -52574 -52575 -52576 -52577 -52578 -52579 -52580 -52581 -52582 -52583 -52584 -52585 -52586 -52587 -52588 -52589 -52590 -52591 -52592 -52593 -52594 -52595 -52596 -52597 -52598 -52599 -52600 -52601 -52602 -52603 -52604 -52605 -52606 -52607 -52608 -52609 -52610 -52611 -52612 -52613 -52614 -52615 -52616 -52617 -52618 -52619 -52620 -52621 -52622 -52623 -52624 -52625 -52626 -52627 -52628 -52629 -52630 -52631 -52632 -52633 -52634 -52635 -52636 -52637 -52638 -52639 -52640 -52641 -52642 -52643 -52644 -52645 -52646 -52647 -52648 -52649 -52650 -52651 -52652 -52653 -52654 -52655 -52656 -52657 -52658 -52659 -52660 -52661 -52662 -52663 -52664 -52665 -52666 -52667 -52668 -52669 -52670 -52671 -52672 -52673 -52674 -52675 -52676 -52677 -52678 -52679 -52680 -52681 -52682 -52683 -52684 -52685 -52686 -52687 -52688 -52689 -52690 -52691 -52692 -52693 -52694 -52695 -52696 -52697 -52698 -52699 -52700 -52701 -52702 -52703 -52704 -52705 -52706 -52707 -52708 -52709 -52710 -52711 -52712 -52713 -52714 -52715 -52716 -52717 -52718 -52719 -52720 -52721 -52722 -52723 -52724 -52725 -52726 -52727 -52728 -52729 -52730 -52731 -52732 -52733 -52734 -52735 -52736 -52737 -52738 -52739 -52740 -52741 -52742 -52743 -52744 -52745 -52746 -52747 -52748 -52749 -52750 -52751 -52752 -52753 -52754 -52755 -52756 -52757 -52758 -52759 -52760 -52761 -52762 -52763 -52764 -52765 -52766 -52767 -52768 -52769 -52770 -52771 -52772 -52773 -52774 -52775 -52776 -52777 -52778 -52779 -52780 -52781 -52782 -52783 -52784 -52785 -52786 -52787 -52788 -52789 -52790 -52791 -52792 -52793 -52794 -52795 -52796 -52797 -52798 -52799 -52800 -52801 -52802 -52803 -52804 -52805 -52806 -52807 -52808 -52809 -52810 -52811 -52812 -52813 -52814 -52815 -52816 -52817 -52818 -52819 -52820 -52821 -52822 -52823 -52824 -52825 -52826 -52827 -52828 -52829 -52830 -52831 -52832 -52833 -52834 -52835 -52836 -52837 -52838 -52839 -52840 -52841 -52842 -52843 -52844 -52845 -52846 -52847 -52848 -52849 -52850 -52851 -52852 -52853 -52854 -52855 -52856 -52857 -52858 -52859 -52860 -52861 -52862 -52863 -52864 -52865 -52866 -52867 -52868 -52869 -52870 -52871 -52872 -52873 -52874 -52875 -52876 -52877 -52878 -52879 -52880 -52881 -52882 -52883 -52884 -52885 -52886 -52887 -52888 -52889 -52890 -52891 -52892 -52893 -52894 -52895 -52896 -52897 -52898 -52899 -52900 -52901 -52902 -52903 -52904 -52905 -52906 -52907 -52908 -52909 -52910 -52911 -52912 -52913 -52914 -52915 -52916 -52917 -52918 -52919 -52920 -52921 -52922 -52923 -52924 -52925 -52926 -52927 -52928 -52929 -52930 -52931 -52932 -52933 -52934 -52935 -52936 -52937 -52938 -52939 -52940 -52941 -52942 -52943 -52944 -52945 -52946 -52947 -52948 -52949 -52950 -52951 -52952 -52953 -52954 -52955 -52956 -52957 -52958 -52959 -52960 -52961 -52962 -52963 -52964 -52965 -52966 -52967 -52968 -52969 -52970 -52971 -52972 -52973 -52974 -52975 -52976 -52977 -52978 -52979 -52980 -52981 -52982 -52983 -52984 -52985 -52986 -52987 -52988 -52989 -52990 -52991 -52992 -52993 -52994 -52995 -52996 -52997 -52998 -52999 -53000 -53001 -53002 -53003 -53004 -53005 -53006 -53007 -53008 -53009 -53010 -53011 -53012 -53013 -53014 -53015 -53016 -53017 -53018 -53019 -53020 -53021 -53022 -53023 -53024 -53025 -53026 -53027 -53028 -53029 -53030 -53031 -53032 -53033 -53034 -53035 -53036 -53037 -53038 -53039 -53040 -53041 -53042 -53043 -53044 -53045 -53046 -53047 -53048 -53049 -53050 -53051 -53052 -53053 -53054 -53055 -53056 -53057 -53058 -53059 -53060 -53061 -53062 -53063 -53064 -53065 -53066 -53067 -53068 -53069 -53070 -53071 -53072 -53073 -53074 -53075 -53076 -53077 -53078 -53079 -53080 -53081 -53082 -53083 -53084 -53085 -53086 -53087 -53088 -53089 -53090 -53091 -53092 -53093 -53094 -53095 -53096 -53097 -53098 -53099 -53100 -53101 -53102 -53103 -53104 -53105 -53106 -53107 -53108 -53109 -53110 -53111 -53112 -53113 -53114 -53115 -53116 -53117 -53118 -53119 -53120 -53121 -53122 -53123 -53124 -53125 -53126 -53127 -53128 -53129 -53130 -53131 -53132 -53133 -53134 -53135 -53136 -53137 -53138 -53139 -53140 -53141 -53142 -53143 -53144 -53145 -53146 -53147 -53148 -53149 -53150 -53151 -53152 -53153 -53154 -53155 -53156 -53157 -53158 -53159 -53160 -53161 -53162 -53163 -53164 -53165 -53166 -53167 -53168 -53169 -53170 -53171 -53172 -53173 -53174 -53175 -53176 -53177 -53178 -53179 -53180 -53181 -53182 -53183 -53184 -53185 -53186 -53187 -53188 -53189 -53190 -53191 -53192 -53193 -53194 -53195 -53196 -53197 -53198 -53199 -53200 -53201 -53202 -53203 -53204 -53205 -53206 -53207 -53208 -53209 -53210 -53211 -53212 -53213 -53214 -53215 -53216 -53217 -53218 -53219 -53220 -53221 -53222 -53223 -53224 -53225 -53226 -53227 -53228 -53229 -53230 -53231 -53232 -53233 -53234 -53235 -53236 -53237 -53238 -53239 -53240 -53241 -53242 -53243 -53244 -53245 -53246 -53247 -53248 -53249 -53250 -53251 -53252 -53253 -53254 -53255 -53256 -53257 -53258 -53259 -53260 -53261 -53262 -53263 -53264 -53265 -53266 -53267 -53268 -53269 -53270 -53271 -53272 -53273 -53274 -53275 -53276 -53277 -53278 -53279 -53280 -53281 -53282 -53283 -53284 -53285 -53286 -53287 -53288 -53289 -53290 -53291 -53292 -53293 -53294 -53295 -53296 -53297 -53298 -53299 -53300 -53301 -53302 -53303 -53304 -53305 -53306 -53307 -53308 -53309 -53310 -53311 -53312 -53313 -53314 -53315 -53316 -53317 -53318 -53319 -53320 -53321 -53322 -53323 -53324 -53325 -53326 -53327 -53328 -53329 -53330 -53331 -53332 -53333 -53334 -53335 -53336 -53337 -53338 -53339 -53340 -53341 -53342 -53343 -53344 -53345 -53346 -53347 -53348 -53349 -53350 -53351 -53352 -53353 -53354 -53355 -53356 -53357 -53358 -53359 -53360 -53361 -53362 -53363 -53364 -53365 -53366 -53367 -53368 -53369 -53370 -53371 -53372 -53373 -53374 -53375 -53376 -53377 -53378 -53379 -53380 -53381 -53382 -53383 -53384 -53385 -53386 -53387 -53388 -53389 -53390 -53391 -53392 -53393 -53394 -53395 -53396 -53397 -53398 -53399 -53400 -53401 -53402 -53403 -53404 -53405 -53406 -53407 -53408 -53409 -53410 -53411 -53412 -53413 -53414 -53415 -53416 -53417 -53418 -53419 -53420 -53421 -53422 -53423 -53424 -53425 -53426 -53427 -53428 -53429 -53430 -53431 -53432 -53433 -53434 -53435 -53436 -53437 -53438 -53439 -53440 -53441 -53442 -53443 -53444 -53445 -53446 -53447 -53448 -53449 -53450 -53451 -53452 -53453 -53454 -53455 -53456 -53457 -53458 -53459 -53460 -53461 -53462 -53463 -53464 -53465 -53466 -53467 -53468 -53469 -53470 -53471 -53472 -53473 -53474 -53475 -53476 -53477 -53478 -53479 -53480 -53481 -53482 -53483 -53484 -53485 -53486 -53487 -53488 -53489 -53490 -53491 -53492 -53493 -53494 -53495 -53496 -53497 -53498 -53499 -53500 -53501 -53502 -53503 -53504 -53505 -53506 -53507 -53508 -53509 -53510 -53511 -53512 -53513 -53514 -53515 -53516 -53517 -53518 -53519 -53520 -53521 -53522 -53523 -53524 -53525 -53526 -53527 -53528 -53529 -53530 -53531 -53532 -53533 -53534 -53535 -53536 -53537 -53538 -53539 -53540 -53541 -53542 -53543 -53544 -53545 -53546 -53547 -53548 -53549 -53550 -53551 -53552 -53553 -53554 -53555 -53556 -53557 -53558 -53559 -53560 -53561 -53562 -53563 -53564 -53565 -53566 -53567 -53568 -53569 -53570 -53571 -53572 -53573 -53574 -53575 -53576 -53577 -53578 -53579 -53580 -53581 -53582 -53583 -53584 -53585 -53586 -53587 -53588 -53589 -53590 -53591 -53592 -53593 -53594 -53595 -53596 -53597 -53598 -53599 -53600 -53601 -53602 -53603 -53604 -53605 -53606 -53607 -53608 -53609 -53610 -53611 -53612 -53613 -53614 -53615 -53616 -53617 -53618 -53619 -53620 -53621 -53622 -53623 -53624 -53625 -53626 -53627 -53628 -53629 -53630 -53631 -53632 -53633 -53634 -53635 -53636 -53637 -53638 -53639 -53640 -53641 -53642 -53643 -53644 -53645 -53646 -53647 -53648 -53649 -53650 -53651 -53652 -53653 -53654 -53655 -53656 -53657 -53658 -53659 -53660 -53661 -53662 -53663 -53664 -53665 -53666 -53667 -53668 -53669 -53670 -53671 -53672 -53673 -53674 -53675 -53676 -53677 -53678 -53679 -53680 -53681 -53682 -53683 -53684 -53685 -53686 -53687 -53688 -53689 -53690 -53691 -53692 -53693 -53694 -53695 -53696 -53697 -53698 -53699 -53700 -53701 -53702 -53703 -53704 -53705 -53706 -53707 -53708 -53709 -53710 -53711 -53712 -53713 -53714 -53715 -53716 -53717 -53718 -53719 -53720 -53721 -53722 -53723 -53724 -53725 -53726 -53727 -53728 -53729 -53730 -53731 -53732 -53733 -53734 -53735 -53736 -53737 -53738 -53739 -53740 -53741 -53742 -53743 -53744 -53745 -53746 -53747 -53748 -53749 -53750 -53751 -53752 -53753 -53754 -53755 -53756 -53757 -53758 -53759 -53760 -53761 -53762 -53763 -53764 -53765 -53766 -53767 -53768 -53769 -53770 -53771 -53772 -53773 -53774 -53775 -53776 -53777 -53778 -53779 -53780 -53781 -53782 -53783 -53784 -53785 -53786 -53787 -53788 -53789 -53790 -53791 -53792 -53793 -53794 -53795 -53796 -53797 -53798 -53799 -53800 -53801 -53802 -53803 -53804 -53805 -53806 -53807 -53808 -53809 -53810 -53811 -53812 -53813 -53814 -53815 -53816 -53817 -53818 -53819 -53820 -53821 -53822 -53823 -53824 -53825 -53826 -53827 -53828 -53829 -53830 -53831 -53832 -53833 -53834 -53835 -53836 -53837 -53838 -53839 -53840 -53841 -53842 -53843 -53844 -53845 -53846 -53847 -53848 -53849 -53850 -53851 -53852 -53853 -53854 -53855 -53856 -53857 -53858 -53859 -53860 -53861 -53862 -53863 -53864 -53865 -53866 -53867 -53868 -53869 -53870 -53871 -53872 -53873 -53874 -53875 -53876 -53877 -53878 -53879 -53880 -53881 -53882 -53883 -53884 -53885 -53886 -53887 -53888 -53889 -53890 -53891 -53892 -53893 -53894 -53895 -53896 -53897 -53898 -53899 -53900 -53901 -53902 -53903 -53904 -53905 -53906 -53907 -53908 -53909 -53910 -53911 -53912 -53913 -53914 -53915 -53916 -53917 -53918 -53919 -53920 -53921 -53922 -53923 -53924 -53925 -53926 -53927 -53928 -53929 -53930 -53931 -53932 -53933 -53934 -53935 -53936 -53937 -53938 -53939 -53940 -53941 -53942 -53943 -53944 -53945 -53946 -53947 -53948 -53949 -53950 -53951 -53952 -53953 -53954 -53955 -53956 -53957 -53958 -53959 -53960 -53961 -53962 -53963 -53964 -53965 -53966 -53967 -53968 -53969 -53970 -53971 -53972 -53973 -53974 -53975 -53976 -53977 -53978 -53979 -53980 -53981 -53982 -53983 -53984 -53985 -53986 -53987 -53988 -53989 -53990 -53991 -53992 -53993 -53994 -53995 -53996 -53997 -53998 -53999 -54000 -54001 -54002 -54003 -54004 -54005 -54006 -54007 -54008 -54009 -54010 -54011 -54012 -54013 -54014 -54015 -54016 -54017 -54018 -54019 -54020 -54021 -54022 -54023 -54024 -54025 -54026 -54027 -54028 -54029 -54030 -54031 -54032 -54033 -54034 -54035 -54036 -54037 -54038 -54039 -54040 -54041 -54042 -54043 -54044 -54045 -54046 -54047 -54048 -54049 -54050 -54051 -54052 -54053 -54054 -54055 -54056 -54057 -54058 -54059 -54060 -54061 -54062 -54063 -54064 -54065 -54066 -54067 -54068 -54069 -54070 -54071 -54072 -54073 -54074 -54075 -54076 -54077 -54078 -54079 -54080 -54081 -54082 -54083 -54084 -54085 -54086 -54087 -54088 -54089 -54090 -54091 -54092 -54093 -54094 -54095 -54096 -54097 -54098 -54099 -54100 -54101 -54102 -54103 -54104 -54105 -54106 -54107 -54108 -54109 -54110 -54111 -54112 -54113 -54114 -54115 -54116 -54117 -54118 -54119 -54120 -54121 -54122 -54123 -54124 -54125 -54126 -54127 -54128 -54129 -54130 -54131 -54132 -54133 -54134 -54135 -54136 -54137 -54138 -54139 -54140 -54141 -54142 -54143 -54144 -54145 -54146 -54147 -54148 -54149 -54150 -54151 -54152 -54153 -54154 -54155 -54156 -54157 -54158 -54159 -54160 -54161 -54162 -54163 -54164 -54165 -54166 -54167 -54168 -54169 -54170 -54171 -54172 -54173 -54174 -54175 -54176 -54177 -54178 -54179 -54180 -54181 -54182 -54183 -54184 -54185 -54186 -54187 -54188 -54189 -54190 -54191 -54192 -54193 -54194 -54195 -54196 -54197 -54198 -54199 -54200 -54201 -54202 -54203 -54204 -54205 -54206 -54207 -54208 -54209 -54210 -54211 -54212 -54213 -54214 -54215 -54216 -54217 -54218 -54219 -54220 -54221 -54222 -54223 -54224 -54225 -54226 -54227 -54228 -54229 -54230 -54231 -54232 -54233 -54234 -54235 -54236 -54237 -54238 -54239 -54240 -54241 -54242 -54243 -54244 -54245 -54246 -54247 -54248 -54249 -54250 -54251 -54252 -54253 -54254 -54255 -54256 -54257 -54258 -54259 -54260 -54261 -54262 -54263 -54264 -54265 -54266 -54267 -54268 -54269 -54270 -54271 -54272 -54273 -54274 -54275 -54276 -54277 -54278 -54279 -54280 -54281 -54282 -54283 -54284 -54285 -54286 -54287 -54288 -54289 -54290 -54291 -54292 -54293 -54294 -54295 -54296 -54297 -54298 -54299 -54300 -54301 -54302 -54303 -54304 -54305 -54306 -54307 -54308 -54309 -54310 -54311 -54312 -54313 -54314 -54315 -54316 -54317 -54318 -54319 -54320 -54321 -54322 -54323 -54324 -54325 -54326 -54327 -54328 -54329 -54330 -54331 -54332 -54333 -54334 -54335 -54336 -54337 -54338 -54339 -54340 -54341 -54342 -54343 -54344 -54345 -54346 -54347 -54348 -54349 -54350 -54351 -54352 -54353 -54354 -54355 -54356 -54357 -54358 -54359 -54360 -54361 -54362 -54363 -54364 -54365 -54366 -54367 -54368 -54369 -54370 -54371 -54372 -54373 -54374 -54375 -54376 -54377 -54378 -54379 -54380 -54381 -54382 -54383 -54384 -54385 -54386 -54387 -54388 -54389 -54390 -54391 -54392 -54393 -54394 -54395 -54396 -54397 -54398 -54399 -54400 -54401 -54402 -54403 -54404 -54405 -54406 -54407 -54408 -54409 -54410 -54411 -54412 -54413 -54414 -54415 -54416 -54417 -54418 -54419 -54420 -54421 -54422 -54423 -54424 -54425 -54426 -54427 -54428 -54429 -54430 -54431 -54432 -54433 -54434 -54435 -54436 -54437 -54438 -54439 -54440 -54441 -54442 -54443 -54444 -54445 -54446 -54447 -54448 -54449 -54450 -54451 -54452 -54453 -54454 -54455 -54456 -54457 -54458 -54459 -54460 -54461 -54462 -54463 -54464 -54465 -54466 -54467 -54468 -54469 -54470 -54471 -54472 -54473 -54474 -54475 -54476 -54477 -54478 -54479 -54480 -54481 -54482 -54483 -54484 -54485 -54486 -54487 -54488 -54489 -54490 -54491 -54492 -54493 -54494 -54495 -54496 -54497 -54498 -54499 -54500 -54501 -54502 -54503 -54504 -54505 -54506 -54507 -54508 -54509 -54510 -54511 -54512 -54513 -54514 -54515 -54516 -54517 -54518 -54519 -54520 -54521 -54522 -54523 -54524 -54525 -54526 -54527 -54528 -54529 -54530 -54531 -54532 -54533 -54534 -54535 -54536 -54537 -54538 -54539 -54540 -54541 -54542 -54543 -54544 -54545 -54546 -54547 -54548 -54549 -54550 -54551 -54552 -54553 -54554 -54555 -54556 -54557 -54558 -54559 -54560 -54561 -54562 -54563 -54564 -54565 -54566 -54567 -54568 -54569 -54570 -54571 -54572 -54573 -54574 -54575 -54576 -54577 -54578 -54579 -54580 -54581 -54582 -54583 -54584 -54585 -54586 -54587 -54588 -54589 -54590 -54591 -54592 -54593 -54594 -54595 -54596 -54597 -54598 -54599 -54600 -54601 -54602 -54603 -54604 -54605 -54606 -54607 -54608 -54609 -54610 -54611 -54612 -54613 -54614 -54615 -54616 -54617 -54618 -54619 -54620 -54621 -54622 -54623 -54624 -54625 -54626 -54627 -54628 -54629 -54630 -54631 -54632 -54633 -54634 -54635 -54636 -54637 -54638 -54639 -54640 -54641 -54642 -54643 -54644 -54645 -54646 -54647 -54648 -54649 -54650 -54651 -54652 -54653 -54654 -54655 -54656 -54657 -54658 -54659 -54660 -54661 -54662 -54663 -54664 -54665 -54666 -54667 -54668 -54669 -54670 -54671 -54672 -54673 -54674 -54675 -54676 -54677 -54678 -54679 -54680 -54681 -54682 -54683 -54684 -54685 -54686 -54687 -54688 -54689 -54690 -54691 -54692 -54693 -54694 -54695 -54696 -54697 -54698 -54699 -54700 -54701 -54702 -54703 -54704 -54705 -54706 -54707 -54708 -54709 -54710 -54711 -54712 -54713 -54714 -54715 -54716 -54717 -54718 -54719 -54720 -54721 -54722 -54723 -54724 -54725 -54726 -54727 -54728 -54729 -54730 -54731 -54732 -54733 -54734 -54735 -54736 -54737 -54738 -54739 -54740 -54741 -54742 -54743 -54744 -54745 -54746 -54747 -54748 -54749 -54750 -54751 -54752 -54753 -54754 -54755 -54756 -54757 -54758 -54759 -54760 -54761 -54762 -54763 -54764 -54765 -54766 -54767 -54768 -54769 -54770 -54771 -54772 -54773 -54774 -54775 -54776 -54777 -54778 -54779 -54780 -54781 -54782 -54783 -54784 -54785 -54786 -54787 -54788 -54789 -54790 -54791 -54792 -54793 -54794 -54795 -54796 -54797 -54798 -54799 -54800 -54801 -54802 -54803 -54804 -54805 -54806 -54807 -54808 -54809 -54810 -54811 -54812 -54813 -54814 -54815 -54816 -54817 -54818 -54819 -54820 -54821 -54822 -54823 -54824 -54825 -54826 -54827 -54828 -54829 -54830 -54831 -54832 -54833 -54834 -54835 -54836 -54837 -54838 -54839 -54840 -54841 -54842 -54843 -54844 -54845 -54846 -54847 -54848 -54849 -54850 -54851 -54852 -54853 -54854 -54855 -54856 -54857 -54858 -54859 -54860 -54861 -54862 -54863 -54864 -54865 -54866 -54867 -54868 -54869 -54870 -54871 -54872 -54873 -54874 -54875 -54876 -54877 -54878 -54879 -54880 -54881 -54882 -54883 -54884 -54885 -54886 -54887 -54888 -54889 -54890 -54891 -54892 -54893 -54894 -54895 -54896 -54897 -54898 -54899 -54900 -54901 -54902 -54903 -54904 -54905 -54906 -54907 -54908 -54909 -54910 -54911 -54912 -54913 -54914 -54915 -54916 -54917 -54918 -54919 -54920 -54921 -54922 -54923 -54924 -54925 -54926 -54927 -54928 -54929 -54930 -54931 -54932 -54933 -54934 -54935 -54936 -54937 -54938 -54939 -54940 -54941 -54942 -54943 -54944 -54945 -54946 -54947 -54948 -54949 -54950 -54951 -54952 -54953 -54954 -54955 -54956 -54957 -54958 -54959 -54960 -54961 -54962 -54963 -54964 -54965 -54966 -54967 -54968 -54969 -54970 -54971 -54972 -54973 -54974 -54975 -54976 -54977 -54978 -54979 -54980 -54981 -54982 -54983 -54984 -54985 -54986 -54987 -54988 -54989 -54990 -54991 -54992 -54993 -54994 -54995 -54996 -54997 -54998 -54999 -55000 -55001 -55002 -55003 -55004 -55005 -55006 -55007 -55008 -55009 -55010 -55011 -55012 -55013 -55014 -55015 -55016 -55017 -55018 -55019 -55020 -55021 -55022 -55023 -55024 -55025 -55026 -55027 -55028 -55029 -55030 -55031 -55032 -55033 -55034 -55035 -55036 -55037 -55038 -55039 -55040 -55041 -55042 -55043 -55044 -55045 -55046 -55047 -55048 -55049 -55050 -55051 -55052 -55053 -55054 -55055 -55056 -55057 -55058 -55059 -55060 -55061 -55062 -55063 -55064 -55065 -55066 -55067 -55068 -55069 -55070 -55071 -55072 -55073 -55074 -55075 -55076 -55077 -55078 -55079 -55080 -55081 -55082 -55083 -55084 -55085 -55086 -55087 -55088 -55089 -55090 -55091 -55092 -55093 -55094 -55095 -55096 -55097 -55098 -55099 -55100 -55101 -55102 -55103 -55104 -55105 -55106 -55107 -55108 -55109 -55110 -55111 -55112 -55113 -55114 -55115 -55116 -55117 -55118 -55119 -55120 -55121 -55122 -55123 -55124 -55125 -55126 -55127 -55128 -55129 -55130 -55131 -55132 -55133 -55134 -55135 -55136 -55137 -55138 -55139 -55140 -55141 -55142 -55143 -55144 -55145 -55146 -55147 -55148 -55149 -55150 -55151 -55152 -55153 -55154 -55155 -55156 -55157 -55158 -55159 -55160 -55161 -55162 -55163 -55164 -55165 -55166 -55167 -55168 -55169 -55170 -55171 -55172 -55173 -55174 -55175 -55176 -55177 -55178 -55179 -55180 -55181 -55182 -55183 -55184 -55185 -55186 -55187 -55188 -55189 -55190 -55191 -55192 -55193 -55194 -55195 -55196 -55197 -55198 -55199 -55200 -55201 -55202 -55203 -55204 -55205 -55206 -55207 -55208 -55209 -55210 -55211 -55212 -55213 -55214 -55215 -55216 -55217 -55218 -55219 -55220 -55221 -55222 -55223 -55224 -55225 -55226 -55227 -55228 -55229 -55230 -55231 -55232 -55233 -55234 -55235 -55236 -55237 -55238 -55239 -55240 -55241 -55242 -55243 -55244 -55245 -55246 -55247 -55248 -55249 -55250 -55251 -55252 -55253 -55254 -55255 -55256 -55257 -55258 -55259 -55260 -55261 -55262 -55263 -55264 -55265 -55266 -55267 -55268 -55269 -55270 -55271 -55272 -55273 -55274 -55275 -55276 -55277 -55278 -55279 -55280 -55281 -55282 -55283 -55284 -55285 -55286 -55287 -55288 -55289 -55290 -55291 -55292 -55293 -55294 -55295 -55296 -55297 -55298 -55299 -55300 -55301 -55302 -55303 -55304 -55305 -55306 -55307 -55308 -55309 -55310 -55311 -55312 -55313 -55314 -55315 -55316 -55317 -55318 -55319 -55320 -55321 -55322 -55323 -55324 -55325 -55326 -55327 -55328 -55329 -55330 -55331 -55332 -55333 -55334 -55335 -55336 -55337 -55338 -55339 -55340 -55341 -55342 -55343 -55344 -55345 -55346 -55347 -55348 -55349 -55350 -55351 -55352 -55353 -55354 -55355 -55356 -55357 -55358 -55359 -55360 -55361 -55362 -55363 -55364 -55365 -55366 -55367 -55368 -55369 -55370 -55371 -55372 -55373 -55374 -55375 -55376 -55377 -55378 -55379 -55380 -55381 -55382 -55383 -55384 -55385 -55386 -55387 -55388 -55389 -55390 -55391 -55392 -55393 -55394 -55395 -55396 -55397 -55398 -55399 -55400 -55401 -55402 -55403 -55404 -55405 -55406 -55407 -55408 -55409 -55410 -55411 -55412 -55413 -55414 -55415 -55416 -55417 -55418 -55419 -55420 -55421 -55422 -55423 -55424 -55425 -55426 -55427 -55428 -55429 -55430 -55431 -55432 -55433 -55434 -55435 -55436 -55437 -55438 -55439 -55440 -55441 -55442 -55443 -55444 -55445 -55446 -55447 -55448 -55449 -55450 -55451 -55452 -55453 -55454 -55455 -55456 -55457 -55458 -55459 -55460 -55461 -55462 -55463 -55464 -55465 -55466 -55467 -55468 -55469 -55470 -55471 -55472 -55473 -55474 -55475 -55476 -55477 -55478 -55479 -55480 -55481 -55482 -55483 -55484 -55485 -55486 -55487 -55488 -55489 -55490 -55491 -55492 -55493 -55494 -55495 -55496 -55497 -55498 -55499 -55500 -55501 -55502 -55503 -55504 -55505 -55506 -55507 -55508 -55509 -55510 -55511 -55512 -55513 -55514 -55515 -55516 -55517 -55518 -55519 -55520 -55521 -55522 -55523 -55524 -55525 -55526 -55527 -55528 -55529 -55530 -55531 -55532 -55533 -55534 -55535 -55536 -55537 -55538 -55539 -55540 -55541 -55542 -55543 -55544 -55545 -55546 -55547 -55548 -55549 -55550 -55551 -55552 -55553 -55554 -55555 -55556 -55557 -55558 -55559 -55560 -55561 -55562 -55563 -55564 -55565 -55566 -55567 -55568 -55569 -55570 -55571 -55572 -55573 -55574 -55575 -55576 -55577 -55578 -55579 -55580 -55581 -55582 -55583 -55584 -55585 -55586 -55587 -55588 -55589 -55590 -55591 -55592 -55593 -55594 -55595 -55596 -55597 -55598 -55599 -55600 -55601 -55602 -55603 -55604 -55605 -55606 -55607 -55608 -55609 -55610 -55611 -55612 -55613 -55614 -55615 -55616 -55617 -55618 -55619 -55620 -55621 -55622 -55623 -55624 -55625 -55626 -55627 -55628 -55629 -55630 -55631 -55632 -55633 -55634 -55635 -55636 -55637 -55638 -55639 -55640 -55641 -55642 -55643 -55644 -55645 -55646 -55647 -55648 -55649 -55650 -55651 -55652 -55653 -55654 -55655 -55656 -55657 -55658 -55659 -55660 -55661 -55662 -55663 -55664 -55665 -55666 -55667 -55668 -55669 -55670 -55671 -55672 -55673 -55674 -55675 -55676 -55677 -55678 -55679 -55680 -55681 -55682 -55683 -55684 -55685 -55686 -55687 -55688 -55689 -55690 -55691 -55692 -55693 -55694 -55695 -55696 -55697 -55698 -55699 -55700 -55701 -55702 -55703 -55704 -55705 -55706 -55707 -55708 -55709 -55710 -55711 -55712 -55713 -55714 -55715 -55716 -55717 -55718 -55719 -55720 -55721 -55722 -55723 -55724 -55725 -55726 -55727 -55728 -55729 -55730 -55731 -55732 -55733 -55734 -55735 -55736 -55737 -55738 -55739 -55740 -55741 -55742 -55743 -55744 -55745 -55746 -55747 -55748 -55749 -55750 -55751 -55752 -55753 -55754 -55755 -55756 -55757 -55758 -55759 -55760 -55761 -55762 -55763 -55764 -55765 -55766 -55767 -55768 -55769 -55770 -55771 -55772 -55773 -55774 -55775 -55776 -55777 -55778 -55779 -55780 -55781 -55782 -55783 -55784 -55785 -55786 -55787 -55788 -55789 -55790 -55791 -55792 -55793 -55794 -55795 -55796 -55797 -55798 -55799 -55800 -55801 -55802 -55803 -55804 -55805 -55806 -55807 -55808 -55809 -55810 -55811 -55812 -55813 -55814 -55815 -55816 -55817 -55818 -55819 -55820 -55821 -55822 -55823 -55824 -55825 -55826 -55827 -55828 -55829 -55830 -55831 -55832 -55833 -55834 -55835 -55836 -55837 -55838 -55839 -55840 -55841 -55842 -55843 -55844 -55845 -55846 -55847 -55848 -55849 -55850 -55851 -55852 -55853 -55854 -55855 -55856 -55857 -55858 -55859 -55860 -55861 -55862 -55863 -55864 -55865 -55866 -55867 -55868 -55869 -55870 -55871 -55872 -55873 -55874 -55875 -55876 -55877 -55878 -55879 -55880 -55881 -55882 -55883 -55884 -55885 -55886 -55887 -55888 -55889 -55890 -55891 -55892 -55893 -55894 -55895 -55896 -55897 -55898 -55899 -55900 -55901 -55902 -55903 -55904 -55905 -55906 -55907 -55908 -55909 -55910 -55911 -55912 -55913 -55914 -55915 -55916 -55917 -55918 -55919 -55920 -55921 -55922 -55923 -55924 -55925 -55926 -55927 -55928 -55929 -55930 -55931 -55932 -55933 -55934 -55935 -55936 -55937 -55938 -55939 -55940 -55941 -55942 -55943 -55944 -55945 -55946 -55947 -55948 -55949 -55950 -55951 -55952 -55953 -55954 -55955 -55956 -55957 -55958 -55959 -55960 -55961 -55962 -55963 -55964 -55965 -55966 -55967 -55968 -55969 -55970 -55971 -55972 -55973 -55974 -55975 -55976 -55977 -55978 -55979 -55980 -55981 -55982 -55983 -55984 -55985 -55986 -55987 -55988 -55989 -55990 -55991 -55992 -55993 -55994 -55995 -55996 -55997 -55998 -55999 -56000 -56001 -56002 -56003 -56004 -56005 -56006 -56007 -56008 -56009 -56010 -56011 -56012 -56013 -56014 -56015 -56016 -56017 -56018 -56019 -56020 -56021 -56022 -56023 -56024 -56025 -56026 -56027 -56028 -56029 -56030 -56031 -56032 -56033 -56034 -56035 -56036 -56037 -56038 -56039 -56040 -56041 -56042 -56043 -56044 -56045 -56046 -56047 -56048 -56049 -56050 -56051 -56052 -56053 -56054 -56055 -56056 -56057 -56058 -56059 -56060 -56061 -56062 -56063 -56064 -56065 -56066 -56067 -56068 -56069 -56070 -56071 -56072 -56073 -56074 -56075 -56076 -56077 -56078 -56079 -56080 -56081 -56082 -56083 -56084 -56085 -56086 -56087 -56088 -56089 -56090 -56091 -56092 -56093 -56094 -56095 -56096 -56097 -56098 -56099 -56100 -56101 -56102 -56103 -56104 -56105 -56106 -56107 -56108 -56109 -56110 -56111 -56112 -56113 -56114 -56115 -56116 -56117 -56118 -56119 -56120 -56121 -56122 -56123 -56124 -56125 -56126 -56127 -56128 -56129 -56130 -56131 -56132 -56133 -56134 -56135 -56136 -56137 -56138 -56139 -56140 -56141 -56142 -56143 -56144 -56145 -56146 -56147 -56148 -56149 -56150 -56151 -56152 -56153 -56154 -56155 -56156 -56157 -56158 -56159 -56160 -56161 -56162 -56163 -56164 -56165 -56166 -56167 -56168 -56169 -56170 -56171 -56172 -56173 -56174 -56175 -56176 -56177 -56178 -56179 -56180 -56181 -56182 -56183 -56184 -56185 -56186 -56187 -56188 -56189 -56190 -56191 -56192 -56193 -56194 -56195 -56196 -56197 -56198 -56199 -56200 -56201 -56202 -56203 -56204 -56205 -56206 -56207 -56208 -56209 -56210 -56211 -56212 -56213 -56214 -56215 -56216 -56217 -56218 -56219 -56220 -56221 -56222 -56223 -56224 -56225 -56226 -56227 -56228 -56229 -56230 -56231 -56232 -56233 -56234 -56235 -56236 -56237 -56238 -56239 -56240 -56241 -56242 -56243 -56244 -56245 -56246 -56247 -56248 -56249 -56250 -56251 -56252 -56253 -56254 -56255 -56256 -56257 -56258 -56259 -56260 -56261 -56262 -56263 -56264 -56265 -56266 -56267 -56268 -56269 -56270 -56271 -56272 -56273 -56274 -56275 -56276 -56277 -56278 -56279 -56280 -56281 -56282 -56283 -56284 -56285 -56286 -56287 -56288 -56289 -56290 -56291 -56292 -56293 -56294 -56295 -56296 -56297 -56298 -56299 -56300 -56301 -56302 -56303 -56304 -56305 -56306 -56307 -56308 -56309 -56310 -56311 -56312 -56313 -56314 -56315 -56316 -56317 -56318 -56319 -56320 -56321 -56322 -56323 -56324 -56325 -56326 -56327 -56328 -56329 -56330 -56331 -56332 -56333 -56334 -56335 -56336 -56337 -56338 -56339 -56340 -56341 -56342 -56343 -56344 -56345 -56346 -56347 -56348 -56349 -56350 -56351 -56352 -56353 -56354 -56355 -56356 -56357 -56358 -56359 -56360 -56361 -56362 -56363 -56364 -56365 -56366 -56367 -56368 -56369 -56370 -56371 -56372 -56373 -56374 -56375 -56376 -56377 -56378 -56379 -56380 -56381 -56382 -56383 -56384 -56385 -56386 -56387 -56388 -56389 -56390 -56391 -56392 -56393 -56394 -56395 -56396 -56397 -56398 -56399 -56400 -56401 -56402 -56403 -56404 -56405 -56406 -56407 -56408 -56409 -56410 -56411 -56412 -56413 -56414 -56415 -56416 -56417 -56418 -56419 -56420 -56421 -56422 -56423 -56424 -56425 -56426 -56427 -56428 -56429 -56430 -56431 -56432 -56433 -56434 -56435 -56436 -56437 -56438 -56439 -56440 -56441 -56442 -56443 -56444 -56445 -56446 -56447 -56448 -56449 -56450 -56451 -56452 -56453 -56454 -56455 -56456 -56457 -56458 -56459 -56460 -56461 -56462 -56463 -56464 -56465 -56466 -56467 -56468 -56469 -56470 -56471 -56472 -56473 -56474 -56475 -56476 -56477 -56478 -56479 -56480 -56481 -56482 -56483 -56484 -56485 -56486 -56487 -56488 -56489 -56490 -56491 -56492 -56493 -56494 -56495 -56496 -56497 -56498 -56499 -56500 -56501 -56502 -56503 -56504 -56505 -56506 -56507 -56508 -56509 -56510 -56511 -56512 -56513 -56514 -56515 -56516 -56517 -56518 -56519 -56520 -56521 -56522 -56523 -56524 -56525 -56526 -56527 -56528 -56529 -56530 -56531 -56532 -56533 -56534 -56535 -56536 -56537 -56538 -56539 -56540 -56541 -56542 -56543 -56544 -56545 -56546 -56547 -56548 -56549 -56550 -56551 -56552 -56553 -56554 -56555 -56556 -56557 -56558 -56559 -56560 -56561 -56562 -56563 -56564 -56565 -56566 -56567 -56568 -56569 -56570 -56571 -56572 -56573 -56574 -56575 -56576 -56577 -56578 -56579 -56580 -56581 -56582 -56583 -56584 -56585 -56586 -56587 -56588 -56589 -56590 -56591 -56592 -56593 -56594 -56595 -56596 -56597 -56598 -56599 -56600 -56601 -56602 -56603 -56604 -56605 -56606 -56607 -56608 -56609 -56610 -56611 -56612 -56613 -56614 -56615 -56616 -56617 -56618 -56619 -56620 -56621 -56622 -56623 -56624 -56625 -56626 -56627 -56628 -56629 -56630 -56631 -56632 -56633 -56634 -56635 -56636 -56637 -56638 -56639 -56640 -56641 -56642 -56643 -56644 -56645 -56646 -56647 -56648 -56649 -56650 -56651 -56652 -56653 -56654 -56655 -56656 -56657 -56658 -56659 -56660 -56661 -56662 -56663 -56664 -56665 -56666 -56667 -56668 -56669 -56670 -56671 -56672 -56673 -56674 -56675 -56676 -56677 -56678 -56679 -56680 -56681 -56682 -56683 -56684 -56685 -56686 -56687 -56688 -56689 -56690 -56691 -56692 -56693 -56694 -56695 -56696 -56697 -56698 -56699 -56700 -56701 -56702 -56703 -56704 -56705 -56706 -56707 -56708 -56709 -56710 -56711 -56712 -56713 -56714 -56715 -56716 -56717 -56718 -56719 -56720 -56721 -56722 -56723 -56724 -56725 -56726 -56727 -56728 -56729 -56730 -56731 -56732 -56733 -56734 -56735 -56736 -56737 -56738 -56739 -56740 -56741 -56742 -56743 -56744 -56745 -56746 -56747 -56748 -56749 -56750 -56751 -56752 -56753 -56754 -56755 -56756 -56757 -56758 -56759 -56760 -56761 -56762 -56763 -56764 -56765 -56766 -56767 -56768 -56769 -56770 -56771 -56772 -56773 -56774 -56775 -56776 -56777 -56778 -56779 -56780 -56781 -56782 -56783 -56784 -56785 -56786 -56787 -56788 -56789 -56790 -56791 -56792 -56793 -56794 -56795 -56796 -56797 -56798 -56799 -56800 -56801 -56802 -56803 -56804 -56805 -56806 -56807 -56808 -56809 -56810 -56811 -56812 -56813 -56814 -56815 -56816 -56817 -56818 -56819 -56820 -56821 -56822 -56823 -56824 -56825 -56826 -56827 -56828 -56829 -56830 -56831 -56832 -56833 -56834 -56835 -56836 -56837 -56838 -56839 -56840 -56841 -56842 -56843 -56844 -56845 -56846 -56847 -56848 -56849 -56850 -56851 -56852 -56853 -56854 -56855 -56856 -56857 -56858 -56859 -56860 -56861 -56862 -56863 -56864 -56865 -56866 -56867 -56868 -56869 -56870 -56871 -56872 -56873 -56874 -56875 -56876 -56877 -56878 -56879 -56880 -56881 -56882 -56883 -56884 -56885 -56886 -56887 -56888 -56889 -56890 -56891 -56892 -56893 -56894 -56895 -56896 -56897 -56898 -56899 -56900 -56901 -56902 -56903 -56904 -56905 -56906 -56907 -56908 -56909 -56910 -56911 -56912 -56913 -56914 -56915 -56916 -56917 -56918 -56919 -56920 -56921 -56922 -56923 -56924 -56925 -56926 -56927 -56928 -56929 -56930 -56931 -56932 -56933 -56934 -56935 -56936 -56937 -56938 -56939 -56940 -56941 -56942 -56943 -56944 -56945 -56946 -56947 -56948 -56949 -56950 -56951 -56952 -56953 -56954 -56955 -56956 -56957 -56958 -56959 -56960 -56961 -56962 -56963 -56964 -56965 -56966 -56967 -56968 -56969 -56970 -56971 -56972 -56973 -56974 -56975 -56976 -56977 -56978 -56979 -56980 -56981 -56982 -56983 -56984 -56985 -56986 -56987 -56988 -56989 -56990 -56991 -56992 -56993 -56994 -56995 -56996 -56997 -56998 -56999 -57000 -57001 -57002 -57003 -57004 -57005 -57006 -57007 -57008 -57009 -57010 -57011 -57012 -57013 -57014 -57015 -57016 -57017 -57018 -57019 -57020 -57021 -57022 -57023 -57024 -57025 -57026 -57027 -57028 -57029 -57030 -57031 -57032 -57033 -57034 -57035 -57036 -57037 -57038 -57039 -57040 -57041 -57042 -57043 -57044 -57045 -57046 -57047 -57048 -57049 -57050 -57051 -57052 -57053 -57054 -57055 -57056 -57057 -57058 -57059 -57060 -57061 -57062 -57063 -57064 -57065 -57066 -57067 -57068 -57069 -57070 -57071 -57072 -57073 -57074 -57075 -57076 -57077 -57078 -57079 -57080 -57081 -57082 -57083 -57084 -57085 -57086 -57087 -57088 -57089 -57090 -57091 -57092 -57093 -57094 -57095 -57096 -57097 -57098 -57099 -57100 -57101 -57102 -57103 -57104 -57105 -57106 -57107 -57108 -57109 -57110 -57111 -57112 -57113 -57114 -57115 -57116 -57117 -57118 -57119 -57120 -57121 -57122 -57123 -57124 -57125 -57126 -57127 -57128 -57129 -57130 -57131 -57132 -57133 -57134 -57135 -57136 -57137 -57138 -57139 -57140 -57141 -57142 -57143 -57144 -57145 -57146 -57147 -57148 -57149 -57150 -57151 -57152 -57153 -57154 -57155 -57156 -57157 -57158 -57159 -57160 -57161 -57162 -57163 -57164 -57165 -57166 -57167 -57168 -57169 -57170 -57171 -57172 -57173 -57174 -57175 -57176 -57177 -57178 -57179 -57180 -57181 -57182 -57183 -57184 -57185 -57186 -57187 -57188 -57189 -57190 -57191 -57192 -57193 -57194 -57195 -57196 -57197 -57198 -57199 -57200 -57201 -57202 -57203 -57204 -57205 -57206 -57207 -57208 -57209 -57210 -57211 -57212 -57213 -57214 -57215 -57216 -57217 -57218 -57219 -57220 -57221 -57222 -57223 -57224 -57225 -57226 -57227 -57228 -57229 -57230 -57231 -57232 -57233 -57234 -57235 -57236 -57237 -57238 -57239 -57240 -57241 -57242 -57243 -57244 -57245 -57246 -57247 -57248 -57249 -57250 -57251 -57252 -57253 -57254 -57255 -57256 -57257 -57258 -57259 -57260 -57261 -57262 -57263 -57264 -57265 -57266 -57267 -57268 -57269 -57270 -57271 -57272 -57273 -57274 -57275 -57276 -57277 -57278 -57279 -57280 -57281 -57282 -57283 -57284 -57285 -57286 -57287 -57288 -57289 -57290 -57291 -57292 -57293 -57294 -57295 -57296 -57297 -57298 -57299 -57300 -57301 -57302 -57303 -57304 -57305 -57306 -57307 -57308 -57309 -57310 -57311 -57312 -57313 -57314 -57315 -57316 -57317 -57318 -57319 -57320 -57321 -57322 -57323 -57324 -57325 -57326 -57327 -57328 -57329 -57330 -57331 -57332 -57333 -57334 -57335 -57336 -57337 -57338 -57339 -57340 -57341 -57342 -57343 -57344 -57345 -57346 -57347 -57348 -57349 -57350 -57351 -57352 -57353 -57354 -57355 -57356 -57357 -57358 -57359 -57360 -57361 -57362 -57363 -57364 -57365 -57366 -57367 -57368 -57369 -57370 -57371 -57372 -57373 -57374 -57375 -57376 -57377 -57378 -57379 -57380 -57381 -57382 -57383 -57384 -57385 -57386 -57387 -57388 -57389 -57390 -57391 -57392 -57393 -57394 -57395 -57396 -57397 -57398 -57399 -57400 -57401 -57402 -57403 -57404 -57405 -57406 -57407 -57408 -57409 -57410 -57411 -57412 -57413 -57414 -57415 -57416 -57417 -57418 -57419 -57420 -57421 -57422 -57423 -57424 -57425 -57426 -57427 -57428 -57429 -57430 -57431 -57432 -57433 -57434 -57435 -57436 -57437 -57438 -57439 -57440 -57441 -57442 -57443 -57444 -57445 -57446 -57447 -57448 -57449 -57450 -57451 -57452 -57453 -57454 -57455 -57456 -57457 -57458 -57459 -57460 -57461 -57462 -57463 -57464 -57465 -57466 -57467 -57468 -57469 -57470 -57471 -57472 -57473 -57474 -57475 -57476 -57477 -57478 -57479 -57480 -57481 -57482 -57483 -57484 -57485 -57486 -57487 -57488 -57489 -57490 -57491 -57492 -57493 -57494 -57495 -57496 -57497 -57498 -57499 -57500 -57501 -57502 -57503 -57504 -57505 -57506 -57507 -57508 -57509 -57510 -57511 -57512 -57513 -57514 -57515 -57516 -57517 -57518 -57519 -57520 -57521 -57522 -57523 -57524 -57525 -57526 -57527 -57528 -57529 -57530 -57531 -57532 -57533 -57534 -57535 -57536 -57537 -57538 -57539 -57540 -57541 -57542 -57543 -57544 -57545 -57546 -57547 -57548 -57549 -57550 -57551 -57552 -57553 -57554 -57555 -57556 -57557 -57558 -57559 -57560 -57561 -57562 -57563 -57564 -57565 -57566 -57567 -57568 -57569 -57570 -57571 -57572 -57573 -57574 -57575 -57576 -57577 -57578 -57579 -57580 -57581 -57582 -57583 -57584 -57585 -57586 -57587 -57588 -57589 -57590 -57591 -57592 -57593 -57594 -57595 -57596 -57597 -57598 -57599 -57600 -57601 -57602 -57603 -57604 -57605 -57606 -57607 -57608 -57609 -57610 -57611 -57612 -57613 -57614 -57615 -57616 -57617 -57618 -57619 -57620 -57621 -57622 -57623 -57624 -57625 -57626 -57627 -57628 -57629 -57630 -57631 -57632 -57633 -57634 -57635 -57636 -57637 -57638 -57639 -57640 -57641 -57642 -57643 -57644 -57645 -57646 -57647 -57648 -57649 -57650 -57651 -57652 -57653 -57654 -57655 -57656 -57657 -57658 -57659 -57660 -57661 -57662 -57663 -57664 -57665 -57666 -57667 -57668 -57669 -57670 -57671 -57672 -57673 -57674 -57675 -57676 -57677 -57678 -57679 -57680 -57681 -57682 -57683 -57684 -57685 -57686 -57687 -57688 -57689 -57690 -57691 -57692 -57693 -57694 -57695 -57696 -57697 -57698 -57699 -57700 -57701 -57702 -57703 -57704 -57705 -57706 -57707 -57708 -57709 -57710 -57711 -57712 -57713 -57714 -57715 -57716 -57717 -57718 -57719 -57720 -57721 -57722 -57723 -57724 -57725 -57726 -57727 -57728 -57729 -57730 -57731 -57732 -57733 -57734 -57735 -57736 -57737 -57738 -57739 -57740 -57741 -57742 -57743 -57744 -57745 -57746 -57747 -57748 -57749 -57750 -57751 -57752 -57753 -57754 -57755 -57756 -57757 -57758 -57759 -57760 -57761 -57762 -57763 -57764 -57765 -57766 -57767 -57768 -57769 -57770 -57771 -57772 -57773 -57774 -57775 -57776 -57777 -57778 -57779 -57780 -57781 -57782 -57783 -57784 -57785 -57786 -57787 -57788 -57789 -57790 -57791 -57792 -57793 -57794 -57795 -57796 -57797 -57798 -57799 -57800 -57801 -57802 -57803 -57804 -57805 -57806 -57807 -57808 -57809 -57810 -57811 -57812 -57813 -57814 -57815 -57816 -57817 -57818 -57819 -57820 -57821 -57822 -57823 -57824 -57825 -57826 -57827 -57828 -57829 -57830 -57831 -57832 -57833 -57834 -57835 -57836 -57837 -57838 -57839 -57840 -57841 -57842 -57843 -57844 -57845 -57846 -57847 -57848 -57849 -57850 -57851 -57852 -57853 -57854 -57855 -57856 -57857 -57858 -57859 -57860 -57861 -57862 -57863 -57864 -57865 -57866 -57867 -57868 -57869 -57870 -57871 -57872 -57873 -57874 -57875 -57876 -57877 -57878 -57879 -57880 -57881 -57882 -57883 -57884 -57885 -57886 -57887 -57888 -57889 -57890 -57891 -57892 -57893 -57894 -57895 -57896 -57897 -57898 -57899 -57900 -57901 -57902 -57903 -57904 -57905 -57906 -57907 -57908 -57909 -57910 -57911 -57912 -57913 -57914 -57915 -57916 -57917 -57918 -57919 -57920 -57921 -57922 -57923 -57924 -57925 -57926 -57927 -57928 -57929 -57930 -57931 -57932 -57933 -57934 -57935 -57936 -57937 -57938 -57939 -57940 -57941 -57942 -57943 -57944 -57945 -57946 -57947 -57948 -57949 -57950 -57951 -57952 -57953 -57954 -57955 -57956 -57957 -57958 -57959 -57960 -57961 -57962 -57963 -57964 -57965 -57966 -57967 -57968 -57969 -57970 -57971 -57972 -57973 -57974 -57975 -57976 -57977 -57978 -57979 -57980 -57981 -57982 -57983 -57984 -57985 -57986 -57987 -57988 -57989 -57990 -57991 -57992 -57993 -57994 -57995 -57996 -57997 -57998 -57999 -58000 -58001 -58002 -58003 -58004 -58005 -58006 -58007 -58008 -58009 -58010 -58011 -58012 -58013 -58014 -58015 -58016 -58017 -58018 -58019 -58020 -58021 -58022 -58023 -58024 -58025 -58026 -58027 -58028 -58029 -58030 -58031 -58032 -58033 -58034 -58035 -58036 -58037 -58038 -58039 -58040 -58041 -58042 -58043 -58044 -58045 -58046 -58047 -58048 -58049 -58050 -58051 -58052 -58053 -58054 -58055 -58056 -58057 -58058 -58059 -58060 -58061 -58062 -58063 -58064 -58065 -58066 -58067 -58068 -58069 -58070 -58071 -58072 -58073 -58074 -58075 -58076 -58077 -58078 -58079 -58080 -58081 -58082 -58083 -58084 -58085 -58086 -58087 -58088 -58089 -58090 -58091 -58092 -58093 -58094 -58095 -58096 -58097 -58098 -58099 -58100 -58101 -58102 -58103 -58104 -58105 -58106 -58107 -58108 -58109 -58110 -58111 -58112 -58113 -58114 -58115 -58116 -58117 -58118 -58119 -58120 -58121 -58122 -58123 -58124 -58125 -58126 -58127 -58128 -58129 -58130 -58131 -58132 -58133 -58134 -58135 -58136 -58137 -58138 -58139 -58140 -58141 -58142 -58143 -58144 -58145 -58146 -58147 -58148 -58149 -58150 -58151 -58152 -58153 -58154 -58155 -58156 -58157 -58158 -58159 -58160 -58161 -58162 -58163 -58164 -58165 -58166 -58167 -58168 -58169 -58170 -58171 -58172 -58173 -58174 -58175 -58176 -58177 -58178 -58179 -58180 -58181 -58182 -58183 -58184 -58185 -58186 -58187 -58188 -58189 -58190 -58191 -58192 -58193 -58194 -58195 -58196 -58197 -58198 -58199 -58200 -58201 -58202 -58203 -58204 -58205 -58206 -58207 -58208 -58209 -58210 -58211 -58212 -58213 -58214 -58215 -58216 -58217 -58218 -58219 -58220 -58221 -58222 -58223 -58224 -58225 -58226 -58227 -58228 -58229 -58230 -58231 -58232 -58233 -58234 -58235 -58236 -58237 -58238 -58239 -58240 -58241 -58242 -58243 -58244 -58245 -58246 -58247 -58248 -58249 -58250 -58251 -58252 -58253 -58254 -58255 -58256 -58257 -58258 -58259 -58260 -58261 -58262 -58263 -58264 -58265 -58266 -58267 -58268 -58269 -58270 -58271 -58272 -58273 -58274 -58275 -58276 -58277 -58278 -58279 -58280 -58281 -58282 -58283 -58284 -58285 -58286 -58287 -58288 -58289 -58290 -58291 -58292 -58293 -58294 -58295 -58296 -58297 -58298 -58299 -58300 -58301 -58302 -58303 -58304 -58305 -58306 -58307 -58308 -58309 -58310 -58311 -58312 -58313 -58314 -58315 -58316 -58317 -58318 -58319 -58320 -58321 -58322 -58323 -58324 -58325 -58326 -58327 -58328 -58329 -58330 -58331 -58332 -58333 -58334 -58335 -58336 -58337 -58338 -58339 -58340 -58341 -58342 -58343 -58344 -58345 -58346 -58347 -58348 -58349 -58350 -58351 -58352 -58353 -58354 -58355 -58356 -58357 -58358 -58359 -58360 -58361 -58362 -58363 -58364 -58365 -58366 -58367 -58368 -58369 -58370 -58371 -58372 -58373 -58374 -58375 -58376 -58377 -58378 -58379 -58380 -58381 -58382 -58383 -58384 -58385 -58386 -58387 -58388 -58389 -58390 -58391 -58392 -58393 -58394 -58395 -58396 -58397 -58398 -58399 -58400 -58401 -58402 -58403 -58404 -58405 -58406 -58407 -58408 -58409 -58410 -58411 -58412 -58413 -58414 -58415 -58416 -58417 -58418 -58419 -58420 -58421 -58422 -58423 -58424 -58425 -58426 -58427 -58428 -58429 -58430 -58431 -58432 -58433 -58434 -58435 -58436 -58437 -58438 -58439 -58440 -58441 -58442 -58443 -58444 -58445 -58446 -58447 -58448 -58449 -58450 -58451 -58452 -58453 -58454 -58455 -58456 -58457 -58458 -58459 -58460 -58461 -58462 -58463 -58464 -58465 -58466 -58467 -58468 -58469 -58470 -58471 -58472 -58473 -58474 -58475 -58476 -58477 -58478 -58479 -58480 -58481 -58482 -58483 -58484 -58485 -58486 -58487 -58488 -58489 -58490 -58491 -58492 -58493 -58494 -58495 -58496 -58497 -58498 -58499 -58500 -58501 -58502 -58503 -58504 -58505 -58506 -58507 -58508 -58509 -58510 -58511 -58512 -58513 -58514 -58515 -58516 -58517 -58518 -58519 -58520 -58521 -58522 -58523 -58524 -58525 -58526 -58527 -58528 -58529 -58530 -58531 -58532 -58533 -58534 -58535 -58536 -58537 -58538 -58539 -58540 -58541 -58542 -58543 -58544 -58545 -58546 -58547 -58548 -58549 -58550 -58551 -58552 -58553 -58554 -58555 -58556 -58557 -58558 -58559 -58560 -58561 -58562 -58563 -58564 -58565 -58566 -58567 -58568 -58569 -58570 -58571 -58572 -58573 -58574 -58575 -58576 -58577 -58578 -58579 -58580 -58581 -58582 -58583 -58584 -58585 -58586 -58587 -58588 -58589 -58590 -58591 -58592 -58593 -58594 -58595 -58596 -58597 -58598 -58599 -58600 -58601 -58602 -58603 -58604 -58605 -58606 -58607 -58608 -58609 -58610 -58611 -58612 -58613 -58614 -58615 -58616 -58617 -58618 -58619 -58620 -58621 -58622 -58623 -58624 -58625 -58626 -58627 -58628 -58629 -58630 -58631 -58632 -58633 -58634 -58635 -58636 -58637 -58638 -58639 -58640 -58641 -58642 -58643 -58644 -58645 -58646 -58647 -58648 -58649 -58650 -58651 -58652 -58653 -58654 -58655 -58656 -58657 -58658 -58659 -58660 -58661 -58662 -58663 -58664 -58665 -58666 -58667 -58668 -58669 -58670 -58671 -58672 -58673 -58674 -58675 -58676 -58677 -58678 -58679 -58680 -58681 -58682 -58683 -58684 -58685 -58686 -58687 -58688 -58689 -58690 -58691 -58692 -58693 -58694 -58695 -58696 -58697 -58698 -58699 -58700 -58701 -58702 -58703 -58704 -58705 -58706 -58707 -58708 -58709 -58710 -58711 -58712 -58713 -58714 -58715 -58716 -58717 -58718 -58719 -58720 -58721 -58722 -58723 -58724 -58725 -58726 -58727 -58728 -58729 -58730 -58731 -58732 -58733 -58734 -58735 -58736 -58737 -58738 -58739 -58740 -58741 -58742 -58743 -58744 -58745 -58746 -58747 -58748 -58749 -58750 -58751 -58752 -58753 -58754 -58755 -58756 -58757 -58758 -58759 -58760 -58761 -58762 -58763 -58764 -58765 -58766 -58767 -58768 -58769 -58770 -58771 -58772 -58773 -58774 -58775 -58776 -58777 -58778 -58779 -58780 -58781 -58782 -58783 -58784 -58785 -58786 -58787 -58788 -58789 -58790 -58791 -58792 -58793 -58794 -58795 -58796 -58797 -58798 -58799 -58800 -58801 -58802 -58803 -58804 -58805 -58806 -58807 -58808 -58809 -58810 -58811 -58812 -58813 -58814 -58815 -58816 -58817 -58818 -58819 -58820 -58821 -58822 -58823 -58824 -58825 -58826 -58827 -58828 -58829 -58830 -58831 -58832 -58833 -58834 -58835 -58836 -58837 -58838 -58839 -58840 -58841 -58842 -58843 -58844 -58845 -58846 -58847 -58848 -58849 -58850 -58851 -58852 -58853 -58854 -58855 -58856 -58857 -58858 -58859 -58860 -58861 -58862 -58863 -58864 -58865 -58866 -58867 -58868 -58869 -58870 -58871 -58872 -58873 -58874 -58875 -58876 -58877 -58878 -58879 -58880 -58881 -58882 -58883 -58884 -58885 -58886 -58887 -58888 -58889 -58890 -58891 -58892 -58893 -58894 -58895 -58896 -58897 -58898 -58899 -58900 -58901 -58902 -58903 -58904 -58905 -58906 -58907 -58908 -58909 -58910 -58911 -58912 -58913 -58914 -58915 -58916 -58917 -58918 -58919 -58920 -58921 -58922 -58923 -58924 -58925 -58926 -58927 -58928 -58929 -58930 -58931 -58932 -58933 -58934 -58935 -58936 -58937 -58938 -58939 -58940 -58941 -58942 -58943 -58944 -58945 -58946 -58947 -58948 -58949 -58950 -58951 -58952 -58953 -58954 -58955 -58956 -58957 -58958 -58959 -58960 -58961 -58962 -58963 -58964 -58965 -58966 -58967 -58968 -58969 -58970 -58971 -58972 -58973 -58974 -58975 -58976 -58977 -58978 -58979 -58980 -58981 -58982 -58983 -58984 -58985 -58986 -58987 -58988 -58989 -58990 -58991 -58992 -58993 -58994 -58995 -58996 -58997 -58998 -58999 -59000 -59001 -59002 -59003 -59004 -59005 -59006 -59007 -59008 -59009 -59010 -59011 -59012 -59013 -59014 -59015 -59016 -59017 -59018 -59019 -59020 -59021 -59022 -59023 -59024 -59025 -59026 -59027 -59028 -59029 -59030 -59031 -59032 -59033 -59034 -59035 -59036 -59037 -59038 -59039 -59040 -59041 -59042 -59043 -59044 -59045 -59046 -59047 -59048 -59049 -59050 -59051 -59052 -59053 -59054 -59055 -59056 -59057 -59058 -59059 -59060 -59061 -59062 -59063 -59064 -59065 -59066 -59067 -59068 -59069 -59070 -59071 -59072 -59073 -59074 -59075 -59076 -59077 -59078 -59079 -59080 -59081 -59082 -59083 -59084 -59085 -59086 -59087 -59088 -59089 -59090 -59091 -59092 -59093 -59094 -59095 -59096 -59097 -59098 -59099 -59100 -59101 -59102 -59103 -59104 -59105 -59106 -59107 -59108 -59109 -59110 -59111 -59112 -59113 -59114 -59115 -59116 -59117 -59118 -59119 -59120 -59121 -59122 -59123 -59124 -59125 -59126 -59127 -59128 -59129 -59130 -59131 -59132 -59133 -59134 -59135 -59136 -59137 -59138 -59139 -59140 -59141 -59142 -59143 -59144 -59145 -59146 -59147 -59148 -59149 -59150 -59151 -59152 -59153 -59154 -59155 -59156 -59157 -59158 -59159 -59160 -59161 -59162 -59163 -59164 -59165 -59166 -59167 -59168 -59169 -59170 -59171 -59172 -59173 -59174 -59175 -59176 -59177 -59178 -59179 -59180 -59181 -59182 -59183 -59184 -59185 -59186 -59187 -59188 -59189 -59190 -59191 -59192 -59193 -59194 -59195 -59196 -59197 -59198 -59199 -59200 -59201 -59202 -59203 -59204 -59205 -59206 -59207 -59208 -59209 -59210 -59211 -59212 -59213 -59214 -59215 -59216 -59217 -59218 -59219 -59220 -59221 -59222 -59223 -59224 -59225 -59226 -59227 -59228 -59229 -59230 -59231 -59232 -59233 -59234 -59235 -59236 -59237 -59238 -59239 -59240 -59241 -59242 -59243 -59244 -59245 -59246 -59247 -59248 -59249 -59250 -59251 -59252 -59253 -59254 -59255 -59256 -59257 -59258 -59259 -59260 -59261 -59262 -59263 -59264 -59265 -59266 -59267 -59268 -59269 -59270 -59271 -59272 -59273 -59274 -59275 -59276 -59277 -59278 -59279 -59280 -59281 -59282 -59283 -59284 -59285 -59286 -59287 -59288 -59289 -59290 -59291 -59292 -59293 -59294 -59295 -59296 -59297 -59298 -59299 -59300 -59301 -59302 -59303 -59304 -59305 -59306 -59307 -59308 -59309 -59310 -59311 -59312 -59313 -59314 -59315 -59316 -59317 -59318 -59319 -59320 -59321 -59322 -59323 -59324 -59325 -59326 -59327 -59328 -59329 -59330 -59331 -59332 -59333 -59334 -59335 -59336 -59337 -59338 -59339 -59340 -59341 -59342 -59343 -59344 -59345 -59346 -59347 -59348 -59349 -59350 -59351 -59352 -59353 -59354 -59355 -59356 -59357 -59358 -59359 -59360 -59361 -59362 -59363 -59364 -59365 -59366 -59367 -59368 -59369 -59370 -59371 -59372 -59373 -59374 -59375 -59376 -59377 -59378 -59379 -59380 -59381 -59382 -59383 -59384 -59385 -59386 -59387 -59388 -59389 -59390 -59391 -59392 -59393 -59394 -59395 -59396 -59397 -59398 -59399 -59400 -59401 -59402 -59403 -59404 -59405 -59406 -59407 -59408 -59409 -59410 -59411 -59412 -59413 -59414 -59415 -59416 -59417 -59418 -59419 -59420 -59421 -59422 -59423 -59424 -59425 -59426 -59427 -59428 -59429 -59430 -59431 -59432 -59433 -59434 -59435 -59436 -59437 -59438 -59439 -59440 -59441 -59442 -59443 -59444 -59445 -59446 -59447 -59448 -59449 -59450 -59451 -59452 -59453 -59454 -59455 -59456 -59457 -59458 -59459 -59460 -59461 -59462 -59463 -59464 -59465 -59466 -59467 -59468 -59469 -59470 -59471 -59472 -59473 -59474 -59475 -59476 -59477 -59478 -59479 -59480 -59481 -59482 -59483 -59484 -59485 -59486 -59487 -59488 -59489 -59490 -59491 -59492 -59493 -59494 -59495 -59496 -59497 -59498 -59499 -59500 -59501 -59502 -59503 -59504 -59505 -59506 -59507 -59508 -59509 -59510 -59511 -59512 -59513 -59514 -59515 -59516 -59517 -59518 -59519 -59520 -59521 -59522 -59523 -59524 -59525 -59526 -59527 -59528 -59529 -59530 -59531 -59532 -59533 -59534 -59535 -59536 -59537 -59538 -59539 -59540 -59541 -59542 -59543 -59544 -59545 -59546 -59547 -59548 -59549 -59550 -59551 -59552 -59553 -59554 -59555 -59556 -59557 -59558 -59559 -59560 -59561 -59562 -59563 -59564 -59565 -59566 -59567 -59568 -59569 -59570 -59571 -59572 -59573 -59574 -59575 -59576 -59577 -59578 -59579 -59580 -59581 -59582 -59583 -59584 -59585 -59586 -59587 -59588 -59589 -59590 -59591 -59592 -59593 -59594 -59595 -59596 -59597 -59598 -59599 -59600 -59601 -59602 -59603 -59604 -59605 -59606 -59607 -59608 -59609 -59610 -59611 -59612 -59613 -59614 -59615 -59616 -59617 -59618 -59619 -59620 -59621 -59622 -59623 -59624 -59625 -59626 -59627 -59628 -59629 -59630 -59631 -59632 -59633 -59634 -59635 -59636 -59637 -59638 -59639 -59640 -59641 -59642 -59643 -59644 -59645 -59646 -59647 -59648 -59649 -59650 -59651 -59652 -59653 -59654 -59655 -59656 -59657 -59658 -59659 -59660 -59661 -59662 -59663 -59664 -59665 -59666 -59667 -59668 -59669 -59670 -59671 -59672 -59673 -59674 -59675 -59676 -59677 -59678 -59679 -59680 -59681 -59682 -59683 -59684 -59685 -59686 -59687 -59688 -59689 -59690 -59691 -59692 -59693 -59694 -59695 -59696 -59697 -59698 -59699 -59700 -59701 -59702 -59703 -59704 -59705 -59706 -59707 -59708 -59709 -59710 -59711 -59712 -59713 -59714 -59715 -59716 -59717 -59718 -59719 -59720 -59721 -59722 -59723 -59724 -59725 -59726 -59727 -59728 -59729 -59730 -59731 -59732 -59733 -59734 -59735 -59736 -59737 -59738 -59739 -59740 -59741 -59742 -59743 -59744 -59745 -59746 -59747 -59748 -59749 -59750 -59751 -59752 -59753 -59754 -59755 -59756 -59757 -59758 -59759 -59760 -59761 -59762 -59763 -59764 -59765 -59766 -59767 -59768 -59769 -59770 -59771 -59772 -59773 -59774 -59775 -59776 -59777 -59778 -59779 -59780 -59781 -59782 -59783 -59784 -59785 -59786 -59787 -59788 -59789 -59790 -59791 -59792 -59793 -59794 -59795 -59796 -59797 -59798 -59799 -59800 -59801 -59802 -59803 -59804 -59805 -59806 -59807 -59808 -59809 -59810 -59811 -59812 -59813 -59814 -59815 -59816 -59817 -59818 -59819 -59820 -59821 -59822 -59823 -59824 -59825 -59826 -59827 -59828 -59829 -59830 -59831 -59832 -59833 -59834 -59835 -59836 -59837 -59838 -59839 -59840 -59841 -59842 -59843 -59844 -59845 -59846 -59847 -59848 -59849 -59850 -59851 -59852 -59853 -59854 -59855 -59856 -59857 -59858 -59859 -59860 -59861 -59862 -59863 -59864 -59865 -59866 -59867 -59868 -59869 -59870 -59871 -59872 -59873 -59874 -59875 -59876 -59877 -59878 -59879 -59880 -59881 -59882 -59883 -59884 -59885 -59886 -59887 -59888 -59889 -59890 -59891 -59892 -59893 -59894 -59895 -59896 -59897 -59898 -59899 -59900 -59901 -59902 -59903 -59904 -59905 -59906 -59907 -59908 -59909 -59910 -59911 -59912 -59913 -59914 -59915 -59916 -59917 -59918 -59919 -59920 -59921 -59922 -59923 -59924 -59925 -59926 -59927 -59928 -59929 -59930 -59931 -59932 -59933 -59934 -59935 -59936 -59937 -59938 -59939 -59940 -59941 -59942 -59943 -59944 -59945 -59946 -59947 -59948 -59949 -59950 -59951 -59952 -59953 -59954 -59955 -59956 -59957 -59958 -59959 -59960 -59961 -59962 -59963 -59964 -59965 -59966 -59967 -59968 -59969 -59970 -59971 -59972 -59973 -59974 -59975 -59976 -59977 -59978 -59979 -59980 -59981 -59982 -59983 -59984 -59985 -59986 -59987 -59988 -59989 -59990 -59991 -59992 -59993 -59994 -59995 -59996 -59997 -59998 -59999 -[] -[[]] -[[],[0]] -[[],[0],[0,1]] -[[],[0],[0,1],[0,1,2]] -[[],[0],[0,1],[0,1,2],[0,1,2,3]] -[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4]] -[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5]] -[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6]] -[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7]] -[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8]] -[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9]] -[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10]] -[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11]] -[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12]] -[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13]] -[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14]] -[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15]] -[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16]] -[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17]] -[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18]] -[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19]] -[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20]] -[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21]] -[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22]] -[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23]] -[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24]] -[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25]] -[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26]] -[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27]] -[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28]] -[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29]] -[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30]] -[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31]] -[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32]] -[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33]] -[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34]] -[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35]] -[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36]] -[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37]] -[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38]] -[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39]] -[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40]] -[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41]] -[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42]] -[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43]] -[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44]] -[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45]] -[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46]] -[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47]] -[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48]] -[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49]] -[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50]] -[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51]] -[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52]] -[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53]] -[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54]] -[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55]] -[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56]] -[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57]] -[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58]] -[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59]] -[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60]] -[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61]] -[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62]] -[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63]] -[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64]] -[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65]] -[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66]] -[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67]] -[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68]] -[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69]] -[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70]] -[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71]] -[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72]] -[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73]] -[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74]] -[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75]] -[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76]] -[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77]] -[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78]] -[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79]] -[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80]] -[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81]] -[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82]] -[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83]] -[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84]] -[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85]] -[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86]] -[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87]] -[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88]] -[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89]] -[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90]] -[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91]] -[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92]] -[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93]] -[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94]] -[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95]] -[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96]] -[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97]] -[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98]] -[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99]] -[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100]] -[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101]] -[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102]] -[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103]] -[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104]] -[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105]] -[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106]] -[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107]] -[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108]] -[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109]] -[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110]] -[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111]] -[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112]] -[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113]] -[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113,114]] -[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113,114],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113,114,115]] -[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113,114],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113,114,115],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113,114,115,116]] -[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113,114],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113,114,115],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113,114,115,116],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113,114,115,116,117]] -[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113,114],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113,114,115],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113,114,115,116],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113,114,115,116,117],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113,114,115,116,117,118]] -[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113,114],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113,114,115],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113,114,115,116],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113,114,115,116,117],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113,114,115,116,117,118],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113,114,115,116,117,118,119]] -[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113,114],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113,114,115],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113,114,115,116],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113,114,115,116,117],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113,114,115,116,117,118],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113,114,115,116,117,118,119],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113,114,115,116,117,118,119,120]] -[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113,114],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113,114,115],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113,114,115,116],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113,114,115,116,117],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113,114,115,116,117,118],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113,114,115,116,117,118,119],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113,114,115,116,117,118,119,120],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113,114,115,116,117,118,119,120,121]] -[[],[0],[0,1],[0,1,2],[0,1,2,3],[0,1,2,3,4],[0,1,2,3,4,5],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7,8],[0,1,2,3,4,5,6,7,8,9],[0,1,2,3,4,5,6,7,8,9,10],[0,1,2,3,4,5,6,7,8,9,10,11],[0,1,2,3,4,5,6,7,8,9,10,11,12],[0,1,2,3,4,5,6,7,8,9,10,11,12,13],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113,114],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113,114,115],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113,114,115,116],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113,114,115,116,117],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113,114,115,116,117,118],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113,114,115,116,117,118,119],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113,114,115,116,117,118,119,120],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113,114,115,116,117,118,119,120,121],[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113,114,115,116,117,118,119,120,121,122]] +Code: 241. DB::Exception: Memory limit (total) exceeded: would use 11.19 MiB (attempt to allocate chunk of 6462982 bytes), maximum: 1.00 B: while receiving packet from localhost:9000. (MEMORY_LIMIT_EXCEEDED) +60000 diff --git a/tests/queries/0_stateless/02003_memory_limit_in_client.sh b/tests/queries/0_stateless/02003_memory_limit_in_client.sh index 29e4f1ae283..b3e2ae57c52 100755 --- a/tests/queries/0_stateless/02003_memory_limit_in_client.sh +++ b/tests/queries/0_stateless/02003_memory_limit_in_client.sh @@ -5,10 +5,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CURDIR"/../shell_config.sh # Big quiery. Memory limit -${CLICKHOUSE_CLIENT} --max_memory_usage_in_client=1 --testmode --query "SELECT arrayMap(x -> range(x), range(number)) FROM numbers(1000) -- { serverError 241; }" +${CLICKHOUSE_CLIENT} --max_memory_usage_in_client=1 --testmode --query "SELECT arrayMap(x -> range(x), range(number)) FROM numbers(1000) -- { serverError 241; }" 2>&1 | grep "Memory limit" # small quiery. In max untracked memory range -${CLICKHOUSE_CLIENT} --max_memory_usage_in_client=1 --testmode --query "SELECT * FROM system.numbers LIMIT 60000" - -# Have anough memory (unlimited). Better not to set limit, because of adress sanitizer -${CLICKHOUSE_CLIENT} --testmode --query "SELECT arrayMap(x -> range(x), range(number)) FROM numbers(125)" +${CLICKHOUSE_CLIENT} --max_memory_usage_in_client=1 --testmode --query "SELECT * FROM (SELECT * FROM system.numbers LIMIT 600000) as num WHERE num.number=60000" From 1f60cd735daf2f79f5bd7b949a2776cda631dae3 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 6 Aug 2021 17:52:23 +0300 Subject: [PATCH 215/599] Revert changes in 01710_projection_fetch --- .../queries/0_stateless/01710_projection_fetch.reference | 8 ++++---- tests/queries/0_stateless/01710_projection_fetch.sql | 4 ++-- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/tests/queries/0_stateless/01710_projection_fetch.reference b/tests/queries/0_stateless/01710_projection_fetch.reference index 6fc199d6ba2..abce5410b26 100644 --- a/tests/queries/0_stateless/01710_projection_fetch.reference +++ b/tests/queries/0_stateless/01710_projection_fetch.reference @@ -10,8 +10,8 @@ 3 3 4 4 0 -CREATE TABLE default.tp_2\n(\n `x` Int32,\n `y` Int32,\n PROJECTION p\n (\n SELECT \n x,\n y\n ORDER BY x\n ),\n PROJECTION pp\n (\n SELECT \n x,\n count()\n GROUP BY x\n )\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/{shard}/01710_projection_fetch_default\', \'2_{replica}\')\nORDER BY y\nSETTINGS min_rows_for_compact_part = 2, min_rows_for_wide_part = 4, min_bytes_for_compact_part = 16, min_bytes_for_wide_part = 32, index_granularity = 8192 +CREATE TABLE default.tp_2\n(\n `x` Int32,\n `y` Int32,\n PROJECTION p\n (\n SELECT \n x,\n y\n ORDER BY x\n ),\n PROJECTION pp\n (\n SELECT \n x,\n count()\n GROUP BY x\n )\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/{shard}/01710_projection_fetch_default\', \'2_{replica}\')\nORDER BY y\nSETTINGS min_rows_for_compact_part = 2, min_rows_for_wide_part = 4, min_bytes_for_compact_part = 16, min_bytes_for_wide_part = 32, index_granularity = 8192 2 -CREATE TABLE default.tp_2\n(\n `x` Int32,\n `y` Int32,\n PROJECTION p\n (\n SELECT \n x,\n y\n ORDER BY x\n ),\n PROJECTION pp\n (\n SELECT \n x,\n count()\n GROUP BY x\n )\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/{shard}/01710_projection_fetch_default\', \'2_{replica}\')\nORDER BY y\nSETTINGS min_rows_for_compact_part = 2, min_rows_for_wide_part = 4, min_bytes_for_compact_part = 16, min_bytes_for_wide_part = 32, index_granularity = 8192 -CREATE TABLE default.tp_2\n(\n `x` Int32,\n `y` Int32,\n PROJECTION p\n (\n SELECT \n x,\n y\n ORDER BY x\n ),\n PROJECTION pp\n (\n SELECT \n x,\n count()\n GROUP BY x\n )\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/{shard}/01710_projection_fetch_default\', \'2_{replica}\')\nORDER BY y\nSETTINGS min_rows_for_compact_part = 2, min_rows_for_wide_part = 4, min_bytes_for_compact_part = 16, min_bytes_for_wide_part = 32, index_granularity = 8192 -CREATE TABLE default.tp_2\n(\n `x` Int32,\n `y` Int32,\n PROJECTION p\n (\n SELECT \n x,\n y\n ORDER BY x\n )\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/{shard}/01710_projection_fetch_default\', \'2_{replica}\')\nORDER BY y\nSETTINGS min_rows_for_compact_part = 2, min_rows_for_wide_part = 4, min_bytes_for_compact_part = 16, min_bytes_for_wide_part = 32, index_granularity = 8192 +CREATE TABLE default.tp_2\n(\n `x` Int32,\n `y` Int32,\n PROJECTION p\n (\n SELECT \n x,\n y\n ORDER BY x\n ),\n PROJECTION pp\n (\n SELECT \n x,\n count()\n GROUP BY x\n )\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/{shard}/01710_projection_fetch_default\', \'2_{replica}\')\nORDER BY y\nSETTINGS min_rows_for_compact_part = 2, min_rows_for_wide_part = 4, min_bytes_for_compact_part = 16, min_bytes_for_wide_part = 32, index_granularity = 8192 +CREATE TABLE default.tp_2\n(\n `x` Int32,\n `y` Int32,\n PROJECTION p\n (\n SELECT \n x,\n y\n ORDER BY x\n ),\n PROJECTION pp\n (\n SELECT \n x,\n count()\n GROUP BY x\n )\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/{shard}/01710_projection_fetch_default\', \'2_{replica}\')\nORDER BY y\nSETTINGS min_rows_for_compact_part = 2, min_rows_for_wide_part = 4, min_bytes_for_compact_part = 16, min_bytes_for_wide_part = 32, index_granularity = 8192 +CREATE TABLE default.tp_2\n(\n `x` Int32,\n `y` Int32,\n PROJECTION p\n (\n SELECT \n x,\n y\n ORDER BY x\n )\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/{shard}/01710_projection_fetch_default\', \'2_{replica}\')\nORDER BY y\nSETTINGS min_rows_for_compact_part = 2, min_rows_for_wide_part = 4, min_bytes_for_compact_part = 16, min_bytes_for_wide_part = 32, index_granularity = 8192 diff --git a/tests/queries/0_stateless/01710_projection_fetch.sql b/tests/queries/0_stateless/01710_projection_fetch.sql index 5781d63788f..7e4f6cc1d9a 100644 --- a/tests/queries/0_stateless/01710_projection_fetch.sql +++ b/tests/queries/0_stateless/01710_projection_fetch.sql @@ -1,9 +1,9 @@ drop table if exists tp_1; drop table if exists tp_2; -create table tp_1 (x Int32, y Int32, projection p (select x, y order by x)) engine = ReplicatedMergeTree('/clickhouse/tables/{database}/{shard}/01710_projection_fetch_' || currentDatabase(), '1_{replica}') order by y settings min_rows_for_compact_part = 2, min_rows_for_wide_part = 4, min_bytes_for_compact_part = 16, min_bytes_for_wide_part = 32; +create table tp_1 (x Int32, y Int32, projection p (select x, y order by x)) engine = ReplicatedMergeTree('/clickhouse/tables/{shard}/01710_projection_fetch_' || currentDatabase(), '1_{replica}') order by y settings min_rows_for_compact_part = 2, min_rows_for_wide_part = 4, min_bytes_for_compact_part = 16, min_bytes_for_wide_part = 32; -create table tp_2 (x Int32, y Int32, projection p (select x, y order by x)) engine = ReplicatedMergeTree('/clickhouse/tables/{database}/{shard}/01710_projection_fetch_' || currentDatabase(), '2_{replica}') order by y settings min_rows_for_compact_part = 2, min_rows_for_wide_part = 4, min_bytes_for_compact_part = 16, min_bytes_for_wide_part = 32; +create table tp_2 (x Int32, y Int32, projection p (select x, y order by x)) engine = ReplicatedMergeTree('/clickhouse/tables/{shard}/01710_projection_fetch_' || currentDatabase(), '2_{replica}') order by y settings min_rows_for_compact_part = 2, min_rows_for_wide_part = 4, min_bytes_for_compact_part = 16, min_bytes_for_wide_part = 32; insert into tp_1 select number, number from numbers(3); From 27421284297ad34bf3b114ecbf19cadc0082c709 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Fri, 6 Aug 2021 18:03:05 +0300 Subject: [PATCH 216/599] Try update contrib/zlib-ng --- contrib/zlib-ng | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/zlib-ng b/contrib/zlib-ng index db232d30b4c..6a5e93b9007 160000 --- a/contrib/zlib-ng +++ b/contrib/zlib-ng @@ -1 +1 @@ -Subproject commit db232d30b4c72fd58e6d7eae2d12cebf9c3d90db +Subproject commit 6a5e93b9007782115f7f7e5235dedc81c4f1facb From 08590874b36af08bc8f2f2ad2e7302869d077d27 Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Fri, 6 Aug 2021 18:03:31 +0300 Subject: [PATCH 217/599] Minor fix. --- .../functions/string-functions.md | 89 ++++++++++++++++++- 1 file changed, 85 insertions(+), 4 deletions(-) diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index 3c4ab2cd99b..5dda7ec1d9f 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -45,7 +45,7 @@ The result type is UInt64. ## leftPad {#leftpad} -Pads the current string from the left with a specified string (multiple times, if needed) until the resulting string reaches the given length. +Pads the current string from the left with a specified string (multiple times, if needed) until the resulting string reaches the given length. Similarly to the MySQL `LPAD` function. **Syntax** @@ -83,14 +83,55 @@ Result: └────────────────────────┘ ``` -## rightPad {#rightpad} +## leftPadUTF8 {#leftpadutf8} -Pads the current string from the right with a specified string (multiple times, if needed) until the resulting string reaches the given length. +Pads the current string from the left with a specified string (multiple times, if needed) until the resulting string reaches the given length. Similarly to the MySQL `LPAD` function. **Syntax** ``` sql -leftPad('string','length', 'string2`) +leftPadUTF8('string','length', 'string2`) +``` + +**Arguments** + +- `string` — Input UTF-8 string, that need to be padded. [String](../data-types/string.md). +- `length` — The length of the resulting string once the input string pads. [UInt](../data-types/int-uint.md). If the value is less than input string length, then string is returned as-is. +- `string2` — The string to pad the current input string with. [String](../data-types/string.md). + +[String](../data-types/string.md) + +**Returned value(s)** + +- The resulting UTF-8 string reaches the given length. + +Type: [String](../data-types/string.md). + +**Example** + +Query: + +``` sql +SELECT leftPadUTF8('абвг', 7, '*'); +``` + +Result: + +``` text +┌─leftPadUTF8('абвг', 7, '*')─┐ +│ ***абвг │ +└─────────────────────────────┘ + +``` + +## rightPad {#rightpad} + +Pads the current string from the right with a specified string (multiple times, if needed) until the resulting string reaches the given length. Similarly to the MySQL `RPAD` function. + +**Syntax** + +``` sql +rightPad('string','length', 'string2`) ``` **Arguments** @@ -123,6 +164,46 @@ Result: └─────────────────────────┘ ``` +## rightPadUTF8 {#rightpadutf8} + +Pads the current UTF-8 string from the right with a specified string (multiple times, if needed) until the resulting string reaches the given length. Similarly to the MySQL `RPAD` function. + +**Syntax** + +``` sql +rightPadUTF8('string','length', 'string2`) +``` + +**Arguments** + +- `string` — Input UTF-8 string, that need to be padded. [String](../data-types/string.md). +- `length` — The length of the resulting string once the input string pads. [UInt](../data-types/int-uint.md). If the value is less than input string length, then string is returned as-is. +- `string2` — The string to pad the current input string with. [String](../data-types/string.md). + +[String](../data-types/string.md) + +**Returned value(s)** + +- The resulting UTF-8 string reaches the given length. + +Type: [String](../data-types/string.md). + +**Example** + +Query: + +``` sql +SELECT rightPadUTF8('абвг', 7, '*'); +``` + +Result: + +``` text +┌─rightPadUTF8('абвг', 7, '*')─┐ +│ абвг*** │ +└──────────────────────────────┘ +``` + ## lower, lcase {#lower} Converts ASCII Latin symbols in a string to lowercase. From 0a06470b69ebd4f7eeee7fc0f19fe56f09b62d0f Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Fri, 6 Aug 2021 18:12:31 +0300 Subject: [PATCH 218/599] tmp --- .../Formats/Impl/ArrowColumnToCHColumn.cpp | 49 +++++++++++++++++-- .../Formats/Impl/CHColumnToArrowColumn.cpp | 29 ++++++++++- 2 files changed, 73 insertions(+), 5 deletions(-) diff --git a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp index 4d358b3e84b..772e8a51d22 100644 --- a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp +++ b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp @@ -12,6 +12,7 @@ #include #include #include +#include #include #include #include @@ -34,9 +35,7 @@ #define FOR_ARROW_NUMERIC_TYPES(M) \ M(arrow::Type::UINT8, DB::UInt8) \ M(arrow::Type::INT8, DB::Int8) \ - M(arrow::Type::UINT16, DB::UInt16) \ M(arrow::Type::INT16, DB::Int16) \ - M(arrow::Type::UINT32, DB::UInt32) \ M(arrow::Type::INT32, DB::Int32) \ M(arrow::Type::UINT64, DB::UInt64) \ M(arrow::Type::INT64, DB::Int64) \ @@ -156,6 +155,30 @@ static ColumnWithTypeAndName readColumnWithBooleanData(std::shared_ptr & arrow_column, const String & column_name) +{ + auto internal_type = std::make_shared(); + auto internal_column = internal_type->createColumn(); + PaddedPODArray & column_data = assert_cast &>(*internal_column).getData(); + column_data.reserve(arrow_column->length()); + + for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i) + { + arrow::UInt16Array & chunk = dynamic_cast(*(arrow_column->chunk(chunk_i))); + + for (size_t value_i = 0, length = static_cast(chunk.length()); value_i < length; ++value_i) + { + UInt16 days_num = static_cast(chunk.Value(value_i)); + if (days_num > DATE_LUT_MAX_DAY_NUM) + throw Exception{ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE, + "Input value {} of a column \"{}\" is greater than max allowed Date value, which is {}", days_num, column_name, DATE_LUT_MAX_DAY_NUM}; + + column_data.emplace_back(days_num); + } + } + return {std::move(internal_column), std::move(internal_type), column_name}; +} + static ColumnWithTypeAndName readColumnWithDate32Data(std::shared_ptr & arrow_column, const String & column_name) { auto internal_type = std::make_shared(); @@ -183,7 +206,7 @@ static ColumnWithTypeAndName readColumnWithDate32Data(std::shared_ptr & arrow_column, const String & column_name) { - auto internal_type = std::make_shared(); + auto internal_type = std::make_shared(); auto internal_column = internal_type->createColumn(); auto & column_data = assert_cast &>(*internal_column).getData(); column_data.reserve(arrow_column->length()); @@ -200,6 +223,22 @@ static ColumnWithTypeAndName readColumnWithDate64Data(std::shared_ptr & arrow_column, const String & column_name) +{ + auto internal_type = std::make_shared(); + auto internal_column = internal_type->createColumn(); + auto & column_data = assert_cast &>(*internal_column).getData(); + column_data.reserve(arrow_column->length()); + + for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i) + { + auto & chunk = dynamic_cast(*(arrow_column->chunk(chunk_i))); + for (size_t value_i = 0, length = static_cast(chunk.length()); value_i < length; ++value_i) + column_data.emplace_back(chunk.Value(value_i)); + } + return {std::move(internal_column), std::move(internal_type), column_name}; +} + static ColumnWithTypeAndName readColumnWithTimestampData(std::shared_ptr & arrow_column, const String & column_name) { auto internal_type = std::make_shared(); @@ -353,6 +392,10 @@ static ColumnWithTypeAndName readColumnFromArrowColumn( return readColumnWithDate32Data(arrow_column, column_name); case arrow::Type::DATE64: return readColumnWithDate64Data(arrow_column, column_name); + case arrow::Type::UINT16: + return readColumnWithDateData(arrow_column, column_name); + case arrow::Type::UINT32: + return readColumnWithDateTimeData(arrow_column, column_name); case arrow::Type::TIMESTAMP: return readColumnWithTimestampData(arrow_column, column_name); #if defined(ARCADIA_BUILD) diff --git a/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp b/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp index 5928a26a7f9..8734f9c7279 100644 --- a/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp +++ b/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp @@ -72,6 +72,7 @@ namespace DB {"Date", arrow::uint16()}, /// uint16 is used instead of date32, because Apache Arrow cannot correctly serialize Date32Array. {"DateTime", arrow::uint32()}, /// uint32 is used instead of date64, because we don't need milliseconds. + {"Date32", arrow::date32()}, {"String", arrow::binary()}, {"FixedString", arrow::binary()}, @@ -335,7 +336,6 @@ namespace DB size_t end) { const PaddedPODArray & internal_data = assert_cast &>(*write_column).getData(); - //arrow::Date32Builder date_builder; arrow::UInt16Builder & builder = assert_cast(*array_builder); arrow::Status status; @@ -344,7 +344,6 @@ namespace DB if (null_bytemap && (*null_bytemap)[value_i]) status = builder.AppendNull(); else - /// Implicitly converts UInt16 to Int32 status = builder.Append(internal_data[value_i]); checkStatus(status, write_column->getName(), format_name); } @@ -373,6 +372,28 @@ namespace DB } } + static void fillArrowArrayWithDate32ColumnData( + ColumnPtr write_column, + const PaddedPODArray * null_bytemap, + const String & format_name, + arrow::ArrayBuilder* array_builder, + size_t start, + size_t end) + { + const PaddedPODArray & internal_data = assert_cast &>(*write_column).getData(); + arrow::Date32Builder & builder = assert_cast(*array_builder); + arrow::Status status; + + for (size_t value_i = start; value_i < end; ++value_i) + { + if (null_bytemap && (*null_bytemap)[value_i]) + status = builder.AppendNull(); + else + status = builder.Append(internal_data[value_i]); + checkStatus(status, write_column->getName(), format_name); + } + } + static void fillArrowArray( const String & column_name, ColumnPtr & column, @@ -411,6 +432,10 @@ namespace DB { fillArrowArrayWithDateTimeColumnData(column, null_bytemap, format_name, array_builder, start, end); } + else if (isDate32(column_type)) + { + fillArrowArrayWithDate32ColumnData(column, null_bytemap, format_name, array_builder, start, end); + } else if (isArray(column_type)) { fillArrowArrayWithArrayColumnData(column_name, column, column_type, null_bytemap, array_builder, format_name, start, end, dictionary_values); From 51198a1654cc7fa97b7f13fdf8ce7acdd654bc77 Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Fri, 6 Aug 2021 18:24:04 +0300 Subject: [PATCH 219/599] Minor fix. --- docs/en/sql-reference/data-types/string.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/data-types/string.md b/docs/en/sql-reference/data-types/string.md index cb3a70ec7f8..2cf11ac85a3 100644 --- a/docs/en/sql-reference/data-types/string.md +++ b/docs/en/sql-reference/data-types/string.md @@ -15,6 +15,6 @@ When creating tables, numeric parameters for string fields can be set (e.g. `VAR ClickHouse does not have the concept of encodings. Strings can contain an arbitrary set of bytes, which are stored and output as-is. If you need to store texts, we recommend using UTF-8 encoding. At the very least, if your terminal uses UTF-8 (as recommended), you can read and write your values without making conversions. Similarly, certain functions for working with strings have separate variations that work under the assumption that the string contains a set of bytes representing a UTF-8 encoded text. -For example, the ‘length’ function calculates the string length in bytes, while the ‘lengthUTF8’ function calculates the string length in Unicode code points, assuming that the value is UTF-8 encoded. +For example, the [length](../functions/string-functions.md#length) function calculates the string length in bytes, while the [lengthUTF8](../functions/string-functions.md#lengthutf8) function calculates the string length in Unicode code points, assuming that the value is UTF-8 encoded. [Original article](https://clickhouse.tech/docs/en/data_types/string/) From 28901ef55f7c4a71b346329a27aae0549816be3e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 6 Aug 2021 17:25:24 +0200 Subject: [PATCH 220/599] Fix 01300_client_save_history_when_terminated_long It was broken because of colored output --- .../01300_client_save_history_when_terminated_long.expect | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01300_client_save_history_when_terminated_long.expect b/tests/queries/0_stateless/01300_client_save_history_when_terminated_long.expect index 7fed383dc38..0c53f5658d1 100755 --- a/tests/queries/0_stateless/01300_client_save_history_when_terminated_long.expect +++ b/tests/queries/0_stateless/01300_client_save_history_when_terminated_long.expect @@ -27,7 +27,7 @@ close spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT" expect ":) " send -- "\[A" -expect "SELECT 'for the history'" +expect "for the history" # Will check that Ctrl+C clears current line. send -- "\3" From 3e0a6bb77c571ff2c1cedc91d0e84e9164e42a5d Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Fri, 6 Aug 2021 18:26:27 +0300 Subject: [PATCH 221/599] Minor fix. --- .../sql-reference/functions/string-functions.md | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index 5dda7ec1d9f..d9bbaeec329 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -50,14 +50,14 @@ Pads the current string from the left with a specified string (multiple times, i **Syntax** ``` sql -leftPad('string','length', 'string2`) +leftPad('string','length', 'pad_string') ``` **Arguments** - `string` — Input string, that need to be padded. [String](../data-types/string.md). - `length` — The length of the resulting string once the input string pads. [UInt](../data-types/int-uint.md). If the value is less than input string length, then string is returned as-is. -- `string2` — The string to pad the current input string with. [String](../data-types/string.md). +- `pad_string` — The string to pad the current input string with. [String](../data-types/string.md). [String](../data-types/string.md) @@ -90,14 +90,14 @@ Pads the current string from the left with a specified string (multiple times, i **Syntax** ``` sql -leftPadUTF8('string','length', 'string2`) +leftPadUTF8('string','length', 'pad_string') ``` **Arguments** - `string` — Input UTF-8 string, that need to be padded. [String](../data-types/string.md). - `length` — The length of the resulting string once the input string pads. [UInt](../data-types/int-uint.md). If the value is less than input string length, then string is returned as-is. -- `string2` — The string to pad the current input string with. [String](../data-types/string.md). +- `pad_string` — The string to pad the current input string with. [String](../data-types/string.md). [String](../data-types/string.md) @@ -131,14 +131,14 @@ Pads the current string from the right with a specified string (multiple times, **Syntax** ``` sql -rightPad('string','length', 'string2`) +rightPad('string','length', 'pad_string') ``` **Arguments** - `string` — Input string, that need to be padded. [String](../data-types/string.md). - `length` — The length of the resulting string once the input string pads. [UInt](../data-types/int-uint.md). If the value is less than input string length, then string is returned as-is. -- `string2` — The string to pad the current input string with. [String](../data-types/string.md). +- `pad_string` — The string to pad the current input string with. [String](../data-types/string.md). [String](../data-types/string.md) @@ -171,14 +171,14 @@ Pads the current UTF-8 string from the right with a specified string (multiple t **Syntax** ``` sql -rightPadUTF8('string','length', 'string2`) +rightPadUTF8('string','length', 'pad_string') ``` **Arguments** - `string` — Input UTF-8 string, that need to be padded. [String](../data-types/string.md). - `length` — The length of the resulting string once the input string pads. [UInt](../data-types/int-uint.md). If the value is less than input string length, then string is returned as-is. -- `string2` — The string to pad the current input string with. [String](../data-types/string.md). +- `pad_string` — The string to pad the current input string with. [String](../data-types/string.md). [String](../data-types/string.md) From b728bdb5ec86018aad2ddb513431e5260651a24a Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Fri, 6 Aug 2021 18:36:56 +0300 Subject: [PATCH 222/599] Minor fix. --- docs/en/sql-reference/functions/string-functions.md | 4 ---- 1 file changed, 4 deletions(-) diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index d9bbaeec329..767abfd4bac 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -140,8 +140,6 @@ rightPad('string','length', 'pad_string') - `length` — The length of the resulting string once the input string pads. [UInt](../data-types/int-uint.md). If the value is less than input string length, then string is returned as-is. - `pad_string` — The string to pad the current input string with. [String](../data-types/string.md). -[String](../data-types/string.md) - **Returned value(s)** - The resulting string reaches the given length. @@ -180,8 +178,6 @@ rightPadUTF8('string','length', 'pad_string') - `length` — The length of the resulting string once the input string pads. [UInt](../data-types/int-uint.md). If the value is less than input string length, then string is returned as-is. - `pad_string` — The string to pad the current input string with. [String](../data-types/string.md). -[String](../data-types/string.md) - **Returned value(s)** - The resulting UTF-8 string reaches the given length. From 8180b47ba135477dbb02c9ac70bfc3f726b02181 Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Fri, 6 Aug 2021 18:44:46 +0300 Subject: [PATCH 223/599] Minor fix. --- docs/en/sql-reference/functions/string-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index 767abfd4bac..f99fd698b0e 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -206,7 +206,7 @@ Converts ASCII Latin symbols in a string to lowercase. ## upper, ucase {#upper} -Converts ASCII Latin symbols in a string to uppercase.pd +Converts ASCII Latin symbols in a string to uppercase. ## lowerUTF8 {#lowerutf8} From b1ff4ca81a5940126b095258f385b2c27abcdb71 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 6 Aug 2021 18:03:27 +0200 Subject: [PATCH 224/599] Fix 01176_mysql_client_interactive and work with mariadb client --- src/Formats/FormatFactory.cpp | 4 ++++ .../0_stateless/01176_mysql_client_interactive.expect | 5 ++--- 2 files changed, 6 insertions(+), 3 deletions(-) diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index d2d6d92dea3..0132a37113a 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -314,6 +314,10 @@ OutputFormatPtr FormatFactory::getOutputFormatParallelIfPossible( auto format_settings = _format_settings ? *_format_settings : getFormatSettings(context); + /// If we're handling MySQL protocol connection right now then MySQLWire is only allowed output format. + if (format_settings.mysql_wire.sequence_id && (name != "MySQLWire")) + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "MySQL protocol does not support custom output formats"); + const Settings & settings = context->getSettingsRef(); if (settings.output_format_parallel_formatting && getCreators(name).supports_parallel_formatting diff --git a/tests/queries/0_stateless/01176_mysql_client_interactive.expect b/tests/queries/0_stateless/01176_mysql_client_interactive.expect index 2337b7d01fe..6f0cea48e76 100755 --- a/tests/queries/0_stateless/01176_mysql_client_interactive.expect +++ b/tests/queries/0_stateless/01176_mysql_client_interactive.expect @@ -12,7 +12,7 @@ expect_after { set basedir [file dirname $argv0] spawn bash -c "source $basedir/../shell_config.sh ; \$MYSQL_CLIENT_BINARY \$MYSQL_CLIENT_OPT" -expect "mysql> " +expect -nocase -re "mysql.*> " send -- "USE system;\r" expect "Database changed" @@ -41,8 +41,7 @@ expect "+---------------+-------------+" expect "| 10 | 45 |" expect "+---------------+-------------+" expect "1 row in set" -expect "Read 10 rows, 80.00 B" -expect "mysql> " +expect -nocase -re "mysql.*> " send -- "quit;\r" expect eof From 8439135960b1dba64d15dc3e95be42530847ff4d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 6 Aug 2021 18:28:44 +0200 Subject: [PATCH 225/599] Try to make the test faster --- tests/queries/0_stateless/00738_lock_for_inner_table.sh | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/00738_lock_for_inner_table.sh b/tests/queries/0_stateless/00738_lock_for_inner_table.sh index 6cbed42baea..37d5755fb0b 100755 --- a/tests/queries/0_stateless/00738_lock_for_inner_table.sh +++ b/tests/queries/0_stateless/00738_lock_for_inner_table.sh @@ -11,9 +11,7 @@ uuid=$(${CLICKHOUSE_CLIENT} --query "SELECT reinterpretAsUUID(currentDatabase()) echo "DROP TABLE IF EXISTS tab_00738 SYNC; DROP TABLE IF EXISTS mv SYNC; --- create table with fsync and 20 partitions for slower INSERT --- (since increasing number of records will make it significantly slower in debug build, but not in release) -CREATE TABLE tab_00738(a Int) ENGINE = MergeTree() ORDER BY a PARTITION BY a%20 SETTINGS fsync_after_insert=1; +CREATE TABLE tab_00738(a Int) ENGINE = MergeTree() ORDER BY a; -- The matview will take at least 2 seconds to be finished (10000000 * 0.0000002) CREATE MATERIALIZED VIEW mv UUID '$uuid' ENGINE = Log AS SELECT sleepEachRow(0.0000002) FROM tab_00738;" | ${CLICKHOUSE_CLIENT} -n From 3cbeab89595c348a7a84fcb5fc80a9fe4926058a Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 6 Aug 2021 21:16:06 +0300 Subject: [PATCH 226/599] Fix incorrect row-level filtering --- src/Storages/MergeTree/MergeTreeRangeReader.cpp | 17 +++++++++++------ 1 file changed, 11 insertions(+), 6 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeRangeReader.cpp b/src/Storages/MergeTree/MergeTreeRangeReader.cpp index 2347280a4a0..f1db130caaf 100644 --- a/src/Storages/MergeTree/MergeTreeRangeReader.cpp +++ b/src/Storages/MergeTree/MergeTreeRangeReader.cpp @@ -859,8 +859,7 @@ static ColumnPtr combineFilters(ColumnPtr first, ColumnPtr second) return first; } - auto mut_first = IColumn::mutate(std::move(first)); - FilterDescription firsrt_descr(*mut_first); + FilterDescription firsrt_descr(*first); size_t bytes_in_first_filter = countBytesInFilter(*firsrt_descr.data); checkCombindeFiltersSize(bytes_in_first_filter, second->size()); @@ -868,13 +867,20 @@ static ColumnPtr combineFilters(ColumnPtr first, ColumnPtr second) ConstantFilterDescription second_const_descr(*second); if (second_const_descr.always_true) - return mut_first; + return first; if (second_const_descr.always_false) - return second->cloneResized(mut_first->size()); + return second->cloneResized(first->size()); FilterDescription second_descr(*second); - auto & first_data = const_cast(*firsrt_descr.data); + + MutableColumnPtr mut_first; + if (firsrt_descr.data_holder) + mut_first = IColumn::mutate(std::move(firsrt_descr.data_holder)); + else + mut_first = IColumn::mutate(std::move(first)); + + auto & first_data = typeid_cast(mut_first.get())->getData(); const auto * second_data = second_descr.data->data(); for (auto & val : first_data) @@ -968,7 +974,6 @@ void MergeTreeRangeReader::executePrewhereActionsAndFilterColumns(ReadResult & r { row_level_filter = combineFilters(std::move(row_level_filter), filter); result.setFilter(row_level_filter); - } else result.setFilter(filter); From e068ea391338fc9d5860e136a474a2b6626bd5a4 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 6 Aug 2021 21:51:43 +0300 Subject: [PATCH 227/599] Add test. --- tests/queries/0_stateless/02002_row_level_filter_bug.reference | 3 +++ 1 file changed, 3 insertions(+) create mode 100644 tests/queries/0_stateless/02002_row_level_filter_bug.reference diff --git a/tests/queries/0_stateless/02002_row_level_filter_bug.reference b/tests/queries/0_stateless/02002_row_level_filter_bug.reference new file mode 100644 index 00000000000..ea14f6fa1dd --- /dev/null +++ b/tests/queries/0_stateless/02002_row_level_filter_bug.reference @@ -0,0 +1,3 @@ +600 +AWD 3130 2021-07-15 600 +0 From 28c33ce7c446796b6dbd221d9259955170539158 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 6 Aug 2021 21:53:22 +0300 Subject: [PATCH 228/599] Add test. --- .../0_stateless/02002_row_level_filter_bug.sh | 75 +++++++++++++++++++ 1 file changed, 75 insertions(+) create mode 100755 tests/queries/0_stateless/02002_row_level_filter_bug.sh diff --git a/tests/queries/0_stateless/02002_row_level_filter_bug.sh b/tests/queries/0_stateless/02002_row_level_filter_bug.sh new file mode 100755 index 00000000000..6c2422a235f --- /dev/null +++ b/tests/queries/0_stateless/02002_row_level_filter_bug.sh @@ -0,0 +1,75 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + + +$CLICKHOUSE_CLIENT --query "drop table if exists test_table" +$CLICKHOUSE_CLIENT --query "CREATE TABLE test_table +( + a UInt16 DEFAULT 0, + c LowCardinality(String) DEFAULT '', + t_date LowCardinality(String) DEFAULT '', + ex LowCardinality(String) DEFAULT '', + team LowCardinality(String) DEFAULT '', + g LowCardinality(String) DEFAULT '', + mt FixedString(1) DEFAULT ' ', + rw_ts Int64 DEFAULT 0, + exr_t Int64 DEFAULT 0, + en UInt16 DEFAULT 0, + f_t Int64 DEFAULT 0, + j UInt64 DEFAULT 0, + oj UInt64 DEFAULT 0 +) +ENGINE = MergeTree +PARTITION BY (c, t_date) +ORDER BY (ex, team, g, mt, rw_ts, exr_t, en, f_t, j, oj) +SETTINGS index_granularity = 8192" + +$CLICKHOUSE_CLIENT --query " +INSERT INTO test_table(t_date, c,team, a) SELECT +arrayJoin([toDate('2021-07-15'),toDate('2021-07-16')]) as t_date, +arrayJoin(['aur','rua']) as c, +arrayJoin(['AWD','ZZZ']) as team, +arrayJoin([3183,3106,0,3130,3108,3126,3109,3107,3182,3180,3129,3128,3125,3266]) as a +FROM numbers(600);" + +$CLICKHOUSE_CLIENT --query "DROP ROLE IF exists AWD;" +$CLICKHOUSE_CLIENT --query "create role AWD;" +$CLICKHOUSE_CLIENT --query "REVOKE ALL ON *.* FROM AWD;" + +$CLICKHOUSE_CLIENT --query "DROP USER IF EXISTS AWD_user;" +$CLICKHOUSE_CLIENT --query "CREATE USER AWD_user IDENTIFIED WITH SHA256_PASSWORD BY 'AWD_pwd' DEFAULT ROLE AWD;" + +$CLICKHOUSE_CLIENT --query "GRANT SELECT ON test_table TO AWD;" + +$CLICKHOUSE_CLIENT --query "DROP ROW POLICY IF EXISTS ttt_bu_test_table_AWD ON test_table;" +$CLICKHOUSE_CLIENT --query "CREATE ROW POLICY ttt_bu_test_table_AWD ON test_table FOR SELECT USING team = 'AWD' TO AWD;" + +$CLICKHOUSE_CLIENT --user=AWD_user --password=AWD_pwd --query " +SELECT count() AS count + FROM test_table +WHERE + t_date = '2021-07-15' AND c = 'aur' AND a=3130; +" + +$CLICKHOUSE_CLIENT --user=AWD_user --password=AWD_pwd --query " +SELECT + team, + a, + t_date, + count() AS count +FROM test_table +WHERE (t_date = '2021-07-15') AND (c = 'aur') AND (a = 3130) +GROUP BY + team, + a, + t_date; +" + +$CLICKHOUSE_CLIENT --user=AWD_user --password=AWD_pwd --query " +SELECT count() AS count +FROM test_table +WHERE (t_date = '2021-07-15') AND (c = 'aur') AND (a = 313) +" From 74dd3a6af69aaa2daf88e7da4238bae19916b56d Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 6 Aug 2021 22:10:54 +0300 Subject: [PATCH 229/599] More fixes. --- src/Processors/QueryPlan/ReadFromMergeTree.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 5fc32e2db69..4276160f514 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -992,8 +992,7 @@ void ReadFromMergeTree::initializePipeline(QueryPipeline & pipeline, const Build }); } - Block cur_header = result_projection ? result_projection->getResultColumns() - : pipe.getHeader(); + Block cur_header = pipe.getHeader(); auto append_actions = [&result_projection](ActionsDAGPtr actions) { From e2cee2576d797b55d74e82a50f96b6d2d5f44727 Mon Sep 17 00:00:00 2001 From: Mike Kot Date: Fri, 6 Aug 2021 22:33:21 +0300 Subject: [PATCH 230/599] Fixing reading of /proc/meminfo --- src/Interpreters/AsynchronousMetrics.cpp | 83 ++++++++++++++---------- 1 file changed, 50 insertions(+), 33 deletions(-) diff --git a/src/Interpreters/AsynchronousMetrics.cpp b/src/Interpreters/AsynchronousMetrics.cpp index d708ff4f9e0..fe34d5c5bc6 100644 --- a/src/Interpreters/AsynchronousMetrics.cpp +++ b/src/Interpreters/AsynchronousMetrics.cpp @@ -779,43 +779,60 @@ void AsynchronousMetrics::update(std::chrono::system_clock::time_point update_ti uint64_t kb = 0; readText(kb, *meminfo); - if (kb) + + if (!kb) { - skipWhitespaceIfAny(*meminfo, true); - assertString("kB", *meminfo); + skipToNextLineOrEOF(*meminfo); + continue; + } - uint64_t bytes = kb * 1024; + skipWhitespaceIfAny(*meminfo, true); - if (name == "MemTotal:") - { - new_values["OSMemoryTotal"] = bytes; - } - else if (name == "MemFree:") - { - /// We cannot simply name this metric "Free", because it confuses users. - /// See https://www.linuxatemyram.com/ - /// For convenience we also provide OSMemoryFreePlusCached, that should be somewhat similar to OSMemoryAvailable. + /** + * Not all entries in /proc/meminfo contain the kB suffix, e.g. + * HugePages_Total: 0 + * HugePages_Free: 0 + * We simply skip such entries as they're not needed + */ + if (*meminfo->position() == '\n') + { + skipToNextLineOrEOF(*meminfo); + continue; + } - free_plus_cached_bytes += bytes; - new_values["OSMemoryFreeWithoutCached"] = bytes; - } - else if (name == "MemAvailable:") - { - new_values["OSMemoryAvailable"] = bytes; - } - else if (name == "Buffers:") - { - new_values["OSMemoryBuffers"] = bytes; - } - else if (name == "Cached:") - { - free_plus_cached_bytes += bytes; - new_values["OSMemoryCached"] = bytes; - } - else if (name == "SwapCached:") - { - new_values["OSMemorySwapCached"] = bytes; - } + assertString("kB", *meminfo); + + uint64_t bytes = kb * 1024; + + if (name == "MemTotal:") + { + new_values["OSMemoryTotal"] = bytes; + } + else if (name == "MemFree:") + { + /// We cannot simply name this metric "Free", because it confuses users. + /// See https://www.linuxatemyram.com/ + /// For convenience we also provide OSMemoryFreePlusCached, that should be somewhat similar to OSMemoryAvailable. + + free_plus_cached_bytes += bytes; + new_values["OSMemoryFreeWithoutCached"] = bytes; + } + else if (name == "MemAvailable:") + { + new_values["OSMemoryAvailable"] = bytes; + } + else if (name == "Buffers:") + { + new_values["OSMemoryBuffers"] = bytes; + } + else if (name == "Cached:") + { + free_plus_cached_bytes += bytes; + new_values["OSMemoryCached"] = bytes; + } + else if (name == "SwapCached:") + { + new_values["OSMemorySwapCached"] = bytes; } skipToNextLineOrEOF(*meminfo); From 1fcbd1e196403ae43a16d5e75e1b7b740930aa5a Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 6 Aug 2021 23:46:29 +0300 Subject: [PATCH 231/599] Mark more tests as long --- ...ence => 00563_insert_into_remote_and_zookeeper_long.reference} | 0 ...keeper.sql => 00563_insert_into_remote_and_zookeeper_long.sql} | 0 ...32_quorum_insert_simple_test_1_parts_zookeeper_long.reference} | 0 ...=> 00732_quorum_insert_simple_test_1_parts_zookeeper_long.sql} | 0 4 files changed, 0 insertions(+), 0 deletions(-) rename tests/queries/0_stateless/{00563_insert_into_remote_and_zookeeper.reference => 00563_insert_into_remote_and_zookeeper_long.reference} (100%) rename tests/queries/0_stateless/{00563_insert_into_remote_and_zookeeper.sql => 00563_insert_into_remote_and_zookeeper_long.sql} (100%) rename tests/queries/0_stateless/{00732_quorum_insert_simple_test_1_parts_zookeeper.reference => 00732_quorum_insert_simple_test_1_parts_zookeeper_long.reference} (100%) rename tests/queries/0_stateless/{00732_quorum_insert_simple_test_1_parts_zookeeper.sql => 00732_quorum_insert_simple_test_1_parts_zookeeper_long.sql} (100%) diff --git a/tests/queries/0_stateless/00563_insert_into_remote_and_zookeeper.reference b/tests/queries/0_stateless/00563_insert_into_remote_and_zookeeper_long.reference similarity index 100% rename from tests/queries/0_stateless/00563_insert_into_remote_and_zookeeper.reference rename to tests/queries/0_stateless/00563_insert_into_remote_and_zookeeper_long.reference diff --git a/tests/queries/0_stateless/00563_insert_into_remote_and_zookeeper.sql b/tests/queries/0_stateless/00563_insert_into_remote_and_zookeeper_long.sql similarity index 100% rename from tests/queries/0_stateless/00563_insert_into_remote_and_zookeeper.sql rename to tests/queries/0_stateless/00563_insert_into_remote_and_zookeeper_long.sql diff --git a/tests/queries/0_stateless/00732_quorum_insert_simple_test_1_parts_zookeeper.reference b/tests/queries/0_stateless/00732_quorum_insert_simple_test_1_parts_zookeeper_long.reference similarity index 100% rename from tests/queries/0_stateless/00732_quorum_insert_simple_test_1_parts_zookeeper.reference rename to tests/queries/0_stateless/00732_quorum_insert_simple_test_1_parts_zookeeper_long.reference diff --git a/tests/queries/0_stateless/00732_quorum_insert_simple_test_1_parts_zookeeper.sql b/tests/queries/0_stateless/00732_quorum_insert_simple_test_1_parts_zookeeper_long.sql similarity index 100% rename from tests/queries/0_stateless/00732_quorum_insert_simple_test_1_parts_zookeeper.sql rename to tests/queries/0_stateless/00732_quorum_insert_simple_test_1_parts_zookeeper_long.sql From 5e2bfd5ba1c3331401b5f75a8bbdb25a948c9cb4 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 6 Aug 2021 21:03:38 +0300 Subject: [PATCH 232/599] fix partition id validation --- src/Storages/MergeTree/MergeTreeData.cpp | 8 ++--- src/Storages/MergeTree/MergeTreePartInfo.cpp | 32 ++++++------------- src/Storages/MergeTree/MergeTreePartInfo.h | 2 +- .../01925_broken_partition_id_zookeeper.sql | 10 ++++++ 4 files changed, 23 insertions(+), 29 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 60ff3d094b7..4730bf9f47c 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -3213,8 +3213,7 @@ String MergeTreeData::getPartitionIDFromQuery(const ASTPtr & ast, ContextPtr loc if (!partition_ast.value) { - if (!MergeTreePartInfo::validatePartitionID(partition_ast.id, format_version)) - throw Exception("Invalid partition format: " + partition_ast.id, ErrorCodes::INVALID_PARTITION_VALUE); + MergeTreePartInfo::validatePartitionID(partition_ast.id, format_version); return partition_ast.id; } @@ -3225,10 +3224,7 @@ String MergeTreeData::getPartitionIDFromQuery(const ASTPtr & ast, ContextPtr loc if (partition_lit && partition_lit->value.getType() == Field::Types::String) { String partition_id = partition_lit->value.get(); - if (partition_id.size() != 6 || !std::all_of(partition_id.begin(), partition_id.end(), isNumericASCII)) - throw Exception( - "Invalid partition format: " + partition_id + ". Partition should consist of 6 digits: YYYYMM", - ErrorCodes::INVALID_PARTITION_VALUE); + MergeTreePartInfo::validatePartitionID(partition_id, format_version); return partition_id; } } diff --git a/src/Storages/MergeTree/MergeTreePartInfo.cpp b/src/Storages/MergeTree/MergeTreePartInfo.cpp index ccb26a0999e..6a98e666c34 100644 --- a/src/Storages/MergeTree/MergeTreePartInfo.cpp +++ b/src/Storages/MergeTree/MergeTreePartInfo.cpp @@ -9,6 +9,7 @@ namespace DB namespace ErrorCodes { extern const int BAD_DATA_PART_NAME; + extern const int INVALID_PARTITION_VALUE; } @@ -21,38 +22,25 @@ MergeTreePartInfo MergeTreePartInfo::fromPartName(const String & part_name, Merg } -bool MergeTreePartInfo::validatePartitionID(const String & partition_id, MergeTreeDataFormatVersion format_version) +void MergeTreePartInfo::validatePartitionID(const String & partition_id, MergeTreeDataFormatVersion format_version) { if (partition_id.empty()) - return false; - - ReadBufferFromString in(partition_id); + throw Exception(ErrorCodes::INVALID_PARTITION_VALUE, "Partition id is empty"); if (format_version < MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING) { - UInt32 min_yyyymmdd = 0; - UInt32 max_yyyymmdd = 0; - if (!tryReadIntText(min_yyyymmdd, in) - || !checkChar('_', in) - || !tryReadIntText(max_yyyymmdd, in) - || !checkChar('_', in)) - { - return false; - } + if (partition_id.size() != 6 || !std::all_of(partition_id.begin(), partition_id.end(), isNumericASCII)) + throw Exception(ErrorCodes::INVALID_PARTITION_VALUE, + "Invalid partition format: {}. Partition should consist of 6 digits: YYYYMM", + partition_id); } else { - while (!in.eof()) - { - char c; - readChar(c, in); - - if (c == '_') - break; - } + auto is_valid_char = [](char c) { return c == '-' || isAlphaNumericASCII(c); }; + if (!std::all_of(partition_id.begin(), partition_id.end(), is_valid_char)) + throw Exception(ErrorCodes::INVALID_PARTITION_VALUE, "Invalid partition format: {}", partition_id); } - return in.eof(); } bool MergeTreePartInfo::tryParsePartName(const String & part_name, MergeTreePartInfo * part_info, MergeTreeDataFormatVersion format_version) diff --git a/src/Storages/MergeTree/MergeTreePartInfo.h b/src/Storages/MergeTree/MergeTreePartInfo.h index 87f96ed5038..be856c1f157 100644 --- a/src/Storages/MergeTree/MergeTreePartInfo.h +++ b/src/Storages/MergeTree/MergeTreePartInfo.h @@ -88,7 +88,7 @@ struct MergeTreePartInfo } /// Simple sanity check for partition ID. Checking that it's not too long or too short, doesn't contain a lot of '_'. - static bool validatePartitionID(const String & partition_id, MergeTreeDataFormatVersion format_version); + static void validatePartitionID(const String & partition_id, MergeTreeDataFormatVersion format_version); static MergeTreePartInfo fromPartName(const String & part_name, MergeTreeDataFormatVersion format_version); // -V1071 diff --git a/tests/queries/0_stateless/01925_broken_partition_id_zookeeper.sql b/tests/queries/0_stateless/01925_broken_partition_id_zookeeper.sql index baf6c1fbf8f..07e490d0ce0 100644 --- a/tests/queries/0_stateless/01925_broken_partition_id_zookeeper.sql +++ b/tests/queries/0_stateless/01925_broken_partition_id_zookeeper.sql @@ -14,3 +14,13 @@ ALTER TABLE broken_partition DROP PARTITION ID '20210325_0_13241_6_12747'; --{se ALTER TABLE broken_partition DROP PARTITION ID '20210325_0_13241_6_12747'; --{serverError 248} DROP TABLE IF EXISTS broken_partition; + +DROP TABLE IF EXISTS old_partition_key; + +CREATE TABLE old_partition_key (sd Date, dh UInt64, ak UInt32, ed Date) ENGINE=MergeTree(sd, dh, (ak, ed, dh), 8192); + +ALTER TABLE old_partition_key DROP PARTITION ID '20210325_0_13241_6_12747'; --{serverError 248} + +ALTER TABLE old_partition_key DROP PARTITION ID '202103'; + +DROP TABLE old_partition_key; From 6189757fed5e429c32f488d7ae6d9ad9d6df085c Mon Sep 17 00:00:00 2001 From: Braulio Valdivielso Date: Fri, 6 Aug 2021 23:29:49 +0100 Subject: [PATCH 233/599] change http error response As reported in #27171, the logic that generates the HTTP error response didn't work in case the configuration hadn't specified a `http_port`. The new logic checks whether the `http_port` was set in the first place. --- src/Server/TCPHandler.cpp | 27 +++++++++++++++++++++++---- 1 file changed, 23 insertions(+), 4 deletions(-) diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 269c33d952e..7774c9ca56e 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -32,6 +32,7 @@ #include #include #include +#include #include @@ -46,6 +47,26 @@ namespace DB { +namespace +{ +std::string formatHTTPErrorResponse(const Poco::Util::AbstractConfiguration& config) +{ + std::string result = fmt::format( + "HTTP/1.0 400 Bad Request\r\n\r\n" + "Port {} is for clickhouse-client program\r\n", + config.getString("tcp_port")); + + if (config.has("http_port")) { + result += fmt::format( + "You must use port {} for HTTP.\r\n", + config.getString("http_port")); + } + + return result; +} +} + + namespace ErrorCodes { extern const int LOGICAL_ERROR; @@ -922,10 +943,8 @@ void TCPHandler::receiveHello() */ if (packet_type == 'G' || packet_type == 'P') { - writeString("HTTP/1.0 400 Bad Request\r\n\r\n" - "Port " + server.config().getString("tcp_port") + " is for clickhouse-client program.\r\n" - "You must use port " + server.config().getString("http_port") + " for HTTP.\r\n", - *out); + writeString(formatHTTPErrorResponse(server.config()), + *out); throw Exception("Client has connected to wrong port", ErrorCodes::CLIENT_HAS_CONNECTED_TO_WRONG_PORT); } From 9a45458faf1b2d567f408739afe9b46ddffa35be Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Fri, 6 Aug 2021 23:55:04 +0000 Subject: [PATCH 234/599] Add bzip2 file compression --- .gitmodules | 3 + CMakeLists.txt | 1 + cmake/find/bzip2.cmake | 19 +++ contrib/CMakeLists.txt | 4 + contrib/bzip2-cmake/CMakeLists.txt | 23 +++ src/CMakeLists.txt | 5 + src/Common/ErrorCodes.cpp | 2 + src/Common/config.h.in | 1 + src/IO/Bzip2ReadBuffer.cpp | 97 ++++++++++++ src/IO/Bzip2ReadBuffer.h | 33 +++++ src/IO/Bzip2WriteBuffer.cpp | 138 ++++++++++++++++++ src/IO/Bzip2WriteBuffer.h | 37 +++++ src/IO/CompressionMethod.cpp | 18 ++- src/IO/CompressionMethod.h | 3 +- src/IO/ya.make | 2 + ...StorageSystemBuildOptions.generated.cpp.in | 1 + .../0_stateless/02003_compress_bz2.reference | 1 + .../queries/0_stateless/02003_compress_bz2.sh | 13 ++ 18 files changed, 397 insertions(+), 4 deletions(-) create mode 100644 cmake/find/bzip2.cmake create mode 100644 contrib/bzip2-cmake/CMakeLists.txt create mode 100644 src/IO/Bzip2ReadBuffer.cpp create mode 100644 src/IO/Bzip2ReadBuffer.h create mode 100644 src/IO/Bzip2WriteBuffer.cpp create mode 100644 src/IO/Bzip2WriteBuffer.h create mode 100644 tests/queries/0_stateless/02003_compress_bz2.reference create mode 100755 tests/queries/0_stateless/02003_compress_bz2.sh diff --git a/.gitmodules b/.gitmodules index 43c878427ec..6bcb2f18c61 100644 --- a/.gitmodules +++ b/.gitmodules @@ -243,3 +243,6 @@ [submodule "contrib/s2geometry"] path = contrib/s2geometry url = https://github.com/ClickHouse-Extras/s2geometry.git +[submodule "contrib/bzip2"] + path = contrib/bzip2 + url = https://gitlab.com/federicomenaquintero/bzip2.git diff --git a/CMakeLists.txt b/CMakeLists.txt index 24022c256ec..d3cb5f70c83 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -543,6 +543,7 @@ include (cmake/find/nuraft.cmake) include (cmake/find/yaml-cpp.cmake) include (cmake/find/s2geometry.cmake) include (cmake/find/nlp.cmake) +include (cmake/find/bzip2.cmake) if(NOT USE_INTERNAL_PARQUET_LIBRARY) set (ENABLE_ORC OFF CACHE INTERNAL "") diff --git a/cmake/find/bzip2.cmake b/cmake/find/bzip2.cmake new file mode 100644 index 00000000000..15532a67c00 --- /dev/null +++ b/cmake/find/bzip2.cmake @@ -0,0 +1,19 @@ +option(ENABLE_BZIP2 "Enable bzip2 compression support" ${ENABLE_LIBRARIES}) + +if (NOT ENABLE_BZIP2) + message (STATUS "bzip2 compression disabled") + return() +endif() + +if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/bzip2/bzlib.h") + message (WARNING "submodule contrib/bzip2 is missing. to fix try run: \n git submodule update --init --recursive") + message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find internal bzip2 library") + set (USE_NLP 0) + return() +endif () + +set (USE_BZIP2 1) +set (BZIP2_INCLUDE_DIR "${ClickHouse_SOURCE_DIR}/contrib/bzip2") +set (BZIP2_LIBRARY bzip2) + +message (STATUS "Using bzip2=${USE_BZIP2}: ${BZIP2_INCLUDE_DIR} : ${BZIP2_LIBRARY}") diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index 82cddb0ace0..e6e098a05b3 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -334,6 +334,10 @@ if (USE_NLP) add_subdirectory(lemmagen-c-cmake) endif() +if (USE_BZIP2) + add_subdirectory(bzip2-cmake) +endif() + if (USE_SQLITE) add_subdirectory(sqlite-cmake) endif() diff --git a/contrib/bzip2-cmake/CMakeLists.txt b/contrib/bzip2-cmake/CMakeLists.txt new file mode 100644 index 00000000000..a9d2efa43c1 --- /dev/null +++ b/contrib/bzip2-cmake/CMakeLists.txt @@ -0,0 +1,23 @@ +set(BZIP2_SOURCE_DIR "${ClickHouse_SOURCE_DIR}/contrib/bzip2") +set(BZIP2_BINARY_DIR "${ClickHouse_BINARY_DIR}/contrib/bzip2") + +set(SRCS + "${BZIP2_SOURCE_DIR}/blocksort.c" + "${BZIP2_SOURCE_DIR}/huffman.c" + "${BZIP2_SOURCE_DIR}/crctable.c" + "${BZIP2_SOURCE_DIR}/randtable.c" + "${BZIP2_SOURCE_DIR}/compress.c" + "${BZIP2_SOURCE_DIR}/decompress.c" + "${BZIP2_SOURCE_DIR}/bzlib.c" +) + +# From bzip2/CMakeLists.txt +set(BZ_VERSION "1.0.7") +configure_file ( + "${BZIP2_SOURCE_DIR}/bz_version.h.in" + "${BZIP2_BINARY_DIR}/bz_version.h" +) + +add_library(bzip2 ${SRCS}) + +target_include_directories(bzip2 PUBLIC "${BZIP2_SOURCE_DIR}" "${BZIP2_BINARY_DIR}") diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index a99201e4aaa..2853a2e1a81 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -479,6 +479,11 @@ if (USE_NLP) dbms_target_link_libraries (PUBLIC lemmagen) endif() +if (USE_BZIP2) + target_link_libraries (clickhouse_common_io PRIVATE ${BZIP2_LIBRARY}) + target_include_directories (clickhouse_common_io SYSTEM BEFORE PRIVATE ${BZIP2_INCLUDE_DIR}) +endif() + include ("${ClickHouse_SOURCE_DIR}/cmake/add_check.cmake") if (ENABLE_TESTS AND USE_GTEST) diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index 7904d0ac61d..2bb378f13b5 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -567,6 +567,8 @@ M(1000, POCO_EXCEPTION) \ M(1001, STD_EXCEPTION) \ M(1002, UNKNOWN_EXCEPTION) \ + M(1003, BZIP2_STREAM_DECODER_FAILED) \ + M(1004, BZIP2_STREAM_ENCODER_FAILED) \ /* See END */ diff --git a/src/Common/config.h.in b/src/Common/config.h.in index 0665b1717ed..bf118e2507f 100644 --- a/src/Common/config.h.in +++ b/src/Common/config.h.in @@ -19,3 +19,4 @@ #cmakedefine01 USE_DATASKETCHES #cmakedefine01 USE_YAML_CPP #cmakedefine01 CLICKHOUSE_SPLIT_BINARY +#cmakedefine01 USE_BZIP2 diff --git a/src/IO/Bzip2ReadBuffer.cpp b/src/IO/Bzip2ReadBuffer.cpp new file mode 100644 index 00000000000..e264ce75444 --- /dev/null +++ b/src/IO/Bzip2ReadBuffer.cpp @@ -0,0 +1,97 @@ +#if !defined(ARCADIA_BUILD) +# include +#endif + +#if USE_BZIP2 +# include +# include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int BZIP2_STREAM_DECODER_FAILED; +} + + +class Bzip2ReadBuffer::Bzip2StateWrapper +{ +public: + Bzip2StateWrapper() + { + memset(&stream, 0, sizeof(stream)); + + int ret = BZ2_bzDecompressInit(&stream, 0, 0); + + if (ret != BZ_OK) + throw Exception( + ErrorCodes::BZIP2_STREAM_DECODER_FAILED, + "bzip2 stream encoder init failed: error code: {}", + ret); + } + + ~Bzip2StateWrapper() + { + BZ2_bzDecompressEnd(&stream); + } + + bz_stream stream; +}; + +Bzip2ReadBuffer::Bzip2ReadBuffer(std::unique_ptr in_, size_t buf_size, char *existing_memory, size_t alignment) + : BufferWithOwnMemory(buf_size, existing_memory, alignment) + , in(std::move(in_)) + , bz(std::make_unique()) + , eof(false) +{ +} + +Bzip2ReadBuffer::~Bzip2ReadBuffer() = default; + +bool Bzip2ReadBuffer::nextImpl() +{ + if (eof) + return false; + + if (!bz->stream.avail_in) + { + in->nextIfAtEnd(); + bz->stream.avail_in = in->buffer().end() - in->position(); + bz->stream.next_in = in->position(); + } + + bz->stream.avail_out = internal_buffer.size(); + bz->stream.next_out = internal_buffer.begin(); + + int ret = BZ2_bzDecompress(&bz->stream); + + in->position() = in->buffer().end() - bz->stream.avail_in; + working_buffer.resize(internal_buffer.size() - bz->stream.avail_out); + + if (ret == BZ_STREAM_END) + { + if (in->eof()) + { + eof = true; + return !working_buffer.empty(); + } + else + { + throw Exception( + ErrorCodes::BZIP2_STREAM_DECODER_FAILED, + "bzip2 decoder finished, but input stream has not exceeded: error code: {}", ret); + } + } + + if (ret != BZ_OK) + throw Exception( + ErrorCodes::BZIP2_STREAM_DECODER_FAILED, + "bzip2 stream decoder failed: error code: {}", + ret); + + return true; +} +} + +#endif diff --git a/src/IO/Bzip2ReadBuffer.h b/src/IO/Bzip2ReadBuffer.h new file mode 100644 index 00000000000..dc113800683 --- /dev/null +++ b/src/IO/Bzip2ReadBuffer.h @@ -0,0 +1,33 @@ +#pragma once + +#include +#include + + +namespace DB +{ + +class Bzip2ReadBuffer : public BufferWithOwnMemory +{ +public: + Bzip2ReadBuffer( + std::unique_ptr in_, + size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, + char * existing_memory = nullptr, + size_t alignment = 0); + + ~Bzip2ReadBuffer() override; + +private: + bool nextImpl() override; + + std::unique_ptr in; + + class Bzip2StateWrapper; + std::unique_ptr bz; + + bool eof; +}; + +} + diff --git a/src/IO/Bzip2WriteBuffer.cpp b/src/IO/Bzip2WriteBuffer.cpp new file mode 100644 index 00000000000..7bf20a964cd --- /dev/null +++ b/src/IO/Bzip2WriteBuffer.cpp @@ -0,0 +1,138 @@ +#if !defined(ARCADIA_BUILD) +# include +#endif + +#if USE_BROTLI +# include +# include + +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int BZIP2_STREAM_ENCODER_FAILED; +} + + +class Bzip2WriteBuffer::Bzip2StateWrapper +{ +public: + Bzip2StateWrapper(int compression_level) + { + memset(&stream, 0, sizeof(stream)); + + int ret = BZ2_bzCompressInit(&stream, compression_level, 0, 0); + + if (ret != BZ_OK) + throw Exception( + ErrorCodes::BZIP2_STREAM_ENCODER_FAILED, + "bzip2 stream encoder init failed: error code: {}", + ret); + } + + ~Bzip2StateWrapper() + { + BZ2_bzCompressEnd(&stream); + } + + bz_stream stream; +}; + +Bzip2WriteBuffer::Bzip2WriteBuffer(std::unique_ptr out_, int compression_level, size_t buf_size, char * existing_memory, size_t alignment) + : BufferWithOwnMemory(buf_size, existing_memory, alignment) + , bz(std::make_unique(compression_level)) + , out(std::move(out_)) +{ +} + +Bzip2WriteBuffer::~Bzip2WriteBuffer() +{ + /// FIXME move final flush into the caller + MemoryTracker::LockExceptionInThread lock(VariableContext::Global); + finish(); +} + +void Bzip2WriteBuffer::nextImpl() +{ + if (!offset()) + { + return; + } + + bz->stream.next_in = working_buffer.begin(); + bz->stream.avail_in = offset(); + + try + { + do + { + out->nextIfAtEnd(); + bz->stream.next_out = out->position(); + bz->stream.avail_out = out->buffer().end() - out->position(); + + int ret = BZ2_bzCompress(&bz->stream, BZ_RUN); + + out->position() = out->buffer().end() - bz->stream.avail_out; + + if (ret != BZ_RUN_OK) + throw Exception( + ErrorCodes::BZIP2_STREAM_ENCODER_FAILED, + "bzip2 stream encoder failed: error code: {}", + ret); + + } + while (bz->stream.avail_in > 0); + } + catch (...) + { + /// Do not try to write next time after exception. + out->position() = out->buffer().begin(); + throw; + } +} + +void Bzip2WriteBuffer::finish() +{ + if (finished) + return; + + try + { + finishImpl(); + out->finalize(); + finished = true; + } + catch (...) + { + /// Do not try to flush next time after exception. + out->position() = out->buffer().begin(); + finished = true; + throw; + } +} + +void Bzip2WriteBuffer::finishImpl() +{ + next(); + + out->nextIfAtEnd(); + bz->stream.next_out = out->position(); + bz->stream.avail_out = out->buffer().end() - out->position(); + + int ret = BZ2_bzCompress(&bz->stream, BZ_FINISH); + + out->position() = out->buffer().end() - bz->stream.avail_out; + + if (ret != BZ_STREAM_END && ret != BZ_FINISH_OK) + throw Exception( + ErrorCodes::BZIP2_STREAM_ENCODER_FAILED, + "bzip2 stream encoder failed: error code: {}", + ret); +} + +} + +#endif diff --git a/src/IO/Bzip2WriteBuffer.h b/src/IO/Bzip2WriteBuffer.h new file mode 100644 index 00000000000..72f14a344e3 --- /dev/null +++ b/src/IO/Bzip2WriteBuffer.h @@ -0,0 +1,37 @@ +#pragma once + +#include +#include + +namespace DB +{ + +class Bzip2WriteBuffer : public BufferWithOwnMemory +{ +public: + Bzip2WriteBuffer( + std::unique_ptr out_, + int compression_level, + size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, + char * existing_memory = nullptr, + size_t alignment = 0); + + ~Bzip2WriteBuffer() override; + + void finalize() override { finish(); } + +private: + void nextImpl() override; + + void finish(); + void finishImpl(); + + class Bzip2StateWrapper; + std::unique_ptr bz; + + std::unique_ptr out; + + bool finished = false; +}; + +} diff --git a/src/IO/CompressionMethod.cpp b/src/IO/CompressionMethod.cpp index c6bb5232c0c..e02475c1d83 100644 --- a/src/IO/CompressionMethod.cpp +++ b/src/IO/CompressionMethod.cpp @@ -10,6 +10,8 @@ #include #include #include +#include +#include #if !defined(ARCADIA_BUILD) # include @@ -40,6 +42,8 @@ std::string toContentEncodingName(CompressionMethod method) return "xz"; case CompressionMethod::Zstd: return "zstd"; + case CompressionMethod::Bzip2: + return "bz2"; case CompressionMethod::None: return ""; } @@ -69,11 +73,13 @@ CompressionMethod chooseCompressionMethod(const std::string & path, const std::s return CompressionMethod::Xz; if (method_str == "zstd" || method_str == "zst") return CompressionMethod::Zstd; + if (method_str == "bz2") + return CompressionMethod::Bzip2; if (hint.empty() || hint == "auto" || hint == "none") return CompressionMethod::None; throw Exception( - "Unknown compression method " + hint + ". Only 'auto', 'none', 'gzip', 'deflate', 'br', 'xz', 'zstd' are supported as compression methods", + "Unknown compression method " + hint + ". Only 'auto', 'none', 'gzip', 'deflate', 'br', 'xz', 'zstd', 'bz2' are supported as compression methods", ErrorCodes::NOT_IMPLEMENTED); } @@ -91,7 +97,10 @@ std::unique_ptr wrapReadBufferWithCompressionMethod( return std::make_unique(std::move(nested), buf_size, existing_memory, alignment); if (method == CompressionMethod::Zstd) return std::make_unique(std::move(nested), buf_size, existing_memory, alignment); - +#if USE_BZIP2 + if (method == CompressionMethod::Bzip2) + return std::make_unique(std::move(nested), buf_size, existing_memory, alignment); +#endif if (method == CompressionMethod::None) return nested; @@ -114,7 +123,10 @@ std::unique_ptr wrapWriteBufferWithCompressionMethod( if (method == CompressionMethod::Zstd) return std::make_unique(std::move(nested), level, buf_size, existing_memory, alignment); - +#if USE_BZIP2 + if (method == CompressionMethod::Bzip2) + return std::make_unique(std::move(nested), level, buf_size, existing_memory, alignment); +#endif if (method == CompressionMethod::None) return nested; diff --git a/src/IO/CompressionMethod.h b/src/IO/CompressionMethod.h index 6f2d87b45cf..cf034229f77 100644 --- a/src/IO/CompressionMethod.h +++ b/src/IO/CompressionMethod.h @@ -31,7 +31,8 @@ enum class CompressionMethod /// Zstd compressor /// This option corresponds to HTTP Content-Encoding: zstd Zstd, - Brotli + Brotli, + Bzip2 }; /// How the compression method is named in HTTP. diff --git a/src/IO/ya.make b/src/IO/ya.make index 9e35a062a96..7723464be6f 100644 --- a/src/IO/ya.make +++ b/src/IO/ya.make @@ -23,6 +23,8 @@ SRCS( AIOContextPool.cpp BrotliReadBuffer.cpp BrotliWriteBuffer.cpp + Bzip2ReadBuffer.cpp + Bzip2WriteBuffer.cpp CascadeWriteBuffer.cpp CompressionMethod.cpp DoubleConverter.cpp diff --git a/src/Storages/System/StorageSystemBuildOptions.generated.cpp.in b/src/Storages/System/StorageSystemBuildOptions.generated.cpp.in index 8fe574da643..6bb97355151 100644 --- a/src/Storages/System/StorageSystemBuildOptions.generated.cpp.in +++ b/src/Storages/System/StorageSystemBuildOptions.generated.cpp.in @@ -50,6 +50,7 @@ const char * auto_config_build[] "USE_LDAP", "@USE_LDAP@", "TZDATA_VERSION", "@TZDATA_VERSION@", "USE_KRB5", "@USE_KRB5@", + "USE_BZIP2", "@USE_BZIP2@", nullptr, nullptr }; diff --git a/tests/queries/0_stateless/02003_compress_bz2.reference b/tests/queries/0_stateless/02003_compress_bz2.reference new file mode 100644 index 00000000000..8ab686eafeb --- /dev/null +++ b/tests/queries/0_stateless/02003_compress_bz2.reference @@ -0,0 +1 @@ +Hello, World! diff --git a/tests/queries/0_stateless/02003_compress_bz2.sh b/tests/queries/0_stateless/02003_compress_bz2.sh new file mode 100755 index 00000000000..e3b670d8da7 --- /dev/null +++ b/tests/queries/0_stateless/02003_compress_bz2.sh @@ -0,0 +1,13 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +USER_FILES_PATH=$(clickhouse-client --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') + +${CLICKHOUSE_CLIENT} --query "SELECT * FROM (SELECT 'Hello, World!' as String) INTO OUTFILE '${USER_FILES_PATH}/bz2_compression.bz2'" +bzip2 -t ${USER_FILES_PATH}/bz2_compression.bz2 +${CLICKHOUSE_CLIENT} --query "SELECT * FROM file('${USER_FILES_PATH}/bz2_compression.bz2', 'TabSeparated', 'col String')" + +rm -f "${USER_FILES_PATH}/bz2_compression.bz2" From aa33a7add163c6668949a1f1056ec529975ae8ef Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Sat, 7 Aug 2021 13:07:41 +0800 Subject: [PATCH 235/599] Add settings to check hyperscan regexp length. --- src/Core/Settings.h | 2 ++ .../FunctionsMultiStringFuzzySearch.h | 16 +++++++++- src/Functions/FunctionsMultiStringSearch.h | 16 +++++++++- src/Functions/hyperscanRegexpChecker.cpp | 29 +++++++++++++++++++ src/Functions/hyperscanRegexpChecker.h | 10 +++++++ ...02004_max_hyperscan_regex_length.reference | 6 ++++ .../02004_max_hyperscan_regex_length.sql | 26 +++++++++++++++++ 7 files changed, 103 insertions(+), 2 deletions(-) create mode 100644 src/Functions/hyperscanRegexpChecker.cpp create mode 100644 src/Functions/hyperscanRegexpChecker.h create mode 100644 tests/queries/0_stateless/02004_max_hyperscan_regex_length.reference create mode 100644 tests/queries/0_stateless/02004_max_hyperscan_regex_length.sql diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 20404089210..d3493677af5 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -377,6 +377,8 @@ class IColumn; M(Bool, 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', 'postgresql' and 'odbc' table functions.", 0) \ \ M(Bool, allow_hyperscan, true, "Allow functions that use Hyperscan library. Disable to avoid potentially long compilation times and excessive resource usage.", 0) \ + M(UInt64, max_hyperscan_regexp_length, 0, "Max length of regexp than can be used in hyperscan multi-match functions. Zero means unlimited.", 0) \ + M(UInt64, max_hyperscan_regexp_total_length, 0, "Max total length of all regexps than can be used in hyperscan multi-match functions. Zero means unlimited.", 0) \ M(Bool, allow_simdjson, true, "Allow using simdjson library in 'JSON*' functions if AVX2 instructions are available. If disabled rapidjson will be used.", 0) \ M(Bool, allow_introspection_functions, false, "Allow functions for introspection of ELF and DWARF for query profiling. These functions are slow and may impose security considerations.", 0) \ \ diff --git a/src/Functions/FunctionsMultiStringFuzzySearch.h b/src/Functions/FunctionsMultiStringFuzzySearch.h index 209efb0fc2f..a2d0c972abb 100644 --- a/src/Functions/FunctionsMultiStringFuzzySearch.h +++ b/src/Functions/FunctionsMultiStringFuzzySearch.h @@ -10,6 +10,7 @@ #include #include #include +#include #include #include #include @@ -40,7 +41,13 @@ public: throw Exception( "Hyperscan functions are disabled, because setting 'allow_hyperscan' is set to 0", ErrorCodes::FUNCTION_NOT_ALLOWED); - return std::make_shared(); + return std::make_shared( + context->getSettingsRef().max_hyperscan_regexp_length, context->getSettingsRef().max_hyperscan_regexp_total_length); + } + + FunctionsMultiStringFuzzySearch(size_t max_hyperscan_regexp_length_, size_t max_hyperscan_regexp_total_length_) + : max_hyperscan_regexp_length(max_hyperscan_regexp_length_), max_hyperscan_regexp_total_length(max_hyperscan_regexp_total_length_) + { } String getName() const override { return name; } @@ -113,6 +120,9 @@ public: for (const auto & el : src_arr) refs.emplace_back(el.get()); + if (Impl::is_using_hyperscan) + checkRegexp(refs, max_hyperscan_regexp_length, max_hyperscan_regexp_total_length); + auto col_res = ColumnVector::create(); auto col_offsets = ColumnArray::ColumnOffsets::create(); @@ -131,6 +141,10 @@ public: else return col_res; } + +private: + size_t max_hyperscan_regexp_length; + size_t max_hyperscan_regexp_total_length; }; } diff --git a/src/Functions/FunctionsMultiStringSearch.h b/src/Functions/FunctionsMultiStringSearch.h index 08b4668940e..3dd2e8bfd09 100644 --- a/src/Functions/FunctionsMultiStringSearch.h +++ b/src/Functions/FunctionsMultiStringSearch.h @@ -10,6 +10,7 @@ #include #include #include +#include #include #include #include @@ -53,7 +54,13 @@ public: throw Exception( "Hyperscan functions are disabled, because setting 'allow_hyperscan' is set to 0", ErrorCodes::FUNCTION_NOT_ALLOWED); - return std::make_shared(); + return std::make_shared( + context->getSettingsRef().max_hyperscan_regexp_length, context->getSettingsRef().max_hyperscan_regexp_total_length); + } + + FunctionsMultiStringSearch(size_t max_hyperscan_regexp_length_, size_t max_hyperscan_regexp_total_length_) + : max_hyperscan_regexp_length(max_hyperscan_regexp_length_), max_hyperscan_regexp_total_length(max_hyperscan_regexp_total_length_) + { } String getName() const override { return name; } @@ -105,6 +112,9 @@ public: for (const auto & el : src_arr) refs.emplace_back(el.get()); + if (Impl::is_using_hyperscan) + checkRegexp(refs, max_hyperscan_regexp_length, max_hyperscan_regexp_total_length); + auto col_res = ColumnVector::create(); auto col_offsets = ColumnArray::ColumnOffsets::create(); @@ -122,6 +132,10 @@ public: else return col_res; } + +private: + size_t max_hyperscan_regexp_length; + size_t max_hyperscan_regexp_total_length; }; } diff --git a/src/Functions/hyperscanRegexpChecker.cpp b/src/Functions/hyperscanRegexpChecker.cpp new file mode 100644 index 00000000000..b3c46e34daa --- /dev/null +++ b/src/Functions/hyperscanRegexpChecker.cpp @@ -0,0 +1,29 @@ +#include + +#include + +namespace DB +{ +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; +} + +void checkRegexp(const std::vector & refs, size_t max_hyperscan_regexp_length, size_t max_hyperscan_regexp_total_length) +{ + if (max_hyperscan_regexp_length > 0 || max_hyperscan_regexp_total_length > 0) + { + size_t total_regexp_length = 0; + for (const auto & pattern : refs) + { + if (max_hyperscan_regexp_length > 0 && pattern.size > max_hyperscan_regexp_length) + throw Exception("Regexp length too large", ErrorCodes::BAD_ARGUMENTS); + total_regexp_length += pattern.size; + } + + if (max_hyperscan_regexp_total_length > 0 && total_regexp_length > max_hyperscan_regexp_total_length) + throw Exception("Total regexp lengths too large", ErrorCodes::BAD_ARGUMENTS); + } +} + +} diff --git a/src/Functions/hyperscanRegexpChecker.h b/src/Functions/hyperscanRegexpChecker.h new file mode 100644 index 00000000000..f2988120899 --- /dev/null +++ b/src/Functions/hyperscanRegexpChecker.h @@ -0,0 +1,10 @@ +#pragma once + +#include + +namespace DB +{ + +void checkRegexp(const std::vector & refs, size_t max_hyperscan_regexp_length, size_t max_hyperscan_regexp_total_length); + +} diff --git a/tests/queries/0_stateless/02004_max_hyperscan_regex_length.reference b/tests/queries/0_stateless/02004_max_hyperscan_regex_length.reference new file mode 100644 index 00000000000..87e68c152c3 --- /dev/null +++ b/tests/queries/0_stateless/02004_max_hyperscan_regex_length.reference @@ -0,0 +1,6 @@ +1 +1 +[1] +1 +1 +[1] diff --git a/tests/queries/0_stateless/02004_max_hyperscan_regex_length.sql b/tests/queries/0_stateless/02004_max_hyperscan_regex_length.sql new file mode 100644 index 00000000000..6058ef2f430 --- /dev/null +++ b/tests/queries/0_stateless/02004_max_hyperscan_regex_length.sql @@ -0,0 +1,26 @@ +set max_hyperscan_regexp_length = 1; +set max_hyperscan_regexp_total_length = 1; + +select multiMatchAny('123', ['1']); +select multiMatchAny('123', ['12']); -- { serverError 36 } +select multiMatchAny('123', ['1', '2']); -- { serverError 36 } + +select multiMatchAnyIndex('123', ['1']); +select multiMatchAnyIndex('123', ['12']); -- { serverError 36 } +select multiMatchAnyIndex('123', ['1', '2']); -- { serverError 36 } + +select multiMatchAllIndices('123', ['1']); +select multiMatchAllIndices('123', ['12']); -- { serverError 36 } +select multiMatchAllIndices('123', ['1', '2']); -- { serverError 36 } + +select multiFuzzyMatchAny('123', 0, ['1']); +select multiFuzzyMatchAny('123', 0, ['12']); -- { serverError 36 } +select multiFuzzyMatchAny('123', 0, ['1', '2']); -- { serverError 36 } + +select multiFuzzyMatchAnyIndex('123', 0, ['1']); +select multiFuzzyMatchAnyIndex('123', 0, ['12']); -- { serverError 36 } +select multiFuzzyMatchAnyIndex('123', 0, ['1', '2']); -- { serverError 36 } + +select multiFuzzyMatchAllIndices('123', 0, ['1']); +select multiFuzzyMatchAllIndices('123', 0, ['12']); -- { serverError 36 } +select multiFuzzyMatchAllIndices('123', 0, ['1', '2']); -- { serverError 36 } From c6a6bc8b1a3db25e891bf70e3b8e342465d3deca Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Sat, 7 Aug 2021 06:18:14 +0000 Subject: [PATCH 236/599] Add submodule bzip2 --- contrib/bzip2 | 1 + 1 file changed, 1 insertion(+) create mode 160000 contrib/bzip2 diff --git a/contrib/bzip2 b/contrib/bzip2 new file mode 160000 index 00000000000..bf905ea2251 --- /dev/null +++ b/contrib/bzip2 @@ -0,0 +1 @@ +Subproject commit bf905ea2251191ff9911ae7ec0cfc35d41f9f7f6 From 00301cf36af223cd989682dcf00bd99f1c4facd7 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Sat, 7 Aug 2021 06:20:59 +0000 Subject: [PATCH 237/599] Ignore test in fasttest --- docker/test/fasttest/run.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/docker/test/fasttest/run.sh b/docker/test/fasttest/run.sh index 6419ea3659c..a0dcf07c41d 100755 --- a/docker/test/fasttest/run.sh +++ b/docker/test/fasttest/run.sh @@ -312,6 +312,7 @@ function run_tests 01798_uniq_theta_sketch 01799_long_uniq_theta_sketch 01890_stem # depends on libstemmer_c + 02003_compress_bz2 # depends on bzip2 collate collation _orc_ From 5ef59d48287b1642e9cea139731fa647d7a731f8 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Sat, 7 Aug 2021 14:26:08 +0800 Subject: [PATCH 238/599] Add setting to log formatted query into system.query_log --- src/Core/Settings.h | 1 + src/Interpreters/QueryLog.cpp | 2 ++ src/Interpreters/QueryLog.h | 1 + src/Interpreters/executeQuery.cpp | 6 ++++++ .../0_stateless/02005_log_formatted_queries.reference | 3 +++ tests/queries/0_stateless/02005_log_formatted_queries.sql | 5 +++++ 6 files changed, 18 insertions(+) create mode 100644 tests/queries/0_stateless/02005_log_formatted_queries.reference create mode 100644 tests/queries/0_stateless/02005_log_formatted_queries.sql diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 20404089210..2ee7ad283b4 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -169,6 +169,7 @@ class IColumn; M(Int64, os_thread_priority, 0, "If non zero - set corresponding 'nice' value for query processing threads. Can be used to adjust query priority for OS scheduler.", 0) \ \ M(Bool, log_queries, 1, "Log requests and write the log to the system table.", 0) \ + M(Bool, log_formatted_queries, 0, "Log formatted queries and write the log to the system table.", 0) \ M(LogQueriesType, log_queries_min_type, QueryLogElementType::QUERY_START, "Minimal type in query_log to log, possible values (from low to high): QUERY_START, QUERY_FINISH, EXCEPTION_BEFORE_START, EXCEPTION_WHILE_PROCESSING.", 0) \ M(Milliseconds, log_queries_min_query_duration_ms, 0, "Minimal time for the query to run, to get to the query_log/query_thread_log.", 0) \ M(UInt64, log_queries_cut_to_length, 100000, "If query length is greater than specified threshold (in bytes), then cut query when writing to query log. Also limit length of printed query in ordinary text log.", 0) \ diff --git a/src/Interpreters/QueryLog.cpp b/src/Interpreters/QueryLog.cpp index 3f668e5e0ab..0f7ff579f5d 100644 --- a/src/Interpreters/QueryLog.cpp +++ b/src/Interpreters/QueryLog.cpp @@ -57,6 +57,7 @@ NamesAndTypesList QueryLogElement::getNamesAndTypes() {"current_database", std::make_shared()}, {"query", std::make_shared()}, + {"formatted_query", std::make_shared()}, {"normalized_query_hash", std::make_shared()}, {"query_kind", std::make_shared(std::make_shared())}, {"databases", std::make_shared( @@ -151,6 +152,7 @@ void QueryLogElement::appendToBlock(MutableColumns & columns) const columns[i++]->insertData(current_database.data(), current_database.size()); columns[i++]->insertData(query.data(), query.size()); + columns[i++]->insertData(formatted_query.data(), formatted_query.size()); columns[i++]->insert(normalized_query_hash); columns[i++]->insertData(query_kind.data(), query_kind.size()); diff --git a/src/Interpreters/QueryLog.h b/src/Interpreters/QueryLog.h index 0aa02104306..aad3e56190b 100644 --- a/src/Interpreters/QueryLog.h +++ b/src/Interpreters/QueryLog.h @@ -51,6 +51,7 @@ struct QueryLogElement String current_database; String query; + String formatted_query; UInt64 normalized_query_hash{}; String query_kind; diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 3756f1b2765..1b59f3bc7df 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -265,7 +265,11 @@ static void onExceptionBeforeStart(const String & query_for_logging, ContextPtr // Try log query_kind if ast is valid if (ast) + { elem.query_kind = ast->getQueryKindString(); + if (settings.log_formatted_queries) + elem.formatted_query = queryToString(ast); + } // We don't calculate databases, tables and columns when the query isn't able to start @@ -641,6 +645,8 @@ static std::tuple executeQueryImpl( elem.current_database = context->getCurrentDatabase(); elem.query = query_for_logging; + if (settings.log_formatted_queries) + elem.formatted_query = queryToString(ast); elem.normalized_query_hash = normalizedQueryHash(query_for_logging); elem.client_info = client_info; diff --git a/tests/queries/0_stateless/02005_log_formatted_queries.reference b/tests/queries/0_stateless/02005_log_formatted_queries.reference new file mode 100644 index 00000000000..3ddd8b0d64f --- /dev/null +++ b/tests/queries/0_stateless/02005_log_formatted_queries.reference @@ -0,0 +1,3 @@ +02005_log_formatted_queries.sql +select \'02005_log_formatted_queries.sql\' from system.one; SELECT \'02005_log_formatted_queries.sql\' FROM system.one +select \'02005_log_formatted_queries.sql\' from system.one; SELECT \'02005_log_formatted_queries.sql\' FROM system.one diff --git a/tests/queries/0_stateless/02005_log_formatted_queries.sql b/tests/queries/0_stateless/02005_log_formatted_queries.sql new file mode 100644 index 00000000000..62f839af0f0 --- /dev/null +++ b/tests/queries/0_stateless/02005_log_formatted_queries.sql @@ -0,0 +1,5 @@ +set log_formatted_queries = 1; + +select '02005_log_formatted_queries.sql' from system.one; +system flush logs; +select query, formatted_query from system.query_log where current_database = currentDatabase() and query = 'select \'02005_log_formatted_queries.sql\' from system.one;' and event_date >= yesterday() and event_time > now() - interval 5 minute; From c20bf2fbe48a2b7cba4709c9fc587d7de8791756 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Sat, 7 Aug 2021 10:36:12 +0300 Subject: [PATCH 239/599] Update docs/ru/sql-reference/functions/string-functions.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/sql-reference/functions/string-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/string-functions.md b/docs/ru/sql-reference/functions/string-functions.md index 097ff593ef7..39498fcea9f 100644 --- a/docs/ru/sql-reference/functions/string-functions.md +++ b/docs/ru/sql-reference/functions/string-functions.md @@ -15,7 +15,7 @@ toc_title: "Функции для работы со строками" empty(x) ``` -Строка считается непустой, если содержит хотя бы один байт, пусть даже это пробел или нулевой байт. UUID считается пустой, если он содержит только нули (нулевой UUID). +Строка считается непустой, если содержит хотя бы один байт, пусть даже это пробел или нулевой байт. UUID считается пустым, если он содержит только нули (нулевой UUID). **Параметры** From 3a69ad9d2bfa32b02e7a848682cf3403f9d89919 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Sat, 7 Aug 2021 10:36:19 +0300 Subject: [PATCH 240/599] Update docs/en/sql-reference/functions/string-functions.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/en/sql-reference/functions/string-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index 0156b4331ba..d54bb16abbc 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -46,7 +46,7 @@ Result: └───────────┘ ``` -## notEmpty {#notempty} +## notempty {#notempty} Checks whether the input string is not empty. From c9124d979d439a9be000260f150d7bf9382b4ce9 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Sat, 7 Aug 2021 10:36:28 +0300 Subject: [PATCH 241/599] Update docs/ru/engines/database-engines/materialized-mysql.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/engines/database-engines/materialized-mysql.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/engines/database-engines/materialized-mysql.md b/docs/ru/engines/database-engines/materialized-mysql.md index 2f57b9f5a84..9ad89628e0b 100644 --- a/docs/ru/engines/database-engines/materialized-mysql.md +++ b/docs/ru/engines/database-engines/materialized-mysql.md @@ -93,7 +93,7 @@ CREATE DATABASE mysql ENGINE = MaterializedMySQL('localhost:3306', 'db', 'user', Кроме ограничений на типы данных, существует несколько ограничений по сравнению с базами данных MySQL, которые следует решить до того, как станет возможной репликация: -- Каждая таблица в `MySQL` должна содержать `PRIMARY KEY`. +- Каждая таблица в MySQL должна содержать `PRIMARY KEY`. - Репликация для таблиц, содержащих строки со значениями полей `ENUM` вне диапазона значений (определяется размерностью `ENUM`), не будет работать. ### DDL-запросы {#ddl-queries} From 6c71fcdeb7b5532c4010aa6e9de5c15484fe93d3 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Sat, 7 Aug 2021 10:36:38 +0300 Subject: [PATCH 242/599] Update docs/en/engines/database-engines/materialized-mysql.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/en/engines/database-engines/materialized-mysql.md | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/docs/en/engines/database-engines/materialized-mysql.md b/docs/en/engines/database-engines/materialized-mysql.md index 163c86df84f..8a6975e4f2b 100644 --- a/docs/en/engines/database-engines/materialized-mysql.md +++ b/docs/en/engines/database-engines/materialized-mysql.md @@ -29,11 +29,11 @@ ENGINE = MaterializedMySQL('host:port', ['database' | database], 'user', 'passwo **Engine Settings** -- `max_rows_in_buffer` — Max rows that data is allowed to cache in memory (for single table and the cache data unable to query). When rows is exceeded, the data will be materialized. Default: `65 505`. -- `max_bytes_in_buffer` — Max bytes that data is allowed to cache in memory (for single table and the cache data unable to query). when rows is exceeded, the data will be materialized. Default: `1 048 576`. -- `max_rows_in_buffers` — Max rows that data is allowed to cache in memory (for database and the cache data unable to query). when rows is exceeded, the data will be materialized. Default: `65505`. -- `max_bytes_in_buffers` — Max bytes that data is allowed to cache in memory (for database and the cache data unable to query). when rows is exceeded, the data will be materialized. Default: `1 048 576`. -- `max_flush_data_time` — Max milliseconds that data is allowed to cache in memory (for database and the cache data unable to query). when this time is exceeded, the data will be materialized. Default: `1000`. +- `max_rows_in_buffer` — Maximum number of rows that data is allowed to cache in memory (for single table and the cache data unable to query). When this number is exceeded, the data will be materialized. Default: `65 505`. +- `max_bytes_in_buffer` — Maximum number of bytes that data is allowed to cache in memory (for single table and the cache data unable to query). When this number is exceeded, the data will be materialized. Default: `1 048 576`. +- `max_rows_in_buffers` — Maximum number of rows that data is allowed to cache in memory (for database and the cache data unable to query). When this number is exceeded, the data will be materialized. Default: `65 505`. +- `max_bytes_in_buffers` — Maximum number of bytes that data is allowed to cache in memory (for database and the cache data unable to query). When this number is exceeded, the data will be materialized. Default: `1 048 576`. +- `max_flush_data_time` — Maximum number of milliseconds that data is allowed to cache in memory (for database and the cache data unable to query). When this time is exceeded, the data will be materialized. Default: `1000`. - `max_wait_time_when_mysql_unavailable` — Retry interval when MySQL is not available (milliseconds). Negative value disable retry. Default: `1000`. - `allows_query_when_mysql_lost` — Allow query materialized table when MySql is lost. Default: `0` (`false`). From c6fcf90bf0f444d4d081f501771493bbe62e6fa3 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Sat, 7 Aug 2021 10:36:48 +0300 Subject: [PATCH 243/599] Update docs/en/engines/database-engines/materialized-mysql.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/en/engines/database-engines/materialized-mysql.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/engines/database-engines/materialized-mysql.md b/docs/en/engines/database-engines/materialized-mysql.md index 8a6975e4f2b..815fa152fee 100644 --- a/docs/en/engines/database-engines/materialized-mysql.md +++ b/docs/en/engines/database-engines/materialized-mysql.md @@ -35,7 +35,7 @@ ENGINE = MaterializedMySQL('host:port', ['database' | database], 'user', 'passwo - `max_bytes_in_buffers` — Maximum number of bytes that data is allowed to cache in memory (for database and the cache data unable to query). When this number is exceeded, the data will be materialized. Default: `1 048 576`. - `max_flush_data_time` — Maximum number of milliseconds that data is allowed to cache in memory (for database and the cache data unable to query). When this time is exceeded, the data will be materialized. Default: `1000`. - `max_wait_time_when_mysql_unavailable` — Retry interval when MySQL is not available (milliseconds). Negative value disable retry. Default: `1000`. -- `allows_query_when_mysql_lost` — Allow query materialized table when MySql is lost. Default: `0` (`false`). +- `allows_query_when_mysql_lost` — Allows to query a materialized table when MySQL is lost. Default: `0` (`false`). ```sql CREATE DATABASE mysql ENGINE = MaterializedMySQL('localhost:3306', 'db', 'user', '***') From 2a98353208e0e88f1ba52e448c0467f635d8aefd Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Sat, 7 Aug 2021 10:36:52 +0300 Subject: [PATCH 244/599] Update docs/en/engines/database-engines/materialized-mysql.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/en/engines/database-engines/materialized-mysql.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/engines/database-engines/materialized-mysql.md b/docs/en/engines/database-engines/materialized-mysql.md index 815fa152fee..0e6528fe414 100644 --- a/docs/en/engines/database-engines/materialized-mysql.md +++ b/docs/en/engines/database-engines/materialized-mysql.md @@ -44,7 +44,7 @@ CREATE DATABASE mysql ENGINE = MaterializedMySQL('localhost:3306', 'db', 'user', max_wait_time_when_mysql_unavailable=10000; ``` -**Settings on Mysql-server Side** +**Settings on MySQL-server Side** For the correct work of `MaterializeMySQL`, there are few mandatory `MySQL`-side configuration settings that must be set: From 8201b5605f4899d439ca5c4a8dcd4abc41af2bdb Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Sat, 7 Aug 2021 10:37:14 +0300 Subject: [PATCH 245/599] Update docs/ru/engines/database-engines/materialized-mysql.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- .../engines/database-engines/materialized-mysql.md | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/docs/ru/engines/database-engines/materialized-mysql.md b/docs/ru/engines/database-engines/materialized-mysql.md index 9ad89628e0b..13f33a2e952 100644 --- a/docs/ru/engines/database-engines/materialized-mysql.md +++ b/docs/ru/engines/database-engines/materialized-mysql.md @@ -29,12 +29,12 @@ ENGINE = MaterializedMySQL('host:port', ['database' | database], 'user', 'passwo **Настройки движка** -- `max_rows_in_buffer` — максимальное количество строк, содержимое которых может кэшироваться в памяти (для одной таблицы и данных кэша, которые невозможно запросить). При превышении количества строк, данные будут материализованы. Значение по умолчанию: `65 505`. -- `max_bytes_in_buffer` — максимальное количество байтов, которое разрешено кэшировать в памяти (для одной таблицы и данных кэша, которые невозможно запросить). При превышении количества строк, данные будут материализованы. Значение по умолчанию: `1 048 576`. -- `max_rows_in_buffers` — максимальное количество строк, содержимое которых может кэшироваться в памяти (для базы данных и данных кэша, которые невозможно запросить). При превышении количества строк, данные будут материализованы. Значение по умолчанию: `65 505`. -- `max_bytes_in_buffers` — максимальное количество байтов, которое разрешено кэшировать данным в памяти (для базы данных и данных кэша, которые невозможно запросить). При превышении количества строк, данные будут материализованы. Значение по умолчанию: `1 048 576`. -- `max_flush_data_time` — максимальное время в миллисекундах, в течение которого разрешено кэшировать данные в памяти (для базы данных и данных кэша, которые невозможно запросить). При превышении количества указанного периода, данные будут материализованы. Значение по умолчанию: `1000`. -- `max_wait_time_when_mysql_unavailable` — интервал повторных попыток, если MySQL недоступен. Указывается в миллисекундах. Отрицательное значение отключает повторные попытки. Значение по умолчанию: `1000`. +- `max_rows_in_buffer` — максимальное количество строк, содержимое которых может кешироваться в памяти (для одной таблицы и данных кеша, которые невозможно запросить). При превышении количества строк, данные будут материализованы. Значение по умолчанию: `65 505`. +- `max_bytes_in_buffer` — максимальное количество байтов, которое разрешено кешировать в памяти (для одной таблицы и данных кеша, которые невозможно запросить). При превышении количества строк, данные будут материализованы. Значение по умолчанию: `1 048 576`. +- `max_rows_in_buffers` — максимальное количество строк, содержимое которых может кешироваться в памяти (для базы данных и данных кеша, которые невозможно запросить). При превышении количества строк, данные будут материализованы. Значение по умолчанию: `65 505`. +- `max_bytes_in_buffers` — максимальное количество байтов, которое разрешено кешировать данным в памяти (для базы данных и данных кеша, которые невозможно запросить). При превышении количества строк, данные будут материализованы. Значение по умолчанию: `1 048 576`. +- `max_flush_data_time` — максимальное время в миллисекундах, в течение которого разрешено кешировать данные в памяти (для базы данных и данных кеша, которые невозможно запросить). При превышении количества указанного периода, данные будут материализованы. Значение по умолчанию: `1000`. +- `max_wait_time_when_mysql_unavailable` — интервал между повторными попытками, если MySQL недоступен. Указывается в миллисекундах. Отрицательное значение отключает повторные попытки. Значение по умолчанию: `1000`. - `allows_query_when_mysql_lost` — разрешить запрос материализованной таблицы при потере MySQL. Значение по умолчанию: `0` (`false`). ```sql From 094b191d708a6884bc9b36765d8ccc5c68e0a667 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Sat, 7 Aug 2021 10:37:22 +0300 Subject: [PATCH 246/599] Update docs/ru/engines/database-engines/materialized-mysql.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/engines/database-engines/materialized-mysql.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/engines/database-engines/materialized-mysql.md b/docs/ru/engines/database-engines/materialized-mysql.md index 13f33a2e952..2f2f79281ee 100644 --- a/docs/ru/engines/database-engines/materialized-mysql.md +++ b/docs/ru/engines/database-engines/materialized-mysql.md @@ -46,7 +46,7 @@ CREATE DATABASE mysql ENGINE = MaterializedMySQL('localhost:3306', 'db', 'user', **Настройки на стороне MySQL-сервера** -Для правильной работы `MaterializeMySQL` следует обязательно указать на MySQL сервере следующие параметры конфигурации: +Для правильной работы `MaterializeMySQL` следует обязательно указать на сервере MySQL следующие параметры конфигурации: - `default_authentication_plugin = mysql_native_password` — `MaterializeMySQL` может авторизоваться только с помощью этого метода. - `gtid_mode = on` — ведение журнала на основе GTID является обязательным для обеспечения правильной репликации. From a63d7bd36f53e1cb9055cc5c9518159596ce2407 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Sat, 7 Aug 2021 10:38:34 +0300 Subject: [PATCH 247/599] Update docs/en/engines/database-engines/materialized-mysql.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/en/engines/database-engines/materialized-mysql.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/engines/database-engines/materialized-mysql.md b/docs/en/engines/database-engines/materialized-mysql.md index 0e6528fe414..a80bfe5f6bc 100644 --- a/docs/en/engines/database-engines/materialized-mysql.md +++ b/docs/en/engines/database-engines/materialized-mysql.md @@ -34,7 +34,7 @@ ENGINE = MaterializedMySQL('host:port', ['database' | database], 'user', 'passwo - `max_rows_in_buffers` — Maximum number of rows that data is allowed to cache in memory (for database and the cache data unable to query). When this number is exceeded, the data will be materialized. Default: `65 505`. - `max_bytes_in_buffers` — Maximum number of bytes that data is allowed to cache in memory (for database and the cache data unable to query). When this number is exceeded, the data will be materialized. Default: `1 048 576`. - `max_flush_data_time` — Maximum number of milliseconds that data is allowed to cache in memory (for database and the cache data unable to query). When this time is exceeded, the data will be materialized. Default: `1000`. -- `max_wait_time_when_mysql_unavailable` — Retry interval when MySQL is not available (milliseconds). Negative value disable retry. Default: `1000`. +- `max_wait_time_when_mysql_unavailable` — Retry interval when MySQL is not available (milliseconds). Negative value disables retry. Default: `1000`. - `allows_query_when_mysql_lost` — Allows to query a materialized table when MySQL is lost. Default: `0` (`false`). ```sql From 0febbd4a1def319cef1f4da457eed59c1c2679f6 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Sat, 7 Aug 2021 08:42:54 +0000 Subject: [PATCH 248/599] Fix build --- src/IO/Bzip2WriteBuffer.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/IO/Bzip2WriteBuffer.cpp b/src/IO/Bzip2WriteBuffer.cpp index 7bf20a964cd..41cb972966c 100644 --- a/src/IO/Bzip2WriteBuffer.cpp +++ b/src/IO/Bzip2WriteBuffer.cpp @@ -20,7 +20,7 @@ namespace ErrorCodes class Bzip2WriteBuffer::Bzip2StateWrapper { public: - Bzip2StateWrapper(int compression_level) + explicit Bzip2StateWrapper(int compression_level) { memset(&stream, 0, sizeof(stream)); From 9ca422f0c5380124b57270ad18505e572d1b3afb Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 7 Aug 2021 08:11:40 +0000 Subject: [PATCH 249/599] Introduce CAST for internal usage --- src/Functions/FunctionsConversion.cpp | 3 + src/Functions/FunctionsConversion.h | 147 +++++++++++------- src/Interpreters/ActionsDAG.cpp | 6 +- .../ConvertStringsToEnumVisitor.cpp | 9 +- src/Interpreters/MutationsInterpreter.cpp | 4 +- ...OptimizeIfWithConstantConditionVisitor.cpp | 3 +- src/Interpreters/addTypeConversionToAST.cpp | 2 +- src/Interpreters/castColumn.cpp | 2 +- src/Interpreters/inplaceBlockConversions.cpp | 4 +- src/Parsers/ASTFunctionHelpers.h | 16 ++ .../Impl/ConstantExpressionTemplate.cpp | 2 +- src/Storages/MergeTree/KeyCondition.cpp | 2 +- .../configs/users.xml | 6 + .../test.py | 36 +++++ 14 files changed, 169 insertions(+), 73 deletions(-) create mode 100644 src/Parsers/ASTFunctionHelpers.h create mode 100644 tests/integration/test_alter_update_cast_keep_nullable/configs/users.xml create mode 100644 tests/integration/test_alter_update_cast_keep_nullable/test.py diff --git a/src/Functions/FunctionsConversion.cpp b/src/Functions/FunctionsConversion.cpp index d7686318efc..cdbd32b189c 100644 --- a/src/Functions/FunctionsConversion.cpp +++ b/src/Functions/FunctionsConversion.cpp @@ -43,6 +43,9 @@ void registerFunctionsConversion(FunctionFactory & factory) factory.registerFunction(); + factory.registerFunction>(FunctionFactory::CaseInsensitive); + factory.registerFunction>(); + factory.registerFunction>(); factory.registerFunction>(FunctionFactory::CaseInsensitive); factory.registerFunction>(); factory.registerFunction>(); diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index 67a02e3fd34..b085e29a786 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -2412,7 +2412,8 @@ private: std::optional diagnostic; }; -struct NameCast { static constexpr auto name = "CAST"; }; +struct CastName { static constexpr auto name = "CAST"; }; +struct CastInternalName { static constexpr auto name = "_CAST"; }; enum class CastType { @@ -2421,17 +2422,26 @@ enum class CastType accurateOrNull }; -class FunctionCast final : public IFunctionBase +class FunctionCastBase : public IFunctionBase +{ +public: + using MonotonicityForRange = std::function; + using Diagnostic = ExecutableFunctionCast::Diagnostic; +}; + +template +class FunctionCast final : public FunctionCastBase { public: using WrapperType = std::function; - using MonotonicityForRange = std::function; - using Diagnostic = ExecutableFunctionCast::Diagnostic; - FunctionCast(const char * name_, MonotonicityForRange && monotonicity_for_range_ - , const DataTypes & argument_types_, const DataTypePtr & return_type_ - , std::optional diagnostic_, CastType cast_type_) - : name(name_), monotonicity_for_range(std::move(monotonicity_for_range_)) + FunctionCast(const char * cast_name_ + , MonotonicityForRange && monotonicity_for_range_ + , const DataTypes & argument_types_ + , const DataTypePtr & return_type_ + , std::optional diagnostic_ + , CastType cast_type_) + : cast_name(cast_name_), monotonicity_for_range(std::move(monotonicity_for_range_)) , argument_types(argument_types_), return_type(return_type_), diagnostic(std::move(diagnostic_)) , cast_type(cast_type_) { @@ -2445,7 +2455,7 @@ public: try { return std::make_unique( - prepareUnpackDictionaries(getArgumentTypes()[0], getResultType()), name, diagnostic); + prepareUnpackDictionaries(getArgumentTypes()[0], getResultType()), cast_name, diagnostic); } catch (Exception & e) { @@ -2456,7 +2466,7 @@ public: } } - String getName() const override { return name; } + String getName() const override { return cast_name; } bool isDeterministic() const override { return true; } bool isDeterministicInScopeOfQuery() const override { return true; } @@ -2473,7 +2483,7 @@ public: private: - const char * name; + const char * cast_name; MonotonicityForRange monotonicity_for_range; DataTypes argument_types; @@ -2515,7 +2525,7 @@ private: { /// In case when converting to Nullable type, we apply different parsing rule, /// that will not throw an exception but return NULL in case of malformed input. - FunctionPtr function = FunctionConvertFromString::create(); + FunctionPtr function = FunctionConvertFromString::create(); return createFunctionAdaptor(function, from_type); } else if (!can_apply_accurate_cast) @@ -2539,12 +2549,12 @@ private: { if (wrapper_cast_type == CastType::accurate) { - result_column = ConvertImpl::execute( + result_column = ConvertImpl::execute( arguments, result_type, input_rows_count, AccurateConvertStrategyAdditions()); } else { - result_column = ConvertImpl::execute( + result_column = ConvertImpl::execute( arguments, result_type, input_rows_count, AccurateOrNullConvertStrategyAdditions()); } @@ -2559,7 +2569,7 @@ private: { if (wrapper_cast_type == CastType::accurateOrNull) { - auto nullable_column_wrapper = FunctionCast::createToNullableColumnWrapper(); + auto nullable_column_wrapper = FunctionCast::createToNullableColumnWrapper(); return nullable_column_wrapper(arguments, result_type, column_nullable, input_rows_count); } else @@ -2631,7 +2641,7 @@ private: { AccurateConvertStrategyAdditions additions; additions.scale = scale; - result_column = ConvertImpl::execute( + result_column = ConvertImpl::execute( arguments, result_type, input_rows_count, additions); return true; @@ -2640,7 +2650,7 @@ private: { AccurateOrNullConvertStrategyAdditions additions; additions.scale = scale; - result_column = ConvertImpl::execute( + result_column = ConvertImpl::execute( arguments, result_type, input_rows_count, additions); return true; @@ -2653,14 +2663,14 @@ private: /// Consistent with CAST(Nullable(String) AS Nullable(Numbers)) /// In case when converting to Nullable type, we apply different parsing rule, /// that will not throw an exception but return NULL in case of malformed input. - result_column = ConvertImpl::execute( + result_column = ConvertImpl::execute( arguments, result_type, input_rows_count, scale); return true; } } - result_column = ConvertImpl::execute(arguments, result_type, input_rows_count, scale); + result_column = ConvertImpl::execute(arguments, result_type, input_rows_count, scale); return true; }); @@ -2670,7 +2680,7 @@ private: { if (wrapper_cast_type == CastType::accurateOrNull) { - auto nullable_column_wrapper = FunctionCast::createToNullableColumnWrapper(); + auto nullable_column_wrapper = FunctionCast::createToNullableColumnWrapper(); return nullable_column_wrapper(arguments, result_type, column_nullable, input_rows_count); } else @@ -2990,7 +3000,7 @@ private: template WrapperType createStringToEnumWrapper() const { - const char * function_name = name; + const char * function_name = cast_name; return [function_name] ( ColumnsWithTypeAndName & arguments, const DataTypePtr & res_type, const ColumnNullable * nullable_col, size_t /*input_rows_count*/) { @@ -3324,7 +3334,7 @@ private: class MonotonicityHelper { public: - using MonotonicityForRange = FunctionCast::MonotonicityForRange; + using MonotonicityForRange = FunctionCastBase::MonotonicityForRange; template static auto monotonicityForType(const DataType * const) @@ -3382,35 +3392,24 @@ public: } }; -template -class CastOverloadResolver : public IFunctionOverloadResolver + +/* + * CastInternal does not preserve nullability of the data type, + * i.e. CastInternal(toNullable(toInt8(1)) as Int32) will be Int32(1). + * + * Cast preserves nullability according to setting `cast_keep_nullable`, + * i.e. Cast(toNullable(toInt8(1)) as Int32) will be Nullable(Int32(1)) if `cast_keep_nullable` == 1. +**/ +template +class CastOverloadResolverImpl : public IFunctionOverloadResolver { public: - using MonotonicityForRange = FunctionCast::MonotonicityForRange; - using Diagnostic = FunctionCast::Diagnostic; - - static constexpr auto accurate_cast_name = "accurateCast"; - static constexpr auto accurate_cast_or_null_name = "accurateCastOrNull"; - static constexpr auto cast_name = "CAST"; + using MonotonicityForRange = FunctionCastBase::MonotonicityForRange; + using Diagnostic = FunctionCastBase::Diagnostic; static constexpr auto name = cast_type == CastType::accurate - ? accurate_cast_name - : (cast_type == CastType::accurateOrNull ? accurate_cast_or_null_name : cast_name); - - static FunctionOverloadResolverPtr create(ContextPtr context) - { - return createImpl(context->getSettingsRef().cast_keep_nullable); - } - - static FunctionOverloadResolverPtr createImpl(bool keep_nullable, std::optional diagnostic = {}) - { - return std::make_unique(keep_nullable, std::move(diagnostic)); - } - - - explicit CastOverloadResolver(bool keep_nullable_, std::optional diagnostic_ = {}) - : keep_nullable(keep_nullable_), diagnostic(std::move(diagnostic_)) - {} + ? CastName::accurate_cast_name + : (cast_type == CastType::accurateOrNull ? CastName::accurate_cast_or_null_name : CastName::cast_name); String getName() const override { return name; } @@ -3418,6 +3417,24 @@ public: ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; } + explicit CastOverloadResolverImpl(std::optional diagnostic_, bool keep_nullable_) + : diagnostic(std::move(diagnostic_)), keep_nullable(keep_nullable_) + { + } + + static FunctionOverloadResolverPtr create(ContextPtr context) + { + if constexpr (internal) + return createImpl(); + return createImpl({}, context->getSettingsRef().cast_keep_nullable); + } + + static FunctionOverloadResolverPtr createImpl(std::optional diagnostic = {}, bool keep_nullable = false) + { + assert(!internal || !keep_nullable); + return std::make_unique(std::move(diagnostic), keep_nullable); + } + protected: FunctionBasePtr buildImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & return_type) const override @@ -3428,7 +3445,7 @@ protected: data_types[i] = arguments[i].type; auto monotonicity = MonotonicityHelper::getMonotonicityInformation(arguments.front().type, return_type.get()); - return std::make_unique(name, std::move(monotonicity), data_types, return_type, diagnostic, cast_type); + return std::make_unique>(name, std::move(monotonicity), data_types, return_type, diagnostic, cast_type); } DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override @@ -3448,23 +3465,41 @@ protected: DataTypePtr type = DataTypeFactory::instance().get(type_col->getValue()); if constexpr (cast_type == CastType::accurateOrNull) - { return makeNullable(type); - } - else - { - if (keep_nullable && arguments.front().type->isNullable()) - return makeNullable(type); + + if constexpr (internal) return type; - } + + if (keep_nullable && arguments.front().type->isNullable()) + return makeNullable(type); + + return type; } bool useDefaultImplementationForNulls() const override { return false; } bool useDefaultImplementationForLowCardinalityColumns() const override { return false; } private: - bool keep_nullable; std::optional diagnostic; + bool keep_nullable; }; + +struct CastOverloadName +{ + static constexpr auto cast_name = "CAST"; + static constexpr auto accurate_cast_name = "CastAccurate"; + static constexpr auto accurate_cast_or_null_name = "CastAccurateOrNull"; +}; + +struct CastInternalOverloadName +{ + static constexpr auto cast_name = "_CAST"; + static constexpr auto accurate_cast_name = "_CastAccurate"; + static constexpr auto accurate_cast_or_null_name = "_CastAccurateOrNull"; +}; + +template using CastOverloadResolver = CastOverloadResolverImpl; +template using CastInternalOverloadResolver = CastOverloadResolverImpl; + } diff --git a/src/Interpreters/ActionsDAG.cpp b/src/Interpreters/ActionsDAG.cpp index 63b0345b372..a42d6053e9a 100644 --- a/src/Interpreters/ActionsDAG.cpp +++ b/src/Interpreters/ActionsDAG.cpp @@ -1110,8 +1110,8 @@ ActionsDAGPtr ActionsDAG::makeConvertingActions( const auto * right_arg = &actions_dag->addColumn(std::move(column)); const auto * left_arg = dst_node; - FunctionCast::Diagnostic diagnostic = {dst_node->result_name, res_elem.name}; - FunctionOverloadResolverPtr func_builder_cast = CastOverloadResolver::createImpl(false, std::move(diagnostic)); + FunctionCastBase::Diagnostic diagnostic = {dst_node->result_name, res_elem.name}; + FunctionOverloadResolverPtr func_builder_cast = CastInternalOverloadResolver::createImpl(std::move(diagnostic)); NodeRawConstPtrs children = { left_arg, right_arg }; dst_node = &actions_dag->addFunction(func_builder_cast, std::move(children), {}); @@ -1876,7 +1876,7 @@ ActionsDAGPtr ActionsDAG::cloneActionsForFilterPushDown( predicate->children = {left_arg, right_arg}; auto arguments = prepareFunctionArguments(predicate->children); - FunctionOverloadResolverPtr func_builder_cast = CastOverloadResolver::createImpl(false); + FunctionOverloadResolverPtr func_builder_cast = CastInternalOverloadResolver::createImpl(); predicate->function_builder = func_builder_cast; predicate->function_base = predicate->function_builder->build(arguments); diff --git a/src/Interpreters/ConvertStringsToEnumVisitor.cpp b/src/Interpreters/ConvertStringsToEnumVisitor.cpp index fa2e0b6613a..e483bc9b5b6 100644 --- a/src/Interpreters/ConvertStringsToEnumVisitor.cpp +++ b/src/Interpreters/ConvertStringsToEnumVisitor.cpp @@ -43,11 +43,11 @@ void changeIfArguments(ASTPtr & first, ASTPtr & second) String enum_string = makeStringsEnum(values); auto enum_literal = std::make_shared(enum_string); - auto first_cast = makeASTFunction("CAST"); + auto first_cast = makeASTFunction("_CAST"); first_cast->arguments->children.push_back(first); first_cast->arguments->children.push_back(enum_literal); - auto second_cast = makeASTFunction("CAST"); + auto second_cast = makeASTFunction("_CAST"); second_cast->arguments->children.push_back(second); second_cast->arguments->children.push_back(enum_literal); @@ -65,12 +65,12 @@ void changeTransformArguments(ASTPtr & array_to, ASTPtr & other) String enum_string = makeStringsEnum(values); - auto array_cast = makeASTFunction("CAST"); + auto array_cast = makeASTFunction("_CAST"); array_cast->arguments->children.push_back(array_to); array_cast->arguments->children.push_back(std::make_shared("Array(" + enum_string + ")")); array_to = array_cast; - auto other_cast = makeASTFunction("CAST"); + auto other_cast = makeASTFunction("_CAST"); other_cast->arguments->children.push_back(other); other_cast->arguments->children.push_back(std::make_shared(enum_string)); other = other_cast; @@ -183,4 +183,3 @@ void ConvertStringsToEnumMatcher::visit(ASTFunction & function_node, Data & data } } - diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index fe0594bb58f..a5c625ef4f8 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -503,10 +503,10 @@ ASTPtr MutationsInterpreter::prepare(bool dry_run) } } - auto updated_column = makeASTFunction("CAST", + auto updated_column = makeASTFunction("_CAST", makeASTFunction("if", condition, - makeASTFunction("CAST", + makeASTFunction("_CAST", update_expr->clone(), type_literal), std::make_shared(column)), diff --git a/src/Interpreters/OptimizeIfWithConstantConditionVisitor.cpp b/src/Interpreters/OptimizeIfWithConstantConditionVisitor.cpp index a8e2d371e05..a9814ce50f5 100644 --- a/src/Interpreters/OptimizeIfWithConstantConditionVisitor.cpp +++ b/src/Interpreters/OptimizeIfWithConstantConditionVisitor.cpp @@ -1,6 +1,7 @@ #include #include #include +#include #include #include #include @@ -29,7 +30,7 @@ static bool tryExtractConstValueFromCondition(const ASTPtr & condition, bool & v /// cast of numeric constant in condition to UInt8 if (const auto * function = condition->as()) { - if (function->name == "CAST") + if (isFunctionCast(function)) { if (const auto * expr_list = function->arguments->as()) { diff --git a/src/Interpreters/addTypeConversionToAST.cpp b/src/Interpreters/addTypeConversionToAST.cpp index ba67ec762a9..2f766880253 100644 --- a/src/Interpreters/addTypeConversionToAST.cpp +++ b/src/Interpreters/addTypeConversionToAST.cpp @@ -20,7 +20,7 @@ namespace ErrorCodes ASTPtr addTypeConversionToAST(ASTPtr && ast, const String & type_name) { - auto func = makeASTFunction("CAST", ast, std::make_shared(type_name)); + auto func = makeASTFunction("_CAST", ast, std::make_shared(type_name)); if (ASTWithAlias * ast_with_alias = dynamic_cast(ast.get())) { diff --git a/src/Interpreters/castColumn.cpp b/src/Interpreters/castColumn.cpp index 181cca1e017..3356d37ba7f 100644 --- a/src/Interpreters/castColumn.cpp +++ b/src/Interpreters/castColumn.cpp @@ -21,7 +21,7 @@ static ColumnPtr castColumn(const ColumnWithTypeAndName & arg, const DataTypePtr } }; - FunctionOverloadResolverPtr func_builder_cast = CastOverloadResolver::createImpl(false); + FunctionOverloadResolverPtr func_builder_cast = CastInternalOverloadResolver::createImpl(); auto func_cast = func_builder_cast->build(arguments); diff --git a/src/Interpreters/inplaceBlockConversions.cpp b/src/Interpreters/inplaceBlockConversions.cpp index 26cf6912bc7..e40e0635a85 100644 --- a/src/Interpreters/inplaceBlockConversions.cpp +++ b/src/Interpreters/inplaceBlockConversions.cpp @@ -52,7 +52,7 @@ void addDefaultRequiredExpressionsRecursively( RequiredSourceColumnsVisitor(columns_context).visit(column_default_expr); NameSet required_columns_names = columns_context.requiredColumns(); - auto expr = makeASTFunction("CAST", column_default_expr, std::make_shared(columns.get(required_column_name).type->getName())); + auto expr = makeASTFunction("_CAST", column_default_expr, std::make_shared(columns.get(required_column_name).type->getName())); if (is_column_in_query && convert_null_to_default) expr = makeASTFunction("ifNull", std::make_shared(required_column_name), std::move(expr)); @@ -101,7 +101,7 @@ ASTPtr convertRequiredExpressions(Block & block, const NamesAndTypesList & requi continue; auto cast_func = makeASTFunction( - "CAST", std::make_shared(required_column.name), std::make_shared(required_column.type->getName())); + "_CAST", std::make_shared(required_column.name), std::make_shared(required_column.type->getName())); conversion_expr_list->children.emplace_back(setAlias(cast_func, required_column.name)); diff --git a/src/Parsers/ASTFunctionHelpers.h b/src/Parsers/ASTFunctionHelpers.h new file mode 100644 index 00000000000..76da2dd1501 --- /dev/null +++ b/src/Parsers/ASTFunctionHelpers.h @@ -0,0 +1,16 @@ +#pragma once + +#include + + +namespace DB +{ + +static bool isFunctionCast(const ASTFunction * function) +{ + if (function) + return function->name == "CAST" || function->name == "_CAST"; + return false; +} + +} diff --git a/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp b/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp index cc9ae5e65bb..1f780a206dd 100644 --- a/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp +++ b/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp @@ -639,7 +639,7 @@ void ConstantExpressionTemplate::TemplateStructure::addNodesToCastResult(const I expr = makeASTFunction("assumeNotNull", std::move(expr)); } - expr = makeASTFunction("CAST", std::move(expr), std::make_shared(result_column_type.getName())); + expr = makeASTFunction("_CAST", std::move(expr), std::make_shared(result_column_type.getName())); if (null_as_default) { diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index 235cadfba11..e6dfdb859b8 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -1367,7 +1367,7 @@ bool KeyCondition::tryParseAtomFromAST(const ASTPtr & node, ContextPtr context, { ColumnsWithTypeAndName arguments{ {nullptr, key_expr_type, ""}, {DataTypeString().createColumnConst(1, common_type->getName()), common_type, ""}}; - FunctionOverloadResolverPtr func_builder_cast = CastOverloadResolver::createImpl(false); + FunctionOverloadResolverPtr func_builder_cast = CastInternalOverloadResolver::createImpl(); auto func_cast = func_builder_cast->build(arguments); /// If we know the given range only contains one value, then we treat all functions as positive monotonic. diff --git a/tests/integration/test_alter_update_cast_keep_nullable/configs/users.xml b/tests/integration/test_alter_update_cast_keep_nullable/configs/users.xml new file mode 100644 index 00000000000..16b032daacb --- /dev/null +++ b/tests/integration/test_alter_update_cast_keep_nullable/configs/users.xml @@ -0,0 +1,6 @@ + + + + 1 + + diff --git a/tests/integration/test_alter_update_cast_keep_nullable/test.py b/tests/integration/test_alter_update_cast_keep_nullable/test.py new file mode 100644 index 00000000000..497a9e21d94 --- /dev/null +++ b/tests/integration/test_alter_update_cast_keep_nullable/test.py @@ -0,0 +1,36 @@ +import pytest +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__) + +node1 = cluster.add_instance('node1', user_configs=['configs/users.xml'], with_zookeeper=True) + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + yield cluster + finally: + cluster.shutdown() + +def test_cast_keep_nullable(started_cluster): + setting = node1.query("SELECT value FROM system.settings WHERE name='cast_keep_nullable'") + assert(setting.strip() == "1") + + result = node1.query(""" + DROP TABLE IF EXISTS t; + CREATE TABLE t (x UInt64) ENGINE = MergeTree ORDER BY tuple(); + INSERT INTO t SELECT number FROM numbers(10); + SELECT * FROM t; + """) + assert(result.strip() == "0\n1\n2\n3\n4\n5\n6\n7\n8\n9") + + error = node1.query_and_get_error(""" + SET mutations_sync = 1; + ALTER TABLE t UPDATE x = x % 3 = 0 ? NULL : x WHERE x % 2 = 1;  + """) + assert("DB::Exception: Cannot convert NULL value to non-Nullable type" in error) + + result = node1.query("SELECT * FROM t;") + assert(result.strip() == "0\n1\n2\n3\n4\n5\n6\n7\n8\n9") + From 365e743831cefc1ffece62d8a7f3379fec94be94 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 7 Aug 2021 09:03:23 +0000 Subject: [PATCH 250/599] Temporarily set cast_keep_nullable = 1 --- src/Core/Settings.h | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 20404089210..5b75ca0d3ab 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -432,7 +432,8 @@ class IColumn; M(String, function_implementation, "", "Choose function implementation for specific target or variant (experimental). If empty enable all of them.", 0) \ M(Bool, allow_experimental_geo_types, false, "Allow geo data types such as Point, Ring, Polygon, MultiPolygon", 0) \ M(Bool, data_type_default_nullable, false, "Data types without NULL or NOT NULL will make Nullable", 0) \ - M(Bool, cast_keep_nullable, false, "CAST operator keep Nullable for result data type", 0) \ + /** Temporarily set to true, to check how tests will feel.*/ \ + M(Bool, cast_keep_nullable, true, "CAST operator keep Nullable for result data type", 0) \ M(Bool, alter_partition_verbose_result, false, "Output information about affected parts. Currently works only for FREEZE and ATTACH commands.", 0) \ M(Bool, allow_experimental_database_materialized_mysql, false, "Allow to create database with Engine=MaterializedMySQL(...).", 0) \ M(Bool, allow_experimental_database_materialized_postgresql, false, "Allow to create database with Engine=MaterializedPostgreSQL(...).", 0) \ From bc7eaad5caffc2feb23feef7e04ad4cb6ce1b52a Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 7 Aug 2021 12:35:59 +0300 Subject: [PATCH 251/599] Fix --- tests/integration/test_library_bridge/test.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/integration/test_library_bridge/test.py b/tests/integration/test_library_bridge/test.py index 607afb6db5f..97b2ccfbdbe 100644 --- a/tests/integration/test_library_bridge/test.py +++ b/tests/integration/test_library_bridge/test.py @@ -100,6 +100,7 @@ def test_load_ids(ch_cluster): if instance.is_built_with_memory_sanitizer(): pytest.skip("Memory Sanitizer cannot work with third-party shared libraries") + instance.query('DROP DICTIONARY IF EXISTS lib_dict_c') instance.query(''' CREATE DICTIONARY lib_dict_c (key UInt64, value1 UInt64, value2 UInt64, value3 UInt64) PRIMARY KEY key SOURCE(library(PATH '/etc/clickhouse-server/config.d/dictionaries_lib/dict_lib.so')) @@ -263,6 +264,7 @@ def test_bridge_dies_with_parent(ch_cluster): assert clickhouse_pid is None assert bridge_pid is None instance.start_clickhouse(20) + instance.query('DROP DICTIONARY lib_dict_c') if __name__ == '__main__': From 8f5ecb5ed6fdacd11dacb97b436382992e28c2f1 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 7 Aug 2021 12:50:42 +0300 Subject: [PATCH 252/599] Mark more tests as long --- ...=> 01650_drop_part_and_deduplication_zookeeper_long.reference} | 0 ...r.sql => 01650_drop_part_and_deduplication_zookeeper_long.sql} | 0 2 files changed, 0 insertions(+), 0 deletions(-) rename tests/queries/0_stateless/{01650_drop_part_and_deduplication_zookeeper.reference => 01650_drop_part_and_deduplication_zookeeper_long.reference} (100%) rename tests/queries/0_stateless/{01650_drop_part_and_deduplication_zookeeper.sql => 01650_drop_part_and_deduplication_zookeeper_long.sql} (100%) diff --git a/tests/queries/0_stateless/01650_drop_part_and_deduplication_zookeeper.reference b/tests/queries/0_stateless/01650_drop_part_and_deduplication_zookeeper_long.reference similarity index 100% rename from tests/queries/0_stateless/01650_drop_part_and_deduplication_zookeeper.reference rename to tests/queries/0_stateless/01650_drop_part_and_deduplication_zookeeper_long.reference diff --git a/tests/queries/0_stateless/01650_drop_part_and_deduplication_zookeeper.sql b/tests/queries/0_stateless/01650_drop_part_and_deduplication_zookeeper_long.sql similarity index 100% rename from tests/queries/0_stateless/01650_drop_part_and_deduplication_zookeeper.sql rename to tests/queries/0_stateless/01650_drop_part_and_deduplication_zookeeper_long.sql From bf0297ccab4c3298213807a6d2ff596b8f07bef5 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Sat, 7 Aug 2021 09:51:35 +0000 Subject: [PATCH 253/599] Fix stateless test --- tests/queries/0_stateless/02003_compress_bz2.sh | 12 ++++++++---- 1 file changed, 8 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/02003_compress_bz2.sh b/tests/queries/0_stateless/02003_compress_bz2.sh index e3b670d8da7..790a520ef24 100755 --- a/tests/queries/0_stateless/02003_compress_bz2.sh +++ b/tests/queries/0_stateless/02003_compress_bz2.sh @@ -5,9 +5,13 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CURDIR"/../shell_config.sh USER_FILES_PATH=$(clickhouse-client --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') +WORKING_FOLDER_02003="${USER_FILES_PATH}/${CLICKHOUSE_DATABASE}" -${CLICKHOUSE_CLIENT} --query "SELECT * FROM (SELECT 'Hello, World!' as String) INTO OUTFILE '${USER_FILES_PATH}/bz2_compression.bz2'" -bzip2 -t ${USER_FILES_PATH}/bz2_compression.bz2 -${CLICKHOUSE_CLIENT} --query "SELECT * FROM file('${USER_FILES_PATH}/bz2_compression.bz2', 'TabSeparated', 'col String')" +rm -rf "${WORKING_FOLDER_02003}" +mkdir ${WORKING_FOLDER_02003} -rm -f "${USER_FILES_PATH}/bz2_compression.bz2" +${CLICKHOUSE_CLIENT} --query "SELECT * FROM (SELECT 'Hello, World!' as String) INTO OUTFILE '${WORKING_FOLDER_02003}/bz2_compression.bz2'" +bzip2 -t ${WORKING_FOLDER_02003}/bz2_compression.bz2 +${CLICKHOUSE_CLIENT} --query "SELECT * FROM file('${WORKING_FOLDER_02003}/bz2_compression.bz2', 'TabSeparated', 'col String')" + +rm -rf "${WORKING_FOLDER_02003}" From ec9e82fb0c648ea98950913337d2bdf83cadc3e9 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 7 Aug 2021 10:12:59 +0000 Subject: [PATCH 254/599] Fix some tests --- src/Functions/FunctionsConversion.h | 8 ++++---- .../0_stateless/01029_early_constant_folding.reference | 2 +- .../01324_if_transform_strings_to_enum.reference | 2 +- .../01455_nullable_type_with_if_agg_combinator.sql | 2 ++ .../01611_constant_folding_subqueries.reference | 2 +- 5 files changed, 9 insertions(+), 7 deletions(-) diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index b085e29a786..c277997593c 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -3488,15 +3488,15 @@ private: struct CastOverloadName { static constexpr auto cast_name = "CAST"; - static constexpr auto accurate_cast_name = "CastAccurate"; - static constexpr auto accurate_cast_or_null_name = "CastAccurateOrNull"; + static constexpr auto accurate_cast_name = "accurateCast"; + static constexpr auto accurate_cast_or_null_name = "accurateCastOrNull"; }; struct CastInternalOverloadName { static constexpr auto cast_name = "_CAST"; - static constexpr auto accurate_cast_name = "_CastAccurate"; - static constexpr auto accurate_cast_or_null_name = "_CastAccurateOrNull"; + static constexpr auto accurate_cast_name = "accurate_Cast"; + static constexpr auto accurate_cast_or_null_name = "accurate_CastOrNull"; }; template using CastOverloadResolver = CastOverloadResolverImpl; diff --git a/tests/queries/0_stateless/01029_early_constant_folding.reference b/tests/queries/0_stateless/01029_early_constant_folding.reference index 88139b7e2b8..abcb2ddc6a7 100644 --- a/tests/queries/0_stateless/01029_early_constant_folding.reference +++ b/tests/queries/0_stateless/01029_early_constant_folding.reference @@ -2,7 +2,7 @@ SELECT 1 WHERE 0 SELECT 1 SELECT 1 -WHERE (1 IN (0, 2)) AND (2 = (identity(CAST(2, \'Nullable(UInt8)\')) AS subquery)) +WHERE (1 IN (0, 2)) AND (2 = (identity(_CAST(2, \'Nullable(UInt8)\')) AS subquery)) SELECT 1 WHERE 1 IN (( SELECT arrayJoin([1, 2, 3]) diff --git a/tests/queries/0_stateless/01324_if_transform_strings_to_enum.reference b/tests/queries/0_stateless/01324_if_transform_strings_to_enum.reference index 99298d28bad..994e3f24aaf 100644 --- a/tests/queries/0_stateless/01324_if_transform_strings_to_enum.reference +++ b/tests/queries/0_stateless/01324_if_transform_strings_to_enum.reference @@ -8,7 +8,7 @@ yahoo other other other -SELECT transform(number, [2, 4, 6], CAST([\'google\', \'yandex\', \'yahoo\'], \'Array(Enum8(\\\'google\\\' = 1, \\\'other\\\' = 2, \\\'yahoo\\\' = 3, \\\'yandex\\\' = 4))\'), CAST(\'other\', \'Enum8(\\\'google\\\' = 1, \\\'other\\\' = 2, \\\'yahoo\\\' = 3, \\\'yandex\\\' = 4)\')) +SELECT transform(number, [2, 4, 6], _CAST([\'google\', \'yandex\', \'yahoo\'], \'Array(Enum8(\\\'google\\\' = 1, \\\'other\\\' = 2, \\\'yahoo\\\' = 3, \\\'yandex\\\' = 4))\'), _CAST(\'other\', \'Enum8(\\\'google\\\' = 1, \\\'other\\\' = 2, \\\'yahoo\\\' = 3, \\\'yandex\\\' = 4)\')) FROM system.numbers LIMIT 10 google diff --git a/tests/queries/0_stateless/01455_nullable_type_with_if_agg_combinator.sql b/tests/queries/0_stateless/01455_nullable_type_with_if_agg_combinator.sql index 852660117f5..0e951af73d0 100644 --- a/tests/queries/0_stateless/01455_nullable_type_with_if_agg_combinator.sql +++ b/tests/queries/0_stateless/01455_nullable_type_with_if_agg_combinator.sql @@ -1,3 +1,5 @@ +SET cast_keep_nullable = 0; + -- Value nullable SELECT anyIf(CAST(number, 'Nullable(UInt8)'), number = 3) AS a, toTypeName(a) FROM numbers(2); -- Value and condition nullable diff --git a/tests/queries/0_stateless/01611_constant_folding_subqueries.reference b/tests/queries/0_stateless/01611_constant_folding_subqueries.reference index 6128cd109e2..c3df2314112 100644 --- a/tests/queries/0_stateless/01611_constant_folding_subqueries.reference +++ b/tests/queries/0_stateless/01611_constant_folding_subqueries.reference @@ -5,7 +5,7 @@ SELECT (SELECT * FROM system.numbers LIMIT 1 OFFSET 1) AS n, toUInt64(10 / n) FO 1,10 EXPLAIN SYNTAX SELECT (SELECT * FROM system.numbers LIMIT 1 OFFSET 1) AS n, toUInt64(10 / n); SELECT - identity(CAST(0, \'Nullable(UInt64)\')) AS n, + identity(_CAST(0, \'Nullable(UInt64)\')) AS n, toUInt64(10 / n) SELECT * FROM (WITH (SELECT * FROM system.numbers LIMIT 1 OFFSET 1) AS n, toUInt64(10 / n) as q SELECT * FROM system.one WHERE q > 0); 0 From 3dcff2124cef5e9af8a0b13a494bdef8b2ad4d7e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E8=91=A3=E6=B5=B7=E9=95=94?= Date: Sat, 7 Aug 2021 17:42:09 +0800 Subject: [PATCH 255/599] rename bitmapSubsetOffsetLimit to subBitmap and modify the rb_offset_limit function logic --- .../functions/bitmap-functions.md | 6 +-- .../functions/bitmap-functions.md | 6 +-- .../AggregateFunctionGroupBitmapData.h | 46 +++++-------------- src/Functions/FunctionsBitmap.h | 2 +- .../0_stateless/00829_bitmap_function.sql | 18 ++++---- 5 files changed, 28 insertions(+), 50 deletions(-) diff --git a/docs/en/sql-reference/functions/bitmap-functions.md b/docs/en/sql-reference/functions/bitmap-functions.md index 2becaefbc53..f8d1fdc69fa 100644 --- a/docs/en/sql-reference/functions/bitmap-functions.md +++ b/docs/en/sql-reference/functions/bitmap-functions.md @@ -125,14 +125,14 @@ Result: └───────────────────────────┘ ``` -## bitmapSubsetOffsetLimit {#bitmapsubsetoffsetlimit} +## subBitmap {#subBitmap} Creates a subset of bitmap limit the results to `cardinality_limit` with offset of `offset`. **Syntax** ``` sql -bitmapSubsetOffsetLimit(bitmap, offset, cardinality_limit) +subBitmap(bitmap, offset, cardinality_limit) ``` **Arguments** @@ -152,7 +152,7 @@ Type: `Bitmap object`. Query: ``` sql -SELECT bitmapToArray(bitmapSubsetOffsetLimit(bitmapBuild([0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,100,200,500]), toUInt32(10), toUInt32(10))) AS res; +SELECT bitmapToArray(subBitmap(bitmapBuild([0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,100,200,500]), toUInt32(10), toUInt32(10))) AS res; ``` Result: diff --git a/docs/zh/sql-reference/functions/bitmap-functions.md b/docs/zh/sql-reference/functions/bitmap-functions.md index e9480d3e411..c4c1b7293f6 100644 --- a/docs/zh/sql-reference/functions/bitmap-functions.md +++ b/docs/zh/sql-reference/functions/bitmap-functions.md @@ -88,11 +88,11 @@ SELECT bitmapToArray(bitmapSubsetLimit(bitmapBuild([0,1,2,3,4,5,6,7,8,9,10,11,12 │ [30,31,32,33,100,200,500] │ └───────────────────────────┘ -## bitmapSubsetOffsetLimit {#bitmapsubsetoffsetlimit} +## subBitmap {#subBitmap} 将位图跳过`offset`个元素,限制大小为`limit`个的结果转换为另一个位图。 - bitmapSubsetOffsetLimit(bitmap, offset, limit) + subBitmap(bitmap, offset, limit) **参数** @@ -103,7 +103,7 @@ SELECT bitmapToArray(bitmapSubsetLimit(bitmapBuild([0,1,2,3,4,5,6,7,8,9,10,11,12 **示例** ``` sql -SELECT bitmapToArray(bitmapSubsetOffsetLimit(bitmapBuild([0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,100,200,500]), toUInt32(10), toUInt32(10))) AS res +SELECT bitmapToArray(subBitmap(bitmapBuild([0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,100,200,500]), toUInt32(10), toUInt32(10))) AS res ``` ```text diff --git a/src/AggregateFunctions/AggregateFunctionGroupBitmapData.h b/src/AggregateFunctions/AggregateFunctionGroupBitmapData.h index 972d137042e..cad732b1b8f 100644 --- a/src/AggregateFunctions/AggregateFunctionGroupBitmapData.h +++ b/src/AggregateFunctions/AggregateFunctionGroupBitmapData.h @@ -586,48 +586,26 @@ public: if (isSmall()) { + UInt64 count = 0; UInt64 offset_count = 0; - std::vector answer; - for (const auto & x : small) - { - T val = x.getValue(); - if (offset_count >= offset) - { - answer.push_back(val); - } - else - { - offset_count++; - } - } - if (limit < answer.size()) - { - std::nth_element(answer.begin(), answer.begin() + limit, answer.end()); - answer.resize(limit); - } + auto it = small.begin(); + for (;it != small.end() && offset_count < offset; ++it) + ++offset_count; - for (const auto & elem : answer) - r1.add(elem); - return answer.size(); + for (;it != small.end() && count < limit; ++it, ++count) + r1.add(it->getValue()); + return count; } else { UInt64 count = 0; UInt64 offset_count = 0; - for (auto it = rb->begin(); it != rb->end(); ++it) - { - offset_count++; - if (offset_count <= offset) - continue; + auto it = rb->begin(); + for (;it != rb->end() && offset_count < offset; ++it) + ++offset_count; - if (count < limit) - { - r1.add(*it); - ++count; - } - else - break; - } + for (;it != rb->end() && count < limit; ++it, ++count) + r1.add(*it); return count; } } diff --git a/src/Functions/FunctionsBitmap.h b/src/Functions/FunctionsBitmap.h index b72ee06dc01..8155dbdcf8e 100644 --- a/src/Functions/FunctionsBitmap.h +++ b/src/Functions/FunctionsBitmap.h @@ -463,7 +463,7 @@ public: struct BitmapSubsetOffsetLimitImpl { public: - static constexpr auto name = "bitmapSubsetOffsetLimit"; + static constexpr auto name = "subBitmap"; template static void apply( const AggregateFunctionGroupBitmapData & bitmap_data_0, diff --git a/tests/queries/0_stateless/00829_bitmap_function.sql b/tests/queries/0_stateless/00829_bitmap_function.sql index 0854f788a50..fde0176de5b 100644 --- a/tests/queries/0_stateless/00829_bitmap_function.sql +++ b/tests/queries/0_stateless/00829_bitmap_function.sql @@ -286,22 +286,22 @@ select bitmapToArray(bitmapSubsetLimit(bitmapBuild([ 0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33, 100,200,500]), toUInt32(100), toUInt16(200))); --- bitmapSubsetOffsetLimit: +-- subBitmap: ---- Empty -SELECT bitmapToArray(bitmapSubsetOffsetLimit(bitmapBuild(emptyArrayUInt32()), toUInt8(0), toUInt32(10))); -SELECT bitmapToArray(bitmapSubsetOffsetLimit(bitmapBuild(emptyArrayUInt16()), toUInt32(0), toUInt64(10))); +SELECT bitmapToArray(subBitmap(bitmapBuild(emptyArrayUInt32()), toUInt8(0), toUInt32(10))); +SELECT bitmapToArray(subBitmap(bitmapBuild(emptyArrayUInt16()), toUInt32(0), toUInt64(10))); ---- Small -select bitmapToArray(bitmapSubsetOffsetLimit(bitmapBuild([1,5,7,9]), toUInt8(0), toUInt32(4))); -select bitmapToArray(bitmapSubsetOffsetLimit(bitmapBuild([1,5,7,9]), toUInt32(1), toUInt64(4))); -select bitmapToArray(bitmapSubsetOffsetLimit(bitmapBuild([1,5,7,9]), toUInt16(1), toUInt32(2))); +select bitmapToArray(subBitmap(bitmapBuild([1,5,7,9]), toUInt8(0), toUInt32(4))); +select bitmapToArray(subBitmap(bitmapBuild([1,5,7,9]), toUInt32(1), toUInt64(4))); +select bitmapToArray(subBitmap(bitmapBuild([1,5,7,9]), toUInt16(1), toUInt32(2))); ---- Large -select bitmapToArray(bitmapSubsetOffsetLimit(bitmapBuild([ +select bitmapToArray(subBitmap(bitmapBuild([ 0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33, 100,200,500]), toUInt32(0), toUInt32(10))); -select bitmapToArray(bitmapSubsetOffsetLimit(bitmapBuild([ +select bitmapToArray(subBitmap(bitmapBuild([ 0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33, 100,200,500]), toUInt32(30), toUInt32(200))); -select bitmapToArray(bitmapSubsetOffsetLimit(bitmapBuild([ +select bitmapToArray(subBitmap(bitmapBuild([ 0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33, 100,200,500]), toUInt32(34), toUInt16(3))); From 3fdc2aae1917e5d404f98d0f8bf7742ebee10a6f Mon Sep 17 00:00:00 2001 From: Braulio Valdivielso Date: Sat, 7 Aug 2021 11:54:54 +0100 Subject: [PATCH 256/599] test that the tcp handler returns correct http responses I have added two tests. One checks the usual scenario where the instance's config specifies a `http_port`. In that case, we expect the `http_port` to be part of the response. The other scenario checks what happens when the config doesn't specify a `http_port`. It's hard to nail down the expectation there without hardcoding the error response (which is not what we generally want) but I'm mainly checking that no HTTP port is mentioned. This particular test would have failed before my previous fix, because the TCP handler wouldn't have managed to send back an HTTP response and `requests.post` would have thrown an exception. --- .../__init__.py | 0 .../configs/config.d/http-port-31337.xml | 3 ++ .../configs/config.d/no-http-port.xml | 3 ++ .../test_case.py | 42 +++++++++++++++++++ 4 files changed, 48 insertions(+) create mode 100644 tests/integration/test_tcp_handler_http_responses/__init__.py create mode 100644 tests/integration/test_tcp_handler_http_responses/configs/config.d/http-port-31337.xml create mode 100644 tests/integration/test_tcp_handler_http_responses/configs/config.d/no-http-port.xml create mode 100644 tests/integration/test_tcp_handler_http_responses/test_case.py diff --git a/tests/integration/test_tcp_handler_http_responses/__init__.py b/tests/integration/test_tcp_handler_http_responses/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_tcp_handler_http_responses/configs/config.d/http-port-31337.xml b/tests/integration/test_tcp_handler_http_responses/configs/config.d/http-port-31337.xml new file mode 100644 index 00000000000..23ce561b5b2 --- /dev/null +++ b/tests/integration/test_tcp_handler_http_responses/configs/config.d/http-port-31337.xml @@ -0,0 +1,3 @@ + + 31337 + diff --git a/tests/integration/test_tcp_handler_http_responses/configs/config.d/no-http-port.xml b/tests/integration/test_tcp_handler_http_responses/configs/config.d/no-http-port.xml new file mode 100644 index 00000000000..86dab92bd91 --- /dev/null +++ b/tests/integration/test_tcp_handler_http_responses/configs/config.d/no-http-port.xml @@ -0,0 +1,3 @@ + + + diff --git a/tests/integration/test_tcp_handler_http_responses/test_case.py b/tests/integration/test_tcp_handler_http_responses/test_case.py new file mode 100644 index 00000000000..38b5ba909a7 --- /dev/null +++ b/tests/integration/test_tcp_handler_http_responses/test_case.py @@ -0,0 +1,42 @@ +"""Test HTTP responses given by the TCP Handler.""" +from pathlib import Path +import pytest +from helpers.cluster import ClickHouseCluster +import requests + +cluster = ClickHouseCluster(__file__) + +node_with_http = cluster.add_instance( + 'node_with_http', + main_configs=["configs/config.d/http-port-31337.xml"] +) +HTTP_PORT = 31337 + +node_without_http = cluster.add_instance( + 'node_without_http', + main_configs=["configs/config.d/no-http-port.xml"] +) + +@pytest.fixture(scope="module") +def start_cluster(): + try: + cluster.start() + yield cluster + + finally: + cluster.shutdown() + +def test_request_to_http_full_instance(start_cluster): + response = requests.get( + f'http://{node_with_http.ip_address}:9000' + ) + assert response.status_code == 400 + assert str(HTTP_PORT) in response.text + +def test_request_to_http_less_instance(start_cluster): + response = requests.post( + f'http://{node_without_http.ip_address}:9000' + ) + assert response.status_code == 400 + assert str(HTTP_PORT) not in response.text + assert "8123" not in response.text From 08beb313e88f24ff72facd09d048f8f47b4c5827 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 7 Aug 2021 15:06:01 +0300 Subject: [PATCH 257/599] Mark more tests as long --- ...ce => 01213_alter_rename_primary_key_zookeeper_long.reference} | 0 ...eper.sql => 01213_alter_rename_primary_key_zookeeper_long.sql} | 0 ...nce => 01451_replicated_detach_drop_and_quorum_long.reference} | 0 ...uorum.sql => 01451_replicated_detach_drop_and_quorum_long.sql} | 0 ...per.reference => 01761_alter_decimal_zookeeper_long.reference} | 0 ...cimal_zookeeper.sql => 01761_alter_decimal_zookeeper_long.sql} | 0 6 files changed, 0 insertions(+), 0 deletions(-) rename tests/queries/0_stateless/{01213_alter_rename_primary_key_zookeeper.reference => 01213_alter_rename_primary_key_zookeeper_long.reference} (100%) rename tests/queries/0_stateless/{01213_alter_rename_primary_key_zookeeper.sql => 01213_alter_rename_primary_key_zookeeper_long.sql} (100%) rename tests/queries/0_stateless/{01451_replicated_detach_drop_and_quorum.reference => 01451_replicated_detach_drop_and_quorum_long.reference} (100%) rename tests/queries/0_stateless/{01451_replicated_detach_drop_and_quorum.sql => 01451_replicated_detach_drop_and_quorum_long.sql} (100%) rename tests/queries/0_stateless/{01761_alter_decimal_zookeeper.reference => 01761_alter_decimal_zookeeper_long.reference} (100%) rename tests/queries/0_stateless/{01761_alter_decimal_zookeeper.sql => 01761_alter_decimal_zookeeper_long.sql} (100%) diff --git a/tests/queries/0_stateless/01213_alter_rename_primary_key_zookeeper.reference b/tests/queries/0_stateless/01213_alter_rename_primary_key_zookeeper_long.reference similarity index 100% rename from tests/queries/0_stateless/01213_alter_rename_primary_key_zookeeper.reference rename to tests/queries/0_stateless/01213_alter_rename_primary_key_zookeeper_long.reference diff --git a/tests/queries/0_stateless/01213_alter_rename_primary_key_zookeeper.sql b/tests/queries/0_stateless/01213_alter_rename_primary_key_zookeeper_long.sql similarity index 100% rename from tests/queries/0_stateless/01213_alter_rename_primary_key_zookeeper.sql rename to tests/queries/0_stateless/01213_alter_rename_primary_key_zookeeper_long.sql diff --git a/tests/queries/0_stateless/01451_replicated_detach_drop_and_quorum.reference b/tests/queries/0_stateless/01451_replicated_detach_drop_and_quorum_long.reference similarity index 100% rename from tests/queries/0_stateless/01451_replicated_detach_drop_and_quorum.reference rename to tests/queries/0_stateless/01451_replicated_detach_drop_and_quorum_long.reference diff --git a/tests/queries/0_stateless/01451_replicated_detach_drop_and_quorum.sql b/tests/queries/0_stateless/01451_replicated_detach_drop_and_quorum_long.sql similarity index 100% rename from tests/queries/0_stateless/01451_replicated_detach_drop_and_quorum.sql rename to tests/queries/0_stateless/01451_replicated_detach_drop_and_quorum_long.sql diff --git a/tests/queries/0_stateless/01761_alter_decimal_zookeeper.reference b/tests/queries/0_stateless/01761_alter_decimal_zookeeper_long.reference similarity index 100% rename from tests/queries/0_stateless/01761_alter_decimal_zookeeper.reference rename to tests/queries/0_stateless/01761_alter_decimal_zookeeper_long.reference diff --git a/tests/queries/0_stateless/01761_alter_decimal_zookeeper.sql b/tests/queries/0_stateless/01761_alter_decimal_zookeeper_long.sql similarity index 100% rename from tests/queries/0_stateless/01761_alter_decimal_zookeeper.sql rename to tests/queries/0_stateless/01761_alter_decimal_zookeeper_long.sql From 07e1047b4f1c979aee35f47355164c554bb7358c Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 7 Aug 2021 16:30:12 +0300 Subject: [PATCH 258/599] Mark more tests as long --- ...ce => 01493_alter_remove_no_property_zookeeper_long.reference} | 0 ...eper.sql => 01493_alter_remove_no_property_zookeeper_long.sql} | 0 2 files changed, 0 insertions(+), 0 deletions(-) rename tests/queries/0_stateless/{01493_alter_remove_no_property_zookeeper.reference => 01493_alter_remove_no_property_zookeeper_long.reference} (100%) rename tests/queries/0_stateless/{01493_alter_remove_no_property_zookeeper.sql => 01493_alter_remove_no_property_zookeeper_long.sql} (100%) diff --git a/tests/queries/0_stateless/01493_alter_remove_no_property_zookeeper.reference b/tests/queries/0_stateless/01493_alter_remove_no_property_zookeeper_long.reference similarity index 100% rename from tests/queries/0_stateless/01493_alter_remove_no_property_zookeeper.reference rename to tests/queries/0_stateless/01493_alter_remove_no_property_zookeeper_long.reference diff --git a/tests/queries/0_stateless/01493_alter_remove_no_property_zookeeper.sql b/tests/queries/0_stateless/01493_alter_remove_no_property_zookeeper_long.sql similarity index 100% rename from tests/queries/0_stateless/01493_alter_remove_no_property_zookeeper.sql rename to tests/queries/0_stateless/01493_alter_remove_no_property_zookeeper_long.sql From 01b4bd3f91806d8c806a5dfc9af21c69b99099e5 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 7 Aug 2021 11:31:53 +0000 Subject: [PATCH 259/599] Fix some more tests --- src/Functions/FunctionsConversion.h | 2 +- .../0_stateless/00974_low_cardinality_cast.sql | 2 ++ .../01402_cast_nullable_string_to_enum.sql | 2 ++ tests/queries/0_stateless/01536_fuzz_cast.sql | 1 + .../0_stateless/01655_plan_optimizations.reference | 2 +- .../queries/0_stateless/01655_plan_optimizations.sh | 2 +- .../0_stateless/01720_join_implicit_cast.sql.j2 | 4 ++-- ..._optimize_skip_unused_shards_rewrite_in.reference | 12 ++++++------ 8 files changed, 16 insertions(+), 11 deletions(-) diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index c277997593c..362249b3b27 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -3470,7 +3470,7 @@ protected: if constexpr (internal) return type; - if (keep_nullable && arguments.front().type->isNullable()) + if (keep_nullable && arguments.front().type->isNullable() && type->canBeInsideNullable()) return makeNullable(type); return type; diff --git a/tests/queries/0_stateless/00974_low_cardinality_cast.sql b/tests/queries/0_stateless/00974_low_cardinality_cast.sql index e369a8c169e..b52c00513d3 100644 --- a/tests/queries/0_stateless/00974_low_cardinality_cast.sql +++ b/tests/queries/0_stateless/00974_low_cardinality_cast.sql @@ -1,3 +1,5 @@ +SET cast_keep_nullable = 0; + SELECT CAST('Hello' AS LowCardinality(Nullable(String))); SELECT CAST(Null AS LowCardinality(Nullable(String))); SELECT CAST(CAST('Hello' AS LowCardinality(Nullable(String))) AS String); diff --git a/tests/queries/0_stateless/01402_cast_nullable_string_to_enum.sql b/tests/queries/0_stateless/01402_cast_nullable_string_to_enum.sql index aa8e8be6673..3b53e593095 100644 --- a/tests/queries/0_stateless/01402_cast_nullable_string_to_enum.sql +++ b/tests/queries/0_stateless/01402_cast_nullable_string_to_enum.sql @@ -1,3 +1,5 @@ +SET cast_keep_nullable = 0; + -- https://github.com/ClickHouse/ClickHouse/issues/5818#issuecomment-619628445 SELECT CAST(CAST(NULL AS Nullable(String)) AS Nullable(Enum8('Hello' = 1))); SELECT CAST(CAST(NULL AS Nullable(FixedString(1))) AS Nullable(Enum8('Hello' = 1))); diff --git a/tests/queries/0_stateless/01536_fuzz_cast.sql b/tests/queries/0_stateless/01536_fuzz_cast.sql index 436d76b5c4c..fb1303549b6 100644 --- a/tests/queries/0_stateless/01536_fuzz_cast.sql +++ b/tests/queries/0_stateless/01536_fuzz_cast.sql @@ -1 +1,2 @@ +SET cast_keep_nullable = 0; SELECT CAST(arrayJoin([NULL, '', '', NULL, '', NULL, '01.02.2017 03:04\005GMT', '', NULL, '01/02/2017 03:04:05 MSK01/02/\0017 03:04:05 MSK', '', NULL, '03/04/201903/04/201903/04/\001903/04/2019']), 'Enum8(\'a\' = 1, \'b\' = 2)') AS x; -- { serverError 349 } diff --git a/tests/queries/0_stateless/01655_plan_optimizations.reference b/tests/queries/0_stateless/01655_plan_optimizations.reference index 22f5a2e73e3..22876207862 100644 --- a/tests/queries/0_stateless/01655_plan_optimizations.reference +++ b/tests/queries/0_stateless/01655_plan_optimizations.reference @@ -56,7 +56,7 @@ Filter column: notEquals(y, 0) 9 10 > one condition of filter should be pushed down after aggregating, other condition is casted Filter column -FUNCTION CAST(minus(s, 4) :: 1, UInt8 :: 3) -> and(notEquals(y, 0), minus(s, 4)) +FUNCTION _CAST(minus(s, 4) :: 1, UInt8 :: 3) -> and(notEquals(y, 0), minus(s, 4)) Aggregating Filter column: notEquals(y, 0) 0 1 diff --git a/tests/queries/0_stateless/01655_plan_optimizations.sh b/tests/queries/0_stateless/01655_plan_optimizations.sh index 148e6157773..b835bae0e27 100755 --- a/tests/queries/0_stateless/01655_plan_optimizations.sh +++ b/tests/queries/0_stateless/01655_plan_optimizations.sh @@ -56,7 +56,7 @@ $CLICKHOUSE_CLIENT -q " select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y ) where y != 0 and s - 4 settings enable_optimize_predicate_expression=0" | - grep -o "Aggregating\|Filter column\|Filter column: notEquals(y, 0)\|FUNCTION CAST(minus(s, 4) :: 1, UInt8 :: 3) -> and(notEquals(y, 0), minus(s, 4))" + grep -o "Aggregating\|Filter column\|Filter column: notEquals(y, 0)\|FUNCTION _CAST(minus(s, 4) :: 1, UInt8 :: 3) -> and(notEquals(y, 0), minus(s, 4))" $CLICKHOUSE_CLIENT -q " select s, y from ( select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y diff --git a/tests/queries/0_stateless/01720_join_implicit_cast.sql.j2 b/tests/queries/0_stateless/01720_join_implicit_cast.sql.j2 index f7760c38163..f2b13e9824b 100644 --- a/tests/queries/0_stateless/01720_join_implicit_cast.sql.j2 +++ b/tests/queries/0_stateless/01720_join_implicit_cast.sql.j2 @@ -48,8 +48,8 @@ SELECT any(toTypeName(a)) == 'Int32' AND any(toTypeName(b)) == 'Nullable(Int64)' SELECT any(toTypeName(a)) == 'Int32' AND any(toTypeName(b)) == 'Nullable(Int64)' FROM t_ab1 RIGHT JOIN t_ab2 USING (a, b); SELECT any(toTypeName(a)) == 'Int32' AND any(toTypeName(b)) == 'Nullable(Int64)' FROM t_ab1 INNER JOIN t_ab2 USING (a, b); -SELECT * FROM ( SELECT a, b as "CAST(a, Int32)" FROM t_ab1 ) t_ab1 FULL JOIN t_ab2 ON (t_ab1.a == t_ab2.a); -- { serverError 44 } -SELECT * FROM ( SELECT a, b as "CAST(a, Int32)" FROM t_ab1 ) t_ab1 FULL JOIN t_ab2 USING (a) FORMAT Null; +SELECT * FROM ( SELECT a, b as "_CAST(a, Int32)" FROM t_ab1 ) t_ab1 FULL JOIN t_ab2 ON (t_ab1.a == t_ab2.a); -- { serverError 44 } +SELECT * FROM ( SELECT a, b as "_CAST(a, Int32)" FROM t_ab1 ) t_ab1 FULL JOIN t_ab2 USING (a) FORMAT Null; {% endfor %} diff --git a/tests/queries/0_stateless/01756_optimize_skip_unused_shards_rewrite_in.reference b/tests/queries/0_stateless/01756_optimize_skip_unused_shards_rewrite_in.reference index 972f4c89bdf..66fbe8a5d1c 100644 --- a/tests/queries/0_stateless/01756_optimize_skip_unused_shards_rewrite_in.reference +++ b/tests/queries/0_stateless/01756_optimize_skip_unused_shards_rewrite_in.reference @@ -1,17 +1,17 @@ (0, 2) 0 0 0 0 -WITH CAST(\'default\', \'Nullable(String)\') AS id_no SELECT one.dummy, ignore(id_no) FROM system.one WHERE dummy IN (0, 2) -WITH CAST(\'default\', \'Nullable(String)\') AS id_no SELECT one.dummy, ignore(id_no) FROM system.one WHERE dummy IN (0, 2) +WITH _CAST(\'default\', \'Nullable(String)\') AS id_no SELECT one.dummy, ignore(id_no) FROM system.one WHERE dummy IN (0, 2) +WITH _CAST(\'default\', \'Nullable(String)\') AS id_no SELECT one.dummy, ignore(id_no) FROM system.one WHERE dummy IN (0, 2) optimize_skip_unused_shards_rewrite_in(0, 2) 0 0 -WITH CAST(\'default\', \'Nullable(String)\') AS id_02 SELECT one.dummy, ignore(id_02) FROM system.one WHERE dummy IN tuple(0) -WITH CAST(\'default\', \'Nullable(String)\') AS id_02 SELECT one.dummy, ignore(id_02) FROM system.one WHERE dummy IN tuple(2) +WITH _CAST(\'default\', \'Nullable(String)\') AS id_02 SELECT one.dummy, ignore(id_02) FROM system.one WHERE dummy IN tuple(0) +WITH _CAST(\'default\', \'Nullable(String)\') AS id_02 SELECT one.dummy, ignore(id_02) FROM system.one WHERE dummy IN tuple(2) optimize_skip_unused_shards_rewrite_in(2,) -WITH CAST(\'default\', \'Nullable(String)\') AS id_2 SELECT one.dummy, ignore(id_2) FROM system.one WHERE dummy IN tuple(2) +WITH _CAST(\'default\', \'Nullable(String)\') AS id_2 SELECT one.dummy, ignore(id_2) FROM system.one WHERE dummy IN tuple(2) optimize_skip_unused_shards_rewrite_in(0,) 0 0 -WITH CAST(\'default\', \'Nullable(String)\') AS id_0 SELECT one.dummy, ignore(id_0) FROM system.one WHERE dummy IN tuple(0) +WITH _CAST(\'default\', \'Nullable(String)\') AS id_0 SELECT one.dummy, ignore(id_0) FROM system.one WHERE dummy IN tuple(0) 0 0 errors From 86ceaaadb9a40bb00eb74e93a2cd05a7664db7e6 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 7 Aug 2021 17:40:09 +0300 Subject: [PATCH 260/599] Mark more tests as long --- ...ce => 01747_alter_partition_key_enum_zookeeper_long.reference} | 0 ...eper.sql => 01747_alter_partition_key_enum_zookeeper_long.sql} | 0 2 files changed, 0 insertions(+), 0 deletions(-) rename tests/queries/0_stateless/{01747_alter_partition_key_enum_zookeeper.reference => 01747_alter_partition_key_enum_zookeeper_long.reference} (100%) rename tests/queries/0_stateless/{01747_alter_partition_key_enum_zookeeper.sql => 01747_alter_partition_key_enum_zookeeper_long.sql} (100%) diff --git a/tests/queries/0_stateless/01747_alter_partition_key_enum_zookeeper.reference b/tests/queries/0_stateless/01747_alter_partition_key_enum_zookeeper_long.reference similarity index 100% rename from tests/queries/0_stateless/01747_alter_partition_key_enum_zookeeper.reference rename to tests/queries/0_stateless/01747_alter_partition_key_enum_zookeeper_long.reference diff --git a/tests/queries/0_stateless/01747_alter_partition_key_enum_zookeeper.sql b/tests/queries/0_stateless/01747_alter_partition_key_enum_zookeeper_long.sql similarity index 100% rename from tests/queries/0_stateless/01747_alter_partition_key_enum_zookeeper.sql rename to tests/queries/0_stateless/01747_alter_partition_key_enum_zookeeper_long.sql From ef1a73e78e50ddc4a549ccbd8134a1e6e463dbac Mon Sep 17 00:00:00 2001 From: Anna <42538400+adevyatova@users.noreply.github.com> Date: Sat, 7 Aug 2021 18:10:55 +0300 Subject: [PATCH 261/599] Update other-functions.md --- docs/en/sql-reference/functions/other-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index fd710b89012..9618bb04b2c 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -2222,7 +2222,7 @@ Type: [Array](../../sql-reference/data-types/array.md)([String](../../sql-refere ## defaultRoles {#default-roles} -Returns the names of the roles which are enabled by default for the current user when he logins. Initially these are all roles granted to the current user (see [GRANT](../../sql-reference/statements/grant/#grant-select)). The list of the user roles can be changed lately with the [SET DEFAULT ROLE](../../sql-reference/statements/set-role.md#set-default-role-statement) statement. +Returns the names of the roles which are enabled by default for the current user when he logins. Initially these are all roles granted to the current user (see [GRANT](../../sql-reference/statements/grant/#grant-select)). But this list can be changed with the [SET DEFAULT ROLE](../../sql-reference/statements/set-role.md#set-default-role-statement) statement. **Syntax** From b5a6e18511f7df9c86151a14d7827e4ab630f46c Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Sat, 7 Aug 2021 15:15:49 +0000 Subject: [PATCH 262/599] Small fixes --- .gitmodules | 2 +- src/Common/ErrorCodes.cpp | 4 ++-- tests/queries/0_stateless/02003_compress_bz2.sh | 2 +- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/.gitmodules b/.gitmodules index 6bcb2f18c61..37b22527eb4 100644 --- a/.gitmodules +++ b/.gitmodules @@ -245,4 +245,4 @@ url = https://github.com/ClickHouse-Extras/s2geometry.git [submodule "contrib/bzip2"] path = contrib/bzip2 - url = https://gitlab.com/federicomenaquintero/bzip2.git + url = https://github.com/ClickHouse-Extras/bzip2.git diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index 2bb378f13b5..04f10fb536a 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -561,14 +561,14 @@ M(591, SQLITE_ENGINE_ERROR) \ M(592, DATA_ENCRYPTION_ERROR) \ M(593, ZERO_COPY_REPLICATION_ERROR) \ + M(594, BZIP2_STREAM_DECODER_FAILED) \ + M(595, BZIP2_STREAM_ENCODER_FAILED) \ \ M(998, POSTGRESQL_CONNECTION_FAILURE) \ M(999, KEEPER_EXCEPTION) \ M(1000, POCO_EXCEPTION) \ M(1001, STD_EXCEPTION) \ M(1002, UNKNOWN_EXCEPTION) \ - M(1003, BZIP2_STREAM_DECODER_FAILED) \ - M(1004, BZIP2_STREAM_ENCODER_FAILED) \ /* See END */ diff --git a/tests/queries/0_stateless/02003_compress_bz2.sh b/tests/queries/0_stateless/02003_compress_bz2.sh index 790a520ef24..b65ce2f233b 100755 --- a/tests/queries/0_stateless/02003_compress_bz2.sh +++ b/tests/queries/0_stateless/02003_compress_bz2.sh @@ -8,7 +8,7 @@ USER_FILES_PATH=$(clickhouse-client --query "select _path,_file from file('nonex WORKING_FOLDER_02003="${USER_FILES_PATH}/${CLICKHOUSE_DATABASE}" rm -rf "${WORKING_FOLDER_02003}" -mkdir ${WORKING_FOLDER_02003} +mkdir "${WORKING_FOLDER_02003}" ${CLICKHOUSE_CLIENT} --query "SELECT * FROM (SELECT 'Hello, World!' as String) INTO OUTFILE '${WORKING_FOLDER_02003}/bz2_compression.bz2'" bzip2 -t ${WORKING_FOLDER_02003}/bz2_compression.bz2 From 5523e5e1533a1d69689eba4ab8308af2e8264779 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sat, 7 Aug 2021 18:48:52 +0300 Subject: [PATCH 263/599] Update Settings.h --- src/Core/Settings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index d3493677af5..b1e9cc66a56 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -378,7 +378,7 @@ class IColumn; \ M(Bool, allow_hyperscan, true, "Allow functions that use Hyperscan library. Disable to avoid potentially long compilation times and excessive resource usage.", 0) \ M(UInt64, max_hyperscan_regexp_length, 0, "Max length of regexp than can be used in hyperscan multi-match functions. Zero means unlimited.", 0) \ - M(UInt64, max_hyperscan_regexp_total_length, 0, "Max total length of all regexps than can be used in hyperscan multi-match functions. Zero means unlimited.", 0) \ + M(UInt64, max_hyperscan_regexp_total_length, 0, "Max total length of all regexps than can be used in hyperscan multi-match functions (per every function). Zero means unlimited.", 0) \ M(Bool, allow_simdjson, true, "Allow using simdjson library in 'JSON*' functions if AVX2 instructions are available. If disabled rapidjson will be used.", 0) \ M(Bool, allow_introspection_functions, false, "Allow functions for introspection of ELF and DWARF for query profiling. These functions are slow and may impose security considerations.", 0) \ \ From 27f489b41fd833d48d5f7ff25a45144f4a02d5f1 Mon Sep 17 00:00:00 2001 From: Dmitriy Date: Sat, 7 Aug 2021 18:50:24 +0300 Subject: [PATCH 264/599] Correction of the description MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Скорректировал описание. --- .../engines/database-engines/materialized-postgresql.md | 2 +- .../engines/database-engines/materialized-postgresql.md | 8 ++++---- .../table-engines/integrations/materialized-postgresql.md | 2 +- 3 files changed, 6 insertions(+), 6 deletions(-) diff --git a/docs/en/engines/database-engines/materialized-postgresql.md b/docs/en/engines/database-engines/materialized-postgresql.md index a5e83140651..89c7c803bb3 100644 --- a/docs/en/engines/database-engines/materialized-postgresql.md +++ b/docs/en/engines/database-engines/materialized-postgresql.md @@ -5,7 +5,7 @@ toc_title: MaterializedPostgreSQL # [experimental] MaterializedPostgreSQL {#materialize-postgresql} -Creates ClickHouse database with all the tables existing in PostgreSQL, and all the data in those tables. +Creates ClickHouse database with an initial data dump of PostgreSQL database tables and starts replication process, i.e. executes background job to apply new changes as they happen on PostgreSQL database tables in the remote PostgreSQL database. ClickHouse server works as PostgreSQL replica. It reads WAL and performs DML queries. DDL is not replicated, but can be handled (described below). diff --git a/docs/ru/engines/database-engines/materialized-postgresql.md b/docs/ru/engines/database-engines/materialized-postgresql.md index 8ac4c9d3e38..7c3ba234fd6 100644 --- a/docs/ru/engines/database-engines/materialized-postgresql.md +++ b/docs/ru/engines/database-engines/materialized-postgresql.md @@ -5,9 +5,9 @@ toc_title: MaterializedPostgreSQL # [экспериментальный] MaterializedPostgreSQL {#materialize-postgresql} -Создает базу данных ClickHouse со всеми таблицами, существующими в PostgreSQL, и всеми данными в этих таблицах. +Создает базу данных ClickHouse с исходным дампом данных таблиц PostgreSQL и запускает процесс репликации, т.е. выполняется применение новых изменений в фоне, как эти изменения происходят в таблице PostgreSQL в удаленной базе данных PostgreSQL. -Сервер ClickHouse работает как реплика PostgreSQL. Он читает файл `binlog` и выполняет DDL и DML-запросы. +Сервер ClickHouse работает как реплика PostgreSQL. Он читает WAL и выполняет DML запросы. Данные, полученные в результате DDL запросов, не реплицируются, но сами запросы могут быть обработаны (описано ниже). ## Создание базы данных {#creating-a-database} @@ -56,8 +56,8 @@ postgres# CREATE unique INDEX postgres_table_index on postgres_table(a, c, e); postgres# ALTER TABLE postgres_table REPLICA IDENTITY USING INDEX postgres_table_index; ``` -Первичный ключ всегда проверяется первым. Если он отсутствует, то проверяется индекс, определенный как индекс идентичности реплики. -Если индекс используется в качестве идентификатора реплики, то в таблице должен быть только один такой индекс. +Первичный ключ всегда проверяется первым. Если он отсутствует, то проверяется индекс, определенный как replica identity index (репликационный идентификатор). +Если индекс используется в качестве репликационного идентификатора, то в таблице должен быть только один такой индекс. Вы можете проверить, какой тип используется для указанной таблицы, выполнив следующую команду: ``` bash diff --git a/docs/ru/engines/table-engines/integrations/materialized-postgresql.md b/docs/ru/engines/table-engines/integrations/materialized-postgresql.md index 34dc0a893fe..0f707749f07 100644 --- a/docs/ru/engines/table-engines/integrations/materialized-postgresql.md +++ b/docs/ru/engines/table-engines/integrations/materialized-postgresql.md @@ -29,7 +29,7 @@ PRIMARY KEY key; 1. Настройка [wal_level](https://postgrespro.ru/docs/postgrespro/10/runtime-config-wal) должна иметь значение `logical`, параметр `max_replication_slots` должен быть равен по меньшей мере `2` в конфигурационном файле в PostgreSQL. -2. Таблица, созданная с помощью движка `MaterializedPostgreSQL`, должна иметь первичный ключ — такой же, как индекс идентичности реплики (по умолчанию: первичный ключ) таблицы PostgreSQL (смотрите [индекс идентичности реплики](../../../engines/database-engines/materialized-postgresql.md#requirements)). +2. Таблица, созданная с помощью движка `MaterializedPostgreSQL`, должна иметь первичный ключ — такой же, как replica identity index (по умолчанию: первичный ключ) таблицы PostgreSQL (смотрите [replica identity index](../../../engines/database-engines/materialized-postgresql.md#requirements)). 3. Допускается только база данных [Atomic](https://en.wikipedia.org/wiki/Atomicity_(database_systems)). From 161aa4020693410d4b799575a86317073a06cb7f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 7 Aug 2021 19:30:59 +0300 Subject: [PATCH 265/599] Fix build; add comments --- contrib/croaring-cmake/CMakeLists.txt | 22 +++++++++++++++------- src/Common/memory.cpp | 25 +++++++++++++++++++++++++ src/Common/memory.h | 23 ----------------------- 3 files changed, 40 insertions(+), 30 deletions(-) create mode 100644 src/Common/memory.cpp diff --git a/contrib/croaring-cmake/CMakeLists.txt b/contrib/croaring-cmake/CMakeLists.txt index 1c61eb39cc8..f0cb378864b 100644 --- a/contrib/croaring-cmake/CMakeLists.txt +++ b/contrib/croaring-cmake/CMakeLists.txt @@ -25,10 +25,18 @@ target_include_directories(roaring PRIVATE "${LIBRARY_DIR}/include/roaring") target_include_directories(roaring SYSTEM BEFORE PUBLIC "${LIBRARY_DIR}/include") target_include_directories(roaring SYSTEM BEFORE PUBLIC "${LIBRARY_DIR}/cpp") -target_compile_definitions(roaring PRIVATE - -Dmalloc=clickhouse_malloc - -Dcalloc=clickhouse_calloc - -Drealloc=clickhouse_realloc - -Dreallocarray=clickhouse_reallocarray - -Dfree=clickhouse_free - -Dposix_memalign=clickhouse_posix_memalign) +# We redirect malloc/free family of functions to different functions that will track memory in ClickHouse. +# It will make this library depend on linking to 'clickhouse_common_io' library that is not done explicitly via 'target_link_libraries'. +# And we check that all libraries dependencies are satisfied and all symbols are resolved if we do build with shared libraries. +# That's why we enable it only in static build. +# Also note that we exploit implicit function declarations. + +if (USE_STATIC_LIBRARIES) + target_compile_definitions(roaring PRIVATE + -Dmalloc=clickhouse_malloc + -Dcalloc=clickhouse_calloc + -Drealloc=clickhouse_realloc + -Dreallocarray=clickhouse_reallocarray + -Dfree=clickhouse_free + -Dposix_memalign=clickhouse_posix_memalign) +endif () diff --git a/src/Common/memory.cpp b/src/Common/memory.cpp new file mode 100644 index 00000000000..a79d3572071 --- /dev/null +++ b/src/Common/memory.cpp @@ -0,0 +1,25 @@ +#if defined(OS_DARWIN) && defined(BUNDLED_STATIC_JEMALLOC) + +extern "C" +{ + extern void zone_register(); +} + +struct InitializeJemallocZoneAllocatorForOSX +{ + InitializeJemallocZoneAllocatorForOSX() + { + /// In case of OSX jemalloc register itself as a default zone allocator. + /// + /// But when you link statically then zone_register() will not be called, + /// and even will be optimized out: + /// + /// It is ok to call it twice (i.e. in case of shared libraries) + /// Since zone_register() is a no-op if the default zone is already replaced with something. + /// + /// https://github.com/jemalloc/jemalloc/issues/708 + zone_register(); + } +} initializeJemallocZoneAllocatorForOSX; + +#endif diff --git a/src/Common/memory.h b/src/Common/memory.h index 0aa99cdf70d..84c6af9a1be 100644 --- a/src/Common/memory.h +++ b/src/Common/memory.h @@ -62,29 +62,6 @@ inline ALWAYS_INLINE void deleteSized(void * ptr, std::size_t size [[maybe_unuse # include #endif -#if defined(OS_DARWIN) && defined(BUNDLED_STATIC_JEMALLOC) -extern "C" -{ -extern void zone_register(); -} - -struct InitializeJemallocZoneAllocatorForOSX -{ - InitializeJemallocZoneAllocatorForOSX() - { - /// In case of OSX jemalloc register itself as a default zone allocator. - /// - /// But when you link statically then zone_register() will not be called, - /// and even will be optimized out: - /// - /// It is ok to call it twice (i.e. in case of shared libraries) - /// Since zone_register() is a no-op if the default zone is already replaced with something. - /// - /// https://github.com/jemalloc/jemalloc/issues/708 - zone_register(); - } -} initializeJemallocZoneAllocatorForOSX; -#endif inline ALWAYS_INLINE size_t getActualAllocationSize(size_t size) { From ac3abb8fd28a54bc871f53562ee0e6d9cd724442 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sat, 7 Aug 2021 19:46:19 +0300 Subject: [PATCH 266/599] Update test.py --- tests/integration/test_merge_tree_s3_failover/test.py | 3 +++ 1 file changed, 3 insertions(+) diff --git a/tests/integration/test_merge_tree_s3_failover/test.py b/tests/integration/test_merge_tree_s3_failover/test.py index d19236f4425..56d9441aba6 100644 --- a/tests/integration/test_merge_tree_s3_failover/test.py +++ b/tests/integration/test_merge_tree_s3_failover/test.py @@ -68,7 +68,10 @@ def drop_table(cluster): # S3 request will be failed for an appropriate part file write. FILES_PER_PART_BASE = 5 # partition.dat, default_compression_codec.txt, count.txt, columns.txt, checksums.txt FILES_PER_PART_WIDE = FILES_PER_PART_BASE + 1 + 1 + 3 * 2 # Primary index, MinMax, Mark and data file for column(s) + +# In debug build there are additional requests (from MergeTreeDataPartWriterWide.cpp:554 due to additional validation). FILES_PER_PART_WIDE_DEBUG = 2 # Additional requests to S3 in debug build + FILES_PER_PART_COMPACT = FILES_PER_PART_BASE + 1 + 1 + 2 FILES_PER_PART_COMPACT_DEBUG = 0 From a833d1cf9efdb85923d8fde5ac712f45239ffe62 Mon Sep 17 00:00:00 2001 From: Braulio Valdivielso Date: Sat, 7 Aug 2021 19:57:19 +0100 Subject: [PATCH 267/599] fixup! change http error response --- src/Server/TCPHandler.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 7774c9ca56e..0339b144f09 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -56,7 +56,8 @@ std::string formatHTTPErrorResponse(const Poco::Util::AbstractConfiguration& con "Port {} is for clickhouse-client program\r\n", config.getString("tcp_port")); - if (config.has("http_port")) { + if (config.has("http_port")) + { result += fmt::format( "You must use port {} for HTTP.\r\n", config.getString("http_port")); From c62de36e4c790d82d59e5531c15a4f789bc4505c Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 8 Aug 2021 00:55:40 +0300 Subject: [PATCH 268/599] Mark more tests as long --- ... => 01650_fetch_patition_with_macro_in_zk_path_long.reference} | 0 ...th.sql => 01650_fetch_patition_with_macro_in_zk_path_long.sql} | 0 2 files changed, 0 insertions(+), 0 deletions(-) rename tests/queries/0_stateless/{01650_fetch_patition_with_macro_in_zk_path.reference => 01650_fetch_patition_with_macro_in_zk_path_long.reference} (100%) rename tests/queries/0_stateless/{01650_fetch_patition_with_macro_in_zk_path.sql => 01650_fetch_patition_with_macro_in_zk_path_long.sql} (100%) diff --git a/tests/queries/0_stateless/01650_fetch_patition_with_macro_in_zk_path.reference b/tests/queries/0_stateless/01650_fetch_patition_with_macro_in_zk_path_long.reference similarity index 100% rename from tests/queries/0_stateless/01650_fetch_patition_with_macro_in_zk_path.reference rename to tests/queries/0_stateless/01650_fetch_patition_with_macro_in_zk_path_long.reference diff --git a/tests/queries/0_stateless/01650_fetch_patition_with_macro_in_zk_path.sql b/tests/queries/0_stateless/01650_fetch_patition_with_macro_in_zk_path_long.sql similarity index 100% rename from tests/queries/0_stateless/01650_fetch_patition_with_macro_in_zk_path.sql rename to tests/queries/0_stateless/01650_fetch_patition_with_macro_in_zk_path_long.sql From 8ac1f3b73d3be60e7ee620ba846339474b86ad19 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sun, 8 Aug 2021 04:24:23 +0300 Subject: [PATCH 269/599] Update projection.md --- docs/en/sql-reference/statements/alter/projection.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/statements/alter/projection.md b/docs/en/sql-reference/statements/alter/projection.md index 9d762a07234..07a13fc23c4 100644 --- a/docs/en/sql-reference/statements/alter/projection.md +++ b/docs/en/sql-reference/statements/alter/projection.md @@ -11,7 +11,7 @@ The following operations are available: - `ALTER TABLE [db].name DROP PROJECTION name` - Removes projection description from tables metadata and deletes projection files from disk. -- `ALTER TABLE [db.]table MATERIALIZE PROJECTION name IN PARTITION partition_name` - The query rebuilds the projection `name` in the partition `partition_name`. Implemented as a [mutation](../../../../sql-reference/statements/alter/index.md#mutations). +- `ALTER TABLE [db.]table MATERIALIZE PROJECTION name IN PARTITION partition_name` - The query rebuilds the projection `name` in the partition `partition_name`. Implemented as a [mutation](../../../sql-reference/statements/alter/index.md#mutations). - `ALTER TABLE [db.]table CLEAR PROJECTION name IN PARTITION partition_name` - Deletes projection files from disk without removing description. @@ -20,4 +20,4 @@ The commands ADD, DROP and CLEAR are lightweight in a sense that they only chang Also, they are replicated, syncing projections metadata via ZooKeeper. !!! note "Note" - Projection manipulation is supported only for tables with [`*MergeTree`](../../../../engines/table-engines/mergetree-family/mergetree.md) engine (including [replicated](../../../../engines/table-engines/mergetree-family/replication.md) variants). + Projection manipulation is supported only for tables with [`*MergeTree`](../../../engines/table-engines/mergetree-family/mergetree.md) engine (including [replicated](../../../engines/table-engines/mergetree-family/replication.md) variants). From 3f8c1a1433552d306ee8d70942b7a046ccceb47d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 8 Aug 2021 04:36:34 +0300 Subject: [PATCH 270/599] Add a test --- .../queries/0_stateless/01961_roaring_memory_tracking.reference | 0 tests/queries/0_stateless/01961_roaring_memory_tracking.sql | 2 ++ 2 files changed, 2 insertions(+) create mode 100644 tests/queries/0_stateless/01961_roaring_memory_tracking.reference create mode 100644 tests/queries/0_stateless/01961_roaring_memory_tracking.sql diff --git a/tests/queries/0_stateless/01961_roaring_memory_tracking.reference b/tests/queries/0_stateless/01961_roaring_memory_tracking.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/01961_roaring_memory_tracking.sql b/tests/queries/0_stateless/01961_roaring_memory_tracking.sql new file mode 100644 index 00000000000..57d71cdc91c --- /dev/null +++ b/tests/queries/0_stateless/01961_roaring_memory_tracking.sql @@ -0,0 +1,2 @@ +SET max_memory_usage = '100M'; +SELECT cityHash64(rand() % 1000) as n, groupBitmapState(number) FROM numbers_mt(2000000000) GROUP BY n; -- { serverError 241 } From ba3fbbaab0de908886920decfb720492d03ce1f4 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sun, 8 Aug 2021 04:43:59 +0300 Subject: [PATCH 271/599] Update ReplicatedMergeTreeMergeStrategyPicker.cpp --- .../MergeTree/ReplicatedMergeTreeMergeStrategyPicker.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeMergeStrategyPicker.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeMergeStrategyPicker.cpp index 2d611dc0c97..4d8261da2ee 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeMergeStrategyPicker.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeMergeStrategyPicker.cpp @@ -146,7 +146,7 @@ void ReplicatedMergeTreeMergeStrategyPicker::refreshState() { if (execute_merges_on_single_replica_time_threshold > 0) { - LOG_WARNING(storage.log, "Can't find current replica in the active replicas list, or too few active replicas to use execute_merges_on_single_replica_time_threshold!"); + LOG_WARNING(storage.log, "Can't find current replica in the active replicas list, or too few active replicas to use 'execute_merges_on_single_replica_time_threshold'"); /// we can reset the settings w/o lock (it's atomic) execute_merges_on_single_replica_time_threshold = 0; } From 34e6f5c66b427915f955bf48717fb80fae89b73f Mon Sep 17 00:00:00 2001 From: fuwhu Date: Sun, 8 Aug 2021 10:22:39 +0800 Subject: [PATCH 272/599] Correct the key data type used in mapContains. --- src/Functions/map.cpp | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/src/Functions/map.cpp b/src/Functions/map.cpp index dea82470393..edd40e05304 100644 --- a/src/Functions/map.cpp +++ b/src/Functions/map.cpp @@ -184,9 +184,11 @@ public: { bool is_const = isColumnConst(*arguments[0].column); const ColumnMap * col_map = is_const ? checkAndGetColumnConstData(arguments[0].column.get()) : checkAndGetColumn(arguments[0].column.get()); - if (!col_map) + const DataTypeMap * map_type = checkAndGetDataType(arguments[0].type.get()); + if (!col_map || !map_type) throw Exception{"First argument for function " + getName() + " must be a map", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; + auto key_type = map_type->getKeyType(); const auto & nested_column = col_map->getNestedColumn(); const auto & keys_data = col_map->getNestedData().getColumn(0); @@ -196,7 +198,7 @@ public: { { is_const ? ColumnConst::create(std::move(column_array), keys_data.size()) : std::move(column_array), - std::make_shared(result_type), + std::make_shared(key_type), "" }, arguments[1] From b8a679158c5cc115a79a0a7ce87ec771e8641fb6 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 8 Aug 2021 09:36:36 +0300 Subject: [PATCH 273/599] Fix 01951_distributed_push_down_limit reference --- .../0_stateless/01951_distributed_push_down_limit.reference | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/01951_distributed_push_down_limit.reference b/tests/queries/0_stateless/01951_distributed_push_down_limit.reference index 9e803a171c4..ca3bbc4cb33 100644 --- a/tests/queries/0_stateless/01951_distributed_push_down_limit.reference +++ b/tests/queries/0_stateless/01951_distributed_push_down_limit.reference @@ -1,7 +1,7 @@ -- { echo } explain select * from remote('127.{1,2}', view(select * from numbers(1e6))) order by number limit 10 settings distributed_push_down_limit=0; Expression (Projection) - Limit (preliminary LIMIT) + Limit (preliminary LIMIT (without OFFSET)) MergingSorted (Merge sorted streams after aggregation stage for ORDER BY) SettingQuotaAndLimits (Set limits and quota after reading from storage) Union @@ -16,11 +16,11 @@ Expression (Projection) ReadFromRemote (Read from remote replica) explain select * from remote('127.{1,2}', view(select * from numbers(1e6))) order by number limit 10 settings distributed_push_down_limit=1; Expression (Projection) - Limit (preliminary LIMIT) + Limit (preliminary LIMIT (without OFFSET)) MergingSorted (Merge sorted streams after aggregation stage for ORDER BY) SettingQuotaAndLimits (Set limits and quota after reading from storage) Union - Limit (preliminary LIMIT) + Limit (preliminary LIMIT (with OFFSET)) MergingSorted (Merge sorted streams for ORDER BY) MergeSorting (Merge sorted blocks for ORDER BY) PartialSorting (Sort each block for ORDER BY) From 3be3c503aa03ae664254661f5775ee16fb95605e Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 8 Aug 2021 09:58:07 +0300 Subject: [PATCH 274/599] Fix some comments --- src/Interpreters/InterpreterSelectQuery.cpp | 2 +- src/Storages/StorageDistributed.cpp | 1 - 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 9e0185b96cd..49ebd3d48b0 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -1378,7 +1378,7 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, const BlockInpu /// Limit is no longer needed if there is prelimit. /// - /// NOTE: that LIMIT cannot be applied of OFFSET should not be applied, + /// NOTE: that LIMIT cannot be applied if OFFSET should not be applied, /// since LIMIT will apply OFFSET too. /// This is the case for various optimizations for distributed queries, /// and when LIMIT cannot be applied it will be applied on the initiator anyway. diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index ff734d02462..bad2abf3cdc 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -553,7 +553,6 @@ std::optional StorageDistributed::getOptimizedQueryP if (const ASTPtr order_by = select.orderBy()) return default_stage; - // LIMIT BY // LIMIT // OFFSET if (select.limitLength() || select.limitOffset()) From f19ff3d61c4e36622a262977337c92a7bfeee016 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 8 Aug 2021 10:10:36 +0300 Subject: [PATCH 275/599] Fix 02003_WithMergeableStateAfterAggregationAndLimit_LIMIT_BY_LIMIT_OFFSET --- ...onAndLimit_LIMIT_BY_LIMIT_OFFSET.reference | 14 +++++++----- ...regationAndLimit_LIMIT_BY_LIMIT_OFFSET.sql | 22 ++++++++++++++----- 2 files changed, 24 insertions(+), 12 deletions(-) diff --git a/tests/queries/0_stateless/02003_WithMergeableStateAfterAggregationAndLimit_LIMIT_BY_LIMIT_OFFSET.reference b/tests/queries/0_stateless/02003_WithMergeableStateAfterAggregationAndLimit_LIMIT_BY_LIMIT_OFFSET.reference index 08fe4251b69..6cdb80167d1 100644 --- a/tests/queries/0_stateless/02003_WithMergeableStateAfterAggregationAndLimit_LIMIT_BY_LIMIT_OFFSET.reference +++ b/tests/queries/0_stateless/02003_WithMergeableStateAfterAggregationAndLimit_LIMIT_BY_LIMIT_OFFSET.reference @@ -1,16 +1,17 @@ -- { echo } SELECT * FROM remote('127.{1,2}', view( - SELECT * - FROM numbers(10) + SELECT number%20 number + FROM numbers(40) + WHERE (number % 2) = (shardNum() - 1) ), number) GROUP BY number ORDER BY number ASC LIMIT 1 BY number LIMIT 5, 5 SETTINGS - optimize_skip_unused_shards = 1, - optimize_distributed_group_by_sharding_key = 1, + optimize_skip_unused_shards=1, + optimize_distributed_group_by_sharding_key=1, distributed_push_down_limit=1; 5 6 @@ -19,8 +20,9 @@ SETTINGS 9 SELECT * FROM remote('127.{1,2}', view( - SELECT * - FROM numbers(10) + SELECT number%20 number + FROM numbers(40) + WHERE (number % 2) = (shardNum() - 1) ), number) GROUP BY number ORDER BY number ASC diff --git a/tests/queries/0_stateless/02003_WithMergeableStateAfterAggregationAndLimit_LIMIT_BY_LIMIT_OFFSET.sql b/tests/queries/0_stateless/02003_WithMergeableStateAfterAggregationAndLimit_LIMIT_BY_LIMIT_OFFSET.sql index 75c06704ab9..1a446a80603 100644 --- a/tests/queries/0_stateless/02003_WithMergeableStateAfterAggregationAndLimit_LIMIT_BY_LIMIT_OFFSET.sql +++ b/tests/queries/0_stateless/02003_WithMergeableStateAfterAggregationAndLimit_LIMIT_BY_LIMIT_OFFSET.sql @@ -1,21 +1,31 @@ +-- Here we use a trick with shardNum() to generate unique data on each shard. +-- Since distributed_group_by_no_merge=2 will use WithMergeableStateAfterAggregationAndLimit, +-- which assume that the data on shards is unique +-- (LIMIT BY will be applied only on shards, not on the initiator). + +-- To distinguish echoing from the comments above we use SELECT FORMAT Null. +SELECT '' FORMAT Null; + -- { echo } SELECT * FROM remote('127.{1,2}', view( - SELECT * - FROM numbers(10) + SELECT number%20 number + FROM numbers(40) + WHERE (number % 2) = (shardNum() - 1) ), number) GROUP BY number ORDER BY number ASC LIMIT 1 BY number LIMIT 5, 5 SETTINGS - optimize_skip_unused_shards = 1, - optimize_distributed_group_by_sharding_key = 1, + optimize_skip_unused_shards=1, + optimize_distributed_group_by_sharding_key=1, distributed_push_down_limit=1; SELECT * FROM remote('127.{1,2}', view( - SELECT * - FROM numbers(10) + SELECT number%20 number + FROM numbers(40) + WHERE (number % 2) = (shardNum() - 1) ), number) GROUP BY number ORDER BY number ASC From 7a36b7a21285a03a00415b92430f6a37e9cc17e5 Mon Sep 17 00:00:00 2001 From: Artur <613623@mail.ru> Date: Sun, 8 Aug 2021 12:22:33 +0000 Subject: [PATCH 276/599] correct tests --- .../02003_memory_limit_in_client.expect | 40 +++++++++++++++++++ 1 file changed, 40 insertions(+) create mode 100755 tests/queries/0_stateless/02003_memory_limit_in_client.expect diff --git a/tests/queries/0_stateless/02003_memory_limit_in_client.expect b/tests/queries/0_stateless/02003_memory_limit_in_client.expect new file mode 100755 index 00000000000..49b81240829 --- /dev/null +++ b/tests/queries/0_stateless/02003_memory_limit_in_client.expect @@ -0,0 +1,40 @@ +#!/usr/bin/expect -f + +# This is a test for system.warnings. Testing in interactive mode is necessary, +# as we want to see certain warnings from client + +log_user 0 +set timeout 60 +match_max 100000 + +# A default timeout action is to do nothing, change it to fail +expect_after { + timeout { + exit 1 + } +} + +set basedir [file dirname $argv0] +spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --max_memory_usage_in_client=1" +expect ":) " + +send -- "SELECT arrayMap(x -> range(x), range(number)) FROM numbers(1000)\r" +expect "Code: 241" + +expect ":) " + +# Exit. +send -- "\4" +expect eof + +set basedir [file dirname $argv0] +spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --max_memory_usage_in_client=1" +expect ":) " + +send -- "SELECT * FROM (SELECT * FROM system.numbers LIMIT 600000) as num WHERE num.number=60000\r" +expect "60000" +expect ":) " + +# Exit. +send -- "\4" +expect eof From a59ce96242c1815377da0cc8d81831f2e71a29be Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Sun, 8 Aug 2021 22:29:37 +0800 Subject: [PATCH 277/599] Skip test in fasttest --- docker/test/fasttest/run.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/docker/test/fasttest/run.sh b/docker/test/fasttest/run.sh index 6419ea3659c..0b4d476eaee 100755 --- a/docker/test/fasttest/run.sh +++ b/docker/test/fasttest/run.sh @@ -279,6 +279,7 @@ function run_tests 00926_multimatch 00929_multi_match_edit_distance 01681_hyperscan_debug_assertion + 02004_max_hyperscan_regex_length 01176_mysql_client_interactive # requires mysql client 01031_mutations_interpreter_and_context From 57a7a2075933c6666893e5be95a59cf722ec56f7 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Sun, 8 Aug 2021 18:25:53 +0300 Subject: [PATCH 278/599] Update 02002_row_level_filter_bug.sh --- tests/queries/0_stateless/02002_row_level_filter_bug.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02002_row_level_filter_bug.sh b/tests/queries/0_stateless/02002_row_level_filter_bug.sh index 6c2422a235f..7e19374c123 100755 --- a/tests/queries/0_stateless/02002_row_level_filter_bug.sh +++ b/tests/queries/0_stateless/02002_row_level_filter_bug.sh @@ -40,7 +40,7 @@ $CLICKHOUSE_CLIENT --query "create role AWD;" $CLICKHOUSE_CLIENT --query "REVOKE ALL ON *.* FROM AWD;" $CLICKHOUSE_CLIENT --query "DROP USER IF EXISTS AWD_user;" -$CLICKHOUSE_CLIENT --query "CREATE USER AWD_user IDENTIFIED WITH SHA256_PASSWORD BY 'AWD_pwd' DEFAULT ROLE AWD;" +$CLICKHOUSE_CLIENT --query "CREATE USER AWD_user IDENTIFIED WITH plaintext_password BY 'AWD_pwd' DEFAULT ROLE AWD;" $CLICKHOUSE_CLIENT --query "GRANT SELECT ON test_table TO AWD;" From 702d9955c0e04b15203b250f95ba4eae5347df9f Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 8 Aug 2021 12:38:24 +0300 Subject: [PATCH 279/599] Fix distributed queries with zero shards and aggregation --- src/Storages/StorageDistributed.cpp | 16 ++++++++++++++-- ...mize_skip_unused_shards_zero_shards.reference | 4 ++++ ...9_optimize_skip_unused_shards_zero_shards.sql | 6 +++--- 3 files changed, 21 insertions(+), 5 deletions(-) diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index bad2abf3cdc..fcd0e255e5c 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -415,17 +415,22 @@ QueryProcessingStage::Enum StorageDistributed::getQueryProcessingStage( ClusterPtr cluster = getCluster(); query_info.cluster = cluster; + size_t nodes = getClusterQueriedNodes(settings, cluster); + /// Always calculate optimized cluster here, to avoid conditions during read() /// (Anyway it will be calculated in the read()) - if (getClusterQueriedNodes(settings, cluster) > 1 && settings.optimize_skip_unused_shards) + if (nodes > 1 && settings.optimize_skip_unused_shards) { ClusterPtr optimized_cluster = getOptimizedCluster(local_context, metadata_snapshot, query_info.query); if (optimized_cluster) { LOG_DEBUG(log, "Skipping irrelevant shards - the query will be sent to the following shards of the cluster (shard numbers): {}", makeFormattedListOfShards(optimized_cluster)); + cluster = optimized_cluster; query_info.optimized_cluster = cluster; + + nodes = getClusterQueriedNodes(settings, cluster); } else { @@ -460,7 +465,7 @@ QueryProcessingStage::Enum StorageDistributed::getQueryProcessingStage( /// If there is only one node, the query can be fully processed by the /// shard, initiator will work as a proxy only. - if (getClusterQueriedNodes(settings, cluster) == 1) + if (nodes == 1) { /// In case the query was processed to /// WithMergeableStateAfterAggregation/WithMergeableStateAfterAggregationAndLimit @@ -469,6 +474,13 @@ QueryProcessingStage::Enum StorageDistributed::getQueryProcessingStage( /// relevant for Distributed over Distributed return std::max(to_stage, QueryProcessingStage::Complete); } + else if (nodes == 0) + { + /// In case of 0 shards, the query should be processed fully on the initiator, + /// since we need to apply aggregations. + /// That's why we need to return FetchColumns. + return QueryProcessingStage::FetchColumns; + } auto optimized_stage = getOptimizedQueryProcessingStage(query_info, settings); if (optimized_stage) diff --git a/tests/queries/0_stateless/01759_optimize_skip_unused_shards_zero_shards.reference b/tests/queries/0_stateless/01759_optimize_skip_unused_shards_zero_shards.reference index e69de29bb2d..109c1835a6e 100644 --- a/tests/queries/0_stateless/01759_optimize_skip_unused_shards_zero_shards.reference +++ b/tests/queries/0_stateless/01759_optimize_skip_unused_shards_zero_shards.reference @@ -0,0 +1,4 @@ +-- { echo } +select * from remote('127.{1,2}', system, one, dummy) where 0 settings optimize_skip_unused_shards=1, force_optimize_skip_unused_shards=1; +select count() from remote('127.{1,2}', system, one, dummy) where 0 settings optimize_skip_unused_shards=1, force_optimize_skip_unused_shards=1; +0 diff --git a/tests/queries/0_stateless/01759_optimize_skip_unused_shards_zero_shards.sql b/tests/queries/0_stateless/01759_optimize_skip_unused_shards_zero_shards.sql index 2ddf318313f..08b9581ace2 100644 --- a/tests/queries/0_stateless/01759_optimize_skip_unused_shards_zero_shards.sql +++ b/tests/queries/0_stateless/01759_optimize_skip_unused_shards_zero_shards.sql @@ -1,3 +1,3 @@ -create table dist_01756 (dummy UInt8) ENGINE = Distributed('test_cluster_two_shards', 'system', 'one', dummy); -select ignore(1), * from dist_01756 where 0 settings optimize_skip_unused_shards=1, force_optimize_skip_unused_shards=1; -drop table dist_01756; +-- { echo } +select * from remote('127.{1,2}', system, one, dummy) where 0 settings optimize_skip_unused_shards=1, force_optimize_skip_unused_shards=1; +select count() from remote('127.{1,2}', system, one, dummy) where 0 settings optimize_skip_unused_shards=1, force_optimize_skip_unused_shards=1; From 83d871b701ec8391e2a7025e6636ef6ef977a3f7 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 8 Aug 2021 11:29:39 +0300 Subject: [PATCH 280/599] Do not miss exceptions from the ThreadPool --- src/Common/ThreadPool.cpp | 13 +++++++++++++ 1 file changed, 13 insertions(+) diff --git a/src/Common/ThreadPool.cpp b/src/Common/ThreadPool.cpp index e6ccf405e9f..8ef85d82a1d 100644 --- a/src/Common/ThreadPool.cpp +++ b/src/Common/ThreadPool.cpp @@ -192,6 +192,9 @@ template ThreadPoolImpl::~ThreadPoolImpl() { finalize(); + /// wait() hadn't been called, log exception at least. + if (first_exception) + DB::tryLogException(first_exception, __PRETTY_FUNCTION__); } template @@ -270,11 +273,21 @@ void ThreadPoolImpl::worker(typename std::list::iterator thread_ } catch (...) { + ALLOW_ALLOCATIONS_IN_SCOPE; + /// job should be reset before decrementing scheduled_jobs to /// ensure that the Job destroyed before wait() returns. job = {}; { + /// In case thread pool will not be terminated on exception + /// (this is the case for GlobalThreadPool), + /// than first_exception may be overwritten and got lost, + /// and this usually is an error, since this will finish the thread, + /// and for this the caller may not be ready. + if (!shutdown_on_exception) + DB::tryLogException(std::current_exception(), __PRETTY_FUNCTION__); + std::unique_lock lock(mutex); if (!first_exception) first_exception = std::current_exception(); // NOLINT From 5139067631fe29adf3f2c8178ed50dffc74b54c0 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 5 Aug 2021 21:04:11 +0300 Subject: [PATCH 281/599] Guard BackgroundJobsExecutor from thread termination in case of uncaught exception --- src/Storages/MergeTree/BackgroundJobsExecutor.cpp | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/src/Storages/MergeTree/BackgroundJobsExecutor.cpp b/src/Storages/MergeTree/BackgroundJobsExecutor.cpp index 36803ba5197..e4b96e55c87 100644 --- a/src/Storages/MergeTree/BackgroundJobsExecutor.cpp +++ b/src/Storages/MergeTree/BackgroundJobsExecutor.cpp @@ -180,10 +180,16 @@ void IBackgroundJobExecutor::triggerTask() } void IBackgroundJobExecutor::backgroundTaskFunction() +try { if (!scheduleJob()) scheduleTask(/* with_backoff = */ true); } +catch (...) /// Catch any exception to avoid thread termination. +{ + tryLogCurrentException(__PRETTY_FUNCTION__); + scheduleTask(/* with_backoff = */ true); +} IBackgroundJobExecutor::~IBackgroundJobExecutor() { From 7964355ecf162a34311070a455b9732a2ae7cf77 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 5 Aug 2021 21:04:11 +0300 Subject: [PATCH 282/599] Fix mutation stuck on invalid partitions in non-replicated MergeTree v2: Do not try to process empty mutations Found with flaky check [1]. [1]: https://clickhouse-test-reports.s3.yandex.net/27248/66e8c0833392c20ba8dba3780f2b0d5c18f8194e/functional_stateless_tests_flaky_check_(address).html#fail1 --- src/Storages/StorageMergeTree.cpp | 40 ++++++++++++++----- ...invalid_partition_mutation_stuck.reference | 0 ...02004_invalid_partition_mutation_stuck.sql | 33 +++++++++++++++ 3 files changed, 62 insertions(+), 11 deletions(-) create mode 100644 tests/queries/0_stateless/02004_invalid_partition_mutation_stuck.reference create mode 100644 tests/queries/0_stateless/02004_invalid_partition_mutation_stuck.sql diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 4f0046eecba..6142115cbc8 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -958,9 +958,19 @@ std::shared_ptr StorageMergeTree::se if (!commands_for_size_validation.empty()) { - MutationsInterpreter interpreter( - shared_from_this(), metadata_snapshot, commands_for_size_validation, getContext(), false); - commands_size += interpreter.evaluateCommandsSize(); + try + { + MutationsInterpreter interpreter( + shared_from_this(), metadata_snapshot, commands_for_size_validation, getContext(), false); + commands_size += interpreter.evaluateCommandsSize(); + } + catch (...) + { + MergeTreeMutationEntry & entry = it->second; + entry.latest_fail_time = time(nullptr); + entry.latest_fail_reason = getCurrentExceptionMessage(false); + continue; + } } if (current_ast_elements + commands_size >= max_ast_elements) @@ -970,17 +980,25 @@ std::shared_ptr StorageMergeTree::se commands.insert(commands.end(), it->second.commands.begin(), it->second.commands.end()); } - auto new_part_info = part->info; - new_part_info.mutation = current_mutations_by_version.rbegin()->first; + if (!commands.empty()) + { + auto new_part_info = part->info; + new_part_info.mutation = current_mutations_by_version.rbegin()->first; - future_part.parts.push_back(part); - future_part.part_info = new_part_info; - future_part.name = part->getNewName(new_part_info); - future_part.type = part->getType(); + future_part.parts.push_back(part); + future_part.part_info = new_part_info; + future_part.name = part->getNewName(new_part_info); + future_part.type = part->getType(); - tagger = std::make_unique(future_part, MergeTreeDataMergerMutator::estimateNeededDiskSpace({part}), *this, metadata_snapshot, true); - return std::make_shared(future_part, std::move(tagger), commands); + tagger = std::make_unique(future_part, MergeTreeDataMergerMutator::estimateNeededDiskSpace({part}), *this, metadata_snapshot, true); + return std::make_shared(future_part, std::move(tagger), commands); + } } + + /// Notify in case of errors + std::unique_lock lock(mutation_wait_mutex); + mutation_wait_event.notify_all(); + return {}; } diff --git a/tests/queries/0_stateless/02004_invalid_partition_mutation_stuck.reference b/tests/queries/0_stateless/02004_invalid_partition_mutation_stuck.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02004_invalid_partition_mutation_stuck.sql b/tests/queries/0_stateless/02004_invalid_partition_mutation_stuck.sql new file mode 100644 index 00000000000..481a5565095 --- /dev/null +++ b/tests/queries/0_stateless/02004_invalid_partition_mutation_stuck.sql @@ -0,0 +1,33 @@ +SET mutations_sync=2; + +DROP TABLE IF EXISTS rep_data; +CREATE TABLE rep_data +( + p Int, + t DateTime, + INDEX idx t TYPE minmax GRANULARITY 1 +) +ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/rep_data', '1') +PARTITION BY p +ORDER BY t +SETTINGS number_of_free_entries_in_pool_to_execute_mutation=0; +INSERT INTO rep_data VALUES (1, now()); +ALTER TABLE rep_data MATERIALIZE INDEX idx IN PARTITION ID 'NO_SUCH_PART'; -- { serverError 248 } +ALTER TABLE rep_data MATERIALIZE INDEX idx IN PARTITION ID '1'; +ALTER TABLE rep_data MATERIALIZE INDEX idx IN PARTITION ID '2'; + +DROP TABLE IF EXISTS data; +CREATE TABLE data +( + p Int, + t DateTime, + INDEX idx t TYPE minmax GRANULARITY 1 +) +ENGINE = MergeTree +PARTITION BY p +ORDER BY t +SETTINGS number_of_free_entries_in_pool_to_execute_mutation=0; +INSERT INTO data VALUES (1, now()); +ALTER TABLE data MATERIALIZE INDEX idx IN PARTITION ID 'NO_SUCH_PART'; -- { serverError 341 } +ALTER TABLE data MATERIALIZE INDEX idx IN PARTITION ID '1'; +ALTER TABLE data MATERIALIZE INDEX idx IN PARTITION ID '2'; From 91d7f3daa7ac8ceb9c38767ceeda909bd5388548 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 6 Aug 2021 01:04:02 +0300 Subject: [PATCH 283/599] Fix lock-order-inversion while notifying about finished mutations TSAN reports [1]: WARNING: ThreadSanitizer: lock-order-inversion (potential deadlock) (pid=36) Cycle in lock order graph: M16388 (0x7b7400011d68) => M1030334152907497744 (0x000000000000) => M16388 Mutex M1030334152907497744 acquired here while holding mutex M16388 in thread T4: 0 pthread_mutex_lock (clickhouse+0x967d536) 1 std::__1::__libcpp_mutex_lock(pthread_mutex_t*) obj-x86_64-linux-gnu/../contrib/libcxx/include/__threading_support:405:10 (clickhouse+0x1b25c7d9) 2 std::__1::mutex::lock() obj-x86_64-linux-gnu/../contrib/libcxx/src/mutex.cpp:33:14 (clickhouse+0x1b25c7d9) 3 std::__1::lock_guard::lock_guard(std::__1::mutex&) obj-x86_64-linux-gnu/../contrib/libcxx/include/__mutex_base:91:27 (clickhouse+0x15220cd9) 4 DB::StorageMergeTree::getIncompleteMutationsStatus(long, std::__1::set, std::__1::allocator >, std::__1::less, std::__1::allocator > >, std:> 5 DB::StorageMergeTree::waitForMutation(long, std::__1::basic_string, std::__1::allocator > const&)::$_0::operator()() const obj-x86_64-linux-gnu/../src/Storages/StorageMergeTree.cpp:464:36 (clickhouse+0x1521e2b1) 6 void std::__1::condition_variable::wait, std::__1::allocator > const&)::$_0>(std::__1::unique_lock&, DB::StorageMergeTree::waitForMutation(lon> 7 DB::StorageMergeTree::waitForMutation(long, std::__1::basic_string, std::__1::allocator > const&) obj-x86_64-linux-gnu/../src/Storages/StorageMergeTree.cpp:469:29 (clickhouse+0x1521e2b1) 8 DB::StorageMergeTree::mutate(DB::MutationCommands const&, std::__1::shared_ptr) obj-x86_64-linux-gnu/../src/Storages/StorageMergeTree.cpp:496:9 (clickhouse+0x15221738) 9 DB::InterpreterAlterQuery::execute() obj-x86_64-linux-gnu/../src/Interpreters/InterpreterAlterQuery.cpp:113:16 (clickhouse+0x141182f6) 10 DB::executeQueryImpl(char const*, char const*, std::__1::shared_ptr, bool, DB::QueryProcessingStage::Enum, bool, DB::ReadBuffer*) obj-x86_64-linux-gnu/../src/Interpreters/executeQuery.cpp:560:32 (clickhouse+0x149152f6) 11 DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, std::__1::shared_ptr, bool, DB::QueryProcessingStage::Enum, bool) obj-x86_64-linux-gnu/../src/Interpreters/executeQuery.cpp:909:30 (clickhous> 12 DB::TCPHandler::runImpl() obj-x86_64-linux-gnu/../src/Server/TCPHandler.cpp:313:24 (clickhouse+0x153270af) Hint: use TSAN_OPTIONS=second_deadlock_stack=1 to get more informative warning message Mutex M16388 acquired here while holding mutex M1030334152907497744 in thread T59: 0 pthread_mutex_lock (clickhouse+0x967d536) 1 std::__1::__libcpp_mutex_lock(pthread_mutex_t*) obj-x86_64-linux-gnu/../contrib/libcxx/include/__threading_support:405:10 (clickhouse+0x1b25c7d9) 2 std::__1::mutex::lock() obj-x86_64-linux-gnu/../contrib/libcxx/src/mutex.cpp:33:14 (clickhouse+0x1b25c7d9) 3 std::__1::unique_lock::unique_lock(std::__1::mutex&) obj-x86_64-linux-gnu/../contrib/libcxx/include/__mutex_base:119:61 (clickhouse+0x15226c31) 4 DB::StorageMergeTree::selectPartsToMutate(std::__1::shared_ptr const&, std::__1::basic_string, std::__1::allocator >*, std::__1::shared_ptr&) obj-x86_64-linux-> 5 DB::StorageMergeTree::scheduleDataProcessingJob(DB::IBackgroundJobExecutor&) obj-x86_64-linux-gnu/../src/Storages/StorageMergeTree.cpp:1060:28 (clickhouse+0x15228b10) 6 DB::BackgroundJobsExecutor::scheduleJob() obj-x86_64-linux-gnu/../src/Storages/MergeTree/BackgroundJobsExecutor.cpp:229:17 (clickhouse+0x14f441dc) 7 DB::IBackgroundJobExecutor::backgroundTaskFunction() obj-x86_64-linux-gnu/../src/Storages/MergeTree/BackgroundJobsExecutor.cpp:185:10 (clickhouse+0x14f438ed) [1]: https://clickhouse-test-reports.s3.yandex.net/27248/4f3b80ff33c846465983aa2bc9ae9490e1118b15/fuzzer_tsan/report.htmlfail1 --- src/Storages/StorageMergeTree.cpp | 14 ++++++++++---- 1 file changed, 10 insertions(+), 4 deletions(-) diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 6142115cbc8..01dc3b17fc3 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -995,10 +995,6 @@ std::shared_ptr StorageMergeTree::se } } - /// Notify in case of errors - std::unique_lock lock(mutation_wait_mutex); - mutation_wait_event.notify_all(); - return {}; } @@ -1053,6 +1049,7 @@ bool StorageMergeTree::scheduleDataProcessingJob(IBackgroundJobExecutor & execut auto share_lock = lockForShare(RWLockImpl::NO_QUERY, getSettings()->lock_acquire_timeout_for_background_operations); + bool has_mutations; { std::unique_lock lock(currently_processing_in_background_mutex); if (merger_mutator.merges_blocker.isCancelled()) @@ -1061,6 +1058,15 @@ bool StorageMergeTree::scheduleDataProcessingJob(IBackgroundJobExecutor & execut merge_entry = selectPartsToMerge(metadata_snapshot, false, {}, false, nullptr, share_lock, lock); if (!merge_entry) mutate_entry = selectPartsToMutate(metadata_snapshot, nullptr, share_lock); + + has_mutations = !current_mutations_by_version.empty(); + } + + if (!mutate_entry && has_mutations) + { + /// Notify in case of errors + std::lock_guard lock(mutation_wait_mutex); + mutation_wait_event.notify_all(); } if (merge_entry) From 038241b6ed6d35a90a19c739b1fc277c28dd6972 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 5 Aug 2021 21:09:17 +0300 Subject: [PATCH 284/599] Add new index data skipping minmax index format for proper Nullable support Note, that it cannot be done w/o new extension, since index does not have any header. v2: use IDisk interface for existence check v3: remove extra file existence check v4: fix MATERIALIZE INDEX --- .../MergeTree/MergeTreeDataMergerMutator.cpp | 13 ++- .../MergeTreeDataPartWriterOnDisk.cpp | 7 +- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 5 +- .../MergeTree/MergeTreeIndexFullText.cpp | 7 +- .../MergeTree/MergeTreeIndexFullText.h | 2 +- .../MergeTreeIndexGranuleBloomFilter.cpp | 8 +- .../MergeTreeIndexGranuleBloomFilter.h | 3 +- .../MergeTree/MergeTreeIndexMinMax.cpp | 93 +++++++++++-------- src/Storages/MergeTree/MergeTreeIndexMinMax.h | 5 +- .../MergeTree/MergeTreeIndexReader.cpp | 47 +++++++--- src/Storages/MergeTree/MergeTreeIndexReader.h | 5 +- src/Storages/MergeTree/MergeTreeIndexSet.cpp | 8 +- src/Storages/MergeTree/MergeTreeIndexSet.h | 2 +- src/Storages/MergeTree/MergeTreeIndices.h | 46 ++++++++- .../01410_nullable_key_and_index.sql | 8 +- 15 files changed, 178 insertions(+), 81 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 6279d2d7d6f..baea7e72b21 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -1663,7 +1663,12 @@ NameToNameVector MergeTreeDataMergerMutator::collectFilesForRenames( { if (command.type == MutationCommand::Type::DROP_INDEX) { - if (source_part->checksums.has(INDEX_FILE_PREFIX + command.column_name + ".idx")) + if (source_part->checksums.has(INDEX_FILE_PREFIX + command.column_name + ".idx2")) + { + rename_vector.emplace_back(INDEX_FILE_PREFIX + command.column_name + ".idx2", ""); + rename_vector.emplace_back(INDEX_FILE_PREFIX + command.column_name + mrk_extension, ""); + } + else if (source_part->checksums.has(INDEX_FILE_PREFIX + command.column_name + ".idx")) { rename_vector.emplace_back(INDEX_FILE_PREFIX + command.column_name + ".idx", ""); rename_vector.emplace_back(INDEX_FILE_PREFIX + command.column_name + mrk_extension, ""); @@ -1749,6 +1754,7 @@ NameSet MergeTreeDataMergerMutator::collectFilesToSkip( for (const auto & index : indices_to_recalc) { files_to_skip.insert(index->getFileName() + ".idx"); + files_to_skip.insert(index->getFileName() + ".idx2"); files_to_skip.insert(index->getFileName() + mrk_extension); } for (const auto & projection : projections_to_recalc) @@ -1893,8 +1899,11 @@ std::set MergeTreeDataMergerMutator::getIndicesToRecalculate( { const auto & index = indices[i]; + bool has_index = + source_part->checksums.has(INDEX_FILE_PREFIX + index.name + ".idx") || + source_part->checksums.has(INDEX_FILE_PREFIX + index.name + ".idx2"); // If we ask to materialize and it already exists - if (!source_part->checksums.has(INDEX_FILE_PREFIX + index.name + ".idx") && materialized_indices.count(index.name)) + if (!has_index && materialized_indices.count(index.name)) { if (indices_to_recalc.insert(index_factory.get(index)).second) { diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp index 9902add9847..4263640c1e0 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp @@ -9,11 +9,6 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -namespace -{ - constexpr auto INDEX_FILE_EXTENSION = ".idx"; -} - void MergeTreeDataPartWriterOnDisk::Stream::finalize() { compressed.next(); @@ -165,7 +160,7 @@ void MergeTreeDataPartWriterOnDisk::initSkipIndices() std::make_unique( stream_name, data_part->volume->getDisk(), - part_path + stream_name, INDEX_FILE_EXTENSION, + part_path + stream_name, index_helper->getSerializedFileExtension(), part_path + stream_name, marks_file_extension, default_codec, settings.max_compress_block_size)); skip_indices_aggregators.push_back(index_helper->createIndexAggregator()); diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 0b5351dcf01..f60acca12a7 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -1457,9 +1457,10 @@ MarkRanges MergeTreeDataSelectExecutor::filterMarksUsingIndex( size_t & granules_dropped, Poco::Logger * log) { - if (!part->volume->getDisk()->exists(part->getFullRelativePath() + index_helper->getFileName() + ".idx")) + const std::string & path_prefix = part->getFullRelativePath() + index_helper->getFileName(); + if (!index_helper->getDeserializedFormat(part->volume->getDisk(), path_prefix)) { - LOG_DEBUG(log, "File for index {} does not exist. Skipping it.", backQuote(index_helper->index.name)); + LOG_DEBUG(log, "File for index {} does not exist ({}.*). Skipping it.", backQuote(index_helper->index.name), path_prefix); return ranges; } diff --git a/src/Storages/MergeTree/MergeTreeIndexFullText.cpp b/src/Storages/MergeTree/MergeTreeIndexFullText.cpp index 10136cd1069..1c71d77b334 100644 --- a/src/Storages/MergeTree/MergeTreeIndexFullText.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexFullText.cpp @@ -101,14 +101,17 @@ MergeTreeIndexGranuleFullText::MergeTreeIndexGranuleFullText( void MergeTreeIndexGranuleFullText::serializeBinary(WriteBuffer & ostr) const { if (empty()) - throw Exception("Attempt to write empty fulltext index " + backQuote(index_name), ErrorCodes::LOGICAL_ERROR); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Attempt to write empty fulltext index {}.", backQuote(index_name)); for (const auto & bloom_filter : bloom_filters) ostr.write(reinterpret_cast(bloom_filter.getFilter().data()), params.filter_size); } -void MergeTreeIndexGranuleFullText::deserializeBinary(ReadBuffer & istr) +void MergeTreeIndexGranuleFullText::deserializeBinary(ReadBuffer & istr, MergeTreeIndexVersion version) { + if (version != 1) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown index version {}.", version); + for (auto & bloom_filter : bloom_filters) { istr.read(reinterpret_cast( diff --git a/src/Storages/MergeTree/MergeTreeIndexFullText.h b/src/Storages/MergeTree/MergeTreeIndexFullText.h index 1385621f97f..d34cbc61da2 100644 --- a/src/Storages/MergeTree/MergeTreeIndexFullText.h +++ b/src/Storages/MergeTree/MergeTreeIndexFullText.h @@ -45,7 +45,7 @@ struct MergeTreeIndexGranuleFullText final : public IMergeTreeIndexGranule ~MergeTreeIndexGranuleFullText() override = default; void serializeBinary(WriteBuffer & ostr) const override; - void deserializeBinary(ReadBuffer & istr) override; + void deserializeBinary(ReadBuffer & istr, MergeTreeIndexVersion version) override; bool empty() const override { return !has_elems; } diff --git a/src/Storages/MergeTree/MergeTreeIndexGranuleBloomFilter.cpp b/src/Storages/MergeTree/MergeTreeIndexGranuleBloomFilter.cpp index b513437fbe1..6a027b8cb8e 100644 --- a/src/Storages/MergeTree/MergeTreeIndexGranuleBloomFilter.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexGranuleBloomFilter.cpp @@ -84,10 +84,12 @@ bool MergeTreeIndexGranuleBloomFilter::empty() const return !total_rows; } -void MergeTreeIndexGranuleBloomFilter::deserializeBinary(ReadBuffer & istr) +void MergeTreeIndexGranuleBloomFilter::deserializeBinary(ReadBuffer & istr, MergeTreeIndexVersion version) { if (!empty()) - throw Exception("Cannot read data to a non-empty bloom filter index.", ErrorCodes::LOGICAL_ERROR); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot read data to a non-empty bloom filter index."); + if (version != 1) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown index version {}.", version); readVarUInt(total_rows, istr); for (auto & filter : bloom_filters) @@ -102,7 +104,7 @@ void MergeTreeIndexGranuleBloomFilter::deserializeBinary(ReadBuffer & istr) void MergeTreeIndexGranuleBloomFilter::serializeBinary(WriteBuffer & ostr) const { if (empty()) - throw Exception("Attempt to write empty bloom filter index.", ErrorCodes::LOGICAL_ERROR); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Attempt to write empty bloom filter index."); static size_t atom_size = 8; writeVarUInt(total_rows, ostr); diff --git a/src/Storages/MergeTree/MergeTreeIndexGranuleBloomFilter.h b/src/Storages/MergeTree/MergeTreeIndexGranuleBloomFilter.h index cdd4b92f80c..82bd91138a7 100644 --- a/src/Storages/MergeTree/MergeTreeIndexGranuleBloomFilter.h +++ b/src/Storages/MergeTree/MergeTreeIndexGranuleBloomFilter.h @@ -16,8 +16,7 @@ public: bool empty() const override; void serializeBinary(WriteBuffer & ostr) const override; - - void deserializeBinary(ReadBuffer & istr) override; + void deserializeBinary(ReadBuffer & istr, MergeTreeIndexVersion version) override; const std::vector & getFilters() const { return bloom_filters; } diff --git a/src/Storages/MergeTree/MergeTreeIndexMinMax.cpp b/src/Storages/MergeTree/MergeTreeIndexMinMax.cpp index ebf553295be..3a83afbd280 100644 --- a/src/Storages/MergeTree/MergeTreeIndexMinMax.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexMinMax.cpp @@ -40,28 +40,12 @@ void MergeTreeIndexGranuleMinMax::serializeBinary(WriteBuffer & ostr) const const DataTypePtr & type = index_sample_block.getByPosition(i).type; auto serialization = type->getDefaultSerialization(); - if (!type->isNullable()) - { - serialization->serializeBinary(hyperrectangle[i].left, ostr); - serialization->serializeBinary(hyperrectangle[i].right, ostr); - } - else - { - /// NOTE: that this serialization differs from - /// IMergeTreeDataPart::MinMaxIndex::store() due to preserve - /// backward compatibility. - bool is_null = hyperrectangle[i].left.isNull() || hyperrectangle[i].right.isNull(); // one is enough - writeBinary(is_null, ostr); - if (!is_null) - { - serialization->serializeBinary(hyperrectangle[i].left, ostr); - serialization->serializeBinary(hyperrectangle[i].right, ostr); - } - } + serialization->serializeBinary(hyperrectangle[i].left, ostr); + serialization->serializeBinary(hyperrectangle[i].right, ostr); } } -void MergeTreeIndexGranuleMinMax::deserializeBinary(ReadBuffer & istr) +void MergeTreeIndexGranuleMinMax::deserializeBinary(ReadBuffer & istr, MergeTreeIndexVersion version) { hyperrectangle.clear(); Field min_val; @@ -72,29 +56,53 @@ void MergeTreeIndexGranuleMinMax::deserializeBinary(ReadBuffer & istr) const DataTypePtr & type = index_sample_block.getByPosition(i).type; auto serialization = type->getDefaultSerialization(); - if (!type->isNullable()) + switch (version) { - serialization->deserializeBinary(min_val, istr); - serialization->deserializeBinary(max_val, istr); - } - else - { - /// NOTE: that this serialization differs from - /// IMergeTreeDataPart::MinMaxIndex::load() due to preserve - /// backward compatibility. - bool is_null; - readBinary(is_null, istr); - if (!is_null) - { + case 1: + if (!type->isNullable()) + { + serialization->deserializeBinary(min_val, istr); + serialization->deserializeBinary(max_val, istr); + } + else + { + /// NOTE: that this serialization differs from + /// IMergeTreeDataPart::MinMaxIndex::load() to preserve + /// backward compatibility. + /// + /// But this is deprecated format, so this is OK. + + bool is_null; + readBinary(is_null, istr); + if (!is_null) + { + serialization->deserializeBinary(min_val, istr); + serialization->deserializeBinary(max_val, istr); + } + else + { + min_val = Null(); + max_val = Null(); + } + } + break; + + /// New format with proper Nullable support for values that includes Null values + case 2: serialization->deserializeBinary(min_val, istr); serialization->deserializeBinary(max_val, istr); - } - else - { - min_val = Null(); - max_val = Null(); - } + + // NULL_LAST + if (min_val.isNull()) + min_val = PositiveInfinity(); + if (max_val.isNull()) + max_val = PositiveInfinity(); + + break; + default: + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown index version {}.", version); } + hyperrectangle.emplace_back(min_val, true, max_val, true); } } @@ -203,6 +211,15 @@ bool MergeTreeIndexMinMax::mayBenefitFromIndexForIn(const ASTPtr & node) const return false; } +MergeTreeIndexFormat MergeTreeIndexMinMax::getDeserializedFormat(const DiskPtr disk, const std::string & relative_path_prefix) const +{ + if (disk->exists(relative_path_prefix + ".idx2")) + return {2, ".idx2"}; + else if (disk->exists(relative_path_prefix + ".idx")) + return {1, ".idx"}; + return {0 /* unknown */, ""}; +} + MergeTreeIndexPtr minmaxIndexCreator( const IndexDescription & index) { diff --git a/src/Storages/MergeTree/MergeTreeIndexMinMax.h b/src/Storages/MergeTree/MergeTreeIndexMinMax.h index 97b9b874484..0e05e25fb36 100644 --- a/src/Storages/MergeTree/MergeTreeIndexMinMax.h +++ b/src/Storages/MergeTree/MergeTreeIndexMinMax.h @@ -21,7 +21,7 @@ struct MergeTreeIndexGranuleMinMax final : public IMergeTreeIndexGranule ~MergeTreeIndexGranuleMinMax() override = default; void serializeBinary(WriteBuffer & ostr) const override; - void deserializeBinary(ReadBuffer & istr) override; + void deserializeBinary(ReadBuffer & istr, MergeTreeIndexVersion version) override; bool empty() const override { return hyperrectangle.empty(); } @@ -81,6 +81,9 @@ public: const SelectQueryInfo & query, ContextPtr context) const override; bool mayBenefitFromIndexForIn(const ASTPtr & node) const override; + + const char* getSerializedFileExtension() const override { return ".idx2"; } + MergeTreeIndexFormat getDeserializedFormat(const DiskPtr disk, const std::string & path_prefix) const override; }; } diff --git a/src/Storages/MergeTree/MergeTreeIndexReader.cpp b/src/Storages/MergeTree/MergeTreeIndexReader.cpp index eaba247009b..0a0f2511914 100644 --- a/src/Storages/MergeTree/MergeTreeIndexReader.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexReader.cpp @@ -1,5 +1,29 @@ #include +namespace +{ + +using namespace DB; + +std::unique_ptr makeIndexReader( + const std::string & extension, + MergeTreeIndexPtr index, + MergeTreeData::DataPartPtr part, + size_t marks_count, + const MarkRanges & all_mark_ranges, + MergeTreeReaderSettings settings) +{ + return std::make_unique( + part->volume->getDisk(), + part->getFullRelativePath() + index->getFileName(), extension, marks_count, + all_mark_ranges, + std::move(settings), nullptr, nullptr, + part->getFileSizeOrZero(index->getFileName() + extension), + &part->index_granularity_info, + ReadBufferFromFileBase::ProfileCallback{}, CLOCK_MONOTONIC_COARSE); +} + +} namespace DB { @@ -7,27 +31,28 @@ namespace DB MergeTreeIndexReader::MergeTreeIndexReader( MergeTreeIndexPtr index_, MergeTreeData::DataPartPtr part_, size_t marks_count_, const MarkRanges & all_mark_ranges_, MergeTreeReaderSettings settings) - : index(index_), stream( - part_->volume->getDisk(), - part_->getFullRelativePath() + index->getFileName(), ".idx", marks_count_, - all_mark_ranges_, - std::move(settings), nullptr, nullptr, - part_->getFileSizeOrZero(index->getFileName() + ".idx"), - &part_->index_granularity_info, - ReadBufferFromFileBase::ProfileCallback{}, CLOCK_MONOTONIC_COARSE) + : index(index_) { - stream.seekToStart(); + const std::string & path_prefix = part_->getFullRelativePath() + index->getFileName(); + auto index_format = index->getDeserializedFormat(part_->volume->getDisk(), path_prefix); + + stream = makeIndexReader(index_format.extension, index_, part_, marks_count_, all_mark_ranges_, std::move(settings)); + version = index_format.version; + + stream->seekToStart(); } +MergeTreeIndexReader::~MergeTreeIndexReader() = default; + void MergeTreeIndexReader::seek(size_t mark) { - stream.seekToMark(mark); + stream->seekToMark(mark); } MergeTreeIndexGranulePtr MergeTreeIndexReader::read() { auto granule = index->createIndexGranule(); - granule->deserializeBinary(*stream.data_buffer); + granule->deserializeBinary(*stream->data_buffer, version); return granule; } diff --git a/src/Storages/MergeTree/MergeTreeIndexReader.h b/src/Storages/MergeTree/MergeTreeIndexReader.h index 68d681458be..4facd43c175 100644 --- a/src/Storages/MergeTree/MergeTreeIndexReader.h +++ b/src/Storages/MergeTree/MergeTreeIndexReader.h @@ -1,5 +1,6 @@ #pragma once +#include #include #include #include @@ -16,6 +17,7 @@ public: size_t marks_count_, const MarkRanges & all_mark_ranges_, MergeTreeReaderSettings settings); + ~MergeTreeIndexReader(); void seek(size_t mark); @@ -23,7 +25,8 @@ public: private: MergeTreeIndexPtr index; - MergeTreeReaderStream stream; + std::unique_ptr stream; + uint8_t version = 0; }; } diff --git a/src/Storages/MergeTree/MergeTreeIndexSet.cpp b/src/Storages/MergeTree/MergeTreeIndexSet.cpp index 6cee80983d6..024b87c9a3e 100644 --- a/src/Storages/MergeTree/MergeTreeIndexSet.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexSet.cpp @@ -48,8 +48,7 @@ MergeTreeIndexGranuleSet::MergeTreeIndexGranuleSet( void MergeTreeIndexGranuleSet::serializeBinary(WriteBuffer & ostr) const { if (empty()) - throw Exception( - "Attempt to write empty set index " + backQuote(index_name), ErrorCodes::LOGICAL_ERROR); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Attempt to write empty set index {}.", backQuote(index_name)); const auto & size_type = DataTypePtr(std::make_shared()); auto size_serialization = size_type->getDefaultSerialization(); @@ -80,8 +79,11 @@ void MergeTreeIndexGranuleSet::serializeBinary(WriteBuffer & ostr) const } } -void MergeTreeIndexGranuleSet::deserializeBinary(ReadBuffer & istr) +void MergeTreeIndexGranuleSet::deserializeBinary(ReadBuffer & istr, MergeTreeIndexVersion version) { + if (version != 1) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown index version {}.", version); + block.clear(); Field field_rows; diff --git a/src/Storages/MergeTree/MergeTreeIndexSet.h b/src/Storages/MergeTree/MergeTreeIndexSet.h index 28afe4f714d..23b336d274b 100644 --- a/src/Storages/MergeTree/MergeTreeIndexSet.h +++ b/src/Storages/MergeTree/MergeTreeIndexSet.h @@ -28,7 +28,7 @@ struct MergeTreeIndexGranuleSet final : public IMergeTreeIndexGranule MutableColumns && columns_); void serializeBinary(WriteBuffer & ostr) const override; - void deserializeBinary(ReadBuffer & istr) override; + void deserializeBinary(ReadBuffer & istr, MergeTreeIndexVersion version) override; size_t size() const { return block.rows(); } bool empty() const override { return !size(); } diff --git a/src/Storages/MergeTree/MergeTreeIndices.h b/src/Storages/MergeTree/MergeTreeIndices.h index 674daeb480d..557af891b74 100644 --- a/src/Storages/MergeTree/MergeTreeIndices.h +++ b/src/Storages/MergeTree/MergeTreeIndices.h @@ -4,6 +4,7 @@ #include #include #include +#include #include #include #include @@ -17,13 +18,37 @@ constexpr auto INDEX_FILE_PREFIX = "skp_idx_"; namespace DB { +using MergeTreeIndexVersion = uint8_t; +struct MergeTreeIndexFormat +{ + MergeTreeIndexVersion version; + const char* extension; + + operator bool() const { return version != 0; } +}; + /// Stores some info about a single block of data. struct IMergeTreeIndexGranule { virtual ~IMergeTreeIndexGranule() = default; + /// Serialize always last version. virtual void serializeBinary(WriteBuffer & ostr) const = 0; - virtual void deserializeBinary(ReadBuffer & istr) = 0; + + /// Version of the index to deserialize: + /// + /// - 2 -- minmax index for proper Nullable support, + /// - 1 -- everything else. + /// + /// Implementation is responsible for version check, + /// and throw LOGICAL_ERROR in case of unsupported version. + /// + /// See also: + /// - IMergeTreeIndex::getSerializedFileExtension() + /// - IMergeTreeIndex::getDeserializedFormat() + /// - MergeTreeDataMergerMutator::collectFilesToSkip() + /// - MergeTreeDataMergerMutator::collectFilesForRenames() + virtual void deserializeBinary(ReadBuffer & istr, MergeTreeIndexVersion version) = 0; virtual bool empty() const = 0; }; @@ -73,9 +98,26 @@ struct IMergeTreeIndex virtual ~IMergeTreeIndex() = default; - /// gets filename without extension + /// Returns filename without extension. String getFileName() const { return INDEX_FILE_PREFIX + index.name; } + /// Returns extension for serialization. + /// Reimplement if you want new index format. + /// + /// NOTE: In case getSerializedFileExtension() is reimplemented, + /// getDeserializedFormat() should be reimplemented too, + /// and check all previous extensions too + /// (to avoid breaking backward compatibility). + virtual const char* getSerializedFileExtension() const { return ".idx"; } + + /// Returns extension for deserialization. + /// + /// Return pair. + virtual MergeTreeIndexFormat getDeserializedFormat(const DiskPtr, const std::string & /* relative_path_prefix */) const + { + return {1, ".idx"}; + } + /// Checks whether the column is in data skipping index. virtual bool mayBenefitFromIndexForIn(const ASTPtr & node) const = 0; diff --git a/tests/queries/0_stateless/01410_nullable_key_and_index.sql b/tests/queries/0_stateless/01410_nullable_key_and_index.sql index 24ddb226c16..ba473b5c29a 100644 --- a/tests/queries/0_stateless/01410_nullable_key_and_index.sql +++ b/tests/queries/0_stateless/01410_nullable_key_and_index.sql @@ -49,15 +49,11 @@ SET force_primary_key = 0; SELECT * FROM nullable_minmax_index ORDER BY k; SET max_rows_to_read = 6; SELECT * FROM nullable_minmax_index WHERE v IS NULL; --- NOTE: granuals with Null values cannot be filtred in data skipping indexes, --- due to backward compatibility -SET max_rows_to_read = 0; +SET max_rows_to_read = 8; SELECT * FROM nullable_minmax_index WHERE v IS NOT NULL; SET max_rows_to_read = 6; SELECT * FROM nullable_minmax_index WHERE v > 2; --- NOTE: granuals with Null values cannot be filtred in data skipping indexes, --- due to backward compatibility -SET max_rows_to_read = 0; +SET max_rows_to_read = 4; SELECT * FROM nullable_minmax_index WHERE v <= 2; DROP TABLE nullable_key; From 0677a6c6cb463aec8ece376e8d53aa02d1442466 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 2 Aug 2021 21:07:07 +0300 Subject: [PATCH 285/599] Enable distributed_push_down_limit by default --- src/Core/Settings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 2ee7ad283b4..74efcea41db 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -123,7 +123,7 @@ class IColumn; \ M(UInt64, parallel_distributed_insert_select, 0, "Process distributed INSERT SELECT query in the same cluster on local tables on every shard, if 1 SELECT is executed on each shard, if 2 SELECT and INSERT is executed on each shard", 0) \ M(UInt64, distributed_group_by_no_merge, 0, "If 1, Do not merge aggregation states from different servers for distributed queries (shards will process query up to the Complete stage, initiator just proxies the data from the shards). If 2 the initiator will apply ORDER BY and LIMIT stages (it is not in case when shard process query up to the Complete stage)", 0) \ - M(UInt64, distributed_push_down_limit, 0, "If 1, LIMIT will be applied on each shard separatelly. Usually you don't need to use it, since this will be done automatically if it is possible, i.e. for simple query SELECT FROM LIMIT.", 0) \ + M(UInt64, distributed_push_down_limit, 1, "If 1, LIMIT will be applied on each shard separatelly. Usually you don't need to use it, since this will be done automatically if it is possible, i.e. for simple query SELECT FROM LIMIT.", 0) \ M(Bool, optimize_distributed_group_by_sharding_key, false, "Optimize GROUP BY sharding_key queries (by avoiding costly aggregation on the initiator server).", 0) \ M(UInt64, optimize_skip_unused_shards_limit, 1000, "Limit for number of sharding key values, turns off optimize_skip_unused_shards if the limit is reached", 0) \ M(Bool, optimize_skip_unused_shards, false, "Assumes that data is distributed by sharding_key. Optimization to skip unused shards if SELECT query filters by sharding_key.", 0) \ From 940a537c66844abfca127b6dd347909341534a93 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 6 Aug 2021 07:49:45 +0300 Subject: [PATCH 286/599] Fix 01710_projection_fetch flakiness --- tests/queries/0_stateless/01710_projection_fetch.sql | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/01710_projection_fetch.sql b/tests/queries/0_stateless/01710_projection_fetch.sql index 7e4f6cc1d9a..09750a72289 100644 --- a/tests/queries/0_stateless/01710_projection_fetch.sql +++ b/tests/queries/0_stateless/01710_projection_fetch.sql @@ -18,23 +18,23 @@ select * from tp_2 order by x; -- test projection creation, materialization, clear and drop alter table tp_1 add projection pp (select x, count() group by x); system sync replica tp_2; -select count() from system.projection_parts where table = 'tp_2' and name = 'pp' and active; +select count() from system.projection_parts where database = currentDatabase() and table = 'tp_2' and name = 'pp' and active; show create table tp_2; -- all other three operations are mutations set mutations_sync = 2; alter table tp_1 materialize projection pp; -select count() from system.projection_parts where table = 'tp_2' and name = 'pp' and active; +select count() from system.projection_parts where database = currentDatabase() and table = 'tp_2' and name = 'pp' and active; show create table tp_2; alter table tp_1 clear projection pp; system sync replica tp_2; -select * from system.projection_parts where table = 'tp_2' and name = 'pp' and active; +select * from system.projection_parts where database = currentDatabase() and table = 'tp_2' and name = 'pp' and active; show create table tp_2; alter table tp_1 drop projection pp; system sync replica tp_2; -select * from system.projection_parts where table = 'tp_2' and name = 'pp' and active; +select * from system.projection_parts where database = currentDatabase() and table = 'tp_2' and name = 'pp' and active; show create table tp_2; drop table if exists tp_1; From 2bfdf0d5a1e7652f07edac5a13be5c7b13151500 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 6 Aug 2021 07:49:46 +0300 Subject: [PATCH 287/599] check-style: check system.projection_parts for database filter --- utils/check-style/check-style | 14 +++++++++++--- 1 file changed, 11 insertions(+), 3 deletions(-) diff --git a/utils/check-style/check-style b/utils/check-style/check-style index 05b8a1485e4..c0f8dbfc096 100755 --- a/utils/check-style/check-style +++ b/utils/check-style/check-style @@ -99,13 +99,21 @@ for test_case in "${tests_with_query_log[@]}"; do } || echo "Queries to system.query_log/system.query_thread_log does not have current_database = currentDatabase() condition in $test_case" done -# Queries to system.tables/system.parts/system.detached_parts/system.parts_columns/system.columns should have database = currentDatabase() condition +# Queries to: +# - system.tables +# - system.parts +# - system.detached_parts +# - system.parts_columns +# - system.columns +# - system.projection_parts +# should have database = currentDatabase() condition +# # NOTE: it is not that accuate, but at least something. tests_with_database_column=( $( find $ROOT_PATH/tests/queries -iname '*.sql' -or -iname '*.sh' -or -iname '*.py' -or -iname '*.j2' | grep -vP $EXCLUDE_DIRS | grep -v -x -e $ROOT_PATH/tests/queries/query_test.py | - xargs grep --with-filename -e system.tables -e system.parts -e system.detached_parts -e system.parts_columns -e system.columns | cut -d: -f1 | sort -u + xargs grep --with-filename -e system.tables -e system.parts -e system.detached_parts -e system.parts_columns -e system.columns -e system.projection_parts | cut -d: -f1 | sort -u ) ) for test_case in "${tests_with_database_column[@]}"; do grep -qE database.*currentDatabase "$test_case" || { @@ -114,7 +122,7 @@ for test_case in "${tests_with_database_column[@]}"; do # explicit database grep -qE "database[ ]*=[ ]*'" "$test_case" } || { - echo "Queries to system.tables/system.parts/system.detached_parts/system.parts_columns/system.columns does not have database = currentDatabase()/\$CLICKHOUSE_DATABASE condition in $test_case" + echo "Queries to system.tables/system.parts/system.detached_parts/system.parts_columns/system.columns/system.projection_parts does not have database = currentDatabase()/\$CLICKHOUSE_DATABASE condition in $test_case" } done From 5abe33e1a38cd44e585ec106468edc475d7170cb Mon Sep 17 00:00:00 2001 From: kssenii Date: Sun, 8 Aug 2021 20:12:12 +0300 Subject: [PATCH 288/599] Better --- .../InterpreterIntersectOrExcept.cpp | 45 ++++++++++------- .../InterpreterIntersectOrExcept.h | 18 ++++--- .../QueryPlan/IntersectOrExceptStep.cpp | 50 +++++++++++++++++-- .../QueryPlan/IntersectOrExceptStep.h | 6 ++- .../Transforms/IntersectOrExceptTransform.cpp | 40 +++++++-------- .../Transforms/IntersectOrExceptTransform.h | 30 +++++------ 6 files changed, 119 insertions(+), 70 deletions(-) diff --git a/src/Interpreters/InterpreterIntersectOrExcept.cpp b/src/Interpreters/InterpreterIntersectOrExcept.cpp index c85bd29e16f..c85c39824d8 100644 --- a/src/Interpreters/InterpreterIntersectOrExcept.cpp +++ b/src/Interpreters/InterpreterIntersectOrExcept.cpp @@ -8,6 +8,8 @@ #include #include #include +#include + namespace DB { @@ -17,16 +19,20 @@ namespace ErrorCodes extern const int INTERSECT_OR_EXCEPT_RESULT_STRUCTURES_MISMATCH; } -InterpreterIntersectOrExcept::InterpreterIntersectOrExcept(const ASTPtr & query_ptr_, ContextPtr context_) - : query_ptr(query_ptr_), context(Context::createCopy(context_)) +InterpreterIntersectOrExcept::InterpreterIntersectOrExcept(const ASTPtr & query_ptr, ContextPtr context_) + : context(Context::createCopy(context_)) + , is_except(query_ptr->as()->is_except) { ASTIntersectOrExcept * ast = query_ptr->as(); + size_t num_children = ast->children.size(); + if (!num_children) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: no children in ASTIntersectOrExceptQuery"); + nested_interpreters.resize(num_children); + for (size_t i = 0; i < num_children; ++i) - { nested_interpreters[i] = buildCurrentChildInterpreter(ast->children[i]); - } Blocks headers(num_children); for (size_t query_num = 0; query_num < num_children; ++query_num) @@ -35,8 +41,7 @@ InterpreterIntersectOrExcept::InterpreterIntersectOrExcept(const ASTPtr & query_ result_header = getCommonHeader(headers); } - -Block InterpreterIntersectOrExcept::getCommonHeader(const Blocks & headers) +Block InterpreterIntersectOrExcept::getCommonHeader(const Blocks & headers) const { size_t num_selects = headers.size(); Block common_header = headers.front(); @@ -45,16 +50,12 @@ Block InterpreterIntersectOrExcept::getCommonHeader(const Blocks & headers) for (size_t query_num = 1; query_num < num_selects; ++query_num) { if (headers[query_num].columns() != num_columns) - throw Exception( - "Different number of columns in " - + toString(query_ptr->as()->is_except ? "EXCEPT" : "INTERSECT") - + " elements:\n" + common_header.dumpNames() + "\nand\n" - + headers[query_num].dumpNames() + "\n", - ErrorCodes::INTERSECT_OR_EXCEPT_RESULT_STRUCTURES_MISMATCH); + throw Exception(ErrorCodes::INTERSECT_OR_EXCEPT_RESULT_STRUCTURES_MISMATCH, + "Different number of columns in {} elements:\n {} \nand\n {}", + getName(), common_header.dumpNames(), headers[query_num].dumpNames()); } std::vector columns(num_selects); - for (size_t column_num = 0; column_num < num_columns; ++column_num) { for (size_t i = 0; i < num_selects; ++i) @@ -67,7 +68,6 @@ Block InterpreterIntersectOrExcept::getCommonHeader(const Blocks & headers) return common_header; } - std::unique_ptr InterpreterIntersectOrExcept::buildCurrentChildInterpreter(const ASTPtr & ast_ptr_) { @@ -80,7 +80,6 @@ InterpreterIntersectOrExcept::buildCurrentChildInterpreter(const ASTPtr & ast_pt void InterpreterIntersectOrExcept::buildQueryPlan(QueryPlan & query_plan) { size_t num_plans = nested_interpreters.size(); - std::vector> plans(num_plans); DataStreams data_streams(num_plans); @@ -88,12 +87,23 @@ void InterpreterIntersectOrExcept::buildQueryPlan(QueryPlan & query_plan) { plans[i] = std::make_unique(); nested_interpreters[i]->buildQueryPlan(*plans[i]); + + if (!blocksHaveEqualStructure(plans[i]->getCurrentDataStream().header, result_header)) + { + auto actions_dag = ActionsDAG::makeConvertingActions( + plans[i]->getCurrentDataStream().header.getColumnsWithTypeAndName(), + result_header.getColumnsWithTypeAndName(), + ActionsDAG::MatchColumnsMode::Position); + auto converting_step = std::make_unique(plans[i]->getCurrentDataStream(), std::move(actions_dag)); + converting_step->setStepDescription("Conversion before UNION"); + plans[i]->addStep(std::move(converting_step)); + } + data_streams[i] = plans[i]->getCurrentDataStream(); } auto max_threads = context->getSettingsRef().max_threads; - auto step = std::make_unique( - query_ptr->as()->is_except, std::move(data_streams), result_header, max_threads); + auto step = std::make_unique(is_except, std::move(data_streams), max_threads); query_plan.unitePlans(std::move(step), std::move(plans)); } @@ -113,4 +123,5 @@ BlockIO InterpreterIntersectOrExcept::execute() return res; } + } diff --git a/src/Interpreters/InterpreterIntersectOrExcept.h b/src/Interpreters/InterpreterIntersectOrExcept.h index 0069dc02f1d..34a58c0c05a 100644 --- a/src/Interpreters/InterpreterIntersectOrExcept.h +++ b/src/Interpreters/InterpreterIntersectOrExcept.h @@ -16,20 +16,22 @@ class InterpreterIntersectOrExcept : public IInterpreter public: InterpreterIntersectOrExcept(const ASTPtr & query_ptr_, ContextPtr context_); - /// Builds QueryPlan for current query. - virtual void buildQueryPlan(QueryPlan & query_plan); - BlockIO execute() override; private: - ASTPtr query_ptr; - ContextPtr context; - Block result_header; - std::vector> nested_interpreters; - Block getCommonHeader(const Blocks & headers); + String getName() const { return is_except ? "EXCEPT" : "INTERSECT"; } + + Block getCommonHeader(const Blocks & headers) const; std::unique_ptr buildCurrentChildInterpreter(const ASTPtr & ast_ptr_); + + void buildQueryPlan(QueryPlan & query_plan); + + ContextPtr context; + bool is_except; + Block result_header; + std::vector> nested_interpreters; }; } diff --git a/src/Processors/QueryPlan/IntersectOrExceptStep.cpp b/src/Processors/QueryPlan/IntersectOrExceptStep.cpp index 28f34bda5db..f04885f4640 100644 --- a/src/Processors/QueryPlan/IntersectOrExceptStep.cpp +++ b/src/Processors/QueryPlan/IntersectOrExceptStep.cpp @@ -1,18 +1,36 @@ #include +#include #include #include #include +#include #include #include + namespace DB { -IntersectOrExceptStep::IntersectOrExceptStep(bool is_except_, DataStreams input_streams_, Block result_header, size_t max_threads_) - : is_except(is_except_), header(std::move(result_header)), max_threads(max_threads_) +Block IntersectOrExceptStep::checkHeaders(const DataStreams & input_streams_) const +{ + if (input_streams_.empty()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot perform {} on empty set of query plan steps", getName()); + + Block res = input_streams_.front().header; + for (const auto & stream : input_streams_) + assertBlocksHaveEqualStructure(stream.header, res, "IntersectOrExceptStep"); + + return res; +} + +IntersectOrExceptStep::IntersectOrExceptStep(bool is_except_, DataStreams input_streams_, size_t max_threads_) + : is_except(is_except_), header(checkHeaders(input_streams_)), max_threads(max_threads_) { input_streams = std::move(input_streams_); - output_stream = DataStream{.header = header}; + if (input_streams.size() == 1) + output_stream = input_streams.front(); + else + output_stream = DataStream{.header = header}; } QueryPipelinePtr IntersectOrExceptStep::updatePipeline(QueryPipelines pipelines, const BuildQueryPipelineSettings &) @@ -20,8 +38,30 @@ QueryPipelinePtr IntersectOrExceptStep::updatePipeline(QueryPipelines pipelines, auto pipeline = std::make_unique(); QueryPipelineProcessorsCollector collector(*pipeline, this); - pipelines[0]->addTransform(std::make_shared(header, pipelines[0]->getNumStreams(), 1)); - pipelines[1]->addTransform(std::make_shared(header, pipelines[1]->getNumStreams(), 1)); + if (pipelines.empty()) + { + pipeline->init(Pipe(std::make_shared(output_stream->header))); + processors = collector.detachProcessors(); + return pipeline; + } + + for (auto & cur_pipeline : pipelines) + { + /// Just in case. + if (!isCompatibleHeader(cur_pipeline->getHeader(), getOutputStream().header)) + { + auto converting_dag = ActionsDAG::makeConvertingActions( + cur_pipeline->getHeader().getColumnsWithTypeAndName(), + getOutputStream().header.getColumnsWithTypeAndName(), + ActionsDAG::MatchColumnsMode::Name); + + auto converting_actions = std::make_shared(std::move(converting_dag)); + cur_pipeline->addSimpleTransform([&](const Block & cur_header) + { + return std::make_shared(cur_header, converting_actions); + }); + } + } *pipeline = QueryPipeline::unitePipelines(std::move(pipelines), max_threads); pipeline->addTransform(std::make_shared(is_except, header)); diff --git a/src/Processors/QueryPlan/IntersectOrExceptStep.h b/src/Processors/QueryPlan/IntersectOrExceptStep.h index d2b515bb1c4..7938a9adad5 100644 --- a/src/Processors/QueryPlan/IntersectOrExceptStep.h +++ b/src/Processors/QueryPlan/IntersectOrExceptStep.h @@ -8,14 +8,17 @@ class IntersectOrExceptStep : public IQueryPlanStep { public: /// max_threads is used to limit the number of threads for result pipeline. - IntersectOrExceptStep(bool is_except_, DataStreams input_streams_, Block result_header, size_t max_threads_ = 0); + IntersectOrExceptStep(bool is_except_, DataStreams input_streams_, size_t max_threads_ = 0); String getName() const override { return is_except ? "Except" : "Intersect"; } QueryPipelinePtr updatePipeline(QueryPipelines pipelines, const BuildQueryPipelineSettings & settings) override; void describePipeline(FormatSettings & settings) const override; + private: + Block checkHeaders(const DataStreams & input_streams_) const; + bool is_except; Block header; size_t max_threads; @@ -23,4 +26,3 @@ private: }; } - diff --git a/src/Processors/Transforms/IntersectOrExceptTransform.cpp b/src/Processors/Transforms/IntersectOrExceptTransform.cpp index 199498bf762..e5e8ff705c8 100644 --- a/src/Processors/Transforms/IntersectOrExceptTransform.cpp +++ b/src/Processors/Transforms/IntersectOrExceptTransform.cpp @@ -1,9 +1,12 @@ #include + namespace DB { + IntersectOrExceptTransform::IntersectOrExceptTransform(bool is_except_, const Block & header_) - : IProcessor(InputPorts(2, header_), {header_}), is_except(is_except_), output(outputs.front()) + : IProcessor(InputPorts(2, header_), {header_}) + , is_except(is_except_) { const Names & columns = header_.getNames(); size_t num_columns = columns.empty() ? header_.columns() : columns.size(); @@ -11,18 +14,17 @@ IntersectOrExceptTransform::IntersectOrExceptTransform(bool is_except_, const Bl key_columns_pos.reserve(columns.size()); for (size_t i = 0; i < num_columns; ++i) { - auto pos = columns.empty() ? i : header_.getPositionByName(columns[i]); - - const auto & col = header_.getByPosition(pos).column; - - if (!(col && isColumnConst(*col))) - key_columns_pos.emplace_back(pos); + auto pos = columns.empty() ? i + : header_.getPositionByName(columns[i]); + key_columns_pos.emplace_back(pos); } } + IntersectOrExceptTransform::Status IntersectOrExceptTransform::prepare() { - /// Check can output. + auto & output = outputs.front(); + if (output.isFinished()) { for (auto & in : inputs) @@ -32,14 +34,8 @@ IntersectOrExceptTransform::Status IntersectOrExceptTransform::prepare() if (!output.canPush()) { - if (inputs.front().isFinished()) - { - inputs.back().setNotNeeded(); - } - else - { - inputs.front().setNotNeeded(); - } + for (auto & input : inputs) + input.setNotNeeded(); return Status::PortFull; } @@ -74,10 +70,9 @@ IntersectOrExceptTransform::Status IntersectOrExceptTransform::prepare() if (!has_input) { input.setNeeded(); + if (!input.hasData()) - { return Status::NeedData; - } current_input_chunk = input.pull(); has_input = true; @@ -86,6 +81,7 @@ IntersectOrExceptTransform::Status IntersectOrExceptTransform::prepare() return Status::Ready; } + void IntersectOrExceptTransform::work() { if (!finished_second_input) @@ -101,17 +97,17 @@ void IntersectOrExceptTransform::work() has_input = false; } + template void IntersectOrExceptTransform::addToSet(Method & method, const ColumnRawPtrs & columns, size_t rows, SetVariants & variants) const { typename Method::State state(columns, key_sizes, nullptr); for (size_t i = 0; i < rows; ++i) - { state.emplaceKey(method.data, i, variants.string_pool); - } } + template size_t IntersectOrExceptTransform::buildFilter( Method & method, const ColumnRawPtrs & columns, IColumn::Filter & filter, size_t rows, SetVariants & variants) const @@ -129,6 +125,7 @@ size_t IntersectOrExceptTransform::buildFilter( return new_rows_num; } + void IntersectOrExceptTransform::accumulate(Chunk chunk) { auto num_rows = chunk.getNumRows(); @@ -136,6 +133,7 @@ void IntersectOrExceptTransform::accumulate(Chunk chunk) ColumnRawPtrs column_ptrs; column_ptrs.reserve(key_columns_pos.size()); + for (auto pos : key_columns_pos) column_ptrs.emplace_back(columns[pos].get()); @@ -155,6 +153,7 @@ void IntersectOrExceptTransform::accumulate(Chunk chunk) } } + void IntersectOrExceptTransform::filter(Chunk & chunk) { auto num_rows = chunk.getNumRows(); @@ -162,6 +161,7 @@ void IntersectOrExceptTransform::filter(Chunk & chunk) ColumnRawPtrs column_ptrs; column_ptrs.reserve(key_columns_pos.size()); + for (auto pos : key_columns_pos) column_ptrs.emplace_back(columns[pos].get()); diff --git a/src/Processors/Transforms/IntersectOrExceptTransform.h b/src/Processors/Transforms/IntersectOrExceptTransform.h index ebe73fdeb26..3c2b9581d6d 100644 --- a/src/Processors/Transforms/IntersectOrExceptTransform.h +++ b/src/Processors/Transforms/IntersectOrExceptTransform.h @@ -12,17 +12,18 @@ class IntersectOrExceptTransform : public IProcessor public: IntersectOrExceptTransform(bool is_except_, const Block & header_); - Status prepare() override; - void work() override; - String getName() const override { return is_except ? "Except" : "Intersect"; } +protected: + Status prepare() override; + + void work() override; + private: + bool is_except; bool push_empty_chunk = false; Chunk empty_chunk; - - bool is_except; ColumnNumbers key_columns_pos; SetVariants data; Sizes key_sizes; @@ -30,24 +31,17 @@ private: Chunk current_output_chunk; bool finished_second_input = false; bool has_input = false; - OutputPort & output; void accumulate(Chunk chunk); + void filter(Chunk & chunk); - template - void addToSet( - Method & method, - const ColumnRawPtrs & key_columns, - size_t rows, - SetVariants & variants) const; template - size_t buildFilter( - Method & method, - const ColumnRawPtrs & columns, - IColumn::Filter & filter, - size_t rows, - SetVariants & variants) const; + void addToSet(Method & method, const ColumnRawPtrs & key_columns, size_t rows, SetVariants & variants) const; + + template + size_t buildFilter(Method & method, const ColumnRawPtrs & columns, + IColumn::Filter & filter, size_t rows, SetVariants & variants) const; }; } From 0f67acf6482ae1b5d736a01144255c8daf60e7eb Mon Sep 17 00:00:00 2001 From: kssenii Date: Sun, 8 Aug 2021 20:16:22 +0300 Subject: [PATCH 289/599] Add test --- ...02004_intersect_except_operators.reference | 25 +++++++++++++++++++ .../02004_intersect_except_operators.sql | 8 ++++++ 2 files changed, 33 insertions(+) create mode 100644 tests/queries/0_stateless/02004_intersect_except_operators.reference create mode 100644 tests/queries/0_stateless/02004_intersect_except_operators.sql diff --git a/tests/queries/0_stateless/02004_intersect_except_operators.reference b/tests/queries/0_stateless/02004_intersect_except_operators.reference new file mode 100644 index 00000000000..763a5872cce --- /dev/null +++ b/tests/queries/0_stateless/02004_intersect_except_operators.reference @@ -0,0 +1,25 @@ +-- { echo } +select 1 intersect select 1; +1 +select 2 intersect select 1; +select 1 except select 1; +select 2 except select 1; +2 +select number from numbers(5, 5) intersect select number from numbers(20); +5 +6 +7 +8 +9 +select number from numbers(10) except select number from numbers(5); +5 +6 +7 +8 +9 +select number, number+10 from numbers(12) except select number+5, number+15 from numbers(10); +0 10 +1 11 +2 12 +3 13 +4 14 diff --git a/tests/queries/0_stateless/02004_intersect_except_operators.sql b/tests/queries/0_stateless/02004_intersect_except_operators.sql new file mode 100644 index 00000000000..d0416fd899e --- /dev/null +++ b/tests/queries/0_stateless/02004_intersect_except_operators.sql @@ -0,0 +1,8 @@ +-- { echo } +select 1 intersect select 1; +select 2 intersect select 1; +select 1 except select 1; +select 2 except select 1; +select number from numbers(5, 5) intersect select number from numbers(20); +select number from numbers(10) except select number from numbers(5); +select number, number+10 from numbers(12) except select number+5, number+15 from numbers(10); From e45a09e2f542b9aa7ec512493febe805e4b36449 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 8 Aug 2021 21:09:39 +0300 Subject: [PATCH 290/599] Fix wrong PR #27184 --- src/Core/Settings.h | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 2ee7ad283b4..0a3c50d8612 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -499,6 +499,7 @@ class IColumn; M(Bool, enable_debug_queries, false, "Obsolete setting, does nothing.", 0) \ M(Bool, allow_experimental_database_atomic, true, "Obsolete setting, does nothing.", 0) \ M(Bool, allow_experimental_bigint_types, true, "Obsolete setting, does nothing.", 0) \ + M(Bool, allow_experimental_window_functions, true, "Obsolete setting, does nothing.", 0) \ M(HandleKafkaErrorMode, handle_kafka_error_mode, HandleKafkaErrorMode::DEFAULT, "Obsolete setting, does nothing.", 0) \ M(Bool, database_replicated_ddl_output, true, "Obsolete setting, does nothing.", 0) \ /** The section above is for obsolete settings. Do not add anything there. */ From 7631242a3bf66a33a73d923cc12c596619dee78d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 8 Aug 2021 22:14:01 +0300 Subject: [PATCH 291/599] Add UK Property Paid data --- .../getting-started/example-datasets/index.md | 1 + .../example-datasets/uk-price-paid.md | 325 ++++++++++++++++++ 2 files changed, 326 insertions(+) create mode 100644 docs/en/getting-started/example-datasets/uk-price-paid.md diff --git a/docs/en/getting-started/example-datasets/index.md b/docs/en/getting-started/example-datasets/index.md index 53007c33306..669bdf31a61 100644 --- a/docs/en/getting-started/example-datasets/index.md +++ b/docs/en/getting-started/example-datasets/index.md @@ -15,6 +15,7 @@ The list of documented datasets: - [Recipes](../../getting-started/example-datasets/recipes.md) - [OnTime](../../getting-started/example-datasets/ontime.md) - [New York Taxi Data](../../getting-started/example-datasets/nyc-taxi.md) +- [UK Property Price Paid](../../getting-started/example-datasets/uk-price-paid.md) - [Star Schema Benchmark](../../getting-started/example-datasets/star-schema.md) - [WikiStat](../../getting-started/example-datasets/wikistat.md) - [Terabyte of Click Logs from Criteo](../../getting-started/example-datasets/criteo.md) diff --git a/docs/en/getting-started/example-datasets/uk-price-paid.md b/docs/en/getting-started/example-datasets/uk-price-paid.md new file mode 100644 index 00000000000..cb4baa55553 --- /dev/null +++ b/docs/en/getting-started/example-datasets/uk-price-paid.md @@ -0,0 +1,325 @@ +--- +toc_priority: 20 +toc_title: UK Property Price Paid +--- + +# UK Property Price Paid + +The dataset contains data about prices paid for real-estate property in England and Wales. The data is available since year 1995. +The size of the dataset in uncompressed form is about 4 GiB and it will take about 272 MiB in ClickHouse. + +Source: https://www.gov.uk/government/statistical-data-sets/price-paid-data-downloads +Description of the fields: https://www.gov.uk/guidance/about-the-price-paid-data + +Contains HM Land Registry data © Crown copyright and database right 2021. This data is licensed under the Open Government Licence v3.0. + +## Download the Dataset + +``` +wget http://prod.publicdata.landregistry.gov.uk.s3-website-eu-west-1.amazonaws.com/pp-complete.csv +``` + +Download will took about 2 minutes with good internet connection. + +## Create the Table + +``` +CREATE TABLE uk_price_paid +( + price UInt32, + date Date, + postcode1 LowCardinality(String), + postcode2 LowCardinality(String), + type Enum8('terraced' = 1, 'semi-detached' = 2, 'detached' = 3, 'flat' = 4, 'other' = 0), + is_new UInt8, + duration Enum8('freehold' = 1, 'leasehold' = 2, 'unknown' = 0), + addr1 String, + addr2 String, + street LowCardinality(String), + locality LowCardinality(String), + town LowCardinality(String), + district LowCardinality(String), + county LowCardinality(String), + category UInt8 +) ENGINE = MergeTree ORDER BY (postcode1, postcode2, addr1, addr2); +``` + +## Preprocess and Import Data + +We will use `clickhouse-local` tool for data preprocessing and `clickhouse-client` to upload it. + +In this example, we define the structure of source data from the CSV file and specidy a query to preprocess the data with `clickhouse-local`. + +The preprocessing is: +- splitting the postcode to two different columns `postcode1` and `postcode2` that is better for storage and queries; +- coverting the `time` field to date as it only contains 00:00 time; +- ignoring the `uuid` field because we don't need it for analysis; +- transforming `type` and `duration` to more readable Enum fields with function `transform`; +- transforming `is_new` and `category` fields from single-character string (`Y`/`N` and `A`/`B`) to UInt8 field with 0 and 1. + +Preprocessed data is piped directly to `clickhouse-client` to be inserted into ClickHouse table in streaming fashion. + +``` +clickhouse-local --input-format CSV --structure ' + uuid String, + price UInt32, + time DateTime, + postcode String, + a String, + b String, + c String, + addr1 String, + addr2 String, + street String, + locality String, + town String, + district String, + county String, + d String, + e String +' --query " + WITH splitByChar(' ', postcode) AS p + SELECT + price, + toDate(time) AS date, + p[1] AS postcode1, + p[2] AS postcode2, + transform(a, ['T', 'S', 'D', 'F', 'O'], ['terraced', 'semi-detached', 'detached', 'flat', 'other']) AS type, + b = 'Y' AS is_new, + transform(c, ['F', 'L', 'U'], ['freehold', 'leasehold', 'unknown']) AS duration, + addr1, + addr2, + street, + locality, + town, + district, + county, + d = 'B' AS category + FROM table" --date_time_input_format best_effort < pp-complete.csv | clickhouse-client --query "INSERT INTO uk_price_paid FORMAT TSV" +``` + +It will took about 40 seconds. + +## Validate the Data + +``` +SELECT count() FROM uk_price_paid +26248711 +``` + +The size of dataset in ClickHouse is just 226 MiB: + +``` +SELECT formatReadableSize(total_bytes) FROM system.tables WHERE name = 'uk_price_paid' +226.40 MiB +``` + +## Run Some Queries + +### Average price per year: + +``` +SELECT toYear(date) AS year, round(avg(price)) AS price, bar(price, 0, 1000000, 80) FROM uk_price_paid GROUP BY year ORDER BY year + +┌─year─┬──price─┬─bar(round(avg(price)), 0, 1000000, 80)─┐ +│ 1995 │ 67932 │ █████▍ │ +│ 1996 │ 71505 │ █████▋ │ +│ 1997 │ 78532 │ ██████▎ │ +│ 1998 │ 85435 │ ██████▋ │ +│ 1999 │ 96036 │ ███████▋ │ +│ 2000 │ 107478 │ ████████▌ │ +│ 2001 │ 118886 │ █████████▌ │ +│ 2002 │ 137940 │ ███████████ │ +│ 2003 │ 155888 │ ████████████▍ │ +│ 2004 │ 178885 │ ██████████████▎ │ +│ 2005 │ 189350 │ ███████████████▏ │ +│ 2006 │ 203528 │ ████████████████▎ │ +│ 2007 │ 219377 │ █████████████████▌ │ +│ 2008 │ 217056 │ █████████████████▎ │ +│ 2009 │ 213419 │ █████████████████ │ +│ 2010 │ 236110 │ ██████████████████▊ │ +│ 2011 │ 232804 │ ██████████████████▌ │ +│ 2012 │ 238366 │ ███████████████████ │ +│ 2013 │ 256931 │ ████████████████████▌ │ +│ 2014 │ 279917 │ ██████████████████████▍ │ +│ 2015 │ 297264 │ ███████████████████████▋ │ +│ 2016 │ 313197 │ █████████████████████████ │ +│ 2017 │ 346070 │ ███████████████████████████▋ │ +│ 2018 │ 350117 │ ████████████████████████████ │ +│ 2019 │ 351010 │ ████████████████████████████ │ +│ 2020 │ 368974 │ █████████████████████████████▌ │ +│ 2021 │ 384351 │ ██████████████████████████████▋ │ +└──────┴────────┴────────────────────────────────────────┘ + +27 rows in set. Elapsed: 0.027 sec. Processed 26.25 million rows, 157.49 MB (955.96 million rows/s., 5.74 GB/s.) +``` + +### Average price per year in London: + +``` +SELECT toYear(date) AS year, round(avg(price)) AS price, bar(price, 0, 2000000, 100) FROM uk_price_paid WHERE town = 'LONDON' GROUP BY year ORDER BY year + +┌─year─┬───price─┬─bar(round(avg(price)), 0, 2000000, 100)───────────────┐ +│ 1995 │ 109112 │ █████▍ │ +│ 1996 │ 118667 │ █████▊ │ +│ 1997 │ 136518 │ ██████▋ │ +│ 1998 │ 152983 │ ███████▋ │ +│ 1999 │ 180633 │ █████████ │ +│ 2000 │ 215830 │ ██████████▋ │ +│ 2001 │ 232996 │ ███████████▋ │ +│ 2002 │ 263672 │ █████████████▏ │ +│ 2003 │ 278394 │ █████████████▊ │ +│ 2004 │ 304665 │ ███████████████▏ │ +│ 2005 │ 322875 │ ████████████████▏ │ +│ 2006 │ 356192 │ █████████████████▋ │ +│ 2007 │ 404055 │ ████████████████████▏ │ +│ 2008 │ 420741 │ █████████████████████ │ +│ 2009 │ 427754 │ █████████████████████▍ │ +│ 2010 │ 480306 │ ████████████████████████ │ +│ 2011 │ 496274 │ ████████████████████████▋ │ +│ 2012 │ 519441 │ █████████████████████████▊ │ +│ 2013 │ 616209 │ ██████████████████████████████▋ │ +│ 2014 │ 724144 │ ████████████████████████████████████▏ │ +│ 2015 │ 792112 │ ███████████████████████████████████████▌ │ +│ 2016 │ 843568 │ ██████████████████████████████████████████▏ │ +│ 2017 │ 982566 │ █████████████████████████████████████████████████▏ │ +│ 2018 │ 1016845 │ ██████████████████████████████████████████████████▋ │ +│ 2019 │ 1043277 │ ████████████████████████████████████████████████████▏ │ +│ 2020 │ 1003963 │ ██████████████████████████████████████████████████▏ │ +│ 2021 │ 940794 │ ███████████████████████████████████████████████ │ +└──────┴─────────┴───────────────────────────────────────────────────────┘ + +27 rows in set. Elapsed: 0.024 sec. Processed 26.25 million rows, 76.88 MB (1.08 billion rows/s., 3.15 GB/s.) +``` + +Something happened in 2013. I don't have a clue. Maybe you have a clue what happened in 2020? + +### The most expensive neighborhoods: + +``` +SELECT + town, + district, + count() AS c, + round(avg(price)) AS price, + bar(price, 0, 5000000, 100) +FROM uk_price_paid +WHERE date >= '2020-01-01' +GROUP BY + town, + district +HAVING c >= 100 +ORDER BY price DESC +LIMIT 100 + +Query id: df8c0a98-4713-4f0e-9690-5f73b52f7206 + +┌─town─────────────────┬─district───────────────┬────c─┬───price─┬─bar(round(avg(price)), 0, 5000000, 100)────────────────────────────┐ +│ LONDON │ CITY OF WESTMINSTER │ 3372 │ 3305225 │ ██████████████████████████████████████████████████████████████████ │ +│ LONDON │ CITY OF LONDON │ 257 │ 3294478 │ █████████████████████████████████████████████████████████████████▊ │ +│ LONDON │ KENSINGTON AND CHELSEA │ 2367 │ 2342422 │ ██████████████████████████████████████████████▋ │ +│ LEATHERHEAD │ ELMBRIDGE │ 108 │ 1927143 │ ██████████████████████████████████████▌ │ +│ VIRGINIA WATER │ RUNNYMEDE │ 142 │ 1868819 │ █████████████████████████████████████▍ │ +│ LONDON │ CAMDEN │ 2815 │ 1736788 │ ██████████████████████████████████▋ │ +│ THORNTON HEATH │ CROYDON │ 521 │ 1733051 │ ██████████████████████████████████▋ │ +│ WINDLESHAM │ SURREY HEATH │ 103 │ 1717255 │ ██████████████████████████████████▎ │ +│ BARNET │ ENFIELD │ 115 │ 1503458 │ ██████████████████████████████ │ +│ OXFORD │ SOUTH OXFORDSHIRE │ 298 │ 1275200 │ █████████████████████████▌ │ +│ LONDON │ ISLINGTON │ 2458 │ 1274308 │ █████████████████████████▍ │ +│ COBHAM │ ELMBRIDGE │ 364 │ 1260005 │ █████████████████████████▏ │ +│ LONDON │ HOUNSLOW │ 618 │ 1215682 │ ████████████████████████▎ │ +│ ASCOT │ WINDSOR AND MAIDENHEAD │ 379 │ 1215146 │ ████████████████████████▎ │ +│ LONDON │ RICHMOND UPON THAMES │ 654 │ 1207551 │ ████████████████████████▏ │ +│ BEACONSFIELD │ BUCKINGHAMSHIRE │ 307 │ 1186220 │ ███████████████████████▋ │ +│ RICHMOND │ RICHMOND UPON THAMES │ 805 │ 1100420 │ ██████████████████████ │ +│ LONDON │ HAMMERSMITH AND FULHAM │ 2888 │ 1062959 │ █████████████████████▎ │ +│ WEYBRIDGE │ ELMBRIDGE │ 607 │ 1027161 │ ████████████████████▌ │ +│ RADLETT │ HERTSMERE │ 265 │ 1015896 │ ████████████████████▎ │ +│ SALCOMBE │ SOUTH HAMS │ 124 │ 1014393 │ ████████████████████▎ │ +│ BURFORD │ WEST OXFORDSHIRE │ 102 │ 993100 │ ███████████████████▋ │ +│ ESHER │ ELMBRIDGE │ 454 │ 969770 │ ███████████████████▍ │ +│ HINDHEAD │ WAVERLEY │ 128 │ 967786 │ ███████████████████▎ │ +│ BROCKENHURST │ NEW FOREST │ 121 │ 967046 │ ███████████████████▎ │ +│ LEATHERHEAD │ GUILDFORD │ 191 │ 964489 │ ███████████████████▎ │ +│ GERRARDS CROSS │ BUCKINGHAMSHIRE │ 376 │ 958555 │ ███████████████████▏ │ +│ EAST MOLESEY │ ELMBRIDGE │ 181 │ 943457 │ ██████████████████▋ │ +│ OLNEY │ MILTON KEYNES │ 220 │ 942892 │ ██████████████████▋ │ +│ CHALFONT ST GILES │ BUCKINGHAMSHIRE │ 135 │ 926950 │ ██████████████████▌ │ +│ HENLEY-ON-THAMES │ SOUTH OXFORDSHIRE │ 509 │ 905732 │ ██████████████████ │ +│ KINGSTON UPON THAMES │ KINGSTON UPON THAMES │ 889 │ 899689 │ █████████████████▊ │ +│ BELVEDERE │ BEXLEY │ 313 │ 895336 │ █████████████████▊ │ +│ CRANBROOK │ TUNBRIDGE WELLS │ 404 │ 888190 │ █████████████████▋ │ +│ LONDON │ EALING │ 2460 │ 865893 │ █████████████████▎ │ +│ MAIDENHEAD │ BUCKINGHAMSHIRE │ 114 │ 863814 │ █████████████████▎ │ +│ LONDON │ MERTON │ 1958 │ 857192 │ █████████████████▏ │ +│ GUILDFORD │ WAVERLEY │ 131 │ 854447 │ █████████████████ │ +│ LONDON │ HACKNEY │ 3088 │ 846571 │ ████████████████▊ │ +│ LYMM │ WARRINGTON │ 285 │ 839920 │ ████████████████▋ │ +│ HARPENDEN │ ST ALBANS │ 606 │ 836994 │ ████████████████▋ │ +│ LONDON │ WANDSWORTH │ 6113 │ 832292 │ ████████████████▋ │ +│ LONDON │ SOUTHWARK │ 3612 │ 831319 │ ████████████████▋ │ +│ BERKHAMSTED │ DACORUM │ 502 │ 830356 │ ████████████████▌ │ +│ KINGS LANGLEY │ DACORUM │ 137 │ 821358 │ ████████████████▍ │ +│ TONBRIDGE │ TUNBRIDGE WELLS │ 339 │ 806736 │ ████████████████▏ │ +│ EPSOM │ REIGATE AND BANSTEAD │ 157 │ 805903 │ ████████████████ │ +│ WOKING │ GUILDFORD │ 161 │ 803283 │ ████████████████ │ +│ STOCKBRIDGE │ TEST VALLEY │ 168 │ 801973 │ ████████████████ │ +│ TEDDINGTON │ RICHMOND UPON THAMES │ 539 │ 798591 │ ███████████████▊ │ +│ OXFORD │ VALE OF WHITE HORSE │ 329 │ 792907 │ ███████████████▋ │ +│ LONDON │ BARNET │ 3624 │ 789583 │ ███████████████▋ │ +│ TWICKENHAM │ RICHMOND UPON THAMES │ 1090 │ 787760 │ ███████████████▋ │ +│ LUTON │ CENTRAL BEDFORDSHIRE │ 196 │ 786051 │ ███████████████▋ │ +│ TONBRIDGE │ MAIDSTONE │ 277 │ 785746 │ ███████████████▋ │ +│ TOWCESTER │ WEST NORTHAMPTONSHIRE │ 186 │ 783532 │ ███████████████▋ │ +│ LONDON │ LAMBETH │ 4832 │ 783422 │ ███████████████▋ │ +│ LUTTERWORTH │ HARBOROUGH │ 515 │ 781775 │ ███████████████▋ │ +│ WOODSTOCK │ WEST OXFORDSHIRE │ 135 │ 777499 │ ███████████████▌ │ +│ ALRESFORD │ WINCHESTER │ 196 │ 775577 │ ███████████████▌ │ +│ LONDON │ NEWHAM │ 2942 │ 768551 │ ███████████████▎ │ +│ ALDERLEY EDGE │ CHESHIRE EAST │ 168 │ 768280 │ ███████████████▎ │ +│ MARLOW │ BUCKINGHAMSHIRE │ 301 │ 762784 │ ███████████████▎ │ +│ BILLINGSHURST │ CHICHESTER │ 134 │ 760920 │ ███████████████▏ │ +│ LONDON │ TOWER HAMLETS │ 4183 │ 759635 │ ███████████████▏ │ +│ MIDHURST │ CHICHESTER │ 245 │ 759101 │ ███████████████▏ │ +│ THAMES DITTON │ ELMBRIDGE │ 227 │ 753347 │ ███████████████ │ +│ POTTERS BAR │ WELWYN HATFIELD │ 163 │ 752926 │ ███████████████ │ +│ REIGATE │ REIGATE AND BANSTEAD │ 555 │ 740961 │ ██████████████▋ │ +│ TADWORTH │ REIGATE AND BANSTEAD │ 477 │ 738997 │ ██████████████▋ │ +│ SEVENOAKS │ SEVENOAKS │ 1074 │ 734658 │ ██████████████▋ │ +│ PETWORTH │ CHICHESTER │ 138 │ 732432 │ ██████████████▋ │ +│ BOURNE END │ BUCKINGHAMSHIRE │ 127 │ 730742 │ ██████████████▌ │ +│ PURLEY │ CROYDON │ 540 │ 727721 │ ██████████████▌ │ +│ OXTED │ TANDRIDGE │ 320 │ 726078 │ ██████████████▌ │ +│ LONDON │ HARINGEY │ 2988 │ 724573 │ ██████████████▍ │ +│ BANSTEAD │ REIGATE AND BANSTEAD │ 373 │ 713834 │ ██████████████▎ │ +│ PINNER │ HARROW │ 480 │ 712166 │ ██████████████▏ │ +│ MALMESBURY │ WILTSHIRE │ 293 │ 707747 │ ██████████████▏ │ +│ RICKMANSWORTH │ THREE RIVERS │ 732 │ 705400 │ ██████████████ │ +│ SLOUGH │ BUCKINGHAMSHIRE │ 359 │ 705002 │ ██████████████ │ +│ GREAT MISSENDEN │ BUCKINGHAMSHIRE │ 214 │ 704904 │ ██████████████ │ +│ READING │ SOUTH OXFORDSHIRE │ 295 │ 701697 │ ██████████████ │ +│ HYTHE │ FOLKESTONE AND HYTHE │ 457 │ 700334 │ ██████████████ │ +│ WELWYN │ WELWYN HATFIELD │ 217 │ 699649 │ █████████████▊ │ +│ CHIGWELL │ EPPING FOREST │ 242 │ 697869 │ █████████████▊ │ +│ BARNET │ BARNET │ 906 │ 695680 │ █████████████▊ │ +│ HASLEMERE │ CHICHESTER │ 120 │ 694028 │ █████████████▊ │ +│ LEATHERHEAD │ MOLE VALLEY │ 748 │ 692026 │ █████████████▋ │ +│ LONDON │ BRENT │ 1945 │ 690799 │ █████████████▋ │ +│ HASLEMERE │ WAVERLEY │ 258 │ 690765 │ █████████████▋ │ +│ NORTHWOOD │ HILLINGDON │ 252 │ 690753 │ █████████████▋ │ +│ WALTON-ON-THAMES │ ELMBRIDGE │ 871 │ 689431 │ █████████████▋ │ +│ INGATESTONE │ BRENTWOOD │ 150 │ 688345 │ █████████████▋ │ +│ OXFORD │ OXFORD │ 1761 │ 686114 │ █████████████▋ │ +│ CHISLEHURST │ BROMLEY │ 410 │ 682892 │ █████████████▋ │ +│ KINGS LANGLEY │ THREE RIVERS │ 109 │ 682320 │ █████████████▋ │ +│ ASHTEAD │ MOLE VALLEY │ 280 │ 680483 │ █████████████▌ │ +│ WOKING │ SURREY HEATH │ 269 │ 679035 │ █████████████▌ │ +│ ASCOT │ BRACKNELL FOREST │ 160 │ 678632 │ █████████████▌ │ +└──────────────────────┴────────────────────────┴──────┴─────────┴────────────────────────────────────────────────────────────────────┘ + +100 rows in set. Elapsed: 0.039 sec. Processed 26.25 million rows, 278.03 MB (674.32 million rows/s., 7.14 GB/s.) +``` + +### Test it in Playground + +The data is uploaded to ClickHouse Playground, [example](https://gh-api.clickhouse.tech/play?user=play#U0VMRUNUIHRvd24sIGRpc3RyaWN0LCBjb3VudCgpIEFTIGMsIHJvdW5kKGF2ZyhwcmljZSkpIEFTIHByaWNlLCBiYXIocHJpY2UsIDAsIDUwMDAwMDAsIDEwMCkgRlJPTSB1a19wcmljZV9wYWlkIFdIRVJFIGRhdGUgPj0gJzIwMjAtMDEtMDEnIEdST1VQIEJZIHRvd24sIGRpc3RyaWN0IEhBVklORyBjID49IDEwMCBPUkRFUiBCWSBwcmljZSBERVNDIExJTUlUIDEwMA==). From b368c12400a6fd7688d6623eb3d6c592d6d148f4 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 8 Aug 2021 22:15:18 +0300 Subject: [PATCH 292/599] Add UK Property Paid data --- docs/en/getting-started/example-datasets/uk-price-paid.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/getting-started/example-datasets/uk-price-paid.md b/docs/en/getting-started/example-datasets/uk-price-paid.md index cb4baa55553..3ff77aa5700 100644 --- a/docs/en/getting-started/example-datasets/uk-price-paid.md +++ b/docs/en/getting-started/example-datasets/uk-price-paid.md @@ -6,7 +6,7 @@ toc_title: UK Property Price Paid # UK Property Price Paid The dataset contains data about prices paid for real-estate property in England and Wales. The data is available since year 1995. -The size of the dataset in uncompressed form is about 4 GiB and it will take about 272 MiB in ClickHouse. +The size of the dataset in uncompressed form is about 4 GiB and it will take about 226 MiB in ClickHouse. Source: https://www.gov.uk/government/statistical-data-sets/price-paid-data-downloads Description of the fields: https://www.gov.uk/guidance/about-the-price-paid-data From 0f5d3963bd8a76dedc4d47ce157c1e04c5e97b7d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 8 Aug 2021 22:16:11 +0300 Subject: [PATCH 293/599] Add UK Property Paid data --- docs/en/getting-started/example-datasets/uk-price-paid.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/getting-started/example-datasets/uk-price-paid.md b/docs/en/getting-started/example-datasets/uk-price-paid.md index 3ff77aa5700..23d2db69af6 100644 --- a/docs/en/getting-started/example-datasets/uk-price-paid.md +++ b/docs/en/getting-started/example-datasets/uk-price-paid.md @@ -48,7 +48,7 @@ CREATE TABLE uk_price_paid We will use `clickhouse-local` tool for data preprocessing and `clickhouse-client` to upload it. -In this example, we define the structure of source data from the CSV file and specidy a query to preprocess the data with `clickhouse-local`. +In this example, we define the structure of source data from the CSV file and specify a query to preprocess the data with `clickhouse-local`. The preprocessing is: - splitting the postcode to two different columns `postcode1` and `postcode2` that is better for storage and queries; From 844c0cf0ab6cfa52e6d8afa10e01aa4213616f0d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 8 Aug 2021 22:16:53 +0300 Subject: [PATCH 294/599] Add UK Property Paid data --- docs/en/getting-started/example-datasets/uk-price-paid.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/getting-started/example-datasets/uk-price-paid.md b/docs/en/getting-started/example-datasets/uk-price-paid.md index 23d2db69af6..f328ea51a24 100644 --- a/docs/en/getting-started/example-datasets/uk-price-paid.md +++ b/docs/en/getting-started/example-datasets/uk-price-paid.md @@ -19,7 +19,7 @@ Contains HM Land Registry data © Crown copyright and database right 2021. This wget http://prod.publicdata.landregistry.gov.uk.s3-website-eu-west-1.amazonaws.com/pp-complete.csv ``` -Download will took about 2 minutes with good internet connection. +Download will take about 2 minutes with good internet connection. ## Create the Table @@ -98,7 +98,7 @@ clickhouse-local --input-format CSV --structure ' FROM table" --date_time_input_format best_effort < pp-complete.csv | clickhouse-client --query "INSERT INTO uk_price_paid FORMAT TSV" ``` -It will took about 40 seconds. +It will take about 40 seconds. ## Validate the Data From 8a320967e636cf8924921fe39b37ceca633871d1 Mon Sep 17 00:00:00 2001 From: Sergei Semin Date: Sun, 8 Aug 2021 23:26:35 +0300 Subject: [PATCH 295/599] fix description of changing url when cloning project --- docs/ru/development/developer-instruction.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/development/developer-instruction.md b/docs/ru/development/developer-instruction.md index 391d28d5a89..c3bf81b58c7 100644 --- a/docs/ru/development/developer-instruction.md +++ b/docs/ru/development/developer-instruction.md @@ -43,7 +43,7 @@ ClickHouse не работает и не собирается на 32-битны git clone --recursive git@github.com:ClickHouse/ClickHouse.git cd ClickHouse -Замените *yandex* на имя вашего аккаунта на GitHub. +Замените первое вхождение слова `ClickHouse` в команде для git на имя вашего аккаунта на GitHub. Эта команда создаст директорию ClickHouse, содержащую рабочую копию проекта. From f55e113755406a02239981fe7c0c810a25163d95 Mon Sep 17 00:00:00 2001 From: Sergei Semin Date: Sun, 8 Aug 2021 23:36:02 +0300 Subject: [PATCH 296/599] remove extra english text already translated to russian --- docs/ru/development/developer-instruction.md | 1 - 1 file changed, 1 deletion(-) diff --git a/docs/ru/development/developer-instruction.md b/docs/ru/development/developer-instruction.md index 391d28d5a89..097b2532132 100644 --- a/docs/ru/development/developer-instruction.md +++ b/docs/ru/development/developer-instruction.md @@ -92,7 +92,6 @@ ClickHouse не работает и не собирается на 32-битны # Две последние команды могут быть объединены вместе: git submodule update --init -The next commands would help you to reset all submodules to the initial state (!WARING! - any changes inside will be deleted): Следующие команды помогут сбросить все сабмодули в изначальное состояние (!ВНИМАНИЕ! - все изменения в сабмодулях будут утеряны): # Synchronizes submodules' remote URL with .gitmodules From 63f624c0b986a84d5e0e0778e0f44dc9790c247e Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sun, 8 Aug 2021 23:38:06 +0300 Subject: [PATCH 297/599] Update coordinates.md --- docs/en/sql-reference/functions/geo/coordinates.md | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/docs/en/sql-reference/functions/geo/coordinates.md b/docs/en/sql-reference/functions/geo/coordinates.md index fd1d46e48ff..537a78869d6 100644 --- a/docs/en/sql-reference/functions/geo/coordinates.md +++ b/docs/en/sql-reference/functions/geo/coordinates.md @@ -41,6 +41,13 @@ SELECT greatCircleDistance(55.755831, 37.617673, -55.755831, -37.617673) └───────────────────────────────────────────────────────────────────┘ ``` +## geoDistance + +Similar to `greatCircleDistance` but calculates the distance on WGS-84 ellipsoid instead of sphere. This is more precise approximation of the Earth Geoid. +The performance is the same as for `greatCircleDistance` (no performance drawback). It is recommended to use `geoDistance` to calculate the distances on Earth. + +Technical note: for close enough points we calculate the distance using planar approximation with the metric on the tangent plane on the midpoint of the coordinates. + ## greatCircleAngle {#greatcircleangle} Calculates the central angle between two points on the Earth’s surface using [the great-circle formula](https://en.wikipedia.org/wiki/Great-circle_distance). From f18e1aa2de5e9f111291cef1f718c8d86466eeb8 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sun, 8 Aug 2021 23:38:51 +0300 Subject: [PATCH 298/599] Update coordinates.md --- docs/en/sql-reference/functions/geo/coordinates.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/geo/coordinates.md b/docs/en/sql-reference/functions/geo/coordinates.md index 537a78869d6..2d9bb41fc27 100644 --- a/docs/en/sql-reference/functions/geo/coordinates.md +++ b/docs/en/sql-reference/functions/geo/coordinates.md @@ -46,7 +46,7 @@ SELECT greatCircleDistance(55.755831, 37.617673, -55.755831, -37.617673) Similar to `greatCircleDistance` but calculates the distance on WGS-84 ellipsoid instead of sphere. This is more precise approximation of the Earth Geoid. The performance is the same as for `greatCircleDistance` (no performance drawback). It is recommended to use `geoDistance` to calculate the distances on Earth. -Technical note: for close enough points we calculate the distance using planar approximation with the metric on the tangent plane on the midpoint of the coordinates. +Technical note: for close enough points we calculate the distance using planar approximation with the metric on the tangent plane at the midpoint of the coordinates. ## greatCircleAngle {#greatcircleangle} From 8b0cfdda686d593254490be6be4ee4336398c870 Mon Sep 17 00:00:00 2001 From: Sergei Semin Date: Sun, 8 Aug 2021 23:40:45 +0300 Subject: [PATCH 299/599] translate some text from english to russian in developer-instruction.md --- docs/ru/development/developer-instruction.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/development/developer-instruction.md b/docs/ru/development/developer-instruction.md index 391d28d5a89..f23c2589c82 100644 --- a/docs/ru/development/developer-instruction.md +++ b/docs/ru/development/developer-instruction.md @@ -140,7 +140,7 @@ ClickHouse использует для сборки некоторое коли Впрочем, наша среда continuous integration проверяет около десятка вариантов сборки, включая gcc, но сборка с помощью gcc непригодна для использования в продакшене. -On Ubuntu/Debian you can use the automatic installation script (check [official webpage](https://apt.llvm.org/)) +На Ubuntu и Debian вы можете использовать скрипт для автоматической установки (см. [официальный сайт](https://apt.llvm.org/)) ```bash sudo bash -c "$(wget -O - https://apt.llvm.org/llvm.sh)" From d159c83fb84081eb4e8ca15854fd5c8bb8aab5dd Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 9 Aug 2021 00:02:25 +0300 Subject: [PATCH 300/599] Add OpenSky dataset --- .../getting-started/example-datasets/index.md | 1 + .../example-datasets/opensky.md | 384 ++++++++++++++++++ 2 files changed, 385 insertions(+) create mode 100644 docs/en/getting-started/example-datasets/opensky.md diff --git a/docs/en/getting-started/example-datasets/index.md b/docs/en/getting-started/example-datasets/index.md index 669bdf31a61..9d4aec6eca2 100644 --- a/docs/en/getting-started/example-datasets/index.md +++ b/docs/en/getting-started/example-datasets/index.md @@ -14,6 +14,7 @@ The list of documented datasets: - [Anonymized Yandex.Metrica Dataset](../../getting-started/example-datasets/metrica.md) - [Recipes](../../getting-started/example-datasets/recipes.md) - [OnTime](../../getting-started/example-datasets/ontime.md) +- [OpenSky](../../getting-started/example-datasets/opensky.md) - [New York Taxi Data](../../getting-started/example-datasets/nyc-taxi.md) - [UK Property Price Paid](../../getting-started/example-datasets/uk-price-paid.md) - [Star Schema Benchmark](../../getting-started/example-datasets/star-schema.md) diff --git a/docs/en/getting-started/example-datasets/opensky.md b/docs/en/getting-started/example-datasets/opensky.md new file mode 100644 index 00000000000..c9f6729cdd7 --- /dev/null +++ b/docs/en/getting-started/example-datasets/opensky.md @@ -0,0 +1,384 @@ +--- +toc_priority: 20 +toc_title: OpenSky +--- + +# Crowdsourced air traffic data from The OpenSky Network 2020 + +"The data in this dataset is derived and cleaned from the full OpenSky dataset to illustrate the development of air traffic during the COVID-19 pandemic. It spans all flights seen by the network's more than 2500 members since 1 January 2019. More data will be periodically included in the dataset until the end of the COVID-19 pandemic". + +Source: https://zenodo.org/record/5092942#.YRBCyTpRXYd + +Martin Strohmeier, Xavier Olive, Jannis Lübbe, Matthias Schäfer, and Vincent Lenders +"Crowdsourced air traffic data from the OpenSky Network 2019–2020" +Earth System Science Data 13(2), 2021 +https://doi.org/10.5194/essd-13-357-2021 + +## Download the Dataset + +``` +wget -O- https://zenodo.org/record/5092942 | grep -oP 'https://zenodo.org/record/5092942/files/flightlist_\d+_\d+\.csv\.gz' | xargs wget +``` + +Download will take about 2 minutes with good internet connection. There are 30 files with total size of 4.3 GB. + +## Create the Table + +``` +CREATE TABLE opensky +( + callsign String, + number String, + icao24 String, + registration String, + typecode String, + origin String, + destination String, + firstseen DateTime, + lastseen DateTime, + day DateTime, + latitude_1 Float64, + longitude_1 Float64, + altitude_1 Float64, + latitude_2 Float64, + longitude_2 Float64, + altitude_2 Float64 +) ENGINE = MergeTree ORDER BY (origin, destination, callsign); +``` + +## Import Data + +Upload data into ClickHouse in parallel: + +``` +ls -1 flightlist_*.csv.gz | xargs -P100 -I{} bash -c ' + gzip -c -d "{}" | clickhouse-client --date_time_input_format best_effort --query "INSERT INTO opensky FORMAT CSVWithNames"' +``` + +Here we pass the list of files (`ls -1 flightlist_*.csv.gz`) to `xargs` for parallel processing. +`xargs -P100` specifies to use up to 100 parallel workers but as we only have 30 files, the number of workers will be only 30. + +For every file, `xargs` will run a script with `bash -c`. The script has substitution in form of `{}` and the `xargs` command will substitute the filename to it (we have asked it for xargs with `-I{}`). + +The script will decompress the file (`gzip -c -d "{}"`) to standard output (`-c` parameter) and the output is redirected to `clickhouse-client`. + +Finally, `clickhouse-client` will do insertion. It will read input data in `CSVWithNames` format. We also asked to parse DateTime fields with extended parser (`--date_time_input_format best_effort`) to recognize ISO-8601 format with timezone offsets. + +Parallel upload takes 24 seconds. + +If you don't like parallel upload, here is sequential variant: +``` +for file in flightlist_*.csv.gz; do gzip -c -d "$file" | clickhouse-client --date_time_input_format best_effort --query "INSERT INTO opensky FORMAT CSVWithNames"; done +``` + +## Validate the Data + +``` +SELECT count() FROM opensky +66010819 +``` + +The size of dataset in ClickHouse is just 2.64 GiB: + +``` +SELECT formatReadableSize(total_bytes) FROM system.tables WHERE name = 'opensky' +2.64 GiB +``` + +## Run Some Queries + +Total distance travelled is 68 billion kilometers: + +``` +SELECT formatReadableQuantity(sum(geoDistance(longitude_1, latitude_1, longitude_2, latitude_2)) / 1000) FROM opensky + +┌─formatReadableQuantity(divide(sum(geoDistance(longitude_1, latitude_1, longitude_2, latitude_2)), 1000))─┐ +│ 68.72 billion │ +└──────────────────────────────────────────────────────────────────────────────────────────────────────────┘ +``` + +Average flight distance is around 1000 km. +``` +SELECT avg(geoDistance(longitude_1, latitude_1, longitude_2, latitude_2)) FROM opensky + +┌─avg(geoDistance(longitude_1, latitude_1, longitude_2, latitude_2))─┐ +│ 1041090.6465708319 │ +└────────────────────────────────────────────────────────────────────┘ +``` + +### Most busy origin airports and the average distance seen: + +``` +SELECT + origin, + count(), + round(avg(geoDistance(longitude_1, latitude_1, longitude_2, latitude_2))) AS distance, + bar(distance, 0, 10000000, 100) AS bar +FROM opensky +WHERE origin != '' +GROUP BY origin +ORDER BY count() DESC +LIMIT 100 + +Query id: f9010ea5-97d0-45a3-a5bd-9657906cd105 + + ┌─origin─┬─count()─┬─distance─┬─bar────────────────────────────────────┐ + 1. │ KORD │ 745007 │ 1546108 │ ███████████████▍ │ + 2. │ KDFW │ 696702 │ 1358721 │ █████████████▌ │ + 3. │ KATL │ 667286 │ 1169661 │ ███████████▋ │ + 4. │ KDEN │ 582709 │ 1287742 │ ████████████▊ │ + 5. │ KLAX │ 581952 │ 2628393 │ ██████████████████████████▎ │ + 6. │ KLAS │ 447789 │ 1336967 │ █████████████▎ │ + 7. │ KPHX │ 428558 │ 1345635 │ █████████████▍ │ + 8. │ KSEA │ 412592 │ 1757317 │ █████████████████▌ │ + 9. │ KCLT │ 404612 │ 880355 │ ████████▋ │ + 10. │ VIDP │ 363074 │ 1445052 │ ██████████████▍ │ + 11. │ EDDF │ 362643 │ 2263960 │ ██████████████████████▋ │ + 12. │ KSFO │ 361869 │ 2445732 │ ████████████████████████▍ │ + 13. │ KJFK │ 349232 │ 2996550 │ █████████████████████████████▊ │ + 14. │ KMSP │ 346010 │ 1287328 │ ████████████▋ │ + 15. │ LFPG │ 344748 │ 2206203 │ ██████████████████████ │ + 16. │ EGLL │ 341370 │ 3216593 │ ████████████████████████████████▏ │ + 17. │ EHAM │ 340272 │ 2116425 │ █████████████████████▏ │ + 18. │ KEWR │ 337696 │ 1826545 │ ██████████████████▎ │ + 19. │ KPHL │ 320762 │ 1291761 │ ████████████▊ │ + 20. │ OMDB │ 308855 │ 2855706 │ ████████████████████████████▌ │ + 21. │ UUEE │ 307098 │ 1555122 │ ███████████████▌ │ + 22. │ KBOS │ 304416 │ 1621675 │ ████████████████▏ │ + 23. │ LEMD │ 291787 │ 1695097 │ ████████████████▊ │ + 24. │ YSSY │ 272979 │ 1875298 │ ██████████████████▋ │ + 25. │ KMIA │ 265121 │ 1923542 │ ███████████████████▏ │ + 26. │ ZGSZ │ 263497 │ 745086 │ ███████▍ │ + 27. │ EDDM │ 256691 │ 1361453 │ █████████████▌ │ + 28. │ WMKK │ 254264 │ 1626688 │ ████████████████▎ │ + 29. │ CYYZ │ 251192 │ 2175026 │ █████████████████████▋ │ + 30. │ KLGA │ 248699 │ 1106935 │ ███████████ │ + 31. │ VHHH │ 248473 │ 3457658 │ ██████████████████████████████████▌ │ + 32. │ RJTT │ 243477 │ 1272744 │ ████████████▋ │ + 33. │ KBWI │ 241440 │ 1187060 │ ███████████▋ │ + 34. │ KIAD │ 239558 │ 1683485 │ ████████████████▋ │ + 35. │ KIAH │ 234202 │ 1538335 │ ███████████████▍ │ + 36. │ KFLL │ 223447 │ 1464410 │ ██████████████▋ │ + 37. │ KDAL │ 212055 │ 1082339 │ ██████████▋ │ + 38. │ KDCA │ 207883 │ 1013359 │ ██████████▏ │ + 39. │ LIRF │ 207047 │ 1427965 │ ██████████████▎ │ + 40. │ PANC │ 206007 │ 2525359 │ █████████████████████████▎ │ + 41. │ LTFJ │ 205415 │ 860470 │ ████████▌ │ + 42. │ KDTW │ 204020 │ 1106716 │ ███████████ │ + 43. │ VABB │ 201679 │ 1300865 │ █████████████ │ + 44. │ OTHH │ 200797 │ 3759544 │ █████████████████████████████████████▌ │ + 45. │ KMDW │ 200796 │ 1232551 │ ████████████▎ │ + 46. │ KSAN │ 198003 │ 1495195 │ ██████████████▊ │ + 47. │ KPDX │ 197760 │ 1269230 │ ████████████▋ │ + 48. │ SBGR │ 197624 │ 2041697 │ ████████████████████▍ │ + 49. │ VOBL │ 189011 │ 1040180 │ ██████████▍ │ + 50. │ LEBL │ 188956 │ 1283190 │ ████████████▋ │ + 51. │ YBBN │ 188011 │ 1253405 │ ████████████▌ │ + 52. │ LSZH │ 187934 │ 1572029 │ ███████████████▋ │ + 53. │ YMML │ 187643 │ 1870076 │ ██████████████████▋ │ + 54. │ RCTP │ 184466 │ 2773976 │ ███████████████████████████▋ │ + 55. │ KSNA │ 180045 │ 778484 │ ███████▋ │ + 56. │ EGKK │ 176420 │ 1694770 │ ████████████████▊ │ + 57. │ LOWW │ 176191 │ 1274833 │ ████████████▋ │ + 58. │ UUDD │ 176099 │ 1368226 │ █████████████▋ │ + 59. │ RKSI │ 173466 │ 3079026 │ ██████████████████████████████▋ │ + 60. │ EKCH │ 172128 │ 1229895 │ ████████████▎ │ + 61. │ KOAK │ 171119 │ 1114447 │ ███████████▏ │ + 62. │ RPLL │ 170122 │ 1440735 │ ██████████████▍ │ + 63. │ KRDU │ 167001 │ 830521 │ ████████▎ │ + 64. │ KAUS │ 164524 │ 1256198 │ ████████████▌ │ + 65. │ KBNA │ 163242 │ 1022726 │ ██████████▏ │ + 66. │ KSDF │ 162655 │ 1380867 │ █████████████▋ │ + 67. │ ENGM │ 160732 │ 910108 │ █████████ │ + 68. │ LIMC │ 160696 │ 1564620 │ ███████████████▋ │ + 69. │ KSJC │ 159278 │ 1081125 │ ██████████▋ │ + 70. │ KSTL │ 157984 │ 1026699 │ ██████████▎ │ + 71. │ UUWW │ 156811 │ 1261155 │ ████████████▌ │ + 72. │ KIND │ 153929 │ 987944 │ █████████▊ │ + 73. │ ESSA │ 153390 │ 1203439 │ ████████████ │ + 74. │ KMCO │ 153351 │ 1508657 │ ███████████████ │ + 75. │ KDVT │ 152895 │ 74048 │ ▋ │ + 76. │ VTBS │ 152645 │ 2255591 │ ██████████████████████▌ │ + 77. │ CYVR │ 149574 │ 2027413 │ ████████████████████▎ │ + 78. │ EIDW │ 148723 │ 1503985 │ ███████████████ │ + 79. │ LFPO │ 143277 │ 1152964 │ ███████████▌ │ + 80. │ EGSS │ 140830 │ 1348183 │ █████████████▍ │ + 81. │ KAPA │ 140776 │ 420441 │ ████▏ │ + 82. │ KHOU │ 138985 │ 1068806 │ ██████████▋ │ + 83. │ KTPA │ 138033 │ 1338223 │ █████████████▍ │ + 84. │ KFFZ │ 137333 │ 55397 │ ▌ │ + 85. │ NZAA │ 136092 │ 1581264 │ ███████████████▋ │ + 86. │ YPPH │ 133916 │ 1271550 │ ████████████▋ │ + 87. │ RJBB │ 133522 │ 1805623 │ ██████████████████ │ + 88. │ EDDL │ 133018 │ 1265919 │ ████████████▋ │ + 89. │ ULLI │ 130501 │ 1197108 │ ███████████▊ │ + 90. │ KIWA │ 127195 │ 250876 │ ██▌ │ + 91. │ KTEB │ 126969 │ 1189414 │ ███████████▊ │ + 92. │ VOMM │ 125616 │ 1127757 │ ███████████▎ │ + 93. │ LSGG │ 123998 │ 1049101 │ ██████████▍ │ + 94. │ LPPT │ 122733 │ 1779187 │ █████████████████▋ │ + 95. │ WSSS │ 120493 │ 3264122 │ ████████████████████████████████▋ │ + 96. │ EBBR │ 118539 │ 1579939 │ ███████████████▋ │ + 97. │ VTBD │ 118107 │ 661627 │ ██████▌ │ + 98. │ KVNY │ 116326 │ 692960 │ ██████▊ │ + 99. │ EDDT │ 115122 │ 941740 │ █████████▍ │ +100. │ EFHK │ 114860 │ 1629143 │ ████████████████▎ │ + └────────┴─────────┴──────────┴────────────────────────────────────────┘ + +100 rows in set. Elapsed: 0.186 sec. Processed 48.31 million rows, 2.17 GB (259.27 million rows/s., 11.67 GB/s.) +``` + +### Number of flights from three major Moscow airports, weekly: + +``` +SELECT + toMonday(day) AS k, + count() AS c, + bar(c, 0, 10000, 100) AS bar +FROM opensky +WHERE origin IN ('UUEE', 'UUDD', 'UUWW') +GROUP BY k +ORDER BY k ASC + +Query id: 1b446157-9519-4cc4-a1cb-178dfcc15a8e + + ┌──────────k─┬────c─┬─bar──────────────────────────────────────────────────────────────────────────┐ + 1. │ 2018-12-31 │ 5248 │ ████████████████████████████████████████████████████▍ │ + 2. │ 2019-01-07 │ 6302 │ ███████████████████████████████████████████████████████████████ │ + 3. │ 2019-01-14 │ 5701 │ █████████████████████████████████████████████████████████ │ + 4. │ 2019-01-21 │ 5638 │ ████████████████████████████████████████████████████████▍ │ + 5. │ 2019-01-28 │ 5731 │ █████████████████████████████████████████████████████████▎ │ + 6. │ 2019-02-04 │ 5683 │ ████████████████████████████████████████████████████████▋ │ + 7. │ 2019-02-11 │ 5759 │ █████████████████████████████████████████████████████████▌ │ + 8. │ 2019-02-18 │ 5736 │ █████████████████████████████████████████████████████████▎ │ + 9. │ 2019-02-25 │ 5873 │ ██████████████████████████████████████████████████████████▋ │ + 10. │ 2019-03-04 │ 5965 │ ███████████████████████████████████████████████████████████▋ │ + 11. │ 2019-03-11 │ 5900 │ ███████████████████████████████████████████████████████████ │ + 12. │ 2019-03-18 │ 5823 │ ██████████████████████████████████████████████████████████▏ │ + 13. │ 2019-03-25 │ 5899 │ ██████████████████████████████████████████████████████████▊ │ + 14. │ 2019-04-01 │ 6043 │ ████████████████████████████████████████████████████████████▍ │ + 15. │ 2019-04-08 │ 6098 │ ████████████████████████████████████████████████████████████▊ │ + 16. │ 2019-04-15 │ 6196 │ █████████████████████████████████████████████████████████████▊ │ + 17. │ 2019-04-22 │ 6486 │ ████████████████████████████████████████████████████████████████▋ │ + 18. │ 2019-04-29 │ 6682 │ ██████████████████████████████████████████████████████████████████▋ │ + 19. │ 2019-05-06 │ 6739 │ ███████████████████████████████████████████████████████████████████▍ │ + 20. │ 2019-05-13 │ 6600 │ ██████████████████████████████████████████████████████████████████ │ + 21. │ 2019-05-20 │ 6575 │ █████████████████████████████████████████████████████████████████▋ │ + 22. │ 2019-05-27 │ 6786 │ ███████████████████████████████████████████████████████████████████▋ │ + 23. │ 2019-06-03 │ 6872 │ ████████████████████████████████████████████████████████████████████▋ │ + 24. │ 2019-06-10 │ 7045 │ ██████████████████████████████████████████████████████████████████████▍ │ + 25. │ 2019-06-17 │ 7045 │ ██████████████████████████████████████████████████████████████████████▍ │ + 26. │ 2019-06-24 │ 6852 │ ████████████████████████████████████████████████████████████████████▌ │ + 27. │ 2019-07-01 │ 7248 │ ████████████████████████████████████████████████████████████████████████▍ │ + 28. │ 2019-07-08 │ 7284 │ ████████████████████████████████████████████████████████████████████████▋ │ + 29. │ 2019-07-15 │ 7142 │ ███████████████████████████████████████████████████████████████████████▍ │ + 30. │ 2019-07-22 │ 7108 │ ███████████████████████████████████████████████████████████████████████ │ + 31. │ 2019-07-29 │ 7251 │ ████████████████████████████████████████████████████████████████████████▌ │ + 32. │ 2019-08-05 │ 7403 │ ██████████████████████████████████████████████████████████████████████████ │ + 33. │ 2019-08-12 │ 7457 │ ██████████████████████████████████████████████████████████████████████████▌ │ + 34. │ 2019-08-19 │ 7502 │ ███████████████████████████████████████████████████████████████████████████ │ + 35. │ 2019-08-26 │ 7540 │ ███████████████████████████████████████████████████████████████████████████▍ │ + 36. │ 2019-09-02 │ 7237 │ ████████████████████████████████████████████████████████████████████████▎ │ + 37. │ 2019-09-09 │ 7328 │ █████████████████████████████████████████████████████████████████████████▎ │ + 38. │ 2019-09-16 │ 5566 │ ███████████████████████████████████████████████████████▋ │ + 39. │ 2019-09-23 │ 7049 │ ██████████████████████████████████████████████████████████████████████▍ │ + 40. │ 2019-09-30 │ 6880 │ ████████████████████████████████████████████████████████████████████▋ │ + 41. │ 2019-10-07 │ 6518 │ █████████████████████████████████████████████████████████████████▏ │ + 42. │ 2019-10-14 │ 6688 │ ██████████████████████████████████████████████████████████████████▊ │ + 43. │ 2019-10-21 │ 6667 │ ██████████████████████████████████████████████████████████████████▋ │ + 44. │ 2019-10-28 │ 6303 │ ███████████████████████████████████████████████████████████████ │ + 45. │ 2019-11-04 │ 6298 │ ██████████████████████████████████████████████████████████████▊ │ + 46. │ 2019-11-11 │ 6137 │ █████████████████████████████████████████████████████████████▎ │ + 47. │ 2019-11-18 │ 6051 │ ████████████████████████████████████████████████████████████▌ │ + 48. │ 2019-11-25 │ 5820 │ ██████████████████████████████████████████████████████████▏ │ + 49. │ 2019-12-02 │ 5942 │ ███████████████████████████████████████████████████████████▍ │ + 50. │ 2019-12-09 │ 4891 │ ████████████████████████████████████████████████▊ │ + 51. │ 2019-12-16 │ 5682 │ ████████████████████████████████████████████████████████▋ │ + 52. │ 2019-12-23 │ 6111 │ █████████████████████████████████████████████████████████████ │ + 53. │ 2019-12-30 │ 5870 │ ██████████████████████████████████████████████████████████▋ │ + 54. │ 2020-01-06 │ 5953 │ ███████████████████████████████████████████████████████████▌ │ + 55. │ 2020-01-13 │ 5698 │ ████████████████████████████████████████████████████████▊ │ + 56. │ 2020-01-20 │ 5339 │ █████████████████████████████████████████████████████▍ │ + 57. │ 2020-01-27 │ 5566 │ ███████████████████████████████████████████████████████▋ │ + 58. │ 2020-02-03 │ 5801 │ ██████████████████████████████████████████████████████████ │ + 59. │ 2020-02-10 │ 5692 │ ████████████████████████████████████████████████████████▊ │ + 60. │ 2020-02-17 │ 5912 │ ███████████████████████████████████████████████████████████ │ + 61. │ 2020-02-24 │ 6031 │ ████████████████████████████████████████████████████████████▎ │ + 62. │ 2020-03-02 │ 6105 │ █████████████████████████████████████████████████████████████ │ + 63. │ 2020-03-09 │ 5823 │ ██████████████████████████████████████████████████████████▏ │ + 64. │ 2020-03-16 │ 4659 │ ██████████████████████████████████████████████▌ │ + 65. │ 2020-03-23 │ 3720 │ █████████████████████████████████████▏ │ + 66. │ 2020-03-30 │ 1720 │ █████████████████▏ │ + 67. │ 2020-04-06 │ 849 │ ████████▍ │ + 68. │ 2020-04-13 │ 710 │ ███████ │ + 69. │ 2020-04-20 │ 725 │ ███████▏ │ + 70. │ 2020-04-27 │ 920 │ █████████▏ │ + 71. │ 2020-05-04 │ 859 │ ████████▌ │ + 72. │ 2020-05-11 │ 1047 │ ██████████▍ │ + 73. │ 2020-05-18 │ 1135 │ ███████████▎ │ + 74. │ 2020-05-25 │ 1266 │ ████████████▋ │ + 75. │ 2020-06-01 │ 1793 │ █████████████████▊ │ + 76. │ 2020-06-08 │ 1979 │ ███████████████████▋ │ + 77. │ 2020-06-15 │ 2297 │ ██████████████████████▊ │ + 78. │ 2020-06-22 │ 2788 │ ███████████████████████████▊ │ + 79. │ 2020-06-29 │ 3389 │ █████████████████████████████████▊ │ + 80. │ 2020-07-06 │ 3545 │ ███████████████████████████████████▍ │ + 81. │ 2020-07-13 │ 3569 │ ███████████████████████████████████▋ │ + 82. │ 2020-07-20 │ 3784 │ █████████████████████████████████████▋ │ + 83. │ 2020-07-27 │ 3960 │ ███████████████████████████████████████▌ │ + 84. │ 2020-08-03 │ 4323 │ ███████████████████████████████████████████▏ │ + 85. │ 2020-08-10 │ 4581 │ █████████████████████████████████████████████▋ │ + 86. │ 2020-08-17 │ 4791 │ ███████████████████████████████████████████████▊ │ + 87. │ 2020-08-24 │ 4928 │ █████████████████████████████████████████████████▎ │ + 88. │ 2020-08-31 │ 4687 │ ██████████████████████████████████████████████▋ │ + 89. │ 2020-09-07 │ 4643 │ ██████████████████████████████████████████████▍ │ + 90. │ 2020-09-14 │ 4594 │ █████████████████████████████████████████████▊ │ + 91. │ 2020-09-21 │ 4478 │ ████████████████████████████████████████████▋ │ + 92. │ 2020-09-28 │ 4382 │ ███████████████████████████████████████████▋ │ + 93. │ 2020-10-05 │ 4261 │ ██████████████████████████████████████████▌ │ + 94. │ 2020-10-12 │ 4243 │ ██████████████████████████████████████████▍ │ + 95. │ 2020-10-19 │ 3941 │ ███████████████████████████████████████▍ │ + 96. │ 2020-10-26 │ 3616 │ ████████████████████████████████████▏ │ + 97. │ 2020-11-02 │ 3586 │ ███████████████████████████████████▋ │ + 98. │ 2020-11-09 │ 3403 │ ██████████████████████████████████ │ + 99. │ 2020-11-16 │ 3336 │ █████████████████████████████████▎ │ +100. │ 2020-11-23 │ 3230 │ ████████████████████████████████▎ │ +101. │ 2020-11-30 │ 3183 │ ███████████████████████████████▋ │ +102. │ 2020-12-07 │ 3285 │ ████████████████████████████████▋ │ +103. │ 2020-12-14 │ 3367 │ █████████████████████████████████▋ │ +104. │ 2020-12-21 │ 3748 │ █████████████████████████████████████▍ │ +105. │ 2020-12-28 │ 3986 │ ███████████████████████████████████████▋ │ +106. │ 2021-01-04 │ 3906 │ ███████████████████████████████████████ │ +107. │ 2021-01-11 │ 3425 │ ██████████████████████████████████▎ │ +108. │ 2021-01-18 │ 3144 │ ███████████████████████████████▍ │ +109. │ 2021-01-25 │ 3115 │ ███████████████████████████████▏ │ +110. │ 2021-02-01 │ 3285 │ ████████████████████████████████▋ │ +111. │ 2021-02-08 │ 3321 │ █████████████████████████████████▏ │ +112. │ 2021-02-15 │ 3475 │ ██████████████████████████████████▋ │ +113. │ 2021-02-22 │ 3549 │ ███████████████████████████████████▍ │ +114. │ 2021-03-01 │ 3755 │ █████████████████████████████████████▌ │ +115. │ 2021-03-08 │ 3080 │ ██████████████████████████████▋ │ +116. │ 2021-03-15 │ 3789 │ █████████████████████████████████████▊ │ +117. │ 2021-03-22 │ 3804 │ ██████████████████████████████████████ │ +118. │ 2021-03-29 │ 4238 │ ██████████████████████████████████████████▍ │ +119. │ 2021-04-05 │ 4307 │ ███████████████████████████████████████████ │ +120. │ 2021-04-12 │ 4225 │ ██████████████████████████████████████████▎ │ +121. │ 2021-04-19 │ 4391 │ ███████████████████████████████████████████▊ │ +122. │ 2021-04-26 │ 4868 │ ████████████████████████████████████████████████▋ │ +123. │ 2021-05-03 │ 4977 │ █████████████████████████████████████████████████▋ │ +124. │ 2021-05-10 │ 5164 │ ███████████████████████████████████████████████████▋ │ +125. │ 2021-05-17 │ 4986 │ █████████████████████████████████████████████████▋ │ +126. │ 2021-05-24 │ 5024 │ ██████████████████████████████████████████████████▏ │ +127. │ 2021-05-31 │ 4824 │ ████████████████████████████████████████████████▏ │ +128. │ 2021-06-07 │ 5652 │ ████████████████████████████████████████████████████████▌ │ +129. │ 2021-06-14 │ 5613 │ ████████████████████████████████████████████████████████▏ │ +130. │ 2021-06-21 │ 6061 │ ████████████████████████████████████████████████████████████▌ │ +131. │ 2021-06-28 │ 2554 │ █████████████████████████▌ │ + └────────────┴──────┴──────────────────────────────────────────────────────────────────────────────┘ + +131 rows in set. Elapsed: 0.014 sec. Processed 655.36 thousand rows, 11.14 MB (47.56 million rows/s., 808.48 MB/s.) +``` + +### Test it in Playground + +The data is uploaded to ClickHouse Playground, [example](https://gh-api.clickhouse.tech/play?user=play#U0VMRUNUCiAgICBvcmlnaW4sCiAgICBjb3VudCgpLAogICAgcm91bmQoYXZnKGdlb0Rpc3RhbmNlKGxvbmdpdHVkZV8xLCBsYXRpdHVkZV8xLCBsb25naXR1ZGVfMiwgbGF0aXR1ZGVfMikpKSBBUyBkaXN0YW5jZSwKICAgIGJhcihkaXN0YW5jZSwgMCwgMTAwMDAwMDAsIDEwMCkgQVMgYmFyCkZST00gb3BlbnNreQpXSEVSRSBvcmlnaW4gIT0gJycKR1JPVVAgQlkgb3JpZ2luCk9SREVSIEJZIGNvdW50KCkgREVTQwpMSU1JVCAxMDA=). From 578e55184e984c9ecb6d537fd74d2d1f4e46e08f Mon Sep 17 00:00:00 2001 From: Sergei Semin Date: Mon, 9 Aug 2021 00:32:36 +0300 Subject: [PATCH 301/599] add info about clang version choice --- docs/ru/development/developer-instruction.md | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/docs/ru/development/developer-instruction.md b/docs/ru/development/developer-instruction.md index f23c2589c82..4474733fe3e 100644 --- a/docs/ru/development/developer-instruction.md +++ b/docs/ru/development/developer-instruction.md @@ -163,7 +163,7 @@ sudo bash -c "$(wget -O - https://apt.llvm.org/llvm.sh)" export CC=clang CXX=clang++ cmake .. -Переменная CC отвечает за компилятор C (сокращение от слов C Compiler), переменная CXX отвечает за выбор компилятора C++ (символ X - это как плюс, но положенный набок, ради того, чтобы превратить его в букву). +Переменная CC отвечает за компилятор C (сокращение от слов C Compiler), переменная CXX отвечает за выбор компилятора C++ (символ X - это как плюс, но положенный набок, ради того, чтобы превратить его в букву). При получении ошибки типа `Could not find compiler set in environment variable CC: clang` необходимо указать в значениях для переменных CC и CXX явную версию компилятора, например, `clang-12` и `clang++-12`. Для более быстрой сборки, можно использовать debug вариант - сборку без оптимизаций. Для этого, укажите параметр `-D CMAKE_BUILD_TYPE=Debug`: @@ -195,6 +195,14 @@ sudo bash -c "$(wget -O - https://apt.llvm.org/llvm.sh)" В процессе сборки могут появится сообщения `libprotobuf WARNING` про protobuf файлы в библиотеке libhdfs2. Это не имеет значения. +В случае получения ошибок вида `error: variable 'y' set but not used [-Werror,-Wunused-but-set-variable]` ножно попробовать использовать другую версию компилятора сlang. Например, на момент написания данного текста описанная выше команда по установке clang для Ubuntu 20.04 по-умолчанию устанавливает clang-13, с которым возникает эта ошибка. Для решения проблемы можно установить clang-12 с помощью команд: +```bash +wget https://apt.llvm.org/llvm.sh +chmod +x llvm.sh +sudo ./llvm.sh 12 +``` +И далее использовать именно его, указав соответствующую версию при установке переменных окружения CC и CXX перед вызовом cmake. + При успешной сборке, вы получите готовый исполняемый файл `ClickHouse/build/programs/clickhouse`: ls -l programs/clickhouse From 06e48f39b0756b0973536f9321314d03f55729ed Mon Sep 17 00:00:00 2001 From: olgarev Date: Sun, 8 Aug 2021 23:19:00 +0000 Subject: [PATCH 302/599] New page - External Disks for Storing Data --- docs/en/engines/table-engines/integrations/hdfs.md | 2 +- docs/en/engines/table-engines/integrations/s3.md | 2 +- docs/en/operations/storing-data.md | 14 ++++++++++++++ docs/ru/engines/table-engines/integrations/hdfs.md | 2 +- docs/ru/engines/table-engines/integrations/s3.md | 2 +- docs/ru/operations/storing-data.md | 14 ++++++++++++++ 6 files changed, 32 insertions(+), 4 deletions(-) create mode 100644 docs/en/operations/storing-data.md create mode 100644 docs/ru/operations/storing-data.md diff --git a/docs/en/engines/table-engines/integrations/hdfs.md b/docs/en/engines/table-engines/integrations/hdfs.md index 948dd7b10ef..677354b4f97 100644 --- a/docs/en/engines/table-engines/integrations/hdfs.md +++ b/docs/en/engines/table-engines/integrations/hdfs.md @@ -51,7 +51,7 @@ SELECT * FROM hdfs_engine_table LIMIT 2 ## Implementation Details {#implementation-details} - Reads and writes can be parallel. -- Zero-copy replication is supported, which means that if the data is stored remotely on several machines and needs to be synchronized, then only the metadata is replicated (paths to the data parts), but not the data itself. +- [Zero-copy](../../../operations/storing-data.md#zero-copy) replication is supported. - Not supported: - `ALTER` and `SELECT...SAMPLE` operations. - Indexes. diff --git a/docs/en/engines/table-engines/integrations/s3.md b/docs/en/engines/table-engines/integrations/s3.md index e2aa7dd6bc7..7249e24aff9 100644 --- a/docs/en/engines/table-engines/integrations/s3.md +++ b/docs/en/engines/table-engines/integrations/s3.md @@ -57,7 +57,7 @@ For more information about virtual columns see [here](../../../engines/table-eng ## Implementation Details {#implementation-details} - Reads and writes can be parallel -- Zero-copy replication is supported, which means that if the data is stored remotely on several machines and needs to be synchronized, then only the metadata is replicated (paths to the data parts), but not the data itself. +- [Zero-copy](../../../operations/storing-data.md#zero-copy) replication is supported. - Not supported: - `ALTER` and `SELECT...SAMPLE` operations. - Indexes. diff --git a/docs/en/operations/storing-data.md b/docs/en/operations/storing-data.md new file mode 100644 index 00000000000..08811819acb --- /dev/null +++ b/docs/en/operations/storing-data.md @@ -0,0 +1,14 @@ +--- +toc_priority: 68 +toc_title: External Disks for Storing Data +--- + +# External Disks for Storing Data {#external-disks} + +Data, processed in ClickHouse, is usually stored in the local file system — on the same machine with the ClickHouse server. That requires large-capacity disks, which can be expensive enough. To avoid that you can store the data remotely — on [Amazon s3](https://aws.amazon.com/s3/) disks or in the Hadoop Distributed File System ([HDFS](https://hadoop.apache.org/docs/current/hadoop-project-dist/hadoop-hdfs/HdfsDesign.html)). + +To work with data stored on `Amazon s3` disks use [s3](../engines/table-engines/integrations/s3.md) table engine, and to work with data in the Hadoop Distributed File System — [HDFS](../engines/table-engines/integrations/hdfs.md) table engine. + +## Zero-copy Replication {#zero-copy} + +ClickHouse supports zero-copy replication for `s3` and `HDFS` disks, which means that if the data is stored remotely on several machines and needs to be synchronized, then only the metadata is replicated (paths to the data parts), but not the data itself. \ No newline at end of file diff --git a/docs/ru/engines/table-engines/integrations/hdfs.md b/docs/ru/engines/table-engines/integrations/hdfs.md index 30cbff987cc..5949cc8a0d7 100644 --- a/docs/ru/engines/table-engines/integrations/hdfs.md +++ b/docs/ru/engines/table-engines/integrations/hdfs.md @@ -47,7 +47,7 @@ SELECT * FROM hdfs_engine_table LIMIT 2 ## Детали реализации {#implementation-details} - Поддерживается многопоточное чтение и запись. -- Поддерживается репликация без копирования данных ("zero-copy"): если данные хранятся на нескольких удаленных машинах, то при синхронизации пересылаются только метаданные (пути к кускам данных), а сами данные не копируются. +- Поддерживается репликация без копирования данных ([zero-copy](../../../operations/storing-data.md#zero-copy)). - Не поддерживается: - использование операций `ALTER` и `SELECT...SAMPLE`; - индексы. diff --git a/docs/ru/engines/table-engines/integrations/s3.md b/docs/ru/engines/table-engines/integrations/s3.md index 53053aa8695..5895bd43d2f 100644 --- a/docs/ru/engines/table-engines/integrations/s3.md +++ b/docs/ru/engines/table-engines/integrations/s3.md @@ -47,7 +47,7 @@ SELECT * FROM s3_engine_table LIMIT 2; ## Детали реализации {#implementation-details} - Чтение и запись могут быть параллельными. -- Поддерживается репиликация без копирования данных (zero-copy): если данные хранятся на нескольких удаленных машинах, то при синхронизации пересылаются только метаданные (пути к кускам данных), а сами данные не копируются. +- Поддерживается репликация без копирования данных ([zero-copy](../../../operations/storing-data.md#zero-copy)). - Не поддерживаются: - запросы `ALTER` и `SELECT...SAMPLE`, - индексы. diff --git a/docs/ru/operations/storing-data.md b/docs/ru/operations/storing-data.md new file mode 100644 index 00000000000..854e6401514 --- /dev/null +++ b/docs/ru/operations/storing-data.md @@ -0,0 +1,14 @@ +--- +toc_priority: 68 +toc_title: "Хранение данных на внешних дисках" +--- + +# Хранение данных на внешних дисках {#external-disks} + +Данные, которые обрабатываются в ClickHouse, обычно хранятся в файловой системе локально — на том же компьютере, где развернут сервер ClickHouse. При этом для хранения данных требуются диски большого объема, которые могут быть довольно дорогостоящими. Решением проблемы может стать хранение данных в удаленной файловой системе — например, на дисках [Amazon s3](https://aws.amazon.com/s3/) или в распределенной дисковой системе Hadoop ([HDFS](https://hadoop.apache.org/docs/current/hadoop-project-dist/hadoop-hdfs/HdfsDesign.html)). + +Для работы с данными, хранящимися на дисках `Amazon s3`, используйте движок [s3](../engines/table-engines/integrations/s3.md), а для работы с данными в распределенной файловой системе Hadoop — движок [HDFS](../engines/table-engines/integrations/hdfs.md). + +## Репликация без копирования данных {#zero-copy} + +Для дисков `s3` и `HDFS` в ClickHouse поддерживается репликация без копирования данных (zero-copy): если данные хранятся на нескольких удаленных машинах, то при синхронизации пересылаются только метаданные (пути к кускам данных), а сами данные не копируются. \ No newline at end of file From 86f3905eb520a2202d47c912f880dcb15a210cda Mon Sep 17 00:00:00 2001 From: olgarev Date: Sun, 8 Aug 2021 23:23:48 +0000 Subject: [PATCH 303/599] Upd --- docs/ru/engines/table-engines/integrations/s3.md | 4 ---- 1 file changed, 4 deletions(-) diff --git a/docs/ru/engines/table-engines/integrations/s3.md b/docs/ru/engines/table-engines/integrations/s3.md index 7a77cfe183c..5895bd43d2f 100644 --- a/docs/ru/engines/table-engines/integrations/s3.md +++ b/docs/ru/engines/table-engines/integrations/s3.md @@ -47,11 +47,7 @@ SELECT * FROM s3_engine_table LIMIT 2; ## Детали реализации {#implementation-details} - Чтение и запись могут быть параллельными. -<<<<<<< HEAD - Поддерживается репликация без копирования данных ([zero-copy](../../../operations/storing-data.md#zero-copy)). -======= -- Поддерживается репликация без копирования данных (zero-copy): если данные хранятся на нескольких удаленных машинах, то при синхронизации пересылаются только метаданные (пути к кускам данных), а сами данные не копируются. ->>>>>>> fbf65fb1c4cce3dc58cb439c01b706daf23df68e - Не поддерживаются: - запросы `ALTER` и `SELECT...SAMPLE`, - индексы. From 9ce7669e1fa3eef3e4a633e58012a2bf7c3de28a Mon Sep 17 00:00:00 2001 From: olgarev Date: Mon, 9 Aug 2021 01:25:22 +0000 Subject: [PATCH 304/599] Initial --- .../statements/select/distinct.md | 128 ++++++++++++------ .../sql-reference/statements/select/index.md | 6 +- .../statements/select/distinct.md | 61 +++++++-- .../sql-reference/statements/select/index.md | 6 +- 4 files changed, 149 insertions(+), 52 deletions(-) diff --git a/docs/en/sql-reference/statements/select/distinct.md b/docs/en/sql-reference/statements/select/distinct.md index 87154cba05a..1c739aab190 100644 --- a/docs/en/sql-reference/statements/select/distinct.md +++ b/docs/en/sql-reference/statements/select/distinct.md @@ -6,6 +6,96 @@ toc_title: DISTINCT If `SELECT DISTINCT` is specified, only unique rows will remain in a query result. Thus only a single row will remain out of all the sets of fully matching rows in the result. +You can narrow the list of columns which must have unique values: `SELECT DISTINCT ON (column1, column2,...)`. If the columns are not specified, all of them are taken into consideration. + +Consider the table: + +```text +┌─a─┬─b─┬─c─┐ +│ 1 │ 1 │ 1 │ +│ 1 │ 1 │ 1 │ +│ 2 │ 2 │ 2 │ +│ 2 │ 2 │ 2 │ +│ 1 │ 1 │ 2 │ +│ 1 │ 2 │ 2 │ +└───┴───┴───┘ +``` + +Using `DISTINCT` without specifying columns: + +```sql +SELECT DISTINCT * FROM t1; +``` + +```text +┌─a─┬─b─┬─c─┐ +│ 1 │ 1 │ 1 │ +│ 2 │ 2 │ 2 │ +│ 1 │ 1 │ 2 │ +│ 1 │ 2 │ 2 │ +└───┴───┴───┘ +``` + +Using `DISTINCT` with specified columns: + +```sql +SELECT DISTINCT ON (a,b) * FROM t1; +``` + +```text +┌─a─┬─b─┬─c─┐ +│ 1 │ 1 │ 1 │ +│ 2 │ 2 │ 2 │ +│ 1 │ 2 │ 2 │ +└───┴───┴───┘ +``` + +## DISTINCT and ORDER BY {#distinct-orderby} + +ClickHouse supports using the `DISTINCT` and `ORDER BY` clauses for different columns in one query. The `DISTINCT` clause is executed before the `ORDER BY` clause. + +Consider the table: + +``` text +┌─a─┬─b─┐ +│ 2 │ 1 │ +│ 1 │ 2 │ +│ 3 │ 3 │ +│ 2 │ 4 │ +└───┴───┘ +``` + +Selecting data: + +```sql +SELECT DISTINCT a FROM t1 ORDER BY b ASC; +``` + +``` text +┌─a─┐ +│ 2 │ +│ 1 │ +│ 3 │ +└───┘ +``` +Selecting data with the different sorting direction: + +```sql +SELECT DISTINCT a FROM t1 ORDER BY b DESC; +``` + +``` text +┌─a─┐ +│ 3 │ +│ 1 │ +│ 2 │ +└───┘ +``` + +Row `2, 4` was cut before sorting. + +Take this implementation specificity into account when programming queries. + ## Null Processing {#null-processing} `DISTINCT` works with [NULL](../../../sql-reference/syntax.md#null-literal) as if `NULL` were a specific value, and `NULL==NULL`. In other words, in the `DISTINCT` results, different combinations with `NULL` occur only once. It differs from `NULL` processing in most other contexts. @@ -18,41 +108,3 @@ It is possible to obtain the same result by applying [GROUP BY](../../../sql-ref - When [ORDER BY](../../../sql-reference/statements/select/order-by.md) is omitted and [LIMIT](../../../sql-reference/statements/select/limit.md) is defined, the query stops running immediately after the required number of different rows has been read. - Data blocks are output as they are processed, without waiting for the entire query to finish running. -## Examples {#examples} - -ClickHouse supports using the `DISTINCT` and `ORDER BY` clauses for different columns in one query. The `DISTINCT` clause is executed before the `ORDER BY` clause. - -Example table: - -``` text -┌─a─┬─b─┐ -│ 2 │ 1 │ -│ 1 │ 2 │ -│ 3 │ 3 │ -│ 2 │ 4 │ -└───┴───┘ -``` - -When selecting data with the `SELECT DISTINCT a FROM t1 ORDER BY b ASC` query, we get the following result: - -``` text -┌─a─┐ -│ 2 │ -│ 1 │ -│ 3 │ -└───┘ -``` - -If we change the sorting direction `SELECT DISTINCT a FROM t1 ORDER BY b DESC`, we get the following result: - -``` text -┌─a─┐ -│ 3 │ -│ 1 │ -│ 2 │ -└───┘ -``` - -Row `2, 4` was cut before sorting. - -Take this implementation specificity into account when programming queries. diff --git a/docs/en/sql-reference/statements/select/index.md b/docs/en/sql-reference/statements/select/index.md index 04273ca1d4d..4e96bae8493 100644 --- a/docs/en/sql-reference/statements/select/index.md +++ b/docs/en/sql-reference/statements/select/index.md @@ -13,7 +13,7 @@ toc_title: Overview ``` sql [WITH expr_list|(subquery)] -SELECT [DISTINCT] expr_list +SELECT [DISTINCT [ON (column1, column2, ...)]] expr_list [FROM [db.]table | (subquery) | table_function] [FINAL] [SAMPLE sample_coeff] [ARRAY JOIN ...] @@ -36,6 +36,8 @@ All clauses are optional, except for the required list of expressions immediatel Specifics of each optional clause are covered in separate sections, which are listed in the same order as they are executed: - [WITH clause](../../../sql-reference/statements/select/with.md) +- [SELECT clause](#select-clause) +- [DISTINCT clause](../../../sql-reference/statements/select/distinct.md) - [FROM clause](../../../sql-reference/statements/select/from.md) - [SAMPLE clause](../../../sql-reference/statements/select/sample.md) - [JOIN clause](../../../sql-reference/statements/select/join.md) @@ -44,8 +46,6 @@ Specifics of each optional clause are covered in separate sections, which are li - [GROUP BY clause](../../../sql-reference/statements/select/group-by.md) - [LIMIT BY clause](../../../sql-reference/statements/select/limit-by.md) - [HAVING clause](../../../sql-reference/statements/select/having.md) -- [SELECT clause](#select-clause) -- [DISTINCT clause](../../../sql-reference/statements/select/distinct.md) - [LIMIT clause](../../../sql-reference/statements/select/limit.md) - [OFFSET clause](../../../sql-reference/statements/select/offset.md) - [UNION clause](../../../sql-reference/statements/select/union.md) diff --git a/docs/ru/sql-reference/statements/select/distinct.md b/docs/ru/sql-reference/statements/select/distinct.md index f57c2a42593..42c1df64540 100644 --- a/docs/ru/sql-reference/statements/select/distinct.md +++ b/docs/ru/sql-reference/statements/select/distinct.md @@ -6,19 +6,51 @@ toc_title: DISTINCT Если указан `SELECT DISTINCT`, то в результате запроса останутся только уникальные строки. Таким образом, из всех наборов полностью совпадающих строк в результате останется только одна строка. -## Обработка NULL {#null-processing} +Вы можете указать столбцы, по которым хотите отбирать уникальные значения: `SELECT DISTINCT ON (column1, column2,...)`. Если столбцы не указаны, то отбираются строки, в которых значения уникальны во всех столбцах. -`DISTINCT` работает с [NULL](../../syntax.md#null-literal) как-будто `NULL` — обычное значение и `NULL==NULL`. Другими словами, в результате `DISTINCT`, различные комбинации с `NULL` встретятся только один раз. Это отличается от обработки `NULL` в большинстве других контекстов. +Рассмотрим таблицу: -## Альтернативы {#alternatives} +```text +┌─a─┬─b─┬─c─┐ +│ 1 │ 1 │ 1 │ +│ 1 │ 1 │ 1 │ +│ 2 │ 2 │ 2 │ +│ 2 │ 2 │ 2 │ +│ 1 │ 1 │ 2 │ +│ 1 │ 2 │ 2 │ +└───┴───┴───┘ +``` -Такой же результат можно получить, применив секцию [GROUP BY](group-by.md) для того же набора значений, которые указан в секции `SELECT`, без использования каких-либо агрегатных функций. Но есть от `GROUP BY` несколько отличий: +Использование `DISTINCT` без указания столбцов: -- `DISTINCT` может применяться вместе с `GROUP BY`. -- Когда секция [ORDER BY](order-by.md) опущена, а секция [LIMIT](limit.md) присутствует, запрос прекращает выполнение сразу после считывания необходимого количества различных строк. -- Блоки данных выводятся по мере их обработки, не дожидаясь завершения выполнения всего запроса. +```sql +SELECT DISTINCT * FROM t1; +``` -## Примеры {#examples} +```text +┌─a─┬─b─┬─c─┐ +│ 1 │ 1 │ 1 │ +│ 2 │ 2 │ 2 │ +│ 1 │ 1 │ 2 │ +│ 1 │ 2 │ 2 │ +└───┴───┴───┘ +``` + +Использование `DISTINCT` с указанием столбцов: + +```sql +SELECT DISTINCT ON (a,b) * FROM t1; +``` + +```text +┌─a─┬─b─┬─c─┐ +│ 1 │ 1 │ 1 │ +│ 2 │ 2 │ 2 │ +│ 1 │ 2 │ 2 │ +└───┴───┴───┘ +``` + +## DISTINCT и ORDER BY {#distinct-orderby} ClickHouse поддерживает использование секций `DISTINCT` и `ORDER BY` для разных столбцов в одном запросе. Секция `DISTINCT` выполняется до секции `ORDER BY`. @@ -56,3 +88,16 @@ ClickHouse поддерживает использование секций `DIS Ряд `2, 4` был разрезан перед сортировкой. Учитывайте эту специфику при разработке запросов. + +## Обработка NULL {#null-processing} + +`DISTINCT` работает с [NULL](../../syntax.md#null-literal) как-будто `NULL` — обычное значение и `NULL==NULL`. Другими словами, в результате `DISTINCT`, различные комбинации с `NULL` встретятся только один раз. Это отличается от обработки `NULL` в большинстве других контекстов. + +## Альтернативы {#alternatives} + +Можно получить такой же результат, применив [GROUP BY](group-by.md) для того же набора значений, которые указан в секции `SELECT`, без использования каких-либо агрегатных функций. Но есть несколько отличий от `GROUP BY`: + +- `DISTINCT` может применяться вместе с `GROUP BY`. +- Когда секция [ORDER BY](order-by.md) опущена, а секция [LIMIT](limit.md) присутствует, запрос прекращает выполнение сразу после считывания необходимого количества различных строк. +- Блоки данных выводятся по мере их обработки, не дожидаясь завершения выполнения всего запроса. + diff --git a/docs/ru/sql-reference/statements/select/index.md b/docs/ru/sql-reference/statements/select/index.md index a0a862cbf55..c2820bc7be4 100644 --- a/docs/ru/sql-reference/statements/select/index.md +++ b/docs/ru/sql-reference/statements/select/index.md @@ -11,7 +11,7 @@ toc_title: "Обзор" ``` sql [WITH expr_list|(subquery)] -SELECT [DISTINCT] expr_list +SELECT [DISTINCT [ON (column1, column2, ...)]] expr_list [FROM [db.]table | (subquery) | table_function] [FINAL] [SAMPLE sample_coeff] [ARRAY JOIN ...] @@ -34,6 +34,8 @@ SELECT [DISTINCT] expr_list Особенности каждой необязательной секции рассматриваются в отдельных разделах, которые перечислены в том же порядке, в каком они выполняются: - [Секция WITH](with.md) +- [Секция SELECT](#select-clause) +- [Секция DISTINCT](distinct.md) - [Секция FROM](from.md) - [Секция SAMPLE](sample.md) - [Секция JOIN](join.md) @@ -42,8 +44,6 @@ SELECT [DISTINCT] expr_list - [Секция GROUP BY](group-by.md) - [Секция LIMIT BY](limit-by.md) - [Секция HAVING](having.md) -- [Секция SELECT](#select-clause) -- [Секция DISTINCT](distinct.md) - [Секция LIMIT](limit.md) [Секция OFFSET](offset.md) - [Секция UNION ALL](union.md) From a5a60a045170e68d3374552e56fb7acc95449776 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 9 Aug 2021 05:20:41 +0000 Subject: [PATCH 305/599] Fix tests --- .../test_alter_update_cast_keep_nullable/__init__.py | 0 .../0_stateless/00597_push_down_predicate_long.reference | 2 +- 2 files changed, 1 insertion(+), 1 deletion(-) create mode 100644 tests/integration/test_alter_update_cast_keep_nullable/__init__.py diff --git a/tests/integration/test_alter_update_cast_keep_nullable/__init__.py b/tests/integration/test_alter_update_cast_keep_nullable/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/00597_push_down_predicate_long.reference b/tests/queries/0_stateless/00597_push_down_predicate_long.reference index 3eaa1139c5d..f6f1320c2f8 100644 --- a/tests/queries/0_stateless/00597_push_down_predicate_long.reference +++ b/tests/queries/0_stateless/00597_push_down_predicate_long.reference @@ -114,7 +114,7 @@ FROM ( SELECT 1 AS id, - identity(CAST(1, \'Nullable(UInt8)\')) AS subquery + identity(_CAST(1, \'Nullable(UInt8)\')) AS subquery WHERE subquery = 1 ) WHERE subquery = 1 From 7fa4c92408c90ae93e9f4dc8b731203dc7532508 Mon Sep 17 00:00:00 2001 From: lehasm Date: Mon, 9 Aug 2021 08:48:17 +0300 Subject: [PATCH 306/599] Update docs/en/engines/database-engines/atomic.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/en/engines/database-engines/atomic.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/engines/database-engines/atomic.md b/docs/en/engines/database-engines/atomic.md index dd9689d3e10..a6fe85ef64b 100644 --- a/docs/en/engines/database-engines/atomic.md +++ b/docs/en/engines/database-engines/atomic.md @@ -29,7 +29,7 @@ CREATE TABLE name UUID '28f1c61c-2970-457a-bffe-454156ddcfef' (n UInt64) ENGINE ### DROP/DETACH TABLE {#drop-detach-table} -On `DROP TABLE` no data is removed, database `Atomic` just marks table as dropped by moving metadata to `/clickhouse_path/metadata_dropped/` and notifies background thread. Delay before final table data deletion is specify by [database_atomic_delay_before_drop_table_sec](../../operations/server-configuration-parameters/settings.md#database_atomic_delay_before_drop_table_sec) setting. +On `DROP TABLE` no data is removed, database `Atomic` just marks table as dropped by moving metadata to `/clickhouse_path/metadata_dropped/` and notifies background thread. Delay before final table data deletion is specified by the [database_atomic_delay_before_drop_table_sec](../../operations/server-configuration-parameters/settings.md#database_atomic_delay_before_drop_table_sec) setting. You can specify synchronous mode using `SYNC` modifier. Use the [database_atomic_wait_for_drop_and_detach_synchronously](../../operations/settings/settings.md#database_atomic_wait_for_drop_and_detach_synchronously) setting to do this. In this case `DROP` waits for running `SELECT`, `INSERT` and other queries which are using the table to finish. Table will be actually removed when it's not in use. ### EXCHANGE TABLES/DICTIONARIES {#exchange-tables} From dcb750bb357ce490a217b1a8755dd87957ec35a1 Mon Sep 17 00:00:00 2001 From: lehasm Date: Mon, 9 Aug 2021 08:48:45 +0300 Subject: [PATCH 307/599] Update docs/ru/engines/database-engines/atomic.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/engines/database-engines/atomic.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/engines/database-engines/atomic.md b/docs/ru/engines/database-engines/atomic.md index 5578b6d7422..8ccb3b968cf 100644 --- a/docs/ru/engines/database-engines/atomic.md +++ b/docs/ru/engines/database-engines/atomic.md @@ -25,7 +25,7 @@ CREATE TABLE name UUID '28f1c61c-2970-457a-bffe-454156ddcfef' (n UInt64) ENGINE ``` ### RENAME TABLE {#rename-table} -Запросы [RENAME](../../sql-reference/statements/rename.md) выполняются без изменения UUID и перемещения табличных данных. Эти запросы не ожидают завершения использующих таблицу запросов и будут выполнены мгновенно. +Запросы [RENAME](../../sql-reference/statements/rename.md) выполняются без изменения UUID и перемещения табличных данных. Эти запросы не ожидают завершения использующих таблицу запросов и выполняются мгновенно. ### DROP/DETACH TABLE {#drop-detach-table} From 6f149630c3eee021339a7385b675e01d50f7c4c6 Mon Sep 17 00:00:00 2001 From: lehasm Date: Mon, 9 Aug 2021 08:49:09 +0300 Subject: [PATCH 308/599] Update docs/en/engines/database-engines/atomic.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/en/engines/database-engines/atomic.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/engines/database-engines/atomic.md b/docs/en/engines/database-engines/atomic.md index a6fe85ef64b..1e555a0a502 100644 --- a/docs/en/engines/database-engines/atomic.md +++ b/docs/en/engines/database-engines/atomic.md @@ -25,7 +25,7 @@ CREATE TABLE name UUID '28f1c61c-2970-457a-bffe-454156ddcfef' (n UInt64) ENGINE ``` ### RENAME TABLE {#rename-table} -[RENAME](../../sql-reference/statements/rename.md) queries are performed without changing UUID and moving table data. These queries do not wait for the completion of queries using the table and will be executed instantly. +[RENAME](../../sql-reference/statements/rename.md) queries are performed without changing UUID and moving table data. These queries do not wait for the completion of queries using the table and are executed instantly. ### DROP/DETACH TABLE {#drop-detach-table} From 9893e0ce59a1b1def90e17eaacafee0b5db19829 Mon Sep 17 00:00:00 2001 From: lehasm Date: Mon, 9 Aug 2021 08:49:19 +0300 Subject: [PATCH 309/599] Update docs/ru/sql-reference/statements/attach.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/sql-reference/statements/attach.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/statements/attach.md b/docs/ru/sql-reference/statements/attach.md index d998f41d629..fab469ec6c8 100644 --- a/docs/ru/sql-reference/statements/attach.md +++ b/docs/ru/sql-reference/statements/attach.md @@ -3,7 +3,7 @@ toc_priority: 40 toc_title: ATTACH --- -# Выражение ATTACH {#attach} +# ATTACH {#attach} Выполняет подключение таблицы или словаря, например, при перемещении базы данных на другой сервер. From e2ba3f7fd4b22ae2c89d9e277b05613ac513d32a Mon Sep 17 00:00:00 2001 From: lehasm Date: Mon, 9 Aug 2021 08:49:29 +0300 Subject: [PATCH 310/599] Update docs/ru/sql-reference/statements/exchange.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/sql-reference/statements/exchange.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/statements/exchange.md b/docs/ru/sql-reference/statements/exchange.md index e809fc5863c..81dea27ddb6 100644 --- a/docs/ru/sql-reference/statements/exchange.md +++ b/docs/ru/sql-reference/statements/exchange.md @@ -3,7 +3,7 @@ toc_priority: 49 toc_title: EXCHANGE --- -# Выражение EXCHANGE {#exchange} +# EXCHANGE {#exchange} Атомарно обменивает имена двух таблиц или словарей. Это действие также можно выполнить с помощью запроса [RENAME](./rename.md), используя третье временное имя, но в таком случае действие неатомарно. From 7a45d32da5075b7527c328b4acaa6f7312592a1d Mon Sep 17 00:00:00 2001 From: lehasm Date: Mon, 9 Aug 2021 08:49:39 +0300 Subject: [PATCH 311/599] Update docs/ru/sql-reference/statements/rename.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/sql-reference/statements/rename.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/statements/rename.md b/docs/ru/sql-reference/statements/rename.md index 727c9d96615..f026be304c6 100644 --- a/docs/ru/sql-reference/statements/rename.md +++ b/docs/ru/sql-reference/statements/rename.md @@ -3,7 +3,7 @@ toc_priority: 48 toc_title: RENAME --- -# Выражение RENAME {#misc_operations-rename} +# RENAME {#misc_operations-rename} Переименовывает базы данных, таблицы или словари. Несколько сущностей могут быть переименованы в одном запросе. Обратите внимание, что запрос `RENAME` с несколькими сущностями это неатомарная операция. Чтобы обменять имена атомарно, используйте выражение [EXCHANGE](./exchange.md). From bf344ecf2006c7dd624f8c65b817377174a2dcc7 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 9 Aug 2021 09:47:50 +0300 Subject: [PATCH 312/599] Mark 01710_projection_fetch as long --- ...tion_fetch.reference => 01710_projection_fetch_long.reference} | 0 ...01710_projection_fetch.sql => 01710_projection_fetch_long.sql} | 0 2 files changed, 0 insertions(+), 0 deletions(-) rename tests/queries/0_stateless/{01710_projection_fetch.reference => 01710_projection_fetch_long.reference} (100%) rename tests/queries/0_stateless/{01710_projection_fetch.sql => 01710_projection_fetch_long.sql} (100%) diff --git a/tests/queries/0_stateless/01710_projection_fetch.reference b/tests/queries/0_stateless/01710_projection_fetch_long.reference similarity index 100% rename from tests/queries/0_stateless/01710_projection_fetch.reference rename to tests/queries/0_stateless/01710_projection_fetch_long.reference diff --git a/tests/queries/0_stateless/01710_projection_fetch.sql b/tests/queries/0_stateless/01710_projection_fetch_long.sql similarity index 100% rename from tests/queries/0_stateless/01710_projection_fetch.sql rename to tests/queries/0_stateless/01710_projection_fetch_long.sql From ba97ffb909f90ed8f346db6c982c9a21936ded0d Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 7 Aug 2021 12:55:03 +0300 Subject: [PATCH 313/599] Add system.mutations for database filter check --- utils/check-style/check-style | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/utils/check-style/check-style b/utils/check-style/check-style index c0f8dbfc096..a4923c3cb19 100755 --- a/utils/check-style/check-style +++ b/utils/check-style/check-style @@ -106,6 +106,7 @@ done # - system.parts_columns # - system.columns # - system.projection_parts +# - system.mutations # should have database = currentDatabase() condition # # NOTE: it is not that accuate, but at least something. @@ -113,7 +114,7 @@ tests_with_database_column=( $( find $ROOT_PATH/tests/queries -iname '*.sql' -or -iname '*.sh' -or -iname '*.py' -or -iname '*.j2' | grep -vP $EXCLUDE_DIRS | grep -v -x -e $ROOT_PATH/tests/queries/query_test.py | - xargs grep --with-filename -e system.tables -e system.parts -e system.detached_parts -e system.parts_columns -e system.columns -e system.projection_parts | cut -d: -f1 | sort -u + xargs grep --with-filename -e system.tables -e system.parts -e system.detached_parts -e system.parts_columns -e system.columns -e system.projection_parts -e system.mutations | cut -d: -f1 | sort -u ) ) for test_case in "${tests_with_database_column[@]}"; do grep -qE database.*currentDatabase "$test_case" || { @@ -122,7 +123,7 @@ for test_case in "${tests_with_database_column[@]}"; do # explicit database grep -qE "database[ ]*=[ ]*'" "$test_case" } || { - echo "Queries to system.tables/system.parts/system.detached_parts/system.parts_columns/system.columns/system.projection_parts does not have database = currentDatabase()/\$CLICKHOUSE_DATABASE condition in $test_case" + echo "Queries to system.tables/system.parts/system.detached_parts/system.parts_columns/system.columns/system.projection_parts/system.mutations does not have database = currentDatabase()/\$CLICKHOUSE_DATABASE condition in $test_case" } done From 673f3b4d8d0ccea831dfa13be8f8147a51b7c09e Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 7 Aug 2021 12:57:58 +0300 Subject: [PATCH 314/599] check-style: refactor database column filter check --- utils/check-style/check-style | 20 +++++++++++--------- 1 file changed, 11 insertions(+), 9 deletions(-) diff --git a/utils/check-style/check-style b/utils/check-style/check-style index a4923c3cb19..4363ba0cfae 100755 --- a/utils/check-style/check-style +++ b/utils/check-style/check-style @@ -100,13 +100,15 @@ for test_case in "${tests_with_query_log[@]}"; do done # Queries to: -# - system.tables -# - system.parts -# - system.detached_parts -# - system.parts_columns -# - system.columns -# - system.projection_parts -# - system.mutations +tables_with_database_column=( + system.tables + system.parts + system.detached_parts + system.parts_columns + system.columns + system.projection_parts + system.mutations +) # should have database = currentDatabase() condition # # NOTE: it is not that accuate, but at least something. @@ -114,7 +116,7 @@ tests_with_database_column=( $( find $ROOT_PATH/tests/queries -iname '*.sql' -or -iname '*.sh' -or -iname '*.py' -or -iname '*.j2' | grep -vP $EXCLUDE_DIRS | grep -v -x -e $ROOT_PATH/tests/queries/query_test.py | - xargs grep --with-filename -e system.tables -e system.parts -e system.detached_parts -e system.parts_columns -e system.columns -e system.projection_parts -e system.mutations | cut -d: -f1 | sort -u + xargs grep --with-filename $(printf -- "-e %s " "${tables_with_database_column[@]}") | cut -d: -f1 | sort -u ) ) for test_case in "${tests_with_database_column[@]}"; do grep -qE database.*currentDatabase "$test_case" || { @@ -123,7 +125,7 @@ for test_case in "${tests_with_database_column[@]}"; do # explicit database grep -qE "database[ ]*=[ ]*'" "$test_case" } || { - echo "Queries to system.tables/system.parts/system.detached_parts/system.parts_columns/system.columns/system.projection_parts/system.mutations does not have database = currentDatabase()/\$CLICKHOUSE_DATABASE condition in $test_case" + echo "Queries to ${tables_with_database_column[*]} does not have database = currentDatabase()/\$CLICKHOUSE_DATABASE condition in $test_case" } done From 8d97585da803725167cd623b88f438b5307d1124 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 7 Aug 2021 13:02:30 +0300 Subject: [PATCH 315/599] tests: add filter by database for queries to system.mutations --- tests/queries/0_stateless/00652_mergetree_mutations.sh | 4 ++-- .../0_stateless/00652_replicated_mutations_zookeeper.sh | 4 ++-- .../01049_zookeeper_synchronous_mutations_long.sql | 8 ++++---- tests/queries/0_stateless/01070_modify_ttl.sql | 4 ++-- .../01079_parallel_alter_detach_table_zookeeper.sh | 4 ++-- .../01079_parallel_alter_modify_zookeeper_long.sh | 4 ++-- .../01090_zookeeper_mutations_and_insert_quorum_long.sql | 2 +- 7 files changed, 15 insertions(+), 15 deletions(-) diff --git a/tests/queries/0_stateless/00652_mergetree_mutations.sh b/tests/queries/0_stateless/00652_mergetree_mutations.sh index 7c7117d5f75..989efd5ecec 100755 --- a/tests/queries/0_stateless/00652_mergetree_mutations.sh +++ b/tests/queries/0_stateless/00652_mergetree_mutations.sh @@ -43,7 +43,7 @@ wait_for_mutation "mutations" "mutation_7.txt" ${CLICKHOUSE_CLIENT} --query="SELECT d, x, s, m FROM mutations ORDER BY d, x" # Check the contents of the system.mutations table. ${CLICKHOUSE_CLIENT} --query="SELECT mutation_id, command, block_numbers.partition_id, block_numbers.number, parts_to_do, is_done \ - FROM system.mutations WHERE table = 'mutations' ORDER BY mutation_id" + FROM system.mutations WHERE database = '$CLICKHOUSE_DATABASE' and table = 'mutations' ORDER BY mutation_id" ${CLICKHOUSE_CLIENT} --query="DROP TABLE mutations" @@ -71,6 +71,6 @@ ${CLICKHOUSE_CLIENT} --query="INSERT INTO mutations_cleaner(x) VALUES (4)" sleep 0.1 # Check that the first mutation is cleaned -${CLICKHOUSE_CLIENT} --query="SELECT mutation_id, command, is_done FROM system.mutations WHERE table = 'mutations_cleaner' ORDER BY mutation_id" +${CLICKHOUSE_CLIENT} --query="SELECT mutation_id, command, is_done FROM system.mutations WHERE database = '$CLICKHOUSE_DATABASE' and table = 'mutations_cleaner' ORDER BY mutation_id" ${CLICKHOUSE_CLIENT} --query="DROP TABLE mutations_cleaner" diff --git a/tests/queries/0_stateless/00652_replicated_mutations_zookeeper.sh b/tests/queries/0_stateless/00652_replicated_mutations_zookeeper.sh index 3ec6e4e3e90..3995cb3348c 100755 --- a/tests/queries/0_stateless/00652_replicated_mutations_zookeeper.sh +++ b/tests/queries/0_stateless/00652_replicated_mutations_zookeeper.sh @@ -42,7 +42,7 @@ ${CLICKHOUSE_CLIENT} --query "SYSTEM SYNC REPLICA mutations_r2" ${CLICKHOUSE_CLIENT} --query="SELECT d, x, s, m FROM mutations_r2 ORDER BY d, x" # Check the contents of the system.mutations table. ${CLICKHOUSE_CLIENT} --query="SELECT mutation_id, command, block_numbers.partition_id, block_numbers.number, parts_to_do, is_done \ - FROM system.mutations WHERE table = 'mutations_r2' ORDER BY mutation_id" + FROM system.mutations WHERE database = '$CLICKHOUSE_DATABASE' and table = 'mutations_r2' ORDER BY mutation_id" ${CLICKHOUSE_CLIENT} --query="SELECT '*** Test mutations cleaner ***'" @@ -76,7 +76,7 @@ ${CLICKHOUSE_CLIENT} --query="ALTER TABLE mutations_cleaner_r1 DELETE WHERE x = sleep 1.5 # Check that the first mutation is cleaned -${CLICKHOUSE_CLIENT} --query="SELECT mutation_id, command, is_done FROM system.mutations WHERE table = 'mutations_cleaner_r2' ORDER BY mutation_id" +${CLICKHOUSE_CLIENT} --query="SELECT mutation_id, command, is_done FROM system.mutations WHERE database = '$CLICKHOUSE_DATABASE' and table = 'mutations_cleaner_r2' ORDER BY mutation_id" ${CLICKHOUSE_CLIENT} --query="DROP TABLE mutations_r1" ${CLICKHOUSE_CLIENT} --query="DROP TABLE mutations_r2" diff --git a/tests/queries/0_stateless/01049_zookeeper_synchronous_mutations_long.sql b/tests/queries/0_stateless/01049_zookeeper_synchronous_mutations_long.sql index e369b500c35..688847b5c59 100644 --- a/tests/queries/0_stateless/01049_zookeeper_synchronous_mutations_long.sql +++ b/tests/queries/0_stateless/01049_zookeeper_synchronous_mutations_long.sql @@ -13,12 +13,12 @@ SYSTEM SYNC REPLICA table_for_synchronous_mutations2; ALTER TABLE table_for_synchronous_mutations1 UPDATE v1 = v1 + 1 WHERE 1 SETTINGS mutations_sync = 2; -SELECT is_done FROM system.mutations where table = 'table_for_synchronous_mutations1'; +SELECT is_done FROM system.mutations where database = currentDatabase() and table = 'table_for_synchronous_mutations1'; -- Another mutation, just to be sure, that previous finished ALTER TABLE table_for_synchronous_mutations1 UPDATE v1 = v1 + 1 WHERE 1 SETTINGS mutations_sync = 2; -SELECT is_done FROM system.mutations where table = 'table_for_synchronous_mutations1'; +SELECT is_done FROM system.mutations where database = currentDatabase() and table = 'table_for_synchronous_mutations1'; DROP TABLE IF EXISTS table_for_synchronous_mutations1; DROP TABLE IF EXISTS table_for_synchronous_mutations2; @@ -33,11 +33,11 @@ INSERT INTO table_for_synchronous_mutations_no_replication select number, number ALTER TABLE table_for_synchronous_mutations_no_replication UPDATE v1 = v1 + 1 WHERE 1 SETTINGS mutations_sync = 2; -SELECT is_done FROM system.mutations where table = 'table_for_synchronous_mutations_no_replication'; +SELECT is_done FROM system.mutations where database = currentDatabase() and table = 'table_for_synchronous_mutations_no_replication'; -- Another mutation, just to be sure, that previous finished ALTER TABLE table_for_synchronous_mutations_no_replication UPDATE v1 = v1 + 1 WHERE 1 SETTINGS mutations_sync = 2; -SELECT is_done FROM system.mutations where table = 'table_for_synchronous_mutations_no_replication'; +SELECT is_done FROM system.mutations where database = currentDatabase() and table = 'table_for_synchronous_mutations_no_replication'; DROP TABLE IF EXISTS table_for_synchronous_mutations_no_replication; diff --git a/tests/queries/0_stateless/01070_modify_ttl.sql b/tests/queries/0_stateless/01070_modify_ttl.sql index 4e842948afe..c2746034a40 100644 --- a/tests/queries/0_stateless/01070_modify_ttl.sql +++ b/tests/queries/0_stateless/01070_modify_ttl.sql @@ -61,7 +61,7 @@ alter table ttl modify column s String ttl i % 3 = 0 ? today() - 10 : toDate('21 select i, s, t from ttl order by i; -- MATERIALIZE TTL ran only once -select count() from system.mutations where table = 'ttl' and is_done; +select count() from system.mutations where database = currentDatabase() and table = 'ttl' and is_done; select '============='; drop table if exists ttl; @@ -69,6 +69,6 @@ drop table if exists ttl; -- Nothing changed, don't run mutation create table ttl (i Int, s String ttl toDate('2000-01-02')) engine = MergeTree order by i; alter table ttl modify column s String ttl toDate('2000-01-02'); -select count() from system.mutations where table = 'ttl' and is_done; +select count() from system.mutations where database = currentDatabase() and table = 'ttl' and is_done; drop table if exists ttl; diff --git a/tests/queries/0_stateless/01079_parallel_alter_detach_table_zookeeper.sh b/tests/queries/0_stateless/01079_parallel_alter_detach_table_zookeeper.sh index 83f3196253a..3166b2153e8 100755 --- a/tests/queries/0_stateless/01079_parallel_alter_detach_table_zookeeper.sh +++ b/tests/queries/0_stateless/01079_parallel_alter_detach_table_zookeeper.sh @@ -103,8 +103,8 @@ done for i in $(seq $REPLICAS); do $CLICKHOUSE_CLIENT --query "SYSTEM SYNC REPLICA concurrent_alter_detach_$i" $CLICKHOUSE_CLIENT --query "SELECT SUM(toUInt64(value1)) > $INITIAL_SUM FROM concurrent_alter_detach_$i" - $CLICKHOUSE_CLIENT --query "SELECT COUNT() FROM system.mutations WHERE is_done=0 and table = 'concurrent_alter_detach_$i'" # all mutations have to be done - $CLICKHOUSE_CLIENT --query "SELECT * FROM system.mutations WHERE is_done=0 and table = 'concurrent_alter_detach_$i'" # all mutations have to be done + $CLICKHOUSE_CLIENT --query "SELECT COUNT() FROM system.mutations WHERE database = '$CLICKHOUSE_DATABASE' and is_done=0 and table = 'concurrent_alter_detach_$i'" # all mutations have to be done + $CLICKHOUSE_CLIENT --query "SELECT * FROM system.mutations WHERE database = '$CLICKHOUSE_DATABASE' and is_done=0 and table = 'concurrent_alter_detach_$i'" # all mutations have to be done $CLICKHOUSE_CLIENT --query "SELECT * FROM system.replication_queue WHERE table = 'concurrent_alter_detach_$i' and (type = 'ALTER_METADATA' or type = 'MUTATE_PART')" # all mutations and alters have to be done $CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS concurrent_alter_detach_$i" done diff --git a/tests/queries/0_stateless/01079_parallel_alter_modify_zookeeper_long.sh b/tests/queries/0_stateless/01079_parallel_alter_modify_zookeeper_long.sh index 9cca73b5eef..37d880bdce7 100755 --- a/tests/queries/0_stateless/01079_parallel_alter_modify_zookeeper_long.sh +++ b/tests/queries/0_stateless/01079_parallel_alter_modify_zookeeper_long.sh @@ -115,8 +115,8 @@ done for i in $(seq $REPLICAS); do $CLICKHOUSE_CLIENT --query "SYSTEM SYNC REPLICA concurrent_alter_mt_$i" $CLICKHOUSE_CLIENT --query "SELECT SUM(toUInt64(value1)) > $INITIAL_SUM FROM concurrent_alter_mt_$i" - $CLICKHOUSE_CLIENT --query "SELECT COUNT() FROM system.mutations WHERE is_done=0 and table = 'concurrent_alter_mt_$i'" # all mutations have to be done - $CLICKHOUSE_CLIENT --query "SELECT * FROM system.mutations WHERE is_done=0 and table = 'concurrent_alter_mt_$i'" + $CLICKHOUSE_CLIENT --query "SELECT COUNT() FROM system.mutations WHERE database = '$CLICKHOUSE_DATABASE' and is_done=0 and table = 'concurrent_alter_mt_$i'" # all mutations have to be done + $CLICKHOUSE_CLIENT --query "SELECT * FROM system.mutations WHERE database = '$CLICKHOUSE_DATABASE' and is_done=0 and table = 'concurrent_alter_mt_$i'" $CLICKHOUSE_CLIENT --query "SELECT * FROM system.replication_queue WHERE table = 'concurrent_alter_mt_$i' and (type = 'ALTER_METADATA' or type = 'MUTATE_PART')" $CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS concurrent_alter_mt_$i" done diff --git a/tests/queries/0_stateless/01090_zookeeper_mutations_and_insert_quorum_long.sql b/tests/queries/0_stateless/01090_zookeeper_mutations_and_insert_quorum_long.sql index d38c639458d..6eebb7de042 100644 --- a/tests/queries/0_stateless/01090_zookeeper_mutations_and_insert_quorum_long.sql +++ b/tests/queries/0_stateless/01090_zookeeper_mutations_and_insert_quorum_long.sql @@ -13,7 +13,7 @@ ALTER TABLE mutations_and_quorum1 DELETE WHERE something = 'test1' SETTINGS muta SELECT COUNT() FROM mutations_and_quorum1; SELECT COUNT() FROM mutations_and_quorum2; -SELECT COUNT() FROM system.mutations WHERE table like 'mutations_and_quorum%' and is_done = 0; +SELECT COUNT() FROM system.mutations WHERE database = currentDatabase() AND table like 'mutations_and_quorum%' and is_done = 0; DROP TABLE IF EXISTS mutations_and_quorum1; DROP TABLE IF EXISTS mutations_and_quorum2; From f46b5dbf1ad432b43fa41b5ded0779f32a91611b Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 9 Aug 2021 09:51:37 +0300 Subject: [PATCH 316/599] Increase insert_quorum_timeout in 01090_zookeeper_mutations_and_insert_quorum_long --- .../01090_zookeeper_mutations_and_insert_quorum_long.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01090_zookeeper_mutations_and_insert_quorum_long.sql b/tests/queries/0_stateless/01090_zookeeper_mutations_and_insert_quorum_long.sql index 6eebb7de042..555db6962a7 100644 --- a/tests/queries/0_stateless/01090_zookeeper_mutations_and_insert_quorum_long.sql +++ b/tests/queries/0_stateless/01090_zookeeper_mutations_and_insert_quorum_long.sql @@ -4,7 +4,7 @@ DROP TABLE IF EXISTS mutations_and_quorum2; CREATE TABLE mutations_and_quorum1 (`server_date` Date, `something` String) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_01090/mutations_and_quorum', '1') PARTITION BY toYYYYMM(server_date) ORDER BY (server_date, something); CREATE TABLE mutations_and_quorum2 (`server_date` Date, `something` String) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_01090/mutations_and_quorum', '2') PARTITION BY toYYYYMM(server_date) ORDER BY (server_date, something); -SET insert_quorum=2, insert_quorum_parallel=0; +SET insert_quorum=2, insert_quorum_parallel=0, insert_quorum_timeout=3000000; INSERT INTO mutations_and_quorum1 VALUES ('2019-01-01', 'test1'), ('2019-02-01', 'test2'), ('2019-03-01', 'test3'), ('2019-04-01', 'test4'), ('2019-05-01', 'test1'), ('2019-06-01', 'test2'), ('2019-07-01', 'test3'), ('2019-08-01', 'test4'), ('2019-09-01', 'test1'), ('2019-10-01', 'test2'), ('2019-11-01', 'test3'), ('2019-12-01', 'test4'); From 75ded13df7e2bca56b74c8e1dd151a755717a9fd Mon Sep 17 00:00:00 2001 From: Vladimir C Date: Mon, 9 Aug 2021 10:52:04 +0300 Subject: [PATCH 317/599] Minor updates in Join Table Engine docs --- docs/en/engines/table-engines/special/join.md | 4 ++-- docs/ru/engines/table-engines/special/join.md | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/en/engines/table-engines/special/join.md b/docs/en/engines/table-engines/special/join.md index 68fbede6ccc..4e4a5e9fc03 100644 --- a/docs/en/engines/table-engines/special/join.md +++ b/docs/en/engines/table-engines/special/join.md @@ -42,7 +42,7 @@ If the server restarts incorrectly, the data block on the disk might get lost or You can use `INSERT` queries to add data to the `Join`-engine tables. If the table was created with the `ANY` strictness, data for duplicate keys are ignored. With the `ALL` strictness, all rows are added. -You cannot perform a `SELECT` query directly from the table. Instead, use one of the following methods: +Main use-cases for `Join`-engine tables are following: - Place the table to the right side in a `JOIN` clause. - Call the [joinGet](../../../sql-reference/functions/other-functions.md#joinget) function, which lets you extract data from the table the same way as from a dictionary. @@ -64,7 +64,7 @@ When creating a table, the following settings are applied: The `Join`-engine tables can’t be used in `GLOBAL JOIN` operations. -The `Join`-engine allows use [join_use_nulls](../../../operations/settings/settings.md#join_use_nulls) setting in the `CREATE TABLE` statement. And [SELECT](../../../sql-reference/statements/select/index.md) query allows use `join_use_nulls` too. If you have different `join_use_nulls` settings, you can get an error joining table. It depends on kind of JOIN. When you use [joinGet](../../../sql-reference/functions/other-functions.md#joinget) function, you have to use the same `join_use_nulls` setting in `CRATE TABLE` and `SELECT` statements. +The `Join`-engine allows to specify [join_use_nulls](../../../operations/settings/settings.md#join_use_nulls) setting in the `CREATE TABLE` statement. [SELECT](../../../sql-reference/statements/select/index.md) query should have the same `join_use_nulls` value. ## Usage Examples {#example} diff --git a/docs/ru/engines/table-engines/special/join.md b/docs/ru/engines/table-engines/special/join.md index 4735637e8c1..a4d2b2af11f 100644 --- a/docs/ru/engines/table-engines/special/join.md +++ b/docs/ru/engines/table-engines/special/join.md @@ -39,7 +39,7 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] Для добавления данных в таблицы с движком `Join` используйте запрос `INSERT`. Если таблица создавалась со строгостью `ANY`, то данные с повторяющимися ключами игнорируются. Если задавалась строгость `ALL`, то добавляются все строки. -Из таблиц нельзя выбрать данные с помощью запроса `SELECT`. Вместо этого, используйте один из следующих методов: +Основные применения `Join` таблиц: - Используйте таблицу как правую в секции `JOIN`. - Используйте функцию [joinGet](../../../engines/table-engines/special/join.md#joinget), которая позволяет извлекать данные из таблицы таким же образом как из словаря. @@ -61,7 +61,7 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] Таблицы с движком `Join` нельзя использовать в операциях `GLOBAL JOIN`. -Движок `Join` позволяет использовать параметр [join_use_nulls](../../../operations/settings/settings.md#join_use_nulls) в запросе `CREATE TABLE`, который также можно использовать в запросе [SELECT](../../../engines/table-engines/special/join.md). Если у вас разные настройки `join_use_nulls`, вы можете получить сообщение об ошибке при объединении таблиц. Это зависит от типа соединения. Когда вы используете функцию [joinGet](../../../engines/table-engines/special/join.md#joinget), вам необходимо использовать один и тот же параметр `join_use_nulls` в запросах `CRATE TABLE` и `SELECT`. +Движок `Join` позволяет использовать настройку [join_use_nulls](../../../operations/settings/settings.md#join_use_nulls) в запросе `CREATE TABLE`. Необходимо использовать одно и то же значение параметра `join_use_nulls` в запросах `CRATE TABLE` и `SELECT`. ## Примеры использования {#example} From 54a1f3b241d68ae4bdfaf94991ea279be28f8817 Mon Sep 17 00:00:00 2001 From: Vladimir C Date: Mon, 9 Aug 2021 10:52:26 +0300 Subject: [PATCH 318/599] Update docs/ru/engines/table-engines/special/join.md --- docs/ru/engines/table-engines/special/join.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/ru/engines/table-engines/special/join.md b/docs/ru/engines/table-engines/special/join.md index a4d2b2af11f..28bbfe6dea4 100644 --- a/docs/ru/engines/table-engines/special/join.md +++ b/docs/ru/engines/table-engines/special/join.md @@ -41,8 +41,8 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] Основные применения `Join` таблиц: -- Используйте таблицу как правую в секции `JOIN`. -- Используйте функцию [joinGet](../../../engines/table-engines/special/join.md#joinget), которая позволяет извлекать данные из таблицы таким же образом как из словаря. +- Использование в правой части секции `JOIN`. +- Извлечение данных из таблицы таким же образом как из словаря с помощью функции [joinGet](../../../engines/table-engines/special/join.md#joinget). ### Удаление данных {#deleting-data} From 039d4673edb96df06f3626c44485592d39304819 Mon Sep 17 00:00:00 2001 From: benbiti Date: Mon, 9 Aug 2021 15:56:59 +0800 Subject: [PATCH 319/599] fix metric BackgroundMessageBrokerSchedulePoolTask --- src/Interpreters/Context.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 635af6f3cb7..bda174776e0 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -1731,7 +1731,7 @@ BackgroundSchedulePool & Context::getMessageBrokerSchedulePool() const if (!shared->message_broker_schedule_pool) shared->message_broker_schedule_pool.emplace( settings.background_message_broker_schedule_pool_size, - CurrentMetrics::BackgroundDistributedSchedulePoolTask, + CurrentMetrics::BackgroundMessageBrokerSchedulePoolTask, "BgMBSchPool"); return *shared->message_broker_schedule_pool; } From e2b59f7caed4a647fb9eda756ed9854c7e755987 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 9 Aug 2021 11:57:53 +0300 Subject: [PATCH 320/599] Add test to parallel skip list. --- tests/queries/skip_list.json | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/queries/skip_list.json b/tests/queries/skip_list.json index 579a2636ad5..a2c56bcf5b9 100644 --- a/tests/queries/skip_list.json +++ b/tests/queries/skip_list.json @@ -529,6 +529,7 @@ "01946_test_zstd_decompression_with_escape_sequence_at_the_end_of_buffer", "01946_test_wrong_host_name_access", "01213_alter_rename_with_default_zookeeper", /// Warning: Removing leftovers from table. - "02001_add_default_database_to_system_users" ///create user + "02001_add_default_database_to_system_users", ///create user + "02002_row_level_filter_bug" ///create user ] } From 904f53ba8bef8ee950d3941a79983ba95fc1fc72 Mon Sep 17 00:00:00 2001 From: Artur <613623@mail.ru> Date: Mon, 9 Aug 2021 08:58:49 +0000 Subject: [PATCH 321/599] correct reference --- .../02003_memory_limit_in_client.reference | 2 -- .../0_stateless/02003_memory_limit_in_client.sh | 11 ----------- 2 files changed, 13 deletions(-) delete mode 100755 tests/queries/0_stateless/02003_memory_limit_in_client.sh diff --git a/tests/queries/0_stateless/02003_memory_limit_in_client.reference b/tests/queries/0_stateless/02003_memory_limit_in_client.reference index 52b628cf6d5..e69de29bb2d 100644 --- a/tests/queries/0_stateless/02003_memory_limit_in_client.reference +++ b/tests/queries/0_stateless/02003_memory_limit_in_client.reference @@ -1,2 +0,0 @@ -Code: 241. DB::Exception: Memory limit (total) exceeded: would use 11.19 MiB (attempt to allocate chunk of 6462982 bytes), maximum: 1.00 B: while receiving packet from localhost:9000. (MEMORY_LIMIT_EXCEEDED) -60000 diff --git a/tests/queries/0_stateless/02003_memory_limit_in_client.sh b/tests/queries/0_stateless/02003_memory_limit_in_client.sh deleted file mode 100755 index b3e2ae57c52..00000000000 --- a/tests/queries/0_stateless/02003_memory_limit_in_client.sh +++ /dev/null @@ -1,11 +0,0 @@ -#!/usr/bin/env bash - -CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -# shellcheck source=../shell_config.sh -. "$CURDIR"/../shell_config.sh - -# Big quiery. Memory limit -${CLICKHOUSE_CLIENT} --max_memory_usage_in_client=1 --testmode --query "SELECT arrayMap(x -> range(x), range(number)) FROM numbers(1000) -- { serverError 241; }" 2>&1 | grep "Memory limit" - -# small quiery. In max untracked memory range -${CLICKHOUSE_CLIENT} --max_memory_usage_in_client=1 --testmode --query "SELECT * FROM (SELECT * FROM system.numbers LIMIT 600000) as num WHERE num.number=60000" From fb32b1a9c01a6632119b6c78ac6448a25693d8fc Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 9 Aug 2021 12:00:12 +0300 Subject: [PATCH 322/599] Update skip_list.json --- tests/queries/skip_list.json | 2 -- 1 file changed, 2 deletions(-) diff --git a/tests/queries/skip_list.json b/tests/queries/skip_list.json index fdc7c9144f3..5078dc9a256 100644 --- a/tests/queries/skip_list.json +++ b/tests/queries/skip_list.json @@ -500,8 +500,6 @@ "01902_table_function_merge_db_repr", "01946_test_zstd_decompression_with_escape_sequence_at_the_end_of_buffer", "01946_test_wrong_host_name_access", - "01213_alter_rename_with_default_zookeeper", /// Warning: Removing leftovers from table. - "02001_add_default_database_to_system_users", ///create user "01493_alter_remove_properties_zookeeper", "00510_materizlized_view_and_deduplication_zookeeper", /// static UUID "00509_extended_storage_definition_syntax_zookeeper", /// leftovers From 4a00e402ae6279e9716b087cccc5edd2da5e6381 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 9 Aug 2021 12:09:09 +0300 Subject: [PATCH 323/599] Fix spelling. --- .../MergeTree/MergeTreeRangeReader.cpp | 22 +++++++++---------- 1 file changed, 11 insertions(+), 11 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeRangeReader.cpp b/src/Storages/MergeTree/MergeTreeRangeReader.cpp index f1db130caaf..744576f158b 100644 --- a/src/Storages/MergeTree/MergeTreeRangeReader.cpp +++ b/src/Storages/MergeTree/MergeTreeRangeReader.cpp @@ -835,7 +835,7 @@ Columns MergeTreeRangeReader::continueReadingChain(ReadResult & result, size_t & return columns; } -static void checkCombindeFiltersSize(size_t bytes_in_first_filter, size_t second_filter_size) +static void checkCombinedFiltersSize(size_t bytes_in_first_filter, size_t second_filter_size) { if (bytes_in_first_filter != second_filter_size) throw Exception(ErrorCodes::LOGICAL_ERROR, @@ -845,24 +845,24 @@ static void checkCombindeFiltersSize(size_t bytes_in_first_filter, size_t second static ColumnPtr combineFilters(ColumnPtr first, ColumnPtr second) { - ConstantFilterDescription firsrt_const_descr(*first); + ConstantFilterDescription first_const_descr(*first); - if (firsrt_const_descr.always_true) + if (first_const_descr.always_true) { - checkCombindeFiltersSize(first->size(), second->size()); + checkCombinedFiltersSize(first->size(), second->size()); return second; } - if (firsrt_const_descr.always_false) + if (first_const_descr.always_false) { - checkCombindeFiltersSize(0, second->size()); + checkCombinedFiltersSize(0, second->size()); return first; } - FilterDescription firsrt_descr(*first); + FilterDescription first_descr(*first); - size_t bytes_in_first_filter = countBytesInFilter(*firsrt_descr.data); - checkCombindeFiltersSize(bytes_in_first_filter, second->size()); + size_t bytes_in_first_filter = countBytesInFilter(*first_descr.data); + checkCombinedFiltersSize(bytes_in_first_filter, second->size()); ConstantFilterDescription second_const_descr(*second); @@ -875,8 +875,8 @@ static ColumnPtr combineFilters(ColumnPtr first, ColumnPtr second) FilterDescription second_descr(*second); MutableColumnPtr mut_first; - if (firsrt_descr.data_holder) - mut_first = IColumn::mutate(std::move(firsrt_descr.data_holder)); + if (first_descr.data_holder) + mut_first = IColumn::mutate(std::move(first_descr.data_holder)); else mut_first = IColumn::mutate(std::move(first)); From 484c3a5d2ee964a9728c53f064e040bb84f64a99 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 9 Aug 2021 11:55:17 +0000 Subject: [PATCH 324/599] Update kafka test --- tests/integration/test_storage_kafka/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index 947b71b5f96..2ce1c1676ad 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -2753,7 +2753,7 @@ def test_kafka_formats_with_broken_message(kafka_cluster): # broken message "(0,'BAD','AM',0.5,1)", ], - 'expected':r'''{"raw_message":"(0,'BAD','AM',0.5,1)","error":"Cannot parse string 'BAD' as UInt16: syntax error at begin of string. Note: there are toUInt16OrZero and toUInt16OrNull functions, which returns zero\/NULL instead of throwing exception.: while executing 'FUNCTION CAST(assumeNotNull(_dummy_0) :: 2, 'UInt16' :: 1) -> CAST(assumeNotNull(_dummy_0), 'UInt16') UInt16 : 4'"}''', + 'expected':r'''{"raw_message":"(0,'BAD','AM',0.5,1)","error":"Cannot parse string 'BAD' as UInt16: syntax error at begin of string. Note: there are toUInt16OrZero and toUInt16OrNull functions, which returns zero\/NULL instead of throwing exception.: while executing 'FUNCTION _CAST(assumeNotNull(_dummy_0) :: 2, 'UInt16' :: 1) -> _CAST(assumeNotNull(_dummy_0), 'UInt16') UInt16 : 4'"}''', 'supports_empty_value': True, 'printable':True, }, From 57e5407fd84d200cbcdad497ee474e43198f2532 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 9 Aug 2021 15:19:28 +0300 Subject: [PATCH 325/599] Fix test from fuzzer. --- src/Interpreters/ExpressionActions.cpp | 7 +++++++ tests/queries/0_stateless/02003_bug_from_23515.sql | 4 ++++ 2 files changed, 11 insertions(+) diff --git a/src/Interpreters/ExpressionActions.cpp b/src/Interpreters/ExpressionActions.cpp index 6797947a101..6c299eb8930 100644 --- a/src/Interpreters/ExpressionActions.cpp +++ b/src/Interpreters/ExpressionActions.cpp @@ -342,6 +342,13 @@ static void executeAction(const ExpressionActions::Action & action, ExecutionCon res_column.type = action.node->result_type; res_column.name = action.node->result_name; + if (action.node->column) + { + /// Do not execut function if it's result is already known. + res_column.column = action.node->column->cloneResized(num_rows); + break; + } + ColumnsWithTypeAndName arguments(action.arguments.size()); for (size_t i = 0; i < arguments.size(); ++i) { diff --git a/tests/queries/0_stateless/02003_bug_from_23515.sql b/tests/queries/0_stateless/02003_bug_from_23515.sql index a1600c539c4..febd59b0755 100644 --- a/tests/queries/0_stateless/02003_bug_from_23515.sql +++ b/tests/queries/0_stateless/02003_bug_from_23515.sql @@ -3,4 +3,8 @@ create table tab (a LowCardinality(String), b LowCardinality(String)) engine = M insert into tab values ('1', 'a'), ('2', 'b'); SELECT a = '1' FROM tab WHERE a = '1' and b='a'; + +-- Fuzzed +SELECT * FROM tab WHERE (a = '1') AND 0 AND (b = 'a'); + drop table if exists tab; From 65bb4ff74409e46a612c218a9da6699d3ab719bd Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 9 Aug 2021 14:29:35 +0200 Subject: [PATCH 326/599] Unify mysql output format checks --- src/Formats/FormatFactory.cpp | 14 ++++---------- 1 file changed, 4 insertions(+), 10 deletions(-) diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index 0132a37113a..cd55c66fbdf 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -212,13 +212,11 @@ BlockOutputStreamPtr FormatFactory::getOutputStreamParallelIfPossible( const Settings & settings = context->getSettingsRef(); bool parallel_formatting = settings.output_format_parallel_formatting; + auto format_settings = _format_settings ? *_format_settings : getFormatSettings(context); - if (output_getter && parallel_formatting && getCreators(name).supports_parallel_formatting - && !settings.output_format_json_array_of_rows) + if (output_getter && parallel_formatting && getCreators(name).supports_parallel_formatting && !settings.output_format_json_array_of_rows + && !format_settings.mysql_wire.sequence_id) { - auto format_settings = _format_settings - ? *_format_settings : getFormatSettings(context); - auto formatter_creator = [output_getter, sample, callback, format_settings] (WriteBuffer & output) -> OutputFormatPtr { return output_getter(output, sample, {std::move(callback)}, format_settings);}; @@ -314,14 +312,10 @@ OutputFormatPtr FormatFactory::getOutputFormatParallelIfPossible( auto format_settings = _format_settings ? *_format_settings : getFormatSettings(context); - /// If we're handling MySQL protocol connection right now then MySQLWire is only allowed output format. - if (format_settings.mysql_wire.sequence_id && (name != "MySQLWire")) - throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "MySQL protocol does not support custom output formats"); - const Settings & settings = context->getSettingsRef(); if (settings.output_format_parallel_formatting && getCreators(name).supports_parallel_formatting - && !settings.output_format_json_array_of_rows) + && !settings.output_format_json_array_of_rows && !format_settings.mysql_wire.sequence_id) { auto formatter_creator = [output_getter, sample, callback, format_settings] (WriteBuffer & output) -> OutputFormatPtr From d3667b348addae3518dddc2954c3bc5267a17a25 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Mon, 9 Aug 2021 16:16:31 +0300 Subject: [PATCH 327/599] Update other-functions.md --- docs/en/sql-reference/functions/other-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index 9618bb04b2c..577fdd668a2 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -2222,7 +2222,7 @@ Type: [Array](../../sql-reference/data-types/array.md)([String](../../sql-refere ## defaultRoles {#default-roles} -Returns the names of the roles which are enabled by default for the current user when he logins. Initially these are all roles granted to the current user (see [GRANT](../../sql-reference/statements/grant/#grant-select)). But this list can be changed with the [SET DEFAULT ROLE](../../sql-reference/statements/set-role.md#set-default-role-statement) statement. +Returns the names of the roles which are enabled by default for the current user when he logins. Initially these are all roles granted to the current user (see [GRANT](../../sql-reference/statements/grant/#grant-select)), but that can be changed with the [SET DEFAULT ROLE](../../sql-reference/statements/set-role.md#set-default-role-statement) statement. **Syntax** From 0f5bef68aa699b3434111ce30ea104cd8324629a Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Mon, 9 Aug 2021 16:19:31 +0300 Subject: [PATCH 328/599] Update docs/en/sql-reference/functions/string-functions.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/en/sql-reference/functions/string-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index f99fd698b0e..13235019da8 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -50,7 +50,7 @@ Pads the current string from the left with a specified string (multiple times, i **Syntax** ``` sql -leftPad('string','length', 'pad_string') +leftPad('string', 'length'[, 'pad_string']) ``` **Arguments** From 0efd744c886646d6876d8850fab14af7ec97ae29 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Mon, 9 Aug 2021 16:19:39 +0300 Subject: [PATCH 329/599] Update docs/en/sql-reference/functions/string-functions.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/en/sql-reference/functions/string-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index 13235019da8..05ea7c3e0bc 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -56,7 +56,7 @@ leftPad('string', 'length'[, 'pad_string']) **Arguments** - `string` — Input string, that need to be padded. [String](../data-types/string.md). -- `length` — The length of the resulting string once the input string pads. [UInt](../data-types/int-uint.md). If the value is less than input string length, then string is returned as-is. +- `length` — The length of the resulting string. [UInt](../data-types/int-uint.md). If the value is less than the input string length, then the input string is returned as-is. - `pad_string` — The string to pad the current input string with. [String](../data-types/string.md). [String](../data-types/string.md) From aede1f4994a70ef7058e418039bbfaeb7b69f7f4 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Mon, 9 Aug 2021 16:19:49 +0300 Subject: [PATCH 330/599] Update docs/en/sql-reference/functions/string-functions.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/en/sql-reference/functions/string-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index 05ea7c3e0bc..e903e2cf8ba 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -57,7 +57,7 @@ leftPad('string', 'length'[, 'pad_string']) - `string` — Input string, that need to be padded. [String](../data-types/string.md). - `length` — The length of the resulting string. [UInt](../data-types/int-uint.md). If the value is less than the input string length, then the input string is returned as-is. -- `pad_string` — The string to pad the current input string with. [String](../data-types/string.md). +- `pad_string` — The string to pad the input string with. [String](../data-types/string.md). Optional. If not specified, then the input string is padded with spaces. [String](../data-types/string.md) From 324ed5a9b91e9a5760e8c7d78b78ed568af4354e Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Mon, 9 Aug 2021 16:20:02 +0300 Subject: [PATCH 331/599] Update docs/en/sql-reference/functions/string-functions.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/en/sql-reference/functions/string-functions.md | 1 - 1 file changed, 1 deletion(-) diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index e903e2cf8ba..7d25928e284 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -59,7 +59,6 @@ leftPad('string', 'length'[, 'pad_string']) - `length` — The length of the resulting string. [UInt](../data-types/int-uint.md). If the value is less than the input string length, then the input string is returned as-is. - `pad_string` — The string to pad the input string with. [String](../data-types/string.md). Optional. If not specified, then the input string is padded with spaces. -[String](../data-types/string.md) **Returned value(s)** From fdbe179ba5e3cf9c3fe3ce91eea5d26af34d0585 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Mon, 9 Aug 2021 16:20:13 +0300 Subject: [PATCH 332/599] Update docs/en/sql-reference/functions/string-functions.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/en/sql-reference/functions/string-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index 7d25928e284..1280fbaf158 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -62,7 +62,7 @@ leftPad('string', 'length'[, 'pad_string']) **Returned value(s)** -- The resulting string reaches the given length. +- The resulting string of the given length. Type: [String](../data-types/string.md). From dd4f48c9fd85f222ffb7a7a2e118a1b5e0f76c02 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Mon, 9 Aug 2021 16:20:20 +0300 Subject: [PATCH 333/599] Update docs/en/sql-reference/functions/string-functions.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/en/sql-reference/functions/string-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index 1280fbaf158..80c163b3cd0 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -71,7 +71,7 @@ Type: [String](../data-types/string.md). Query: ``` sql -SELECT leftPad('abc', 7, '*'); +SELECT leftPad('abc', 7, '*'), leftPad('def', 7); ``` Result: From 8f3111f2979f24757c6c9489a125e395e004647a Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Mon, 9 Aug 2021 16:20:26 +0300 Subject: [PATCH 334/599] Update docs/en/sql-reference/functions/string-functions.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/en/sql-reference/functions/string-functions.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index 80c163b3cd0..d6285b9994a 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -77,9 +77,9 @@ SELECT leftPad('abc', 7, '*'), leftPad('def', 7); Result: ``` text -┌─leftPad('abc', 7, '*')─┐ -│ ****abc │ -└────────────────────────┘ +┌─leftPad('abc', 7, '*')─┬─leftPad('def', 7)─┐ +│ ****abc │ def │ +└────────────────────────┴───────────────────┘ ``` ## leftPadUTF8 {#leftpadutf8} From 6c037273f87fea037357091ec0bfc26624543ce1 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Mon, 9 Aug 2021 16:20:32 +0300 Subject: [PATCH 335/599] Update docs/en/sql-reference/functions/string-functions.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/en/sql-reference/functions/string-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index d6285b9994a..bbd165f5aa8 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -89,7 +89,7 @@ Pads the current string from the left with a specified string (multiple times, i **Syntax** ``` sql -leftPadUTF8('string','length', 'pad_string') +leftPadUTF8('string','length'[, 'pad_string']) ``` **Arguments** From be082ff33ef35f3d8c62d36d210d56b5bc3f51d2 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Mon, 9 Aug 2021 16:20:38 +0300 Subject: [PATCH 336/599] Update docs/en/sql-reference/functions/string-functions.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/en/sql-reference/functions/string-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index bbd165f5aa8..9390819987d 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -55,7 +55,7 @@ leftPad('string', 'length'[, 'pad_string']) **Arguments** -- `string` — Input string, that need to be padded. [String](../data-types/string.md). +- `string` — Input string, that needs to be padded. [String](../data-types/string.md). - `length` — The length of the resulting string. [UInt](../data-types/int-uint.md). If the value is less than the input string length, then the input string is returned as-is. - `pad_string` — The string to pad the input string with. [String](../data-types/string.md). Optional. If not specified, then the input string is padded with spaces. From e618e3d2f4c5b9bad0d235b176de43dac553760b Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Mon, 9 Aug 2021 16:20:43 +0300 Subject: [PATCH 337/599] Update docs/en/sql-reference/functions/string-functions.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/en/sql-reference/functions/string-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index 9390819987d..5e2fe5d9d66 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -94,7 +94,7 @@ leftPadUTF8('string','length'[, 'pad_string']) **Arguments** -- `string` — Input UTF-8 string, that need to be padded. [String](../data-types/string.md). +- `string` — Input UTF-8 string, that needs to be padded. [String](../data-types/string.md). - `length` — The length of the resulting string once the input string pads. [UInt](../data-types/int-uint.md). If the value is less than input string length, then string is returned as-is. - `pad_string` — The string to pad the current input string with. [String](../data-types/string.md). From bdfd5a8896d115359e14b8b07d098dd1710efaa8 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Mon, 9 Aug 2021 16:21:09 +0300 Subject: [PATCH 338/599] Update docs/en/sql-reference/functions/string-functions.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/en/sql-reference/functions/string-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index 5e2fe5d9d66..eb04f18aa9d 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -95,7 +95,7 @@ leftPadUTF8('string','length'[, 'pad_string']) **Arguments** - `string` — Input UTF-8 string, that needs to be padded. [String](../data-types/string.md). -- `length` — The length of the resulting string once the input string pads. [UInt](../data-types/int-uint.md). If the value is less than input string length, then string is returned as-is. +- `length` — The length of the resulting string. [UInt](../data-types/int-uint.md). If the value is less than the input string length, then the input string is returned as-is. - `pad_string` — The string to pad the current input string with. [String](../data-types/string.md). [String](../data-types/string.md) From dc5aaddf2c84f8a8ccdb3a0a9e6d4f5e0aa43251 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Mon, 9 Aug 2021 16:21:14 +0300 Subject: [PATCH 339/599] Update docs/en/sql-reference/functions/string-functions.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/en/sql-reference/functions/string-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index eb04f18aa9d..1d0d4603b2b 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -96,7 +96,7 @@ leftPadUTF8('string','length'[, 'pad_string']) - `string` — Input UTF-8 string, that needs to be padded. [String](../data-types/string.md). - `length` — The length of the resulting string. [UInt](../data-types/int-uint.md). If the value is less than the input string length, then the input string is returned as-is. -- `pad_string` — The string to pad the current input string with. [String](../data-types/string.md). +- `pad_string` — The string to pad the input string with. [String](../data-types/string.md). Optional. If not specified, then the input string is padded with spaces. [String](../data-types/string.md) From 63f1f140dc8355431a74d83e83a2e53d1ef96538 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Mon, 9 Aug 2021 16:21:22 +0300 Subject: [PATCH 340/599] Update docs/en/sql-reference/functions/string-functions.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/en/sql-reference/functions/string-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index 1d0d4603b2b..e68c4c8c5c0 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -111,7 +111,7 @@ Type: [String](../data-types/string.md). Query: ``` sql -SELECT leftPadUTF8('абвг', 7, '*'); +SELECT leftPadUTF8('абвг', 7, '*'), leftPadUTF8('дежз', 7); ``` Result: From 56121b5b33e3edbe1b0c9f0b3fe85ace2269f2b2 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Mon, 9 Aug 2021 16:21:28 +0300 Subject: [PATCH 341/599] Update docs/en/sql-reference/functions/string-functions.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/en/sql-reference/functions/string-functions.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index e68c4c8c5c0..c659ba64763 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -117,9 +117,9 @@ SELECT leftPadUTF8('абвг', 7, '*'), leftPadUTF8('дежз', 7); Result: ``` text -┌─leftPadUTF8('абвг', 7, '*')─┐ -│ ***абвг │ -└─────────────────────────────┘ +┌─leftPadUTF8('абвг', 7, '*')─┬─leftPadUTF8('дежз', 7)─┐ +│ ***абвг │ дежз │ +└─────────────────────────────┴────────────────────────┘ ``` From 3cd877e9247921c543c2c66d1d14dd8e969088e0 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Mon, 9 Aug 2021 16:21:34 +0300 Subject: [PATCH 342/599] Update docs/en/sql-reference/functions/string-functions.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/en/sql-reference/functions/string-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index c659ba64763..16d47fd3e9e 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -102,7 +102,7 @@ leftPadUTF8('string','length'[, 'pad_string']) **Returned value(s)** -- The resulting UTF-8 string reaches the given length. +- The resulting UTF-8 string of the given length. Type: [String](../data-types/string.md). From 9994be7bc094376e6191bc64ae66148ca306496d Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Mon, 9 Aug 2021 16:21:59 +0300 Subject: [PATCH 343/599] Update docs/en/sql-reference/functions/string-functions.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/en/sql-reference/functions/string-functions.md | 1 - 1 file changed, 1 deletion(-) diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index 16d47fd3e9e..f141114b6b5 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -98,7 +98,6 @@ leftPadUTF8('string','length'[, 'pad_string']) - `length` — The length of the resulting string. [UInt](../data-types/int-uint.md). If the value is less than the input string length, then the input string is returned as-is. - `pad_string` — The string to pad the input string with. [String](../data-types/string.md). Optional. If not specified, then the input string is padded with spaces. -[String](../data-types/string.md) **Returned value(s)** From eda68ababc7e752b2c179e71400089bf393ba902 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Mon, 9 Aug 2021 16:22:06 +0300 Subject: [PATCH 344/599] Update docs/en/sql-reference/functions/string-functions.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/en/sql-reference/functions/string-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index f141114b6b5..c3d4f2c8531 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -84,7 +84,7 @@ Result: ## leftPadUTF8 {#leftpadutf8} -Pads the current string from the left with a specified string (multiple times, if needed) until the resulting string reaches the given length. Similarly to the MySQL `LPAD` function. +Pads the current string from the left with a specified string (multiple times, if needed) until the resulting string reaches the given length. Similarly to the MySQL `LPAD` function. While in the [leftPad](#leftpad) function the length is measured in bytes, here in the `leftPadUTF8` function it is measured in code points. **Syntax** From b5d3bc34d2e4aaf17a0c8eade86e8897e4a44f36 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 9 Aug 2021 16:27:19 +0300 Subject: [PATCH 345/599] Try fix integration tests. --- src/DataStreams/PostgreSQLBlockInputStream.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/DataStreams/PostgreSQLBlockInputStream.cpp b/src/DataStreams/PostgreSQLBlockInputStream.cpp index f4734ae0e3d..7f8949740df 100644 --- a/src/DataStreams/PostgreSQLBlockInputStream.cpp +++ b/src/DataStreams/PostgreSQLBlockInputStream.cpp @@ -73,7 +73,9 @@ void PostgreSQLSource::init(const Block & sample_block) template void PostgreSQLSource::onStart() { - tx = std::make_shared(connection_holder->get()); + if (connection_holder) + tx = std::make_shared(connection_holder->get()); + stream = std::make_unique(*tx, pqxx::from_query, std::string_view(query_str)); } From b90dc1017bfdc4e826cd3181209415bbb0b7e754 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Mon, 9 Aug 2021 16:43:10 +0300 Subject: [PATCH 346/599] fix tests --- ...map_add_map_subtract_on_map_type.reference | 54 +++++++++---------- .../01550_type_map_formats.reference | 6 +-- 2 files changed, 30 insertions(+), 30 deletions(-) diff --git a/tests/queries/0_stateless/01318_map_add_map_subtract_on_map_type.reference b/tests/queries/0_stateless/01318_map_add_map_subtract_on_map_type.reference index 96bafc2c79c..304f7407cf5 100644 --- a/tests/queries/0_stateless/01318_map_add_map_subtract_on_map_type.reference +++ b/tests/queries/0_stateless/01318_map_add_map_subtract_on_map_type.reference @@ -18,38 +18,38 @@ {1:3,2:2,8:2} {1:3,2:2,9:2} {1:3,2:2,10:2} -{1:2,2:2} Map(UInt8,UInt64) -{1:2,2:2} Map(UInt16,UInt64) -{1:2,2:2} Map(UInt32,UInt64) -{1:2,2:2} Map(UInt64,UInt64) -{1:2,2:2} Map(UInt128,UInt128) -{1:2,2:2} Map(UInt256,UInt256) -{1:2,2:2} Map(Int16,UInt64) -{1:2,2:2} Map(Int16,Int64) -{1:2,2:2} Map(Int32,Int64) -{1:2,2:2} Map(Int64,Int64) -{1:2,2:2} Map(Int128,Int128) -{1:2,2:2} Map(Int256,Int256) -{1:3.300000023841858,2:2} Map(UInt8,Float64) -{1:3.3000000000000003,2:2} Map(UInt8,Float64) +{1:2,2:2} Map(UInt8, UInt64) +{1:2,2:2} Map(UInt16, UInt64) +{1:2,2:2} Map(UInt32, UInt64) +{1:2,2:2} Map(UInt64, UInt64) +{1:2,2:2} Map(UInt128, UInt128) +{1:2,2:2} Map(UInt256, UInt256) +{1:2,2:2} Map(Int16, UInt64) +{1:2,2:2} Map(Int16, Int64) +{1:2,2:2} Map(Int32, Int64) +{1:2,2:2} Map(Int64, Int64) +{1:2,2:2} Map(Int128, Int128) +{1:2,2:2} Map(Int256, Int256) +{1:3.300000023841858,2:2} Map(UInt8, Float64) +{1:3.3000000000000003,2:2} Map(UInt8, Float64) {'a':1,'b':2} {'a':1,'b':1,'c':1} {'a':1,'b':1,'d':1} -{'a':1,'b':2} Map(String,UInt64) -{'a':1,'b':1,'c':1} Map(String,UInt64) -{'a':1,'b':1,'d':1} Map(String,UInt64) +{'a':1,'b':2} Map(String, UInt64) +{'a':1,'b':1,'c':1} Map(String, UInt64) +{'a':1,'b':1,'d':1} Map(String, UInt64) {'a':1,'b':2} {'a':1,'b':1,'c':1} {'a':1,'b':1,'d':1} -{'a':2} Map(Enum16(\'a\' = 1, \'b\' = 2),Int64) -{'b':2} Map(Enum16(\'a\' = 1, \'b\' = 2),Int64) -{'a':2} Map(Enum8(\'a\' = 1, \'b\' = 2),Int64) -{'b':2} Map(Enum8(\'a\' = 1, \'b\' = 2),Int64) -{'00000000-89ab-cdef-0123-456789abcdef':2} Map(UUID,Int64) -{'11111111-89ab-cdef-0123-456789abcdef':4} Map(UUID,Int64) +{'a':2} Map(Enum16(\'a\' = 1, \'b\' = 2), Int64) +{'b':2} Map(Enum16(\'a\' = 1, \'b\' = 2), Int64) +{'a':2} Map(Enum8(\'a\' = 1, \'b\' = 2), Int64) +{'b':2} Map(Enum8(\'a\' = 1, \'b\' = 2), Int64) +{'00000000-89ab-cdef-0123-456789abcdef':2} Map(UUID, Int64) +{'11111111-89ab-cdef-0123-456789abcdef':4} Map(UUID, Int64) {1:0,2:0} Map(UInt8,UInt64) -{1:18446744073709551615,2:18446744073709551615} Map(UInt8,UInt64) +{1:18446744073709551615,2:18446744073709551615} Map(UInt8, UInt64) {1:-1,2:-1} Map(UInt8,Int64) -{1:-1.0999999761581423,2:0} Map(UInt8,Float64) -{1:-1,2:-1} Map(UInt8,Int64) -{1:-2,2:-2,3:1} Map(UInt8,Int64) +{1:-1.0999999761581423,2:0} Map(UInt8, Float64) +{1:-1,2:-1} Map(UInt8, Int64) +{1:-2,2:-2,3:1} Map(UInt8, Int64) diff --git a/tests/queries/0_stateless/01550_type_map_formats.reference b/tests/queries/0_stateless/01550_type_map_formats.reference index ca081db75a2..998473ef63a 100644 --- a/tests/queries/0_stateless/01550_type_map_formats.reference +++ b/tests/queries/0_stateless/01550_type_map_formats.reference @@ -4,15 +4,15 @@ JSON [ { "name": "m", - "type": "Map(String,UInt32)" + "type": "Map(String, UInt32)" }, { "name": "m1", - "type": "Map(String,Date)" + "type": "Map(String, Date)" }, { "name": "m2", - "type": "Map(String,Array(UInt32))" + "type": "Map(String, Array(UInt32))" } ], From 497b54fe33618c250099787a6279441024f8ed19 Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Mon, 9 Aug 2021 16:43:17 +0300 Subject: [PATCH 347/599] Fix comments. --- .../functions/string-functions.md | 45 +++++++++---------- .../database-engines/materialized-mysql.md | 6 ++- 2 files changed, 26 insertions(+), 25 deletions(-) diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index c3d4f2c8531..47dfce53ccc 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -45,7 +45,7 @@ The result type is UInt64. ## leftPad {#leftpad} -Pads the current string from the left with a specified string (multiple times, if needed) until the resulting string reaches the given length. Similarly to the MySQL `LPAD` function. +Pads the current string from the left with spaces or a specified string (multiple times, if needed) until the resulting string reaches the given length. Similarly to the MySQL `LPAD` function. **Syntax** @@ -59,7 +59,6 @@ leftPad('string', 'length'[, 'pad_string']) - `length` — The length of the resulting string. [UInt](../data-types/int-uint.md). If the value is less than the input string length, then the input string is returned as-is. - `pad_string` — The string to pad the input string with. [String](../data-types/string.md). Optional. If not specified, then the input string is padded with spaces. - **Returned value(s)** - The resulting string of the given length. @@ -84,7 +83,7 @@ Result: ## leftPadUTF8 {#leftpadutf8} -Pads the current string from the left with a specified string (multiple times, if needed) until the resulting string reaches the given length. Similarly to the MySQL `LPAD` function. While in the [leftPad](#leftpad) function the length is measured in bytes, here in the `leftPadUTF8` function it is measured in code points. +Pads the current string from the left with spaces or a specified string (multiple times, if needed) until the resulting string reaches the given length. Similarly to the MySQL `LPAD` function. While in the [leftPad](#leftpad) function the length is measured in bytes, here in the `leftPadUTF8` function it is measured in code points. **Syntax** @@ -124,23 +123,23 @@ Result: ## rightPad {#rightpad} -Pads the current string from the right with a specified string (multiple times, if needed) until the resulting string reaches the given length. Similarly to the MySQL `RPAD` function. +Pads the current string from the right with spaces or a specified string (multiple times, if needed) until the resulting string reaches the given length. Similarly to the MySQL `RPAD` function. **Syntax** ``` sql -rightPad('string','length', 'pad_string') +rightPad('string', 'length'[, 'pad_string']) ``` **Arguments** -- `string` — Input string, that need to be padded. [String](../data-types/string.md). -- `length` — The length of the resulting string once the input string pads. [UInt](../data-types/int-uint.md). If the value is less than input string length, then string is returned as-is. -- `pad_string` — The string to pad the current input string with. [String](../data-types/string.md). +- `string` — Input string, that needs to be padded. [String](../data-types/string.md). +- `length` — The length of the resulting string. [UInt](../data-types/int-uint.md). If the value is less than the input string length, then the input string is returned as-is. +- `pad_string` — The string to pad the input string with. [String](../data-types/string.md). Optional. If not specified, then the input string is padded with spaces. **Returned value(s)** -- The resulting string reaches the given length. +- The resulting string of the given length. Type: [String](../data-types/string.md). @@ -149,36 +148,36 @@ Type: [String](../data-types/string.md). Query: ``` sql -SELECT rightPad('abc', 7, '*'); +SELECT rightPad('abc', 7, '*'), rightPad('abc', 7); ``` Result: ``` text -┌─rightPad('abc', 7, '*')─┐ -│ abc**** │ -└─────────────────────────┘ +┌─rightPad('abc', 7, '*')─┬─rightPad('abc', 7)─┐ +│ abc**** │ abc │ +└─────────────────────────┴────────────────────┘ ``` ## rightPadUTF8 {#rightpadutf8} -Pads the current UTF-8 string from the right with a specified string (multiple times, if needed) until the resulting string reaches the given length. Similarly to the MySQL `RPAD` function. +Pads the current UTF-8 string from the right with spaces or a specified string (multiple times, if needed) until the resulting string reaches the given length. Similarly to the MySQL `RPAD` function. While in the [rightPad](#rightpad) function the length is measured in bytes, here in the `rightPadUTF8` function it is measured in code points. **Syntax** ``` sql -rightPadUTF8('string','length', 'pad_string') +rightPadUTF8('string','length'[, 'pad_string']) ``` **Arguments** -- `string` — Input UTF-8 string, that need to be padded. [String](../data-types/string.md). -- `length` — The length of the resulting string once the input string pads. [UInt](../data-types/int-uint.md). If the value is less than input string length, then string is returned as-is. -- `pad_string` — The string to pad the current input string with. [String](../data-types/string.md). +- `string` — Input UTF-8 string, that needs to be padded. [String](../data-types/string.md). +- `length` — The length of the resulting string. [UInt](../data-types/int-uint.md). If the value is less than the input string length, then the input string is returned as-is. +- `pad_string` — The string to pad the input string with. [String](../data-types/string.md). Optional. If not specified, then the input string is padded with spaces. **Returned value(s)** -- The resulting UTF-8 string reaches the given length. +- The resulting UTF-8 string of the given length. Type: [String](../data-types/string.md). @@ -187,15 +186,15 @@ Type: [String](../data-types/string.md). Query: ``` sql -SELECT rightPadUTF8('абвг', 7, '*'); +SELECT rightPadUTF8('абвг', 7, '*'), rightPadUTF8('абвг', 7); ``` Result: ``` text -┌─rightPadUTF8('абвг', 7, '*')─┐ -│ абвг*** │ -└──────────────────────────────┘ +┌─rightPadUTF8('абвг', 7, '*')─┬─rightPadUTF8('абвг', 7)─┐ +│ абвг*** │ абвг │ +└──────────────────────────────┴─────────────────────────┘ ``` ## lower, lcase {#lower} diff --git a/docs/ru/engines/database-engines/materialized-mysql.md b/docs/ru/engines/database-engines/materialized-mysql.md index f5f0166c9dc..fc437ff04d4 100644 --- a/docs/ru/engines/database-engines/materialized-mysql.md +++ b/docs/ru/engines/database-engines/materialized-mysql.md @@ -1,10 +1,12 @@ --- toc_priority: 29 -toc_title: MaterializedMySQL +toc_title: "[экспериментальный] MaterializedMySQL" --- -# MaterializedMySQL {#materialized-mysql} +# [экспериментальный] MaterializedMySQL {#materialized-mysql} + +**Это экспериментальная функция, которую не следует использовать в продуктивной среде.** Создает базу данных ClickHouse со всеми таблицами, существующими в MySQL, и всеми данными в этих таблицах. From abeaf60e4ae910dd88c5fd05355d5bc78f25941c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 2 Aug 2021 13:33:16 +0200 Subject: [PATCH 348/599] Improve compatibility with non-whole-minute timezone offsets --- base/common/DateLUTImpl.h | 55 ++++++++++--------- .../01958_partial_hour_timezone.reference | 20 +++++++ .../01958_partial_hour_timezone.sql | 21 +++++++ 3 files changed, 69 insertions(+), 27 deletions(-) create mode 100644 tests/queries/0_stateless/01958_partial_hour_timezone.reference create mode 100644 tests/queries/0_stateless/01958_partial_hour_timezone.sql diff --git a/base/common/DateLUTImpl.h b/base/common/DateLUTImpl.h index 202eb88a361..06f6bbcc6fa 100644 --- a/base/common/DateLUTImpl.h +++ b/base/common/DateLUTImpl.h @@ -251,18 +251,23 @@ private: } template - static inline T roundDown(T x, Divisor divisor) + inline T roundDown(T x, Divisor divisor) const { static_assert(std::is_integral_v && std::is_integral_v); assert(divisor > 0); - if (likely(x >= 0)) - return x / divisor * divisor; + if (likely(offset_is_whole_number_of_hours_during_epoch)) + { + if (likely(x >= 0)) + return x / divisor * divisor; - /// Integer division for negative numbers rounds them towards zero (up). - /// We will shift the number so it will be rounded towards -inf (down). + /// Integer division for negative numbers rounds them towards zero (up). + /// We will shift the number so it will be rounded towards -inf (down). + return (x + 1 - divisor) / divisor * divisor; + } - return (x + 1 - divisor) / divisor * divisor; + Time date = find(x).date; + return date + (x - date) / divisor * divisor; } public: @@ -459,7 +464,21 @@ public: inline unsigned toSecond(Time t) const { - auto res = t % 60; + if (offset_is_whole_number_of_hours_during_epoch) + { + auto res = t % 60; + if (likely(res >= 0)) + return res; + return res + 60; + } + + LUTIndex index = findIndex(t); + UInt32 time = t - lut[index].date; + + if (time >= lut[index].time_at_offset_change()) + time += lut[index].amount_of_offset_change(); + + auto res = time % 60; if (likely(res >= 0)) return res; return res + 60; @@ -486,26 +505,8 @@ public: inline Time toStartOfMinute(Time t) const { return roundDown(t, 60); } inline Time toStartOfFiveMinute(Time t) const { return roundDown(t, 300); } inline Time toStartOfFifteenMinutes(Time t) const { return roundDown(t, 900); } - - inline Time toStartOfTenMinutes(Time t) const - { - if (t >= 0 && offset_is_whole_number_of_hours_during_epoch) - return t / 600 * 600; - - /// More complex logic is for Nepal - it has offset 05:45. Australia/Eucla is also unfortunate. - Time date = find(t).date; - return date + (t - date) / 600 * 600; - } - - /// NOTE: Assuming timezone transitions are multiple of hours. Lord Howe Island in Australia is a notable exception. - inline Time toStartOfHour(Time t) const - { - if (t >= 0 && offset_is_whole_number_of_hours_during_epoch) - return t / 3600 * 3600; - - Time date = find(t).date; - return date + (t - date) / 3600 * 3600; - } + inline Time toStartOfTenMinutes(Time t) const { return roundDown(t, 600); } + inline Time toStartOfHour(Time t) const { return roundDown(t, 3600); } /** Number of calendar day since the beginning of UNIX epoch (1970-01-01 is zero) * We use just two bytes for it. It covers the range up to 2105 and slightly more. diff --git a/tests/queries/0_stateless/01958_partial_hour_timezone.reference b/tests/queries/0_stateless/01958_partial_hour_timezone.reference new file mode 100644 index 00000000000..a86391b491c --- /dev/null +++ b/tests/queries/0_stateless/01958_partial_hour_timezone.reference @@ -0,0 +1,20 @@ +Row 1: +────── +toUnixTimestamp(t): 14459031 +timeZoneOffset(t): -2670 +formatDateTime(t, '%F %T', 'Africa/Monrovia'): 1970-06-17 07:39:21 +toString(t, 'Africa/Monrovia'): 1970-06-17 07:39:21 +toStartOfMinute(t): 1970-06-17 07:39:00 +toStartOfFiveMinute(t): 1970-06-17 07:35:00 +toStartOfFifteenMinutes(t): 1970-06-17 07:30:00 +toStartOfTenMinutes(t): 1970-06-17 07:30:00 +toStartOfHour(t): 1970-06-17 07:00:00 +toStartOfDay(t): 1970-06-17 00:00:00 +toStartOfWeek(t): 1970-06-14 +toStartOfInterval(t, toIntervalSecond(1)): 1970-06-17 07:39:21 +toStartOfInterval(t, toIntervalMinute(1)): 1970-06-17 07:39:00 +toStartOfInterval(t, toIntervalMinute(2)): 1970-06-17 07:38:00 +toStartOfInterval(t, toIntervalMinute(5)): 1970-06-17 07:35:00 +toStartOfInterval(t, toIntervalMinute(60)): 1970-06-17 07:00:00 +addMinutes(t, 1): 1970-06-17 07:40:21 +addMinutes(t, 60): 1970-06-17 08:39:21 diff --git a/tests/queries/0_stateless/01958_partial_hour_timezone.sql b/tests/queries/0_stateless/01958_partial_hour_timezone.sql new file mode 100644 index 00000000000..9bcb03ea4f2 --- /dev/null +++ b/tests/queries/0_stateless/01958_partial_hour_timezone.sql @@ -0,0 +1,21 @@ +-- Appeared in https://github.com/ClickHouse/ClickHouse/pull/26978#issuecomment-890889362 +WITH toDateTime('1970-06-17 07:39:21', 'Africa/Monrovia') as t +SELECT toUnixTimestamp(t), + timeZoneOffset(t), + formatDateTime(t, '%F %T', 'Africa/Monrovia'), + toString(t, 'Africa/Monrovia'), + toStartOfMinute(t), + toStartOfFiveMinute(t), + toStartOfFifteenMinutes(t), + toStartOfTenMinutes(t), + toStartOfHour(t), + toStartOfDay(t), + toStartOfWeek(t), + toStartOfInterval(t, INTERVAL 1 second), + toStartOfInterval(t, INTERVAL 1 minute), + toStartOfInterval(t, INTERVAL 2 minute), + toStartOfInterval(t, INTERVAL 5 minute), + toStartOfInterval(t, INTERVAL 60 minute), + addMinutes(t, 1), + addMinutes(t, 60) +FORMAT Vertical; From 5921f0cc821a1733d5cb5f34d21feff97e82b003 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 2 Aug 2021 15:33:49 +0200 Subject: [PATCH 349/599] Make gcc happy --- base/common/DateLUTImpl.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/base/common/DateLUTImpl.h b/base/common/DateLUTImpl.h index 06f6bbcc6fa..25d15ac01b9 100644 --- a/base/common/DateLUTImpl.h +++ b/base/common/DateLUTImpl.h @@ -466,7 +466,7 @@ public: { if (offset_is_whole_number_of_hours_during_epoch) { - auto res = t % 60; + Time res = t % 60; if (likely(res >= 0)) return res; return res + 60; @@ -478,7 +478,7 @@ public: if (time >= lut[index].time_at_offset_change()) time += lut[index].amount_of_offset_change(); - auto res = time % 60; + Time res = time % 60; if (likely(res >= 0)) return res; return res + 60; From 367b314df680ba33e8e9797760df726688bbc7eb Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 2 Aug 2021 19:34:53 +0200 Subject: [PATCH 350/599] Linter --- base/common/DateLUTImpl.h | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/base/common/DateLUTImpl.h b/base/common/DateLUTImpl.h index 25d15ac01b9..19407226bb5 100644 --- a/base/common/DateLUTImpl.h +++ b/base/common/DateLUTImpl.h @@ -473,7 +473,7 @@ public: } LUTIndex index = findIndex(t); - UInt32 time = t - lut[index].date; + Time time = t - lut[index].date; if (time >= lut[index].time_at_offset_change()) time += lut[index].amount_of_offset_change(); @@ -493,12 +493,15 @@ public: /// also make the special timezones with no whole hour offset such as 'Australia/Lord_Howe' been taken into account. LUTIndex index = findIndex(t); - UInt32 time = t - lut[index].date; + Time time = t - lut[index].date; if (time >= lut[index].time_at_offset_change()) time += lut[index].amount_of_offset_change(); - return time / 60 % 60; + Time res = time / 60 % 60; + if (likely(res >= 0)) + return res; + return res + 60; } /// NOTE: Assuming timezone offset is a multiple of 15 minutes. From 273b1c64882d303919a1e1bf6c9f98c41c3097e0 Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Mon, 9 Aug 2021 16:46:13 +0300 Subject: [PATCH 351/599] Fix comments. --- docs/ru/engines/database-engines/materialized-mysql.md | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/docs/ru/engines/database-engines/materialized-mysql.md b/docs/ru/engines/database-engines/materialized-mysql.md index fc437ff04d4..f5f0166c9dc 100644 --- a/docs/ru/engines/database-engines/materialized-mysql.md +++ b/docs/ru/engines/database-engines/materialized-mysql.md @@ -1,12 +1,10 @@ --- toc_priority: 29 -toc_title: "[экспериментальный] MaterializedMySQL" +toc_title: MaterializedMySQL --- -# [экспериментальный] MaterializedMySQL {#materialized-mysql} - -**Это экспериментальная функция, которую не следует использовать в продуктивной среде.** +# MaterializedMySQL {#materialized-mysql} Создает базу данных ClickHouse со всеми таблицами, существующими в MySQL, и всеми данными в этих таблицах. From 40e9f9d88e09c8072af05cd646799e91448ed593 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 9 Aug 2021 16:47:30 +0300 Subject: [PATCH 352/599] Fix spellign. --- src/Interpreters/ExpressionActions.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/ExpressionActions.cpp b/src/Interpreters/ExpressionActions.cpp index 6c299eb8930..ef5c1f8e48f 100644 --- a/src/Interpreters/ExpressionActions.cpp +++ b/src/Interpreters/ExpressionActions.cpp @@ -344,7 +344,7 @@ static void executeAction(const ExpressionActions::Action & action, ExecutionCon if (action.node->column) { - /// Do not execut function if it's result is already known. + /// Do not execute function if it's result is already known. res_column.column = action.node->column->cloneResized(num_rows); break; } From 62873f510770587752527823b64624c7dde286fe Mon Sep 17 00:00:00 2001 From: Artur Filatenkov <613623@mail.ru> Date: Mon, 9 Aug 2021 16:53:45 +0300 Subject: [PATCH 353/599] comment docs --- docs/en/operations/server-configuration-parameters/settings.md | 3 ++- docs/en/sql-reference/statements/create/table.md | 3 ++- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index aedd1c107c4..d7ffcff35fb 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -69,6 +69,7 @@ If no conditions met for a data part, ClickHouse uses the `lz4` compression. ``` + ## custom_settings_prefixes {#custom_settings_prefixes} List of prefixes for [custom settings](../../operations/settings/index.md#custom_settings). The prefixes must be separated with commas. diff --git a/docs/en/sql-reference/statements/create/table.md b/docs/en/sql-reference/statements/create/table.md index c20981b6bbf..d09ff24efcd 100644 --- a/docs/en/sql-reference/statements/create/table.md +++ b/docs/en/sql-reference/statements/create/table.md @@ -254,6 +254,7 @@ CREATE TABLE codec_example ENGINE = MergeTree() ``` + ## Temporary Tables {#temporary-tables} ClickHouse supports temporary tables which have the following characteristics: From 6002b7b42dd8123b9396205841529b9a5846130d Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Mon, 9 Aug 2021 11:13:19 -0300 Subject: [PATCH 354/599] Update uk-price-paid.md added example with a projection --- .../example-datasets/uk-price-paid.md | 260 +++++++++++++++++- 1 file changed, 258 insertions(+), 2 deletions(-) diff --git a/docs/en/getting-started/example-datasets/uk-price-paid.md b/docs/en/getting-started/example-datasets/uk-price-paid.md index f328ea51a24..f862ee78282 100644 --- a/docs/en/getting-started/example-datasets/uk-price-paid.md +++ b/docs/en/getting-started/example-datasets/uk-price-paid.md @@ -212,8 +212,6 @@ HAVING c >= 100 ORDER BY price DESC LIMIT 100 -Query id: df8c0a98-4713-4f0e-9690-5f73b52f7206 - ┌─town─────────────────┬─district───────────────┬────c─┬───price─┬─bar(round(avg(price)), 0, 5000000, 100)────────────────────────────┐ │ LONDON │ CITY OF WESTMINSTER │ 3372 │ 3305225 │ ██████████████████████████████████████████████████████████████████ │ │ LONDON │ CITY OF LONDON │ 257 │ 3294478 │ █████████████████████████████████████████████████████████████████▊ │ @@ -323,3 +321,261 @@ Query id: df8c0a98-4713-4f0e-9690-5f73b52f7206 ### Test it in Playground The data is uploaded to ClickHouse Playground, [example](https://gh-api.clickhouse.tech/play?user=play#U0VMRUNUIHRvd24sIGRpc3RyaWN0LCBjb3VudCgpIEFTIGMsIHJvdW5kKGF2ZyhwcmljZSkpIEFTIHByaWNlLCBiYXIocHJpY2UsIDAsIDUwMDAwMDAsIDEwMCkgRlJPTSB1a19wcmljZV9wYWlkIFdIRVJFIGRhdGUgPj0gJzIwMjAtMDEtMDEnIEdST1VQIEJZIHRvd24sIGRpc3RyaWN0IEhBVklORyBjID49IDEwMCBPUkRFUiBCWSBwcmljZSBERVNDIExJTUlUIDEwMA==). + +## Let's speed up queries using projections + +[Projections](https://../../sql-reference/statements/alter/projection/) allows to improve queries speed by storing pre-aggregated data + +### Bultd a projection + +``` +-- create an aggregating projection by dimensions (toYear(date), district, town) + +ALTER TABLE uk_price_paid + ADD PROJECTION projection_by_year_district_town + ( + SELECT + toYear(date), + district, + town, + avg(price), + sum(price), + count() + GROUP BY + toYear(date), + district, + town + ); + +-- populate the projection for existing data (without it projection will be +-- created for only newly inserted data) + +ALTER TABLE uk_price_paid + MATERIALIZE PROJECTION projection_by_year_district_town +SETTINGS mutations_sync = 1; +``` + +## Test performarnce + +Let's run the same 3 queries. + +``` +-- enable projections for selects +set allow_experimental_projection_optimization=1; + +-- Q1) Average price per year: + +SELECT + toYear(date) AS year, + round(avg(price)) AS price, + bar(price, 0, 1000000, 80) +FROM uk_price_paid +GROUP BY year +ORDER BY year ASC; + +┌─year─┬──price─┬─bar(round(avg(price)), 0, 1000000, 80)─┐ +│ 1995 │ 67932 │ █████▍ │ +│ 1996 │ 71505 │ █████▋ │ +│ 1997 │ 78532 │ ██████▎ │ +│ 1998 │ 85435 │ ██████▋ │ +│ 1999 │ 96036 │ ███████▋ │ +│ 2000 │ 107478 │ ████████▌ │ +│ 2001 │ 118886 │ █████████▌ │ +│ 2002 │ 137940 │ ███████████ │ +│ 2003 │ 155888 │ ████████████▍ │ +│ 2004 │ 178885 │ ██████████████▎ │ +│ 2005 │ 189350 │ ███████████████▏ │ +│ 2006 │ 203528 │ ████████████████▎ │ +│ 2007 │ 219377 │ █████████████████▌ │ +│ 2008 │ 217056 │ █████████████████▎ │ +│ 2009 │ 213419 │ █████████████████ │ +│ 2010 │ 236110 │ ██████████████████▊ │ +│ 2011 │ 232804 │ ██████████████████▌ │ +│ 2012 │ 238366 │ ███████████████████ │ +│ 2013 │ 256931 │ ████████████████████▌ │ +│ 2014 │ 279917 │ ██████████████████████▍ │ +│ 2015 │ 297264 │ ███████████████████████▋ │ +│ 2016 │ 313197 │ █████████████████████████ │ +│ 2017 │ 346070 │ ███████████████████████████▋ │ +│ 2018 │ 350117 │ ████████████████████████████ │ +│ 2019 │ 351010 │ ████████████████████████████ │ +│ 2020 │ 368974 │ █████████████████████████████▌ │ +│ 2021 │ 384351 │ ██████████████████████████████▋ │ +└──────┴────────┴────────────────────────────────────────┘ + +27 rows in set. Elapsed: 0.003 sec. Processed 106.87 thousand rows, 3.21 MB (31.92 million rows/s., 959.03 MB/s.) + +-- Q2) Average price per year in London: + +SELECT + toYear(date) AS year, + round(avg(price)) AS price, + bar(price, 0, 2000000, 100) +FROM uk_price_paid +WHERE town = 'LONDON' +GROUP BY year +ORDER BY year ASC; + +┌─year─┬───price─┬─bar(round(avg(price)), 0, 2000000, 100)───────────────┐ +│ 1995 │ 109112 │ █████▍ │ +│ 1996 │ 118667 │ █████▊ │ +│ 1997 │ 136518 │ ██████▋ │ +│ 1998 │ 152983 │ ███████▋ │ +│ 1999 │ 180633 │ █████████ │ +│ 2000 │ 215830 │ ██████████▋ │ +│ 2001 │ 232996 │ ███████████▋ │ +│ 2002 │ 263672 │ █████████████▏ │ +│ 2003 │ 278394 │ █████████████▊ │ +│ 2004 │ 304665 │ ███████████████▏ │ +│ 2005 │ 322875 │ ████████████████▏ │ +│ 2006 │ 356192 │ █████████████████▋ │ +│ 2007 │ 404055 │ ████████████████████▏ │ +│ 2008 │ 420741 │ █████████████████████ │ +│ 2009 │ 427754 │ █████████████████████▍ │ +│ 2010 │ 480306 │ ████████████████████████ │ +│ 2011 │ 496274 │ ████████████████████████▋ │ +│ 2012 │ 519441 │ █████████████████████████▊ │ +│ 2013 │ 616209 │ ██████████████████████████████▋ │ +│ 2014 │ 724144 │ ████████████████████████████████████▏ │ +│ 2015 │ 792112 │ ███████████████████████████████████████▌ │ +│ 2016 │ 843568 │ ██████████████████████████████████████████▏ │ +│ 2017 │ 982566 │ █████████████████████████████████████████████████▏ │ +│ 2018 │ 1016845 │ ██████████████████████████████████████████████████▋ │ +│ 2019 │ 1043277 │ ████████████████████████████████████████████████████▏ │ +│ 2020 │ 1003963 │ ██████████████████████████████████████████████████▏ │ +│ 2021 │ 940794 │ ███████████████████████████████████████████████ │ +└──────┴─────────┴───────────────────────────────────────────────────────┘ + +27 rows in set. Elapsed: 0.005 sec. Processed 106.87 thousand rows, 3.53 MB (23.49 million rows/s., 775.95 MB/s.) + +-- Q3) The most expensive neighborhoods: +-- the condition (date >= '2020-01-01') needs to be modified to match projection dimension (toYear(date) >= 2020) + +SELECT + town, + district, + count() AS c, + round(avg(price)) AS price, + bar(price, 0, 5000000, 100) +FROM uk_price_paid +WHERE toYear(date) >= 2020 +GROUP BY + town, + district +HAVING c >= 100 +ORDER BY price DESC +LIMIT 100 + +┌─town─────────────────┬─district───────────────┬────c─┬───price─┬─bar(round(avg(price)), 0, 5000000, 100)────────────────────────────┐ +│ LONDON │ CITY OF WESTMINSTER │ 3372 │ 3305225 │ ██████████████████████████████████████████████████████████████████ │ +│ LONDON │ CITY OF LONDON │ 257 │ 3294478 │ █████████████████████████████████████████████████████████████████▊ │ +│ LONDON │ KENSINGTON AND CHELSEA │ 2367 │ 2342422 │ ██████████████████████████████████████████████▋ │ +│ LEATHERHEAD │ ELMBRIDGE │ 108 │ 1927143 │ ██████████████████████████████████████▌ │ +│ VIRGINIA WATER │ RUNNYMEDE │ 142 │ 1868819 │ █████████████████████████████████████▍ │ +│ LONDON │ CAMDEN │ 2815 │ 1736788 │ ██████████████████████████████████▋ │ +│ THORNTON HEATH │ CROYDON │ 521 │ 1733051 │ ██████████████████████████████████▋ │ +│ WINDLESHAM │ SURREY HEATH │ 103 │ 1717255 │ ██████████████████████████████████▎ │ +│ BARNET │ ENFIELD │ 115 │ 1503458 │ ██████████████████████████████ │ +│ OXFORD │ SOUTH OXFORDSHIRE │ 298 │ 1275200 │ █████████████████████████▌ │ +│ LONDON │ ISLINGTON │ 2458 │ 1274308 │ █████████████████████████▍ │ +│ COBHAM │ ELMBRIDGE │ 364 │ 1260005 │ █████████████████████████▏ │ +│ LONDON │ HOUNSLOW │ 618 │ 1215682 │ ████████████████████████▎ │ +│ ASCOT │ WINDSOR AND MAIDENHEAD │ 379 │ 1215146 │ ████████████████████████▎ │ +│ LONDON │ RICHMOND UPON THAMES │ 654 │ 1207551 │ ████████████████████████▏ │ +│ BEACONSFIELD │ BUCKINGHAMSHIRE │ 307 │ 1186220 │ ███████████████████████▋ │ +│ RICHMOND │ RICHMOND UPON THAMES │ 805 │ 1100420 │ ██████████████████████ │ +│ LONDON │ HAMMERSMITH AND FULHAM │ 2888 │ 1062959 │ █████████████████████▎ │ +│ WEYBRIDGE │ ELMBRIDGE │ 607 │ 1027161 │ ████████████████████▌ │ +│ RADLETT │ HERTSMERE │ 265 │ 1015896 │ ████████████████████▎ │ +│ SALCOMBE │ SOUTH HAMS │ 124 │ 1014393 │ ████████████████████▎ │ +│ BURFORD │ WEST OXFORDSHIRE │ 102 │ 993100 │ ███████████████████▋ │ +│ ESHER │ ELMBRIDGE │ 454 │ 969770 │ ███████████████████▍ │ +│ HINDHEAD │ WAVERLEY │ 128 │ 967786 │ ███████████████████▎ │ +│ BROCKENHURST │ NEW FOREST │ 121 │ 967046 │ ███████████████████▎ │ +│ LEATHERHEAD │ GUILDFORD │ 191 │ 964489 │ ███████████████████▎ │ +│ GERRARDS CROSS │ BUCKINGHAMSHIRE │ 376 │ 958555 │ ███████████████████▏ │ +│ EAST MOLESEY │ ELMBRIDGE │ 181 │ 943457 │ ██████████████████▋ │ +│ OLNEY │ MILTON KEYNES │ 220 │ 942892 │ ██████████████████▋ │ +│ CHALFONT ST GILES │ BUCKINGHAMSHIRE │ 135 │ 926950 │ ██████████████████▌ │ +│ HENLEY-ON-THAMES │ SOUTH OXFORDSHIRE │ 509 │ 905732 │ ██████████████████ │ +│ KINGSTON UPON THAMES │ KINGSTON UPON THAMES │ 889 │ 899689 │ █████████████████▊ │ +│ BELVEDERE │ BEXLEY │ 313 │ 895336 │ █████████████████▊ │ +│ CRANBROOK │ TUNBRIDGE WELLS │ 404 │ 888190 │ █████████████████▋ │ +│ LONDON │ EALING │ 2460 │ 865893 │ █████████████████▎ │ +│ MAIDENHEAD │ BUCKINGHAMSHIRE │ 114 │ 863814 │ █████████████████▎ │ +│ LONDON │ MERTON │ 1958 │ 857192 │ █████████████████▏ │ +│ GUILDFORD │ WAVERLEY │ 131 │ 854447 │ █████████████████ │ +│ LONDON │ HACKNEY │ 3088 │ 846571 │ ████████████████▊ │ +│ LYMM │ WARRINGTON │ 285 │ 839920 │ ████████████████▋ │ +│ HARPENDEN │ ST ALBANS │ 606 │ 836994 │ ████████████████▋ │ +│ LONDON │ WANDSWORTH │ 6113 │ 832292 │ ████████████████▋ │ +│ LONDON │ SOUTHWARK │ 3612 │ 831319 │ ████████████████▋ │ +│ BERKHAMSTED │ DACORUM │ 502 │ 830356 │ ████████████████▌ │ +│ KINGS LANGLEY │ DACORUM │ 137 │ 821358 │ ████████████████▍ │ +│ TONBRIDGE │ TUNBRIDGE WELLS │ 339 │ 806736 │ ████████████████▏ │ +│ EPSOM │ REIGATE AND BANSTEAD │ 157 │ 805903 │ ████████████████ │ +│ WOKING │ GUILDFORD │ 161 │ 803283 │ ████████████████ │ +│ STOCKBRIDGE │ TEST VALLEY │ 168 │ 801973 │ ████████████████ │ +│ TEDDINGTON │ RICHMOND UPON THAMES │ 539 │ 798591 │ ███████████████▊ │ +│ OXFORD │ VALE OF WHITE HORSE │ 329 │ 792907 │ ███████████████▋ │ +│ LONDON │ BARNET │ 3624 │ 789583 │ ███████████████▋ │ +│ TWICKENHAM │ RICHMOND UPON THAMES │ 1090 │ 787760 │ ███████████████▋ │ +│ LUTON │ CENTRAL BEDFORDSHIRE │ 196 │ 786051 │ ███████████████▋ │ +│ TONBRIDGE │ MAIDSTONE │ 277 │ 785746 │ ███████████████▋ │ +│ TOWCESTER │ WEST NORTHAMPTONSHIRE │ 186 │ 783532 │ ███████████████▋ │ +│ LONDON │ LAMBETH │ 4832 │ 783422 │ ███████████████▋ │ +│ LUTTERWORTH │ HARBOROUGH │ 515 │ 781775 │ ███████████████▋ │ +│ WOODSTOCK │ WEST OXFORDSHIRE │ 135 │ 777499 │ ███████████████▌ │ +│ ALRESFORD │ WINCHESTER │ 196 │ 775577 │ ███████████████▌ │ +│ LONDON │ NEWHAM │ 2942 │ 768551 │ ███████████████▎ │ +│ ALDERLEY EDGE │ CHESHIRE EAST │ 168 │ 768280 │ ███████████████▎ │ +│ MARLOW │ BUCKINGHAMSHIRE │ 301 │ 762784 │ ███████████████▎ │ +│ BILLINGSHURST │ CHICHESTER │ 134 │ 760920 │ ███████████████▏ │ +│ LONDON │ TOWER HAMLETS │ 4183 │ 759635 │ ███████████████▏ │ +│ MIDHURST │ CHICHESTER │ 245 │ 759101 │ ███████████████▏ │ +│ THAMES DITTON │ ELMBRIDGE │ 227 │ 753347 │ ███████████████ │ +│ POTTERS BAR │ WELWYN HATFIELD │ 163 │ 752926 │ ███████████████ │ +│ REIGATE │ REIGATE AND BANSTEAD │ 555 │ 740961 │ ██████████████▋ │ +│ TADWORTH │ REIGATE AND BANSTEAD │ 477 │ 738997 │ ██████████████▋ │ +│ SEVENOAKS │ SEVENOAKS │ 1074 │ 734658 │ ██████████████▋ │ +│ PETWORTH │ CHICHESTER │ 138 │ 732432 │ ██████████████▋ │ +│ BOURNE END │ BUCKINGHAMSHIRE │ 127 │ 730742 │ ██████████████▌ │ +│ PURLEY │ CROYDON │ 540 │ 727721 │ ██████████████▌ │ +│ OXTED │ TANDRIDGE │ 320 │ 726078 │ ██████████████▌ │ +│ LONDON │ HARINGEY │ 2988 │ 724573 │ ██████████████▍ │ +│ BANSTEAD │ REIGATE AND BANSTEAD │ 373 │ 713834 │ ██████████████▎ │ +│ PINNER │ HARROW │ 480 │ 712166 │ ██████████████▏ │ +│ MALMESBURY │ WILTSHIRE │ 293 │ 707747 │ ██████████████▏ │ +│ RICKMANSWORTH │ THREE RIVERS │ 732 │ 705400 │ ██████████████ │ +│ SLOUGH │ BUCKINGHAMSHIRE │ 359 │ 705002 │ ██████████████ │ +│ GREAT MISSENDEN │ BUCKINGHAMSHIRE │ 214 │ 704904 │ ██████████████ │ +│ READING │ SOUTH OXFORDSHIRE │ 295 │ 701697 │ ██████████████ │ +│ HYTHE │ FOLKESTONE AND HYTHE │ 457 │ 700334 │ ██████████████ │ +│ WELWYN │ WELWYN HATFIELD │ 217 │ 699649 │ █████████████▊ │ +│ CHIGWELL │ EPPING FOREST │ 242 │ 697869 │ █████████████▊ │ +│ BARNET │ BARNET │ 906 │ 695680 │ █████████████▊ │ +│ HASLEMERE │ CHICHESTER │ 120 │ 694028 │ █████████████▊ │ +│ LEATHERHEAD │ MOLE VALLEY │ 748 │ 692026 │ █████████████▋ │ +│ LONDON │ BRENT │ 1945 │ 690799 │ █████████████▋ │ +│ HASLEMERE │ WAVERLEY │ 258 │ 690765 │ █████████████▋ │ +│ NORTHWOOD │ HILLINGDON │ 252 │ 690753 │ █████████████▋ │ +│ WALTON-ON-THAMES │ ELMBRIDGE │ 871 │ 689431 │ █████████████▋ │ +│ INGATESTONE │ BRENTWOOD │ 150 │ 688345 │ █████████████▋ │ +│ OXFORD │ OXFORD │ 1761 │ 686114 │ █████████████▋ │ +│ CHISLEHURST │ BROMLEY │ 410 │ 682892 │ █████████████▋ │ +│ KINGS LANGLEY │ THREE RIVERS │ 109 │ 682320 │ █████████████▋ │ +│ ASHTEAD │ MOLE VALLEY │ 280 │ 680483 │ █████████████▌ │ +│ WOKING │ SURREY HEATH │ 269 │ 679035 │ █████████████▌ │ +│ ASCOT │ BRACKNELL FOREST │ 160 │ 678632 │ █████████████▌ │ +└──────────────────────┴────────────────────────┴──────┴─────────┴────────────────────────────────────────────────────────────────────┘ + +100 rows in set. Elapsed: 0.005 sec. Processed 12.85 thousand rows, 813.40 KB (2.73 million rows/s., 172.95 MB/s.) +``` + +All 3 queries work much faster and read fewer rows. + +``` +Q1) +no projection: 27 rows in set. Elapsed: 0.027 sec. Processed 26.25 million rows, 157.49 MB (955.96 million rows/s., 5.74 GB/s.) + projection: 27 rows in set. Elapsed: 0.003 sec. Processed 106.87 thousand rows, 3.21 MB (31.92 million rows/s., 959.03 MB/s.) +``` From 0771329bf31d5a3c91c8cd97efa66ea8df75a11c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 9 Aug 2021 16:16:17 +0200 Subject: [PATCH 355/599] Improvements based on review --- base/common/DateLUTImpl.cpp | 4 ++++ base/common/DateLUTImpl.h | 10 ++++------ 2 files changed, 8 insertions(+), 6 deletions(-) diff --git a/base/common/DateLUTImpl.cpp b/base/common/DateLUTImpl.cpp index e7faeb63760..472f24f3805 100644 --- a/base/common/DateLUTImpl.cpp +++ b/base/common/DateLUTImpl.cpp @@ -60,6 +60,7 @@ DateLUTImpl::DateLUTImpl(const std::string & time_zone_) offset_at_start_of_epoch = cctz_time_zone.lookup(cctz_time_zone.lookup(epoch).pre).offset; offset_at_start_of_lut = cctz_time_zone.lookup(cctz_time_zone.lookup(lut_start).pre).offset; offset_is_whole_number_of_hours_during_epoch = true; + offset_is_whole_number_of_minutes_during_epoch = true; cctz::civil_day date = lut_start; @@ -108,6 +109,9 @@ DateLUTImpl::DateLUTImpl(const std::string & time_zone_) if (offset_is_whole_number_of_hours_during_epoch && start_of_day > 0 && start_of_day % 3600) offset_is_whole_number_of_hours_during_epoch = false; + if (offset_is_whole_number_of_minutes_during_epoch && start_of_day > 0 && start_of_day % 60) + offset_is_whole_number_of_minutes_during_epoch = false; + /// If UTC offset was changed this day. /// Change in time zone without transition is possible, e.g. Moscow 1991 Sun, 31 Mar, 02:00 MSK to EEST cctz::time_zone::civil_transition transition{}; diff --git a/base/common/DateLUTImpl.h b/base/common/DateLUTImpl.h index 19407226bb5..85d8403df93 100644 --- a/base/common/DateLUTImpl.h +++ b/base/common/DateLUTImpl.h @@ -193,6 +193,7 @@ private: /// UTC offset at the beginning of the first supported year. Time offset_at_start_of_lut; bool offset_is_whole_number_of_hours_during_epoch; + bool offset_is_whole_number_of_minutes_during_epoch; /// Time zone name. std::string time_zone; @@ -464,7 +465,7 @@ public: inline unsigned toSecond(Time t) const { - if (offset_is_whole_number_of_hours_during_epoch) + if (likely(offset_is_whole_number_of_minutes_during_epoch)) { Time res = t % 60; if (likely(res >= 0)) @@ -478,15 +479,12 @@ public: if (time >= lut[index].time_at_offset_change()) time += lut[index].amount_of_offset_change(); - Time res = time % 60; - if (likely(res >= 0)) - return res; - return res + 60; + return time % 60; } inline unsigned toMinute(Time t) const { - if (t >= 0 && offset_is_whole_number_of_hours_during_epoch) + if (likely(t >= 0 && offset_is_whole_number_of_hours_during_epoch)) return (t / 60) % 60; /// To consider the DST changing situation within this day From d8434539a75f18536c2581dbdb34964a359edbbe Mon Sep 17 00:00:00 2001 From: Artur Filatenkov <613623@mail.ru> Date: Mon, 9 Aug 2021 16:53:45 +0300 Subject: [PATCH 356/599] comment docs --- docs/en/operations/server-configuration-parameters/settings.md | 3 ++- docs/en/sql-reference/statements/create/table.md | 3 ++- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index aedd1c107c4..d7ffcff35fb 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -69,6 +69,7 @@ If no conditions met for a data part, ClickHouse uses the `lz4` compression. ``` + ## custom_settings_prefixes {#custom_settings_prefixes} List of prefixes for [custom settings](../../operations/settings/index.md#custom_settings). The prefixes must be separated with commas. diff --git a/docs/en/sql-reference/statements/create/table.md b/docs/en/sql-reference/statements/create/table.md index c20981b6bbf..d09ff24efcd 100644 --- a/docs/en/sql-reference/statements/create/table.md +++ b/docs/en/sql-reference/statements/create/table.md @@ -254,6 +254,7 @@ CREATE TABLE codec_example ENGINE = MergeTree() ``` + ## Temporary Tables {#temporary-tables} ClickHouse supports temporary tables which have the following characteristics: From fef50918ec5be7bec1945f4a724183d1fc02a62f Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Mon, 9 Aug 2021 11:17:35 -0300 Subject: [PATCH 357/599] Update uk-price-paid.md --- docs/en/getting-started/example-datasets/uk-price-paid.md | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/en/getting-started/example-datasets/uk-price-paid.md b/docs/en/getting-started/example-datasets/uk-price-paid.md index f862ee78282..948ebd35b21 100644 --- a/docs/en/getting-started/example-datasets/uk-price-paid.md +++ b/docs/en/getting-started/example-datasets/uk-price-paid.md @@ -324,12 +324,12 @@ The data is uploaded to ClickHouse Playground, [example](https://gh-api.clickhou ## Let's speed up queries using projections -[Projections](https://../../sql-reference/statements/alter/projection/) allows to improve queries speed by storing pre-aggregated data +[Projections](https://../../sql-reference/statements/alter/projection/) allow to improve queries speed by storing pre-aggregated data. -### Bultd a projection +### Build a projection ``` --- create an aggregating projection by dimensions (toYear(date), district, town) +-- create an aggregate projection by dimensions (toYear(date), district, town) ALTER TABLE uk_price_paid ADD PROJECTION projection_by_year_district_town @@ -355,7 +355,7 @@ ALTER TABLE uk_price_paid SETTINGS mutations_sync = 1; ``` -## Test performarnce +## Test performance Let's run the same 3 queries. From 79fb2f4014a4bd2c005819a79b222c08a84b7525 Mon Sep 17 00:00:00 2001 From: Vladimir Goncharov Date: Mon, 9 Aug 2021 17:22:57 +0300 Subject: [PATCH 358/599] Create projection.md --- .../statements/alter/projection.md | 23 +++++++++++++++++++ 1 file changed, 23 insertions(+) create mode 100644 docs/ru/sql-reference/statements/alter/projection.md diff --git a/docs/ru/sql-reference/statements/alter/projection.md b/docs/ru/sql-reference/statements/alter/projection.md new file mode 100644 index 00000000000..a3d829d21e4 --- /dev/null +++ b/docs/ru/sql-reference/statements/alter/projection.md @@ -0,0 +1,23 @@ +--- +toc_priority: 49 +toc_title: PROJECTION +--- + +# Манипуляции с проекциями {#manipulations-with-projections} + +Доступны следующие операции: + +- `ALTER TABLE [db].name ADD PROJECTION name AS SELECT [GROUP BY] [ORDER BY]` - добавляет описание проекции в метаданные. + +- `ALTER TABLE [db].name DROP PROJECTION name` - удаляет описание проекции из метаданных и удаляет файлы проекции с диска. + +- `ALTER TABLE [db.]table MATERIALIZE PROJECTION name IN PARTITION partition_name` - перестраивает проекцию в указанной партиции. Реализовано как [мутация](../../../sql-reference/statements/alter/index.md#mutations). + +- `ALTER TABLE [db.]table CLEAR PROJECTION name IN PARTITION partition_name` - удаляет файлы проекции с диска без удаления описания. + +Комманды ADD, DROP и CLEAR - легковесны потому что они только меняют метаданные или удаляют файлы. + +Также комманды реплицируется, синхронизируя описание проекций в метаданных с помощью ZooKeeper. + +!!! note "Note" + Манипуляции с проекциями поддерживаются только для таблиц с движком [`*MergeTree`](../../../engines/table-engines/mergetree-family/mergetree.md) (включая [replicated](../../../engines/table-engines/mergetree-family/replication.md) варианты). From 261efb4c49270984c5fa673d9e2558613af0d062 Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Mon, 9 Aug 2021 17:29:26 +0300 Subject: [PATCH 359/599] Add Ru for pads functions. --- .../functions/string-functions.md | 20 ++- .../functions/string-functions.md | 152 ++++++++++++++++++ 2 files changed, 161 insertions(+), 11 deletions(-) diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index 47dfce53ccc..038995c5883 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -59,7 +59,7 @@ leftPad('string', 'length'[, 'pad_string']) - `length` — The length of the resulting string. [UInt](../data-types/int-uint.md). If the value is less than the input string length, then the input string is returned as-is. - `pad_string` — The string to pad the input string with. [String](../data-types/string.md). Optional. If not specified, then the input string is padded with spaces. -**Returned value(s)** +**Returned value** - The resulting string of the given length. @@ -93,14 +93,13 @@ leftPadUTF8('string','length'[, 'pad_string']) **Arguments** -- `string` — Input UTF-8 string, that needs to be padded. [String](../data-types/string.md). +- `string` — Input string, that needs to be padded. [String](../data-types/string.md). - `length` — The length of the resulting string. [UInt](../data-types/int-uint.md). If the value is less than the input string length, then the input string is returned as-is. - `pad_string` — The string to pad the input string with. [String](../data-types/string.md). Optional. If not specified, then the input string is padded with spaces. +**Returned value** -**Returned value(s)** - -- The resulting UTF-8 string of the given length. +- The resulting string of the given length. Type: [String](../data-types/string.md). @@ -118,7 +117,6 @@ Result: ┌─leftPadUTF8('абвг', 7, '*')─┬─leftPadUTF8('дежз', 7)─┐ │ ***абвг │ дежз │ └─────────────────────────────┴────────────────────────┘ - ``` ## rightPad {#rightpad} @@ -137,7 +135,7 @@ rightPad('string', 'length'[, 'pad_string']) - `length` — The length of the resulting string. [UInt](../data-types/int-uint.md). If the value is less than the input string length, then the input string is returned as-is. - `pad_string` — The string to pad the input string with. [String](../data-types/string.md). Optional. If not specified, then the input string is padded with spaces. -**Returned value(s)** +**Returned value** - The resulting string of the given length. @@ -161,7 +159,7 @@ Result: ## rightPadUTF8 {#rightpadutf8} -Pads the current UTF-8 string from the right with spaces or a specified string (multiple times, if needed) until the resulting string reaches the given length. Similarly to the MySQL `RPAD` function. While in the [rightPad](#rightpad) function the length is measured in bytes, here in the `rightPadUTF8` function it is measured in code points. +Pads the current string from the right with spaces or a specified string (multiple times, if needed) until the resulting string reaches the given length. Similarly to the MySQL `RPAD` function. While in the [rightPad](#rightpad) function the length is measured in bytes, here in the `rightPadUTF8` function it is measured in code points. **Syntax** @@ -171,13 +169,13 @@ rightPadUTF8('string','length'[, 'pad_string']) **Arguments** -- `string` — Input UTF-8 string, that needs to be padded. [String](../data-types/string.md). +- `string` — Input string, that needs to be padded. [String](../data-types/string.md). - `length` — The length of the resulting string. [UInt](../data-types/int-uint.md). If the value is less than the input string length, then the input string is returned as-is. - `pad_string` — The string to pad the input string with. [String](../data-types/string.md). Optional. If not specified, then the input string is padded with spaces. -**Returned value(s)** +**Returned value** -- The resulting UTF-8 string of the given length. +- The resulting string of the given length. Type: [String](../data-types/string.md). diff --git a/docs/ru/sql-reference/functions/string-functions.md b/docs/ru/sql-reference/functions/string-functions.md index b587a991db1..1a666c6a0cc 100644 --- a/docs/ru/sql-reference/functions/string-functions.md +++ b/docs/ru/sql-reference/functions/string-functions.md @@ -39,6 +39,158 @@ toc_title: "Функции для работы со строками" Возвращает длину строки в кодовых точках Unicode (не символах), при допущении, что строка содержит набор байтов, являющийся текстом в кодировке UTF-8. Если допущение не выполнено, возвращает какой-нибудь результат (не кидает исключение). Тип результата — UInt64. +## leftPad {#leftpad} + +Заполняет текущую строку слева пробелами или указанной строкой (несколько раз, если необходимо), пока результирующая строка не достигнет заданной длины. Соответствует функции MySQL `LPAD`. + +**Синтаксис** + +``` sql +leftPad('string', 'length'[, 'pad_string']) +``` + +**Параметры** + +- `string` — входная строка, которую необходимо заполнить. [String](../data-types/string.md). +- `length` — длина результирующей строки. [UInt](../data-types/int-uint.md). Если указанное значение меньше, чем длина входной строки, то входная строка возвращается как есть. +- `pad_string` — строка, используемая для заполнения входной строки. [String](../data-types/string.md). Необязательный параметр. Если не указано, то входная строка заполняется пробелами. + +**Возвращаемое значение** + +- Результирующая строка заданной длины. + +Type: [String](../data-types/string.md). + +**Пример** + +Запрос: + +``` sql +SELECT leftPad('abc', 7, '*'), leftPad('def', 7); +``` + +Результат: + +``` text +┌─leftPad('abc', 7, '*')─┬─leftPad('def', 7)─┐ +│ ****abc │ def │ +└────────────────────────┴───────────────────┘ +``` + +## leftPadUTF8 {#leftpadutf8} + +Заполняет текущую строку слева пробелами или указанной строкой (несколько раз, если необходимо), пока результирующая строка не достигнет заданной длины. Соответствует функции MySQL `LPAD`. Функция [leftPad](#leftpad) измеряет длину строки в байтах, а функция `leftPadUTF8` — в кодовых точках Unicode. + +**Синтаксис** + +``` sql +leftPadUTF8('string','length'[, 'pad_string']) +``` + +**Параметры** + +- `string` — входная строка, которую необходимо заполнить. [String](../data-types/string.md). +- `length` — длина результирующей строки. [UInt](../data-types/int-uint.md). Если указанное значение меньше, чем длина входной строки, то входная строка возвращается как есть. +- `pad_string` — строка, используемая для заполнения входной строки. [String](../data-types/string.md). Необязательный параметр. Если не указано, то входная строка заполняется пробелами. + +**Возвращаемое значение** + +- Результирующая строка заданной длины. + +Type: [String](../data-types/string.md). + +**Пример** + +Запрос: + +``` sql +SELECT leftPadUTF8('абвг', 7, '*'), leftPadUTF8('дежз', 7); +``` + +Результат: + +``` text +┌─leftPadUTF8('абвг', 7, '*')─┬─leftPadUTF8('дежз', 7)─┐ +│ ***абвг │ дежз │ +└─────────────────────────────┴────────────────────────┘ +``` + +## rightPad {#rightpad} + +Заполняет текущую строку справа пробелами или указанной строкой (несколько раз, если необходимо), пока результирующая строка не достигнет заданной длины. Соответствует функции MySQL `RPAD`. + +**Синтаксис** + +``` sql +rightPad('string', 'length'[, 'pad_string']) +``` + +**Параметры** + +- `string` — входная строка, которую необходимо заполнить. [String](../data-types/string.md). +- `length` — длина результирующей строки. [UInt](../data-types/int-uint.md). Если указанное значение меньше, чем длина входной строки, то входная строка возвращается как есть. +- `pad_string` — строка, используемая для заполнения входной строки. [String](../data-types/string.md). Необязательный параметр. Если не указано, то входная строка заполняется пробелами. + +**Возвращаемое значение** + +- Результирующая строка заданной длины. + +Type: [String](../data-types/string.md). + +**Пример** + +Запрос: + +``` sql +SELECT rightPad('abc', 7, '*'), rightPad('abc', 7); +``` + +Результат: + +``` text +┌─rightPad('abc', 7, '*')─┬─rightPad('abc', 7)─┐ +│ abc**** │ abc │ +└─────────────────────────┴────────────────────┘ +``` + +## rightPadUTF8 {#rightpadutf8} + +Заполняет текущую строку слева пробелами или указанной строкой (несколько раз, если необходимо), пока результирующая строка не достигнет заданной длины. Соответствует функции MySQL `RPAD`. Функция [rightPad](#rightpad) измеряет длину строки в байтах, а функция `rightPadUTF8` — в кодовых точках Unicode. + +**Синтаксис** + +``` sql +rightPadUTF8('string','length'[, 'pad_string']) +``` + +**Параметры** + +- `string` — входная строка, которую необходимо заполнить. [String](../data-types/string.md). +- `length` — длина результирующей строки. [UInt](../data-types/int-uint.md). Если указанное значение меньше, чем длина входной строки, то входная строка возвращается как есть. +- `pad_string` — строка, используемая для заполнения входной строки. [String](../data-types/string.md). Необязательный параметр. Если не указано, то входная строка заполняется пробелами. + +**Возвращаемое значение** + +- Результирующая строка заданной длины. + +Type: [String](../data-types/string.md). + +**Пример** + +Результат: + +``` sql +SELECT rightPadUTF8('абвг', 7, '*'), rightPadUTF8('абвг', 7); +``` + +Результат: + +``` text +┌─rightPadUTF8('абвг', 7, '*')─┬─rightPadUTF8('абвг', 7)─┐ +│ абвг*** │ абвг │ +└──────────────────────────────┴─────────────────────────┘ +``` + ## lower, lcase {#lower} Переводит ASCII-символы латиницы в строке в нижний регистр. From 7498a1eb3892a220ef278ee04475b1516107c396 Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Mon, 9 Aug 2021 17:30:14 +0300 Subject: [PATCH 360/599] Fix tests and style --- .../Formats/Impl/ArrowColumnToCHColumn.cpp | 74 +++++++++---------- .../Formats/Impl/ArrowColumnToCHColumn.h | 5 +- 2 files changed, 40 insertions(+), 39 deletions(-) diff --git a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp index 772e8a51d22..db7d4df7165 100644 --- a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp +++ b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp @@ -16,7 +16,6 @@ #include #include #include -#include #include #include #include @@ -60,8 +59,6 @@ namespace ErrorCodes { extern const int UNKNOWN_TYPE; extern const int VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE; - extern const int CANNOT_CONVERT_TYPE; - extern const int CANNOT_INSERT_NULL_IN_ORDINARY_COLUMN; extern const int THERE_IS_NO_COLUMN; extern const int BAD_ARGUMENTS; extern const int UNKNOWN_EXCEPTION; @@ -392,6 +389,9 @@ static ColumnWithTypeAndName readColumnFromArrowColumn( return readColumnWithDate32Data(arrow_column, column_name); case arrow::Type::DATE64: return readColumnWithDate64Data(arrow_column, column_name); + // ClickHouse writes Date as arrow UINT16 and DateTime as arrow UINT32, + // so, read UINT16 as Date and UINT32 as DateTime to perform correct conversion + // between Date and DateTime further. case arrow::Type::UINT16: return readColumnWithDateData(arrow_column, column_name); case arrow::Type::UINT32: @@ -510,40 +510,40 @@ static ColumnWithTypeAndName readColumnFromArrowColumn( // Creating CH header by arrow schema. Will be useful in task about inserting // data from file without knowing table structure. -// -//static void checkStatus(const arrow::Status & status, const String & column_name, const String & format_name) -//{ -// if (!status.ok()) -// throw Exception{ErrorCodes::UNKNOWN_EXCEPTION, "Error with a {} column '{}': {}.", format_name, column_name, status.ToString()}; -//} -// -//static Block arrowSchemaToCHHeader(const arrow::Schema & schema, const std::string & format_name) -//{ -// ColumnsWithTypeAndName sample_columns; -// for (const auto & field : schema.fields()) -// { -// /// Create empty arrow column by it's type and convert it to ClickHouse column. -// arrow::MemoryPool* pool = arrow::default_memory_pool(); -// std::unique_ptr array_builder; -// arrow::Status status = MakeBuilder(pool, field->type(), &array_builder); -// checkStatus(status, field->name(), format_name); -// std::shared_ptr arrow_array; -// status = array_builder->Finish(&arrow_array); -// checkStatus(status, field->name(), format_name); -// arrow::ArrayVector array_vector = {arrow_array}; -// auto arrow_column = std::make_shared(array_vector); -// std::unordered_map> dict_values; -// ColumnWithTypeAndName sample_column = readColumnFromArrowColumn(arrow_column, field->name(), format_name, false, dict_values); -// sample_columns.emplace_back(std::move(sample_column)); -// } -// return Block(std::move(sample_columns)); -//} -// -//ArrowColumnToCHColumn::ArrowColumnToCHColumn( -// const arrow::Schema & schema, const std::string & format_name_, bool import_nested_) -// : header(arrowSchemaToCHHeader(schema, format_name_)), format_name(format_name_), import_nested(import_nested_) -//{ -//} + +static void checkStatus(const arrow::Status & status, const String & column_name, const String & format_name) +{ + if (!status.ok()) + throw Exception{ErrorCodes::UNKNOWN_EXCEPTION, "Error with a {} column '{}': {}.", format_name, column_name, status.ToString()}; +} + +static Block arrowSchemaToCHHeader(const arrow::Schema & schema, const std::string & format_name) +{ + ColumnsWithTypeAndName sample_columns; + for (const auto & field : schema.fields()) + { + /// Create empty arrow column by it's type and convert it to ClickHouse column. + arrow::MemoryPool* pool = arrow::default_memory_pool(); + std::unique_ptr array_builder; + arrow::Status status = MakeBuilder(pool, field->type(), &array_builder); + checkStatus(status, field->name(), format_name); + std::shared_ptr arrow_array; + status = array_builder->Finish(&arrow_array); + checkStatus(status, field->name(), format_name); + arrow::ArrayVector array_vector = {arrow_array}; + auto arrow_column = std::make_shared(array_vector); + std::unordered_map> dict_values; + ColumnWithTypeAndName sample_column = readColumnFromArrowColumn(arrow_column, field->name(), format_name, false, dict_values); + sample_columns.emplace_back(std::move(sample_column)); + } + return Block(std::move(sample_columns)); +} + +ArrowColumnToCHColumn::ArrowColumnToCHColumn( + const arrow::Schema & schema, const std::string & format_name_, bool import_nested_) + : header(arrowSchemaToCHHeader(schema, format_name_)), format_name(format_name_), import_nested(import_nested_) +{ +} ArrowColumnToCHColumn::ArrowColumnToCHColumn( const Block & header_, const std::string & format_name_, bool import_nested_) diff --git a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.h b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.h index 3d2fae1420f..abce6759250 100644 --- a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.h +++ b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.h @@ -8,6 +8,7 @@ #include #include +#include #include @@ -24,12 +25,12 @@ public: /// Constructor that create header by arrow schema. It will be useful for inserting /// data from file without knowing table structure. - /// ArrowColumnToCHColumn(const arrow::Schema & schema, const std::string & format_name, bool import_nested_); + ArrowColumnToCHColumn(const arrow::Schema & schema, const std::string & format_name, bool import_nested_); void arrowTableToCHChunk(Chunk & res, std::shared_ptr & table); private: - const Block & header; + const Block header; const std::string format_name; bool import_nested; From f06d445d67b3c270272aa4817ebf5666d2898f01 Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Mon, 9 Aug 2021 17:44:02 +0300 Subject: [PATCH 361/599] Add Ru for pads functions. --- docs/ru/sql-reference/functions/string-functions.md | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/ru/sql-reference/functions/string-functions.md b/docs/ru/sql-reference/functions/string-functions.md index 1a666c6a0cc..f60027b6202 100644 --- a/docs/ru/sql-reference/functions/string-functions.md +++ b/docs/ru/sql-reference/functions/string-functions.md @@ -41,7 +41,7 @@ toc_title: "Функции для работы со строками" ## leftPad {#leftpad} -Заполняет текущую строку слева пробелами или указанной строкой (несколько раз, если необходимо), пока результирующая строка не достигнет заданной длины. Соответствует функции MySQL `LPAD`. +Заполняет текущую строку слева пробелами или указанной строкой (несколько раз, если необходимо), пока результирующая строка не достигнет заданной длины. Соответствует MySQL функции `LPAD`. **Синтаксис** @@ -79,7 +79,7 @@ SELECT leftPad('abc', 7, '*'), leftPad('def', 7); ## leftPadUTF8 {#leftpadutf8} -Заполняет текущую строку слева пробелами или указанной строкой (несколько раз, если необходимо), пока результирующая строка не достигнет заданной длины. Соответствует функции MySQL `LPAD`. Функция [leftPad](#leftpad) измеряет длину строки в байтах, а функция `leftPadUTF8` — в кодовых точках Unicode. +Заполняет текущую строку слева пробелами или указанной строкой (несколько раз, если необходимо), пока результирующая строка не достигнет заданной длины. Соответствует MySQL функции `LPAD`. Функция [leftPad](#leftpad) измеряет длину строки в байтах, а функция `leftPadUTF8` — в кодовых точках Unicode. **Синтаксис** @@ -117,7 +117,7 @@ SELECT leftPadUTF8('абвг', 7, '*'), leftPadUTF8('дежз', 7); ## rightPad {#rightpad} -Заполняет текущую строку справа пробелами или указанной строкой (несколько раз, если необходимо), пока результирующая строка не достигнет заданной длины. Соответствует функции MySQL `RPAD`. +Заполняет текущую строку справа пробелами или указанной строкой (несколько раз, если необходимо), пока результирующая строка не достигнет заданной длины. Соответствует MySQL функции `RPAD`. **Синтаксис** @@ -155,7 +155,7 @@ SELECT rightPad('abc', 7, '*'), rightPad('abc', 7); ## rightPadUTF8 {#rightpadutf8} -Заполняет текущую строку слева пробелами или указанной строкой (несколько раз, если необходимо), пока результирующая строка не достигнет заданной длины. Соответствует функции MySQL `RPAD`. Функция [rightPad](#rightpad) измеряет длину строки в байтах, а функция `rightPadUTF8` — в кодовых точках Unicode. +Заполняет текущую строку слева пробелами или указанной строкой (несколько раз, если необходимо), пока результирующая строка не достигнет заданной длины. Соответствует MySQL функции `RPAD`. Функция [rightPad](#rightpad) измеряет длину строки в байтах, а функция `rightPadUTF8` — в кодовых точках Unicode. **Синтаксис** From 8cc493a3cdd4cd621a7d6b585c542b6f16d88e5d Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 9 Aug 2021 18:09:29 +0300 Subject: [PATCH 362/599] Try fix build. --- src/Common/DenseHashMap.h | 19 +++++++++++++++++++ src/Common/DenseHashSet.h | 19 +++++++++++++++++++ src/Common/SparseHashMap.h | 19 +++++++++++++++++++ src/Core/NamesAndTypes.cpp | 8 ++------ src/Dictionaries/HashedDictionary.h | 10 +--------- src/Storages/MergeTree/IMergeTreeReader.h | 8 ++------ src/Storages/StorageInMemoryMetadata.cpp | 20 +++++--------------- 7 files changed, 67 insertions(+), 36 deletions(-) create mode 100644 src/Common/DenseHashMap.h create mode 100644 src/Common/DenseHashSet.h create mode 100644 src/Common/SparseHashMap.h diff --git a/src/Common/DenseHashMap.h b/src/Common/DenseHashMap.h new file mode 100644 index 00000000000..0097faa5259 --- /dev/null +++ b/src/Common/DenseHashMap.h @@ -0,0 +1,19 @@ +#pragma once + +#if defined(ARCADIA_BUILD) +#define HASH_FUN_H +#endif + +#include + +#if !defined(ARCADIA_BUILD) + template , + class EqualKey = std::equal_to, + class Alloc = google::libc_allocator_with_realloc>> + using DenseHashMap = google::dense_hash_map; +#else + template , + class EqualKey = std::equal_to, + class Alloc = google::sparsehash::libc_allocator_with_realloc>> + using DenseHashMap = google::sparsehash::dense_hash_map; +#endif diff --git a/src/Common/DenseHashSet.h b/src/Common/DenseHashSet.h new file mode 100644 index 00000000000..38d281e1966 --- /dev/null +++ b/src/Common/DenseHashSet.h @@ -0,0 +1,19 @@ +#pragma once + +#if defined(ARCADIA_BUILD) +#define HASH_FUN_H +#endif + +#include + +#if !defined(ARCADIA_BUILD) + template , + class EqualKey = std::equal_to, + class Alloc = google::libc_allocator_with_realloc>> + using DenseHashSet = google::dense_hash_set; +#else +template , + class EqualKey = std::equal_to, + class Alloc = google::sparsehash::libc_allocator_with_realloc>> + using DenseHashSet = google::sparsehash::dense_hash_set; +#endif diff --git a/src/Common/SparseHashMap.h b/src/Common/SparseHashMap.h new file mode 100644 index 00000000000..c5fa07410af --- /dev/null +++ b/src/Common/SparseHashMap.h @@ -0,0 +1,19 @@ +#pragma once + +#if defined(ARCADIA_BUILD) +#define HASH_FUN_H +#endif + +#include + +#if !defined(ARCADIA_BUILD) + template , + class EqualKey = std::equal_to, + class Alloc = google::libc_allocator_with_realloc>> + using SparseHashMap = google::sparse_hash_map; +#else + template , + class EqualKey = std::equal_to, + class Alloc = google::sparsehash::libc_allocator_with_realloc>> + using SparseHashMap = google::sparsehash::sparse_hash_map; +#endif diff --git a/src/Core/NamesAndTypes.cpp b/src/Core/NamesAndTypes.cpp index 91191c73fd0..54f83fc13fc 100644 --- a/src/Core/NamesAndTypes.cpp +++ b/src/Core/NamesAndTypes.cpp @@ -6,7 +6,7 @@ #include #include #include -#include +#include namespace DB @@ -163,11 +163,7 @@ NamesAndTypesList NamesAndTypesList::filter(const Names & names) const NamesAndTypesList NamesAndTypesList::addTypes(const Names & names) const { /// NOTE: It's better to make a map in `IStorage` than to create it here every time again. -#if !defined(ARCADIA_BUILD) - google::dense_hash_map types; -#else - google::sparsehash::dense_hash_map types; -#endif + DenseHashMap types; types.set_empty_key(StringRef()); for (const auto & column : *this) diff --git a/src/Dictionaries/HashedDictionary.h b/src/Dictionaries/HashedDictionary.h index 842e49aa8f0..168f4e3bafa 100644 --- a/src/Dictionaries/HashedDictionary.h +++ b/src/Dictionaries/HashedDictionary.h @@ -5,7 +5,7 @@ #include #include -#include +#include #include #include @@ -125,14 +125,6 @@ private: HashMap, HashMapWithSavedHash>>; -#if !defined(ARCADIA_BUILD) - template - using SparseHashMap = google::sparse_hash_map>; -#else - template - using SparseHashMap = google::sparsehash::sparse_hash_map>; -#endif - template using CollectionTypeSparse = std::conditional_t< dictionary_key_type == DictionaryKeyType::simple, diff --git a/src/Storages/MergeTree/IMergeTreeReader.h b/src/Storages/MergeTree/IMergeTreeReader.h index ab412e48822..8d80719efaf 100644 --- a/src/Storages/MergeTree/IMergeTreeReader.h +++ b/src/Storages/MergeTree/IMergeTreeReader.h @@ -1,9 +1,9 @@ #pragma once #include +#include #include #include -#include namespace DB { @@ -95,11 +95,7 @@ private: /// Actual data type of columns in part -#if !defined(ARCADIA_BUILD) - google::dense_hash_map columns_from_part; -#else - google::sparsehash::dense_hash_map columns_from_part; -#endif + DenseHashMap columns_from_part; }; } diff --git a/src/Storages/StorageInMemoryMetadata.cpp b/src/Storages/StorageInMemoryMetadata.cpp index 7b1a126b1ab..91f69cdac7d 100644 --- a/src/Storages/StorageInMemoryMetadata.cpp +++ b/src/Storages/StorageInMemoryMetadata.cpp @@ -1,7 +1,7 @@ #include -#include -#include +#include +#include #include #include #include @@ -320,12 +320,7 @@ Block StorageInMemoryMetadata::getSampleBlockForColumns( { Block res; -#if !defined(ARCADIA_BUILD) - google::dense_hash_map virtuals_map; -#else - google::sparsehash::dense_hash_map virtuals_map; -#endif - + DenseHashMap virtuals_map; virtuals_map.set_empty_key(StringRef()); /// Virtual columns must be appended after ordinary, because user can @@ -475,13 +470,8 @@ bool StorageInMemoryMetadata::hasSelectQuery() const namespace { -#if !defined(ARCADIA_BUILD) - using NamesAndTypesMap = google::dense_hash_map; - using UniqueStrings = google::dense_hash_set; -#else - using NamesAndTypesMap = google::sparsehash::dense_hash_map; - using UniqueStrings = google::sparsehash::dense_hash_set; -#endif + using NamesAndTypesMap = DenseHashMap; + using UniqueStrings = DenseHashSet; String listOfColumns(const NamesAndTypesList & available_columns) { From 3d201c37c05a488d444ca3eff274260892f0b345 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Mon, 9 Aug 2021 15:42:29 +0000 Subject: [PATCH 363/599] Done --- src/Core/Block.cpp | 14 ++++++++++++++ src/Core/Block.h | 1 + 2 files changed, 15 insertions(+) diff --git a/src/Core/Block.cpp b/src/Core/Block.cpp index efd8de43a3c..96667862e41 100644 --- a/src/Core/Block.cpp +++ b/src/Core/Block.cpp @@ -44,6 +44,13 @@ void Block::initializeIndexByName() } +void Block::reserve(size_t count) +{ + index_by_name.reserve(count); + data.reserve(count); +} + + void Block::insert(size_t position, ColumnWithTypeAndName elem) { if (position > data.size()) @@ -287,6 +294,7 @@ std::string Block::dumpIndex() const Block Block::cloneEmpty() const { Block res; + res.reserve(data.size()); for (const auto & elem : data) res.insert(elem.cloneEmpty()); @@ -364,6 +372,8 @@ Block Block::cloneWithColumns(MutableColumns && columns) const Block res; size_t num_columns = data.size(); + res.reserve(num_columns); + for (size_t i = 0; i < num_columns; ++i) res.insert({ std::move(columns[i]), data[i].type, data[i].name }); @@ -381,6 +391,8 @@ Block Block::cloneWithColumns(const Columns & columns) const throw Exception("Cannot clone block with columns because block has " + toString(num_columns) + " columns, " "but " + toString(columns.size()) + " columns given.", ErrorCodes::LOGICAL_ERROR); + res.reserve(num_columns); + for (size_t i = 0; i < num_columns; ++i) res.insert({ columns[i], data[i].type, data[i].name }); @@ -393,6 +405,8 @@ Block Block::cloneWithoutColumns() const Block res; size_t num_columns = data.size(); + res.reserve(num_columns); + for (size_t i = 0; i < num_columns; ++i) res.insert({ nullptr, data[i].type, data[i].name }); diff --git a/src/Core/Block.h b/src/Core/Block.h index a2d91190795..14f82cecd8d 100644 --- a/src/Core/Block.h +++ b/src/Core/Block.h @@ -152,6 +152,7 @@ public: private: void eraseImpl(size_t position); void initializeIndexByName(); + void reserve(size_t count); /// This is needed to allow function execution over data. /// It is safe because functions does not change column names, so index is unaffected. From c4cf1eae0da5e6b462226c60d19e74e137ae38f5 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 9 Aug 2021 18:46:39 +0300 Subject: [PATCH 364/599] Try fix build. --- src/Common/DenseHashSet.h | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/src/Common/DenseHashSet.h b/src/Common/DenseHashSet.h index 38d281e1966..9741bf5ec5c 100644 --- a/src/Common/DenseHashSet.h +++ b/src/Common/DenseHashSet.h @@ -4,16 +4,16 @@ #define HASH_FUN_H #endif -#include +#include #if !defined(ARCADIA_BUILD) - template , - class EqualKey = std::equal_to, - class Alloc = google::libc_allocator_with_realloc>> - using DenseHashSet = google::dense_hash_set; + template , + class EqualKey = std::equal_to, + class Alloc = google::libc_allocator_with_realloc> + using DenseHashSet = google::dense_hash_set; #else -template , - class EqualKey = std::equal_to, - class Alloc = google::sparsehash::libc_allocator_with_realloc>> - using DenseHashSet = google::sparsehash::dense_hash_set; + template , + class EqualKey = std::equal_to, + class Alloc = google::sparsehash::libc_allocator_with_realloc> + using DenseHashSet = google::sparsehash::dense_hash_set; #endif From 1cd31382a5ea4c5b6679723082f334162614ef57 Mon Sep 17 00:00:00 2001 From: Alexey Boykov <33257111+mathalex@users.noreply.github.com> Date: Mon, 9 Aug 2021 19:52:15 +0300 Subject: [PATCH 365/599] Update conditional-functions.md Fix typo. --- docs/ru/sql-reference/functions/conditional-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/conditional-functions.md b/docs/ru/sql-reference/functions/conditional-functions.md index b191937df51..8a78425efd4 100644 --- a/docs/ru/sql-reference/functions/conditional-functions.md +++ b/docs/ru/sql-reference/functions/conditional-functions.md @@ -7,7 +7,7 @@ toc_title: "Условные функции" ## if {#if} -Условное выражение. В отличии от большинства систем, ClickHouse всегда считает оба выражения `then` и `else`. +Условное выражение. В отличие от большинства систем, ClickHouse всегда считает оба выражения `then` и `else`. **Синтаксис** From 2259387f0cf2a0576f2d0cefb4ee2a3907338b23 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Mon, 9 Aug 2021 20:04:42 +0300 Subject: [PATCH 366/599] make it possible to cancel window functions on ctrl+c --- src/Processors/Transforms/WindowTransform.cpp | 23 +++++++++++++++-- src/Processors/Transforms/WindowTransform.h | 4 ++- tests/clickhouse-test | 2 +- .../01572_kill_window_function.reference | 1 + .../0_stateless/01572_kill_window_function.sh | 25 +++++++++++++++++++ 5 files changed, 51 insertions(+), 4 deletions(-) create mode 100644 tests/queries/0_stateless/01572_kill_window_function.reference create mode 100755 tests/queries/0_stateless/01572_kill_window_function.sh diff --git a/src/Processors/Transforms/WindowTransform.cpp b/src/Processors/Transforms/WindowTransform.cpp index 3ab1a23537b..1b8406682ea 100644 --- a/src/Processors/Transforms/WindowTransform.cpp +++ b/src/Processors/Transforms/WindowTransform.cpp @@ -1166,6 +1166,23 @@ void WindowTransform::appendChunk(Chunk & chunk) // Write out the aggregation results. writeOutCurrentRow(); + if (isCancelled()) + { + // Good time to check if the query is cancelled. Checking once + // per block might not be enough in severe quadratic cases. + // Just leave the work halfway through and return, the 'prepare' + // method will figure out what to do. Note that this doesn't + // handle 'max_execution_time' and other limits, because these + // limits are only updated between blocks. Eventually we should + // start updating them in background and canceling the processor, + // like we do for Ctrl+C handling. + // + // This class is final, so the check should hopefully be + // devirtualized and become a single never-taken branch that is + // basically free. + return; + } + // Move to the next row. The frame will have to be recalculated. // The peer group start is updated at the beginning of the loop, // because current_row might now be past-the-end. @@ -1255,10 +1272,12 @@ IProcessor::Status WindowTransform::prepare() // next_output_block_number, first_not_ready_row, first_block_number, // blocks.size()); - if (output.isFinished()) + if (output.isFinished() || isCancelled()) { // The consumer asked us not to continue (or we decided it ourselves), - // so we abort. + // so we abort. Not sure what the difference between the two conditions + // is, but it seemed that output.isFinished() is not enough to cancel on + // Ctrl+C. Test manually if you change it. input.close(); return Status::Finished; } diff --git a/src/Processors/Transforms/WindowTransform.h b/src/Processors/Transforms/WindowTransform.h index d7211f9edd7..5dc78a34f78 100644 --- a/src/Processors/Transforms/WindowTransform.h +++ b/src/Processors/Transforms/WindowTransform.h @@ -80,8 +80,10 @@ struct RowNumber * the order of input data. This property also trivially holds for the ROWS and * GROUPS frames. For the RANGE frame, the proof requires the additional fact * that the ranges are specified in terms of (the single) ORDER BY column. + * + * `final` is so that the isCancelled() is devirtualized, we call it every row. */ -class WindowTransform : public IProcessor /* public ISimpleTransform */ +class WindowTransform final : public IProcessor { public: WindowTransform( diff --git a/tests/clickhouse-test b/tests/clickhouse-test index b734af0bdea..f6833cfbd09 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -647,7 +647,7 @@ def run_tests_array(all_tests_with_params): failures_chain += 1 status += MSG_FAIL status += print_test_time(total_time) - status += " - having exception:\n{}\n".format( + status += " - having exception in stdout:\n{}\n".format( '\n'.join(stdout.split('\n')[:100])) status += 'Database: ' + testcase_args.testcase_database elif reference_file is None: diff --git a/tests/queries/0_stateless/01572_kill_window_function.reference b/tests/queries/0_stateless/01572_kill_window_function.reference new file mode 100644 index 00000000000..93e7803245e --- /dev/null +++ b/tests/queries/0_stateless/01572_kill_window_function.reference @@ -0,0 +1 @@ +138 diff --git a/tests/queries/0_stateless/01572_kill_window_function.sh b/tests/queries/0_stateless/01572_kill_window_function.sh new file mode 100755 index 00000000000..8a46ae2fd5a --- /dev/null +++ b/tests/queries/0_stateless/01572_kill_window_function.sh @@ -0,0 +1,25 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +set -e -o pipefail + +function wait_for_query_to_start() +{ + while [[ $($CLICKHOUSE_CLIENT --query "SELECT count() FROM system.processes WHERE query_id = '$1' and current_database = currentDatabase()") == 0 ]]; do sleep 0.1; done +} + +query_id="01572_kill_window_function" +$CLICKHOUSE_CLIENT --query_id="$query_id" --query "SELECT count(1048575) OVER (PARTITION BY intDiv(NULL, number) ORDER BY number DESC NULLS FIRST ROWS BETWEEN CURRENT ROW AND 1048575 FOLLOWING) FROM numbers(255, 1048575)" >/dev/null 2>&1 & +client_pid=$! +wait_for_query_to_start "$query_id" +$CLICKHOUSE_CLIENT --query "kill query where query_id = '$query_id' and current_database = currentDatabase() format Null" +wait $client_pid || echo $? + +# We have tested for Ctrl+C. +# The following client flags don't cancel, but should: --max_execution_time, +# --receive_timeout. Probably needs asynchonous calculation of query limits, as +# discussed with Nikolay on TG: https://t.me/c/1214350934/21492 + From 75eb3e318de9f949d8910b868cf0a1805546c810 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 9 Aug 2021 20:47:52 +0300 Subject: [PATCH 367/599] Add menus (yummy dataset) --- .../getting-started/example-datasets/index.md | 1 + .../getting-started/example-datasets/menus.md | 322 ++++++++++++++++++ 2 files changed, 323 insertions(+) create mode 100644 docs/en/getting-started/example-datasets/menus.md diff --git a/docs/en/getting-started/example-datasets/index.md b/docs/en/getting-started/example-datasets/index.md index 9d4aec6eca2..e61c76970a7 100644 --- a/docs/en/getting-started/example-datasets/index.md +++ b/docs/en/getting-started/example-datasets/index.md @@ -17,6 +17,7 @@ The list of documented datasets: - [OpenSky](../../getting-started/example-datasets/opensky.md) - [New York Taxi Data](../../getting-started/example-datasets/nyc-taxi.md) - [UK Property Price Paid](../../getting-started/example-datasets/uk-price-paid.md) +- [What's on the Menu?](../../getting-started/example-datasets/menus.md) - [Star Schema Benchmark](../../getting-started/example-datasets/star-schema.md) - [WikiStat](../../getting-started/example-datasets/wikistat.md) - [Terabyte of Click Logs from Criteo](../../getting-started/example-datasets/criteo.md) diff --git a/docs/en/getting-started/example-datasets/menus.md b/docs/en/getting-started/example-datasets/menus.md new file mode 100644 index 00000000000..e25a1300670 --- /dev/null +++ b/docs/en/getting-started/example-datasets/menus.md @@ -0,0 +1,322 @@ +--- +toc_priority: 21 +toc_title: Menus +--- + +# New York Public Library "What's on the Menu?" Dataset + +The dataset is created by the New York Public Library. It contains historical data on the menus of hotels, restaurants and cafes with the dishes along with their prices. + +Source: http://menus.nypl.org/data +The data is in public domain. + +The data is from library's may be incomplete and difficult for statistical analysis. Nevertheless it is also very yummy. +The size is just 1.3 million records about dishes in the menus (a very small data volume for ClickHouse, but it's still a good example). + +## Download the Dataset + +``` +wget https://s3.amazonaws.com/menusdata.nypl.org/gzips/2021_08_01_07_01_17_data.tgz +``` + +Replace the link to the up to date link from http://menus.nypl.org/data if needed. +Download size is about 35 MB. + +## Unpack the Dataset + +``` +tar xvf 2021_08_01_07_01_17_data.tgz +``` + +Uncompressed size is about 150 MB. + +The data is normalized consisted of four tables: +- Menu: information about menus: the name of the restaurant, the date when menu was seen, etc; +- Dish: information about dishes: the name of the dish along with some characteristic; +- MenuPage: information about the pages in the menus; every page belongs to some menu; +- MenuItem: an item of the menu - a dish along with its price on some menu page: links to dish and menu page. + +## Create the Tables + +``` +CREATE TABLE dish +( + id UInt32, + name String, + description String, + menus_appeared UInt32, + times_appeared Int32, + first_appeared UInt16, + last_appeared UInt16, + lowest_price Decimal64(3), + highest_price Decimal64(3) +) ENGINE = MergeTree ORDER BY id; + +CREATE TABLE menu +( + id UInt32, + name String, + sponsor String, + event String, + venue String, + place String, + physical_description String, + occasion String, + notes String, + call_number String, + keywords String, + language String, + date String, + location String, + location_type String, + currency String, + currency_symbol String, + status String, + page_count UInt16, + dish_count UInt16 +) ENGINE = MergeTree ORDER BY id; + +CREATE TABLE menu_page +( + id UInt32, + menu_id UInt32, + page_number UInt16, + image_id String, + full_height UInt16, + full_width UInt16, + uuid UUID +) ENGINE = MergeTree ORDER BY id; + +CREATE TABLE menu_item +( + id UInt32, + menu_page_id UInt32, + price Decimal64(3), + high_price Decimal64(3), + dish_id UInt32, + created_at DateTime, + updated_at DateTime, + xpos Float64, + ypos Float64 +) ENGINE = MergeTree ORDER BY id; +``` + +We use `Decimal` data type to store prices. Everything else is quite straightforward. + +## Import Data + +Upload data into ClickHouse in parallel: + +``` +clickhouse-client --format_csv_allow_single_quotes 0 --input_format_null_as_default 0 --query "INSERT INTO dish FORMAT CSVWithNames" < Dish.csv +clickhouse-client --format_csv_allow_single_quotes 0 --input_format_null_as_default 0 --query "INSERT INTO menu FORMAT CSVWithNames" < Menu.csv +clickhouse-client --format_csv_allow_single_quotes 0 --input_format_null_as_default 0 --query "INSERT INTO menu_page FORMAT CSVWithNames" < MenuPage.csv +clickhouse-client --format_csv_allow_single_quotes 0 --input_format_null_as_default 0 --date_time_input_format best_effort --query "INSERT INTO menu_item FORMAT CSVWithNames" < MenuItem.csv +``` + +We use `CSVWithNames` format as the data is represented by CSV with header. + +We disable `format_csv_allow_single_quotes` as only double quotes are used for data fields and single quotes can be inside the values and should not confuse the CSV parser. + +We disable `input_format_null_as_default` as our data does not have NULLs. Otherwise ClickHouse will try to parse `\N` sequences and can be confused with `\` in data. + +The setting `--date_time_input_format best_effort` allows to parse `DateTime` fields in wide variety of formats. For example, ISO-8601 without seconds like '2000-01-01 01:02' will be recognized. Without this setting only fixed DateTime format is allowed. + +## Denormalize the Data + +Data is presented in multiple tables in normalized form. It means you have to perform JOINs if you want to query, e.g. dish names from menu items. +For typical analytical tasks it is way more efficient to deal with pre-JOINed data to avoid doing JOIN every time. It is called "denormalized" data. + +We will create a table that will contain all the data JOINed together: + +``` +CREATE TABLE menu_item_denorm +ENGINE = MergeTree ORDER BY (dish_name, created_at) +AS SELECT + price, + high_price, + created_at, + updated_at, + xpos, + ypos, + dish.id AS dish_id, + dish.name AS dish_name, + dish.description AS dish_description, + dish.menus_appeared AS dish_menus_appeared, + dish.times_appeared AS dish_times_appeared, + dish.first_appeared AS dish_first_appeared, + dish.last_appeared AS dish_last_appeared, + dish.lowest_price AS dish_lowest_price, + dish.highest_price AS dish_highest_price, + menu.id AS menu_id, + menu.name AS menu_name, + menu.sponsor AS menu_sponsor, + menu.event AS menu_event, + menu.venue AS menu_venue, + menu.place AS menu_place, + menu.physical_description AS menu_physical_description, + menu.occasion AS menu_occasion, + menu.notes AS menu_notes, + menu.call_number AS menu_call_number, + menu.keywords AS menu_keywords, + menu.language AS menu_language, + menu.date AS menu_date, + menu.location AS menu_location, + menu.location_type AS menu_location_type, + menu.currency AS menu_currency, + menu.currency_symbol AS menu_currency_symbol, + menu.status AS menu_status, + menu.page_count AS menu_page_count, + menu.dish_count AS menu_dish_count +FROM menu_item + JOIN dish ON menu_item.dish_id = dish.id + JOIN menu_page ON menu_item.menu_page_id = menu_page.id + JOIN menu ON menu_page.menu_id = menu.id +``` + +## Validate the Data + +``` +SELECT count() FROM menu_item_denorm +1329175 +``` + +## Run Some Queries + +Averaged historical prices of dishes: + +``` +SELECT + round(toUInt32OrZero(extract(menu_date, '^\\d{4}')), -1) AS d, + count(), + round(avg(price), 2), + bar(avg(price), 0, 100, 100) +FROM menu_item_denorm +WHERE (menu_currency = 'Dollars') AND (d > 0) AND (d < 2022) +GROUP BY d +ORDER BY d ASC + +┌────d─┬─count()─┬─round(avg(price), 2)─┬─bar(avg(price), 0, 100, 100)─┐ +│ 1850 │ 618 │ 1.5 │ █▍ │ +│ 1860 │ 1634 │ 1.29 │ █▎ │ +│ 1870 │ 2215 │ 1.36 │ █▎ │ +│ 1880 │ 3909 │ 1.01 │ █ │ +│ 1890 │ 8837 │ 1.4 │ █▍ │ +│ 1900 │ 176292 │ 0.68 │ ▋ │ +│ 1910 │ 212196 │ 0.88 │ ▊ │ +│ 1920 │ 179590 │ 0.74 │ ▋ │ +│ 1930 │ 73707 │ 0.6 │ ▌ │ +│ 1940 │ 58795 │ 0.57 │ ▌ │ +│ 1950 │ 41407 │ 0.95 │ ▊ │ +│ 1960 │ 51179 │ 1.32 │ █▎ │ +│ 1970 │ 12914 │ 1.86 │ █▋ │ +│ 1980 │ 7268 │ 4.35 │ ████▎ │ +│ 1990 │ 11055 │ 6.03 │ ██████ │ +│ 2000 │ 2467 │ 11.85 │ ███████████▋ │ +│ 2010 │ 597 │ 25.66 │ █████████████████████████▋ │ +└──────┴─────────┴──────────────────────┴──────────────────────────────┘ + +17 rows in set. Elapsed: 0.044 sec. Processed 1.33 million rows, 54.62 MB (30.00 million rows/s., 1.23 GB/s.) +``` + +Take it with a grain of salt. + +### Burger Prices: + +``` +SELECT + round(toUInt32OrZero(extract(menu_date, '^\\d{4}')), -1) AS d, + count(), + round(avg(price), 2), + bar(avg(price), 0, 50, 100) +FROM menu_item_denorm +WHERE (menu_currency = 'Dollars') AND (d > 0) AND (d < 2022) AND (dish_name ILIKE '%burger%') +GROUP BY d +ORDER BY d ASC + +┌────d─┬─count()─┬─round(avg(price), 2)─┬─bar(avg(price), 0, 50, 100)───────────┐ +│ 1880 │ 2 │ 0.42 │ ▋ │ +│ 1890 │ 7 │ 0.85 │ █▋ │ +│ 1900 │ 399 │ 0.49 │ ▊ │ +│ 1910 │ 589 │ 0.68 │ █▎ │ +│ 1920 │ 280 │ 0.56 │ █ │ +│ 1930 │ 74 │ 0.42 │ ▋ │ +│ 1940 │ 119 │ 0.59 │ █▏ │ +│ 1950 │ 134 │ 1.09 │ ██▏ │ +│ 1960 │ 272 │ 0.92 │ █▋ │ +│ 1970 │ 108 │ 1.18 │ ██▎ │ +│ 1980 │ 88 │ 2.82 │ █████▋ │ +│ 1990 │ 184 │ 3.68 │ ███████▎ │ +│ 2000 │ 21 │ 7.14 │ ██████████████▎ │ +│ 2010 │ 6 │ 18.42 │ ████████████████████████████████████▋ │ +└──────┴─────────┴──────────────────────┴───────────────────────────────────────┘ + +14 rows in set. Elapsed: 0.052 sec. Processed 1.33 million rows, 94.15 MB (25.48 million rows/s., 1.80 GB/s.) +``` + +### Vodka: + +``` +SELECT + round(toUInt32OrZero(extract(menu_date, '^\\d{4}')), -1) AS d, + count(), + round(avg(price), 2), + bar(avg(price), 0, 50, 100) +FROM menu_item_denorm +WHERE (menu_currency IN ('Dollars', '')) AND (d > 0) AND (d < 2022) AND (dish_name ILIKE '%vodka%') +GROUP BY d +ORDER BY d ASC + +┌────d─┬─count()─┬─round(avg(price), 2)─┬─bar(avg(price), 0, 50, 100)─┐ +│ 1910 │ 2 │ 0 │ │ +│ 1920 │ 1 │ 0.3 │ ▌ │ +│ 1940 │ 21 │ 0.42 │ ▋ │ +│ 1950 │ 14 │ 0.59 │ █▏ │ +│ 1960 │ 113 │ 2.17 │ ████▎ │ +│ 1970 │ 37 │ 0.68 │ █▎ │ +│ 1980 │ 19 │ 2.55 │ █████ │ +│ 1990 │ 86 │ 3.6 │ ███████▏ │ +│ 2000 │ 2 │ 3.98 │ ███████▊ │ +└──────┴─────────┴──────────────────────┴─────────────────────────────┘ +``` + +### Caviar: + +Let's print caviar prices. Also let's print a name of any dish with caviar. + +``` +SELECT + round(toUInt32OrZero(extract(menu_date, '^\\d{4}')), -1) AS d, + count(), + round(avg(price), 2), + bar(avg(price), 0, 50, 100), + any(dish_name) +FROM menu_item_denorm +WHERE (menu_currency IN ('Dollars', '')) AND (d > 0) AND (d < 2022) AND (dish_name ILIKE '%caviar%') +GROUP BY d +ORDER BY d ASC + +┌────d─┬─count()─┬─round(avg(price), 2)─┬─bar(avg(price), 0, 50, 100)──────┬─any(dish_name)──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┐ +│ 1090 │ 1 │ 0 │ │ Caviar │ +│ 1880 │ 3 │ 0 │ │ Caviar │ +│ 1890 │ 39 │ 0.59 │ █▏ │ Butter and caviar │ +│ 1900 │ 1014 │ 0.34 │ ▋ │ Anchovy Caviar on Toast │ +│ 1910 │ 1588 │ 1.35 │ ██▋ │ 1/1 Brötchen Caviar │ +│ 1920 │ 927 │ 1.37 │ ██▋ │ ASTRAKAN CAVIAR │ +│ 1930 │ 289 │ 1.91 │ ███▋ │ Astrachan caviar │ +│ 1940 │ 201 │ 0.83 │ █▋ │ (SPECIAL) Domestic Caviar Sandwich │ +│ 1950 │ 81 │ 2.27 │ ████▌ │ Beluga Caviar │ +│ 1960 │ 126 │ 2.21 │ ████▍ │ Beluga Caviar │ +│ 1970 │ 105 │ 0.95 │ █▊ │ BELUGA MALOSSOL CAVIAR AMERICAN DRESSING │ +│ 1980 │ 12 │ 7.22 │ ██████████████▍ │ Authentic Iranian Beluga Caviar the world's finest black caviar presented in ice garni and a sampling of chilled 100° Russian vodka │ +│ 1990 │ 74 │ 14.42 │ ████████████████████████████▋ │ Avocado Salad, Fresh cut avocado with caviare │ +│ 2000 │ 3 │ 7.82 │ ███████████████▋ │ Aufgeschlagenes Kartoffelsueppchen mit Forellencaviar │ +│ 2010 │ 6 │ 15.58 │ ███████████████████████████████▏ │ "OYSTERS AND PEARLS" "Sabayon" of Pearl Tapioca with Island Creek Oysters and Russian Sevruga Caviar │ +└──────┴─────────┴──────────────────────┴──────────────────────────────────┴─────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┘ +``` + +At least they have caviar with vodka. Very nice. + +### Test it in Playground + +The data is uploaded to ClickHouse Playground, [example](https://gh-api.clickhouse.tech/play?user=play#U0VMRUNUCiAgICByb3VuZCh0b1VJbnQzMk9yWmVybyhleHRyYWN0KG1lbnVfZGF0ZSwgJ15cXGR7NH0nKSksIC0xKSBBUyBkLAogICAgY291bnQoKSwKICAgIHJvdW5kKGF2ZyhwcmljZSksIDIpLAogICAgYmFyKGF2ZyhwcmljZSksIDAsIDUwLCAxMDApLAogICAgYW55KGRpc2hfbmFtZSkKRlJPTSBtZW51X2l0ZW1fZGVub3JtCldIRVJFIChtZW51X2N1cnJlbmN5IElOICgnRG9sbGFycycsICcnKSkgQU5EIChkID4gMCkgQU5EIChkIDwgMjAyMikgQU5EIChkaXNoX25hbWUgSUxJS0UgJyVjYXZpYXIlJykKR1JPVVAgQlkgZApPUkRFUiBCWSBkIEFTQw==). From d64ee0337aa07816241cdbcd7d1ad2150cbb5374 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 9 Aug 2021 20:49:38 +0300 Subject: [PATCH 368/599] Add menus (yummy dataset) --- docs/en/getting-started/example-datasets/menus.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/getting-started/example-datasets/menus.md b/docs/en/getting-started/example-datasets/menus.md index e25a1300670..1f4592dcaa1 100644 --- a/docs/en/getting-started/example-datasets/menus.md +++ b/docs/en/getting-started/example-datasets/menus.md @@ -10,7 +10,7 @@ The dataset is created by the New York Public Library. It contains historical da Source: http://menus.nypl.org/data The data is in public domain. -The data is from library's may be incomplete and difficult for statistical analysis. Nevertheless it is also very yummy. +The data is from library's archive and it may be incomplete and difficult for statistical analysis. Nevertheless it is also very yummy. The size is just 1.3 million records about dishes in the menus (a very small data volume for ClickHouse, but it's still a good example). ## Download the Dataset From d584a55de79df66a1c6d3d15be9c452b35c25805 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 9 Aug 2021 20:52:41 +0300 Subject: [PATCH 369/599] Add menus (yummy dataset) --- docs/en/getting-started/example-datasets/menus.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docs/en/getting-started/example-datasets/menus.md b/docs/en/getting-started/example-datasets/menus.md index 1f4592dcaa1..87e4c75d0d4 100644 --- a/docs/en/getting-started/example-datasets/menus.md +++ b/docs/en/getting-started/example-datasets/menus.md @@ -280,6 +280,8 @@ ORDER BY d ASC └──────┴─────────┴──────────────────────┴─────────────────────────────┘ ``` +To get vodka we have to write `ILIKE '%vodka%'` and this definitely makes a statement. + ### Caviar: Let's print caviar prices. Also let's print a name of any dish with caviar. From 3b5bcc3ab56e4243d67db92719f22fa0b08b58aa Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Mon, 9 Aug 2021 21:22:40 +0300 Subject: [PATCH 370/599] Update storing-data.md --- docs/ru/operations/storing-data.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/ru/operations/storing-data.md b/docs/ru/operations/storing-data.md index 854e6401514..7af7cf3313d 100644 --- a/docs/ru/operations/storing-data.md +++ b/docs/ru/operations/storing-data.md @@ -5,10 +5,10 @@ toc_title: "Хранение данных на внешних дисках" # Хранение данных на внешних дисках {#external-disks} -Данные, которые обрабатываются в ClickHouse, обычно хранятся в файловой системе локально — на том же компьютере, где развернут сервер ClickHouse. При этом для хранения данных требуются диски большого объема, которые могут быть довольно дорогостоящими. Решением проблемы может стать хранение данных в удаленной файловой системе — например, на дисках [Amazon s3](https://aws.amazon.com/s3/) или в распределенной дисковой системе Hadoop ([HDFS](https://hadoop.apache.org/docs/current/hadoop-project-dist/hadoop-hdfs/HdfsDesign.html)). +Данные, которые обрабатываются в ClickHouse, обычно хранятся в файловой системе локально, где развернут сервер ClickHouse. При этом для хранения данных требуются диски большого объема, которые могут быть довольно дорогостоящими. Решением проблемы может стать хранение данных отдельно от сервера — в распределенных файловых системах — [Amazon s3](https://aws.amazon.com/s3/) или Hadoop ([HDFS](https://hadoop.apache.org/docs/current/hadoop-project-dist/hadoop-hdfs/HdfsDesign.html)). -Для работы с данными, хранящимися на дисках `Amazon s3`, используйте движок [s3](../engines/table-engines/integrations/s3.md), а для работы с данными в распределенной файловой системе Hadoop — движок [HDFS](../engines/table-engines/integrations/hdfs.md). +Для работы с данными, хранящимися в файловой системе `Amazon s3`, используйте движок [s3](../engines/table-engines/integrations/s3.md), а для работы с данными в файловой системе Hadoop — движок [HDFS](../engines/table-engines/integrations/hdfs.md). ## Репликация без копирования данных {#zero-copy} -Для дисков `s3` и `HDFS` в ClickHouse поддерживается репликация без копирования данных (zero-copy): если данные хранятся на нескольких удаленных машинах, то при синхронизации пересылаются только метаданные (пути к кускам данных), а сами данные не копируются. \ No newline at end of file +Для дисков `s3` и `HDFS` в ClickHouse поддерживается репликация без копирования данных (zero-copy): если данные хранятся на нескольких репликах, то при синхронизации пересылаются только метаданные (пути к кускам данных), а сами данные не копируются. From 8f1dd055ef775c8832b541f413f53e2429156718 Mon Sep 17 00:00:00 2001 From: Braulio Valdivielso Date: Mon, 9 Aug 2021 19:21:43 +0100 Subject: [PATCH 371/599] add `namedTupleItems` function I'm trying to address issue #26174. This is just a draft so far, as I don't think it's necessary to force all value types in the tuple to be the same. It should be possible to cast all values to a shared supertype. I also have to add tests --- src/Functions/namedTupleItems.cpp | 129 +++++++++++++++++++++++ src/Functions/registerFunctionsTuple.cpp | 2 + 2 files changed, 131 insertions(+) create mode 100644 src/Functions/namedTupleItems.cpp diff --git a/src/Functions/namedTupleItems.cpp b/src/Functions/namedTupleItems.cpp new file mode 100644 index 00000000000..598fe79b9fa --- /dev/null +++ b/src/Functions/namedTupleItems.cpp @@ -0,0 +1,129 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ +namespace ErrorCodes +{ + extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int ILLEGAL_INDEX; +} + +namespace +{ + +/** Extract element of tuple by constant index or name. The operation is essentially free. + * Also the function looks through Arrays: you can get Array of tuple elements from Array of Tuples. + */ +class FunctionNamedTupleItems : public IFunction +{ +public: + static constexpr auto name = "namedTupleItems"; + static FunctionPtr create(ContextPtr) + { + return std::make_shared(); + } + + String getName() const override + { + return name; + } + + size_t getNumberOfArguments() const override + { + return 1; + } + + bool useDefaultImplementationForConstants() const override + { + return true; + } + + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override + { + // get the type of all the fields in the tuple + const IDataType * col = arguments[0].type.get(); + const DataTypeTuple * tuple = checkAndGetDataType(col); + + if (!tuple) + throw Exception("First argument for function " + getName() + "must " + "be a tuple.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + + const auto& elementTypes = tuple->getElements(); + + if (elementTypes.empty()) + throw Exception("The argument tuple for function " + getName() + "must " + "not be empty.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + const auto& firstElementType = elementTypes[0]; + + auto it = std::find_if( + elementTypes.begin() + 1, + elementTypes.end(), + [&](const auto &other) { + return !firstElementType->equals(*other); + }); + + if (it != elementTypes.end()) + { + throw Exception("TODO: FIX", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + } + + DataTypePtr tupleNameType = std::make_shared(); + DataTypes itemDataTypes ={tupleNameType, + firstElementType}; + + auto itemDataType = std::make_shared(itemDataTypes); + + return std::make_shared(itemDataType); + } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/) const override + { + const IColumn *tuple_col = arguments[0].column.get(); + const DataTypeTuple * tuple = checkAndGetDataType(arguments[0].type.get()); + auto *tuple_col_concrete = assert_cast(tuple_col); + + MutableColumnPtr keys = ColumnString::create(); + MutableColumnPtr values = tuple_col_concrete->getColumn(0).cloneEmpty(); + auto offsets = ColumnVector::create(); + for (size_t row = 0; row < tuple_col_concrete->size(); ++row) + { + for (size_t col = 0; col < tuple_col_concrete->tupleSize(); ++col) + { + const std::string& key = tuple->getElementNames()[col]; + const IColumn& valueColumn = tuple_col_concrete->getColumn(col); + + values->insertFrom(valueColumn, row); + keys->insertData(key.data(), key.size()); + } + offsets->insertValue(tuple_col_concrete->tupleSize() * (row + 1)); + } + + std::vector tupleColumns = { std::move(keys), std::move(values) }; + auto tupleColumn = ColumnTuple::create(std::move(tupleColumns)); + return ColumnArray::create(std::move(tupleColumn), std::move(offsets)); + } +}; + +} + +void registerFunctionNamedTupleItems(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/src/Functions/registerFunctionsTuple.cpp b/src/Functions/registerFunctionsTuple.cpp index 12092e1e7e0..21d2dd3cc0f 100644 --- a/src/Functions/registerFunctionsTuple.cpp +++ b/src/Functions/registerFunctionsTuple.cpp @@ -5,11 +5,13 @@ class FunctionFactory; void registerFunctionTuple(FunctionFactory &); void registerFunctionTupleElement(FunctionFactory &); +void registerFunctionNamedTupleItems(FunctionFactory &); void registerFunctionsTuple(FunctionFactory & factory) { registerFunctionTuple(factory); registerFunctionTupleElement(factory); + registerFunctionNamedTupleItems(factory); } } From 852ee7e26a93bcb2ce20068b820d719d8d3d7024 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Mon, 9 Aug 2021 21:35:34 +0300 Subject: [PATCH 372/599] a more stable test --- .../01572_kill_window_function.reference | 2 +- .../0_stateless/01572_kill_window_function.sh | 35 ++++++++++++++----- 2 files changed, 28 insertions(+), 9 deletions(-) diff --git a/tests/queries/0_stateless/01572_kill_window_function.reference b/tests/queries/0_stateless/01572_kill_window_function.reference index 93e7803245e..d86bac9de59 100644 --- a/tests/queries/0_stateless/01572_kill_window_function.reference +++ b/tests/queries/0_stateless/01572_kill_window_function.reference @@ -1 +1 @@ -138 +OK diff --git a/tests/queries/0_stateless/01572_kill_window_function.sh b/tests/queries/0_stateless/01572_kill_window_function.sh index 8a46ae2fd5a..e596c59aa30 100755 --- a/tests/queries/0_stateless/01572_kill_window_function.sh +++ b/tests/queries/0_stateless/01572_kill_window_function.sh @@ -6,17 +6,36 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) set -e -o pipefail -function wait_for_query_to_start() -{ - while [[ $($CLICKHOUSE_CLIENT --query "SELECT count() FROM system.processes WHERE query_id = '$1' and current_database = currentDatabase()") == 0 ]]; do sleep 0.1; done -} - +# Run a test query that takes very long to run. query_id="01572_kill_window_function" $CLICKHOUSE_CLIENT --query_id="$query_id" --query "SELECT count(1048575) OVER (PARTITION BY intDiv(NULL, number) ORDER BY number DESC NULLS FIRST ROWS BETWEEN CURRENT ROW AND 1048575 FOLLOWING) FROM numbers(255, 1048575)" >/dev/null 2>&1 & client_pid=$! -wait_for_query_to_start "$query_id" -$CLICKHOUSE_CLIENT --query "kill query where query_id = '$query_id' and current_database = currentDatabase() format Null" -wait $client_pid || echo $? + +# First check that clickhouse-client is still alive, and then use one query to +# both kill the test query and verify that it has started. +# If we try to kill it before it starts, the test will fail. +# If it finishes after we check it started, the test will fail. +# That's why it is better to check and kill with the same call. +while kill -0 $client_pid \ + && [ -z "$(CLICKHOUSE_CLIENT --query "kill query where query_id = '$query_id' and current_database = currentDatabase()")" ] +do + sleep 1 +done + +# Wait for the client to terminate. +client_exit_code=0 +wait $client_pid || client_exit_code=$? + +# Note that we still have to check for normal termination, because the test query +# might have completed even before we first tried to kill it. This shouldn't +# really happen because it's very long, but our CI infractructure is known to +# introduce the most unexpected delays. +if [ $client_exit_code -eq 0 ] || [ $client_exit_code -eq 138 ] +then + echo "OK" +else + echo "Got unexpected client exit code $client_exit_code" +fi # We have tested for Ctrl+C. # The following client flags don't cancel, but should: --max_execution_time, From 6f8c4b8fde9f11d19cd52629e557215871672585 Mon Sep 17 00:00:00 2001 From: Braulio Valdivielso Date: Mon, 9 Aug 2021 20:07:43 +0100 Subject: [PATCH 373/599] add some basic tests for namedTupleItems --- .../02007_namedtupleitems.reference | 3 +++ .../0_stateless/02007_namedtupleitems.sql | 21 +++++++++++++++++++ 2 files changed, 24 insertions(+) create mode 100644 tests/queries/0_stateless/02007_namedtupleitems.reference create mode 100644 tests/queries/0_stateless/02007_namedtupleitems.sql diff --git a/tests/queries/0_stateless/02007_namedtupleitems.reference b/tests/queries/0_stateless/02007_namedtupleitems.reference new file mode 100644 index 00000000000..985425a01c5 --- /dev/null +++ b/tests/queries/0_stateless/02007_namedtupleitems.reference @@ -0,0 +1,3 @@ +[('1',1),('2',2),('3',3)] +[('a',(1,2)),('b',(3,4))] +[('a',(5,6)),('b',(7,8))] diff --git a/tests/queries/0_stateless/02007_namedtupleitems.sql b/tests/queries/0_stateless/02007_namedtupleitems.sql new file mode 100644 index 00000000000..cda8a9a654c --- /dev/null +++ b/tests/queries/0_stateless/02007_namedtupleitems.sql @@ -0,0 +1,21 @@ +SELECT namedTupleItems(tuple(1, 2, 3)); + +DROP TABLE IF EXISTS test02007; +CREATE TABLE test02007 ( + col Tuple( + a Tuple(key1 int, key2 int), + b Tuple(key1 int, key3 int) + ) +) ENGINE=Memory(); +INSERT INTO test02007 VALUES (tuple(tuple(1, 2), tuple(3, 4))); +INSERT INTO test02007 VALUES (tuple(tuple(5, 6), tuple(7, 8))); + +SELECT namedTupleItems(col) FROM test02007 ORDER BY col; +DROP TABLE IF EXISTS test02007; +SELECT namedTupleItems(tuple(1, 1.3)); -- this should work, will fix + +SELECT namedTupleItems(tuple(1, [1,2])); -- { serverError 43; } +SELECT namedTupleItems(tuple(1, 'a')) -- { serverError 43; } +SELECT namedTupleItems(33); -- { serverError 43; } + + From 0cdb6f8c786e21aec77588ae66951add7da31b92 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Mon, 9 Aug 2021 22:19:54 +0300 Subject: [PATCH 374/599] fixup --- .../01572_kill_window_function.reference | 4 ++- .../0_stateless/01572_kill_window_function.sh | 26 +++++++------------ 2 files changed, 12 insertions(+), 18 deletions(-) diff --git a/tests/queries/0_stateless/01572_kill_window_function.reference b/tests/queries/0_stateless/01572_kill_window_function.reference index d86bac9de59..f1218bf5bdf 100644 --- a/tests/queries/0_stateless/01572_kill_window_function.reference +++ b/tests/queries/0_stateless/01572_kill_window_function.reference @@ -1 +1,3 @@ -OK +Started +Sent kill request +Exit 138 diff --git a/tests/queries/0_stateless/01572_kill_window_function.sh b/tests/queries/0_stateless/01572_kill_window_function.sh index e596c59aa30..efb70cbf71d 100755 --- a/tests/queries/0_stateless/01572_kill_window_function.sh +++ b/tests/queries/0_stateless/01572_kill_window_function.sh @@ -10,32 +10,24 @@ set -e -o pipefail query_id="01572_kill_window_function" $CLICKHOUSE_CLIENT --query_id="$query_id" --query "SELECT count(1048575) OVER (PARTITION BY intDiv(NULL, number) ORDER BY number DESC NULLS FIRST ROWS BETWEEN CURRENT ROW AND 1048575 FOLLOWING) FROM numbers(255, 1048575)" >/dev/null 2>&1 & client_pid=$! +echo Started -# First check that clickhouse-client is still alive, and then use one query to -# both kill the test query and verify that it has started. -# If we try to kill it before it starts, the test will fail. -# If it finishes after we check it started, the test will fail. -# That's why it is better to check and kill with the same call. -while kill -0 $client_pid \ - && [ -z "$(CLICKHOUSE_CLIENT --query "kill query where query_id = '$query_id' and current_database = currentDatabase()")" ] +# Use one query to both kill the test query and verify that it has started, +# because if we try to kill it before it starts, the test will fail. +while [ -z "$($CLICKHOUSE_CLIENT --query "kill query where query_id = '$query_id' and current_database = currentDatabase()")" ] do + # If we don't yet see the query in the process list, the client should still + # be running. The query is very long. + kill -0 -- $client_pid sleep 1 done +echo Sent kill request # Wait for the client to terminate. client_exit_code=0 wait $client_pid || client_exit_code=$? -# Note that we still have to check for normal termination, because the test query -# might have completed even before we first tried to kill it. This shouldn't -# really happen because it's very long, but our CI infractructure is known to -# introduce the most unexpected delays. -if [ $client_exit_code -eq 0 ] || [ $client_exit_code -eq 138 ] -then - echo "OK" -else - echo "Got unexpected client exit code $client_exit_code" -fi +echo "Exit $client_exit_code" # We have tested for Ctrl+C. # The following client flags don't cancel, but should: --max_execution_time, From 9b28fb464e4db641492f99f117543fcc4e207fa3 Mon Sep 17 00:00:00 2001 From: Sergei Semin Date: Mon, 9 Aug 2021 22:36:01 +0300 Subject: [PATCH 375/599] add info about ninja usage in clion instead of default make --- docs/ru/development/developer-instruction.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/development/developer-instruction.md b/docs/ru/development/developer-instruction.md index 73b00228d93..d23c0bbbdca 100644 --- a/docs/ru/development/developer-instruction.md +++ b/docs/ru/development/developer-instruction.md @@ -241,7 +241,7 @@ sudo ./llvm.sh 12 В качестве простых редакторов кода можно использовать Sublime Text или Visual Studio Code или Kate (все варианты доступны под Linux). -На всякий случай заметим, что CLion самостоятельно создаёт свою build директорию, самостоятельно выбирает тип сборки debug по-умолчанию, для конфигурации использует встроенную в CLion версию CMake вместо установленного вами, а для запуска задач использует make вместо ninja. Это нормально, просто имейте это ввиду, чтобы не возникало путаницы. +На всякий случай заметим, что CLion самостоятельно создаёт свою build директорию, самостоятельно выбирает тип сборки debug по-умолчанию, для конфигурации использует встроенную в CLion версию CMake вместо установленного вами, а для запуска задач использует make вместо ninja (но при желании начиная с версии CLion 2019.3 EAP можно настроить использование ninja, см. подробнее [тут](https://blog.jetbrains.com/clion/2019/10/clion-2019-3-eap-ninja-cmake-generators/)). Это нормально, просто имейте это ввиду, чтобы не возникало путаницы. ## Написание кода {#napisanie-koda} From 3794e9e980c282af5f25fdce7813da79a5464c74 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 9 Aug 2021 23:24:33 +0300 Subject: [PATCH 376/599] Try fix build. --- src/Common/DenseHashMap.h | 1 + src/Common/DenseHashSet.h | 1 + src/Common/SparseHashMap.h | 1 + 3 files changed, 3 insertions(+) diff --git a/src/Common/DenseHashMap.h b/src/Common/DenseHashMap.h index 0097faa5259..d27b8d93a27 100644 --- a/src/Common/DenseHashMap.h +++ b/src/Common/DenseHashMap.h @@ -1,6 +1,7 @@ #pragma once #if defined(ARCADIA_BUILD) +#undef HASH_FUN_H #define HASH_FUN_H #endif diff --git a/src/Common/DenseHashSet.h b/src/Common/DenseHashSet.h index 9741bf5ec5c..750dc829817 100644 --- a/src/Common/DenseHashSet.h +++ b/src/Common/DenseHashSet.h @@ -1,6 +1,7 @@ #pragma once #if defined(ARCADIA_BUILD) +#undef HASH_FUN_H #define HASH_FUN_H #endif diff --git a/src/Common/SparseHashMap.h b/src/Common/SparseHashMap.h index c5fa07410af..22847499bce 100644 --- a/src/Common/SparseHashMap.h +++ b/src/Common/SparseHashMap.h @@ -1,6 +1,7 @@ #pragma once #if defined(ARCADIA_BUILD) +#undef HASH_FUN_H #define HASH_FUN_H #endif From 599cb22061dddf06c62b7eee92a7c8b5b08ba7ef Mon Sep 17 00:00:00 2001 From: Sergei Semin Date: Mon, 9 Aug 2021 23:51:05 +0300 Subject: [PATCH 377/599] add info about speedy shared build MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit from tg chat `ClickHouse developers, public` ``` prll, [06.02.19 12:54] есть специальное заклинание для разработчиков увеличивающее скорость и снижающее жручесть памяти - cmake -DUSE_DEBUG_HELPERS=1 -DUSE_STATIC_LIBRARIES=0 -DSPLIT_SHARED_LIBRARIES=1 -DCLICKHOUSE_SPLIT_BINARY=1 ``` --- docs/ru/development/developer-instruction.md | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/docs/ru/development/developer-instruction.md b/docs/ru/development/developer-instruction.md index d23c0bbbdca..c8f6e6aba44 100644 --- a/docs/ru/development/developer-instruction.md +++ b/docs/ru/development/developer-instruction.md @@ -168,7 +168,13 @@ sudo bash -c "$(wget -O - https://apt.llvm.org/llvm.sh)" cmake -D CMAKE_BUILD_TYPE=Debug .. -Вы можете изменить вариант сборки, выполнив эту команду в директории build. +Для ещё более быстрой сборки и более экономного расхода места на диске при осуществлении сборки можно также использоавть специальные опции: +```bash +cmake -DUSE_DEBUG_HELPERS=1 -DUSE_STATIC_LIBRARIES=0 -DSPLIT_SHARED_LIBRARIES=1 -DCLICKHOUSE_SPLIT_BINARY=1 .. +``` +При этом надо учесть, что получаемые в результате сборки исполнимые файлы будут динамически слинкованы с библиотеками, в результате фактически станут непереносимыми на другие компьютеры (либо для этого нужно будет предпринять значительно больше усилий по сравнению со статической сборкой). Плюсом же в данном случае является значительно меньшее время сборки и значительно меньшее использование места на жёстком диске при сборке. Для целей разработки, когда планируются только отладочные запуске на том же компьютере, где осуществлялась сборка, это может быть наиболее удобным вариантом. + +Вы можете изменить вариант сборки, выполнив новую команду в директории build. Запустите ninja для сборки: From dff51cb10534e1a0673f71d1808501e34171d471 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Mon, 9 Aug 2021 23:56:44 +0300 Subject: [PATCH 378/599] fixup --- tests/queries/0_stateless/01572_kill_window_function.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01572_kill_window_function.sh b/tests/queries/0_stateless/01572_kill_window_function.sh index efb70cbf71d..7103b7f7210 100755 --- a/tests/queries/0_stateless/01572_kill_window_function.sh +++ b/tests/queries/0_stateless/01572_kill_window_function.sh @@ -7,7 +7,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) set -e -o pipefail # Run a test query that takes very long to run. -query_id="01572_kill_window_function" +query_id="01572_kill_window_function-$CLICKHOUSE_DATABASE" $CLICKHOUSE_CLIENT --query_id="$query_id" --query "SELECT count(1048575) OVER (PARTITION BY intDiv(NULL, number) ORDER BY number DESC NULLS FIRST ROWS BETWEEN CURRENT ROW AND 1048575 FOLLOWING) FROM numbers(255, 1048575)" >/dev/null 2>&1 & client_pid=$! echo Started From 9763dd18b1838b0e7edde89242e10c168c86ba60 Mon Sep 17 00:00:00 2001 From: olgarev <56617294+olgarev@users.noreply.github.com> Date: Tue, 10 Aug 2021 00:46:25 +0300 Subject: [PATCH 379/599] Update docs/en/sql-reference/statements/select/distinct.md Co-authored-by: Anna <42538400+adevyatova@users.noreply.github.com> --- docs/en/sql-reference/statements/select/distinct.md | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/docs/en/sql-reference/statements/select/distinct.md b/docs/en/sql-reference/statements/select/distinct.md index 1c739aab190..390afa46248 100644 --- a/docs/en/sql-reference/statements/select/distinct.md +++ b/docs/en/sql-reference/statements/select/distinct.md @@ -6,7 +6,7 @@ toc_title: DISTINCT If `SELECT DISTINCT` is specified, only unique rows will remain in a query result. Thus only a single row will remain out of all the sets of fully matching rows in the result. -You can narrow the list of columns which must have unique values: `SELECT DISTINCT ON (column1, column2,...)`. If the columns are not specified, all of them are taken into consideration. +You can specify the list of columns that must have unique values: `SELECT DISTINCT ON (column1, column2,...)`. If the columns are not specified, all of them are taken into consideration. Consider the table: @@ -107,4 +107,3 @@ It is possible to obtain the same result by applying [GROUP BY](../../../sql-ref - `DISTINCT` can be applied together with `GROUP BY`. - When [ORDER BY](../../../sql-reference/statements/select/order-by.md) is omitted and [LIMIT](../../../sql-reference/statements/select/limit.md) is defined, the query stops running immediately after the required number of different rows has been read. - Data blocks are output as they are processed, without waiting for the entire query to finish running. - From dbed356edf19ff71197a6c20db83e6108f9ef47c Mon Sep 17 00:00:00 2001 From: Braulio Valdivielso Date: Mon, 9 Aug 2021 22:45:16 +0100 Subject: [PATCH 380/599] clean up and add more tests --- src/Functions/namedTupleItems.cpp | 19 +++++++++--------- .../02007_namedtupleitems.reference | 3 +++ .../0_stateless/02007_namedtupleitems.sql | 20 +++++++++++++++---- 3 files changed, 29 insertions(+), 13 deletions(-) diff --git a/src/Functions/namedTupleItems.cpp b/src/Functions/namedTupleItems.cpp index 598fe79b9fa..1fd1397a1ef 100644 --- a/src/Functions/namedTupleItems.cpp +++ b/src/Functions/namedTupleItems.cpp @@ -12,7 +12,6 @@ #include #include - namespace DB { namespace ErrorCodes @@ -58,14 +57,13 @@ public: const DataTypeTuple * tuple = checkAndGetDataType(col); if (!tuple) - throw Exception("First argument for function " + getName() + "must " + throw Exception("First argument for function " + getName() + " must " "be a tuple.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - const auto& elementTypes = tuple->getElements(); - + if (elementTypes.empty()) - throw Exception("The argument tuple for function " + getName() + "must " + throw Exception("The argument tuple for function " + getName() + " must " "not be empty.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); const auto& firstElementType = elementTypes[0]; @@ -73,13 +71,16 @@ public: auto it = std::find_if( elementTypes.begin() + 1, elementTypes.end(), - [&](const auto &other) { + [&](const auto &other) + { return !firstElementType->equals(*other); }); - + if (it != elementTypes.end()) { - throw Exception("TODO: FIX", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + throw Exception("The argument tuple for function " + getName() + " must " + "contain just one type", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); } DataTypePtr tupleNameType = std::make_shared(); @@ -87,7 +88,7 @@ public: firstElementType}; auto itemDataType = std::make_shared(itemDataTypes); - + return std::make_shared(itemDataType); } diff --git a/tests/queries/0_stateless/02007_namedtupleitems.reference b/tests/queries/0_stateless/02007_namedtupleitems.reference index 985425a01c5..90d15b16dcd 100644 --- a/tests/queries/0_stateless/02007_namedtupleitems.reference +++ b/tests/queries/0_stateless/02007_namedtupleitems.reference @@ -1,3 +1,6 @@ [('1',1),('2',2),('3',3)] [('a',(1,2)),('b',(3,4))] [('a',(5,6)),('b',(7,8))] +CPU 3.3 +Memory 5.5 +Disk 6.6 diff --git a/tests/queries/0_stateless/02007_namedtupleitems.sql b/tests/queries/0_stateless/02007_namedtupleitems.sql index cda8a9a654c..02c6e57a0fb 100644 --- a/tests/queries/0_stateless/02007_namedtupleitems.sql +++ b/tests/queries/0_stateless/02007_namedtupleitems.sql @@ -1,4 +1,5 @@ SELECT namedTupleItems(tuple(1, 2, 3)); +-- [('1',1),('2',2),('3',3)] DROP TABLE IF EXISTS test02007; CREATE TABLE test02007 ( @@ -9,13 +10,24 @@ CREATE TABLE test02007 ( ) ENGINE=Memory(); INSERT INTO test02007 VALUES (tuple(tuple(1, 2), tuple(3, 4))); INSERT INTO test02007 VALUES (tuple(tuple(5, 6), tuple(7, 8))); - SELECT namedTupleItems(col) FROM test02007 ORDER BY col; -DROP TABLE IF EXISTS test02007; -SELECT namedTupleItems(tuple(1, 1.3)); -- this should work, will fix +-- [('a',(1,2)),('b',(3,4))] +-- [('a',(5,6)),('b',(7,8))] +DROP TABLE IF EXISTS test02007; +CREATE TABLE test02007 ( + col Tuple(CPU double, Memory double, Disk double) +) ENGINE=Memory(); +INSERT INTO test02007 VALUES (tuple(3.3, 5.5, 6.6)); +SELECT untuple(arrayJoin(namedTupleItems(col))) from test02007; +-- CPU 3.3 +-- Memory 5.5 +-- Disk 6.6 + +DROP TABLE IF EXISTS test02007; +SELECT namedTupleItems(tuple(1, 1.3)); -- { serverError 43; } should it? SELECT namedTupleItems(tuple(1, [1,2])); -- { serverError 43; } -SELECT namedTupleItems(tuple(1, 'a')) -- { serverError 43; } +SELECT namedTupleItems(tuple(1, 'a')); -- { serverError 43; } SELECT namedTupleItems(33); -- { serverError 43; } From eda489dde8508474a0d56e7988e514fd3cb912c3 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 10 Aug 2021 02:58:51 +0300 Subject: [PATCH 381/599] Untangle CPU features specification in CMake --- CMakeLists.txt | 6 -- cmake/cpu_features.cmake | 199 +++++++++++++++++++++------------------ 2 files changed, 109 insertions(+), 96 deletions(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index d3cb5f70c83..1727caea766 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -271,12 +271,6 @@ endif() include(cmake/cpu_features.cmake) -option(ARCH_NATIVE "Add -march=native compiler flag. This makes your binaries non-portable but more performant code may be generated.") - -if (ARCH_NATIVE) - set (COMPILER_FLAGS "${COMPILER_FLAGS} -march=native") -endif () - # Asynchronous unwind tables are needed for Query Profiler. # They are already by default on some platforms but possibly not on all platforms. # Enable it explicitly. diff --git a/cmake/cpu_features.cmake b/cmake/cpu_features.cmake index d12eac2e3c4..46e42329958 100644 --- a/cmake/cpu_features.cmake +++ b/cmake/cpu_features.cmake @@ -5,109 +5,128 @@ include (CMakePushCheckState) cmake_push_check_state () -# gcc -dM -E -mno-sse2 - < /dev/null | sort > gcc-dump-nosse2 -# gcc -dM -E -msse2 - < /dev/null | sort > gcc-dump-sse2 -#define __SSE2__ 1 -#define __SSE2_MATH__ 1 +# The variables HAVE_* determine if compiler has support for the flag to use the corresponding instruction set. +# The options ENABLE_* determine if we will tell compiler to actually use the corresponding instruction set if compiler can do it. -# gcc -dM -E -msse4.1 - < /dev/null | sort > gcc-dump-sse41 -#define __SSE4_1__ 1 +# All of them are unrelated to the instruction set at the host machine +# (you can compile for newer instruction set on old machines and vice versa). -set (TEST_FLAG "-msse4.1") -set (CMAKE_REQUIRED_FLAGS "${TEST_FLAG} -O0") -check_cxx_source_compiles(" - #include - int main() { - auto a = _mm_insert_epi8(__m128i(), 0, 0); - (void)a; - return 0; - } -" HAVE_SSE41) -if (HAVE_SSE41) - set (COMPILER_FLAGS "${COMPILER_FLAGS} ${TEST_FLAG}") -endif () +option (ENABLE_SSSE3 "Use SSSE3 instructions on x86_64" 1) +option (ENABLE_SSE41 "Use SSE4.1 instructions on x86_64" 1) +option (ENABLE_SSE42 "Use SSE4.2 instructions on x86_64" 1) +option (ENABLE_PCLMULQDQ "Use pclmulqdq instructions on x86_64" 1) +option (ENABLE_POPCNT "Use popcnt instructions on x86_64" 1) +option (ENABLE_AVX "Use AVX instructions on x86_64" 0) +option (ENABLE_AVX2 "Use AVX2 instructions on x86_64" 0) -if (ARCH_PPC64LE) - set (COMPILER_FLAGS "${COMPILER_FLAGS} -maltivec -D__SSE2__=1 -DNO_WARN_X86_INTRINSICS") -endif () +option (ARCH_NATIVE "Add -march=native compiler flag. This makes your binaries non-portable but more performant code may be generated. This option overrides ENABLE_* options for specific instruction set. Highly not recommended to use." 0) -# gcc -dM -E -msse4.2 - < /dev/null | sort > gcc-dump-sse42 -#define __SSE4_2__ 1 +if (ARCH_NATIVE) + set (COMPILER_FLAGS "${COMPILER_FLAGS} -march=native") -set (TEST_FLAG "-msse4.2") -set (CMAKE_REQUIRED_FLAGS "${TEST_FLAG} -O0") -check_cxx_source_compiles(" - #include - int main() { - auto a = _mm_crc32_u64(0, 0); - (void)a; - return 0; - } -" HAVE_SSE42) -if (HAVE_SSE42) - set (COMPILER_FLAGS "${COMPILER_FLAGS} ${TEST_FLAG}") -endif () +else () + set (TEST_FLAG "-mssse3") + set (CMAKE_REQUIRED_FLAGS "${TEST_FLAG} -O0") + check_cxx_source_compiles(" + #include + int main() { + __m64 a = _mm_abs_pi8(__m64()); + (void)a; + return 0; + } + " HAVE_SSSE3) + if (HAVE_SSSE3 AND ENABLE_SSSE3) + set (COMPILER_FLAGS "${COMPILER_FLAGS} ${TEST_FLAG}") + endif () -set (TEST_FLAG "-mssse3") -set (CMAKE_REQUIRED_FLAGS "${TEST_FLAG} -O0") -check_cxx_source_compiles(" - #include - int main() { - __m64 a = _mm_abs_pi8(__m64()); - (void)a; - return 0; - } -" HAVE_SSSE3) -set (TEST_FLAG "-mavx") -set (CMAKE_REQUIRED_FLAGS "${TEST_FLAG} -O0") -check_cxx_source_compiles(" - #include - int main() { - auto a = _mm256_insert_epi8(__m256i(), 0, 0); - (void)a; - return 0; - } -" HAVE_AVX) + set (TEST_FLAG "-msse4.1") + set (CMAKE_REQUIRED_FLAGS "${TEST_FLAG} -O0") + check_cxx_source_compiles(" + #include + int main() { + auto a = _mm_insert_epi8(__m128i(), 0, 0); + (void)a; + return 0; + } + " HAVE_SSE41) + if (HAVE_SSE41 AND ENABLE_SSE41) + set (COMPILER_FLAGS "${COMPILER_FLAGS} ${TEST_FLAG}") + endif () -set (TEST_FLAG "-mavx2") -set (CMAKE_REQUIRED_FLAGS "${TEST_FLAG} -O0") -check_cxx_source_compiles(" - #include - int main() { - auto a = _mm256_add_epi16(__m256i(), __m256i()); - (void)a; - return 0; - } -" HAVE_AVX2) + if (ARCH_PPC64LE) + set (COMPILER_FLAGS "${COMPILER_FLAGS} -maltivec -D__SSE2__=1 -DNO_WARN_X86_INTRINSICS") + endif () -set (TEST_FLAG "-mpclmul") -set (CMAKE_REQUIRED_FLAGS "${TEST_FLAG} -O0") -check_cxx_source_compiles(" - #include - int main() { - auto a = _mm_clmulepi64_si128(__m128i(), __m128i(), 0); - (void)a; - return 0; - } -" HAVE_PCLMULQDQ) + set (TEST_FLAG "-msse4.2") + set (CMAKE_REQUIRED_FLAGS "${TEST_FLAG} -O0") + check_cxx_source_compiles(" + #include + int main() { + auto a = _mm_crc32_u64(0, 0); + (void)a; + return 0; + } + " HAVE_SSE42) + if (HAVE_SSE42 AND ENABLE_SSE42) + set (COMPILER_FLAGS "${COMPILER_FLAGS} ${TEST_FLAG}") + endif () -# gcc -dM -E -mpopcnt - < /dev/null | sort > gcc-dump-popcnt -#define __POPCNT__ 1 + set (TEST_FLAG "-mpclmul") + set (CMAKE_REQUIRED_FLAGS "${TEST_FLAG} -O0") + check_cxx_source_compiles(" + #include + int main() { + auto a = _mm_clmulepi64_si128(__m128i(), __m128i(), 0); + (void)a; + return 0; + } + " HAVE_PCLMULQDQ) + if (HAVE_PCLMULQDQ AND ENABLE_PCLMULQDQ) + set (COMPILER_FLAGS "${COMPILER_FLAGS} ${TEST_FLAG}") + endif () -set (TEST_FLAG "-mpopcnt") + set (TEST_FLAG "-mpopcnt") -set (CMAKE_REQUIRED_FLAGS "${TEST_FLAG} -O0") -check_cxx_source_compiles(" - int main() { - auto a = __builtin_popcountll(0); - (void)a; - return 0; - } -" HAVE_POPCNT) + set (CMAKE_REQUIRED_FLAGS "${TEST_FLAG} -O0") + check_cxx_source_compiles(" + int main() { + auto a = __builtin_popcountll(0); + (void)a; + return 0; + } + " HAVE_POPCNT) + if (HAVE_POPCNT AND ENABLE_POPCNT) + set (COMPILER_FLAGS "${COMPILER_FLAGS} ${TEST_FLAG}") + endif () -if (HAVE_POPCNT AND NOT ARCH_AARCH64) - set (COMPILER_FLAGS "${COMPILER_FLAGS} ${TEST_FLAG}") + set (TEST_FLAG "-mavx") + set (CMAKE_REQUIRED_FLAGS "${TEST_FLAG} -O0") + check_cxx_source_compiles(" + #include + int main() { + auto a = _mm256_insert_epi8(__m256i(), 0, 0); + (void)a; + return 0; + } + " HAVE_AVX) + if (HAVE_AVX AND ENABLE_AVX) + set (COMPILER_FLAGS "${COMPILER_FLAGS} ${TEST_FLAG}") + endif () + + set (TEST_FLAG "-mavx2") + set (CMAKE_REQUIRED_FLAGS "${TEST_FLAG} -O0") + check_cxx_source_compiles(" + #include + int main() { + auto a = _mm256_add_epi16(__m256i(), __m256i()); + (void)a; + return 0; + } + " HAVE_AVX2) + if (HAVE_AVX2 AND ENABLE_AVX2) + set (COMPILER_FLAGS "${COMPILER_FLAGS} ${TEST_FLAG}") + endif () endif () cmake_pop_check_state () From 3449e611fd8ea5b332425f5995de63c10cf0c841 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 10 Aug 2021 03:46:03 +0300 Subject: [PATCH 382/599] Allow to build with AVX2 --- contrib/simdjson-cmake/CMakeLists.txt | 3 +++ 1 file changed, 3 insertions(+) diff --git a/contrib/simdjson-cmake/CMakeLists.txt b/contrib/simdjson-cmake/CMakeLists.txt index d3bcf6c046c..862d8dc50f8 100644 --- a/contrib/simdjson-cmake/CMakeLists.txt +++ b/contrib/simdjson-cmake/CMakeLists.txt @@ -4,3 +4,6 @@ set(SIMDJSON_SRC "${SIMDJSON_SRC_DIR}/simdjson.cpp") add_library(simdjson ${SIMDJSON_SRC}) target_include_directories(simdjson SYSTEM PUBLIC "${SIMDJSON_INCLUDE_DIR}" PRIVATE "${SIMDJSON_SRC_DIR}") + +# simdjson is using its own CPU dispatching and get confused if we enable AVX/AVX2 flags. +target_compile_options(simdjson PRIVATE -mno-avx -mno-avx2) From 8983f4bc12c42bc593177a0cc613b1b3e8d4952c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 10 Aug 2021 08:13:08 +0300 Subject: [PATCH 383/599] Better block drawing characters in Web UI --- programs/server/play.html | 15 ++++++++++----- 1 file changed, 10 insertions(+), 5 deletions(-) diff --git a/programs/server/play.html b/programs/server/play.html index 503bb92d03e..2095d7d518a 100644 --- a/programs/server/play.html +++ b/programs/server/play.html @@ -96,11 +96,16 @@ .monospace { /* Prefer fonts that have full hinting info. This is important for non-retina displays. - Also I personally dislike "Ubuntu" font due to the similarity of 'r' and 'г' (it looks very ignorant). - */ + Also I personally dislike "Ubuntu" font due to the similarity of 'r' and 'г' (it looks very ignorant). */ font-family: Liberation Mono, DejaVu Sans Mono, MonoLisa, Consolas, Monospace; } + .monospace-table + { + /* Liberation is worse than DejaVu for block drawing characters. */ + font-family: DejaVu Sans Mono, Liberation Mono, MonoLisa, Consolas, Monospace; + } + .shadow { box-shadow: 0 0 1rem var(--shadow-color); @@ -325,8 +330,8 @@ 🌑🌞

-
-

+        
+

     

@@ -367,7 +372,7 @@ const server_address = document.getElementById('url').value; const url = server_address + - (server_address.indexOf('?') >= 0 ? '&' : '?') + + (server_address.indexOf('?') >= 0 ? '&' : '?') + /// Ask server to allow cross-domain requests. 'add_http_cors_header=1' + '&user=' + encodeURIComponent(user) + From 55b87a788b3bd58c6438e9c802ff083bc1ab320a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 10 Aug 2021 08:18:18 +0300 Subject: [PATCH 384/599] Fix font in Chromium --- programs/server/play.html | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/programs/server/play.html b/programs/server/play.html index 2095d7d518a..8fb32efe98e 100644 --- a/programs/server/play.html +++ b/programs/server/play.html @@ -68,7 +68,7 @@ html, body { /* Personal choice. */ - font-family: Sans-Serif; + font-family: Liberation Sans, DejaVu Sans, sans-serif, Noto Color Emoji, Apple Color Emoji, Segoe UI Emoji; background: var(--background-color); color: var(--text-color); } From 7abec83d5bc6e1f8633764eb37c3ed79643ef539 Mon Sep 17 00:00:00 2001 From: Misko Lee Date: Tue, 10 Aug 2021 13:36:28 +0800 Subject: [PATCH 385/599] add from IP on login fail error message, it's helpful detect traffic source on error message. --- src/Access/IAccessStorage.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Access/IAccessStorage.cpp b/src/Access/IAccessStorage.cpp index 348987899cb..f0fbb95ff4e 100644 --- a/src/Access/IAccessStorage.cpp +++ b/src/Access/IAccessStorage.cpp @@ -455,7 +455,7 @@ UUID IAccessStorage::login( if (!replace_exception_with_cannot_authenticate) throw; - tryLogCurrentException(getLogger(), credentials.getUserName() + ": Authentication failed"); + tryLogCurrentException(getLogger(), "from: " + address.toString() + ", user: " + credentials.getUserName() + ": Authentication failed"); throwCannotAuthenticate(credentials.getUserName()); } } From 4eb4dd97d4d1869d713c901b053fcb18bdb0ae23 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 10 Aug 2021 09:24:12 +0300 Subject: [PATCH 386/599] Support not only 2 inputs, support any size sequence of intersect / except together --- .../InterpreterIntersectOrExcept.cpp | 13 ++-- .../InterpreterIntersectOrExcept.h | 6 +- src/Parsers/ASTIntersectOrExcept.cpp | 55 +++++++++++-- src/Parsers/ASTIntersectOrExcept.h | 16 +++- src/Parsers/ParserIntersectOrExcept.cpp | 61 ++++++++++----- .../QueryPlan/IntersectOrExceptStep.cpp | 7 +- .../QueryPlan/IntersectOrExceptStep.h | 10 ++- .../Transforms/IntersectOrExceptTransform.cpp | 78 +++++++++++++------ .../Transforms/IntersectOrExceptTransform.h | 21 +++-- ...02004_intersect_except_operators.reference | 30 +++++++ .../02004_intersect_except_operators.sql | 10 +++ 11 files changed, 234 insertions(+), 73 deletions(-) diff --git a/src/Interpreters/InterpreterIntersectOrExcept.cpp b/src/Interpreters/InterpreterIntersectOrExcept.cpp index c85c39824d8..d706ab02bfd 100644 --- a/src/Interpreters/InterpreterIntersectOrExcept.cpp +++ b/src/Interpreters/InterpreterIntersectOrExcept.cpp @@ -21,18 +21,21 @@ namespace ErrorCodes InterpreterIntersectOrExcept::InterpreterIntersectOrExcept(const ASTPtr & query_ptr, ContextPtr context_) : context(Context::createCopy(context_)) - , is_except(query_ptr->as()->is_except) { ASTIntersectOrExcept * ast = query_ptr->as(); + auto children = ast->list_of_selects->children; + modes = ast->list_of_modes; + if (modes.size() + 1 != children.size()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Number of modes and number of children are not consistent"); - size_t num_children = ast->children.size(); + size_t num_children = children.size(); if (!num_children) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: no children in ASTIntersectOrExceptQuery"); + throw Exception(ErrorCodes::LOGICAL_ERROR, "No children in ASTIntersectOrExceptQuery"); nested_interpreters.resize(num_children); for (size_t i = 0; i < num_children; ++i) - nested_interpreters[i] = buildCurrentChildInterpreter(ast->children[i]); + nested_interpreters[i] = buildCurrentChildInterpreter(children.at(i)); Blocks headers(num_children); for (size_t query_num = 0; query_num < num_children; ++query_num) @@ -103,7 +106,7 @@ void InterpreterIntersectOrExcept::buildQueryPlan(QueryPlan & query_plan) } auto max_threads = context->getSettingsRef().max_threads; - auto step = std::make_unique(is_except, std::move(data_streams), max_threads); + auto step = std::make_unique(std::move(data_streams), modes, max_threads); query_plan.unitePlans(std::move(step), std::move(plans)); } diff --git a/src/Interpreters/InterpreterIntersectOrExcept.h b/src/Interpreters/InterpreterIntersectOrExcept.h index 34a58c0c05a..8021a25df06 100644 --- a/src/Interpreters/InterpreterIntersectOrExcept.h +++ b/src/Interpreters/InterpreterIntersectOrExcept.h @@ -3,6 +3,8 @@ #include #include #include +#include + namespace DB { @@ -19,7 +21,7 @@ public: BlockIO execute() override; private: - String getName() const { return is_except ? "EXCEPT" : "INTERSECT"; } + String getName() const { return "IntersectExcept"; } Block getCommonHeader(const Blocks & headers) const; @@ -29,9 +31,9 @@ private: void buildQueryPlan(QueryPlan & query_plan); ContextPtr context; - bool is_except; Block result_header; std::vector> nested_interpreters; + ASTIntersectOrExcept::Modes modes; }; } diff --git a/src/Parsers/ASTIntersectOrExcept.cpp b/src/Parsers/ASTIntersectOrExcept.cpp index a05d7ee86c9..7d92055646c 100644 --- a/src/Parsers/ASTIntersectOrExcept.cpp +++ b/src/Parsers/ASTIntersectOrExcept.cpp @@ -1,5 +1,7 @@ #include #include +#include + namespace DB { @@ -8,21 +10,58 @@ ASTPtr ASTIntersectOrExcept::clone() const { auto res = std::make_shared(*this); res->children.clear(); - res->children.push_back(children[0]->clone()); - res->children.push_back(children[1]->clone()); - res->is_except = is_except; + + res->list_of_selects = list_of_selects->clone(); + res->children.push_back(res->list_of_selects); + res->list_of_modes = list_of_modes; + cloneOutputOptions(*res); return res; } void ASTIntersectOrExcept::formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const { - children[0]->formatImpl(settings, state, frame); std::string indent_str = settings.one_line ? "" : std::string(4 * frame.indent, ' '); - settings.ostr << settings.nl_or_ws << indent_str << (settings.hilite ? hilite_keyword : "") - << (is_except ? "EXCEPT" : "INTERSECT") - << (settings.hilite ? hilite_none : "") << settings.nl_or_ws; - children[1]->formatImpl(settings, state, frame); + + auto mode_to_str = [&](auto mode) + { + if (mode == Mode::INTERSECT) + return "INTERSECT"; + else + return "EXCEPT"; + }; + + for (ASTs::const_iterator it = list_of_selects->children.begin(); it != list_of_selects->children.end(); ++it) + { + if (it != list_of_selects->children.begin()) + { + settings.ostr << settings.nl_or_ws << indent_str << (settings.hilite ? hilite_keyword : "") + << mode_to_str(list_of_modes[it - list_of_selects->children.begin() - 1]) + << (settings.hilite ? hilite_none : ""); + } + + if (auto * node = (*it)->as()) + { + settings.ostr << settings.nl_or_ws << indent_str; + + if (node->list_of_selects->children.size() == 1) + { + (node->list_of_selects->children.at(0))->formatImpl(settings, state, frame); + } + else + { + auto sub_query = std::make_shared(); + sub_query->children.push_back(*it); + sub_query->formatImpl(settings, state, frame); + } + } + else + { + if (it != list_of_selects->children.begin()) + settings.ostr << settings.nl_or_ws; + (*it)->formatImpl(settings, state, frame); + } + } } } diff --git a/src/Parsers/ASTIntersectOrExcept.h b/src/Parsers/ASTIntersectOrExcept.h index a02cb9f7d77..0eb8ab3b113 100644 --- a/src/Parsers/ASTIntersectOrExcept.h +++ b/src/Parsers/ASTIntersectOrExcept.h @@ -9,10 +9,22 @@ namespace DB class ASTIntersectOrExcept : public ASTQueryWithOutput { public: - String getID(char) const override { return is_except ? "Except" : "Intersect"; } + String getID(char) const override { return "IntersectExceptQuery"; } + ASTPtr clone() const override; + void formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override; - bool is_except; + + enum class Mode + { + INTERSECT, + EXCEPT + }; + + using Modes = std::vector; + + ASTPtr list_of_selects; + Modes list_of_modes; }; } diff --git a/src/Parsers/ParserIntersectOrExcept.cpp b/src/Parsers/ParserIntersectOrExcept.cpp index 6d5da54fa38..fd1511b8af6 100644 --- a/src/Parsers/ParserIntersectOrExcept.cpp +++ b/src/Parsers/ParserIntersectOrExcept.cpp @@ -4,41 +4,62 @@ #include #include #include +#include +#include + namespace DB { + bool ParserIntersectOrExcept::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { ParserKeyword intersect_keyword("INTERSECT"); ParserKeyword except_keyword("EXCEPT"); - ASTPtr left_node; - ASTPtr right_node; - auto ast = std::make_shared(); - ast->is_except = false; + ASTs elements; + ASTIntersectOrExcept::Modes modes; - if (!ParserSelectWithUnionQuery().parse(pos, left_node, expected) && !ParserSubquery().parse(pos, left_node, expected)) - return false; - - if (!intersect_keyword.ignore(pos)) + auto parse_element = [&]() -> bool { - if (!except_keyword.ignore(pos)) - { + ASTPtr element; + if (!ParserSelectWithUnionQuery().parse(pos, element, expected) && !ParserSubquery().parse(pos, element, expected)) return false; - } - else - { - ast->is_except = true; - } - } - if (!ParserSelectWithUnionQuery().parse(pos, right_node, expected) && !ParserSubquery().parse(pos, right_node, expected)) + elements.push_back(element); + return true; + }; + + auto parse_separator = [&]() -> bool + { + if (!intersect_keyword.ignore(pos)) + { + if (!except_keyword.ignore(pos)) + return false; + + modes.emplace_back(ASTIntersectOrExcept::Mode::EXCEPT); + return true; + } + + modes.emplace_back(ASTIntersectOrExcept::Mode::INTERSECT); + return true; + }; + + if (!ParserUnionList::parseUtil(pos, parse_element, parse_separator)) return false; - ast->children.push_back(left_node); - ast->children.push_back(right_node); + if (modes.empty()) + return false; + + auto list_node = std::make_shared(); + list_node->children = std::move(elements); + + auto intersect_or_except_ast = std::make_shared(); + + node = intersect_or_except_ast; + intersect_or_except_ast->list_of_selects = list_node; + intersect_or_except_ast->children.push_back(intersect_or_except_ast->list_of_selects); + intersect_or_except_ast->list_of_modes = modes; - node = ast; return true; } diff --git a/src/Processors/QueryPlan/IntersectOrExceptStep.cpp b/src/Processors/QueryPlan/IntersectOrExceptStep.cpp index f04885f4640..b1b5c1b8813 100644 --- a/src/Processors/QueryPlan/IntersectOrExceptStep.cpp +++ b/src/Processors/QueryPlan/IntersectOrExceptStep.cpp @@ -23,8 +23,8 @@ Block IntersectOrExceptStep::checkHeaders(const DataStreams & input_streams_) co return res; } -IntersectOrExceptStep::IntersectOrExceptStep(bool is_except_, DataStreams input_streams_, size_t max_threads_) - : is_except(is_except_), header(checkHeaders(input_streams_)), max_threads(max_threads_) +IntersectOrExceptStep::IntersectOrExceptStep(DataStreams input_streams_, const Modes & modes_, size_t max_threads_) + : header(checkHeaders(input_streams_)), modes(modes_), max_threads(max_threads_) { input_streams = std::move(input_streams_); if (input_streams.size() == 1) @@ -63,8 +63,9 @@ QueryPipelinePtr IntersectOrExceptStep::updatePipeline(QueryPipelines pipelines, } } + std::cerr << "size: " << input_streams.size() << std::endl; *pipeline = QueryPipeline::unitePipelines(std::move(pipelines), max_threads); - pipeline->addTransform(std::make_shared(is_except, header)); + pipeline->addTransform(std::make_shared(header, modes)); processors = collector.detachProcessors(); return pipeline; diff --git a/src/Processors/QueryPlan/IntersectOrExceptStep.h b/src/Processors/QueryPlan/IntersectOrExceptStep.h index 7938a9adad5..4eceb820153 100644 --- a/src/Processors/QueryPlan/IntersectOrExceptStep.h +++ b/src/Processors/QueryPlan/IntersectOrExceptStep.h @@ -1,16 +1,20 @@ #pragma once #include +#include + namespace DB { class IntersectOrExceptStep : public IQueryPlanStep { +using Modes = ASTIntersectOrExcept::Modes; + public: /// max_threads is used to limit the number of threads for result pipeline. - IntersectOrExceptStep(bool is_except_, DataStreams input_streams_, size_t max_threads_ = 0); + IntersectOrExceptStep(DataStreams input_streams_, const Modes & modes_, size_t max_threads_ = 0); - String getName() const override { return is_except ? "Except" : "Intersect"; } + String getName() const override { return "IntersectExcept"; } QueryPipelinePtr updatePipeline(QueryPipelines pipelines, const BuildQueryPipelineSettings & settings) override; @@ -19,8 +23,8 @@ public: private: Block checkHeaders(const DataStreams & input_streams_) const; - bool is_except; Block header; + Modes modes; size_t max_threads; Processors processors; }; diff --git a/src/Processors/Transforms/IntersectOrExceptTransform.cpp b/src/Processors/Transforms/IntersectOrExceptTransform.cpp index e5e8ff705c8..70a86855992 100644 --- a/src/Processors/Transforms/IntersectOrExceptTransform.cpp +++ b/src/Processors/Transforms/IntersectOrExceptTransform.cpp @@ -4,9 +4,11 @@ namespace DB { -IntersectOrExceptTransform::IntersectOrExceptTransform(bool is_except_, const Block & header_) - : IProcessor(InputPorts(2, header_), {header_}) - , is_except(is_except_) +IntersectOrExceptTransform::IntersectOrExceptTransform(const Block & header_, const Modes & modes_) + : IProcessor(InputPorts(modes_.size() + 1, header_), {header_}) + , modes(modes_) + , first_input(inputs.begin()) + , second_input(std::next(inputs.begin())) { const Names & columns = header_.getNames(); size_t num_columns = columns.empty() ? header_.columns() : columns.size(); @@ -14,8 +16,7 @@ IntersectOrExceptTransform::IntersectOrExceptTransform(bool is_except_, const Bl key_columns_pos.reserve(columns.size()); for (size_t i = 0; i < num_columns; ++i) { - auto pos = columns.empty() ? i - : header_.getPositionByName(columns[i]); + auto pos = columns.empty() ? i : header_.getPositionByName(columns[i]); key_columns_pos.emplace_back(pos); } } @@ -40,7 +41,7 @@ IntersectOrExceptTransform::Status IntersectOrExceptTransform::prepare() } /// Output if has data. - if (current_output_chunk) + if (current_output_chunk && second_input == inputs.end()) { output.push(std::move(current_output_chunk)); } @@ -53,28 +54,50 @@ IntersectOrExceptTransform::Status IntersectOrExceptTransform::prepare() if (finished_second_input) { - if (inputs.front().isFinished()) + if (first_input->isFinished() || (more && !current_input_chunk)) { - output.finish(); - return Status::Finished; + std::advance(second_input, 1); + + if (second_input == inputs.end()) + { + if (current_output_chunk) + { + output.push(std::move(current_output_chunk)); + } + output.finish(); + return Status::Finished; + } + else + { + more = true; + data.reset(); + finished_second_input = false; + ++current_operator_pos; + } } } - else if (inputs.back().isFinished()) + else if (second_input->isFinished()) { finished_second_input = true; } - InputPort & input = finished_second_input ? inputs.front() : inputs.back(); + InputPort & input = finished_second_input ? *first_input : *second_input; /// Check can input. if (!has_input) { - input.setNeeded(); + if (finished_second_input && more) + { + current_input_chunk = std::move(current_output_chunk); + } + else + { + input.setNeeded(); + if (!input.hasData()) + return Status::NeedData; + current_input_chunk = input.pull(); + } - if (!input.hasData()) - return Status::NeedData; - - current_input_chunk = input.pull(); has_input = true; } @@ -84,6 +107,9 @@ IntersectOrExceptTransform::Status IntersectOrExceptTransform::prepare() void IntersectOrExceptTransform::work() { + if (!data) + data.emplace(); + if (!finished_second_input) { accumulate(std::move(current_input_chunk)); @@ -118,7 +144,7 @@ size_t IntersectOrExceptTransform::buildFilter( for (size_t i = 0; i < rows; ++i) { auto find_result = state.findKey(method.data, i, variants.string_pool); - filter[i] = is_except ? !find_result.isFound() : find_result.isFound(); + filter[i] = modes[current_operator_pos] == ASTIntersectOrExcept::Mode::EXCEPT ? !find_result.isFound() : find_result.isFound(); if (filter[i]) ++new_rows_num; } @@ -137,16 +163,17 @@ void IntersectOrExceptTransform::accumulate(Chunk chunk) for (auto pos : key_columns_pos) column_ptrs.emplace_back(columns[pos].get()); - if (data.empty()) - data.init(SetVariants::chooseMethod(column_ptrs, key_sizes)); + if (data->empty()) + data->init(SetVariants::chooseMethod(column_ptrs, key_sizes)); - switch (data.type) + auto & data_set = *data; + switch (data->type) { case SetVariants::Type::EMPTY: break; #define M(NAME) \ case SetVariants::Type::NAME: \ - addToSet(*data.NAME, column_ptrs, num_rows, data); \ + addToSet(*data_set.NAME, column_ptrs, num_rows, data_set); \ break; APPLY_FOR_SET_VARIANTS(M) #undef M @@ -165,19 +192,20 @@ void IntersectOrExceptTransform::filter(Chunk & chunk) for (auto pos : key_columns_pos) column_ptrs.emplace_back(columns[pos].get()); - if (data.empty()) - data.init(SetVariants::chooseMethod(column_ptrs, key_sizes)); + if (data->empty()) + data->init(SetVariants::chooseMethod(column_ptrs, key_sizes)); IColumn::Filter filter(num_rows); size_t new_rows_num = 0; - switch (data.type) + auto & data_set = *data; + switch (data->type) { case SetVariants::Type::EMPTY: break; #define M(NAME) \ case SetVariants::Type::NAME: \ - new_rows_num = buildFilter(*data.NAME, column_ptrs, filter, num_rows, data); \ + new_rows_num = buildFilter(*data_set.NAME, column_ptrs, filter, num_rows, data_set); \ break; APPLY_FOR_SET_VARIANTS(M) #undef M diff --git a/src/Processors/Transforms/IntersectOrExceptTransform.h b/src/Processors/Transforms/IntersectOrExceptTransform.h index 3c2b9581d6d..5b62ef65d8d 100644 --- a/src/Processors/Transforms/IntersectOrExceptTransform.h +++ b/src/Processors/Transforms/IntersectOrExceptTransform.h @@ -3,16 +3,20 @@ #include #include #include +#include + namespace DB { class IntersectOrExceptTransform : public IProcessor { -public: - IntersectOrExceptTransform(bool is_except_, const Block & header_); +using Modes = ASTIntersectOrExcept::Modes; - String getName() const override { return is_except ? "Except" : "Intersect"; } +public: + IntersectOrExceptTransform(const Block & header_, const Modes & modes); + + String getName() const override { return "IntersectExcept"; } protected: Status prepare() override; @@ -20,15 +24,22 @@ protected: void work() override; private: - bool is_except; + Modes modes; + InputPorts::iterator first_input; + InputPorts::iterator second_input; + size_t current_operator_pos = 0; bool push_empty_chunk = false; Chunk empty_chunk; + ColumnNumbers key_columns_pos; - SetVariants data; + std::optional data; Sizes key_sizes; + Chunk current_input_chunk; Chunk current_output_chunk; + bool more = false; + bool finished_second_input = false; bool has_input = false; diff --git a/tests/queries/0_stateless/02004_intersect_except_operators.reference b/tests/queries/0_stateless/02004_intersect_except_operators.reference index 763a5872cce..d17216a5ec4 100644 --- a/tests/queries/0_stateless/02004_intersect_except_operators.reference +++ b/tests/queries/0_stateless/02004_intersect_except_operators.reference @@ -23,3 +23,33 @@ select number, number+10 from numbers(12) except select number+5, number+15 from 2 12 3 13 4 14 +select 1 except select 2 intersect select 1; +1 +select 1 except select 2 intersect select 2; +select 1 intersect select 1 except select 2; +1 +select 1 intersect select 1 except select 1; +select 1 intersect select 1 except select 2 intersect select 1 except select 3 intersect select 1; +1 +select 1 intersect select 1 except select 2 intersect select 1 except select 3 intersect select 2; +select number from numbers(100) intersect select number from numbers(20, 60) except select number from numbers(30, 20) except select number from numbers(60, 20); +20 +21 +22 +23 +24 +25 +26 +27 +28 +29 +50 +51 +52 +53 +54 +55 +56 +57 +58 +59 diff --git a/tests/queries/0_stateless/02004_intersect_except_operators.sql b/tests/queries/0_stateless/02004_intersect_except_operators.sql index d0416fd899e..971aa262070 100644 --- a/tests/queries/0_stateless/02004_intersect_except_operators.sql +++ b/tests/queries/0_stateless/02004_intersect_except_operators.sql @@ -3,6 +3,16 @@ select 1 intersect select 1; select 2 intersect select 1; select 1 except select 1; select 2 except select 1; + select number from numbers(5, 5) intersect select number from numbers(20); select number from numbers(10) except select number from numbers(5); select number, number+10 from numbers(12) except select number+5, number+15 from numbers(10); + +select 1 except select 2 intersect select 1; +select 1 except select 2 intersect select 2; +select 1 intersect select 1 except select 2; +select 1 intersect select 1 except select 1; +select 1 intersect select 1 except select 2 intersect select 1 except select 3 intersect select 1; +select 1 intersect select 1 except select 2 intersect select 1 except select 3 intersect select 2; + +select number from numbers(100) intersect select number from numbers(20, 60) except select number from numbers(30, 20) except select number from numbers(60, 20); From 07a6903009697d6eca621aa1bb2070a1b637da23 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 10 Aug 2021 09:35:35 +0300 Subject: [PATCH 387/599] Try fix build. --- src/Common/DenseHashMap.h | 7 +++++++ src/Common/DenseHashSet.h | 7 +++++++ src/Common/SparseHashMap.h | 7 +++++++ 3 files changed, 21 insertions(+) diff --git a/src/Common/DenseHashMap.h b/src/Common/DenseHashMap.h index d27b8d93a27..44287c604a7 100644 --- a/src/Common/DenseHashMap.h +++ b/src/Common/DenseHashMap.h @@ -5,7 +5,14 @@ #define HASH_FUN_H #endif +#ifdef __clang__ +#pragma clang diagnostic push +#pragma clang diagnostic ignored "-Wmacro-redefined" +#endif #include +#ifdef __clang__ +#pragma clang diagnostic pop +#endif #if !defined(ARCADIA_BUILD) template , diff --git a/src/Common/DenseHashSet.h b/src/Common/DenseHashSet.h index 750dc829817..ee3838d7be1 100644 --- a/src/Common/DenseHashSet.h +++ b/src/Common/DenseHashSet.h @@ -5,7 +5,14 @@ #define HASH_FUN_H #endif +#ifdef __clang__ +#pragma clang diagnostic push +#pragma clang diagnostic ignored "-Wmacro-redefined" +#endif #include +#ifdef __clang__ +#pragma clang diagnostic pop +#endif #if !defined(ARCADIA_BUILD) template , diff --git a/src/Common/SparseHashMap.h b/src/Common/SparseHashMap.h index 22847499bce..b6a7f1f9b9c 100644 --- a/src/Common/SparseHashMap.h +++ b/src/Common/SparseHashMap.h @@ -5,7 +5,14 @@ #define HASH_FUN_H #endif +#ifdef __clang__ +#pragma clang diagnostic push +#pragma clang diagnostic ignored "-Wmacro-redefined" +#endif #include +#ifdef __clang__ +#pragma clang diagnostic pop +#endif #if !defined(ARCADIA_BUILD) template , From 2306fbe9be189d9066eae32609d80a59585656da Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 10 Aug 2021 10:23:18 +0300 Subject: [PATCH 388/599] Better --- .../InterpreterIntersectOrExcept.cpp | 11 ++--- .../InterpreterIntersectOrExcept.h | 4 +- src/Parsers/ASTIntersectOrExcept.cpp | 8 ++-- src/Parsers/ASTIntersectOrExcept.h | 6 +-- ...t.cpp => ParserIntersectOrExceptQuery.cpp} | 17 ++++--- ...xcept.h => ParserIntersectOrExceptQuery.h} | 2 +- src/Parsers/ParserQueryWithOutput.cpp | 6 +-- .../QueryPlan/IntersectOrExceptStep.cpp | 14 +++--- .../QueryPlan/IntersectOrExceptStep.h | 8 ++-- .../Transforms/IntersectOrExceptTransform.cpp | 48 +++++++++---------- .../Transforms/IntersectOrExceptTransform.h | 13 ++--- 11 files changed, 66 insertions(+), 71 deletions(-) rename src/Parsers/{ParserIntersectOrExcept.cpp => ParserIntersectOrExceptQuery.cpp} (73%) rename src/Parsers/{ParserIntersectOrExcept.h => ParserIntersectOrExceptQuery.h} (80%) diff --git a/src/Interpreters/InterpreterIntersectOrExcept.cpp b/src/Interpreters/InterpreterIntersectOrExcept.cpp index d706ab02bfd..52dcb24ed27 100644 --- a/src/Interpreters/InterpreterIntersectOrExcept.cpp +++ b/src/Interpreters/InterpreterIntersectOrExcept.cpp @@ -23,15 +23,10 @@ InterpreterIntersectOrExcept::InterpreterIntersectOrExcept(const ASTPtr & query_ : context(Context::createCopy(context_)) { ASTIntersectOrExcept * ast = query_ptr->as(); + operators = ast->list_of_operators; + auto children = ast->list_of_selects->children; - modes = ast->list_of_modes; - if (modes.size() + 1 != children.size()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Number of modes and number of children are not consistent"); - size_t num_children = children.size(); - if (!num_children) - throw Exception(ErrorCodes::LOGICAL_ERROR, "No children in ASTIntersectOrExceptQuery"); - nested_interpreters.resize(num_children); for (size_t i = 0; i < num_children; ++i) @@ -106,7 +101,7 @@ void InterpreterIntersectOrExcept::buildQueryPlan(QueryPlan & query_plan) } auto max_threads = context->getSettingsRef().max_threads; - auto step = std::make_unique(std::move(data_streams), modes, max_threads); + auto step = std::make_unique(std::move(data_streams), operators, max_threads); query_plan.unitePlans(std::move(step), std::move(plans)); } diff --git a/src/Interpreters/InterpreterIntersectOrExcept.h b/src/Interpreters/InterpreterIntersectOrExcept.h index 8021a25df06..359be05db8b 100644 --- a/src/Interpreters/InterpreterIntersectOrExcept.h +++ b/src/Interpreters/InterpreterIntersectOrExcept.h @@ -21,7 +21,7 @@ public: BlockIO execute() override; private: - String getName() const { return "IntersectExcept"; } + String getName() const { return "IntersectOrExcept"; } Block getCommonHeader(const Blocks & headers) const; @@ -33,7 +33,7 @@ private: ContextPtr context; Block result_header; std::vector> nested_interpreters; - ASTIntersectOrExcept::Modes modes; + ASTIntersectOrExcept::Operators operators; }; } diff --git a/src/Parsers/ASTIntersectOrExcept.cpp b/src/Parsers/ASTIntersectOrExcept.cpp index 7d92055646c..33ffb76c2f7 100644 --- a/src/Parsers/ASTIntersectOrExcept.cpp +++ b/src/Parsers/ASTIntersectOrExcept.cpp @@ -13,7 +13,7 @@ ASTPtr ASTIntersectOrExcept::clone() const res->list_of_selects = list_of_selects->clone(); res->children.push_back(res->list_of_selects); - res->list_of_modes = list_of_modes; + res->list_of_operators = list_of_operators; cloneOutputOptions(*res); return res; @@ -23,9 +23,9 @@ void ASTIntersectOrExcept::formatQueryImpl(const FormatSettings & settings, Form { std::string indent_str = settings.one_line ? "" : std::string(4 * frame.indent, ' '); - auto mode_to_str = [&](auto mode) + auto operator_to_str = [&](auto current_operator) { - if (mode == Mode::INTERSECT) + if (current_operator == Operator::INTERSECT) return "INTERSECT"; else return "EXCEPT"; @@ -36,7 +36,7 @@ void ASTIntersectOrExcept::formatQueryImpl(const FormatSettings & settings, Form if (it != list_of_selects->children.begin()) { settings.ostr << settings.nl_or_ws << indent_str << (settings.hilite ? hilite_keyword : "") - << mode_to_str(list_of_modes[it - list_of_selects->children.begin() - 1]) + << operator_to_str(list_of_operators[it - list_of_selects->children.begin() - 1]) << (settings.hilite ? hilite_none : ""); } diff --git a/src/Parsers/ASTIntersectOrExcept.h b/src/Parsers/ASTIntersectOrExcept.h index 0eb8ab3b113..9adfdedc497 100644 --- a/src/Parsers/ASTIntersectOrExcept.h +++ b/src/Parsers/ASTIntersectOrExcept.h @@ -15,16 +15,16 @@ public: void formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override; - enum class Mode + enum class Operator { INTERSECT, EXCEPT }; - using Modes = std::vector; + using Operators = std::vector; ASTPtr list_of_selects; - Modes list_of_modes; + Operators list_of_operators; }; } diff --git a/src/Parsers/ParserIntersectOrExcept.cpp b/src/Parsers/ParserIntersectOrExceptQuery.cpp similarity index 73% rename from src/Parsers/ParserIntersectOrExcept.cpp rename to src/Parsers/ParserIntersectOrExceptQuery.cpp index fd1511b8af6..ef6d68f8534 100644 --- a/src/Parsers/ParserIntersectOrExcept.cpp +++ b/src/Parsers/ParserIntersectOrExceptQuery.cpp @@ -2,7 +2,7 @@ #include #include #include -#include +#include #include #include #include @@ -11,13 +11,13 @@ namespace DB { -bool ParserIntersectOrExcept::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) +bool ParserIntersectOrExceptQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { ParserKeyword intersect_keyword("INTERSECT"); ParserKeyword except_keyword("EXCEPT"); ASTs elements; - ASTIntersectOrExcept::Modes modes; + ASTIntersectOrExcept::Operators operators; auto parse_element = [&]() -> bool { @@ -36,18 +36,21 @@ bool ParserIntersectOrExcept::parseImpl(Pos & pos, ASTPtr & node, Expected & exp if (!except_keyword.ignore(pos)) return false; - modes.emplace_back(ASTIntersectOrExcept::Mode::EXCEPT); + operators.emplace_back(ASTIntersectOrExcept::Operator::EXCEPT); return true; } - modes.emplace_back(ASTIntersectOrExcept::Mode::INTERSECT); + operators.emplace_back(ASTIntersectOrExcept::Operator::INTERSECT); return true; }; if (!ParserUnionList::parseUtil(pos, parse_element, parse_separator)) return false; - if (modes.empty()) + if (operators.empty() || elements.empty()) + return false; + + if (operators.size() + 1 != elements.size()) return false; auto list_node = std::make_shared(); @@ -58,7 +61,7 @@ bool ParserIntersectOrExcept::parseImpl(Pos & pos, ASTPtr & node, Expected & exp node = intersect_or_except_ast; intersect_or_except_ast->list_of_selects = list_node; intersect_or_except_ast->children.push_back(intersect_or_except_ast->list_of_selects); - intersect_or_except_ast->list_of_modes = modes; + intersect_or_except_ast->list_of_operators = operators; return true; } diff --git a/src/Parsers/ParserIntersectOrExcept.h b/src/Parsers/ParserIntersectOrExceptQuery.h similarity index 80% rename from src/Parsers/ParserIntersectOrExcept.h rename to src/Parsers/ParserIntersectOrExceptQuery.h index 61cc74cf0a9..d8ba82ba053 100644 --- a/src/Parsers/ParserIntersectOrExcept.h +++ b/src/Parsers/ParserIntersectOrExceptQuery.h @@ -4,7 +4,7 @@ namespace DB { -class ParserIntersectOrExcept : public IParserBase +class ParserIntersectOrExceptQuery : public IParserBase { protected: const char * getName() const override { return "INTERSECT or EXCEPT"; } diff --git a/src/Parsers/ParserQueryWithOutput.cpp b/src/Parsers/ParserQueryWithOutput.cpp index 35355b29ebf..d7d87cac9b9 100644 --- a/src/Parsers/ParserQueryWithOutput.cpp +++ b/src/Parsers/ParserQueryWithOutput.cpp @@ -7,7 +7,7 @@ #include #include #include -#include +#include #include #include #include @@ -31,7 +31,7 @@ namespace DB bool ParserQueryWithOutput::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { ParserShowTablesQuery show_tables_p; - ParserIntersectOrExcept intersect_p; + ParserIntersectOrExceptQuery intersect_except_p; ParserSelectWithUnionQuery select_p; ParserTablePropertiesQuery table_p; ParserDescribeTableQuery describe_table_p; @@ -55,7 +55,7 @@ bool ParserQueryWithOutput::parseImpl(Pos & pos, ASTPtr & node, Expected & expec bool parsed = explain_p.parse(pos, query, expected) - || intersect_p.parse(pos, query, expected) + || intersect_except_p.parse(pos, query, expected) || 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) diff --git a/src/Processors/QueryPlan/IntersectOrExceptStep.cpp b/src/Processors/QueryPlan/IntersectOrExceptStep.cpp index b1b5c1b8813..76f496ba47c 100644 --- a/src/Processors/QueryPlan/IntersectOrExceptStep.cpp +++ b/src/Processors/QueryPlan/IntersectOrExceptStep.cpp @@ -14,17 +14,20 @@ namespace DB Block IntersectOrExceptStep::checkHeaders(const DataStreams & input_streams_) const { if (input_streams_.empty()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot perform {} on empty set of query plan steps", getName()); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot perform intersect/except on empty set of query plan steps"); Block res = input_streams_.front().header; for (const auto & stream : input_streams_) - assertBlocksHaveEqualStructure(stream.header, res, "IntersectOrExceptStep"); + assertBlocksHaveEqualStructure(stream.header, res, "IntersectExceptStep"); return res; } -IntersectOrExceptStep::IntersectOrExceptStep(DataStreams input_streams_, const Modes & modes_, size_t max_threads_) - : header(checkHeaders(input_streams_)), modes(modes_), max_threads(max_threads_) +IntersectOrExceptStep::IntersectOrExceptStep( + DataStreams input_streams_ , const Operators & operators_ , size_t max_threads_) + : header(checkHeaders(input_streams_)) + , operators(operators_) + , max_threads(max_threads_) { input_streams = std::move(input_streams_); if (input_streams.size() == 1) @@ -63,9 +66,8 @@ QueryPipelinePtr IntersectOrExceptStep::updatePipeline(QueryPipelines pipelines, } } - std::cerr << "size: " << input_streams.size() << std::endl; *pipeline = QueryPipeline::unitePipelines(std::move(pipelines), max_threads); - pipeline->addTransform(std::make_shared(header, modes)); + pipeline->addTransform(std::make_shared(header, operators)); processors = collector.detachProcessors(); return pipeline; diff --git a/src/Processors/QueryPlan/IntersectOrExceptStep.h b/src/Processors/QueryPlan/IntersectOrExceptStep.h index 4eceb820153..914a7dce197 100644 --- a/src/Processors/QueryPlan/IntersectOrExceptStep.h +++ b/src/Processors/QueryPlan/IntersectOrExceptStep.h @@ -8,13 +8,13 @@ namespace DB class IntersectOrExceptStep : public IQueryPlanStep { -using Modes = ASTIntersectOrExcept::Modes; +using Operators = ASTIntersectOrExcept::Operators; public: /// max_threads is used to limit the number of threads for result pipeline. - IntersectOrExceptStep(DataStreams input_streams_, const Modes & modes_, size_t max_threads_ = 0); + IntersectOrExceptStep(DataStreams input_streams_, const Operators & operators_, size_t max_threads_ = 0); - String getName() const override { return "IntersectExcept"; } + String getName() const override { return "IntersectOrExcept"; } QueryPipelinePtr updatePipeline(QueryPipelines pipelines, const BuildQueryPipelineSettings & settings) override; @@ -24,7 +24,7 @@ private: Block checkHeaders(const DataStreams & input_streams_) const; Block header; - Modes modes; + Operators operators; size_t max_threads; Processors processors; }; diff --git a/src/Processors/Transforms/IntersectOrExceptTransform.cpp b/src/Processors/Transforms/IntersectOrExceptTransform.cpp index 70a86855992..68d5f6a2e5e 100644 --- a/src/Processors/Transforms/IntersectOrExceptTransform.cpp +++ b/src/Processors/Transforms/IntersectOrExceptTransform.cpp @@ -4,9 +4,13 @@ namespace DB { -IntersectOrExceptTransform::IntersectOrExceptTransform(const Block & header_, const Modes & modes_) - : IProcessor(InputPorts(modes_.size() + 1, header_), {header_}) - , modes(modes_) +/* + * There are always at least two inputs. Number of operators is always number of inputs minus 1. + * input1 {operator1} input2 {operator2} input3 ... +**/ +IntersectOrExceptTransform::IntersectOrExceptTransform(const Block & header_, const Operators & operators_) + : IProcessor(InputPorts(operators_.size() + 1, header_), {header_}) + , operators(operators_) , first_input(inputs.begin()) , second_input(std::next(inputs.begin())) { @@ -30,6 +34,7 @@ IntersectOrExceptTransform::Status IntersectOrExceptTransform::prepare() { for (auto & in : inputs) in.close(); + return Status::Finished; } @@ -37,24 +42,13 @@ IntersectOrExceptTransform::Status IntersectOrExceptTransform::prepare() { for (auto & input : inputs) input.setNotNeeded(); + return Status::PortFull; } - /// Output if has data. - if (current_output_chunk && second_input == inputs.end()) - { - output.push(std::move(current_output_chunk)); - } - - if (push_empty_chunk) - { - output.push(std::move(empty_chunk)); - push_empty_chunk = false; - } - if (finished_second_input) { - if (first_input->isFinished() || (more && !current_input_chunk)) + if (first_input->isFinished() || (use_accumulated_input && !current_input_chunk)) { std::advance(second_input, 1); @@ -64,12 +58,13 @@ IntersectOrExceptTransform::Status IntersectOrExceptTransform::prepare() { output.push(std::move(current_output_chunk)); } + output.finish(); return Status::Finished; } else { - more = true; + use_accumulated_input = true; data.reset(); finished_second_input = false; ++current_operator_pos; @@ -81,20 +76,20 @@ IntersectOrExceptTransform::Status IntersectOrExceptTransform::prepare() finished_second_input = true; } - InputPort & input = finished_second_input ? *first_input : *second_input; - - /// Check can input. if (!has_input) { - if (finished_second_input && more) + if (finished_second_input && use_accumulated_input) { current_input_chunk = std::move(current_output_chunk); } else { + InputPort & input = finished_second_input ? *first_input : *second_input; + input.setNeeded(); if (!input.hasData()) return Status::NeedData; + current_input_chunk = input.pull(); } @@ -107,9 +102,6 @@ IntersectOrExceptTransform::Status IntersectOrExceptTransform::prepare() void IntersectOrExceptTransform::work() { - if (!data) - data.emplace(); - if (!finished_second_input) { accumulate(std::move(current_input_chunk)); @@ -144,7 +136,7 @@ size_t IntersectOrExceptTransform::buildFilter( for (size_t i = 0; i < rows; ++i) { auto find_result = state.findKey(method.data, i, variants.string_pool); - filter[i] = modes[current_operator_pos] == ASTIntersectOrExcept::Mode::EXCEPT ? !find_result.isFound() : find_result.isFound(); + filter[i] = operators[current_operator_pos] == ASTIntersectOrExcept::Operator::EXCEPT ? !find_result.isFound() : find_result.isFound(); if (filter[i]) ++new_rows_num; } @@ -163,6 +155,9 @@ void IntersectOrExceptTransform::accumulate(Chunk chunk) for (auto pos : key_columns_pos) column_ptrs.emplace_back(columns[pos].get()); + if (!data) + data.emplace(); + if (data->empty()) data->init(SetVariants::chooseMethod(column_ptrs, key_sizes)); @@ -192,6 +187,9 @@ void IntersectOrExceptTransform::filter(Chunk & chunk) for (auto pos : key_columns_pos) column_ptrs.emplace_back(columns[pos].get()); + if (!data) + data.emplace(); + if (data->empty()) data->init(SetVariants::chooseMethod(column_ptrs, key_sizes)); diff --git a/src/Processors/Transforms/IntersectOrExceptTransform.h b/src/Processors/Transforms/IntersectOrExceptTransform.h index 5b62ef65d8d..6d0c3516d5d 100644 --- a/src/Processors/Transforms/IntersectOrExceptTransform.h +++ b/src/Processors/Transforms/IntersectOrExceptTransform.h @@ -11,12 +11,12 @@ namespace DB class IntersectOrExceptTransform : public IProcessor { -using Modes = ASTIntersectOrExcept::Modes; +using Operators = ASTIntersectOrExcept::Operators; public: - IntersectOrExceptTransform(const Block & header_, const Modes & modes); + IntersectOrExceptTransform(const Block & header_, const Operators & operators); - String getName() const override { return "IntersectExcept"; } + String getName() const override { return "IntersectOrExcept"; } protected: Status prepare() override; @@ -24,22 +24,19 @@ protected: void work() override; private: - Modes modes; + Operators operators; InputPorts::iterator first_input; InputPorts::iterator second_input; size_t current_operator_pos = 0; - bool push_empty_chunk = false; - Chunk empty_chunk; - ColumnNumbers key_columns_pos; std::optional data; Sizes key_sizes; Chunk current_input_chunk; Chunk current_output_chunk; - bool more = false; + bool use_accumulated_input = false; bool finished_second_input = false; bool has_input = false; From 77adb509cb18b5d036a9e4bdca904bcbf5927332 Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 10 Aug 2021 10:42:56 +0300 Subject: [PATCH 389/599] Fix Nullable const columns in JOIN --- src/Interpreters/join_common.cpp | 3 +++ tests/queries/0_stateless/02007_join_use_nulls.reference | 1 + tests/queries/0_stateless/02007_join_use_nulls.sql | 1 + 3 files changed, 5 insertions(+) create mode 100644 tests/queries/0_stateless/02007_join_use_nulls.reference create mode 100644 tests/queries/0_stateless/02007_join_use_nulls.sql diff --git a/src/Interpreters/join_common.cpp b/src/Interpreters/join_common.cpp index 76bfd7f2899..0ad63bf87b8 100644 --- a/src/Interpreters/join_common.cpp +++ b/src/Interpreters/join_common.cpp @@ -2,6 +2,7 @@ #include #include +#include #include @@ -112,6 +113,8 @@ void convertColumnToNullable(ColumnWithTypeAndName & column) if (!column.column) return; + column.column = column.column->convertToFullColumnIfConst(); + if (column.column->lowCardinality()) { /// Convert nested to nullable, not LowCardinality itself diff --git a/tests/queries/0_stateless/02007_join_use_nulls.reference b/tests/queries/0_stateless/02007_join_use_nulls.reference new file mode 100644 index 00000000000..f66ca605718 --- /dev/null +++ b/tests/queries/0_stateless/02007_join_use_nulls.reference @@ -0,0 +1 @@ +1 2 3 1 3 diff --git a/tests/queries/0_stateless/02007_join_use_nulls.sql b/tests/queries/0_stateless/02007_join_use_nulls.sql new file mode 100644 index 00000000000..f2fbae7ed1b --- /dev/null +++ b/tests/queries/0_stateless/02007_join_use_nulls.sql @@ -0,0 +1 @@ +SELECT *, d.* FROM ( SELECT 1 AS id, 2 AS value ) a SEMI LEFT JOIN ( SELECT 1 AS id, 3 AS values ) AS d USING id SETTINGS join_use_nulls=1; From 7c15d6aa41cfadf3e3e9f85d3a99beaba46377a4 Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 10 Aug 2021 10:50:56 +0300 Subject: [PATCH 390/599] Do not convert const to full in JoinCommon::convertColumnToNullable --- src/Interpreters/join_common.cpp | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/src/Interpreters/join_common.cpp b/src/Interpreters/join_common.cpp index 0ad63bf87b8..97b5dd54da6 100644 --- a/src/Interpreters/join_common.cpp +++ b/src/Interpreters/join_common.cpp @@ -113,8 +113,6 @@ void convertColumnToNullable(ColumnWithTypeAndName & column) if (!column.column) return; - column.column = column.column->convertToFullColumnIfConst(); - if (column.column->lowCardinality()) { /// Convert nested to nullable, not LowCardinality itself @@ -124,7 +122,7 @@ void convertColumnToNullable(ColumnWithTypeAndName & column) col_as_lc->nestedToNullable(); column.column = std::move(mut_col); } - else if (column.column->canBeInsideNullable()) + else if (column.column->canBeInsideNullable() || isColumnConst(*column.column)) { column.column = makeNullable(column.column); } From 84d936355a76a4eca9d8a113064527c776c1d421 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Tue, 10 Aug 2021 11:12:18 +0300 Subject: [PATCH 391/599] Update version_date.tsv after release 21.7.7.47 --- utils/list-versions/version_date.tsv | 1 + 1 file changed, 1 insertion(+) diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index 5a1d2f4c098..151dab728a5 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -1,3 +1,4 @@ +v21.7.7.47-stable 2021-08-09 v21.7.6.39-stable 2021-08-06 v21.7.5.29-stable 2021-07-28 v21.7.4.18-stable 2021-07-17 From 6ebde145eb1833cb9821be015bf0b7c87b53b4ff Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 10 Aug 2021 11:24:22 +0300 Subject: [PATCH 392/599] fix docs --- docs/en/engines/database-engines/materialized-mysql.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/en/engines/database-engines/materialized-mysql.md b/docs/en/engines/database-engines/materialized-mysql.md index 62e58c7876b..ca550776d53 100644 --- a/docs/en/engines/database-engines/materialized-mysql.md +++ b/docs/en/engines/database-engines/materialized-mysql.md @@ -43,10 +43,10 @@ CREATE DATABASE mysql ENGINE = MaterializedMySQL('localhost:3306', 'db', 'user', **Settings on MySQL-server side** -For the correct work of `MaterializeMySQL`, there are few mandatory `MySQL`-side configuration settings that should be set: +For the correct work of `MaterializedMySQL`, there are few mandatory `MySQL`-side configuration settings that should be set: -- `default_authentication_plugin = mysql_native_password` since `MaterializeMySQL` can only authorize with this method. -- `gtid_mode = on` since GTID based logging is a mandatory for providing correct `MaterializeMySQL` replication. Pay attention that while turning this mode `On` you should also specify `enforce_gtid_consistency = on`. +- `default_authentication_plugin = mysql_native_password` since `MaterializedMySQL` can only authorize with this method. +- `gtid_mode = on` since GTID based logging is a mandatory for providing correct `MaterializedMySQL` replication. Pay attention that while turning this mode `On` you should also specify `enforce_gtid_consistency = on`. ## Virtual columns {#virtual-columns} From dfccc85a33f5c607b3d41826a8be4b91d0f79e54 Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 10 Aug 2021 11:37:55 +0300 Subject: [PATCH 393/599] Check const and lc nested can beome nullable in JoinCommon::convertColumnToNullable --- src/Columns/ColumnLowCardinality.h | 1 + src/Interpreters/join_common.cpp | 23 ++++++++++++++++++++--- 2 files changed, 21 insertions(+), 3 deletions(-) diff --git a/src/Columns/ColumnLowCardinality.h b/src/Columns/ColumnLowCardinality.h index 698f65b1281..faf5bb9e712 100644 --- a/src/Columns/ColumnLowCardinality.h +++ b/src/Columns/ColumnLowCardinality.h @@ -187,6 +187,7 @@ public: * So LC(Nullable(T)) would return true, LC(U) -- false. */ bool nestedIsNullable() const { return isColumnNullable(*dictionary.getColumnUnique().getNestedColumn()); } + bool nestedCanBeInsideNullable() const { return dictionary.getColumnUnique().getNestedColumn()->canBeInsideNullable(); } void nestedToNullable() { dictionary.getColumnUnique().nestedToNullable(); } void nestedRemoveNullable() { dictionary.getColumnUnique().nestedRemoveNullable(); } diff --git a/src/Interpreters/join_common.cpp b/src/Interpreters/join_common.cpp index 97b5dd54da6..eb531d0f431 100644 --- a/src/Interpreters/join_common.cpp +++ b/src/Interpreters/join_common.cpp @@ -106,12 +106,29 @@ DataTypePtr convertTypeToNullable(const DataTypePtr & type) return type; } +static bool canBecomeNullable(const ColumnPtr & col) +{ + if (const ColumnConst * col_const = checkAndGetColumn(*col)) + return col_const->getDataColumnPtr()->isNullable() || col_const->getDataColumnPtr()->canBeInsideNullable(); + + if (const ColumnLowCardinality * col_lc = checkAndGetColumn(*col)) + return col_lc->nestedIsNullable() || col_lc->nestedCanBeInsideNullable(); + + return isColumnNullable(*col) || col->canBeInsideNullable(); +} + void convertColumnToNullable(ColumnWithTypeAndName & column) { - column.type = convertTypeToNullable(column.type); - if (!column.column) + { + column.type = convertTypeToNullable(column.type); return; + } + + if (!canBecomeNullable(column.column)) + return; + + column.type = convertTypeToNullable(column.type); if (column.column->lowCardinality()) { @@ -122,7 +139,7 @@ void convertColumnToNullable(ColumnWithTypeAndName & column) col_as_lc->nestedToNullable(); column.column = std::move(mut_col); } - else if (column.column->canBeInsideNullable() || isColumnConst(*column.column)) + else { column.column = makeNullable(column.column); } From d03c867ad9446d49440efb6ea7b3415096b703e5 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 10 Aug 2021 11:44:44 +0300 Subject: [PATCH 394/599] Add Y_IGNORE --- src/IO/Bzip2ReadBuffer.cpp | 2 +- src/IO/Bzip2WriteBuffer.cpp | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/IO/Bzip2ReadBuffer.cpp b/src/IO/Bzip2ReadBuffer.cpp index e264ce75444..99798bca325 100644 --- a/src/IO/Bzip2ReadBuffer.cpp +++ b/src/IO/Bzip2ReadBuffer.cpp @@ -4,7 +4,7 @@ #if USE_BZIP2 # include -# include +# include // Y_IGNORE namespace DB { diff --git a/src/IO/Bzip2WriteBuffer.cpp b/src/IO/Bzip2WriteBuffer.cpp index 41cb972966c..39c5356b792 100644 --- a/src/IO/Bzip2WriteBuffer.cpp +++ b/src/IO/Bzip2WriteBuffer.cpp @@ -4,7 +4,7 @@ #if USE_BROTLI # include -# include +# include // Y_IGNORE #include From 820a6e098773bc5fde2ae2296aa7ce3812ab79a8 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 9 Aug 2021 19:38:29 +0200 Subject: [PATCH 395/599] Refactor MV stage metric setup and teardown --- src/Common/ThreadStatus.cpp | 3 + src/Common/ThreadStatus.h | 7 +- .../PushingToViewsBlockOutputStream.cpp | 267 +++++++----------- .../PushingToViewsBlockOutputStream.h | 8 +- src/Interpreters/ThreadStatusExt.cpp | 9 +- 5 files changed, 122 insertions(+), 172 deletions(-) diff --git a/src/Common/ThreadStatus.cpp b/src/Common/ThreadStatus.cpp index c84195c8834..81c6b8eb1c3 100644 --- a/src/Common/ThreadStatus.cpp +++ b/src/Common/ThreadStatus.cpp @@ -149,6 +149,9 @@ ThreadStatus::~ThreadStatus() if (deleter) deleter(); + + /// Only change current_thread if it's currently being used by this ThreadStatus + /// For example, PushingToViewsBlockOutputStream creates and deletes ThreadStatus instances while running in the main query thread if (current_thread == this) current_thread = nullptr; } diff --git a/src/Common/ThreadStatus.h b/src/Common/ThreadStatus.h index cf3629dce40..dbfb33a320c 100644 --- a/src/Common/ThreadStatus.h +++ b/src/Common/ThreadStatus.h @@ -37,7 +37,7 @@ struct RUsageCounters; struct PerfEventsCounters; class TaskStatsInfoGetter; class InternalTextLogsQueue; -struct ViewInfo; +struct ViewRuntimeData; class QueryViewsLog; using InternalTextLogsQueuePtr = std::shared_ptr; using InternalTextLogsQueueWeakPtr = std::weak_ptr; @@ -216,6 +216,9 @@ public: /// Update ProfileEvents and dumps info to system.query_thread_log void finalizePerformanceCounters(); + /// Set the counters last usage to now + void resetPerformanceCountersLastUsage(); + /// Detaches thread from the thread group and the query, dumps performance counters if they have not been dumped void detachQuery(bool exit_if_already_detached = false, bool thread_exits = false); @@ -231,7 +234,7 @@ protected: void logToQueryThreadLog(QueryThreadLog & thread_log, const String & current_database, std::chrono::time_point now); - void logToQueryViewsLog(const ViewInfo & vinfo); + void logToQueryViewsLog(const ViewRuntimeData & vinfo); void assertState(const std::initializer_list & permitted_states, const char * description = nullptr) const; diff --git a/src/DataStreams/PushingToViewsBlockOutputStream.cpp b/src/DataStreams/PushingToViewsBlockOutputStream.cpp index 27089e98b80..a024b80f391 100644 --- a/src/DataStreams/PushingToViewsBlockOutputStream.cpp +++ b/src/DataStreams/PushingToViewsBlockOutputStream.cpp @@ -167,7 +167,7 @@ PushingToViewsBlockOutputStream::PushingToViewsBlockOutputStream( 0, std::chrono::system_clock::now(), QueryViewsLogElement::ViewStatus::EXCEPTION_BEFORE_START}; - views.emplace_back(ViewInfo{std::move(query), database_table, std::move(out), nullptr, std::move(runtime_stats)}); + views.emplace_back(ViewRuntimeData{std::move(query), database_table, std::move(out), nullptr, std::move(runtime_stats)}); /// Add the view to the query access info so it can appear in system.query_log @@ -200,6 +200,39 @@ Block PushingToViewsBlockOutputStream::getHeader() const return metadata_snapshot->getSampleBlockWithVirtuals(storage->getVirtuals()); } +void inline runViewStage(ViewRuntimeData & view, const std::string & action, std::function stage) +{ + Stopwatch watch; + + auto * original_thread = current_thread; + SCOPE_EXIT({ current_thread = original_thread; }); + + if (view.runtime_stats.thread_status) + { + /// Change thread context to store individual metrics per view. Once the work in done, go back to the original thread + view.runtime_stats.thread_status->resetPerformanceCountersLastUsage(); + current_thread = view.runtime_stats.thread_status.get(); + } + + try + { + stage(); + } + catch (Exception & ex) + { + ex.addMessage(action + " " + view.table_id.getNameForLogs()); + view.setException(std::current_exception()); + } + catch (...) + { + view.setException(std::current_exception()); + } + + if (view.runtime_stats.thread_status) + view.runtime_stats.thread_status->updatePerformanceCounters(); + view.runtime_stats.elapsed_ms += watch.elapsedMilliseconds(); +} + void PushingToViewsBlockOutputStream::write(const Block & block) { /** Throw an exception if the sizes of arrays - elements of nested data structures doesn't match. @@ -237,7 +270,7 @@ void PushingToViewsBlockOutputStream::write(const Block & block) { pool.scheduleOrThrowOnError([&] { setThreadName("PushingToViews"); - process(block, view); + runViewStage(view, "while pushing to view", [&]() { process(block, view); }); }); } pool.wait(); @@ -246,7 +279,7 @@ void PushingToViewsBlockOutputStream::write(const Block & block) { for (auto & view : views) { - process(block, view); + runViewStage(view, "while pushing to view", [&]() { process(block, view); }); } } } @@ -258,7 +291,7 @@ void PushingToViewsBlockOutputStream::writePrefix() for (auto & view : views) { - processPrefix(view); + runViewStage(view, "while writing prefix to view", [&] { view.out->writePrefix(); }); if (view.exception) { logQueryViews(); @@ -275,6 +308,12 @@ void PushingToViewsBlockOutputStream::writeSuffix() if (views.empty()) return; + auto processSuffix = [](ViewRuntimeData & view) { + view.out->writeSuffix(); + view.runtime_stats.setStatus(QueryViewsLogElement::ViewStatus::QUERY_FINISH); + }; + static std::string stageStep = "while writing suffix to view"; + /// Run writeSuffix() for views in separate thread pool. /// In could have been done in PushingToViewsBlockOutputStream::process, however /// it is not good if insert into main table fail but into view succeed. @@ -295,9 +334,20 @@ void PushingToViewsBlockOutputStream::writeSuffix() pool.scheduleOrThrowOnError([&] { setThreadName("PushingToViews"); - processSuffix(view); + runViewStage(view, stageStep, [&] { processSuffix(view); }); if (view.exception) + { exception_count.fetch_add(1, std::memory_order_relaxed); + } + else + { + LOG_TRACE( + log, + "Pushing from {} to {} took {} ms.", + storage->getStorageID().getNameForLogs(), + view.table_id.getNameForLogs(), + view.runtime_stats.elapsed_ms); + } }); } pool.wait(); @@ -312,7 +362,7 @@ void PushingToViewsBlockOutputStream::writeSuffix() exception_happened = true; continue; } - processSuffix(view); + runViewStage(view, stageStep, [&] { processSuffix(view); }); if (view.exception) exception_happened = true; } @@ -337,170 +387,59 @@ void PushingToViewsBlockOutputStream::flush() view.out->flush(); } -void PushingToViewsBlockOutputStream::process(const Block & block, ViewInfo & view) +void PushingToViewsBlockOutputStream::process(const Block & block, ViewRuntimeData & view) { - Stopwatch watch; + BlockInputStreamPtr in; - auto * original_thread = current_thread; - SCOPE_EXIT({ current_thread = original_thread; }); + /// We need keep InterpreterSelectQuery, until the processing will be finished, since: + /// + /// - We copy Context inside InterpreterSelectQuery to support + /// modification of context (Settings) for subqueries + /// - InterpreterSelectQuery lives shorter than query pipeline. + /// It's used just to build the query pipeline and no longer needed + /// - ExpressionAnalyzer and then, Functions, that created in InterpreterSelectQuery, + /// **can** take a reference to Context from InterpreterSelectQuery + /// (the problem raises only when function uses context from the + /// execute*() method, like FunctionDictGet do) + /// - These objects live inside query pipeline (DataStreams) and the reference become dangling. + std::optional select; - if (view.runtime_stats.thread_status) + if (view.runtime_stats.type == QueryViewsLogElement::ViewType::MATERIALIZED) { - /// Change thread context to store individual metrics per view. Once the work in done, go back to the original thread - *view.runtime_stats.thread_status->last_rusage = RUsageCounters::current(); - if (view.runtime_stats.thread_status->taskstats) - view.runtime_stats.thread_status->taskstats->reset(); - current_thread = view.runtime_stats.thread_status.get(); + /// We create a table with the same name as original table and the same alias columns, + /// but it will contain single block (that is INSERT-ed into main table). + /// InterpreterSelectQuery will do processing of alias columns. + + auto local_context = Context::createCopy(select_context); + local_context->addViewSource( + StorageValues::create(storage->getStorageID(), metadata_snapshot->getColumns(), block, storage->getVirtuals())); + select.emplace(view.query, local_context, SelectQueryOptions()); + in = std::make_shared(select->execute().getInputStream()); + + /// Squashing is needed here because the materialized view query can generate a lot of blocks + /// even when only one block is inserted into the parent table (e.g. if the query is a GROUP BY + /// and two-level aggregation is triggered). + in = std::make_shared( + in, getContext()->getSettingsRef().min_insert_block_size_rows, getContext()->getSettingsRef().min_insert_block_size_bytes); + in = std::make_shared(in, view.out->getHeader(), ConvertingBlockInputStream::MatchColumnsMode::Name); + } + else + in = std::make_shared(block); + + in->setProgressCallback([this](const Progress & progress) { + CurrentThread::updateProgressIn(progress); + this->onProgress(progress); + }); + + in->readPrefix(); + + while (Block result_block = in->read()) + { + Nested::validateArraySizes(result_block); + view.out->write(result_block); } - - try - { - BlockInputStreamPtr in; - - /// We need keep InterpreterSelectQuery, until the processing will be finished, since: - /// - /// - We copy Context inside InterpreterSelectQuery to support - /// modification of context (Settings) for subqueries - /// - InterpreterSelectQuery lives shorter than query pipeline. - /// It's used just to build the query pipeline and no longer needed - /// - ExpressionAnalyzer and then, Functions, that created in InterpreterSelectQuery, - /// **can** take a reference to Context from InterpreterSelectQuery - /// (the problem raises only when function uses context from the - /// execute*() method, like FunctionDictGet do) - /// - These objects live inside query pipeline (DataStreams) and the reference become dangling. - std::optional select; - - if (view.runtime_stats.type == QueryViewsLogElement::ViewType::MATERIALIZED) - { - /// We create a table with the same name as original table and the same alias columns, - /// but it will contain single block (that is INSERT-ed into main table). - /// InterpreterSelectQuery will do processing of alias columns. - - auto local_context = Context::createCopy(select_context); - local_context->addViewSource( - StorageValues::create(storage->getStorageID(), metadata_snapshot->getColumns(), block, storage->getVirtuals())); - select.emplace(view.query, local_context, SelectQueryOptions()); - in = std::make_shared(select->execute().getInputStream()); - - /// Squashing is needed here because the materialized view query can generate a lot of blocks - /// even when only one block is inserted into the parent table (e.g. if the query is a GROUP BY - /// and two-level aggregation is triggered). - in = std::make_shared( - in, getContext()->getSettingsRef().min_insert_block_size_rows, getContext()->getSettingsRef().min_insert_block_size_bytes); - in = std::make_shared(in, view.out->getHeader(), ConvertingBlockInputStream::MatchColumnsMode::Name); - } - else - in = std::make_shared(block); - - in->setProgressCallback([this](const Progress & progress) - { - CurrentThread::updateProgressIn(progress); - this->onProgress(progress); - }); - - in->readPrefix(); - - while (Block result_block = in->read()) - { - Nested::validateArraySizes(result_block); - view.out->write(result_block); - } - - in->readSuffix(); - } - catch (Exception & ex) - { - ex.addMessage("while pushing to view " + view.table_id.getNameForLogs()); - view.setException(std::current_exception()); - } - catch (...) - { - view.setException(std::current_exception()); - } - - if (view.runtime_stats.thread_status) - view.runtime_stats.thread_status->updatePerformanceCounters(); - view.runtime_stats.elapsed_ms += watch.elapsedMilliseconds(); -} - -void PushingToViewsBlockOutputStream::processPrefix(ViewInfo & view) -{ - Stopwatch watch; - - auto * original_thread = current_thread; - SCOPE_EXIT({ current_thread = original_thread; }); - - if (view.runtime_stats.thread_status) - { - /// Change thread context to store individual metrics per view. Once the work in done, go back to the original thread - *view.runtime_stats.thread_status->last_rusage = RUsageCounters::current(); - if (view.runtime_stats.thread_status->taskstats) - view.runtime_stats.thread_status->taskstats->reset(); - current_thread = view.runtime_stats.thread_status.get(); - } - - try - { - view.out->writePrefix(); - } - catch (Exception & ex) - { - ex.addMessage("while writing prefix to view " + view.table_id.getNameForLogs()); - view.setException(std::current_exception()); - } - catch (...) - { - view.setException(std::current_exception()); - } - if (view.runtime_stats.thread_status) - view.runtime_stats.thread_status->updatePerformanceCounters(); - view.runtime_stats.elapsed_ms += watch.elapsedMilliseconds(); -} - - -void PushingToViewsBlockOutputStream::processSuffix(ViewInfo & view) -{ - Stopwatch watch; - - auto * original_thread = current_thread; - SCOPE_EXIT({ current_thread = original_thread; }); - - if (view.runtime_stats.thread_status) - { - /// Change thread context to store individual metrics per view. Once the work in done, go back to the original thread - *view.runtime_stats.thread_status->last_rusage = RUsageCounters::current(); - if (view.runtime_stats.thread_status->taskstats) - view.runtime_stats.thread_status->taskstats->reset(); - current_thread = view.runtime_stats.thread_status.get(); - } - - try - { - view.out->writeSuffix(); - view.runtime_stats.setStatus(QueryViewsLogElement::ViewStatus::QUERY_FINISH); - } - catch (Exception & ex) - { - ex.addMessage("while writing suffix to view " + view.table_id.getNameForLogs()); - view.setException(std::current_exception()); - } - catch (...) - { - view.setException(std::current_exception()); - } - if (view.runtime_stats.thread_status) - view.runtime_stats.thread_status->updatePerformanceCounters(); - view.runtime_stats.elapsed_ms += watch.elapsedMilliseconds(); - if (!view.exception) - { - LOG_TRACE( - log, - "Pushing from {} to {} took {} ms.", - storage->getStorageID().getNameForLogs(), - view.table_id.getNameForLogs(), - view.runtime_stats.elapsed_ms); - } + in->readSuffix(); } void PushingToViewsBlockOutputStream::checkExceptionsInViews() diff --git a/src/DataStreams/PushingToViewsBlockOutputStream.h b/src/DataStreams/PushingToViewsBlockOutputStream.h index 591cf9e771c..ba125e28829 100644 --- a/src/DataStreams/PushingToViewsBlockOutputStream.h +++ b/src/DataStreams/PushingToViewsBlockOutputStream.h @@ -16,7 +16,7 @@ namespace DB class ReplicatedMergeTreeSink; -struct ViewInfo +struct ViewRuntimeData { const ASTPtr query; StorageID table_id; @@ -61,13 +61,11 @@ private: ASTPtr query_ptr; Stopwatch main_watch; - std::vector views; + std::vector views; ContextMutablePtr select_context; ContextMutablePtr insert_context; - void process(const Block & block, ViewInfo & view); - static void processPrefix(ViewInfo & view); - void processSuffix(ViewInfo & view); + void process(const Block & block, ViewRuntimeData & view); void checkExceptionsInViews(); void logQueryViews(); }; diff --git a/src/Interpreters/ThreadStatusExt.cpp b/src/Interpreters/ThreadStatusExt.cpp index d11d3b22184..2917a399906 100644 --- a/src/Interpreters/ThreadStatusExt.cpp +++ b/src/Interpreters/ThreadStatusExt.cpp @@ -300,6 +300,13 @@ void ThreadStatus::finalizePerformanceCounters() } } +void ThreadStatus::resetPerformanceCountersLastUsage() +{ + *last_rusage = RUsageCounters::current(); + if (taskstats) + taskstats->reset(); +} + void ThreadStatus::initQueryProfiler() { if (!query_profiled_enabled) @@ -482,7 +489,7 @@ static String getCleanQueryAst(const ASTPtr q, ContextPtr context) return res; } -void ThreadStatus::logToQueryViewsLog(const ViewInfo & vinfo) +void ThreadStatus::logToQueryViewsLog(const ViewRuntimeData & vinfo) { auto query_context_ptr = query_context.lock(); if (!query_context_ptr) From 3545f2a4ee62385244d41edc38c054b10beab049 Mon Sep 17 00:00:00 2001 From: Filatenkov Artur <58165623+FArthur-cmd@users.noreply.github.com> Date: Tue, 10 Aug 2021 11:59:11 +0300 Subject: [PATCH 396/599] Update Client.cpp --- programs/client/Client.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index 2e12c262605..81a50d8336c 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -316,7 +316,7 @@ private: } catch (const Exception & e) { - bool print_stack_trace = config().getBool("stacktrace", false) && e.code() != ErrorCodes::NETWORK_ERROR; + bool print_stack_trace = config().getBool("stacktrace", false) && e.code() != ErrorCodes::NETWORK_ERROR; std::cerr << getExceptionMessage(e, print_stack_trace, true) << std::endl << std::endl; From 0db028bb497be8d4b0ee765dd20f4f98fb9b1827 Mon Sep 17 00:00:00 2001 From: Filatenkov Artur <58165623+FArthur-cmd@users.noreply.github.com> Date: Tue, 10 Aug 2021 12:00:55 +0300 Subject: [PATCH 397/599] Update Client.cpp --- programs/client/Client.cpp | 15 ++++++--------- 1 file changed, 6 insertions(+), 9 deletions(-) diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index 81a50d8336c..eb7c3bd0a74 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -537,17 +537,14 @@ private: MainThreadStatus::getInstance(); - if (config().has("max_memory_usage_in_client")) - { /// Limit on total memory usage - size_t max_client_memory_usage = config().getInt64("max_memory_usage_in_client"); + size_t max_client_memory_usage = config().getInt64("max_memory_usage_in_client", 0 /*default value*/); - if (max_client_memory_usage != 0) - { - total_memory_tracker.setHardLimit(max_client_memory_usage); - total_memory_tracker.setDescription("(total)"); - total_memory_tracker.setMetric(CurrentMetrics::MemoryTracking); - } + if (max_client_memory_usage != 0) + { + total_memory_tracker.setHardLimit(max_client_memory_usage); + total_memory_tracker.setDescription("(total)"); + total_memory_tracker.setMetric(CurrentMetrics::MemoryTracking); } registerFormats(); From 83287827f23e80a617d5390ac3de7c7a505b97b8 Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Tue, 10 Aug 2021 12:17:08 +0300 Subject: [PATCH 398/599] Update success stories on the website front page --- website/templates/index/success.html | 46 ++++++++++++++-------------- 1 file changed, 23 insertions(+), 23 deletions(-) diff --git a/website/templates/index/success.html b/website/templates/index/success.html index 83b5c1427c9..a93efa8bdc5 100644 --- a/website/templates/index/success.html +++ b/website/templates/index/success.html @@ -2,19 +2,9 @@

Success stories

-
-
+
+
- +
-

Speeding up Report API

-

at Geniee

+

Business Intelligence

+

at Deutsche Bank

From d8375a88ec0dd38e5758fd1c79c934d3ec80d355 Mon Sep 17 00:00:00 2001 From: tavplubix Date: Tue, 10 Aug 2021 12:42:01 +0300 Subject: [PATCH 399/599] Update InterpreterCreateQuery.cpp --- src/Interpreters/InterpreterCreateQuery.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index bf2cf6338aa..4c1a3064c3d 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -764,7 +764,7 @@ void InterpreterCreateQuery::assertOrSetUUID(ASTCreateQuery & create, const Data const auto * kind = create.is_dictionary ? "Dictionary" : "Table"; const auto * kind_upper = create.is_dictionary ? "DICTIONARY" : "TABLE"; - if (database->getEngineName() == "Replicated" && getContext()->getClientInfo().query_kind == ClientInfo::QueryKind::SECONDARY_QUERY + if (database->getEngineName() == "Replicated" && getContext()->getClientInfo().is_replicated_database_internal && !internal) { if (create.uuid == UUIDHelpers::Nil) From 315b611963eea220e78f2d821ddf6dee04d76830 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 9 Aug 2021 19:43:17 +0200 Subject: [PATCH 400/599] MV: Use as many threads as views if max_threads = 0 --- .../PushingToViewsBlockOutputStream.cpp | 23 ++++++++++++++++--- 1 file changed, 20 insertions(+), 3 deletions(-) diff --git a/src/DataStreams/PushingToViewsBlockOutputStream.cpp b/src/DataStreams/PushingToViewsBlockOutputStream.cpp index a024b80f391..2bfdba092e3 100644 --- a/src/DataStreams/PushingToViewsBlockOutputStream.cpp +++ b/src/DataStreams/PushingToViewsBlockOutputStream.cpp @@ -200,6 +200,7 @@ Block PushingToViewsBlockOutputStream::getHeader() const return metadata_snapshot->getSampleBlockWithVirtuals(storage->getVirtuals()); } +/// Auxiliary function to do the setup and teardown to run a view individually and collect its metrics inside the view ThreadStatus void inline runViewStage(ViewRuntimeData & view, const std::string & action, std::function stage) { Stopwatch watch; @@ -262,7 +263,9 @@ void PushingToViewsBlockOutputStream::write(const Block & block) if (!settings.deduplicate_blocks_in_dependent_materialized_views && replicated_output && replicated_output->lastBlockIsDuplicate()) return; - const size_t max_threads = std::min(views.size(), (settings.parallel_view_processing ? static_cast(settings.max_threads) : 1)); + size_t max_threads = 1; + if (settings.parallel_view_processing) + max_threads = settings.max_threads ? std::min(static_cast(settings.max_threads), views.size()) : views.size(); if (max_threads > 1) { ThreadPool pool(max_threads); @@ -318,7 +321,9 @@ void PushingToViewsBlockOutputStream::writeSuffix() /// In could have been done in PushingToViewsBlockOutputStream::process, however /// it is not good if insert into main table fail but into view succeed. const Settings & settings = getContext()->getSettingsRef(); - const size_t max_threads = std::min(views.size(), (settings.parallel_view_processing ? static_cast(settings.max_threads) : 1)); + size_t max_threads = 1; + if (settings.parallel_view_processing) + max_threads = settings.max_threads ? std::min(static_cast(settings.max_threads), views.size()) : views.size(); bool exception_happened = false; if (max_threads > 1) { @@ -343,7 +348,8 @@ void PushingToViewsBlockOutputStream::writeSuffix() { LOG_TRACE( log, - "Pushing from {} to {} took {} ms.", + "Pushing (parallel {}) from {} to {} took {} ms.", + max_threads, storage->getStorageID().getNameForLogs(), view.table_id.getNameForLogs(), view.runtime_stats.elapsed_ms); @@ -364,7 +370,18 @@ void PushingToViewsBlockOutputStream::writeSuffix() } runViewStage(view, stageStep, [&] { processSuffix(view); }); if (view.exception) + { exception_happened = true; + } + else + { + LOG_TRACE( + log, + "Pushing (sequentially) from {} to {} took {} ms.", + storage->getStorageID().getNameForLogs(), + view.table_id.getNameForLogs(), + view.runtime_stats.elapsed_ms); + } } } if (exception_happened) From 770aa427f1dc2639679a2c65fefc2c01a21ac8a0 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 10 Aug 2021 11:31:39 +0200 Subject: [PATCH 401/599] MV: Test profile events and view type --- ...query_views_log_current_database.reference | 76 +++++++++++++++++-- ...01927_query_views_log_current_database.sql | 32 +++++--- 2 files changed, 91 insertions(+), 17 deletions(-) diff --git a/tests/queries/0_stateless/01927_query_views_log_current_database.reference b/tests/queries/0_stateless/01927_query_views_log_current_database.reference index ac9247ab9d5..ff9eca2d97f 100644 --- a/tests/queries/0_stateless/01927_query_views_log_current_database.reference +++ b/tests/queries/0_stateless/01927_query_views_log_current_database.reference @@ -1,6 +1,70 @@ -{"stage":"Query log rows","read_rows":"100","written_rows":"201","databases":["_table_function","default"],"tables":["_table_function.numbers","default.table_a","default.table_b","default.table_b_live_view","default.table_c"],"views":["default.matview_a_to_b","default.matview_b_to_c","default.table_b_live_view"]} -{"stage":"Depending views","view_name":"default.matview_a_to_b","status":"QueryFinish","view_target":"default.table_b","view_query":"SELECT toFloat64(a) AS a, b AS count FROM default.table_a","read_rows":"100","written_rows":"100"} -{"stage":"Depending views","view_name":"default.matview_b_to_c","status":"QueryFinish","view_target":"default.table_c","view_query":"SELECT sum(a) AS a FROM default.table_b","read_rows":"100","written_rows":"1"} -{"stage":"Depending views","view_name":"default.table_b_live_view","status":"QueryFinish","view_target":"default.table_b_live_view","view_query":"SELECT sum(a + b) FROM default.table_b","read_rows":"100","written_rows":"0"} -{"stage":"Query log rows 2","read_rows":"50","written_rows":"100","databases":["_table_function","default"],"tables":["_table_function.numbers","default.table_d","default.table_e","default.table_f"],"views":["default.matview_join_d_e"]} -{"stage":"Depending views 2","view_name":"default.matview_join_d_e","status":"QueryFinish","view_target":"default.table_f","view_query":"SELECT table_d.a AS a, table_e.count AS count FROM default.table_d LEFT JOIN default.table_e ON table_d.a = table_e.a","read_rows":"50","written_rows":"50"} +Row 1: +────── +stage: Query log rows +read_rows: 100 +written_rows: 201 +databases: ['_table_function','default'] +tables: ['_table_function.numbers','default.table_a','default.table_b','default.table_b_live_view','default.table_c'] +views: ['default.matview_a_to_b','default.matview_b_to_c','default.table_b_live_view'] +sleep_calls: 200 +sleep_us: 298 +Row 1: +────── +stage: Depending views +view_name: default.matview_a_to_b +view_type: Materialized +status: QueryFinish +view_target: default.table_b +view_query: SELECT toFloat64(a) AS a, b + sleepEachRow(0.000001) AS count FROM default.table_a +read_rows: 100 +written_rows: 100 +sleep_calls: 100 +sleep_us: 99 + +Row 2: +────── +stage: Depending views +view_name: default.matview_b_to_c +view_type: Materialized +status: QueryFinish +view_target: default.table_c +view_query: SELECT sum(a + sleepEachRow(0.000002)) AS a FROM default.table_b +read_rows: 100 +written_rows: 1 +sleep_calls: 100 +sleep_us: 199 + +Row 3: +────── +stage: Depending views +view_name: default.table_b_live_view +view_type: Live +status: QueryFinish +view_target: default.table_b_live_view +view_query: SELECT sum(a + b) FROM default.table_b +read_rows: 100 +written_rows: 0 +sleep_calls: 0 +sleep_us: 0 +Row 1: +────── +stage: Query log rows 2 +read_rows: 50 +written_rows: 100 +databases: ['_table_function','default'] +tables: ['_table_function.numbers','default.table_d','default.table_e','default.table_f'] +views: ['default.matview_join_d_e'] +sleep_calls: 50 +sleep_us: 150 +Row 1: +────── +stage: Depending views 2 +view_name: default.matview_join_d_e +view_type: Materialized +status: QueryFinish +view_target: default.table_f +view_query: SELECT table_d.a AS a, table_e.count + sleepEachRow(0.000003) AS count FROM default.table_d LEFT JOIN default.table_e ON table_d.a = table_e.a +read_rows: 50 +written_rows: 50 +sleep_calls: 50 +sleep_us: 150 diff --git a/tests/queries/0_stateless/01927_query_views_log_current_database.sql b/tests/queries/0_stateless/01927_query_views_log_current_database.sql index b674fbc0882..40ab8c8e16a 100644 --- a/tests/queries/0_stateless/01927_query_views_log_current_database.sql +++ b/tests/queries/0_stateless/01927_query_views_log_current_database.sql @@ -12,9 +12,9 @@ CREATE TABLE table_e (a Float64, count Int64) ENGINE MergeTree ORDER BY a; CREATE TABLE table_f (a Float64, count Int64) ENGINE MergeTree ORDER BY a; -- SETUP MATERIALIZED VIEWS -CREATE MATERIALIZED VIEW matview_a_to_b TO table_b AS SELECT toFloat64(a) AS a, b AS count FROM table_a; -CREATE MATERIALIZED VIEW matview_b_to_c TO table_c AS SELECT SUM(a) as a FROM table_b; -CREATE MATERIALIZED VIEW matview_join_d_e TO table_f AS SELECT table_d.a as a, table_e.count as count FROM table_d LEFT JOIN table_e ON table_d.a = table_e.a; +CREATE MATERIALIZED VIEW matview_a_to_b TO table_b AS SELECT toFloat64(a) AS a, b + sleepEachRow(0.000001) AS count FROM table_a; +CREATE MATERIALIZED VIEW matview_b_to_c TO table_c AS SELECT SUM(a + sleepEachRow(0.000002)) as a FROM table_b; +CREATE MATERIALIZED VIEW matview_join_d_e TO table_f AS SELECT table_d.a as a, table_e.count + sleepEachRow(0.000003) as count FROM table_d LEFT JOIN table_e ON table_d.a = table_e.a; -- SETUP LIVE VIEW ---- table_b_live_view (Int64) @@ -43,21 +43,26 @@ SELECT written_rows, arraySort(databases) as databases, arraySort(tables) as tables, - arraySort(views) as views + arraySort(views) as views, + ProfileEvents['SleepFunctionCalls'] as sleep_calls, + ProfileEvents['SleepFunctionMicroseconds'] as sleep_us FROM system.query_log WHERE query like '-- INSERT 1%INSERT INTO table_a%' AND current_database = currentDatabase() AND event_date >= yesterday() -FORMAT JSONEachRow; +FORMAT Vertical; SELECT 'Depending views' as stage, view_name, + view_type, status, view_target, view_query, read_rows, - written_rows + written_rows, + ProfileEvents['SleepFunctionCalls'] as sleep_calls, + ProfileEvents['SleepFunctionMicroseconds'] as sleep_us FROM system.query_views_log WHERE initial_query_id = ( @@ -69,7 +74,7 @@ WHERE initial_query_id = LIMIT 1 ) ORDER BY view_name -FORMAT JSONEachRow; +FORMAT Vertical; -- CHECK LOGS OF INSERT 2 SELECT @@ -78,21 +83,26 @@ SELECT written_rows, arraySort(databases) as databases, arraySort(tables) as tables, - arraySort(views) as views + arraySort(views) as views, + ProfileEvents['SleepFunctionCalls'] as sleep_calls, + ProfileEvents['SleepFunctionMicroseconds'] as sleep_us FROM system.query_log WHERE query like '-- INSERT 2%INSERT INTO table_d%' AND current_database = currentDatabase() AND event_date >= yesterday() -FORMAT JSONEachRow; +FORMAT Vertical; SELECT 'Depending views 2' as stage, view_name, + view_type, status, view_target, view_query, read_rows, - written_rows + written_rows, + ProfileEvents['SleepFunctionCalls'] as sleep_calls, + ProfileEvents['SleepFunctionMicroseconds'] as sleep_us FROM system.query_views_log WHERE initial_query_id = ( @@ -104,7 +114,7 @@ WHERE initial_query_id = LIMIT 1 ) ORDER BY view_name - FORMAT JSONEachRow; +FORMAT Vertical; -- TEARDOWN DROP TABLE table_b_live_view; From ae3726643d3a0c32a4300d7ede5c017c0b146c24 Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 10 Aug 2021 12:44:20 +0300 Subject: [PATCH 402/599] Support ColumnConst(ColumnLowCardinality) in JoinCommon::convertColumnToNullable --- src/Interpreters/join_common.cpp | 61 ++++++++++++------- .../02007_join_use_nulls.reference | 7 +++ .../0_stateless/02007_join_use_nulls.sql | 12 +++- 3 files changed, 56 insertions(+), 24 deletions(-) diff --git a/src/Interpreters/join_common.cpp b/src/Interpreters/join_common.cpp index eb531d0f431..e9f3e4f3fdd 100644 --- a/src/Interpreters/join_common.cpp +++ b/src/Interpreters/join_common.cpp @@ -106,15 +106,42 @@ DataTypePtr convertTypeToNullable(const DataTypePtr & type) return type; } -static bool canBecomeNullable(const ColumnPtr & col) +/// Convert column to nullable. If column LowCardinality or Const, convert nested column. +/// Returns nullptr if conversion cannot be performed. +static ColumnPtr tryConvertColumnToNullable(const ColumnPtr & col) { - if (const ColumnConst * col_const = checkAndGetColumn(*col)) - return col_const->getDataColumnPtr()->isNullable() || col_const->getDataColumnPtr()->canBeInsideNullable(); + if (isColumnNullable(*col) || col->canBeInsideNullable()) + return makeNullable(col); - if (const ColumnLowCardinality * col_lc = checkAndGetColumn(*col)) - return col_lc->nestedIsNullable() || col_lc->nestedCanBeInsideNullable(); - - return isColumnNullable(*col) || col->canBeInsideNullable(); + if (col->lowCardinality()) + { + auto mut_col = IColumn::mutate(std::move(col)); + ColumnLowCardinality * col_lc = assert_cast(mut_col.get()); + if (col_lc->nestedIsNullable()) + { + return mut_col; + } + else if (col_lc->nestedCanBeInsideNullable()) + { + col_lc->nestedToNullable(); + return mut_col; + } + } + else if (const ColumnConst * col_const = checkAndGetColumn(*col)) + { + const auto & nested = col_const->getDataColumnPtr(); + if (nested->isNullable() || nested->canBeInsideNullable()) + { + return makeNullable(col); + } + else if (nested->lowCardinality()) + { + ColumnPtr nested_nullable = tryConvertColumnToNullable(nested); + if (nested_nullable) + return ColumnConst::create(nested_nullable, col_const->size()); + } + } + return nullptr; } void convertColumnToNullable(ColumnWithTypeAndName & column) @@ -125,23 +152,11 @@ void convertColumnToNullable(ColumnWithTypeAndName & column) return; } - if (!canBecomeNullable(column.column)) - return; - - column.type = convertTypeToNullable(column.type); - - if (column.column->lowCardinality()) + ColumnPtr nullable_column = tryConvertColumnToNullable(column.column); + if (nullable_column) { - /// Convert nested to nullable, not LowCardinality itself - auto mut_col = IColumn::mutate(std::move(column.column)); - ColumnLowCardinality * col_as_lc = assert_cast(mut_col.get()); - if (!col_as_lc->nestedIsNullable()) - col_as_lc->nestedToNullable(); - column.column = std::move(mut_col); - } - else - { - column.column = makeNullable(column.column); + column.type = convertTypeToNullable(column.type); + column.column = std::move(nullable_column); } } diff --git a/tests/queries/0_stateless/02007_join_use_nulls.reference b/tests/queries/0_stateless/02007_join_use_nulls.reference index f66ca605718..30ee87bf91d 100644 --- a/tests/queries/0_stateless/02007_join_use_nulls.reference +++ b/tests/queries/0_stateless/02007_join_use_nulls.reference @@ -1 +1,8 @@ 1 2 3 1 3 +1 UInt8 2 UInt8 3 Nullable(UInt8) +1 LowCardinality(UInt8) 2 LowCardinality(UInt8) 3 LowCardinality(Nullable(UInt8)) +1 LowCardinality(UInt8) 2 LowCardinality(UInt8) 1 LowCardinality(Nullable(UInt8)) +1 UInt8 2 UInt8 3 Nullable(UInt8) +1 UInt8 2 UInt8 1 Nullable(UInt8) 3 Nullable(UInt8) +1 LowCardinality(UInt8) 2 LowCardinality(UInt8) 3 LowCardinality(Nullable(UInt8)) +1 LowCardinality(UInt8) 2 LowCardinality(UInt8) 1 LowCardinality(Nullable(UInt8)) 3 LowCardinality(Nullable(UInt8)) diff --git a/tests/queries/0_stateless/02007_join_use_nulls.sql b/tests/queries/0_stateless/02007_join_use_nulls.sql index f2fbae7ed1b..e08fffce3b7 100644 --- a/tests/queries/0_stateless/02007_join_use_nulls.sql +++ b/tests/queries/0_stateless/02007_join_use_nulls.sql @@ -1 +1,11 @@ -SELECT *, d.* FROM ( SELECT 1 AS id, 2 AS value ) a SEMI LEFT JOIN ( SELECT 1 AS id, 3 AS values ) AS d USING id SETTINGS join_use_nulls=1; +SET join_use_nulls = 1; + +SELECT *, d.* FROM ( SELECT 1 AS id, 2 AS value ) a SEMI LEFT JOIN ( SELECT 1 AS id, 3 AS values ) AS d USING id; + +SELECT id, toTypeName(id), value, toTypeName(value), d.values, toTypeName(d.values) FROM ( SELECT 1 AS id, 2 AS value ) a SEMI LEFT JOIN ( SELECT 1 AS id, 3 AS values ) AS d USING id; +SELECT id, toTypeName(id), value, toTypeName(value), d.values, toTypeName(d.values) FROM ( SELECT toLowCardinality(1) AS id, toLowCardinality(2) AS value ) a SEMI LEFT JOIN ( SELECT toLowCardinality(1) AS id, toLowCardinality(3) AS values ) AS d USING id; +SELECT id, toTypeName(id), value, toTypeName(value), d.id, toTypeName(d.id) FROM ( SELECT toLowCardinality(1) AS id, toLowCardinality(2) AS value ) a SEMI LEFT JOIN ( SELECT toLowCardinality(1) AS id, toLowCardinality(3) AS values ) AS d USING id; +SELECT id, toTypeName(id), value, toTypeName(value), d.values, toTypeName(d.values) FROM ( SELECT 1 AS id, 2 AS value ) a SEMI LEFT JOIN ( SELECT 1 AS id, 3 AS values ) AS d USING id; +SELECT id, toTypeName(id), value, toTypeName(value), d.id, toTypeName(d.id) , d.values, toTypeName(d.values) FROM ( SELECT 1 AS id, 2 AS value ) a SEMI LEFT JOIN ( SELECT 1 AS id, 3 AS values ) AS d USING id; +SELECT id, toTypeName(id), value, toTypeName(value), d.values, toTypeName(d.values) FROM ( SELECT toLowCardinality(1) AS id, toLowCardinality(2) AS value ) a SEMI LEFT JOIN ( SELECT toLowCardinality(1) AS id, toLowCardinality(3) AS values ) AS d USING id; +SELECT id, toTypeName(id), value, toTypeName(value), d.id, toTypeName(d.id) , d.values, toTypeName(d.values) FROM ( SELECT toLowCardinality(1) AS id, toLowCardinality(2) AS value ) a SEMI LEFT JOIN ( SELECT toLowCardinality(1) AS id, toLowCardinality(3) AS values ) AS d USING id; From b28a186727111dfbac3cd389c3a8109ddb331cb9 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 10 Aug 2021 12:08:46 +0200 Subject: [PATCH 403/599] Style --- src/DataStreams/PushingToViewsBlockOutputStream.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/DataStreams/PushingToViewsBlockOutputStream.cpp b/src/DataStreams/PushingToViewsBlockOutputStream.cpp index 2bfdba092e3..c24df2d94c0 100644 --- a/src/DataStreams/PushingToViewsBlockOutputStream.cpp +++ b/src/DataStreams/PushingToViewsBlockOutputStream.cpp @@ -311,11 +311,11 @@ void PushingToViewsBlockOutputStream::writeSuffix() if (views.empty()) return; - auto processSuffix = [](ViewRuntimeData & view) { + auto process_suffix = [](ViewRuntimeData & view) { view.out->writeSuffix(); view.runtime_stats.setStatus(QueryViewsLogElement::ViewStatus::QUERY_FINISH); }; - static std::string stageStep = "while writing suffix to view"; + static std::string stage_step = "while writing suffix to view"; /// Run writeSuffix() for views in separate thread pool. /// In could have been done in PushingToViewsBlockOutputStream::process, however @@ -339,7 +339,7 @@ void PushingToViewsBlockOutputStream::writeSuffix() pool.scheduleOrThrowOnError([&] { setThreadName("PushingToViews"); - runViewStage(view, stageStep, [&] { processSuffix(view); }); + runViewStage(view, stage_step, [&] { process_suffix(view); }); if (view.exception) { exception_count.fetch_add(1, std::memory_order_relaxed); @@ -368,7 +368,7 @@ void PushingToViewsBlockOutputStream::writeSuffix() exception_happened = true; continue; } - runViewStage(view, stageStep, [&] { processSuffix(view); }); + runViewStage(view, stage_step, [&] { process_suffix(view); }); if (view.exception) { exception_happened = true; From 54f161ca1f4a216fcad20fb4a1e4e9444e1663f9 Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Tue, 10 Aug 2021 13:53:35 +0300 Subject: [PATCH 404/599] Add empty for UUID and arrays. --- .../functions/array-functions.md | 85 +++++++++++++++++-- .../functions/string-functions.md | 22 +++-- .../sql-reference/functions/uuid-functions.md | 84 ++++++++++++++++++ 3 files changed, 174 insertions(+), 17 deletions(-) diff --git a/docs/en/sql-reference/functions/array-functions.md b/docs/en/sql-reference/functions/array-functions.md index b56d403edf6..655ccdeea81 100644 --- a/docs/en/sql-reference/functions/array-functions.md +++ b/docs/en/sql-reference/functions/array-functions.md @@ -7,19 +7,88 @@ toc_title: Arrays ## empty {#function-empty} -Returns 1 for an empty array, or 0 for a non-empty array. -The result type is UInt8. -The function also works for strings. +Checks whether the input array is empty. -Can be optimized by enabling the [optimize_functions_to_subcolumns](../../operations/settings/settings.md#optimize-functions-to-subcolumns) setting. With `optimize_functions_to_subcolumns = 1` the function reads only [size0](../../sql-reference/data-types/array.md#array-size) subcolumn instead of reading and processing the whole array column. The query `SELECT empty(arr) FROM table` transforms to `SELECT arr.size0 = 0 FROM TABLE`. +**Syntax** + +``` sql +empty([x]) +``` + +An array is considered empty if it contains all empty elements. + +!!! note "Note" + Can be optimized by enabling the [optimize_functions_to_subcolumns](../../operations/settings/settings.md#optimize-functions-to-subcolumns) setting. With `optimize_functions_to_subcolumns = 1` the function reads only [size0](../../sql-reference/data-types/array.md#array-size) subcolumn instead of reading and processing the whole array column. The query `SELECT empty(arr) FROM table;` transforms to `SELECT arr.size0 = 0 FROM TABLE;`. + +The function also works for [strings](string-functions.md#empty) or [UUID](uuid-functions.md#empty). + +**Arguments** + +- `[x]` — Input array. [Array](../data-types/array.md). + +**Returned value** + +- Returns `1` for an empty array or `0` for a non-empty array. + +Type: [UInt8](../data-types/int-uint.md). + +**Example** + +Query: + +```sql +SELECT empty([]); +``` + +Result: + +```text +┌─empty(array())─┐ +│ 1 │ +└────────────────┘ +``` ## notEmpty {#function-notempty} -Returns 0 for an empty array, or 1 for a non-empty array. -The result type is UInt8. -The function also works for strings. +Checks whether the input array is non-empty. -Can be optimized by enabling the [optimize_functions_to_subcolumns](../../operations/settings/settings.md#optimize-functions-to-subcolumns) setting. With `optimize_functions_to_subcolumns = 1` the function reads only [size0](../../sql-reference/data-types/array.md#array-size) subcolumn instead of reading and processing the whole array column. The query `SELECT notEmpty(arr) FROM table` transforms to `SELECT arr.size0 != 0 FROM TABLE`. +**Syntax** + +``` sql +notEmpty([x]) +``` +An array is considered non-empty if it contains at least one non-empty element. + +!!! note "Note" + Can be optimized by enabling the [optimize_functions_to_subcolumns](../../operations/settings/settings.md#optimize-functions-to-subcolumns) setting. With `optimize_functions_to_subcolumns = 1` the function reads only [size0](../../sql-reference/data-types/array.md#array-size) subcolumn instead of reading and processing the whole array column. The query `SELECT notEmpty(arr) FROM table` transforms to `SELECT arr.size0 != 0 FROM TABLE`. + +The function also works for [strings](string-functions.md#notempty) or [UUID](uuid-functions.md#notempty). + +**Arguments** + +- `[x]` — Input array. [Array](../data-types/array.md). + +**Returned value** + +- Returns `1` for a non-empty array or `0` for an empty array. + +Type: [UInt8](../data-types/int-uint.md). + +**Example** + +Query: + +```sql +SELECT notEmpty([1,2]); +``` + +Result: + +```text +┌─notEmpty([1, 2])─┐ +│ 1 │ +└──────────────────┘ +``` ## length {#array_functions-length} diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index d54bb16abbc..7f3ecb80ee9 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -20,9 +20,11 @@ empty(x) A string is considered non-empty if it contains at least one byte, even if this is a space or a null byte. The UUID is empty if it contains all zeros (zero UUID). +The function also works for [arrays](array-functions.md#function-empty) or [UUID](uuid-functions.md#empty). + **Arguments** -- `x` — Input value. [Array](../data-types/array.md), [String](../data-types/string.md), [UUID](../data-types/uuid.md). +- `x` — Input value. [String](../data-types/string.md). **Returned value** @@ -46,21 +48,23 @@ Result: └───────────┘ ``` -## notempty {#notempty} +## notEmpty {#notempty} -Checks whether the input string is not empty. +Checks whether the input string is non-empty. **Syntax** ``` sql -notempty(x) +notEmpty(x) ``` A string is considered non-empty if it contains at least one byte, even if this is a space or a null byte. The UUID is empty if it contains all zeros (zero UUID). +The function also works for [arrays](array-functions.md#function-notempty) or [UUID](uuid-functions.md#notempty). + **Arguments** -- `x` — Input value. [Array](../data-types/array.md), [String](../data-types/string.md), [UUID](../data-types/uuid.md). +- `x` — Input value. [String](../data-types/string.md). **Returned value** @@ -73,15 +77,15 @@ Type: [UInt8](../data-types/int-uint.md). Query: ```sql -SELECT notempty('text'); +SELECT notEmpty('text'); ``` Result: ```text -┌─empty('')─┐ -│ 1 │ -└───────────┘ +┌─notEmpty('text')─┐ +│ 1 │ +└──────────────────┘ ``` ## length {#length} diff --git a/docs/en/sql-reference/functions/uuid-functions.md b/docs/en/sql-reference/functions/uuid-functions.md index e7e55c699cd..63bd2446f39 100644 --- a/docs/en/sql-reference/functions/uuid-functions.md +++ b/docs/en/sql-reference/functions/uuid-functions.md @@ -37,6 +37,90 @@ SELECT * FROM t_uuid └──────────────────────────────────────┘ ``` +## empty {#empty} + +Checks whether the input UUID is empty. + +**Syntax** + +```sql +empty(UUID) +``` + +The UUID is considered empty if it contains all zeros (zero UUID). + +The function also works for [arrays](array-functions.md#function-empty) or [strings](string-functions.md#empty). + +**Arguments** + +- `x` — Input UUID. [UUID](../data-types/UUID.md). + +**Returned value** + +- Returns `1` for an empty UUID or `0` for a non-empty UUID. + +Type: [UInt8](../data-types/int-uint.md). + +**Example** + +To generate the UUID value, ClickHouse provides the [generateUUIDv4](uuid-function-generate) function. + +Query: + +```sql +SELECT empty(generateUUIDv4()); +``` + +Result: + +```text +┌─empty(generateUUIDv4())─┐ +│ 0 │ +└─────────────────────────┘ +``` + +## notEmpty {#notempty} + +Checks whether the input UUID is non-empty. + +**Syntax** + +```sql +notEmpty(UUID) +``` + +The UUID is considered empty if it contains all zeros (zero UUID). + +The function also works for [arrays](array-functions.md#function-notempty) or [strings](string-functions.md#notempty). + +**Arguments** + +- `x` — Input UUID. [UUID](../data-types/UUID.md). + +**Returned value** + +- Returns `1` for a non-empty UUID or `0` for an empty UUID. + +Type: [UInt8](../data-types/int-uint.md). + +**Example** + +To generate the UUID value, ClickHouse provides the [generateUUIDv4](uuid-function-generate) function. + +Query: + +```sql +SELECT notEmpty(generateUUIDv4()); +``` + +Result: + +```text +┌─notEmpty(generateUUIDv4())─┐ +│ 1 │ +└────────────────────────────┘ +``` + ## toUUID (x) {#touuid-x} Converts String type value to UUID type. From ede536926509a4537ac1db30a7387fa40e26bd4a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 10 Aug 2021 12:55:49 +0200 Subject: [PATCH 405/599] Revert toMinute changes and add additional roundDown fastpath --- base/common/DateLUTImpl.h | 11 ++++------- 1 file changed, 4 insertions(+), 7 deletions(-) diff --git a/base/common/DateLUTImpl.h b/base/common/DateLUTImpl.h index 85d8403df93..d8268981584 100644 --- a/base/common/DateLUTImpl.h +++ b/base/common/DateLUTImpl.h @@ -257,7 +257,7 @@ private: static_assert(std::is_integral_v && std::is_integral_v); assert(divisor > 0); - if (likely(offset_is_whole_number_of_hours_during_epoch)) + if (likely(offset_is_whole_number_of_hours_during_epoch || (offset_is_whole_number_of_minutes_during_epoch && (divisor % 60 == 0)))) { if (likely(x >= 0)) return x / divisor * divisor; @@ -484,22 +484,19 @@ public: inline unsigned toMinute(Time t) const { - if (likely(t >= 0 && offset_is_whole_number_of_hours_during_epoch)) + if (t >= 0 && offset_is_whole_number_of_hours_during_epoch) return (t / 60) % 60; /// To consider the DST changing situation within this day /// also make the special timezones with no whole hour offset such as 'Australia/Lord_Howe' been taken into account. LUTIndex index = findIndex(t); - Time time = t - lut[index].date; + UInt32 time = t - lut[index].date; if (time >= lut[index].time_at_offset_change()) time += lut[index].amount_of_offset_change(); - Time res = time / 60 % 60; - if (likely(res >= 0)) - return res; - return res + 60; + return time / 60 % 60; } /// NOTE: Assuming timezone offset is a multiple of 15 minutes. From e39e995dc3bed730e5fcf2fb4a2ebf3575989cac Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Tue, 10 Aug 2021 14:06:37 +0300 Subject: [PATCH 406/599] Fix errors. --- docs/en/sql-reference/functions/uuid-functions.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/functions/uuid-functions.md b/docs/en/sql-reference/functions/uuid-functions.md index 63bd2446f39..bbfa19dc1c3 100644 --- a/docs/en/sql-reference/functions/uuid-functions.md +++ b/docs/en/sql-reference/functions/uuid-functions.md @@ -63,7 +63,7 @@ Type: [UInt8](../data-types/int-uint.md). **Example** -To generate the UUID value, ClickHouse provides the [generateUUIDv4](uuid-function-generate) function. +To generate the UUID value, ClickHouse provides the [generateUUIDv4](#uuid-function-generate) function. Query: @@ -105,7 +105,7 @@ Type: [UInt8](../data-types/int-uint.md). **Example** -To generate the UUID value, ClickHouse provides the [generateUUIDv4](uuid-function-generate) function. +To generate the UUID value, ClickHouse provides the [generateUUIDv4](#uuid-function-generate) function. Query: From 60c6daa579c9d89fd915a1f267b91b51f9780fc4 Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Tue, 10 Aug 2021 14:09:20 +0300 Subject: [PATCH 407/599] Fix errors. --- docs/en/sql-reference/functions/uuid-functions.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/functions/uuid-functions.md b/docs/en/sql-reference/functions/uuid-functions.md index bbfa19dc1c3..d02f5af6cd0 100644 --- a/docs/en/sql-reference/functions/uuid-functions.md +++ b/docs/en/sql-reference/functions/uuid-functions.md @@ -53,7 +53,7 @@ The function also works for [arrays](array-functions.md#function-empty) or [stri **Arguments** -- `x` — Input UUID. [UUID](../data-types/UUID.md). +- `x` — Input UUID. [UUID](../data-types/uuid.md). **Returned value** @@ -95,7 +95,7 @@ The function also works for [arrays](array-functions.md#function-notempty) or [s **Arguments** -- `x` — Input UUID. [UUID](../data-types/UUID.md). +- `x` — Input UUID. [UUID](../data-types/uuid.md). **Returned value** From 5e3bcb557405db4043e62cefb2af5237c98d5f18 Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Tue, 10 Aug 2021 14:11:41 +0300 Subject: [PATCH 408/599] Fix errors. --- docs/en/sql-reference/functions/uuid-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/uuid-functions.md b/docs/en/sql-reference/functions/uuid-functions.md index d02f5af6cd0..e5ab45bda40 100644 --- a/docs/en/sql-reference/functions/uuid-functions.md +++ b/docs/en/sql-reference/functions/uuid-functions.md @@ -9,7 +9,7 @@ The functions for working with UUID are listed below. ## generateUUIDv4 {#uuid-function-generate} -Generates the [UUID](../../sql-reference/data-types/uuid.md) of [version 4](https://tools.ietf.org/html/rfc4122#section-4.4). +Generates the [UUID](../data-types/uuid.md) of [version 4](https://tools.ietf.org/html/rfc4122#section-4.4). ``` sql generateUUIDv4() From 669a80000d215c15910d0a84d5e0a6a321f6f211 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 10 Aug 2021 13:20:15 +0200 Subject: [PATCH 409/599] Fix 01236_graphite_mt for random timezones --- .../queries/0_stateless/01236_graphite_mt.sql | 34 +++++++++---------- 1 file changed, 17 insertions(+), 17 deletions(-) diff --git a/tests/queries/0_stateless/01236_graphite_mt.sql b/tests/queries/0_stateless/01236_graphite_mt.sql index 88d2d0ccb63..035fdf6af69 100644 --- a/tests/queries/0_stateless/01236_graphite_mt.sql +++ b/tests/queries/0_stateless/01236_graphite_mt.sql @@ -1,26 +1,26 @@ drop table if exists test_graphite; -create table test_graphite (key UInt32, Path String, Time DateTime, Value Float64, Version UInt32, col UInt64) +create table test_graphite (key UInt32, Path String, Time DateTime('UTC'), Value Float64, Version UInt32, col UInt64) engine = GraphiteMergeTree('graphite_rollup') order by key settings index_granularity=10; insert into test_graphite -select 1, 'sum_1', toDateTime(today()) - number * 60 - 30, number, 1, number from numbers(300) union all -select 2, 'sum_1', toDateTime(today()) - number * 60 - 30, number, 1, number from numbers(300) union all -select 1, 'sum_2', toDateTime(today()) - number * 60 - 30, number, 1, number from numbers(300) union all -select 2, 'sum_2', toDateTime(today()) - number * 60 - 30, number, 1, number from numbers(300) union all -select 1, 'max_1', toDateTime(today()) - number * 60 - 30, number, 1, number from numbers(300) union all -select 2, 'max_1', toDateTime(today()) - number * 60 - 30, number, 1, number from numbers(300) union all -select 1, 'max_2', toDateTime(today()) - number * 60 - 30, number, 1, number from numbers(300) union all -select 2, 'max_2', toDateTime(today()) - number * 60 - 30, number, 1, number from numbers(300); +select 1, 'sum_1', toDateTime(today(), 'UTC') - number * 60 - 30, number, 1, number from numbers(300) union all +select 2, 'sum_1', toDateTime(today(), 'UTC') - number * 60 - 30, number, 1, number from numbers(300) union all +select 1, 'sum_2', toDateTime(today(), 'UTC') - number * 60 - 30, number, 1, number from numbers(300) union all +select 2, 'sum_2', toDateTime(today(), 'UTC') - number * 60 - 30, number, 1, number from numbers(300) union all +select 1, 'max_1', toDateTime(today(), 'UTC') - number * 60 - 30, number, 1, number from numbers(300) union all +select 2, 'max_1', toDateTime(today(), 'UTC') - number * 60 - 30, number, 1, number from numbers(300) union all +select 1, 'max_2', toDateTime(today(), 'UTC') - number * 60 - 30, number, 1, number from numbers(300) union all +select 2, 'max_2', toDateTime(today(), 'UTC') - number * 60 - 30, number, 1, number from numbers(300); insert into test_graphite -select 1, 'sum_1', toDateTime(today() - 3) - number * 60 - 30, number, 1, number from numbers(1200) union all -select 2, 'sum_1', toDateTime(today() - 3) - number * 60 - 30, number, 1, number from numbers(1200) union all -select 1, 'sum_2', toDateTime(today() - 3) - number * 60 - 30, number, 1, number from numbers(1200) union all -select 2, 'sum_2', toDateTime(today() - 3) - number * 60 - 30, number, 1, number from numbers(1200) union all -select 1, 'max_1', toDateTime(today() - 3) - number * 60 - 30, number, 1, number from numbers(1200) union all -select 2, 'max_1', toDateTime(today() - 3) - number * 60 - 30, number, 1, number from numbers(1200) union all -select 1, 'max_2', toDateTime(today() - 3) - number * 60 - 30, number, 1, number from numbers(1200) union all -select 2, 'max_2', toDateTime(today() - 3) - number * 60 - 30, number, 1, number from numbers(1200); +select 1, 'sum_1', toDateTime(today() -3 , 'UTC') - number * 60 - 30, number, 1, number from numbers(1200) union all +select 2, 'sum_1', toDateTime(today() -3 , 'UTC') - number * 60 - 30, number, 1, number from numbers(1200) union all +select 1, 'sum_2', toDateTime(today() -3 , 'UTC') - number * 60 - 30, number, 1, number from numbers(1200) union all +select 2, 'sum_2', toDateTime(today() -3 , 'UTC') - number * 60 - 30, number, 1, number from numbers(1200) union all +select 1, 'max_1', toDateTime(today() -3 , 'UTC') - number * 60 - 30, number, 1, number from numbers(1200) union all +select 2, 'max_1', toDateTime(today() -3 , 'UTC') - number * 60 - 30, number, 1, number from numbers(1200) union all +select 1, 'max_2', toDateTime(today() -3 , 'UTC') - number * 60 - 30, number, 1, number from numbers(1200) union all +select 2, 'max_2', toDateTime(today() -3 , 'UTC') - number * 60 - 30, number, 1, number from numbers(1200); optimize table test_graphite final; From 07cc2ec85b69539e56c5bcce25a0bba997160a98 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 10 Aug 2021 15:36:40 +0300 Subject: [PATCH 410/599] Try fix build. --- src/Common/DenseHashMap.h | 15 ++++++--------- src/Common/DenseHashSet.h | 14 +++++--------- src/Common/SparseHashMap.h | 14 +++++--------- 3 files changed, 16 insertions(+), 27 deletions(-) diff --git a/src/Common/DenseHashMap.h b/src/Common/DenseHashMap.h index 44287c604a7..7a9d5a68e2d 100644 --- a/src/Common/DenseHashMap.h +++ b/src/Common/DenseHashMap.h @@ -1,18 +1,13 @@ #pragma once +#include #if defined(ARCADIA_BUILD) -#undef HASH_FUN_H -#define HASH_FUN_H +#define HASH_FUN_H ; +template +struct THash : public std::hash {}; #endif -#ifdef __clang__ -#pragma clang diagnostic push -#pragma clang diagnostic ignored "-Wmacro-redefined" -#endif #include -#ifdef __clang__ -#pragma clang diagnostic pop -#endif #if !defined(ARCADIA_BUILD) template , @@ -24,4 +19,6 @@ class EqualKey = std::equal_to, class Alloc = google::sparsehash::libc_allocator_with_realloc>> using DenseHashMap = google::sparsehash::dense_hash_map; + + #undef THash #endif diff --git a/src/Common/DenseHashSet.h b/src/Common/DenseHashSet.h index ee3838d7be1..004d1c10903 100644 --- a/src/Common/DenseHashSet.h +++ b/src/Common/DenseHashSet.h @@ -1,18 +1,12 @@ #pragma once #if defined(ARCADIA_BUILD) -#undef HASH_FUN_H -#define HASH_FUN_H +#define HASH_FUN_H ; +template +struct THash : public std::hash {}; #endif -#ifdef __clang__ -#pragma clang diagnostic push -#pragma clang diagnostic ignored "-Wmacro-redefined" -#endif #include -#ifdef __clang__ -#pragma clang diagnostic pop -#endif #if !defined(ARCADIA_BUILD) template , @@ -24,4 +18,6 @@ class EqualKey = std::equal_to, class Alloc = google::sparsehash::libc_allocator_with_realloc> using DenseHashSet = google::sparsehash::dense_hash_set; + + #undef THash #endif diff --git a/src/Common/SparseHashMap.h b/src/Common/SparseHashMap.h index b6a7f1f9b9c..651dd22f15d 100644 --- a/src/Common/SparseHashMap.h +++ b/src/Common/SparseHashMap.h @@ -1,18 +1,12 @@ #pragma once #if defined(ARCADIA_BUILD) -#undef HASH_FUN_H -#define HASH_FUN_H +#define HASH_FUN_H ; +template +struct THash : public std::hash {}; #endif -#ifdef __clang__ -#pragma clang diagnostic push -#pragma clang diagnostic ignored "-Wmacro-redefined" -#endif #include -#ifdef __clang__ -#pragma clang diagnostic pop -#endif #if !defined(ARCADIA_BUILD) template , @@ -24,4 +18,6 @@ class EqualKey = std::equal_to, class Alloc = google::sparsehash::libc_allocator_with_realloc>> using SparseHashMap = google::sparsehash::sparse_hash_map; + + #undef THash #endif From 7324c7cd6b32a32d23f8a605c6afd770d7b9daa2 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 10 Aug 2021 15:35:39 +0200 Subject: [PATCH 411/599] Style #2 --- src/DataStreams/PushingToViewsBlockOutputStream.cpp | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/src/DataStreams/PushingToViewsBlockOutputStream.cpp b/src/DataStreams/PushingToViewsBlockOutputStream.cpp index c24df2d94c0..aec1209a454 100644 --- a/src/DataStreams/PushingToViewsBlockOutputStream.cpp +++ b/src/DataStreams/PushingToViewsBlockOutputStream.cpp @@ -311,7 +311,8 @@ void PushingToViewsBlockOutputStream::writeSuffix() if (views.empty()) return; - auto process_suffix = [](ViewRuntimeData & view) { + auto process_suffix = [](ViewRuntimeData & view) + { view.out->writeSuffix(); view.runtime_stats.setStatus(QueryViewsLogElement::ViewStatus::QUERY_FINISH); }; @@ -443,7 +444,8 @@ void PushingToViewsBlockOutputStream::process(const Block & block, ViewRuntimeDa else in = std::make_shared(block); - in->setProgressCallback([this](const Progress & progress) { + in->setProgressCallback([this](const Progress & progress) + { CurrentThread::updateProgressIn(progress); this->onProgress(progress); }); From e63c26edb713b8452b0979740020e9205cd3f839 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Tue, 10 Aug 2021 21:47:27 +0800 Subject: [PATCH 412/599] Fix projection materialization with missing columns --- src/Interpreters/MutationsInterpreter.cpp | 5 +++-- src/Interpreters/MutationsInterpreter.h | 2 +- .../MergeTree/MergeTreeDataMergerMutator.cpp | 12 ++++-------- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 5 +++-- ...ection_materialize_with_missing_columns.reference | 0 ...0_projection_materialize_with_missing_columns.sql | 9 +++++++++ 6 files changed, 20 insertions(+), 13 deletions(-) create mode 100644 tests/queries/0_stateless/01710_projection_materialize_with_missing_columns.reference create mode 100644 tests/queries/0_stateless/01710_projection_materialize_with_missing_columns.sql diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index fe0594bb58f..ff2f3e97c4e 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -920,9 +920,10 @@ BlockInputStreamPtr MutationsInterpreter::execute() return result_stream; } -const Block & MutationsInterpreter::getUpdatedHeader() const +Block MutationsInterpreter::getUpdatedHeader() const { - return *updated_header; + // If it's an index/projection materialization, we don't write any data columns, thus empty header is used + return mutation_kind.mutation_kind == MutationKind::MUTATE_INDEX_PROJECTION ? Block{} : *updated_header; } const ColumnDependencies & MutationsInterpreter::getColumnDependencies() const diff --git a/src/Interpreters/MutationsInterpreter.h b/src/Interpreters/MutationsInterpreter.h index c9a589e6b6d..4f8960ae8f7 100644 --- a/src/Interpreters/MutationsInterpreter.h +++ b/src/Interpreters/MutationsInterpreter.h @@ -53,7 +53,7 @@ public: BlockInputStreamPtr execute(); /// Only changed columns. - const Block & getUpdatedHeader() const; + Block getUpdatedHeader() const; const ColumnDependencies & getColumnDependencies() const; diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 6279d2d7d6f..0a92efdf61c 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -1332,11 +1332,8 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor { /// We will modify only some of the columns. Other columns and key values can be copied as-is. NameSet updated_columns; - if (mutation_kind != MutationsInterpreter::MutationKind::MUTATE_INDEX_PROJECTION) - { - for (const auto & name_type : updated_header.getNamesAndTypesList()) - updated_columns.emplace(name_type.name); - } + for (const auto & name_type : updated_header.getNamesAndTypesList()) + updated_columns.emplace(name_type.name); auto indices_to_recalc = getIndicesToRecalculate( in, updated_columns, metadata_snapshot, context, materialized_indices, source_part); @@ -1345,7 +1342,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor NameSet files_to_skip = collectFilesToSkip( source_part, - mutation_kind == MutationsInterpreter::MutationKind::MUTATE_INDEX_PROJECTION ? Block{} : updated_header, + updated_header, indices_to_recalc, mrk_extension, projections_to_recalc); @@ -1413,8 +1410,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor metadata_snapshot, indices_to_recalc, projections_to_recalc, - // If it's an index/projection materialization, we don't write any data columns, thus empty header is used - mutation_kind == MutationsInterpreter::MutationKind::MUTATE_INDEX_PROJECTION ? Block{} : updated_header, + updated_header, new_data_part, in, time_of_mutation, diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 0b5351dcf01..342bcb25b0b 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -361,10 +361,11 @@ QueryPlanPtr MergeTreeDataSelectExecutor::read( pipes.emplace_back(std::move(projection_pipe)); pipes.emplace_back(std::move(ordinary_pipe)); auto pipe = Pipe::unitePipes(std::move(pipes)); - // TODO what if pipe is empty? pipe.resize(1); - auto step = std::make_unique(std::move(pipe), "MergeTree(with projection)"); + auto step = std::make_unique( + std::move(pipe), + fmt::format("MergeTree(with {} projection {})", query_info.projection->desc->type, query_info.projection->desc->name)); auto plan = std::make_unique(); plan->addStep(std::move(step)); return plan; diff --git a/tests/queries/0_stateless/01710_projection_materialize_with_missing_columns.reference b/tests/queries/0_stateless/01710_projection_materialize_with_missing_columns.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/01710_projection_materialize_with_missing_columns.sql b/tests/queries/0_stateless/01710_projection_materialize_with_missing_columns.sql new file mode 100644 index 00000000000..28bf1c050d0 --- /dev/null +++ b/tests/queries/0_stateless/01710_projection_materialize_with_missing_columns.sql @@ -0,0 +1,9 @@ +drop table if exists x; + +create table x (i int) engine MergeTree order by tuple(); +insert into x values (1); +alter table x add column j int; +alter table x add projection p_agg (select sum(j)); +alter table x materialize projection p_agg settings mutations_sync = 1; + +drop table x; From c2cd4d452a9d91440050d775236d62a3bce4257d Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Tue, 10 Aug 2021 17:42:18 +0300 Subject: [PATCH 413/599] Update Settings.h --- src/Core/Settings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 623eb0298dd..05374a398eb 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -434,7 +434,7 @@ class IColumn; M(Bool, allow_experimental_geo_types, false, "Allow geo data types such as Point, Ring, Polygon, MultiPolygon", 0) \ M(Bool, data_type_default_nullable, false, "Data types without NULL or NOT NULL will make Nullable", 0) \ /** Temporarily set to true, to check how tests will feel.*/ \ - M(Bool, cast_keep_nullable, true, "CAST operator keep Nullable for result data type", 0) \ + M(Bool, cast_keep_nullable, false, "CAST operator keep Nullable for result data type", 0) \ M(Bool, alter_partition_verbose_result, false, "Output information about affected parts. Currently works only for FREEZE and ATTACH commands.", 0) \ M(Bool, allow_experimental_database_materialized_mysql, false, "Allow to create database with Engine=MaterializedMySQL(...).", 0) \ M(Bool, allow_experimental_database_materialized_postgresql, false, "Allow to create database with Engine=MaterializedPostgreSQL(...).", 0) \ From 8bae14acb13dc6116256218ba3f03c78f7f374b2 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 10 Aug 2021 17:01:56 +0200 Subject: [PATCH 414/599] Separate minute intervals Instead of mixing multiple rounding, use toStartOfMinuteInterval for all minute related calculations --- base/common/DateLUTImpl.h | 33 ++++++++++++++++----------------- 1 file changed, 16 insertions(+), 17 deletions(-) diff --git a/base/common/DateLUTImpl.h b/base/common/DateLUTImpl.h index d8268981584..17f7a3d525c 100644 --- a/base/common/DateLUTImpl.h +++ b/base/common/DateLUTImpl.h @@ -257,7 +257,7 @@ private: static_assert(std::is_integral_v && std::is_integral_v); assert(divisor > 0); - if (likely(offset_is_whole_number_of_hours_during_epoch || (offset_is_whole_number_of_minutes_during_epoch && (divisor % 60 == 0)))) + if (likely(offset_is_whole_number_of_hours_during_epoch)) { if (likely(x >= 0)) return x / divisor * divisor; @@ -500,10 +500,10 @@ public: } /// NOTE: Assuming timezone offset is a multiple of 15 minutes. - inline Time toStartOfMinute(Time t) const { return roundDown(t, 60); } - inline Time toStartOfFiveMinute(Time t) const { return roundDown(t, 300); } - inline Time toStartOfFifteenMinutes(Time t) const { return roundDown(t, 900); } - inline Time toStartOfTenMinutes(Time t) const { return roundDown(t, 600); } + inline Time toStartOfMinute(Time t) const { return toStartOfMinuteInterval(t, 1); } + inline Time toStartOfFiveMinute(Time t) const { return toStartOfMinuteInterval(t, 5); } + inline Time toStartOfFifteenMinutes(Time t) const { return toStartOfMinuteInterval(t, 15); } + inline Time toStartOfTenMinutes(Time t) const { return toStartOfMinuteInterval(t, 10); } inline Time toStartOfHour(Time t) const { return roundDown(t, 3600); } /** Number of calendar day since the beginning of UNIX epoch (1970-01-01 is zero) @@ -902,25 +902,24 @@ public: inline Time toStartOfMinuteInterval(Time t, UInt64 minutes) const { - if (minutes == 1) - return toStartOfMinute(t); + UInt64 divisor = 60 * minutes; + if (likely(offset_is_whole_number_of_minutes_during_epoch)) + { + if (likely(t >= 0)) + return t / divisor * divisor; + return (t + 1 - divisor) / divisor * divisor; + } - /** In contrast to "toStartOfHourInterval" function above, - * the minute intervals are not aligned to the midnight. - * You will get unexpected results if for example, you round down to 60 minute interval - * and there was a time shift to 30 minutes. - * - * But this is not specified in docs and can be changed in future. - */ - - UInt64 seconds = 60 * minutes; - return roundDown(t, seconds); + Time date = find(t).date; + return date + (t - date) / divisor * divisor; } inline Time toStartOfSecondInterval(Time t, UInt64 seconds) const { if (seconds == 1) return t; + if (seconds % 60 == 0) + return toStartOfMinuteInterval(t, seconds / 60); return roundDown(t, seconds); } From 3dfa54bfd8afada336fe9449561614fc80aea25b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 10 Aug 2021 17:08:43 +0200 Subject: [PATCH 415/599] Revert "Fix 01236_graphite_mt for random timezones" This reverts commit 669a80000d215c15910d0a84d5e0a6a321f6f211. --- .../queries/0_stateless/01236_graphite_mt.sql | 34 +++++++++---------- 1 file changed, 17 insertions(+), 17 deletions(-) diff --git a/tests/queries/0_stateless/01236_graphite_mt.sql b/tests/queries/0_stateless/01236_graphite_mt.sql index 035fdf6af69..88d2d0ccb63 100644 --- a/tests/queries/0_stateless/01236_graphite_mt.sql +++ b/tests/queries/0_stateless/01236_graphite_mt.sql @@ -1,26 +1,26 @@ drop table if exists test_graphite; -create table test_graphite (key UInt32, Path String, Time DateTime('UTC'), Value Float64, Version UInt32, col UInt64) +create table test_graphite (key UInt32, Path String, Time DateTime, Value Float64, Version UInt32, col UInt64) engine = GraphiteMergeTree('graphite_rollup') order by key settings index_granularity=10; insert into test_graphite -select 1, 'sum_1', toDateTime(today(), 'UTC') - number * 60 - 30, number, 1, number from numbers(300) union all -select 2, 'sum_1', toDateTime(today(), 'UTC') - number * 60 - 30, number, 1, number from numbers(300) union all -select 1, 'sum_2', toDateTime(today(), 'UTC') - number * 60 - 30, number, 1, number from numbers(300) union all -select 2, 'sum_2', toDateTime(today(), 'UTC') - number * 60 - 30, number, 1, number from numbers(300) union all -select 1, 'max_1', toDateTime(today(), 'UTC') - number * 60 - 30, number, 1, number from numbers(300) union all -select 2, 'max_1', toDateTime(today(), 'UTC') - number * 60 - 30, number, 1, number from numbers(300) union all -select 1, 'max_2', toDateTime(today(), 'UTC') - number * 60 - 30, number, 1, number from numbers(300) union all -select 2, 'max_2', toDateTime(today(), 'UTC') - number * 60 - 30, number, 1, number from numbers(300); +select 1, 'sum_1', toDateTime(today()) - number * 60 - 30, number, 1, number from numbers(300) union all +select 2, 'sum_1', toDateTime(today()) - number * 60 - 30, number, 1, number from numbers(300) union all +select 1, 'sum_2', toDateTime(today()) - number * 60 - 30, number, 1, number from numbers(300) union all +select 2, 'sum_2', toDateTime(today()) - number * 60 - 30, number, 1, number from numbers(300) union all +select 1, 'max_1', toDateTime(today()) - number * 60 - 30, number, 1, number from numbers(300) union all +select 2, 'max_1', toDateTime(today()) - number * 60 - 30, number, 1, number from numbers(300) union all +select 1, 'max_2', toDateTime(today()) - number * 60 - 30, number, 1, number from numbers(300) union all +select 2, 'max_2', toDateTime(today()) - number * 60 - 30, number, 1, number from numbers(300); insert into test_graphite -select 1, 'sum_1', toDateTime(today() -3 , 'UTC') - number * 60 - 30, number, 1, number from numbers(1200) union all -select 2, 'sum_1', toDateTime(today() -3 , 'UTC') - number * 60 - 30, number, 1, number from numbers(1200) union all -select 1, 'sum_2', toDateTime(today() -3 , 'UTC') - number * 60 - 30, number, 1, number from numbers(1200) union all -select 2, 'sum_2', toDateTime(today() -3 , 'UTC') - number * 60 - 30, number, 1, number from numbers(1200) union all -select 1, 'max_1', toDateTime(today() -3 , 'UTC') - number * 60 - 30, number, 1, number from numbers(1200) union all -select 2, 'max_1', toDateTime(today() -3 , 'UTC') - number * 60 - 30, number, 1, number from numbers(1200) union all -select 1, 'max_2', toDateTime(today() -3 , 'UTC') - number * 60 - 30, number, 1, number from numbers(1200) union all -select 2, 'max_2', toDateTime(today() -3 , 'UTC') - number * 60 - 30, number, 1, number from numbers(1200); +select 1, 'sum_1', toDateTime(today() - 3) - number * 60 - 30, number, 1, number from numbers(1200) union all +select 2, 'sum_1', toDateTime(today() - 3) - number * 60 - 30, number, 1, number from numbers(1200) union all +select 1, 'sum_2', toDateTime(today() - 3) - number * 60 - 30, number, 1, number from numbers(1200) union all +select 2, 'sum_2', toDateTime(today() - 3) - number * 60 - 30, number, 1, number from numbers(1200) union all +select 1, 'max_1', toDateTime(today() - 3) - number * 60 - 30, number, 1, number from numbers(1200) union all +select 2, 'max_1', toDateTime(today() - 3) - number * 60 - 30, number, 1, number from numbers(1200) union all +select 1, 'max_2', toDateTime(today() - 3) - number * 60 - 30, number, 1, number from numbers(1200) union all +select 2, 'max_2', toDateTime(today() - 3) - number * 60 - 30, number, 1, number from numbers(1200); optimize table test_graphite final; From 476b066668d3d19bdbc8f70ab7abae4a139387f5 Mon Sep 17 00:00:00 2001 From: abel-wang Date: Tue, 10 Aug 2021 23:21:23 +0800 Subject: [PATCH 416/599] Enable with constants. --- src/Interpreters/RequiredSourceColumnsVisitor.cpp | 13 +++++++++++++ 1 file changed, 13 insertions(+) diff --git a/src/Interpreters/RequiredSourceColumnsVisitor.cpp b/src/Interpreters/RequiredSourceColumnsVisitor.cpp index 2f2a68656bc..f6786451c82 100644 --- a/src/Interpreters/RequiredSourceColumnsVisitor.cpp +++ b/src/Interpreters/RequiredSourceColumnsVisitor.cpp @@ -123,6 +123,17 @@ void RequiredSourceColumnsMatcher::visit(const ASTSelectQuery & select, const AS data.addColumnAliasIfAny(*node); } + if (auto & with = select.with()) + { + for (auto & node : with->children) + { + if (const auto * identifier = node->as()) + data.addColumnIdentifier(*identifier); + else + data.addColumnAliasIfAny(*node); + } + } + std::vector out; for (const auto & node : select.children) { @@ -134,6 +145,8 @@ void RequiredSourceColumnsMatcher::visit(const ASTSelectQuery & select, const AS /// revisit select_expression_list (with children) when all the aliases are set Visitor(data).visit(select.select()); + if (auto with = select.with()) + Visitor(data).visit(with); } void RequiredSourceColumnsMatcher::visit(const ASTIdentifier & node, const ASTPtr &, Data & data) From ec0ee2cecf87e3dd72aca9992d304b65e8d6d9db Mon Sep 17 00:00:00 2001 From: abel-wang Date: Tue, 10 Aug 2021 23:22:33 +0800 Subject: [PATCH 417/599] Replace parameters in ASTFunctions with alias. --- src/Interpreters/QueryNormalizer.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Interpreters/QueryNormalizer.cpp b/src/Interpreters/QueryNormalizer.cpp index ea61ade2b49..7c820622c37 100644 --- a/src/Interpreters/QueryNormalizer.cpp +++ b/src/Interpreters/QueryNormalizer.cpp @@ -256,6 +256,9 @@ void QueryNormalizer::visit(ASTPtr & ast, Data & data) visit(*node_select, ast, data); else if (auto * node_param = ast->as()) throw Exception("Query parameter " + backQuote(node_param->name) + " was not set", ErrorCodes::UNKNOWN_QUERY_PARAMETER); + else if (auto * node_function = ast->as()) + if (node_function->parameters) + visit(node_function->parameters, data); /// If we replace the root of the subtree, we will be called again for the new root, in case the alias is replaced by an alias. if (ast.get() != initial_ast.get()) From ee416e79bdaf28dc668ab7a5c593aeb9c32f948f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 10 Aug 2021 17:55:02 +0200 Subject: [PATCH 418/599] 01236_graphite_mt: Use static date and print dates in results --- .../0_stateless/01236_graphite_mt.reference | 688 +++++++++--------- .../queries/0_stateless/01236_graphite_mt.sql | 38 +- 2 files changed, 364 insertions(+), 362 deletions(-) diff --git a/tests/queries/0_stateless/01236_graphite_mt.reference b/tests/queries/0_stateless/01236_graphite_mt.reference index a30d2495265..c68becad53d 100644 --- a/tests/queries/0_stateless/01236_graphite_mt.reference +++ b/tests/queries/0_stateless/01236_graphite_mt.reference @@ -1,344 +1,344 @@ -1 max_1 9 1 0 -1 max_1 19 1 10 -1 max_1 29 1 20 -1 max_1 39 1 30 -1 max_1 49 1 40 -1 max_1 59 1 50 -1 max_1 69 1 60 -1 max_1 79 1 70 -1 max_1 89 1 80 -1 max_1 99 1 90 -1 max_1 109 1 100 -1 max_1 119 1 110 -1 max_1 129 1 120 -1 max_1 139 1 130 -1 max_1 149 1 140 -1 max_1 159 1 150 -1 max_1 169 1 160 -1 max_1 179 1 170 -1 max_1 189 1 180 -1 max_1 199 1 190 -1 max_1 209 1 200 -1 max_1 219 1 210 -1 max_1 229 1 220 -1 max_1 239 1 230 -1 max_1 249 1 240 -1 max_1 259 1 250 -1 max_1 269 1 260 -1 max_1 279 1 270 -1 max_1 289 1 280 -1 max_1 299 1 290 -1 max_1 39 1 0 -1 max_1 139 1 40 -1 max_1 239 1 140 -1 max_1 339 1 240 -1 max_1 439 1 340 -1 max_1 539 1 440 -1 max_1 639 1 540 -1 max_1 739 1 640 -1 max_1 839 1 740 -1 max_1 939 1 840 -1 max_1 1039 1 940 -1 max_1 1139 1 1040 -1 max_1 1199 1 1140 -1 max_2 9 1 0 -1 max_2 19 1 10 -1 max_2 29 1 20 -1 max_2 39 1 30 -1 max_2 49 1 40 -1 max_2 59 1 50 -1 max_2 69 1 60 -1 max_2 79 1 70 -1 max_2 89 1 80 -1 max_2 99 1 90 -1 max_2 109 1 100 -1 max_2 119 1 110 -1 max_2 129 1 120 -1 max_2 139 1 130 -1 max_2 149 1 140 -1 max_2 159 1 150 -1 max_2 169 1 160 -1 max_2 179 1 170 -1 max_2 189 1 180 -1 max_2 199 1 190 -1 max_2 209 1 200 -1 max_2 219 1 210 -1 max_2 229 1 220 -1 max_2 239 1 230 -1 max_2 249 1 240 -1 max_2 259 1 250 -1 max_2 269 1 260 -1 max_2 279 1 270 -1 max_2 289 1 280 -1 max_2 299 1 290 -1 max_2 39 1 0 -1 max_2 139 1 40 -1 max_2 239 1 140 -1 max_2 339 1 240 -1 max_2 439 1 340 -1 max_2 539 1 440 -1 max_2 639 1 540 -1 max_2 739 1 640 -1 max_2 839 1 740 -1 max_2 939 1 840 -1 max_2 1039 1 940 -1 max_2 1139 1 1040 -1 max_2 1199 1 1140 -1 sum_1 45 1 0 -1 sum_1 145 1 10 -1 sum_1 245 1 20 -1 sum_1 345 1 30 -1 sum_1 445 1 40 -1 sum_1 545 1 50 -1 sum_1 645 1 60 -1 sum_1 745 1 70 -1 sum_1 845 1 80 -1 sum_1 945 1 90 -1 sum_1 1045 1 100 -1 sum_1 1145 1 110 -1 sum_1 1245 1 120 -1 sum_1 1345 1 130 -1 sum_1 1445 1 140 -1 sum_1 1545 1 150 -1 sum_1 1645 1 160 -1 sum_1 1745 1 170 -1 sum_1 1845 1 180 -1 sum_1 1945 1 190 -1 sum_1 2045 1 200 -1 sum_1 2145 1 210 -1 sum_1 2245 1 220 -1 sum_1 2345 1 230 -1 sum_1 2445 1 240 -1 sum_1 2545 1 250 -1 sum_1 2645 1 260 -1 sum_1 2745 1 270 -1 sum_1 2845 1 280 -1 sum_1 2945 1 290 -1 sum_1 780 1 0 -1 sum_1 8950 1 40 -1 sum_1 18950 1 140 -1 sum_1 28950 1 240 -1 sum_1 38950 1 340 -1 sum_1 48950 1 440 -1 sum_1 58950 1 540 -1 sum_1 68950 1 640 -1 sum_1 78950 1 740 -1 sum_1 88950 1 840 -1 sum_1 98950 1 940 -1 sum_1 108950 1 1040 -1 sum_1 70170 1 1140 -1 sum_2 45 1 0 -1 sum_2 145 1 10 -1 sum_2 245 1 20 -1 sum_2 345 1 30 -1 sum_2 445 1 40 -1 sum_2 545 1 50 -1 sum_2 645 1 60 -1 sum_2 745 1 70 -1 sum_2 845 1 80 -1 sum_2 945 1 90 -1 sum_2 1045 1 100 -1 sum_2 1145 1 110 -1 sum_2 1245 1 120 -1 sum_2 1345 1 130 -1 sum_2 1445 1 140 -1 sum_2 1545 1 150 -1 sum_2 1645 1 160 -1 sum_2 1745 1 170 -1 sum_2 1845 1 180 -1 sum_2 1945 1 190 -1 sum_2 2045 1 200 -1 sum_2 2145 1 210 -1 sum_2 2245 1 220 -1 sum_2 2345 1 230 -1 sum_2 2445 1 240 -1 sum_2 2545 1 250 -1 sum_2 2645 1 260 -1 sum_2 2745 1 270 -1 sum_2 2845 1 280 -1 sum_2 2945 1 290 -1 sum_2 780 1 0 -1 sum_2 8950 1 40 -1 sum_2 18950 1 140 -1 sum_2 28950 1 240 -1 sum_2 38950 1 340 -1 sum_2 48950 1 440 -1 sum_2 58950 1 540 -1 sum_2 68950 1 640 -1 sum_2 78950 1 740 -1 sum_2 88950 1 840 -1 sum_2 98950 1 940 -1 sum_2 108950 1 1040 -1 sum_2 70170 1 1140 -2 max_1 9 1 0 -2 max_1 19 1 10 -2 max_1 29 1 20 -2 max_1 39 1 30 -2 max_1 49 1 40 -2 max_1 59 1 50 -2 max_1 69 1 60 -2 max_1 79 1 70 -2 max_1 89 1 80 -2 max_1 99 1 90 -2 max_1 109 1 100 -2 max_1 119 1 110 -2 max_1 129 1 120 -2 max_1 139 1 130 -2 max_1 149 1 140 -2 max_1 159 1 150 -2 max_1 169 1 160 -2 max_1 179 1 170 -2 max_1 189 1 180 -2 max_1 199 1 190 -2 max_1 209 1 200 -2 max_1 219 1 210 -2 max_1 229 1 220 -2 max_1 239 1 230 -2 max_1 249 1 240 -2 max_1 259 1 250 -2 max_1 269 1 260 -2 max_1 279 1 270 -2 max_1 289 1 280 -2 max_1 299 1 290 -2 max_1 39 1 0 -2 max_1 139 1 40 -2 max_1 239 1 140 -2 max_1 339 1 240 -2 max_1 439 1 340 -2 max_1 539 1 440 -2 max_1 639 1 540 -2 max_1 739 1 640 -2 max_1 839 1 740 -2 max_1 939 1 840 -2 max_1 1039 1 940 -2 max_1 1139 1 1040 -2 max_1 1199 1 1140 -2 max_2 9 1 0 -2 max_2 19 1 10 -2 max_2 29 1 20 -2 max_2 39 1 30 -2 max_2 49 1 40 -2 max_2 59 1 50 -2 max_2 69 1 60 -2 max_2 79 1 70 -2 max_2 89 1 80 -2 max_2 99 1 90 -2 max_2 109 1 100 -2 max_2 119 1 110 -2 max_2 129 1 120 -2 max_2 139 1 130 -2 max_2 149 1 140 -2 max_2 159 1 150 -2 max_2 169 1 160 -2 max_2 179 1 170 -2 max_2 189 1 180 -2 max_2 199 1 190 -2 max_2 209 1 200 -2 max_2 219 1 210 -2 max_2 229 1 220 -2 max_2 239 1 230 -2 max_2 249 1 240 -2 max_2 259 1 250 -2 max_2 269 1 260 -2 max_2 279 1 270 -2 max_2 289 1 280 -2 max_2 299 1 290 -2 max_2 39 1 0 -2 max_2 139 1 40 -2 max_2 239 1 140 -2 max_2 339 1 240 -2 max_2 439 1 340 -2 max_2 539 1 440 -2 max_2 639 1 540 -2 max_2 739 1 640 -2 max_2 839 1 740 -2 max_2 939 1 840 -2 max_2 1039 1 940 -2 max_2 1139 1 1040 -2 max_2 1199 1 1140 -2 sum_1 45 1 0 -2 sum_1 145 1 10 -2 sum_1 245 1 20 -2 sum_1 345 1 30 -2 sum_1 445 1 40 -2 sum_1 545 1 50 -2 sum_1 645 1 60 -2 sum_1 745 1 70 -2 sum_1 845 1 80 -2 sum_1 945 1 90 -2 sum_1 1045 1 100 -2 sum_1 1145 1 110 -2 sum_1 1245 1 120 -2 sum_1 1345 1 130 -2 sum_1 1445 1 140 -2 sum_1 1545 1 150 -2 sum_1 1645 1 160 -2 sum_1 1745 1 170 -2 sum_1 1845 1 180 -2 sum_1 1945 1 190 -2 sum_1 2045 1 200 -2 sum_1 2145 1 210 -2 sum_1 2245 1 220 -2 sum_1 2345 1 230 -2 sum_1 2445 1 240 -2 sum_1 2545 1 250 -2 sum_1 2645 1 260 -2 sum_1 2745 1 270 -2 sum_1 2845 1 280 -2 sum_1 2945 1 290 -2 sum_1 780 1 0 -2 sum_1 8950 1 40 -2 sum_1 18950 1 140 -2 sum_1 28950 1 240 -2 sum_1 38950 1 340 -2 sum_1 48950 1 440 -2 sum_1 58950 1 540 -2 sum_1 68950 1 640 -2 sum_1 78950 1 740 -2 sum_1 88950 1 840 -2 sum_1 98950 1 940 -2 sum_1 108950 1 1040 -2 sum_1 70170 1 1140 -2 sum_2 45 1 0 -2 sum_2 145 1 10 -2 sum_2 245 1 20 -2 sum_2 345 1 30 -2 sum_2 445 1 40 -2 sum_2 545 1 50 -2 sum_2 645 1 60 -2 sum_2 745 1 70 -2 sum_2 845 1 80 -2 sum_2 945 1 90 -2 sum_2 1045 1 100 -2 sum_2 1145 1 110 -2 sum_2 1245 1 120 -2 sum_2 1345 1 130 -2 sum_2 1445 1 140 -2 sum_2 1545 1 150 -2 sum_2 1645 1 160 -2 sum_2 1745 1 170 -2 sum_2 1845 1 180 -2 sum_2 1945 1 190 -2 sum_2 2045 1 200 -2 sum_2 2145 1 210 -2 sum_2 2245 1 220 -2 sum_2 2345 1 230 -2 sum_2 2445 1 240 -2 sum_2 2545 1 250 -2 sum_2 2645 1 260 -2 sum_2 2745 1 270 -2 sum_2 2845 1 280 -2 sum_2 2945 1 290 -2 sum_2 780 1 0 -2 sum_2 8950 1 40 -2 sum_2 18950 1 140 -2 sum_2 28950 1 240 -2 sum_2 38950 1 340 -2 sum_2 48950 1 440 -2 sum_2 58950 1 540 -2 sum_2 68950 1 640 -2 sum_2 78950 1 740 -2 sum_2 88950 1 840 -2 sum_2 98950 1 940 -2 sum_2 108950 1 1040 -2 sum_2 70170 1 1140 +1 max_1 2021-08-09 23:50:00 9 1 0 +1 max_1 2021-08-09 23:40:00 19 1 10 +1 max_1 2021-08-09 23:30:00 29 1 20 +1 max_1 2021-08-09 23:20:00 39 1 30 +1 max_1 2021-08-09 23:10:00 49 1 40 +1 max_1 2021-08-09 23:00:00 59 1 50 +1 max_1 2021-08-09 22:50:00 69 1 60 +1 max_1 2021-08-09 22:40:00 79 1 70 +1 max_1 2021-08-09 22:30:00 89 1 80 +1 max_1 2021-08-09 22:20:00 99 1 90 +1 max_1 2021-08-09 22:10:00 109 1 100 +1 max_1 2021-08-09 22:00:00 119 1 110 +1 max_1 2021-08-09 21:50:00 129 1 120 +1 max_1 2021-08-09 21:40:00 139 1 130 +1 max_1 2021-08-09 21:30:00 149 1 140 +1 max_1 2021-08-09 21:20:00 159 1 150 +1 max_1 2021-08-09 21:10:00 169 1 160 +1 max_1 2021-08-09 21:00:00 179 1 170 +1 max_1 2021-08-09 20:50:00 189 1 180 +1 max_1 2021-08-09 20:40:00 199 1 190 +1 max_1 2021-08-09 20:30:00 209 1 200 +1 max_1 2021-08-09 20:20:00 219 1 210 +1 max_1 2021-08-09 20:10:00 229 1 220 +1 max_1 2021-08-09 20:00:00 239 1 230 +1 max_1 2021-08-09 19:50:00 249 1 240 +1 max_1 2021-08-09 19:40:00 259 1 250 +1 max_1 2021-08-09 19:30:00 269 1 260 +1 max_1 2021-08-09 19:20:00 279 1 270 +1 max_1 2021-08-09 19:10:00 289 1 280 +1 max_1 2021-08-09 19:00:00 299 1 290 +1 max_1 2021-08-06 23:20:00 39 1 0 +1 max_1 2021-08-06 21:40:00 139 1 40 +1 max_1 2021-08-06 20:00:00 239 1 140 +1 max_1 2021-08-06 18:20:00 339 1 240 +1 max_1 2021-08-06 16:40:00 439 1 340 +1 max_1 2021-08-06 15:00:00 539 1 440 +1 max_1 2021-08-06 13:20:00 639 1 540 +1 max_1 2021-08-06 11:40:00 739 1 640 +1 max_1 2021-08-06 10:00:00 839 1 740 +1 max_1 2021-08-06 08:20:00 939 1 840 +1 max_1 2021-08-06 06:40:00 1039 1 940 +1 max_1 2021-08-06 05:00:00 1139 1 1040 +1 max_1 2021-08-06 03:20:00 1199 1 1140 +1 max_2 2021-08-09 23:50:00 9 1 0 +1 max_2 2021-08-09 23:40:00 19 1 10 +1 max_2 2021-08-09 23:30:00 29 1 20 +1 max_2 2021-08-09 23:20:00 39 1 30 +1 max_2 2021-08-09 23:10:00 49 1 40 +1 max_2 2021-08-09 23:00:00 59 1 50 +1 max_2 2021-08-09 22:50:00 69 1 60 +1 max_2 2021-08-09 22:40:00 79 1 70 +1 max_2 2021-08-09 22:30:00 89 1 80 +1 max_2 2021-08-09 22:20:00 99 1 90 +1 max_2 2021-08-09 22:10:00 109 1 100 +1 max_2 2021-08-09 22:00:00 119 1 110 +1 max_2 2021-08-09 21:50:00 129 1 120 +1 max_2 2021-08-09 21:40:00 139 1 130 +1 max_2 2021-08-09 21:30:00 149 1 140 +1 max_2 2021-08-09 21:20:00 159 1 150 +1 max_2 2021-08-09 21:10:00 169 1 160 +1 max_2 2021-08-09 21:00:00 179 1 170 +1 max_2 2021-08-09 20:50:00 189 1 180 +1 max_2 2021-08-09 20:40:00 199 1 190 +1 max_2 2021-08-09 20:30:00 209 1 200 +1 max_2 2021-08-09 20:20:00 219 1 210 +1 max_2 2021-08-09 20:10:00 229 1 220 +1 max_2 2021-08-09 20:00:00 239 1 230 +1 max_2 2021-08-09 19:50:00 249 1 240 +1 max_2 2021-08-09 19:40:00 259 1 250 +1 max_2 2021-08-09 19:30:00 269 1 260 +1 max_2 2021-08-09 19:20:00 279 1 270 +1 max_2 2021-08-09 19:10:00 289 1 280 +1 max_2 2021-08-09 19:00:00 299 1 290 +1 max_2 2021-08-06 23:20:00 39 1 0 +1 max_2 2021-08-06 21:40:00 139 1 40 +1 max_2 2021-08-06 20:00:00 239 1 140 +1 max_2 2021-08-06 18:20:00 339 1 240 +1 max_2 2021-08-06 16:40:00 439 1 340 +1 max_2 2021-08-06 15:00:00 539 1 440 +1 max_2 2021-08-06 13:20:00 639 1 540 +1 max_2 2021-08-06 11:40:00 739 1 640 +1 max_2 2021-08-06 10:00:00 839 1 740 +1 max_2 2021-08-06 08:20:00 939 1 840 +1 max_2 2021-08-06 06:40:00 1039 1 940 +1 max_2 2021-08-06 05:00:00 1139 1 1040 +1 max_2 2021-08-06 03:20:00 1199 1 1140 +1 sum_1 2021-08-09 23:50:00 45 1 0 +1 sum_1 2021-08-09 23:40:00 145 1 10 +1 sum_1 2021-08-09 23:30:00 245 1 20 +1 sum_1 2021-08-09 23:20:00 345 1 30 +1 sum_1 2021-08-09 23:10:00 445 1 40 +1 sum_1 2021-08-09 23:00:00 545 1 50 +1 sum_1 2021-08-09 22:50:00 645 1 60 +1 sum_1 2021-08-09 22:40:00 745 1 70 +1 sum_1 2021-08-09 22:30:00 845 1 80 +1 sum_1 2021-08-09 22:20:00 945 1 90 +1 sum_1 2021-08-09 22:10:00 1045 1 100 +1 sum_1 2021-08-09 22:00:00 1145 1 110 +1 sum_1 2021-08-09 21:50:00 1245 1 120 +1 sum_1 2021-08-09 21:40:00 1345 1 130 +1 sum_1 2021-08-09 21:30:00 1445 1 140 +1 sum_1 2021-08-09 21:20:00 1545 1 150 +1 sum_1 2021-08-09 21:10:00 1645 1 160 +1 sum_1 2021-08-09 21:00:00 1745 1 170 +1 sum_1 2021-08-09 20:50:00 1845 1 180 +1 sum_1 2021-08-09 20:40:00 1945 1 190 +1 sum_1 2021-08-09 20:30:00 2045 1 200 +1 sum_1 2021-08-09 20:20:00 2145 1 210 +1 sum_1 2021-08-09 20:10:00 2245 1 220 +1 sum_1 2021-08-09 20:00:00 2345 1 230 +1 sum_1 2021-08-09 19:50:00 2445 1 240 +1 sum_1 2021-08-09 19:40:00 2545 1 250 +1 sum_1 2021-08-09 19:30:00 2645 1 260 +1 sum_1 2021-08-09 19:20:00 2745 1 270 +1 sum_1 2021-08-09 19:10:00 2845 1 280 +1 sum_1 2021-08-09 19:00:00 2945 1 290 +1 sum_1 2021-08-06 23:20:00 780 1 0 +1 sum_1 2021-08-06 21:40:00 8950 1 40 +1 sum_1 2021-08-06 20:00:00 18950 1 140 +1 sum_1 2021-08-06 18:20:00 28950 1 240 +1 sum_1 2021-08-06 16:40:00 38950 1 340 +1 sum_1 2021-08-06 15:00:00 48950 1 440 +1 sum_1 2021-08-06 13:20:00 58950 1 540 +1 sum_1 2021-08-06 11:40:00 68950 1 640 +1 sum_1 2021-08-06 10:00:00 78950 1 740 +1 sum_1 2021-08-06 08:20:00 88950 1 840 +1 sum_1 2021-08-06 06:40:00 98950 1 940 +1 sum_1 2021-08-06 05:00:00 108950 1 1040 +1 sum_1 2021-08-06 03:20:00 70170 1 1140 +1 sum_2 2021-08-09 23:50:00 45 1 0 +1 sum_2 2021-08-09 23:40:00 145 1 10 +1 sum_2 2021-08-09 23:30:00 245 1 20 +1 sum_2 2021-08-09 23:20:00 345 1 30 +1 sum_2 2021-08-09 23:10:00 445 1 40 +1 sum_2 2021-08-09 23:00:00 545 1 50 +1 sum_2 2021-08-09 22:50:00 645 1 60 +1 sum_2 2021-08-09 22:40:00 745 1 70 +1 sum_2 2021-08-09 22:30:00 845 1 80 +1 sum_2 2021-08-09 22:20:00 945 1 90 +1 sum_2 2021-08-09 22:10:00 1045 1 100 +1 sum_2 2021-08-09 22:00:00 1145 1 110 +1 sum_2 2021-08-09 21:50:00 1245 1 120 +1 sum_2 2021-08-09 21:40:00 1345 1 130 +1 sum_2 2021-08-09 21:30:00 1445 1 140 +1 sum_2 2021-08-09 21:20:00 1545 1 150 +1 sum_2 2021-08-09 21:10:00 1645 1 160 +1 sum_2 2021-08-09 21:00:00 1745 1 170 +1 sum_2 2021-08-09 20:50:00 1845 1 180 +1 sum_2 2021-08-09 20:40:00 1945 1 190 +1 sum_2 2021-08-09 20:30:00 2045 1 200 +1 sum_2 2021-08-09 20:20:00 2145 1 210 +1 sum_2 2021-08-09 20:10:00 2245 1 220 +1 sum_2 2021-08-09 20:00:00 2345 1 230 +1 sum_2 2021-08-09 19:50:00 2445 1 240 +1 sum_2 2021-08-09 19:40:00 2545 1 250 +1 sum_2 2021-08-09 19:30:00 2645 1 260 +1 sum_2 2021-08-09 19:20:00 2745 1 270 +1 sum_2 2021-08-09 19:10:00 2845 1 280 +1 sum_2 2021-08-09 19:00:00 2945 1 290 +1 sum_2 2021-08-06 23:20:00 780 1 0 +1 sum_2 2021-08-06 21:40:00 8950 1 40 +1 sum_2 2021-08-06 20:00:00 18950 1 140 +1 sum_2 2021-08-06 18:20:00 28950 1 240 +1 sum_2 2021-08-06 16:40:00 38950 1 340 +1 sum_2 2021-08-06 15:00:00 48950 1 440 +1 sum_2 2021-08-06 13:20:00 58950 1 540 +1 sum_2 2021-08-06 11:40:00 68950 1 640 +1 sum_2 2021-08-06 10:00:00 78950 1 740 +1 sum_2 2021-08-06 08:20:00 88950 1 840 +1 sum_2 2021-08-06 06:40:00 98950 1 940 +1 sum_2 2021-08-06 05:00:00 108950 1 1040 +1 sum_2 2021-08-06 03:20:00 70170 1 1140 +2 max_1 2021-08-09 23:50:00 9 1 0 +2 max_1 2021-08-09 23:40:00 19 1 10 +2 max_1 2021-08-09 23:30:00 29 1 20 +2 max_1 2021-08-09 23:20:00 39 1 30 +2 max_1 2021-08-09 23:10:00 49 1 40 +2 max_1 2021-08-09 23:00:00 59 1 50 +2 max_1 2021-08-09 22:50:00 69 1 60 +2 max_1 2021-08-09 22:40:00 79 1 70 +2 max_1 2021-08-09 22:30:00 89 1 80 +2 max_1 2021-08-09 22:20:00 99 1 90 +2 max_1 2021-08-09 22:10:00 109 1 100 +2 max_1 2021-08-09 22:00:00 119 1 110 +2 max_1 2021-08-09 21:50:00 129 1 120 +2 max_1 2021-08-09 21:40:00 139 1 130 +2 max_1 2021-08-09 21:30:00 149 1 140 +2 max_1 2021-08-09 21:20:00 159 1 150 +2 max_1 2021-08-09 21:10:00 169 1 160 +2 max_1 2021-08-09 21:00:00 179 1 170 +2 max_1 2021-08-09 20:50:00 189 1 180 +2 max_1 2021-08-09 20:40:00 199 1 190 +2 max_1 2021-08-09 20:30:00 209 1 200 +2 max_1 2021-08-09 20:20:00 219 1 210 +2 max_1 2021-08-09 20:10:00 229 1 220 +2 max_1 2021-08-09 20:00:00 239 1 230 +2 max_1 2021-08-09 19:50:00 249 1 240 +2 max_1 2021-08-09 19:40:00 259 1 250 +2 max_1 2021-08-09 19:30:00 269 1 260 +2 max_1 2021-08-09 19:20:00 279 1 270 +2 max_1 2021-08-09 19:10:00 289 1 280 +2 max_1 2021-08-09 19:00:00 299 1 290 +2 max_1 2021-08-06 23:20:00 39 1 0 +2 max_1 2021-08-06 21:40:00 139 1 40 +2 max_1 2021-08-06 20:00:00 239 1 140 +2 max_1 2021-08-06 18:20:00 339 1 240 +2 max_1 2021-08-06 16:40:00 439 1 340 +2 max_1 2021-08-06 15:00:00 539 1 440 +2 max_1 2021-08-06 13:20:00 639 1 540 +2 max_1 2021-08-06 11:40:00 739 1 640 +2 max_1 2021-08-06 10:00:00 839 1 740 +2 max_1 2021-08-06 08:20:00 939 1 840 +2 max_1 2021-08-06 06:40:00 1039 1 940 +2 max_1 2021-08-06 05:00:00 1139 1 1040 +2 max_1 2021-08-06 03:20:00 1199 1 1140 +2 max_2 2021-08-09 23:50:00 9 1 0 +2 max_2 2021-08-09 23:40:00 19 1 10 +2 max_2 2021-08-09 23:30:00 29 1 20 +2 max_2 2021-08-09 23:20:00 39 1 30 +2 max_2 2021-08-09 23:10:00 49 1 40 +2 max_2 2021-08-09 23:00:00 59 1 50 +2 max_2 2021-08-09 22:50:00 69 1 60 +2 max_2 2021-08-09 22:40:00 79 1 70 +2 max_2 2021-08-09 22:30:00 89 1 80 +2 max_2 2021-08-09 22:20:00 99 1 90 +2 max_2 2021-08-09 22:10:00 109 1 100 +2 max_2 2021-08-09 22:00:00 119 1 110 +2 max_2 2021-08-09 21:50:00 129 1 120 +2 max_2 2021-08-09 21:40:00 139 1 130 +2 max_2 2021-08-09 21:30:00 149 1 140 +2 max_2 2021-08-09 21:20:00 159 1 150 +2 max_2 2021-08-09 21:10:00 169 1 160 +2 max_2 2021-08-09 21:00:00 179 1 170 +2 max_2 2021-08-09 20:50:00 189 1 180 +2 max_2 2021-08-09 20:40:00 199 1 190 +2 max_2 2021-08-09 20:30:00 209 1 200 +2 max_2 2021-08-09 20:20:00 219 1 210 +2 max_2 2021-08-09 20:10:00 229 1 220 +2 max_2 2021-08-09 20:00:00 239 1 230 +2 max_2 2021-08-09 19:50:00 249 1 240 +2 max_2 2021-08-09 19:40:00 259 1 250 +2 max_2 2021-08-09 19:30:00 269 1 260 +2 max_2 2021-08-09 19:20:00 279 1 270 +2 max_2 2021-08-09 19:10:00 289 1 280 +2 max_2 2021-08-09 19:00:00 299 1 290 +2 max_2 2021-08-06 23:20:00 39 1 0 +2 max_2 2021-08-06 21:40:00 139 1 40 +2 max_2 2021-08-06 20:00:00 239 1 140 +2 max_2 2021-08-06 18:20:00 339 1 240 +2 max_2 2021-08-06 16:40:00 439 1 340 +2 max_2 2021-08-06 15:00:00 539 1 440 +2 max_2 2021-08-06 13:20:00 639 1 540 +2 max_2 2021-08-06 11:40:00 739 1 640 +2 max_2 2021-08-06 10:00:00 839 1 740 +2 max_2 2021-08-06 08:20:00 939 1 840 +2 max_2 2021-08-06 06:40:00 1039 1 940 +2 max_2 2021-08-06 05:00:00 1139 1 1040 +2 max_2 2021-08-06 03:20:00 1199 1 1140 +2 sum_1 2021-08-09 23:50:00 45 1 0 +2 sum_1 2021-08-09 23:40:00 145 1 10 +2 sum_1 2021-08-09 23:30:00 245 1 20 +2 sum_1 2021-08-09 23:20:00 345 1 30 +2 sum_1 2021-08-09 23:10:00 445 1 40 +2 sum_1 2021-08-09 23:00:00 545 1 50 +2 sum_1 2021-08-09 22:50:00 645 1 60 +2 sum_1 2021-08-09 22:40:00 745 1 70 +2 sum_1 2021-08-09 22:30:00 845 1 80 +2 sum_1 2021-08-09 22:20:00 945 1 90 +2 sum_1 2021-08-09 22:10:00 1045 1 100 +2 sum_1 2021-08-09 22:00:00 1145 1 110 +2 sum_1 2021-08-09 21:50:00 1245 1 120 +2 sum_1 2021-08-09 21:40:00 1345 1 130 +2 sum_1 2021-08-09 21:30:00 1445 1 140 +2 sum_1 2021-08-09 21:20:00 1545 1 150 +2 sum_1 2021-08-09 21:10:00 1645 1 160 +2 sum_1 2021-08-09 21:00:00 1745 1 170 +2 sum_1 2021-08-09 20:50:00 1845 1 180 +2 sum_1 2021-08-09 20:40:00 1945 1 190 +2 sum_1 2021-08-09 20:30:00 2045 1 200 +2 sum_1 2021-08-09 20:20:00 2145 1 210 +2 sum_1 2021-08-09 20:10:00 2245 1 220 +2 sum_1 2021-08-09 20:00:00 2345 1 230 +2 sum_1 2021-08-09 19:50:00 2445 1 240 +2 sum_1 2021-08-09 19:40:00 2545 1 250 +2 sum_1 2021-08-09 19:30:00 2645 1 260 +2 sum_1 2021-08-09 19:20:00 2745 1 270 +2 sum_1 2021-08-09 19:10:00 2845 1 280 +2 sum_1 2021-08-09 19:00:00 2945 1 290 +2 sum_1 2021-08-06 23:20:00 780 1 0 +2 sum_1 2021-08-06 21:40:00 8950 1 40 +2 sum_1 2021-08-06 20:00:00 18950 1 140 +2 sum_1 2021-08-06 18:20:00 28950 1 240 +2 sum_1 2021-08-06 16:40:00 38950 1 340 +2 sum_1 2021-08-06 15:00:00 48950 1 440 +2 sum_1 2021-08-06 13:20:00 58950 1 540 +2 sum_1 2021-08-06 11:40:00 68950 1 640 +2 sum_1 2021-08-06 10:00:00 78950 1 740 +2 sum_1 2021-08-06 08:20:00 88950 1 840 +2 sum_1 2021-08-06 06:40:00 98950 1 940 +2 sum_1 2021-08-06 05:00:00 108950 1 1040 +2 sum_1 2021-08-06 03:20:00 70170 1 1140 +2 sum_2 2021-08-09 23:50:00 45 1 0 +2 sum_2 2021-08-09 23:40:00 145 1 10 +2 sum_2 2021-08-09 23:30:00 245 1 20 +2 sum_2 2021-08-09 23:20:00 345 1 30 +2 sum_2 2021-08-09 23:10:00 445 1 40 +2 sum_2 2021-08-09 23:00:00 545 1 50 +2 sum_2 2021-08-09 22:50:00 645 1 60 +2 sum_2 2021-08-09 22:40:00 745 1 70 +2 sum_2 2021-08-09 22:30:00 845 1 80 +2 sum_2 2021-08-09 22:20:00 945 1 90 +2 sum_2 2021-08-09 22:10:00 1045 1 100 +2 sum_2 2021-08-09 22:00:00 1145 1 110 +2 sum_2 2021-08-09 21:50:00 1245 1 120 +2 sum_2 2021-08-09 21:40:00 1345 1 130 +2 sum_2 2021-08-09 21:30:00 1445 1 140 +2 sum_2 2021-08-09 21:20:00 1545 1 150 +2 sum_2 2021-08-09 21:10:00 1645 1 160 +2 sum_2 2021-08-09 21:00:00 1745 1 170 +2 sum_2 2021-08-09 20:50:00 1845 1 180 +2 sum_2 2021-08-09 20:40:00 1945 1 190 +2 sum_2 2021-08-09 20:30:00 2045 1 200 +2 sum_2 2021-08-09 20:20:00 2145 1 210 +2 sum_2 2021-08-09 20:10:00 2245 1 220 +2 sum_2 2021-08-09 20:00:00 2345 1 230 +2 sum_2 2021-08-09 19:50:00 2445 1 240 +2 sum_2 2021-08-09 19:40:00 2545 1 250 +2 sum_2 2021-08-09 19:30:00 2645 1 260 +2 sum_2 2021-08-09 19:20:00 2745 1 270 +2 sum_2 2021-08-09 19:10:00 2845 1 280 +2 sum_2 2021-08-09 19:00:00 2945 1 290 +2 sum_2 2021-08-06 23:20:00 780 1 0 +2 sum_2 2021-08-06 21:40:00 8950 1 40 +2 sum_2 2021-08-06 20:00:00 18950 1 140 +2 sum_2 2021-08-06 18:20:00 28950 1 240 +2 sum_2 2021-08-06 16:40:00 38950 1 340 +2 sum_2 2021-08-06 15:00:00 48950 1 440 +2 sum_2 2021-08-06 13:20:00 58950 1 540 +2 sum_2 2021-08-06 11:40:00 68950 1 640 +2 sum_2 2021-08-06 10:00:00 78950 1 740 +2 sum_2 2021-08-06 08:20:00 88950 1 840 +2 sum_2 2021-08-06 06:40:00 98950 1 940 +2 sum_2 2021-08-06 05:00:00 108950 1 1040 +2 sum_2 2021-08-06 03:20:00 70170 1 1140 diff --git a/tests/queries/0_stateless/01236_graphite_mt.sql b/tests/queries/0_stateless/01236_graphite_mt.sql index 88d2d0ccb63..6f4af186a1c 100644 --- a/tests/queries/0_stateless/01236_graphite_mt.sql +++ b/tests/queries/0_stateless/01236_graphite_mt.sql @@ -1,29 +1,31 @@ + +-- Use DateTime('UTC') to have a common rollup window drop table if exists test_graphite; -create table test_graphite (key UInt32, Path String, Time DateTime, Value Float64, Version UInt32, col UInt64) +create table test_graphite (key UInt32, Path String, Time DateTime('UTC'), Value Float64, Version UInt32, col UInt64) engine = GraphiteMergeTree('graphite_rollup') order by key settings index_granularity=10; insert into test_graphite -select 1, 'sum_1', toDateTime(today()) - number * 60 - 30, number, 1, number from numbers(300) union all -select 2, 'sum_1', toDateTime(today()) - number * 60 - 30, number, 1, number from numbers(300) union all -select 1, 'sum_2', toDateTime(today()) - number * 60 - 30, number, 1, number from numbers(300) union all -select 2, 'sum_2', toDateTime(today()) - number * 60 - 30, number, 1, number from numbers(300) union all -select 1, 'max_1', toDateTime(today()) - number * 60 - 30, number, 1, number from numbers(300) union all -select 2, 'max_1', toDateTime(today()) - number * 60 - 30, number, 1, number from numbers(300) union all -select 1, 'max_2', toDateTime(today()) - number * 60 - 30, number, 1, number from numbers(300) union all -select 2, 'max_2', toDateTime(today()) - number * 60 - 30, number, 1, number from numbers(300); +select 1, 'sum_1', toDateTime('2021-08-10', 'UTC') - number * 60 - 30, number, 1, number from numbers(300) union all +select 2, 'sum_1', toDateTime('2021-08-10', 'UTC') - number * 60 - 30, number, 1, number from numbers(300) union all +select 1, 'sum_2', toDateTime('2021-08-10', 'UTC') - number * 60 - 30, number, 1, number from numbers(300) union all +select 2, 'sum_2', toDateTime('2021-08-10', 'UTC') - number * 60 - 30, number, 1, number from numbers(300) union all +select 1, 'max_1', toDateTime('2021-08-10', 'UTC') - number * 60 - 30, number, 1, number from numbers(300) union all +select 2, 'max_1', toDateTime('2021-08-10', 'UTC') - number * 60 - 30, number, 1, number from numbers(300) union all +select 1, 'max_2', toDateTime('2021-08-10', 'UTC') - number * 60 - 30, number, 1, number from numbers(300) union all +select 2, 'max_2', toDateTime('2021-08-10', 'UTC') - number * 60 - 30, number, 1, number from numbers(300); insert into test_graphite -select 1, 'sum_1', toDateTime(today() - 3) - number * 60 - 30, number, 1, number from numbers(1200) union all -select 2, 'sum_1', toDateTime(today() - 3) - number * 60 - 30, number, 1, number from numbers(1200) union all -select 1, 'sum_2', toDateTime(today() - 3) - number * 60 - 30, number, 1, number from numbers(1200) union all -select 2, 'sum_2', toDateTime(today() - 3) - number * 60 - 30, number, 1, number from numbers(1200) union all -select 1, 'max_1', toDateTime(today() - 3) - number * 60 - 30, number, 1, number from numbers(1200) union all -select 2, 'max_1', toDateTime(today() - 3) - number * 60 - 30, number, 1, number from numbers(1200) union all -select 1, 'max_2', toDateTime(today() - 3) - number * 60 - 30, number, 1, number from numbers(1200) union all -select 2, 'max_2', toDateTime(today() - 3) - number * 60 - 30, number, 1, number from numbers(1200); +select 1, 'sum_1', toDateTime('2021-08-07', 'UTC') - number * 60 - 30, number, 1, number from numbers(1200) union all +select 2, 'sum_1', toDateTime('2021-08-07', 'UTC') - number * 60 - 30, number, 1, number from numbers(1200) union all +select 1, 'sum_2', toDateTime('2021-08-07', 'UTC') - number * 60 - 30, number, 1, number from numbers(1200) union all +select 2, 'sum_2', toDateTime('2021-08-07', 'UTC') - number * 60 - 30, number, 1, number from numbers(1200) union all +select 1, 'max_1', toDateTime('2021-08-07', 'UTC') - number * 60 - 30, number, 1, number from numbers(1200) union all +select 2, 'max_1', toDateTime('2021-08-07', 'UTC') - number * 60 - 30, number, 1, number from numbers(1200) union all +select 1, 'max_2', toDateTime('2021-08-07', 'UTC') - number * 60 - 30, number, 1, number from numbers(1200) union all +select 2, 'max_2', toDateTime('2021-08-07', 'UTC') - number * 60 - 30, number, 1, number from numbers(1200); optimize table test_graphite final; -select key, Path, Value, Version, col from test_graphite order by key, Path, Time desc; +select key, Path, Time, Value, Version, col from test_graphite order by key, Path, Time desc; drop table test_graphite; From 32e4a9a5a5b5d34d215326d6f2bb78bf119b00a9 Mon Sep 17 00:00:00 2001 From: tavplubix Date: Tue, 10 Aug 2021 20:58:36 +0300 Subject: [PATCH 419/599] Update ci-runner.py --- tests/integration/ci-runner.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/ci-runner.py b/tests/integration/ci-runner.py index ecd4cb8d4e7..7aecf2953f8 100755 --- a/tests/integration/ci-runner.py +++ b/tests/integration/ci-runner.py @@ -264,7 +264,7 @@ class ClickhouseIntegrationTestsRunner: out_file = "all_tests.txt" out_file_full = "all_tests_full.txt" cmd = "cd {repo_path}/tests/integration && " \ - "./runner --tmpfs {image_cmd} ' --setup-plan' " \ + "timeout 1h ./runner --tmpfs {image_cmd} ' --setup-plan' " \ "| tee {out_file_full} | grep '::' | sed 's/ (fixtures used:.*//g' | sed 's/^ *//g' | sed 's/ *$//g' " \ "| grep -v 'SKIPPED' | sort -u > {out_file}".format( repo_path=repo_path, image_cmd=image_cmd, out_file=out_file, out_file_full=out_file_full) From 68c8a168b143d2b320e098f6aa1d7cd7a1fa69b4 Mon Sep 17 00:00:00 2001 From: Braulio Valdivielso Date: Tue, 10 Aug 2021 21:33:24 +0100 Subject: [PATCH 420/599] change name to function to "tupleToNameValuePairs" --- src/Functions/registerFunctionsTuple.cpp | 4 ++-- ...medTupleItems.cpp => tupleToNameValuePairs.cpp} | 10 +++++----- ...e => 02007_tuple_to_name_value_pairs.reference} | 0 ...ems.sql => 02007_tuple_to_name_value_pairs.sql} | 14 +++++++------- 4 files changed, 14 insertions(+), 14 deletions(-) rename src/Functions/{namedTupleItems.cpp => tupleToNameValuePairs.cpp} (93%) rename tests/queries/0_stateless/{02007_namedtupleitems.reference => 02007_tuple_to_name_value_pairs.reference} (100%) rename tests/queries/0_stateless/{02007_namedtupleitems.sql => 02007_tuple_to_name_value_pairs.sql} (59%) diff --git a/src/Functions/registerFunctionsTuple.cpp b/src/Functions/registerFunctionsTuple.cpp index 21d2dd3cc0f..33f078675e9 100644 --- a/src/Functions/registerFunctionsTuple.cpp +++ b/src/Functions/registerFunctionsTuple.cpp @@ -5,13 +5,13 @@ class FunctionFactory; void registerFunctionTuple(FunctionFactory &); void registerFunctionTupleElement(FunctionFactory &); -void registerFunctionNamedTupleItems(FunctionFactory &); +void registerFunctionTupleToNameValuePairs(FunctionFactory &); void registerFunctionsTuple(FunctionFactory & factory) { registerFunctionTuple(factory); registerFunctionTupleElement(factory); - registerFunctionNamedTupleItems(factory); + registerFunctionTupleToNameValuePairs(factory); } } diff --git a/src/Functions/namedTupleItems.cpp b/src/Functions/tupleToNameValuePairs.cpp similarity index 93% rename from src/Functions/namedTupleItems.cpp rename to src/Functions/tupleToNameValuePairs.cpp index 1fd1397a1ef..a745800f42e 100644 --- a/src/Functions/namedTupleItems.cpp +++ b/src/Functions/tupleToNameValuePairs.cpp @@ -26,13 +26,13 @@ namespace /** Extract element of tuple by constant index or name. The operation is essentially free. * Also the function looks through Arrays: you can get Array of tuple elements from Array of Tuples. */ -class FunctionNamedTupleItems : public IFunction +class FunctionTupleToNameValuePairs : public IFunction { public: - static constexpr auto name = "namedTupleItems"; + static constexpr auto name = "tupleToNameValuePairs"; static FunctionPtr create(ContextPtr) { - return std::make_shared(); + return std::make_shared(); } String getName() const override @@ -122,9 +122,9 @@ public: } -void registerFunctionNamedTupleItems(FunctionFactory & factory) +void registerFunctionTupleToNameValuePairs(FunctionFactory & factory) { - factory.registerFunction(); + factory.registerFunction(); } } diff --git a/tests/queries/0_stateless/02007_namedtupleitems.reference b/tests/queries/0_stateless/02007_tuple_to_name_value_pairs.reference similarity index 100% rename from tests/queries/0_stateless/02007_namedtupleitems.reference rename to tests/queries/0_stateless/02007_tuple_to_name_value_pairs.reference diff --git a/tests/queries/0_stateless/02007_namedtupleitems.sql b/tests/queries/0_stateless/02007_tuple_to_name_value_pairs.sql similarity index 59% rename from tests/queries/0_stateless/02007_namedtupleitems.sql rename to tests/queries/0_stateless/02007_tuple_to_name_value_pairs.sql index 02c6e57a0fb..17fc5b85b28 100644 --- a/tests/queries/0_stateless/02007_namedtupleitems.sql +++ b/tests/queries/0_stateless/02007_tuple_to_name_value_pairs.sql @@ -1,4 +1,4 @@ -SELECT namedTupleItems(tuple(1, 2, 3)); +SELECT tupleToNameValuePairs(tuple(1, 2, 3)); -- [('1',1),('2',2),('3',3)] DROP TABLE IF EXISTS test02007; @@ -10,7 +10,7 @@ CREATE TABLE test02007 ( ) ENGINE=Memory(); INSERT INTO test02007 VALUES (tuple(tuple(1, 2), tuple(3, 4))); INSERT INTO test02007 VALUES (tuple(tuple(5, 6), tuple(7, 8))); -SELECT namedTupleItems(col) FROM test02007 ORDER BY col; +SELECT tupleToNameValuePairs(col) FROM test02007 ORDER BY col; -- [('a',(1,2)),('b',(3,4))] -- [('a',(5,6)),('b',(7,8))] @@ -19,15 +19,15 @@ CREATE TABLE test02007 ( col Tuple(CPU double, Memory double, Disk double) ) ENGINE=Memory(); INSERT INTO test02007 VALUES (tuple(3.3, 5.5, 6.6)); -SELECT untuple(arrayJoin(namedTupleItems(col))) from test02007; +SELECT untuple(arrayJoin(tupleToNameValuePairs(col))) from test02007; -- CPU 3.3 -- Memory 5.5 -- Disk 6.6 DROP TABLE IF EXISTS test02007; -SELECT namedTupleItems(tuple(1, 1.3)); -- { serverError 43; } should it? -SELECT namedTupleItems(tuple(1, [1,2])); -- { serverError 43; } -SELECT namedTupleItems(tuple(1, 'a')); -- { serverError 43; } -SELECT namedTupleItems(33); -- { serverError 43; } +SELECT tupleToNameValuePairs(tuple(1, 1.3)); -- { serverError 43; } should it? +SELECT tupleToNameValuePairs(tuple(1, [1,2])); -- { serverError 43; } +SELECT tupleToNameValuePairs(tuple(1, 'a')); -- { serverError 43; } +SELECT tupleToNameValuePairs(33); -- { serverError 43; } From a880b8a7a1a074d9c3ef21fff5f4517e2d77fba8 Mon Sep 17 00:00:00 2001 From: Braulio Valdivielso Date: Tue, 10 Aug 2021 21:47:56 +0100 Subject: [PATCH 421/599] fixup! clean up and add more tests --- src/Functions/tupleToNameValuePairs.cpp | 48 ++++++++++--------- .../02007_tuple_to_name_value_pairs.sql | 8 ---- 2 files changed, 25 insertions(+), 31 deletions(-) diff --git a/src/Functions/tupleToNameValuePairs.cpp b/src/Functions/tupleToNameValuePairs.cpp index a745800f42e..d0cea2fc0d4 100644 --- a/src/Functions/tupleToNameValuePairs.cpp +++ b/src/Functions/tupleToNameValuePairs.cpp @@ -23,8 +23,9 @@ namespace ErrorCodes namespace { -/** Extract element of tuple by constant index or name. The operation is essentially free. - * Also the function looks through Arrays: you can get Array of tuple elements from Array of Tuples. +/** Transform a named tuple into an array of pairs, where the first element + * of the pair corresponds to the tuple field name and the second one to the + * tuple value. */ class FunctionTupleToNameValuePairs : public IFunction { @@ -57,30 +58,31 @@ public: const DataTypeTuple * tuple = checkAndGetDataType(col); if (!tuple) - throw Exception("First argument for function " + getName() + " must " - "be a tuple.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "First argument for function {} must be a tuple.", + getName()); - const auto& elementTypes = tuple->getElements(); + const auto & elementTypes = tuple->getElements(); if (elementTypes.empty()) - throw Exception("The argument tuple for function " + getName() + " must " - "not be empty.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "The argument tuple for function {} must not be empty.", + getName()); - const auto& firstElementType = elementTypes[0]; + const auto & firstElementType = elementTypes[0]; - auto it = std::find_if( - elementTypes.begin() + 1, - elementTypes.end(), - [&](const auto &other) - { - return !firstElementType->equals(*other); - }); + bool allValueTypesEqual = std::all_of(elementTypes.begin() + 1, + elementTypes.end(), + [&](const auto &other) + { + return firstElementType->equals(*other); + }); - if (it != elementTypes.end()) + if (!allValueTypesEqual) { - throw Exception("The argument tuple for function " + getName() + " must " - "contain just one type", - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "The argument tuple for function {} must contain just one type.", + getName()); } DataTypePtr tupleNameType = std::make_shared(); @@ -94,9 +96,9 @@ public: ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/) const override { - const IColumn *tuple_col = arguments[0].column.get(); + const IColumn * tuple_col = arguments[0].column.get(); const DataTypeTuple * tuple = checkAndGetDataType(arguments[0].type.get()); - auto *tuple_col_concrete = assert_cast(tuple_col); + auto * tuple_col_concrete = assert_cast(tuple_col); MutableColumnPtr keys = ColumnString::create(); MutableColumnPtr values = tuple_col_concrete->getColumn(0).cloneEmpty(); @@ -105,8 +107,8 @@ public: { for (size_t col = 0; col < tuple_col_concrete->tupleSize(); ++col) { - const std::string& key = tuple->getElementNames()[col]; - const IColumn& valueColumn = tuple_col_concrete->getColumn(col); + const std::string & key = tuple->getElementNames()[col]; + const IColumn & valueColumn = tuple_col_concrete->getColumn(col); values->insertFrom(valueColumn, row); keys->insertData(key.data(), key.size()); diff --git a/tests/queries/0_stateless/02007_tuple_to_name_value_pairs.sql b/tests/queries/0_stateless/02007_tuple_to_name_value_pairs.sql index 17fc5b85b28..f1d2b46a983 100644 --- a/tests/queries/0_stateless/02007_tuple_to_name_value_pairs.sql +++ b/tests/queries/0_stateless/02007_tuple_to_name_value_pairs.sql @@ -1,5 +1,4 @@ SELECT tupleToNameValuePairs(tuple(1, 2, 3)); --- [('1',1),('2',2),('3',3)] DROP TABLE IF EXISTS test02007; CREATE TABLE test02007 ( @@ -11,8 +10,6 @@ CREATE TABLE test02007 ( INSERT INTO test02007 VALUES (tuple(tuple(1, 2), tuple(3, 4))); INSERT INTO test02007 VALUES (tuple(tuple(5, 6), tuple(7, 8))); SELECT tupleToNameValuePairs(col) FROM test02007 ORDER BY col; --- [('a',(1,2)),('b',(3,4))] --- [('a',(5,6)),('b',(7,8))] DROP TABLE IF EXISTS test02007; CREATE TABLE test02007 ( @@ -20,14 +17,9 @@ CREATE TABLE test02007 ( ) ENGINE=Memory(); INSERT INTO test02007 VALUES (tuple(3.3, 5.5, 6.6)); SELECT untuple(arrayJoin(tupleToNameValuePairs(col))) from test02007; --- CPU 3.3 --- Memory 5.5 --- Disk 6.6 DROP TABLE IF EXISTS test02007; SELECT tupleToNameValuePairs(tuple(1, 1.3)); -- { serverError 43; } should it? SELECT tupleToNameValuePairs(tuple(1, [1,2])); -- { serverError 43; } SELECT tupleToNameValuePairs(tuple(1, 'a')); -- { serverError 43; } SELECT tupleToNameValuePairs(33); -- { serverError 43; } - - From 4d2af9c40415eaa2bebfcca679905ef97961bdca Mon Sep 17 00:00:00 2001 From: Braulio Valdivielso Date: Tue, 10 Aug 2021 23:19:50 +0100 Subject: [PATCH 422/599] change type annotation to aid devirtualization --- src/Functions/tupleToNameValuePairs.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/tupleToNameValuePairs.cpp b/src/Functions/tupleToNameValuePairs.cpp index d0cea2fc0d4..05938955313 100644 --- a/src/Functions/tupleToNameValuePairs.cpp +++ b/src/Functions/tupleToNameValuePairs.cpp @@ -100,7 +100,7 @@ public: const DataTypeTuple * tuple = checkAndGetDataType(arguments[0].type.get()); auto * tuple_col_concrete = assert_cast(tuple_col); - MutableColumnPtr keys = ColumnString::create(); + auto keys = ColumnString::create(); MutableColumnPtr values = tuple_col_concrete->getColumn(0).cloneEmpty(); auto offsets = ColumnVector::create(); for (size_t row = 0; row < tuple_col_concrete->size(); ++row) From 65920b26bde69eb93270317d0a04d812ebdb7c29 Mon Sep 17 00:00:00 2001 From: Braulio Valdivielso Date: Tue, 10 Aug 2021 23:20:15 +0100 Subject: [PATCH 423/599] remove unnecessary comment The `tupleToNameValuePairs` function is not expected to cast types of values in a tuple to the least common super type. Users will have to be explicit about casting in their queries --- tests/queries/0_stateless/02007_tuple_to_name_value_pairs.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02007_tuple_to_name_value_pairs.sql b/tests/queries/0_stateless/02007_tuple_to_name_value_pairs.sql index f1d2b46a983..e80df4ebf28 100644 --- a/tests/queries/0_stateless/02007_tuple_to_name_value_pairs.sql +++ b/tests/queries/0_stateless/02007_tuple_to_name_value_pairs.sql @@ -19,7 +19,7 @@ INSERT INTO test02007 VALUES (tuple(3.3, 5.5, 6.6)); SELECT untuple(arrayJoin(tupleToNameValuePairs(col))) from test02007; DROP TABLE IF EXISTS test02007; -SELECT tupleToNameValuePairs(tuple(1, 1.3)); -- { serverError 43; } should it? +SELECT tupleToNameValuePairs(tuple(1, 1.3)); -- { serverError 43; } SELECT tupleToNameValuePairs(tuple(1, [1,2])); -- { serverError 43; } SELECT tupleToNameValuePairs(tuple(1, 'a')); -- { serverError 43; } SELECT tupleToNameValuePairs(33); -- { serverError 43; } From a0a5c0da3256f4e81e2ee19ea1c9c774b638cb5c Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Wed, 11 Aug 2021 06:08:30 +0300 Subject: [PATCH 424/599] Update BackgroundJobsExecutor.cpp --- src/Storages/MergeTree/BackgroundJobsExecutor.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Storages/MergeTree/BackgroundJobsExecutor.cpp b/src/Storages/MergeTree/BackgroundJobsExecutor.cpp index e4b96e55c87..9ff7dd60266 100644 --- a/src/Storages/MergeTree/BackgroundJobsExecutor.cpp +++ b/src/Storages/MergeTree/BackgroundJobsExecutor.cpp @@ -146,6 +146,9 @@ try catch (...) /// Exception while we looking for a task, reschedule { tryLogCurrentException(__PRETTY_FUNCTION__); + + /// Why do we scheduleTask again? + /// To retry on exception, since it may be some temporary exception. scheduleTask(/* with_backoff = */ true); } From 6ac4ad2920af7e5f73cadb81ca6cc1d47a7cf805 Mon Sep 17 00:00:00 2001 From: abel-wang Date: Wed, 11 Aug 2021 11:19:46 +0800 Subject: [PATCH 425/599] add tests --- .../02006_use_constants_in_with_and_select.reference | 5 +++++ 1 file changed, 5 insertions(+) create mode 100644 tests/queries/0_stateless/02006_use_constants_in_with_and_select.reference diff --git a/tests/queries/0_stateless/02006_use_constants_in_with_and_select.reference b/tests/queries/0_stateless/02006_use_constants_in_with_and_select.reference new file mode 100644 index 00000000000..ab7d5dd61aa --- /dev/null +++ b/tests/queries/0_stateless/02006_use_constants_in_with_and_select.reference @@ -0,0 +1,5 @@ +1 [1] +1 [1] +99.9 +0.1 99.9 +[99.9] From b5c1ee9a8a80ae2e29e7c48a6377d8d7f00f2b6c Mon Sep 17 00:00:00 2001 From: abel-wang Date: Wed, 11 Aug 2021 11:32:44 +0800 Subject: [PATCH 426/599] add tests --- ...02006_use_constants_in_with_and_select.sql | 36 +++++++++++++++++++ 1 file changed, 36 insertions(+) create mode 100644 tests/queries/0_stateless/02006_use_constants_in_with_and_select.sql diff --git a/tests/queries/0_stateless/02006_use_constants_in_with_and_select.sql b/tests/queries/0_stateless/02006_use_constants_in_with_and_select.sql new file mode 100644 index 00000000000..91171c9ab7b --- /dev/null +++ b/tests/queries/0_stateless/02006_use_constants_in_with_and_select.sql @@ -0,0 +1,36 @@ +SELECT + 1 AS max_size, + groupArray(max_size)(col) +FROM + ( + SELECT 1 AS col + UNION ALL + SELECT 2 + ); + +WITH 1 AS max_size +SELECT groupArray(max_size)(col) +FROM + ( + SELECT 1 as col + UNION ALL + SELECT 2 + ); + +WITH 0.1 AS level +SELECT quantile(level)(number) +FROM numbers(1000); + +SELECT 0.1 AS level, quantile(level)(number) +FROM numbers(1000); + +WITH + 0.1 AS level, + 1 AS max_size +SELECT groupArray(max_size)(col) +FROM + ( + SELECT quantile(level)(number) AS col + FROM numbers(1000) + ); + From 8515f3b3a2b99b70e82e7e00ed4d5d5323edc8d7 Mon Sep 17 00:00:00 2001 From: fuwhu Date: Wed, 11 Aug 2021 11:40:06 +0800 Subject: [PATCH 427/599] Add metric MaxPushedDDLEntryID. --- programs/server/Server.cpp | 4 +++- src/Common/CurrentMetrics.cpp | 1 + src/Interpreters/DDLWorker.cpp | 16 +++++++++++++++- src/Interpreters/DDLWorker.h | 3 ++- 4 files changed, 21 insertions(+), 3 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 86bb04351b1..c69c48bb23d 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -126,6 +126,7 @@ namespace CurrentMetrics extern const Metric VersionInteger; extern const Metric MemoryTracking; extern const Metric MaxDDLEntryID; + extern const Metric MaxPushedDDLEntryID; } namespace fs = std::filesystem; @@ -1468,7 +1469,8 @@ if (ThreadFuzzer::instance().isEffective()) if (pool_size < 1) throw Exception("distributed_ddl.pool_size should be greater then 0", ErrorCodes::ARGUMENT_OUT_OF_BOUND); global_context->setDDLWorker(std::make_unique(pool_size, ddl_zookeeper_path, global_context, &config(), - "distributed_ddl", "DDLWorker", &CurrentMetrics::MaxDDLEntryID)); + "distributed_ddl", "DDLWorker", + &CurrentMetrics::MaxDDLEntryID, &CurrentMetrics::MaxPushedDDLEntryID)); } for (auto & server : *servers) diff --git a/src/Common/CurrentMetrics.cpp b/src/Common/CurrentMetrics.cpp index f94c3421107..9acefe8a2d8 100644 --- a/src/Common/CurrentMetrics.cpp +++ b/src/Common/CurrentMetrics.cpp @@ -60,6 +60,7 @@ M(BrokenDistributedFilesToInsert, "Number of files for asynchronous insertion into Distributed tables that has been marked as broken. This metric will starts from 0 on start. Number of files for every shard is summed.") \ M(TablesToDropQueueSize, "Number of dropped tables, that are waiting for background data removal.") \ M(MaxDDLEntryID, "Max processed DDL entry of DDLWorker.") \ + M(MaxPushedDDLEntryID, "Max DDL entry of DDLWorker that pushed to zookeeper.") \ M(PartsTemporary, "The part is generating now, it is not in data_parts list.") \ M(PartsPreCommitted, "The part is in data_parts, but not used for SELECTs.") \ M(PartsCommitted, "Active data part, used by current and upcoming SELECTs.") \ diff --git a/src/Interpreters/DDLWorker.cpp b/src/Interpreters/DDLWorker.cpp index 47ca2b72db8..c00f62f5133 100644 --- a/src/Interpreters/DDLWorker.cpp +++ b/src/Interpreters/DDLWorker.cpp @@ -158,15 +158,20 @@ DDLWorker::DDLWorker( const Poco::Util::AbstractConfiguration * config, const String & prefix, const String & logger_name, - const CurrentMetrics::Metric * max_entry_metric_) + const CurrentMetrics::Metric * max_entry_metric_, + const CurrentMetrics::Metric * max_pushed_entry_metric_) : context(Context::createCopy(context_)) , log(&Poco::Logger::get(logger_name)) , pool_size(pool_size_) , max_entry_metric(max_entry_metric_) + , max_pushed_entry_metric(max_pushed_entry_metric_) { if (max_entry_metric) CurrentMetrics::set(*max_entry_metric, 0); + if (max_pushed_entry_metric) + CurrentMetrics::set(*max_pushed_entry_metric, 0); + if (1 < pool_size) { LOG_WARNING(log, "DDLWorker is configured to use multiple threads. " @@ -1046,6 +1051,15 @@ String DDLWorker::enqueueQuery(DDLLogEntry & entry) zookeeper->createAncestors(query_path_prefix); String node_path = zookeeper->create(query_path_prefix, entry.toString(), zkutil::CreateMode::PersistentSequential); + if (max_pushed_entry_metric) + { + String str_buf = node_path.substr(query_path_prefix.length()); + DB::ReadBufferFromString in(str_buf); + CurrentMetrics::Metric id; + readText(id, in); + id = std::max(*max_pushed_entry_metric, id); + CurrentMetrics::set(*max_pushed_entry_metric, id); + } /// We cannot create status dirs in a single transaction with previous request, /// because we don't know node_path until previous request is executed. diff --git a/src/Interpreters/DDLWorker.h b/src/Interpreters/DDLWorker.h index d05b9b27611..d2b7c9d169d 100644 --- a/src/Interpreters/DDLWorker.h +++ b/src/Interpreters/DDLWorker.h @@ -44,7 +44,7 @@ class DDLWorker { public: DDLWorker(int pool_size_, const std::string & zk_root_dir, ContextPtr context_, const Poco::Util::AbstractConfiguration * config, const String & prefix, - const String & logger_name = "DDLWorker", const CurrentMetrics::Metric * max_entry_metric_ = nullptr); + const String & logger_name = "DDLWorker", const CurrentMetrics::Metric * max_entry_metric_ = nullptr, const CurrentMetrics::Metric * max_pushed_entry_metric_ = nullptr); virtual ~DDLWorker(); /// Pushes query into DDL queue, returns path to created node @@ -148,6 +148,7 @@ protected: std::atomic max_id = 0; const CurrentMetrics::Metric * max_entry_metric; + const CurrentMetrics::Metric * max_pushed_entry_metric; }; From 08f8511854fcf4600607f403ca7117d4ce63f7b4 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 11 Aug 2021 06:52:28 +0300 Subject: [PATCH 428/599] Fix Style --- src/Storages/MergeTree/BackgroundJobsExecutor.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/BackgroundJobsExecutor.cpp b/src/Storages/MergeTree/BackgroundJobsExecutor.cpp index 9ff7dd60266..f3d957117e8 100644 --- a/src/Storages/MergeTree/BackgroundJobsExecutor.cpp +++ b/src/Storages/MergeTree/BackgroundJobsExecutor.cpp @@ -146,7 +146,7 @@ try catch (...) /// Exception while we looking for a task, reschedule { tryLogCurrentException(__PRETTY_FUNCTION__); - + /// Why do we scheduleTask again? /// To retry on exception, since it may be some temporary exception. scheduleTask(/* with_backoff = */ true); From 92ef0a9ed71a730108cdc2658d8ad47332e84cbd Mon Sep 17 00:00:00 2001 From: abel-wang Date: Wed, 11 Aug 2021 12:34:41 +0800 Subject: [PATCH 429/599] format test results. --- .../02006_use_constants_in_with_and_select.reference | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/02006_use_constants_in_with_and_select.reference b/tests/queries/0_stateless/02006_use_constants_in_with_and_select.reference index ab7d5dd61aa..bbf008ffdf2 100644 --- a/tests/queries/0_stateless/02006_use_constants_in_with_and_select.reference +++ b/tests/queries/0_stateless/02006_use_constants_in_with_and_select.reference @@ -1,5 +1,5 @@ -1 [1] -1 [1] +1 [1] +[1] 99.9 -0.1 99.9 +0.1 99.9 [99.9] From 5a4684cdc5e7842a60d5e2db1f679148591d4999 Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 11 Aug 2021 09:42:06 +0300 Subject: [PATCH 430/599] Upd changelog for 21.8 --- CHANGELOG.md | 93 ++++++++++++++++++++++++++-------------------------- 1 file changed, 46 insertions(+), 47 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 7b760f15746..2cfd0b3e07e 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,19 +1,19 @@ -### ClickHouse release v21.8.2.19-prestable FIXME!! as compared to v21.7.5.29-stable +### ClickHouse release v21.8, 2021-08-11 #### Backward Incompatible Changes -* Use Map data type for system logs tables (`system.query_log`, `system.query_thread_log`, `system.processes`, `system.OpenTelemetrySpanLog`). These tables will be auto-created with new datatypes; old queries may be incompatible. [#23934](https://github.com/ClickHouse/ClickHouse/pull/23934) ([hexiaoting](https://github.com/hexiaoting)). +* Use `Map` data type for system logs tables (`system.query_log`, `system.query_thread_log`, `system.processes`, `system.OpenTelemetrySpanLog`). These tables will be auto-created with new data types; old queries may be incompatible. [#23934](https://github.com/ClickHouse/ClickHouse/pull/23934) ([hexiaoting](https://github.com/hexiaoting)). #### New Features * Add new functions `leftPad()`, `rightPad()`, `leftPadUTF8()`, `rightPadUTF8()`. [#26075](https://github.com/ClickHouse/ClickHouse/pull/26075) ([Vitaly Baranov](https://github.com/vitlibar)). * Render pipelines as graphs in Web UI if `EXPLAIN PIPELINE graph = 1` query is submitted. [#26067](https://github.com/ClickHouse/ClickHouse/pull/26067) ([alexey-milovidov](https://github.com/alexey-milovidov)). * Add the `FIRST` keyword to the `ADD INDEX` command to be able to add the index at the beginning of the indices list. [#25904](https://github.com/ClickHouse/ClickHouse/pull/25904) ([xjewer](https://github.com/xjewer)). -* Introduce `system.data_skipping_indices` table containing information about existing data skipping indices. This closes [#7659](https://github.com/ClickHouse/ClickHouse/issues/7659). [#25693](https://github.com/ClickHouse/ClickHouse/pull/25693) ([Dmitry Novik](https://github.com/novikd)). +* Introduce `system.data_skipping_indices` table containing information about existing data skipping indices. Close [#7659](https://github.com/ClickHouse/ClickHouse/issues/7659). [#25693](https://github.com/ClickHouse/ClickHouse/pull/25693) ([Dmitry Novik](https://github.com/novikd)). * Add `bin`/`unbin` functions. [#25609](https://github.com/ClickHouse/ClickHouse/pull/25609) ([zhaoyu](https://github.com/zxc111)). -* Support Map type in `mapAdd` and `mapSubtract` functions, add support for `(U)Int128`, `U(Int256)` types in `mapAdd` and `mapSubtract` functions. [#25596](https://github.com/ClickHouse/ClickHouse/pull/25596) ([Ildus Kurbangaliev](https://github.com/ildus)). -* Add support for `DISTINCT ON (columns)` expression, close [#25404](https://github.com/ClickHouse/ClickHouse/issues/25404). [#25589](https://github.com/ClickHouse/ClickHouse/pull/25589) ([Zijie Lu](https://github.com/TszKitLo40)). -* Collect common system metrics (in `system.asynchronous_metrics` and `system.asynchronous_metric_log`) on CPU usage, disk usage, memory usage, IO, network, files, load average, CPU frequencies, thermal sensors, EDAC counters, system uptime; also added metrics about the scheduling jitter and the time spent collecting the metrics. It works similar to `atop` in ClickHouse and allows access to monitoring data even if you have no additional tools installed. This closes [#9430](https://github.com/ClickHouse/ClickHouse/issues/9430). [#24416](https://github.com/ClickHouse/ClickHouse/pull/24416) ([Yegor Levankov](https://github.com/elevankoff)). +* Support `Map` and `(U)Int128`, `U(Int256) types in `mapAdd` and `mapSubtract` functions. [#25596](https://github.com/ClickHouse/ClickHouse/pull/25596) ([Ildus Kurbangaliev](https://github.com/ildus)). +* Support `DISTINCT ON (columns)` expression, close [#25404](https://github.com/ClickHouse/ClickHouse/issues/25404). [#25589](https://github.com/ClickHouse/ClickHouse/pull/25589) ([Zijie Lu](https://github.com/TszKitLo40)). +* Collect common system metrics (in `system.asynchronous_metrics` and `system.asynchronous_metric_log`) on CPU usage, disk usage, memory usage, IO, network, files, load average, CPU frequencies, thermal sensors, EDAC counters, system uptime; also added metrics about the scheduling jitter and the time spent collecting the metrics. It works similar to `atop` in ClickHouse and allows access to monitoring data even if you have no additional tools installed. Close [#9430](https://github.com/ClickHouse/ClickHouse/issues/9430). [#24416](https://github.com/ClickHouse/ClickHouse/pull/24416) ([Yegor Levankov](https://github.com/elevankoff)). * Add support for a part of SQLJSON standard. [#24148](https://github.com/ClickHouse/ClickHouse/pull/24148) ([l1tsolaiki](https://github.com/l1tsolaiki)). * Add MaterializedPostgreSQL table engine and database engine. This database engine allows replicating a whole database or any subset of database tables. [#20470](https://github.com/ClickHouse/ClickHouse/pull/20470) ([Kseniia Sumarokova](https://github.com/kssenii)). * Add an ability to reset a custom setting to default and remove it from the table's metadata. It allows rolling back the change without knowing the system/config's default. Closes [#14449](https://github.com/ClickHouse/ClickHouse/issues/14449). [#17769](https://github.com/ClickHouse/ClickHouse/pull/17769) ([xjewer](https://github.com/xjewer)). @@ -21,76 +21,76 @@ #### Performance Improvements * Improve latency of short queries that require reading from tables with a large number of columns. [#26371](https://github.com/ClickHouse/ClickHouse/pull/26371) ([Anton Popov](https://github.com/CurtizJ)). -* Added option to compile aggregate functions if `compile_aggregate_expressions` setting is on. [#24789](https://github.com/ClickHouse/ClickHouse/pull/24789) ([Maksim Kita](https://github.com/kitaisreal)). +* Compile aggregate functions. Use option `compile_aggregate_expressions` to enable it. [#24789](https://github.com/ClickHouse/ClickHouse/pull/24789) ([Maksim Kita](https://github.com/kitaisreal)). #### Improvements * For a dictionary with a complex key containing only one attribute, allow not wrapping the key expression in tuple for functions `dictGet`, `dictHas`. [#26130](https://github.com/ClickHouse/ClickHouse/pull/26130) ([Maksim Kita](https://github.com/kitaisreal)). -* Support arguments of `AggregateFunction` type in `bin` and `hex` functions. [#26094](https://github.com/ClickHouse/ClickHouse/pull/26094) ([zhaoyu](https://github.com/zxc111)). +* Implement function `bin`/`hex` from `AggregateFunction` states. [#26094](https://github.com/ClickHouse/ClickHouse/pull/26094) ([zhaoyu](https://github.com/zxc111)). * Support arguments of `UUID` type for `empty` and `notEmpty` functions. `UUID` is empty if it is all zeros (nil UUID). Closes [#3446](https://github.com/ClickHouse/ClickHouse/issues/3446). [#25974](https://github.com/ClickHouse/ClickHouse/pull/25974) ([zhaoyu](https://github.com/zxc111)). * Fix error with query `SET SQL_SELECT_LIMIT` in MySQL protocol. Closes [#17115](https://github.com/ClickHouse/ClickHouse/issues/17115). [#25972](https://github.com/ClickHouse/ClickHouse/pull/25972) ([Kseniia Sumarokova](https://github.com/kssenii)). * Web UI: if the value looks like an URL, automatically generate a link. [#25965](https://github.com/ClickHouse/ClickHouse/pull/25965) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* More instrumentation for network interaction: add counters for recv/send bytes; add gauges for recvs/sends. Added missing documentation. Closes [#5897](https://github.com/ClickHouse/ClickHouse/issues/5897). [#25962](https://github.com/ClickHouse/ClickHouse/pull/25962) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* More instrumentation for network interaction: add counters for recv/send bytes; add gauges for recvs/sends. Added missing documentation. Close [#5897](https://github.com/ClickHouse/ClickHouse/issues/5897). [#25962](https://github.com/ClickHouse/ClickHouse/pull/25962) ([alexey-milovidov](https://github.com/alexey-milovidov)). * Add setting `optimize_move_to_prewhere_if_final`. If query has `FINAL`, the optimization `move_to_prewhere` will be enabled only if both `optimize_move_to_prewhere` and `optimize_move_to_prewhere_if_final` are enabled. Closes [#8684](https://github.com/ClickHouse/ClickHouse/issues/8684). [#25940](https://github.com/ClickHouse/ClickHouse/pull/25940) ([Kseniia Sumarokova](https://github.com/kssenii)). -* Allow complex quoted identifiers of JOINed tables. This closes [#17861](https://github.com/ClickHouse/ClickHouse/issues/17861). [#25924](https://github.com/ClickHouse/ClickHouse/pull/25924) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Add support for Unicode (e.g. Chinese, Cyrillic) components in `Nested` data types. This closes [#25594](https://github.com/ClickHouse/ClickHouse/issues/25594). [#25923](https://github.com/ClickHouse/ClickHouse/pull/25923) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Make `sudo service clickhouse-server start` to work on systems with `systemd` like Centos 8. This closes [#14298](https://github.com/ClickHouse/ClickHouse/issues/14298). This closes [#17799](https://github.com/ClickHouse/ClickHouse/issues/17799). [#25921](https://github.com/ClickHouse/ClickHouse/pull/25921) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Allow `quantiles*` functions to work with `aggregate_functions_null_for_empty`. This closes [#25892](https://github.com/ClickHouse/ClickHouse/issues/25892). [#25919](https://github.com/ClickHouse/ClickHouse/pull/25919) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Allow complex quoted identifiers of JOINed tables. Close [#17861](https://github.com/ClickHouse/ClickHouse/issues/17861). [#25924](https://github.com/ClickHouse/ClickHouse/pull/25924) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Add support for Unicode (e.g. Chinese, Cyrillic) components in `Nested` data types. Close [#25594](https://github.com/ClickHouse/ClickHouse/issues/25594). [#25923](https://github.com/ClickHouse/ClickHouse/pull/25923) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Make `sudo service clickhouse-server start` to work on systems with `systemd` like Centos 8. Close [#14298](https://github.com/ClickHouse/ClickHouse/issues/14298). Close [#17799](https://github.com/ClickHouse/ClickHouse/issues/17799). [#25921](https://github.com/ClickHouse/ClickHouse/pull/25921) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Allow `quantiles*` functions to work with `aggregate_functions_null_for_empty`. Close [#25892](https://github.com/ClickHouse/ClickHouse/issues/25892). [#25919](https://github.com/ClickHouse/ClickHouse/pull/25919) ([alexey-milovidov](https://github.com/alexey-milovidov)). * Allow parameters for parametric aggregate functions to be arbitrary constant expressions (e.g., `1 + 2`), not just literals. It also allows using the query parameters (in parameterized queries like `{param:UInt8}`) inside parametric aggregate functions. Closes [#11607](https://github.com/ClickHouse/ClickHouse/issues/11607). [#25910](https://github.com/ClickHouse/ClickHouse/pull/25910) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Correctly throw the exception on the attempt to parse an invalid Date. Closes [#6481](https://github.com/ClickHouse/ClickHouse/issues/6481). [#25909](https://github.com/ClickHouse/ClickHouse/pull/25909) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Add support for queries with a column named `"null"` (it must be specified in backticks or double quotes) and `ON CLUSTER`. Closes [#24035](https://github.com/ClickHouse/ClickHouse/issues/24035). [#25907](https://github.com/ClickHouse/ClickHouse/pull/25907) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Support LowCardinality, Decimal, and UUID for JSON extract. Closes [#24606](https://github.com/ClickHouse/ClickHouse/issues/24606). [#25900](https://github.com/ClickHouse/ClickHouse/pull/25900) ([Kseniia Sumarokova](https://github.com/kssenii)). -* Convert history file from readline format to replxx format. [#25888](https://github.com/ClickHouse/ClickHouse/pull/25888) ([Azat Khuzhin](https://github.com/azat)). -* Fix an extremely rare bug which can lead to intersecting parts after `DROP PART` or background deletion of an empty part. [#25884](https://github.com/ClickHouse/ClickHouse/pull/25884) ([alesapin](https://github.com/alesapin)). -* Better handling of lost parts for ReplicatedMergeTree tables. Fixes rare inconsistencies in ReplicationQueue. Nothing should be visible to the user. Fixes [#10368](https://github.com/ClickHouse/ClickHouse/issues/10368). [#25820](https://github.com/ClickHouse/ClickHouse/pull/25820) ([alesapin](https://github.com/alesapin)). -* Allow to start clickhouse-client with unreadable working directory. [#25817](https://github.com/ClickHouse/ClickHouse/pull/25817) ([ianton-ru](https://github.com/ianton-ru)). -* Fix "No available columns" for Merge() storage. [#25801](https://github.com/ClickHouse/ClickHouse/pull/25801) ([Azat Khuzhin](https://github.com/azat)). +* Correctly throw the exception on the attempt to parse an invalid `Date`. Closes [#6481](https://github.com/ClickHouse/ClickHouse/issues/6481). [#25909](https://github.com/ClickHouse/ClickHouse/pull/25909) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Add support for queries with a column named `"null"` (it must be specified in back-ticks or double quotes) and `ON CLUSTER`. Closes [#24035](https://github.com/ClickHouse/ClickHouse/issues/24035). [#25907](https://github.com/ClickHouse/ClickHouse/pull/25907) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Support `LowCardinality`, `Decimal`, and `UUID` for `JSONExtract`. Closes [#24606](https://github.com/ClickHouse/ClickHouse/issues/24606). [#25900](https://github.com/ClickHouse/ClickHouse/pull/25900) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Convert history file from `readline` format to `replxx` format. [#25888](https://github.com/ClickHouse/ClickHouse/pull/25888) ([Azat Khuzhin](https://github.com/azat)). +* Fix bug which can lead to intersecting parts after `DROP PART` or background deletion of an empty part. [#25884](https://github.com/ClickHouse/ClickHouse/pull/25884) ([alesapin](https://github.com/alesapin)). +* Better handling of lost parts for `ReplicatedMergeTree` tables. Fixes rare inconsistencies in `ReplicationQueue`. Fixes [#10368](https://github.com/ClickHouse/ClickHouse/issues/10368). [#25820](https://github.com/ClickHouse/ClickHouse/pull/25820) ([alesapin](https://github.com/alesapin)). +* Allow starting clickhouse-client with unreadable working directory. [#25817](https://github.com/ClickHouse/ClickHouse/pull/25817) ([ianton-ru](https://github.com/ianton-ru)). +* Fix "No available columns" error for `Merge` storage. [#25801](https://github.com/ClickHouse/ClickHouse/pull/25801) ([Azat Khuzhin](https://github.com/azat)). * MySQL Engine now supports the exchange of column comments between MySQL and ClickHouse. [#25795](https://github.com/ClickHouse/ClickHouse/pull/25795) ([Storozhuk Kostiantyn](https://github.com/sand6255)). * Fix inconsistent behaviour of `GROUP BY` constant on empty set. Closes [#6842](https://github.com/ClickHouse/ClickHouse/issues/6842). [#25786](https://github.com/ClickHouse/ClickHouse/pull/25786) ([Kseniia Sumarokova](https://github.com/kssenii)). * Use `Map` data type for key-value dictionaries in system logs tables (`system.query_log`, `system.query_thread_log`, `system.processes`, `system.opentelemetry_span_log`). Virtual columns are created to support old queries. Closes [#18698](https://github.com/ClickHouse/ClickHouse/issues/18698). Authors @hexiaoting, @sundy-li. [#25773](https://github.com/ClickHouse/ClickHouse/pull/25773) ([Maksim Kita](https://github.com/kitaisreal)). * Cancel already running merges in partition on `DROP PARTITION` and `TRUNCATE` for `ReplicatedMergeTree`. Resolves [#17151](https://github.com/ClickHouse/ClickHouse/issues/17151). [#25684](https://github.com/ClickHouse/ClickHouse/pull/25684) ([tavplubix](https://github.com/tavplubix)). -* MaterializeMySQL now supports `ENUM` data type. [#25676](https://github.com/ClickHouse/ClickHouse/pull/25676) ([Storozhuk Kostiantyn](https://github.com/sand6255)). -* Support materialized and aliased columns in joins, close [#13274](https://github.com/ClickHouse/ClickHouse/issues/13274). [#25634](https://github.com/ClickHouse/ClickHouse/pull/25634) ([Vladimir C](https://github.com/vdimir)). +* Support ENUM` data type for MaterializeMySQL. [#25676](https://github.com/ClickHouse/ClickHouse/pull/25676) ([Storozhuk Kostiantyn](https://github.com/sand6255)). +* Support materialized and aliased columns in JOIN, close [#13274](https://github.com/ClickHouse/ClickHouse/issues/13274). [#25634](https://github.com/ClickHouse/ClickHouse/pull/25634) ([Vladimir C](https://github.com/vdimir)). * Fix possible logical race condition between `ALTER TABLE ... DETACH` and background merges. [#25605](https://github.com/ClickHouse/ClickHouse/pull/25605) ([Azat Khuzhin](https://github.com/azat)). -* Make `NetworkReceiveElapsedMicroseconds` metric to correctly include the time spent waiting for data from the client to `INSERT`. This closes [#9958](https://github.com/ClickHouse/ClickHouse/issues/9958). [#25602](https://github.com/ClickHouse/ClickHouse/pull/25602) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Support `TRUNCATE TABLE` for StorageS3 and StorageHDFS. This closes [#25530](https://github.com/ClickHouse/ClickHouse/issues/25530). [#25550](https://github.com/ClickHouse/ClickHouse/pull/25550) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Make `NetworkReceiveElapsedMicroseconds` metric to correctly include the time spent waiting for data from the client to `INSERT`. Close [#9958](https://github.com/ClickHouse/ClickHouse/issues/9958). [#25602](https://github.com/ClickHouse/ClickHouse/pull/25602) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Support `TRUNCATE TABLE` for StorageS3 and StorageHDFS. Close [#25530](https://github.com/ClickHouse/ClickHouse/issues/25530). [#25550](https://github.com/ClickHouse/ClickHouse/pull/25550) ([Kseniia Sumarokova](https://github.com/kssenii)). * Support for dynamic reloading of config to change number of threads in pool for background jobs execution (merges, mutations, fetches). [#25548](https://github.com/ClickHouse/ClickHouse/pull/25548) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). -* Allow extract of non-string element as string using JSONExtract. This is for [#25414](https://github.com/ClickHouse/ClickHouse/issues/25414). [#25452](https://github.com/ClickHouse/ClickHouse/pull/25452) ([Amos Bird](https://github.com/amosbird)). -* `Database` argument for `StorageMerge` support regular expression. This closes [#776]](https://github.com/ClickHouse/ClickHouse/issues/776). [#25064](https://github.com/ClickHouse/ClickHouse/pull/25064) ([flynn](https://github.com/ucasfl)). -* Support for multiple includes in configuration. It is possible to include users configuration, remote servers configuration from multiple sources. Simply place `` element with `from_zk`, `from_env` or `incl` attribute and it will be replaced with the substitution. [#24404](https://github.com/ClickHouse/ClickHouse/pull/24404) ([nvartolomei](https://github.com/nvartolomei)). +* Allow extracting of non-string element as string using `JSONExtract`. This is for [#25414](https://github.com/ClickHouse/ClickHouse/issues/25414). [#25452](https://github.com/ClickHouse/ClickHouse/pull/25452) ([Amos Bird](https://github.com/amosbird)). +* Support regular expression in `Database` argument for `StorageMerge`. Close [#776](https://github.com/ClickHouse/ClickHouse/issues/776). [#25064](https://github.com/ClickHouse/ClickHouse/pull/25064) ([flynn](https://github.com/ucasfl)). +* Support for multiple includes in configuration. It is possible to include users configuration, remote server configuration from multiple sources. Simply place `` element with `from_zk`, `from_env` or `incl` attribute, and it will be replaced with the substitution. [#24404](https://github.com/ClickHouse/ClickHouse/pull/24404) ([nvartolomei](https://github.com/nvartolomei)). #### Bug Fixes -* Sometimes `SET ROLE` could work incorrectly, this PR fixes that. [#26707](https://github.com/ClickHouse/ClickHouse/pull/26707) ([Vitaly Baranov](https://github.com/vitlibar)). -* Fix potential nullptr dereference in window functions. This fixes [#25276](https://github.com/ClickHouse/ClickHouse/issues/25276). [#26668](https://github.com/ClickHouse/ClickHouse/pull/26668) ([Alexander Kuzmenkov](https://github.com/akuzm)). -* Fix history file conversion if file is empty. [#26589](https://github.com/ClickHouse/ClickHouse/pull/26589) ([Azat Khuzhin](https://github.com/azat)). -* Fix incorrect function names of groupBitmapAnd/Or/Xor. This fixes. [#26557](https://github.com/ClickHouse/ClickHouse/pull/26557) ([Amos Bird](https://github.com/amosbird)). -* Update `chown` cmd check in clickhouse-server docker entrypoint. It fixes the bug 'cluster pod restart failed (or timeout) on kubernetes'. [#26545](https://github.com/ClickHouse/ClickHouse/pull/26545) ([Ky Li](https://github.com/Kylinrix)). +* Fix incorrect `SET ROLE` in some cases. [#26707](https://github.com/ClickHouse/ClickHouse/pull/26707) ([Vitaly Baranov](https://github.com/vitlibar)). +* Fix potential `nullptr` dereference in window functions. Fix [#25276](https://github.com/ClickHouse/ClickHouse/issues/25276). [#26668](https://github.com/ClickHouse/ClickHouse/pull/26668) ([Alexander Kuzmenkov](https://github.com/akuzm)). +* Fix empty history file conversion. [#26589](https://github.com/ClickHouse/ClickHouse/pull/26589) ([Azat Khuzhin](https://github.com/azat)). +* Fix incorrect function names of `groupBitmapAnd/Or/Xor`. Fix [#26557](https://github.com/ClickHouse/ClickHouse/pull/26557) ([Amos Bird](https://github.com/amosbird)). +* Update `chown` cmd check in `clickhouse-server` docker entrypoint. It fixes error 'cluster pod restart failed (or timeout)' on kubernetes. [#26545](https://github.com/ClickHouse/ClickHouse/pull/26545) ([Ky Li](https://github.com/Kylinrix)). * Fix crash in rabbitmq shutdown in case rabbitmq setup was not started. Closes [#26504](https://github.com/ClickHouse/ClickHouse/issues/26504). [#26529](https://github.com/ClickHouse/ClickHouse/pull/26529) ([Kseniia Sumarokova](https://github.com/kssenii)). * Fix issues with `CREATE DICTIONARY` query if dictionary name or database name was quoted. Closes [#26491](https://github.com/ClickHouse/ClickHouse/issues/26491). [#26508](https://github.com/ClickHouse/ClickHouse/pull/26508) ([Maksim Kita](https://github.com/kitaisreal)). -* Fix broken name resolution after rewriting column aliases. This fixes [#26432](https://github.com/ClickHouse/ClickHouse/issues/26432). [#26475](https://github.com/ClickHouse/ClickHouse/pull/26475) ([Amos Bird](https://github.com/amosbird)). +* Fix broken name resolution after rewriting column aliases. Fix [#26432](https://github.com/ClickHouse/ClickHouse/issues/26432). [#26475](https://github.com/ClickHouse/ClickHouse/pull/26475) ([Amos Bird](https://github.com/amosbird)). * Fix some fuzzed msan crash. Fixes [#22517](https://github.com/ClickHouse/ClickHouse/issues/22517). [#26428](https://github.com/ClickHouse/ClickHouse/pull/26428) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fix infinite non joined block stream in `partial_merge_join` close [#26325](https://github.com/ClickHouse/ClickHouse/issues/26325). [#26374](https://github.com/ClickHouse/ClickHouse/pull/26374) ([Vladimir C](https://github.com/vdimir)). -* Fix possible crash when login as dropped user. This PR fixes [#26073](https://github.com/ClickHouse/ClickHouse/issues/26073). [#26363](https://github.com/ClickHouse/ClickHouse/pull/26363) ([Vitaly Baranov](https://github.com/vitlibar)). +* Fix infinite non-joined block stream in `partial_merge_join` close [#26325](https://github.com/ClickHouse/ClickHouse/issues/26325). [#26374](https://github.com/ClickHouse/ClickHouse/pull/26374) ([Vladimir C](https://github.com/vdimir)). +* Fix possible crash when login as dropped user. Fix [#26073](https://github.com/ClickHouse/ClickHouse/issues/26073). [#26363](https://github.com/ClickHouse/ClickHouse/pull/26363) ([Vitaly Baranov](https://github.com/vitlibar)). * Fix `optimize_distributed_group_by_sharding_key` for multiple columns (leads to incorrect result w/ `optimize_skip_unused_shards=1`/`allow_nondeterministic_optimize_skip_unused_shards=1` and multiple columns in sharding key expression). [#26353](https://github.com/ClickHouse/ClickHouse/pull/26353) ([Azat Khuzhin](https://github.com/azat)). * Fixed rare bug in lost replica recovery that may cause replicas to diverge. [#26321](https://github.com/ClickHouse/ClickHouse/pull/26321) ([tavplubix](https://github.com/tavplubix)). * Fix zstd decompression in case there are escape sequences at the end of internal buffer. Closes [#26013](https://github.com/ClickHouse/ClickHouse/issues/26013). [#26314](https://github.com/ClickHouse/ClickHouse/pull/26314) ([Kseniia Sumarokova](https://github.com/kssenii)). -* Fix logical error on join with totals, closing [#26017](https://github.com/ClickHouse/ClickHouse/issues/26017). [#26250](https://github.com/ClickHouse/ClickHouse/pull/26250) ([Vladimir C](https://github.com/vdimir)). -* Remove excessive newline in `thread_name` column in `system.stack_trace` table. This fixes [#24124](https://github.com/ClickHouse/ClickHouse/issues/24124). [#26210](https://github.com/ClickHouse/ClickHouse/pull/26210) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix `joinGet` with LowCarinality columns, close [#25993](https://github.com/ClickHouse/ClickHouse/issues/25993). [#26118](https://github.com/ClickHouse/ClickHouse/pull/26118) ([Vladimir C](https://github.com/vdimir)). +* Fix logical error on join with totals, close [#26017](https://github.com/ClickHouse/ClickHouse/issues/26017). [#26250](https://github.com/ClickHouse/ClickHouse/pull/26250) ([Vladimir C](https://github.com/vdimir)). +* Remove excessive newline in `thread_name` column in `system.stack_trace` table. Fix [#24124](https://github.com/ClickHouse/ClickHouse/issues/24124). [#26210](https://github.com/ClickHouse/ClickHouse/pull/26210) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix `joinGet` with `LowCarinality` columns, close [#25993](https://github.com/ClickHouse/ClickHouse/issues/25993). [#26118](https://github.com/ClickHouse/ClickHouse/pull/26118) ([Vladimir C](https://github.com/vdimir)). * Fix possible crash in `pointInPolygon` if the setting `validate_polygons` is turned off. [#26113](https://github.com/ClickHouse/ClickHouse/pull/26113) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix throwing exception when iterate over non existing remote directory. [#26087](https://github.com/ClickHouse/ClickHouse/pull/26087) ([ianton-ru](https://github.com/ianton-ru)). +* Fix throwing exception when iterate over non-existing remote directory. [#26087](https://github.com/ClickHouse/ClickHouse/pull/26087) ([ianton-ru](https://github.com/ianton-ru)). * Fix rare server crash because of `abort` in ZooKeeper client. Fixes [#25813](https://github.com/ClickHouse/ClickHouse/issues/25813). [#26079](https://github.com/ClickHouse/ClickHouse/pull/26079) ([alesapin](https://github.com/alesapin)). * Fix wrong thread estimation for right subquery join in some cases. Close [#24075](https://github.com/ClickHouse/ClickHouse/issues/24075). [#26052](https://github.com/ClickHouse/ClickHouse/pull/26052) ([Vladimir C](https://github.com/vdimir)). * Fixed incorrect `sequence_id` in MySQL protocol packets that ClickHouse sends on exception during query execution. It might cause MySQL client to reset connection to ClickHouse server. Fixes [#21184](https://github.com/ClickHouse/ClickHouse/issues/21184). [#26051](https://github.com/ClickHouse/ClickHouse/pull/26051) ([tavplubix](https://github.com/tavplubix)). -* Fix possible mismatched header when using normal projection with prewhere. This fixes [#26020](https://github.com/ClickHouse/ClickHouse/issues/26020). [#26038](https://github.com/ClickHouse/ClickHouse/pull/26038) ([Amos Bird](https://github.com/amosbird)). +* Fix possible mismatched header when using normal projection with `PREWHERE`. Fix [#26020](https://github.com/ClickHouse/ClickHouse/issues/26020). [#26038](https://github.com/ClickHouse/ClickHouse/pull/26038) ([Amos Bird](https://github.com/amosbird)). * Fix formatting of type `Map` with integer keys to `JSON`. [#25982](https://github.com/ClickHouse/ClickHouse/pull/25982) ([Anton Popov](https://github.com/CurtizJ)). -* Fix possible deadlock during query profiler stack unwinding. Fixes [#25968](https://github.com/ClickHouse/ClickHouse/issues/25968). [#25970](https://github.com/ClickHouse/ClickHouse/pull/25970) ([Maksim Kita](https://github.com/kitaisreal)). -* Fix crash on call dictGet() with bad arguments. [#25913](https://github.com/ClickHouse/ClickHouse/pull/25913) ([Vitaly Baranov](https://github.com/vitlibar)). +* Fix possible deadlock during query profiler stack unwinding. Fix [#25968](https://github.com/ClickHouse/ClickHouse/issues/25968). [#25970](https://github.com/ClickHouse/ClickHouse/pull/25970) ([Maksim Kita](https://github.com/kitaisreal)). +* Fix crash on call `dictGet()` with bad arguments. [#25913](https://github.com/ClickHouse/ClickHouse/pull/25913) ([Vitaly Baranov](https://github.com/vitlibar)). * Fixed `scram-sha-256` authentication for PostgreSQL engines. Closes [#24516](https://github.com/ClickHouse/ClickHouse/issues/24516). [#25906](https://github.com/ClickHouse/ClickHouse/pull/25906) ([Kseniia Sumarokova](https://github.com/kssenii)). * Fix extremely long backoff for background tasks when the background pool is full. Fixes [#25836](https://github.com/ClickHouse/ClickHouse/issues/25836). [#25893](https://github.com/ClickHouse/ClickHouse/pull/25893) ([alesapin](https://github.com/alesapin)). -* Fix ARM exception handling with non default page size. Fixes [#25512](https://github.com/ClickHouse/ClickHouse/issues/25512). Fixes [#25044](https://github.com/ClickHouse/ClickHouse/issues/25044). Fixes [#24901](https://github.com/ClickHouse/ClickHouse/issues/24901). Fixes [#23183](https://github.com/ClickHouse/ClickHouse/issues/23183). Fixes [#20221](https://github.com/ClickHouse/ClickHouse/issues/20221). Fixes [#19703](https://github.com/ClickHouse/ClickHouse/issues/19703). Fixes [#19028](https://github.com/ClickHouse/ClickHouse/issues/19028). Fixes [#18391](https://github.com/ClickHouse/ClickHouse/issues/18391). Fixes [#18121](https://github.com/ClickHouse/ClickHouse/issues/18121). Fixes [#17994](https://github.com/ClickHouse/ClickHouse/issues/17994). Fixes [#12483](https://github.com/ClickHouse/ClickHouse/issues/12483). [#25854](https://github.com/ClickHouse/ClickHouse/pull/25854) ([Maksim Kita](https://github.com/kitaisreal)). -* Fix sharding_key from column w/o function for remote() (before `select * from remote('127.1', system.one, dummy)` leads to `Unknown column: dummy, there are only columns .` error). [#25824](https://github.com/ClickHouse/ClickHouse/pull/25824) ([Azat Khuzhin](https://github.com/azat)). +* Fix ARM exception handling with non default page size. Fixes [#25512](https://github.com/ClickHouse/ClickHouse/issues/25512), [#25044](https://github.com/ClickHouse/ClickHouse/issues/25044), [#24901](https://github.com/ClickHouse/ClickHouse/issues/24901), [#23183](https://github.com/ClickHouse/ClickHouse/issues/23183), [#20221](https://github.com/ClickHouse/ClickHouse/issues/20221), [#19703](https://github.com/ClickHouse/ClickHouse/issues/19703), [#19028](https://github.com/ClickHouse/ClickHouse/issues/19028), [#18391](https://github.com/ClickHouse/ClickHouse/issues/18391), [#18121](https://github.com/ClickHouse/ClickHouse/issues/18121), [#17994](https://github.com/ClickHouse/ClickHouse/issues/17994), [#12483](https://github.com/ClickHouse/ClickHouse/issues/12483). [#25854](https://github.com/ClickHouse/ClickHouse/pull/25854) ([Maksim Kita](https://github.com/kitaisreal)). +* Fix sharding_key from column w/o function for `remote()` (before `select * from remote('127.1', system.one, dummy)` leads to `Unknown column: dummy, there are only columns .` error). [#25824](https://github.com/ClickHouse/ClickHouse/pull/25824) ([Azat Khuzhin](https://github.com/azat)). * Fixed `Not found column ...` and `Missing column ...` errors when selecting from `MaterializeMySQL`. Fixes [#23708](https://github.com/ClickHouse/ClickHouse/issues/23708), [#24830](https://github.com/ClickHouse/ClickHouse/issues/24830), [#25794](https://github.com/ClickHouse/ClickHouse/issues/25794). [#25822](https://github.com/ClickHouse/ClickHouse/pull/25822) ([tavplubix](https://github.com/tavplubix)). * Fix `optimize_skip_unused_shards_rewrite_in` for non-UInt64 types (may select incorrect shards eventually or throw `Cannot infer type of an empty tuple` or `Function tuple requires at least one argument`). [#25798](https://github.com/ClickHouse/ClickHouse/pull/25798) ([Azat Khuzhin](https://github.com/azat)). * Fix rare bug with `DROP PART` query for `ReplicatedMergeTree` tables which can lead to error message `Unexpected merged part intersecting drop range`. [#25783](https://github.com/ClickHouse/ClickHouse/pull/25783) ([alesapin](https://github.com/alesapin)). @@ -98,7 +98,7 @@ * Allow StorageMerge to access tables with aliases. Closes [#6051](https://github.com/ClickHouse/ClickHouse/issues/6051). [#25694](https://github.com/ClickHouse/ClickHouse/pull/25694) ([Kseniia Sumarokova](https://github.com/kssenii)). * Fix slow dict join in some cases, close [#24209](https://github.com/ClickHouse/ClickHouse/issues/24209). [#25618](https://github.com/ClickHouse/ClickHouse/pull/25618) ([Vladimir C](https://github.com/vdimir)). * Fix `ALTER MODIFY COLUMN` of columns, which participates in TTL expressions. [#25554](https://github.com/ClickHouse/ClickHouse/pull/25554) ([Anton Popov](https://github.com/CurtizJ)). -* Fix assertion in PREWHERE with non-uint8 type, close [#19589](https://github.com/ClickHouse/ClickHouse/issues/19589). [#25484](https://github.com/ClickHouse/ClickHouse/pull/25484) ([Vladimir C](https://github.com/vdimir)). +* Fix assertion in `PREWHERE` with non-UInt8 type, close [#19589](https://github.com/ClickHouse/ClickHouse/issues/19589). [#25484](https://github.com/ClickHouse/ClickHouse/pull/25484) ([Vladimir C](https://github.com/vdimir)). * `CAST` from `Date` to `DateTime` (or `DateTime64`) was not using the timezone of the `DateTime` type. It can also affect the comparison between `Date` and `DateTime`. Inference of the common type for `Date` and `DateTime` also was not using the corresponding timezone. It affected the results of function `if` and array construction. Closes [#24128](https://github.com/ClickHouse/ClickHouse/issues/24128). [#24129](https://github.com/ClickHouse/ClickHouse/pull/24129) ([Maksim Kita](https://github.com/kitaisreal)). #### Build/Testing/Packaging Improvements @@ -106,7 +106,6 @@ * Disabling TestFlows LDAP module due to test fails. [#26065](https://github.com/ClickHouse/ClickHouse/pull/26065) ([vzakaznikov](https://github.com/vzakaznikov)). * Enabling all TestFlows modules and fixing some tests. [#26011](https://github.com/ClickHouse/ClickHouse/pull/26011) ([vzakaznikov](https://github.com/vzakaznikov)). * Add new tests for checking access rights for columns used in filters (`WHERE` / `PREWHERE` / row policy) of the `SELECT` statement after changes in [#24405](https://github.com/ClickHouse/ClickHouse/pull/24405). [#25619](https://github.com/ClickHouse/ClickHouse/pull/25619) ([Vitaly Baranov](https://github.com/vitlibar)). -* Syntax update: changing 'is' to '==' when necessary. [#25559](https://github.com/ClickHouse/ClickHouse/pull/25559) ([MyroTk](https://github.com/MyroTk)). #### Other From 0737925dbe76b11033135ffb73841fe2b4df93d2 Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 11 Aug 2021 09:53:09 +0300 Subject: [PATCH 431/599] Reorder entries in 21.8 changelog --- CHANGELOG.md | 22 +++++++++++----------- 1 file changed, 11 insertions(+), 11 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 2cfd0b3e07e..0b8ca122582 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -6,22 +6,22 @@ #### New Features +* Collect common system metrics (in `system.asynchronous_metrics` and `system.asynchronous_metric_log`) on CPU usage, disk usage, memory usage, IO, network, files, load average, CPU frequencies, thermal sensors, EDAC counters, system uptime; also added metrics about the scheduling jitter and the time spent collecting the metrics. It works similar to `atop` in ClickHouse and allows access to monitoring data even if you have no additional tools installed. Close [#9430](https://github.com/ClickHouse/ClickHouse/issues/9430). [#24416](https://github.com/ClickHouse/ClickHouse/pull/24416) ([Yegor Levankov](https://github.com/elevankoff)). * Add new functions `leftPad()`, `rightPad()`, `leftPadUTF8()`, `rightPadUTF8()`. [#26075](https://github.com/ClickHouse/ClickHouse/pull/26075) ([Vitaly Baranov](https://github.com/vitlibar)). -* Render pipelines as graphs in Web UI if `EXPLAIN PIPELINE graph = 1` query is submitted. [#26067](https://github.com/ClickHouse/ClickHouse/pull/26067) ([alexey-milovidov](https://github.com/alexey-milovidov)). * Add the `FIRST` keyword to the `ADD INDEX` command to be able to add the index at the beginning of the indices list. [#25904](https://github.com/ClickHouse/ClickHouse/pull/25904) ([xjewer](https://github.com/xjewer)). * Introduce `system.data_skipping_indices` table containing information about existing data skipping indices. Close [#7659](https://github.com/ClickHouse/ClickHouse/issues/7659). [#25693](https://github.com/ClickHouse/ClickHouse/pull/25693) ([Dmitry Novik](https://github.com/novikd)). * Add `bin`/`unbin` functions. [#25609](https://github.com/ClickHouse/ClickHouse/pull/25609) ([zhaoyu](https://github.com/zxc111)). * Support `Map` and `(U)Int128`, `U(Int256) types in `mapAdd` and `mapSubtract` functions. [#25596](https://github.com/ClickHouse/ClickHouse/pull/25596) ([Ildus Kurbangaliev](https://github.com/ildus)). * Support `DISTINCT ON (columns)` expression, close [#25404](https://github.com/ClickHouse/ClickHouse/issues/25404). [#25589](https://github.com/ClickHouse/ClickHouse/pull/25589) ([Zijie Lu](https://github.com/TszKitLo40)). -* Collect common system metrics (in `system.asynchronous_metrics` and `system.asynchronous_metric_log`) on CPU usage, disk usage, memory usage, IO, network, files, load average, CPU frequencies, thermal sensors, EDAC counters, system uptime; also added metrics about the scheduling jitter and the time spent collecting the metrics. It works similar to `atop` in ClickHouse and allows access to monitoring data even if you have no additional tools installed. Close [#9430](https://github.com/ClickHouse/ClickHouse/issues/9430). [#24416](https://github.com/ClickHouse/ClickHouse/pull/24416) ([Yegor Levankov](https://github.com/elevankoff)). * Add support for a part of SQLJSON standard. [#24148](https://github.com/ClickHouse/ClickHouse/pull/24148) ([l1tsolaiki](https://github.com/l1tsolaiki)). * Add MaterializedPostgreSQL table engine and database engine. This database engine allows replicating a whole database or any subset of database tables. [#20470](https://github.com/ClickHouse/ClickHouse/pull/20470) ([Kseniia Sumarokova](https://github.com/kssenii)). * Add an ability to reset a custom setting to default and remove it from the table's metadata. It allows rolling back the change without knowing the system/config's default. Closes [#14449](https://github.com/ClickHouse/ClickHouse/issues/14449). [#17769](https://github.com/ClickHouse/ClickHouse/pull/17769) ([xjewer](https://github.com/xjewer)). +* Render pipelines as graphs in Web UI if `EXPLAIN PIPELINE graph = 1` query is submitted. [#26067](https://github.com/ClickHouse/ClickHouse/pull/26067) ([alexey-milovidov](https://github.com/alexey-milovidov)). #### Performance Improvements -* Improve latency of short queries that require reading from tables with a large number of columns. [#26371](https://github.com/ClickHouse/ClickHouse/pull/26371) ([Anton Popov](https://github.com/CurtizJ)). * Compile aggregate functions. Use option `compile_aggregate_expressions` to enable it. [#24789](https://github.com/ClickHouse/ClickHouse/pull/24789) ([Maksim Kita](https://github.com/kitaisreal)). +* Improve latency of short queries that require reading from tables with many columns. [#26371](https://github.com/ClickHouse/ClickHouse/pull/26371) ([Anton Popov](https://github.com/CurtizJ)). #### Improvements @@ -29,16 +29,15 @@ * Implement function `bin`/`hex` from `AggregateFunction` states. [#26094](https://github.com/ClickHouse/ClickHouse/pull/26094) ([zhaoyu](https://github.com/zxc111)). * Support arguments of `UUID` type for `empty` and `notEmpty` functions. `UUID` is empty if it is all zeros (nil UUID). Closes [#3446](https://github.com/ClickHouse/ClickHouse/issues/3446). [#25974](https://github.com/ClickHouse/ClickHouse/pull/25974) ([zhaoyu](https://github.com/zxc111)). * Fix error with query `SET SQL_SELECT_LIMIT` in MySQL protocol. Closes [#17115](https://github.com/ClickHouse/ClickHouse/issues/17115). [#25972](https://github.com/ClickHouse/ClickHouse/pull/25972) ([Kseniia Sumarokova](https://github.com/kssenii)). -* Web UI: if the value looks like an URL, automatically generate a link. [#25965](https://github.com/ClickHouse/ClickHouse/pull/25965) ([alexey-milovidov](https://github.com/alexey-milovidov)). * More instrumentation for network interaction: add counters for recv/send bytes; add gauges for recvs/sends. Added missing documentation. Close [#5897](https://github.com/ClickHouse/ClickHouse/issues/5897). [#25962](https://github.com/ClickHouse/ClickHouse/pull/25962) ([alexey-milovidov](https://github.com/alexey-milovidov)). * Add setting `optimize_move_to_prewhere_if_final`. If query has `FINAL`, the optimization `move_to_prewhere` will be enabled only if both `optimize_move_to_prewhere` and `optimize_move_to_prewhere_if_final` are enabled. Closes [#8684](https://github.com/ClickHouse/ClickHouse/issues/8684). [#25940](https://github.com/ClickHouse/ClickHouse/pull/25940) ([Kseniia Sumarokova](https://github.com/kssenii)). * Allow complex quoted identifiers of JOINed tables. Close [#17861](https://github.com/ClickHouse/ClickHouse/issues/17861). [#25924](https://github.com/ClickHouse/ClickHouse/pull/25924) ([alexey-milovidov](https://github.com/alexey-milovidov)). * Add support for Unicode (e.g. Chinese, Cyrillic) components in `Nested` data types. Close [#25594](https://github.com/ClickHouse/ClickHouse/issues/25594). [#25923](https://github.com/ClickHouse/ClickHouse/pull/25923) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Make `sudo service clickhouse-server start` to work on systems with `systemd` like Centos 8. Close [#14298](https://github.com/ClickHouse/ClickHouse/issues/14298). Close [#17799](https://github.com/ClickHouse/ClickHouse/issues/17799). [#25921](https://github.com/ClickHouse/ClickHouse/pull/25921) ([alexey-milovidov](https://github.com/alexey-milovidov)). * Allow `quantiles*` functions to work with `aggregate_functions_null_for_empty`. Close [#25892](https://github.com/ClickHouse/ClickHouse/issues/25892). [#25919](https://github.com/ClickHouse/ClickHouse/pull/25919) ([alexey-milovidov](https://github.com/alexey-milovidov)). * Allow parameters for parametric aggregate functions to be arbitrary constant expressions (e.g., `1 + 2`), not just literals. It also allows using the query parameters (in parameterized queries like `{param:UInt8}`) inside parametric aggregate functions. Closes [#11607](https://github.com/ClickHouse/ClickHouse/issues/11607). [#25910](https://github.com/ClickHouse/ClickHouse/pull/25910) ([alexey-milovidov](https://github.com/alexey-milovidov)). * Correctly throw the exception on the attempt to parse an invalid `Date`. Closes [#6481](https://github.com/ClickHouse/ClickHouse/issues/6481). [#25909](https://github.com/ClickHouse/ClickHouse/pull/25909) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Add support for queries with a column named `"null"` (it must be specified in back-ticks or double quotes) and `ON CLUSTER`. Closes [#24035](https://github.com/ClickHouse/ClickHouse/issues/24035). [#25907](https://github.com/ClickHouse/ClickHouse/pull/25907) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Support for multiple includes in configuration. It is possible to include users configuration, remote server configuration from multiple sources. Simply place `` element with `from_zk`, `from_env` or `incl` attribute, and it will be replaced with the substitution. [#24404](https://github.com/ClickHouse/ClickHouse/pull/24404) ([nvartolomei](https://github.com/nvartolomei)). +* Support for queries with a column named `"null"` (it must be specified in back-ticks or double quotes) and `ON CLUSTER`. Closes [#24035](https://github.com/ClickHouse/ClickHouse/issues/24035). [#25907](https://github.com/ClickHouse/ClickHouse/pull/25907) ([alexey-milovidov](https://github.com/alexey-milovidov)). * Support `LowCardinality`, `Decimal`, and `UUID` for `JSONExtract`. Closes [#24606](https://github.com/ClickHouse/ClickHouse/issues/24606). [#25900](https://github.com/ClickHouse/ClickHouse/pull/25900) ([Kseniia Sumarokova](https://github.com/kssenii)). * Convert history file from `readline` format to `replxx` format. [#25888](https://github.com/ClickHouse/ClickHouse/pull/25888) ([Azat Khuzhin](https://github.com/azat)). * Fix bug which can lead to intersecting parts after `DROP PART` or background deletion of an empty part. [#25884](https://github.com/ClickHouse/ClickHouse/pull/25884) ([alesapin](https://github.com/alesapin)). @@ -57,22 +56,21 @@ * Support for dynamic reloading of config to change number of threads in pool for background jobs execution (merges, mutations, fetches). [#25548](https://github.com/ClickHouse/ClickHouse/pull/25548) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). * Allow extracting of non-string element as string using `JSONExtract`. This is for [#25414](https://github.com/ClickHouse/ClickHouse/issues/25414). [#25452](https://github.com/ClickHouse/ClickHouse/pull/25452) ([Amos Bird](https://github.com/amosbird)). * Support regular expression in `Database` argument for `StorageMerge`. Close [#776](https://github.com/ClickHouse/ClickHouse/issues/776). [#25064](https://github.com/ClickHouse/ClickHouse/pull/25064) ([flynn](https://github.com/ucasfl)). -* Support for multiple includes in configuration. It is possible to include users configuration, remote server configuration from multiple sources. Simply place `` element with `from_zk`, `from_env` or `incl` attribute, and it will be replaced with the substitution. [#24404](https://github.com/ClickHouse/ClickHouse/pull/24404) ([nvartolomei](https://github.com/nvartolomei)). +* Web UI: if the value looks like a URL, automatically generate a link. [#25965](https://github.com/ClickHouse/ClickHouse/pull/25965) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Make `sudo service clickhouse-server start` to work on systems with `systemd` like Centos 8. Close [#14298](https://github.com/ClickHouse/ClickHouse/issues/14298). Close [#17799](https://github.com/ClickHouse/ClickHouse/issues/17799). [#25921](https://github.com/ClickHouse/ClickHouse/pull/25921) ([alexey-milovidov](https://github.com/alexey-milovidov)). #### Bug Fixes * Fix incorrect `SET ROLE` in some cases. [#26707](https://github.com/ClickHouse/ClickHouse/pull/26707) ([Vitaly Baranov](https://github.com/vitlibar)). * Fix potential `nullptr` dereference in window functions. Fix [#25276](https://github.com/ClickHouse/ClickHouse/issues/25276). [#26668](https://github.com/ClickHouse/ClickHouse/pull/26668) ([Alexander Kuzmenkov](https://github.com/akuzm)). -* Fix empty history file conversion. [#26589](https://github.com/ClickHouse/ClickHouse/pull/26589) ([Azat Khuzhin](https://github.com/azat)). * Fix incorrect function names of `groupBitmapAnd/Or/Xor`. Fix [#26557](https://github.com/ClickHouse/ClickHouse/pull/26557) ([Amos Bird](https://github.com/amosbird)). -* Update `chown` cmd check in `clickhouse-server` docker entrypoint. It fixes error 'cluster pod restart failed (or timeout)' on kubernetes. [#26545](https://github.com/ClickHouse/ClickHouse/pull/26545) ([Ky Li](https://github.com/Kylinrix)). * Fix crash in rabbitmq shutdown in case rabbitmq setup was not started. Closes [#26504](https://github.com/ClickHouse/ClickHouse/issues/26504). [#26529](https://github.com/ClickHouse/ClickHouse/pull/26529) ([Kseniia Sumarokova](https://github.com/kssenii)). * Fix issues with `CREATE DICTIONARY` query if dictionary name or database name was quoted. Closes [#26491](https://github.com/ClickHouse/ClickHouse/issues/26491). [#26508](https://github.com/ClickHouse/ClickHouse/pull/26508) ([Maksim Kita](https://github.com/kitaisreal)). * Fix broken name resolution after rewriting column aliases. Fix [#26432](https://github.com/ClickHouse/ClickHouse/issues/26432). [#26475](https://github.com/ClickHouse/ClickHouse/pull/26475) ([Amos Bird](https://github.com/amosbird)). -* Fix some fuzzed msan crash. Fixes [#22517](https://github.com/ClickHouse/ClickHouse/issues/22517). [#26428](https://github.com/ClickHouse/ClickHouse/pull/26428) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). * Fix infinite non-joined block stream in `partial_merge_join` close [#26325](https://github.com/ClickHouse/ClickHouse/issues/26325). [#26374](https://github.com/ClickHouse/ClickHouse/pull/26374) ([Vladimir C](https://github.com/vdimir)). * Fix possible crash when login as dropped user. Fix [#26073](https://github.com/ClickHouse/ClickHouse/issues/26073). [#26363](https://github.com/ClickHouse/ClickHouse/pull/26363) ([Vitaly Baranov](https://github.com/vitlibar)). * Fix `optimize_distributed_group_by_sharding_key` for multiple columns (leads to incorrect result w/ `optimize_skip_unused_shards=1`/`allow_nondeterministic_optimize_skip_unused_shards=1` and multiple columns in sharding key expression). [#26353](https://github.com/ClickHouse/ClickHouse/pull/26353) ([Azat Khuzhin](https://github.com/azat)). + * `CAST` from `Date` to `DateTime` (or `DateTime64`) was not using the timezone of the `DateTime` type. It can also affect the comparison between `Date` and `DateTime`. Inference of the common type for `Date` and `DateTime` also was not using the corresponding timezone. It affected the results of function `if` and array construction. Closes [#24128](https://github.com/ClickHouse/ClickHouse/issues/24128). [#24129](https://github.com/ClickHouse/ClickHouse/pull/24129) ([Maksim Kita](https://github.com/kitaisreal)). * Fixed rare bug in lost replica recovery that may cause replicas to diverge. [#26321](https://github.com/ClickHouse/ClickHouse/pull/26321) ([tavplubix](https://github.com/tavplubix)). * Fix zstd decompression in case there are escape sequences at the end of internal buffer. Closes [#26013](https://github.com/ClickHouse/ClickHouse/issues/26013). [#26314](https://github.com/ClickHouse/ClickHouse/pull/26314) ([Kseniia Sumarokova](https://github.com/kssenii)). * Fix logical error on join with totals, close [#26017](https://github.com/ClickHouse/ClickHouse/issues/26017). [#26250](https://github.com/ClickHouse/ClickHouse/pull/26250) ([Vladimir C](https://github.com/vdimir)). @@ -99,7 +97,9 @@ * Fix slow dict join in some cases, close [#24209](https://github.com/ClickHouse/ClickHouse/issues/24209). [#25618](https://github.com/ClickHouse/ClickHouse/pull/25618) ([Vladimir C](https://github.com/vdimir)). * Fix `ALTER MODIFY COLUMN` of columns, which participates in TTL expressions. [#25554](https://github.com/ClickHouse/ClickHouse/pull/25554) ([Anton Popov](https://github.com/CurtizJ)). * Fix assertion in `PREWHERE` with non-UInt8 type, close [#19589](https://github.com/ClickHouse/ClickHouse/issues/19589). [#25484](https://github.com/ClickHouse/ClickHouse/pull/25484) ([Vladimir C](https://github.com/vdimir)). -* `CAST` from `Date` to `DateTime` (or `DateTime64`) was not using the timezone of the `DateTime` type. It can also affect the comparison between `Date` and `DateTime`. Inference of the common type for `Date` and `DateTime` also was not using the corresponding timezone. It affected the results of function `if` and array construction. Closes [#24128](https://github.com/ClickHouse/ClickHouse/issues/24128). [#24129](https://github.com/ClickHouse/ClickHouse/pull/24129) ([Maksim Kita](https://github.com/kitaisreal)). +* Fix some fuzzed msan crash. Fixes [#22517](https://github.com/ClickHouse/ClickHouse/issues/22517). [#26428](https://github.com/ClickHouse/ClickHouse/pull/26428) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix empty history file conversion. [#26589](https://github.com/ClickHouse/ClickHouse/pull/26589) ([Azat Khuzhin](https://github.com/azat)). +* Update `chown` cmd check in `clickhouse-server` docker entrypoint. It fixes error 'cluster pod restart failed (or timeout)' on kubernetes. [#26545](https://github.com/ClickHouse/ClickHouse/pull/26545) ([Ky Li](https://github.com/Kylinrix)). #### Build/Testing/Packaging Improvements From 70db20172ec91e9754e964a39eb980885d589bc4 Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 11 Aug 2021 09:58:14 +0300 Subject: [PATCH 432/599] 'Map type for system logs tables' is not backward incompatible in 21.8 changelog --- CHANGELOG.md | 6 +----- 1 file changed, 1 insertion(+), 5 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 0b8ca122582..c7c054a53a8 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,9 +1,5 @@ ### ClickHouse release v21.8, 2021-08-11 -#### Backward Incompatible Changes - -* Use `Map` data type for system logs tables (`system.query_log`, `system.query_thread_log`, `system.processes`, `system.OpenTelemetrySpanLog`). These tables will be auto-created with new data types; old queries may be incompatible. [#23934](https://github.com/ClickHouse/ClickHouse/pull/23934) ([hexiaoting](https://github.com/hexiaoting)). - #### New Features * Collect common system metrics (in `system.asynchronous_metrics` and `system.asynchronous_metric_log`) on CPU usage, disk usage, memory usage, IO, network, files, load average, CPU frequencies, thermal sensors, EDAC counters, system uptime; also added metrics about the scheduling jitter and the time spent collecting the metrics. It works similar to `atop` in ClickHouse and allows access to monitoring data even if you have no additional tools installed. Close [#9430](https://github.com/ClickHouse/ClickHouse/issues/9430). [#24416](https://github.com/ClickHouse/ClickHouse/pull/24416) ([Yegor Levankov](https://github.com/elevankoff)). @@ -25,6 +21,7 @@ #### Improvements +* Use `Map` data type for system logs tables (`system.query_log`, `system.query_thread_log`, `system.processes`, `system.opentelemetry_span_log`). These tables will be auto-created with new data types. Virtual columns are created to support old queries. Closes [#18698](https://github.com/ClickHouse/ClickHouse/issues/18698). [#23934](https://github.com/ClickHouse/ClickHouse/pull/23934), [#25773](https://github.com/ClickHouse/ClickHouse/pull/25773) ([hexiaoting](https://github.com/hexiaoting), [sundy-li](https://github.com/sundy-li)). * For a dictionary with a complex key containing only one attribute, allow not wrapping the key expression in tuple for functions `dictGet`, `dictHas`. [#26130](https://github.com/ClickHouse/ClickHouse/pull/26130) ([Maksim Kita](https://github.com/kitaisreal)). * Implement function `bin`/`hex` from `AggregateFunction` states. [#26094](https://github.com/ClickHouse/ClickHouse/pull/26094) ([zhaoyu](https://github.com/zxc111)). * Support arguments of `UUID` type for `empty` and `notEmpty` functions. `UUID` is empty if it is all zeros (nil UUID). Closes [#3446](https://github.com/ClickHouse/ClickHouse/issues/3446). [#25974](https://github.com/ClickHouse/ClickHouse/pull/25974) ([zhaoyu](https://github.com/zxc111)). @@ -46,7 +43,6 @@ * Fix "No available columns" error for `Merge` storage. [#25801](https://github.com/ClickHouse/ClickHouse/pull/25801) ([Azat Khuzhin](https://github.com/azat)). * MySQL Engine now supports the exchange of column comments between MySQL and ClickHouse. [#25795](https://github.com/ClickHouse/ClickHouse/pull/25795) ([Storozhuk Kostiantyn](https://github.com/sand6255)). * Fix inconsistent behaviour of `GROUP BY` constant on empty set. Closes [#6842](https://github.com/ClickHouse/ClickHouse/issues/6842). [#25786](https://github.com/ClickHouse/ClickHouse/pull/25786) ([Kseniia Sumarokova](https://github.com/kssenii)). -* Use `Map` data type for key-value dictionaries in system logs tables (`system.query_log`, `system.query_thread_log`, `system.processes`, `system.opentelemetry_span_log`). Virtual columns are created to support old queries. Closes [#18698](https://github.com/ClickHouse/ClickHouse/issues/18698). Authors @hexiaoting, @sundy-li. [#25773](https://github.com/ClickHouse/ClickHouse/pull/25773) ([Maksim Kita](https://github.com/kitaisreal)). * Cancel already running merges in partition on `DROP PARTITION` and `TRUNCATE` for `ReplicatedMergeTree`. Resolves [#17151](https://github.com/ClickHouse/ClickHouse/issues/17151). [#25684](https://github.com/ClickHouse/ClickHouse/pull/25684) ([tavplubix](https://github.com/tavplubix)). * Support ENUM` data type for MaterializeMySQL. [#25676](https://github.com/ClickHouse/ClickHouse/pull/25676) ([Storozhuk Kostiantyn](https://github.com/sand6255)). * Support materialized and aliased columns in JOIN, close [#13274](https://github.com/ClickHouse/ClickHouse/issues/13274). [#25634](https://github.com/ClickHouse/ClickHouse/pull/25634) ([Vladimir C](https://github.com/vdimir)). From ec4e8ca5945694e3bcab98e9e77ae22db489bb15 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 11 Aug 2021 10:03:46 +0300 Subject: [PATCH 433/599] Firstly write current exception and then reopen block devices --- src/Interpreters/AsynchronousMetrics.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/AsynchronousMetrics.cpp b/src/Interpreters/AsynchronousMetrics.cpp index 8efe959a623..f041d604516 100644 --- a/src/Interpreters/AsynchronousMetrics.cpp +++ b/src/Interpreters/AsynchronousMetrics.cpp @@ -967,6 +967,8 @@ void AsynchronousMetrics::update(std::chrono::system_clock::time_point update_ti } catch (...) { + tryLogCurrentException(__PRETTY_FUNCTION__); + /// Try to reopen block devices in case of error /// (i.e. ENOENT means that some disk had been replaced, and it may apperas with a new name) try @@ -977,7 +979,6 @@ void AsynchronousMetrics::update(std::chrono::system_clock::time_point update_ti { tryLogCurrentException(__PRETTY_FUNCTION__); } - tryLogCurrentException(__PRETTY_FUNCTION__); } if (net_dev) From eed5052a86a3cc4451df6a0cc9a48a30ceefe029 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 11 Aug 2021 09:54:55 +0300 Subject: [PATCH 434/599] Reopen hwmon sensors on error (/sys/class/hwmon) Sensors maybe recreated (i.e. on module load/unload, or suspend/resume cycle), so descriptors should be reopened. --- src/Interpreters/AsynchronousMetrics.cpp | 135 +++++++++++++---------- src/Interpreters/AsynchronousMetrics.h | 1 + 2 files changed, 79 insertions(+), 57 deletions(-) diff --git a/src/Interpreters/AsynchronousMetrics.cpp b/src/Interpreters/AsynchronousMetrics.cpp index f041d604516..cab87054902 100644 --- a/src/Interpreters/AsynchronousMetrics.cpp +++ b/src/Interpreters/AsynchronousMetrics.cpp @@ -102,6 +102,69 @@ AsynchronousMetrics::AsynchronousMetrics( thermal.emplace_back(std::move(file)); } + for (size_t edac_index = 0;; ++edac_index) + { + String edac_correctable_file = fmt::format("/sys/devices/system/edac/mc/mc{}/ce_count", edac_index); + String edac_uncorrectable_file = fmt::format("/sys/devices/system/edac/mc/mc{}/ue_count", edac_index); + + bool edac_correctable_file_exists = std::filesystem::exists(edac_correctable_file); + bool edac_uncorrectable_file_exists = std::filesystem::exists(edac_uncorrectable_file); + + if (!edac_correctable_file_exists && !edac_uncorrectable_file_exists) + { + if (edac_index == 0) + continue; + else + break; + } + + edac.emplace_back(); + + if (edac_correctable_file_exists) + edac.back().first = openFileIfExists(edac_correctable_file); + if (edac_uncorrectable_file_exists) + edac.back().second = openFileIfExists(edac_uncorrectable_file); + } + + openBlockDevices(); + openSensorsChips(); +#endif +} + +#if defined(OS_LINUX) +void AsynchronousMetrics::openBlockDevices() +{ + LOG_TRACE(log, "Scanning /sys/block"); + + if (!std::filesystem::exists("/sys/block")) + return; + + block_devices_rescan_delay.restart(); + + block_devs.clear(); + + for (const auto & device_dir : std::filesystem::directory_iterator("/sys/block")) + { + String device_name = device_dir.path().filename(); + + /// We are not interested in loopback devices. + if (device_name.starts_with("loop")) + continue; + + std::unique_ptr file = openFileIfExists(device_dir.path() / "stat"); + if (!file) + continue; + + block_devs[device_name] = std::move(file); + } +} + +void AsynchronousMetrics::openSensorsChips() +{ + LOG_TRACE(log, "Scanning /sys/class/hwmon"); + + hwmon_devices.clear(); + for (size_t hwmon_index = 0;; ++hwmon_index) { String hwmon_name_file = fmt::format("/sys/class/hwmon/hwmon{}/name", hwmon_index); @@ -150,61 +213,6 @@ AsynchronousMetrics::AsynchronousMetrics( hwmon_devices[hwmon_name][sensor_name] = std::move(file); } } - - for (size_t edac_index = 0;; ++edac_index) - { - String edac_correctable_file = fmt::format("/sys/devices/system/edac/mc/mc{}/ce_count", edac_index); - String edac_uncorrectable_file = fmt::format("/sys/devices/system/edac/mc/mc{}/ue_count", edac_index); - - bool edac_correctable_file_exists = std::filesystem::exists(edac_correctable_file); - bool edac_uncorrectable_file_exists = std::filesystem::exists(edac_uncorrectable_file); - - if (!edac_correctable_file_exists && !edac_uncorrectable_file_exists) - { - if (edac_index == 0) - continue; - else - break; - } - - edac.emplace_back(); - - if (edac_correctable_file_exists) - edac.back().first = openFileIfExists(edac_correctable_file); - if (edac_uncorrectable_file_exists) - edac.back().second = openFileIfExists(edac_uncorrectable_file); - } - - openBlockDevices(); -#endif -} - -#if defined(OS_LINUX) -void AsynchronousMetrics::openBlockDevices() -{ - LOG_TRACE(log, "Scanning /sys/block"); - - if (!std::filesystem::exists("/sys/block")) - return; - - block_devices_rescan_delay.restart(); - - block_devs.clear(); - - for (const auto & device_dir : std::filesystem::directory_iterator("/sys/block")) - { - String device_name = device_dir.path().filename(); - - /// We are not interested in loopback devices. - if (device_name.starts_with("loop")) - continue; - - std::unique_ptr file = openFileIfExists(device_dir.path() / "stat"); - if (!file) - continue; - - block_devs[device_name] = std::move(file); - } } #endif @@ -1084,9 +1092,9 @@ void AsynchronousMetrics::update(std::chrono::system_clock::time_point update_ti } } - for (const auto & [hwmon_name, sensors] : hwmon_devices) + try { - try + for (const auto & [hwmon_name, sensors] : hwmon_devices) { for (const auto & [sensor_name, sensor_file] : sensors) { @@ -1107,6 +1115,19 @@ void AsynchronousMetrics::update(std::chrono::system_clock::time_point update_ti new_values[fmt::format("Temperature_{}_{}", hwmon_name, sensor_name)] = temperature * 0.001; } } + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + + /// Files can be re-created on: + /// - module load/unload + /// - suspend/resume cycle + /// So file descriptors should be reopened. + try + { + openSensorsChips(); + } catch (...) { tryLogCurrentException(__PRETTY_FUNCTION__); diff --git a/src/Interpreters/AsynchronousMetrics.h b/src/Interpreters/AsynchronousMetrics.h index c8677ac3ced..a5d7f2ab98f 100644 --- a/src/Interpreters/AsynchronousMetrics.h +++ b/src/Interpreters/AsynchronousMetrics.h @@ -184,6 +184,7 @@ private: Stopwatch block_devices_rescan_delay; void openBlockDevices(); + void openSensorsChips(); #endif std::unique_ptr thread; From 3f91f61c3c4e8122b38bbe60bcbe6cde2b181501 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 11 Aug 2021 10:02:34 +0300 Subject: [PATCH 435/599] Reopen EDAC files (/sys/devices/system/edac) --- src/Interpreters/AsynchronousMetrics.cpp | 76 +++++++++++++++--------- src/Interpreters/AsynchronousMetrics.h | 1 + 2 files changed, 48 insertions(+), 29 deletions(-) diff --git a/src/Interpreters/AsynchronousMetrics.cpp b/src/Interpreters/AsynchronousMetrics.cpp index cab87054902..bf0bbe804fe 100644 --- a/src/Interpreters/AsynchronousMetrics.cpp +++ b/src/Interpreters/AsynchronousMetrics.cpp @@ -102,32 +102,9 @@ AsynchronousMetrics::AsynchronousMetrics( thermal.emplace_back(std::move(file)); } - for (size_t edac_index = 0;; ++edac_index) - { - String edac_correctable_file = fmt::format("/sys/devices/system/edac/mc/mc{}/ce_count", edac_index); - String edac_uncorrectable_file = fmt::format("/sys/devices/system/edac/mc/mc{}/ue_count", edac_index); - - bool edac_correctable_file_exists = std::filesystem::exists(edac_correctable_file); - bool edac_uncorrectable_file_exists = std::filesystem::exists(edac_uncorrectable_file); - - if (!edac_correctable_file_exists && !edac_uncorrectable_file_exists) - { - if (edac_index == 0) - continue; - else - break; - } - - edac.emplace_back(); - - if (edac_correctable_file_exists) - edac.back().first = openFileIfExists(edac_correctable_file); - if (edac_uncorrectable_file_exists) - edac.back().second = openFileIfExists(edac_uncorrectable_file); - } - openBlockDevices(); openSensorsChips(); + openEDAC(); #endif } @@ -159,6 +136,37 @@ void AsynchronousMetrics::openBlockDevices() } } +void AsynchronousMetrics::openEDAC() +{ + LOG_TRACE(log, "Scanning /sys/devices/system/edac"); + + edac.clear(); + + for (size_t edac_index = 0;; ++edac_index) + { + String edac_correctable_file = fmt::format("/sys/devices/system/edac/mc/mc{}/ce_count", edac_index); + String edac_uncorrectable_file = fmt::format("/sys/devices/system/edac/mc/mc{}/ue_count", edac_index); + + bool edac_correctable_file_exists = std::filesystem::exists(edac_correctable_file); + bool edac_uncorrectable_file_exists = std::filesystem::exists(edac_uncorrectable_file); + + if (!edac_correctable_file_exists && !edac_uncorrectable_file_exists) + { + if (edac_index == 0) + continue; + else + break; + } + + edac.emplace_back(); + + if (edac_correctable_file_exists) + edac.back().first = openFileIfExists(edac_correctable_file); + if (edac_uncorrectable_file_exists) + edac.back().second = openFileIfExists(edac_uncorrectable_file); + } +} + void AsynchronousMetrics::openSensorsChips() { LOG_TRACE(log, "Scanning /sys/class/hwmon"); @@ -1134,13 +1142,13 @@ void AsynchronousMetrics::update(std::chrono::system_clock::time_point update_ti } } - for (size_t i = 0, size = edac.size(); i < size; ++i) + try { - /// NOTE maybe we need to take difference with previous values. - /// But these metrics should be exceptionally rare, so it's ok to keep them accumulated. - - try + for (size_t i = 0, size = edac.size(); i < size; ++i) { + /// NOTE maybe we need to take difference with previous values. + /// But these metrics should be exceptionally rare, so it's ok to keep them accumulated. + if (edac[i].first) { ReadBufferFromFilePRead & in = *edac[i].first; @@ -1159,6 +1167,16 @@ void AsynchronousMetrics::update(std::chrono::system_clock::time_point update_ti new_values[fmt::format("EDAC{}_Uncorrectable", i)] = errors; } } + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + + /// EDAC files can be re-created on module load/unload + try + { + openEDAC(); + } catch (...) { tryLogCurrentException(__PRETTY_FUNCTION__); diff --git a/src/Interpreters/AsynchronousMetrics.h b/src/Interpreters/AsynchronousMetrics.h index a5d7f2ab98f..409f2dfeec4 100644 --- a/src/Interpreters/AsynchronousMetrics.h +++ b/src/Interpreters/AsynchronousMetrics.h @@ -185,6 +185,7 @@ private: void openBlockDevices(); void openSensorsChips(); + void openEDAC(); #endif std::unique_ptr thread; From 67ebcef978764220a881f9c1d2c9a354ef87bd05 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 11 Aug 2021 10:09:00 +0300 Subject: [PATCH 436/599] Reopen sensors (/sys/class/thermal) --- src/Interpreters/AsynchronousMetrics.cpp | 34 ++++++++++++++++++------ src/Interpreters/AsynchronousMetrics.h | 1 + 2 files changed, 27 insertions(+), 8 deletions(-) diff --git a/src/Interpreters/AsynchronousMetrics.cpp b/src/Interpreters/AsynchronousMetrics.cpp index bf0bbe804fe..fd02aa4abec 100644 --- a/src/Interpreters/AsynchronousMetrics.cpp +++ b/src/Interpreters/AsynchronousMetrics.cpp @@ -88,6 +88,20 @@ AsynchronousMetrics::AsynchronousMetrics( openFileIfExists("/proc/uptime", uptime); openFileIfExists("/proc/net/dev", net_dev); + openSensors(); + openBlockDevices(); + openEDAC(); + openSensorsChips(); +#endif +} + +#if defined(OS_LINUX) +void AsynchronousMetrics::openSensors() +{ + LOG_TRACE(log, "Scanning /sys/class/thermal"); + + thermal.clear(); + for (size_t thermal_device_index = 0;; ++thermal_device_index) { std::unique_ptr file = openFileIfExists(fmt::format("/sys/class/thermal/thermal_zone{}/temp", thermal_device_index)); @@ -101,14 +115,8 @@ AsynchronousMetrics::AsynchronousMetrics( } thermal.emplace_back(std::move(file)); } - - openBlockDevices(); - openSensorsChips(); - openEDAC(); -#endif } -#if defined(OS_LINUX) void AsynchronousMetrics::openBlockDevices() { LOG_TRACE(log, "Scanning /sys/block"); @@ -1083,9 +1091,9 @@ void AsynchronousMetrics::update(std::chrono::system_clock::time_point update_ti } } - for (size_t i = 0, size = thermal.size(); i < size; ++i) + try { - try + for (size_t i = 0, size = thermal.size(); i < size; ++i) { ReadBufferFromFilePRead & in = *thermal[i]; @@ -1094,6 +1102,16 @@ void AsynchronousMetrics::update(std::chrono::system_clock::time_point update_ti readText(temperature, in); new_values[fmt::format("Temperature{}", i)] = temperature * 0.001; } + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + + /// Files maybe re-created on module load/unload + try + { + openSensors(); + } catch (...) { tryLogCurrentException(__PRETTY_FUNCTION__); diff --git a/src/Interpreters/AsynchronousMetrics.h b/src/Interpreters/AsynchronousMetrics.h index 409f2dfeec4..93e77b6bde8 100644 --- a/src/Interpreters/AsynchronousMetrics.h +++ b/src/Interpreters/AsynchronousMetrics.h @@ -183,6 +183,7 @@ private: Stopwatch block_devices_rescan_delay; + void openSensors(); void openBlockDevices(); void openSensorsChips(); void openEDAC(); From 7f247becca1e483489ff46d4af28b3437693f1c9 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 11 Aug 2021 07:32:46 +0000 Subject: [PATCH 437/599] Move some templates --- src/Functions/FunctionsConversion.cpp | 1 + src/Functions/FunctionsConversion.h | 17 ----------------- src/Functions/cast_overload_fwd.h | 24 ++++++++++++++++++++++++ src/Interpreters/ActionsDAG.cpp | 1 + src/Interpreters/castColumn.cpp | 1 + src/Storages/MergeTree/KeyCondition.cpp | 1 + 6 files changed, 28 insertions(+), 17 deletions(-) create mode 100644 src/Functions/cast_overload_fwd.h diff --git a/src/Functions/FunctionsConversion.cpp b/src/Functions/FunctionsConversion.cpp index cdbd32b189c..ae115a6e90d 100644 --- a/src/Functions/FunctionsConversion.cpp +++ b/src/Functions/FunctionsConversion.cpp @@ -1,5 +1,6 @@ #include #include +#include namespace DB diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index 362249b3b27..941a2fae0ab 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -3485,21 +3485,4 @@ private: }; -struct CastOverloadName -{ - static constexpr auto cast_name = "CAST"; - static constexpr auto accurate_cast_name = "accurateCast"; - static constexpr auto accurate_cast_or_null_name = "accurateCastOrNull"; -}; - -struct CastInternalOverloadName -{ - static constexpr auto cast_name = "_CAST"; - static constexpr auto accurate_cast_name = "accurate_Cast"; - static constexpr auto accurate_cast_or_null_name = "accurate_CastOrNull"; -}; - -template using CastOverloadResolver = CastOverloadResolverImpl; -template using CastInternalOverloadResolver = CastOverloadResolverImpl; - } diff --git a/src/Functions/cast_overload_fwd.h b/src/Functions/cast_overload_fwd.h new file mode 100644 index 00000000000..1264d01b5d9 --- /dev/null +++ b/src/Functions/cast_overload_fwd.h @@ -0,0 +1,24 @@ +#pragma once +#include + +namespace DB +{ + +struct CastOverloadName +{ + static constexpr auto cast_name = "CAST"; + static constexpr auto accurate_cast_name = "accurateCast"; + static constexpr auto accurate_cast_or_null_name = "accurateCastOrNull"; +}; + +struct CastInternalOverloadName +{ + static constexpr auto cast_name = "_CAST"; + static constexpr auto accurate_cast_name = "accurate_Cast"; + static constexpr auto accurate_cast_or_null_name = "accurate_CastOrNull"; +}; + +template using CastOverloadResolver = CastOverloadResolverImpl; +template using CastInternalOverloadResolver = CastOverloadResolverImpl; + +} diff --git a/src/Interpreters/ActionsDAG.cpp b/src/Interpreters/ActionsDAG.cpp index a42d6053e9a..664b0d61a4f 100644 --- a/src/Interpreters/ActionsDAG.cpp +++ b/src/Interpreters/ActionsDAG.cpp @@ -7,6 +7,7 @@ #include #include #include +#include #include #include #include diff --git a/src/Interpreters/castColumn.cpp b/src/Interpreters/castColumn.cpp index 3356d37ba7f..4c3c564ce58 100644 --- a/src/Interpreters/castColumn.cpp +++ b/src/Interpreters/castColumn.cpp @@ -1,6 +1,7 @@ #include #include +#include namespace DB { diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index e6dfdb859b8..751ac9fd9d9 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -11,6 +11,7 @@ #include #include #include +#include #include #include #include From 30a95e77a71dffcd2aaebf6e7ffde38d0db61671 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Thu, 29 Jul 2021 23:57:06 +0300 Subject: [PATCH 438/599] Database dictionaries custom query support --- .../CassandraDictionarySource.cpp | 30 +- src/Dictionaries/CassandraDictionarySource.h | 50 +- .../ClickHouseDictionarySource.cpp | 7 +- src/Dictionaries/ClickHouseDictionarySource.h | 1 + src/Dictionaries/ExternalQueryBuilder.cpp | 486 +++++++++++------- src/Dictionaries/ExternalQueryBuilder.h | 21 +- src/Dictionaries/MySQLDictionarySource.cpp | 21 +- src/Dictionaries/MySQLDictionarySource.h | 1 + .../PostgreSQLDictionarySource.cpp | 9 +- src/Dictionaries/PostgreSQLDictionarySource.h | 1 + src/Dictionaries/XDBCDictionarySource.cpp | 10 +- src/Dictionaries/XDBCDictionarySource.h | 1 + 12 files changed, 390 insertions(+), 248 deletions(-) diff --git a/src/Dictionaries/CassandraDictionarySource.cpp b/src/Dictionaries/CassandraDictionarySource.cpp index 8b31b4d6fa2..98928312ec8 100644 --- a/src/Dictionaries/CassandraDictionarySource.cpp +++ b/src/Dictionaries/CassandraDictionarySource.cpp @@ -49,7 +49,7 @@ namespace ErrorCodes extern const int INVALID_CONFIG_PARAMETER; } -CassandraSettings::CassandraSettings( +CassandraDictionarySource::Configuration::Configuration( const Poco::Util::AbstractConfiguration & config, const String & config_prefix) : host(config.getString(config_prefix + ".host")) @@ -66,7 +66,7 @@ CassandraSettings::CassandraSettings( setConsistency(config.getString(config_prefix + ".consistency", "One")); } -void CassandraSettings::setConsistency(const String & config_str) +void CassandraDictionarySource::Configuration::setConsistency(const String & config_str) { if (config_str == "One") consistency = CASS_CONSISTENCY_ONE; @@ -96,19 +96,19 @@ static const size_t max_block_size = 8192; CassandraDictionarySource::CassandraDictionarySource( const DictionaryStructure & dict_struct_, - const CassandraSettings & settings_, + const Configuration & configuration_, const Block & sample_block_) : log(&Poco::Logger::get("CassandraDictionarySource")) , dict_struct(dict_struct_) - , settings(settings_) + , configuration(configuration_) , sample_block(sample_block_) - , query_builder(dict_struct, settings.db, "", settings.table, settings.where, IdentifierQuotingStyle::DoubleQuotes) + , query_builder(dict_struct, configuration.db, "", configuration.table, configuration.query, configuration.where, IdentifierQuotingStyle::DoubleQuotes) { - cassandraCheck(cass_cluster_set_contact_points(cluster, settings.host.c_str())); - if (settings.port) - cassandraCheck(cass_cluster_set_port(cluster, settings.port)); - cass_cluster_set_credentials(cluster, settings.user.c_str(), settings.password.c_str()); - cassandraCheck(cass_cluster_set_consistency(cluster, settings.consistency)); + cassandraCheck(cass_cluster_set_contact_points(cluster, configuration.host.c_str())); + if (configuration.port) + cassandraCheck(cass_cluster_set_port(cluster, configuration.port)); + cass_cluster_set_credentials(cluster, configuration.user.c_str(), configuration.password.c_str()); + cassandraCheck(cass_cluster_set_consistency(cluster, configuration.consistency)); } CassandraDictionarySource::CassandraDictionarySource( @@ -118,14 +118,14 @@ CassandraDictionarySource::CassandraDictionarySource( Block & sample_block_) : CassandraDictionarySource( dict_struct_, - CassandraSettings(config, config_prefix), + Configuration(config, config_prefix), sample_block_) { } void CassandraDictionarySource::maybeAllowFiltering(String & query) const { - if (!settings.allow_filtering) + if (!configuration.allow_filtering) return; query.pop_back(); /// remove semicolon query += " ALLOW FILTERING;"; @@ -141,7 +141,7 @@ Pipe CassandraDictionarySource::loadAll() std::string CassandraDictionarySource::toString() const { - return "Cassandra: " + settings.db + '.' + settings.table; + return "Cassandra: " + configuration.db + '.' + configuration.table; } Pipe CassandraDictionarySource::loadIds(const std::vector & ids) @@ -162,7 +162,7 @@ Pipe CassandraDictionarySource::loadKeys(const Columns & key_columns, const std: for (const auto & row : requested_rows) { SipHash partition_key; - for (size_t i = 0; i < settings.partition_key_prefix; ++i) + for (size_t i = 0; i < configuration.partition_key_prefix; ++i) key_columns[i]->updateHashWithValue(row, partition_key); partitions[partition_key.get64()].push_back(row); } @@ -170,7 +170,7 @@ Pipe CassandraDictionarySource::loadKeys(const Columns & key_columns, const std: Pipes pipes; for (const auto & partition : partitions) { - String query = query_builder.composeLoadKeysQuery(key_columns, partition.second, ExternalQueryBuilder::CASSANDRA_SEPARATE_PARTITION_KEY, settings.partition_key_prefix); + String query = query_builder.composeLoadKeysQuery(key_columns, partition.second, ExternalQueryBuilder::CASSANDRA_SEPARATE_PARTITION_KEY, configuration.partition_key_prefix); maybeAllowFiltering(query); LOG_INFO(log, "Loading keys for partition hash {} using query: {}", partition.first, query); pipes.push_back(Pipe(std::make_shared(getSession(), query, sample_block, max_block_size))); diff --git a/src/Dictionaries/CassandraDictionarySource.h b/src/Dictionaries/CassandraDictionarySource.h index 871e3dc4857..35419d3ea7d 100644 --- a/src/Dictionaries/CassandraDictionarySource.h +++ b/src/Dictionaries/CassandraDictionarySource.h @@ -14,33 +14,35 @@ namespace DB { -struct CassandraSettings -{ - String host; - UInt16 port; - String user; - String password; - String db; - String table; - - CassConsistency consistency; - bool allow_filtering; - /// TODO get information about key from the driver - size_t partition_key_prefix; - size_t max_threads; - String where; - - CassandraSettings(const Poco::Util::AbstractConfiguration & config, const String & config_prefix); - - void setConsistency(const String & config_str); -}; - class CassandraDictionarySource final : public IDictionarySource { public: + + struct Configuration + { + String host; + UInt16 port; + String user; + String password; + String db; + String table; + String query; + + CassConsistency consistency; + bool allow_filtering; + /// TODO get information about key from the driver + size_t partition_key_prefix; + size_t max_threads; + String where; + + Configuration(const Poco::Util::AbstractConfiguration & config, const String & config_prefix); + + void setConsistency(const String & config_str); + }; + CassandraDictionarySource( const DictionaryStructure & dict_struct, - const CassandraSettings & settings_, + const Configuration & configuration, const Block & sample_block); CassandraDictionarySource( @@ -59,7 +61,7 @@ public: DictionarySourcePtr clone() const override { - return std::make_unique(dict_struct, settings, sample_block); + return std::make_unique(dict_struct, configuration, sample_block); } Pipe loadIds(const std::vector & ids) override; @@ -76,7 +78,7 @@ private: Poco::Logger * log; const DictionaryStructure dict_struct; - const CassandraSettings settings; + const Configuration configuration; Block sample_block; ExternalQueryBuilder query_builder; diff --git a/src/Dictionaries/ClickHouseDictionarySource.cpp b/src/Dictionaries/ClickHouseDictionarySource.cpp index 8b2373302c8..0f085a7c1a2 100644 --- a/src/Dictionaries/ClickHouseDictionarySource.cpp +++ b/src/Dictionaries/ClickHouseDictionarySource.cpp @@ -67,7 +67,7 @@ ClickHouseDictionarySource::ClickHouseDictionarySource( : update_time{std::chrono::system_clock::from_time_t(0)} , dict_struct{dict_struct_} , configuration{configuration_} - , query_builder{dict_struct, configuration.db, "", configuration.table, configuration.where, IdentifierQuotingStyle::Backticks} + , query_builder{dict_struct, configuration.db, "", configuration.table, configuration.query, configuration.where, IdentifierQuotingStyle::Backticks} , sample_block{sample_block_} , context(Context::createCopy(context_)) , pool{createPool(configuration)} @@ -83,7 +83,7 @@ ClickHouseDictionarySource::ClickHouseDictionarySource(const ClickHouseDictionar , dict_struct{other.dict_struct} , configuration{other.configuration} , invalidate_query_response{other.invalidate_query_response} - , query_builder{dict_struct, configuration.db, "", configuration.table, configuration.where, IdentifierQuotingStyle::Backticks} + , query_builder{dict_struct, configuration.db, "", configuration.table, configuration.query, configuration.where, IdentifierQuotingStyle::Backticks} , sample_block{other.sample_block} , context(Context::createCopy(other.context)) , pool{createPool(configuration)} @@ -241,7 +241,8 @@ void registerDictionarySourceClickHouse(DictionarySourceFactory & factory) .user = config.getString(settings_config_prefix + ".user", "default"), .password = config.getString(settings_config_prefix + ".password", ""), .db = config.getString(settings_config_prefix + ".db", default_database), - .table = config.getString(settings_config_prefix + ".table"), + .table = config.getString(settings_config_prefix + ".table", ""), + .query = config.getString(settings_config_prefix + ".query", ""), .where = config.getString(settings_config_prefix + ".where", ""), .invalidate_query = config.getString(settings_config_prefix + ".invalidate_query", ""), .update_field = config.getString(settings_config_prefix + ".update_field", ""), diff --git a/src/Dictionaries/ClickHouseDictionarySource.h b/src/Dictionaries/ClickHouseDictionarySource.h index f293c010ec3..2daa296af3e 100644 --- a/src/Dictionaries/ClickHouseDictionarySource.h +++ b/src/Dictionaries/ClickHouseDictionarySource.h @@ -25,6 +25,7 @@ public: const std::string password; const std::string db; const std::string table; + const std::string query; const std::string where; const std::string invalidate_query; const std::string update_field; diff --git a/src/Dictionaries/ExternalQueryBuilder.cpp b/src/Dictionaries/ExternalQueryBuilder.cpp index e0920535e33..deba95c9d02 100644 --- a/src/Dictionaries/ExternalQueryBuilder.cpp +++ b/src/Dictionaries/ExternalQueryBuilder.cpp @@ -21,10 +21,23 @@ ExternalQueryBuilder::ExternalQueryBuilder( const std::string & db_, const std::string & schema_, const std::string & table_, + const std::string & query_, const std::string & where_, IdentifierQuotingStyle quoting_style_) - : dict_struct(dict_struct_), db(db_), schema(schema_), table(table_), where(where_), quoting_style(quoting_style_) -{} + : dict_struct(dict_struct_) + , db(db_) + , schema(schema_) + , table(table_) + , query(query_) + , where(where_) + , quoting_style(quoting_style_) +{ + if (table.empty() && query.empty()) + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Setting `table` or `query` must be non empty"); + + if (!query.empty() && !where.empty()) + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Setting `where` is not supported with `query` parameter"); +} void ExternalQueryBuilder::writeQuoted(const std::string & s, WriteBuffer & out) const @@ -152,74 +165,314 @@ void ExternalQueryBuilder::composeLoadAllQuery(WriteBuffer & out) const std::string ExternalQueryBuilder::composeUpdateQuery(const std::string & update_field, const std::string & time_point) const { WriteBufferFromOwnString out; - composeLoadAllQuery(out); - if (!where.empty()) - writeString(" AND ", out); + if (query.empty()) + { + composeLoadAllQuery(out); + + if (!where.empty()) + writeString(" AND ", out); + else + writeString(" WHERE ", out); + + composeUpdateCondition(update_field, time_point, out); + + writeChar(';', out); + + return out.str(); + } else - writeString(" WHERE ", out); + { + writeString(query, out); - writeString(update_field, out); - writeString(" >= '", out); - writeString(time_point, out); - writeChar('\'', out); + auto condition_position = query.find("{condition}"); + if (condition_position == std::string::npos) + { + writeString(" WHERE ", out); + composeUpdateCondition(update_field, time_point, out); + writeString(";", out); - writeChar(';', out); - return out.str(); + return out.str(); + } + + WriteBufferFromOwnString condition_value_buffer; + composeUpdateCondition(update_field, time_point, condition_value_buffer); + const auto & condition_value = condition_value_buffer.str(); + + auto query_copy = query; + query_copy.replace(condition_position, condition_value.size(), condition_value); + + return query_copy; + } } -std::string ExternalQueryBuilder::composeLoadIdsQuery(const std::vector & ids) +std::string ExternalQueryBuilder::composeLoadIdsQuery(const std::vector & ids) const { if (!dict_struct.id) throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Simple key required for method"); WriteBufferFromOwnString out; - writeString("SELECT ", out); - if (!dict_struct.id->expression.empty()) + if (query.empty()) { - writeParenthesisedString(dict_struct.id->expression, out); - writeString(" AS ", out); - } + writeString("SELECT ", out); - writeQuoted(dict_struct.id->name, out); - - for (const auto & attr : dict_struct.attributes) - { - writeString(", ", out); - - if (!attr.expression.empty()) + if (!dict_struct.id->expression.empty()) { - writeParenthesisedString(attr.expression, out); + writeParenthesisedString(dict_struct.id->expression, out); writeString(" AS ", out); } - writeQuoted(attr.name, out); - } + writeQuoted(dict_struct.id->name, out); - writeString(" FROM ", out); - if (!db.empty()) + for (const auto & attr : dict_struct.attributes) + { + writeString(", ", out); + + if (!attr.expression.empty()) + { + writeParenthesisedString(attr.expression, out); + writeString(" AS ", out); + } + + writeQuoted(attr.name, out); + } + + writeString(" FROM ", out); + if (!db.empty()) + { + writeQuoted(db, out); + writeChar('.', out); + } + if (!schema.empty()) + { + writeQuoted(schema, out); + writeChar('.', out); + } + + writeQuoted(table, out); + + writeString(" WHERE ", out); + + if (!where.empty()) + { + writeString(where, out); + writeString(" AND ", out); + } + + composeIdsCondition(ids, out); + writeString(";", out); + + return out.str(); + } + else { - writeQuoted(db, out); - writeChar('.', out); + writeString(query, out); + + auto condition_position = query.find("{condition}"); + if (condition_position == std::string::npos) + { + writeString(" WHERE ", out); + composeIdsCondition(ids, out); + writeString(";", out); + + return out.str(); + } + + WriteBufferFromOwnString condition_value_buffer; + composeIdsCondition(ids, condition_value_buffer); + const auto & condition_value = condition_value_buffer.str(); + + auto query_copy = query; + query_copy.replace(condition_position, condition_value.size(), condition_value); + + return query_copy; } - if (!schema.empty()) +} + + +std::string ExternalQueryBuilder::composeLoadKeysQuery( + const Columns & key_columns, const std::vector & requested_rows, LoadKeysMethod method, size_t partition_key_prefix) const +{ + if (!dict_struct.key) + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Composite key required for method"); + + if (key_columns.size() != dict_struct.key->size()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "The size of key_columns does not equal to the size of dictionary key"); + + WriteBufferFromOwnString out; + + if (query.empty()) { - writeQuoted(schema, out); - writeChar('.', out); + writeString("SELECT ", out); + + auto first = true; + for (const auto & key_or_attribute : boost::join(*dict_struct.key, dict_struct.attributes)) + { + if (!first) + writeString(", ", out); + + first = false; + + if (!key_or_attribute.expression.empty()) + { + writeParenthesisedString(key_or_attribute.expression, out); + writeString(" AS ", out); + } + + writeQuoted(key_or_attribute.name, out); + } + + writeString(" FROM ", out); + if (!db.empty()) + { + writeQuoted(db, out); + writeChar('.', out); + } + if (!schema.empty()) + { + writeQuoted(schema, out); + writeChar('.', out); + } + + writeQuoted(table, out); + + writeString(" WHERE ", out); + + if (!where.empty()) + { + if (method != CASSANDRA_SEPARATE_PARTITION_KEY) + writeString("(", out); + writeString(where, out); + if (method != CASSANDRA_SEPARATE_PARTITION_KEY) + writeString(") AND (", out); + else + writeString(" AND ", out); + } + + composeKeysCondition(key_columns, requested_rows, method, partition_key_prefix, out); + + writeString(";", out); + + return out.str(); } - - writeQuoted(table, out); - - writeString(" WHERE ", out); - - if (!where.empty()) + else { - writeString(where, out); - writeString(" AND ", out); + writeString(query, out); + + auto condition_position = query.find("{condition}"); + if (condition_position == std::string::npos) + { + writeString(" WHERE ", out); + composeKeysCondition(key_columns, requested_rows, method, partition_key_prefix, out); + writeString(";", out); + + return out.str(); + } + + WriteBufferFromOwnString condition_value_buffer; + composeKeysCondition(key_columns, requested_rows, method, partition_key_prefix, condition_value_buffer); + const auto & condition_value = condition_value_buffer.str(); + + auto query_copy = query; + query_copy.replace(condition_position, condition_value.size(), condition_value); + + return query_copy; + } +} + + +void ExternalQueryBuilder::composeKeyCondition(const Columns & key_columns, size_t row, WriteBuffer & out, + size_t beg, size_t end) const +{ + auto first = true; + for (size_t i = beg; i < end; ++i) + { + if (!first) + writeString(" AND ", out); + + first = false; + + const auto & key_description = (*dict_struct.key)[i]; + + /// key_i=value_i + writeQuoted(key_description.name, out); + writeString("=", out); + key_description.type_serialization->serializeTextQuoted(*key_columns[i], row, out, format_settings); + } +} + + +void ExternalQueryBuilder::composeInWithTuples(const Columns & key_columns, const std::vector & requested_rows, + WriteBuffer & out, size_t beg, size_t end) const +{ + composeKeyTupleDefinition(out, beg, end); + writeString(" IN (", out); + + bool first = true; + for (const auto row : requested_rows) + { + if (!first) + writeString(", ", out); + + first = false; + composeKeyTuple(key_columns, row, out, beg, end); } + writeString(")", out); +} + + +void ExternalQueryBuilder::composeKeyTupleDefinition(WriteBuffer & out, size_t beg, size_t end) const +{ + if (!dict_struct.key) + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Composite key required for method"); + + writeChar('(', out); + + auto first = true; + for (size_t i = beg; i < end; ++i) + { + if (!first) + writeString(", ", out); + + first = false; + writeQuoted((*dict_struct.key)[i].name, out); + } + + writeChar(')', out); +} + + +void ExternalQueryBuilder::composeKeyTuple(const Columns & key_columns, size_t row, WriteBuffer & out, size_t beg, size_t end) const +{ + writeString("(", out); + + auto first = true; + for (size_t i = beg; i < end; ++i) + { + if (!first) + writeString(", ", out); + + first = false; + auto serialization = (*dict_struct.key)[i].type_serialization; + serialization->serializeTextQuoted(*key_columns[i], row, out, format_settings); + } + + writeString(")", out); +} + +void ExternalQueryBuilder::composeUpdateCondition(const std::string & update_field, const std::string & time_point, WriteBuffer & out) +{ + writeString(update_field, out); + writeString(" >= '", out); + writeString(time_point, out); + writeChar('\'', out); +} + +void ExternalQueryBuilder::composeIdsCondition(const std::vector & ids, WriteBuffer & out) const +{ writeQuoted(dict_struct.id->name, out); writeString(" IN (", out); @@ -233,67 +486,12 @@ std::string ExternalQueryBuilder::composeLoadIdsQuery(const std::vector writeString(DB::toString(id), out); } - writeString(");", out); - - return out.str(); + writeString(")", out); } - -std::string ExternalQueryBuilder::composeLoadKeysQuery( - const Columns & key_columns, const std::vector & requested_rows, LoadKeysMethod method, size_t partition_key_prefix) +void ExternalQueryBuilder::composeKeysCondition(const Columns & key_columns, const std::vector & requested_rows, LoadKeysMethod method, size_t partition_key_prefix, WriteBuffer & out) const { - if (!dict_struct.key) - throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Composite key required for method"); - - if (key_columns.size() != dict_struct.key->size()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "The size of key_columns does not equal to the size of dictionary key"); - - WriteBufferFromOwnString out; - writeString("SELECT ", out); - - auto first = true; - for (const auto & key_or_attribute : boost::join(*dict_struct.key, dict_struct.attributes)) - { - if (!first) - writeString(", ", out); - - first = false; - - if (!key_or_attribute.expression.empty()) - { - writeParenthesisedString(key_or_attribute.expression, out); - writeString(" AS ", out); - } - - writeQuoted(key_or_attribute.name, out); - } - - writeString(" FROM ", out); - if (!db.empty()) - { - writeQuoted(db, out); - writeChar('.', out); - } - if (!schema.empty()) - { - writeQuoted(schema, out); - writeChar('.', out); - } - - writeQuoted(table, out); - - writeString(" WHERE ", out); - - if (!where.empty()) - { - if (method != CASSANDRA_SEPARATE_PARTITION_KEY) - writeString("(", out); - writeString(where, out); - if (method != CASSANDRA_SEPARATE_PARTITION_KEY) - writeString(") AND (", out); - else - writeString(" AND ", out); - } + bool first = true; if (method == AND_OR_CHAIN) { @@ -334,92 +532,6 @@ std::string ExternalQueryBuilder::composeLoadKeysQuery( { writeString(")", out); } - - writeString(";", out); - - return out.str(); } - -void ExternalQueryBuilder::composeKeyCondition(const Columns & key_columns, const size_t row, WriteBuffer & out, - size_t beg, size_t end) const -{ - auto first = true; - for (size_t i = beg; i < end; ++i) - { - if (!first) - writeString(" AND ", out); - - first = false; - - const auto & key_description = (*dict_struct.key)[i]; - - /// key_i=value_i - writeQuoted(key_description.name, out); - writeString("=", out); - key_description.type_serialization->serializeTextQuoted(*key_columns[i], row, out, format_settings); - } -} - - -void ExternalQueryBuilder::composeInWithTuples(const Columns & key_columns, const std::vector & requested_rows, - WriteBuffer & out, size_t beg, size_t end) -{ - composeKeyTupleDefinition(out, beg, end); - writeString(" IN (", out); - - bool first = true; - for (const auto row : requested_rows) - { - if (!first) - writeString(", ", out); - - first = false; - composeKeyTuple(key_columns, row, out, beg, end); - } - - writeString(")", out); -} - - -void ExternalQueryBuilder::composeKeyTupleDefinition(WriteBuffer & out, size_t beg, size_t end) const -{ - if (!dict_struct.key) - throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Composite key required for method"); - - writeChar('(', out); - - auto first = true; - for (size_t i = beg; i < end; ++i) - { - if (!first) - writeString(", ", out); - - first = false; - writeQuoted((*dict_struct.key)[i].name, out); - } - - writeChar(')', out); -} - - -void ExternalQueryBuilder::composeKeyTuple(const Columns & key_columns, const size_t row, WriteBuffer & out, size_t beg, size_t end) const -{ - writeString("(", out); - - auto first = true; - for (size_t i = beg; i < end; ++i) - { - if (!first) - writeString(", ", out); - - first = false; - auto serialization = (*dict_struct.key)[i].type_serialization; - serialization->serializeTextQuoted(*key_columns[i], row, out, format_settings); - } - - writeString(")", out); -} - - } diff --git a/src/Dictionaries/ExternalQueryBuilder.h b/src/Dictionaries/ExternalQueryBuilder.h index 9f9ccd65001..9d79ec3e702 100644 --- a/src/Dictionaries/ExternalQueryBuilder.h +++ b/src/Dictionaries/ExternalQueryBuilder.h @@ -21,6 +21,7 @@ struct ExternalQueryBuilder const std::string db; const std::string schema; const std::string table; + const std::string query; const std::string where; IdentifierQuotingStyle quoting_style; @@ -31,6 +32,7 @@ struct ExternalQueryBuilder const std::string & db_, const std::string & schema_, const std::string & table_, + const std::string & query_, const std::string & where_, IdentifierQuotingStyle quoting_style_); @@ -41,7 +43,7 @@ struct ExternalQueryBuilder std::string composeUpdateQuery(const std::string & update_field, const std::string & time_point) const; /** Generate a query to load data by set of UInt64 keys. */ - std::string composeLoadIdsQuery(const std::vector & ids); + std::string composeLoadIdsQuery(const std::vector & ids) const; /** Generate a query to load data by set of composite keys. * There are three methods of specification of composite keys in WHERE: @@ -56,7 +58,7 @@ struct ExternalQueryBuilder CASSANDRA_SEPARATE_PARTITION_KEY, }; - std::string composeLoadKeysQuery(const Columns & key_columns, const std::vector & requested_rows, LoadKeysMethod method, size_t partition_key_prefix = 0); + std::string composeLoadKeysQuery(const Columns & key_columns, const std::vector & requested_rows, LoadKeysMethod method, size_t partition_key_prefix = 0) const; private: @@ -67,16 +69,25 @@ private: /// In the following methods `beg` and `end` specifies which columns to write in expression /// Expression in form (x = c1 AND y = c2 ...) - void composeKeyCondition(const Columns & key_columns, const size_t row, WriteBuffer & out, size_t beg, size_t end) const; + void composeKeyCondition(const Columns & key_columns, size_t row, WriteBuffer & out, size_t beg, size_t end) const; /// Expression in form (x, y, ...) IN ((c1, c2, ...), ...) - void composeInWithTuples(const Columns & key_columns, const std::vector & requested_rows, WriteBuffer & out, size_t beg, size_t end); + void composeInWithTuples(const Columns & key_columns, const std::vector & requested_rows, WriteBuffer & out, size_t beg, size_t end) const; /// Expression in form (x, y, ...) void composeKeyTupleDefinition(WriteBuffer & out, size_t beg, size_t end) const; /// Expression in form (c1, c2, ...) - void composeKeyTuple(const Columns & key_columns, const size_t row, WriteBuffer & out, size_t beg, size_t end) const; + void composeKeyTuple(const Columns & key_columns, size_t row, WriteBuffer & out, size_t beg, size_t end) const; + + /// Compose update condition + static void composeUpdateCondition(const std::string & update_field, const std::string & time_point, WriteBuffer & out); + + /// Compose ids condition + void composeIdsCondition(const std::vector & ids, WriteBuffer & out) const; + + /// Compose keys condition + void composeKeysCondition(const Columns & key_columns, const std::vector & requested_rows, LoadKeysMethod method, size_t partition_key_prefix, WriteBuffer & out) const; /// Write string with specified quoting style. void writeQuoted(const std::string & s, WriteBuffer & out) const; diff --git a/src/Dictionaries/MySQLDictionarySource.cpp b/src/Dictionaries/MySQLDictionarySource.cpp index c7309ddb950..fbb39096fcd 100644 --- a/src/Dictionaries/MySQLDictionarySource.cpp +++ b/src/Dictionaries/MySQLDictionarySource.cpp @@ -22,6 +22,7 @@ static const size_t default_num_tries_on_connection_loss = 3; namespace ErrorCodes { extern const int SUPPORT_IS_DISABLED; + extern const int UNSUPPORTED_METHOD; } void registerDictionarySourceMysql(DictionarySourceFactory & factory) @@ -41,11 +42,19 @@ void registerDictionarySourceMysql(DictionarySourceFactory & factory) auto settings_config_prefix = config_prefix + ".mysql"; + auto table = config.getString(settings_config_prefix + ".table", ""); + auto where = config.getString(settings_config_prefix + ".where", ""); + auto query = config.getString(settings_config_prefix + ".query", ""); + + if (query.empty() && table.empty()) + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "MySQL dictionary source configuration must contain table or query field"); + MySQLDictionarySource::Configuration configuration { .db = config.getString(settings_config_prefix + ".db", ""), - .table = config.getString(settings_config_prefix + ".table"), - .where = config.getString(settings_config_prefix + ".where", ""), + .table = table, + .query = query, + .where = where, .invalidate_query = config.getString(settings_config_prefix + ".invalidate_query", ""), .update_field = config.getString(settings_config_prefix + ".update_field", ""), .update_lag = config.getUInt64(settings_config_prefix + ".update_lag", 1), @@ -94,8 +103,8 @@ MySQLDictionarySource::MySQLDictionarySource( , configuration(configuration_) , pool(std::move(pool_)) , sample_block(sample_block_) - , query_builder(dict_struct, configuration.db, "", configuration.table, configuration.where, IdentifierQuotingStyle::Backticks) - , load_all_query(query_builder.composeLoadAllQuery()) + , query_builder(dict_struct, configuration.db, "", configuration.table, configuration.query, configuration.where, IdentifierQuotingStyle::Backticks) + , load_all_query(configuration.query.empty() ? query_builder.composeLoadAllQuery() : configuration.query) , settings(settings_) { } @@ -108,7 +117,7 @@ MySQLDictionarySource::MySQLDictionarySource(const MySQLDictionarySource & other , configuration(other.configuration) , pool(other.pool) , sample_block(other.sample_block) - , query_builder{dict_struct, configuration.db, "", configuration.table, configuration.where, IdentifierQuotingStyle::Backticks} + , query_builder{dict_struct, configuration.db, "", configuration.table, configuration.query, configuration.where, IdentifierQuotingStyle::Backticks} , load_all_query{other.load_all_query} , last_modification{other.last_modification} , invalidate_query_response{other.invalidate_query_response} @@ -128,7 +137,7 @@ std::string MySQLDictionarySource::getUpdateFieldAndDate() else { update_time = std::chrono::system_clock::now(); - return query_builder.composeLoadAllQuery(); + return load_all_query; } } diff --git a/src/Dictionaries/MySQLDictionarySource.h b/src/Dictionaries/MySQLDictionarySource.h index 49ddc924a86..afa6aa61d28 100644 --- a/src/Dictionaries/MySQLDictionarySource.h +++ b/src/Dictionaries/MySQLDictionarySource.h @@ -35,6 +35,7 @@ public: { const std::string db; const std::string table; + const std::string query; const std::string where; const std::string invalidate_query; const std::string update_field; diff --git a/src/Dictionaries/PostgreSQLDictionarySource.cpp b/src/Dictionaries/PostgreSQLDictionarySource.cpp index f226b7a9165..c03dbaa71c5 100644 --- a/src/Dictionaries/PostgreSQLDictionarySource.cpp +++ b/src/Dictionaries/PostgreSQLDictionarySource.cpp @@ -27,7 +27,7 @@ static const UInt64 max_block_size = 8192; namespace { - ExternalQueryBuilder makeExternalQueryBuilder(const DictionaryStructure & dict_struct, const String & schema, const String & table, const String & where) + ExternalQueryBuilder makeExternalQueryBuilder(const DictionaryStructure & dict_struct, const String & schema, const String & table, const String & query, const String & where) { auto schema_value = schema; auto table_value = table; @@ -41,7 +41,7 @@ namespace } } /// Do not need db because it is already in a connection string. - return {dict_struct, "", schema_value, table_value, where, IdentifierQuotingStyle::DoubleQuotes}; + return {dict_struct, "", schema_value, table_value, query, where, IdentifierQuotingStyle::DoubleQuotes}; } } @@ -56,7 +56,7 @@ PostgreSQLDictionarySource::PostgreSQLDictionarySource( , pool(std::move(pool_)) , sample_block(sample_block_) , log(&Poco::Logger::get("PostgreSQLDictionarySource")) - , query_builder(makeExternalQueryBuilder(dict_struct, configuration.schema, configuration.table, configuration.where)) + , query_builder(makeExternalQueryBuilder(dict_struct, configuration.schema, configuration.table, configuration.query, configuration.where)) , load_all_query(query_builder.composeLoadAllQuery()) { } @@ -69,7 +69,7 @@ PostgreSQLDictionarySource::PostgreSQLDictionarySource(const PostgreSQLDictionar , pool(other.pool) , sample_block(other.sample_block) , log(&Poco::Logger::get("PostgreSQLDictionarySource")) - , query_builder(makeExternalQueryBuilder(dict_struct, configuration.schema, configuration.table, configuration.where)) + , query_builder(makeExternalQueryBuilder(dict_struct, configuration.schema, configuration.table, configuration.query, configuration.where)) , load_all_query(query_builder.composeLoadAllQuery()) , update_time(other.update_time) , invalidate_query_response(other.invalidate_query_response) @@ -198,6 +198,7 @@ void registerDictionarySourcePostgreSQL(DictionarySourceFactory & factory) .db = config.getString(fmt::format("{}.db", settings_config_prefix), ""), .schema = config.getString(fmt::format("{}.schema", settings_config_prefix), ""), .table = config.getString(fmt::format("{}.table", settings_config_prefix), ""), + .query = config.getString(fmt::format("{}.where", settings_config_prefix), ""), .where = config.getString(fmt::format("{}.where", settings_config_prefix), ""), .invalidate_query = config.getString(fmt::format("{}.invalidate_query", settings_config_prefix), ""), .update_field = config.getString(fmt::format("{}.update_field", settings_config_prefix), ""), diff --git a/src/Dictionaries/PostgreSQLDictionarySource.h b/src/Dictionaries/PostgreSQLDictionarySource.h index 28ad28661ed..c5ade4d259a 100644 --- a/src/Dictionaries/PostgreSQLDictionarySource.h +++ b/src/Dictionaries/PostgreSQLDictionarySource.h @@ -26,6 +26,7 @@ public: const String db; const String schema; const String table; + const String query; const String where; const String invalidate_query; const String update_field; diff --git a/src/Dictionaries/XDBCDictionarySource.cpp b/src/Dictionaries/XDBCDictionarySource.cpp index 26b6c24cd2d..80d3df1caab 100644 --- a/src/Dictionaries/XDBCDictionarySource.cpp +++ b/src/Dictionaries/XDBCDictionarySource.cpp @@ -34,6 +34,7 @@ namespace const std::string & db_, const std::string & schema_, const std::string & table_, + const std::string & query_, const std::string & where_, IXDBCBridgeHelper & bridge_) { @@ -59,7 +60,7 @@ namespace bridge_.getName()); } - return {dict_struct_, db_, schema, table, where_, bridge_.getIdentifierQuotingStyle()}; + return {dict_struct_, db_, schema, table, query_, where_, bridge_.getIdentifierQuotingStyle()}; } } @@ -78,7 +79,7 @@ XDBCDictionarySource::XDBCDictionarySource( , dict_struct(dict_struct_) , configuration(configuration_) , sample_block(sample_block_) - , query_builder(makeExternalQueryBuilder(dict_struct, configuration.db, configuration.schema, configuration.table, configuration.where, *bridge_)) + , query_builder(makeExternalQueryBuilder(dict_struct, configuration.db, configuration.schema, configuration.table, configuration.query, configuration.where, *bridge_)) , load_all_query(query_builder.composeLoadAllQuery()) , bridge_helper(bridge_) , bridge_url(bridge_helper->getMainURI()) @@ -119,7 +120,7 @@ std::string XDBCDictionarySource::getUpdateFieldAndDate() else { update_time = std::chrono::system_clock::now(); - return query_builder.composeLoadAllQuery(); + return load_all_query; } } @@ -246,7 +247,8 @@ void registerDictionarySourceXDBC(DictionarySourceFactory & factory) { .db = config.getString(settings_config_prefix + ".db", ""), .schema = config.getString(settings_config_prefix + ".schema", ""), - .table = config.getString(settings_config_prefix + ".table"), + .table = config.getString(settings_config_prefix + ".table", ""), + .query = config.getString(settings_config_prefix + ".query", ""), .where = config.getString(settings_config_prefix + ".where", ""), .invalidate_query = config.getString(settings_config_prefix + ".invalidate_query", ""), .update_field = config.getString(settings_config_prefix + ".update_field", ""), diff --git a/src/Dictionaries/XDBCDictionarySource.h b/src/Dictionaries/XDBCDictionarySource.h index ebced022b62..df31e8a87cf 100644 --- a/src/Dictionaries/XDBCDictionarySource.h +++ b/src/Dictionaries/XDBCDictionarySource.h @@ -32,6 +32,7 @@ public: const std::string db; const std::string schema; const std::string table; + const std::string query; const std::string where; const std::string invalidate_query; const std::string update_field; From f8500633e63861dcfb876d8b093d4235e54f4f91 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Mon, 9 Aug 2021 14:23:44 +0300 Subject: [PATCH 439/599] Added tests --- src/Dictionaries/ExternalQueryBuilder.cpp | 15 +- src/Dictionaries/MySQLDictionarySource.cpp | 2 +- .../PostgreSQLDictionarySource.cpp | 2 +- .../test_dictionaries_mysql/test.py | 128 ++++++++++++++++-- .../test_dictionaries_postgresql/test.py | 117 +++++++++++++++- 5 files changed, 243 insertions(+), 21 deletions(-) diff --git a/src/Dictionaries/ExternalQueryBuilder.cpp b/src/Dictionaries/ExternalQueryBuilder.cpp index deba95c9d02..0f7db8a2b31 100644 --- a/src/Dictionaries/ExternalQueryBuilder.cpp +++ b/src/Dictionaries/ExternalQueryBuilder.cpp @@ -65,10 +65,17 @@ void ExternalQueryBuilder::writeQuoted(const std::string & s, WriteBuffer & out) std::string ExternalQueryBuilder::composeLoadAllQuery() const { - WriteBufferFromOwnString out; - composeLoadAllQuery(out); - writeChar(';', out); - return out.str(); + if (query.empty()) + { + WriteBufferFromOwnString out; + composeLoadAllQuery(out); + writeChar(';', out); + return out.str(); + } + else + { + return query; + } } void ExternalQueryBuilder::composeLoadAllQuery(WriteBuffer & out) const diff --git a/src/Dictionaries/MySQLDictionarySource.cpp b/src/Dictionaries/MySQLDictionarySource.cpp index fbb39096fcd..2eebb6970d0 100644 --- a/src/Dictionaries/MySQLDictionarySource.cpp +++ b/src/Dictionaries/MySQLDictionarySource.cpp @@ -104,7 +104,7 @@ MySQLDictionarySource::MySQLDictionarySource( , pool(std::move(pool_)) , sample_block(sample_block_) , query_builder(dict_struct, configuration.db, "", configuration.table, configuration.query, configuration.where, IdentifierQuotingStyle::Backticks) - , load_all_query(configuration.query.empty() ? query_builder.composeLoadAllQuery() : configuration.query) + , load_all_query(query_builder.composeLoadAllQuery()) , settings(settings_) { } diff --git a/src/Dictionaries/PostgreSQLDictionarySource.cpp b/src/Dictionaries/PostgreSQLDictionarySource.cpp index c03dbaa71c5..21b8809bc85 100644 --- a/src/Dictionaries/PostgreSQLDictionarySource.cpp +++ b/src/Dictionaries/PostgreSQLDictionarySource.cpp @@ -198,7 +198,7 @@ void registerDictionarySourcePostgreSQL(DictionarySourceFactory & factory) .db = config.getString(fmt::format("{}.db", settings_config_prefix), ""), .schema = config.getString(fmt::format("{}.schema", settings_config_prefix), ""), .table = config.getString(fmt::format("{}.table", settings_config_prefix), ""), - .query = config.getString(fmt::format("{}.where", settings_config_prefix), ""), + .query = config.getString(fmt::format("{}.query", settings_config_prefix), ""), .where = config.getString(fmt::format("{}.where", settings_config_prefix), ""), .invalidate_query = config.getString(fmt::format("{}.invalidate_query", settings_config_prefix), ""), .update_field = config.getString(fmt::format("{}.update_field", settings_config_prefix), ""), diff --git a/tests/integration/test_dictionaries_mysql/test.py b/tests/integration/test_dictionaries_mysql/test.py index fe284f71e00..fa3855d1e16 100644 --- a/tests/integration/test_dictionaries_mysql/test.py +++ b/tests/integration/test_dictionaries_mysql/test.py @@ -1,4 +1,5 @@ ## sudo -H pip install PyMySQL +import warnings import pymysql.cursors import pytest from helpers.cluster import ClickHouseCluster @@ -46,23 +47,114 @@ def started_cluster(): cluster.shutdown() -def test_load_mysql_dictionaries(started_cluster): - # Load dictionaries +def test_mysql_dictionaries_custom_query_full_load(started_cluster): + mysql_connection = get_mysql_conn(started_cluster) + + execute_mysql_query(mysql_connection, "CREATE TABLE IF NOT EXISTS test.test_table_1 (id Integer, value_1 Text);") + execute_mysql_query(mysql_connection, "CREATE TABLE IF NOT EXISTS test.test_table_2 (id Integer, value_2 Text);") + execute_mysql_query(mysql_connection, "INSERT INTO test.test_table_1 VALUES (1, 'Value_1');") + execute_mysql_query(mysql_connection, "INSERT INTO test.test_table_2 VALUES (1, 'Value_2');") + query = instance.query - query("SYSTEM RELOAD DICTIONARIES") + query(""" + CREATE DICTIONARY test_dictionary_custom_query + ( + id UInt64, + value_1 String, + value_2 String + ) + PRIMARY KEY id + LAYOUT(FLAT()) + SOURCE(MYSQL( + HOST 'mysql57' + PORT 3306 + USER 'root' + PASSWORD 'clickhouse' + QUERY $doc$SELECT id, value_1, value_2 FROM test.test_table_1 INNER JOIN test.test_table_2 USING (id);$doc$)) + LIFETIME(0) + """) - for n in range(0, 5): - # Create MySQL tables, fill them and create CH dict tables - prepare_mysql_table(started_cluster, 'test', str(n)) + result = query("SELECT id, value_1, value_2 FROM test_dictionary_custom_query") - # Check dictionaries are loaded and have correct number of elements - for n in range(0, 100): - # Force reload of dictionaries (each 10 iteration) - if (n % 10) == 0: - query("SYSTEM RELOAD DICTIONARIES") + assert result == '1\tValue_1\tValue_2\n' - # Check number of row - assert query("SELECT count() FROM `test`.`dict_table_{}`".format('test' + str(n % 5))).rstrip() == '10000' + query("DROP DICTIONARY test_dictionary_custom_query;") + + execute_mysql_query(mysql_connection, "DROP TABLE test.test_table_1;") + execute_mysql_query(mysql_connection, "DROP TABLE test.test_table_2;") + + +def test_mysql_dictionaries_custom_query_partial_load_simple_key(started_cluster): + mysql_connection = get_mysql_conn(started_cluster) + + execute_mysql_query(mysql_connection, "CREATE TABLE IF NOT EXISTS test.test_table_1 (id Integer, value_1 Text);") + execute_mysql_query(mysql_connection, "CREATE TABLE IF NOT EXISTS test.test_table_2 (id Integer, value_2 Text);") + execute_mysql_query(mysql_connection, "INSERT INTO test.test_table_1 VALUES (1, 'Value_1');") + execute_mysql_query(mysql_connection, "INSERT INTO test.test_table_2 VALUES (1, 'Value_2');") + + query = instance.query + query(""" + CREATE DICTIONARY test_dictionary_custom_query + ( + id UInt64, + value_1 String, + value_2 String + ) + PRIMARY KEY id + LAYOUT(DIRECT()) + SOURCE(MYSQL( + HOST 'mysql57' + PORT 3306 + USER 'root' + PASSWORD 'clickhouse' + QUERY $doc$SELECT id, value_1, value_2 FROM test.test_table_1 INNER JOIN test.test_table_2 USING (id) WHERE {condition};$doc$)) + """) + + result = query("SELECT dictGet('test_dictionary_custom_query', ('value_1', 'value_2'), toUInt64(1))") + + assert result == "('Value_1','Value_2')\n" + + query("DROP DICTIONARY test_dictionary_custom_query;") + + execute_mysql_query(mysql_connection, "DROP TABLE test.test_table_1;") + execute_mysql_query(mysql_connection, "DROP TABLE test.test_table_2;") + + +def test_mysql_dictionaries_custom_query_partial_load_complex_key(started_cluster): + mysql_connection = get_mysql_conn(started_cluster) + + execute_mysql_query(mysql_connection, "CREATE TABLE IF NOT EXISTS test.test_table_1 (id Integer, id_key Text, value_1 Text);") + execute_mysql_query(mysql_connection, "CREATE TABLE IF NOT EXISTS test.test_table_2 (id Integer, id_key Text, value_2 Text);") + execute_mysql_query(mysql_connection, "INSERT INTO test.test_table_1 VALUES (1, 'Key', 'Value_1');") + execute_mysql_query(mysql_connection, "INSERT INTO test.test_table_2 VALUES (1, 'Key', 'Value_2');") + + query = instance.query + query(""" + CREATE DICTIONARY test_dictionary_custom_query + ( + id UInt64, + id_key String, + value_1 String, + value_2 String + ) + PRIMARY KEY id, id_key + LAYOUT(COMPLEX_KEY_DIRECT()) + SOURCE(MYSQL( + HOST 'mysql57' + PORT 3306 + USER 'root' + PASSWORD 'clickhouse' + QUERY $doc$SELECT id, id_key, value_1, value_2 FROM test.test_table_1 INNER JOIN test.test_table_2 USING (id, id_key) WHERE {condition};$doc$)) + """) + + result = query("SELECT dictGet('test_dictionary_custom_query', ('value_1', 'value_2'), (toUInt64(1), 'Key'))") + + assert result == "('Value_1','Value_2')\n" + + query("DROP DICTIONARY test_dictionary_custom_query;") + + execute_mysql_query(mysql_connection, "DROP TABLE test.test_table_1;") + execute_mysql_query(mysql_connection, "DROP TABLE test.test_table_2;") def create_mysql_db(mysql_connection, name): @@ -102,9 +194,17 @@ def get_mysql_conn(started_cluster): except Exception as e: errors += [str(e)] time.sleep(1) - + raise Exception("Connection not establised, {}".format(errors)) +def execute_mysql_query(connection, query): + logging.debug("Execute MySQL query:{}".format(query)) + with warnings.catch_warnings(): + warnings.simplefilter("ignore") + with connection.cursor() as cursor: + cursor.execute(query) + connection.commit() + def create_mysql_table(conn, table_name): with conn.cursor() as cursor: cursor.execute(create_table_mysql_template.format(table_name)) diff --git a/tests/integration/test_dictionaries_postgresql/test.py b/tests/integration/test_dictionaries_postgresql/test.py index d35182e15a1..6eb4a04ed2c 100644 --- a/tests/integration/test_dictionaries_postgresql/test.py +++ b/tests/integration/test_dictionaries_postgresql/test.py @@ -90,6 +90,121 @@ def test_load_dictionaries(started_cluster): node1.query("DROP DICTIONARY IF EXISTS {}".format(dict_name)) +def test_postgres_dictionaries_custom_query_full_load(started_cluster): + conn = get_postgres_conn(ip=started_cluster.postgres_ip, database=True, port=started_cluster.postgres_port) + cursor = conn.cursor() + + cursor.execute("CREATE TABLE IF NOT EXISTS test_table_1 (id Integer, value_1 Text);") + cursor.execute("CREATE TABLE IF NOT EXISTS test_table_2 (id Integer, value_2 Text);") + cursor.execute("INSERT INTO test_table_1 VALUES (1, 'Value_1');") + cursor.execute("INSERT INTO test_table_2 VALUES (1, 'Value_2');") + + query = node1.query + query(""" + CREATE DICTIONARY test_dictionary_custom_query + ( + id UInt64, + value_1 String, + value_2 String + ) + PRIMARY KEY id + LAYOUT(FLAT()) + SOURCE(PostgreSQL( + DB 'clickhouse' + HOST '{}' + PORT {} + USER 'postgres' + PASSWORD 'mysecretpassword' + QUERY $doc$SELECT id, value_1, value_2 FROM test_table_1 INNER JOIN test_table_2 USING (id);$doc$)) + LIFETIME(0) + """.format(started_cluster.postgres_ip, started_cluster.postgres_port)) + + result = query("SELECT id, value_1, value_2 FROM test_dictionary_custom_query") + + assert result == '1\tValue_1\tValue_2\n' + + query("DROP DICTIONARY test_dictionary_custom_query;") + + cursor.execute("DROP TABLE test_table_2;") + cursor.execute("DROP TABLE test_table_1;") + + +def test_postgres_dictionaries_custom_query_partial_load_simple_key(started_cluster): + conn = get_postgres_conn(ip=started_cluster.postgres_ip, database=True, port=started_cluster.postgres_port) + cursor = conn.cursor() + + cursor.execute("CREATE TABLE IF NOT EXISTS test_table_1 (id Integer, value_1 Text);") + cursor.execute("CREATE TABLE IF NOT EXISTS test_table_2 (id Integer, value_2 Text);") + cursor.execute("INSERT INTO test_table_1 VALUES (1, 'Value_1');") + cursor.execute("INSERT INTO test_table_2 VALUES (1, 'Value_2');") + + query = node1.query + query(""" + CREATE DICTIONARY test_dictionary_custom_query + ( + id UInt64, + value_1 String, + value_2 String + ) + PRIMARY KEY id + LAYOUT(DIRECT()) + SOURCE(PostgreSQL( + DB 'clickhouse' + HOST '{}' + PORT {} + USER 'postgres' + PASSWORD 'mysecretpassword' + QUERY $doc$SELECT id, value_1, value_2 FROM test_table_1 INNER JOIN test_table_2 USING (id) WHERE {{condition}};$doc$)) + """.format(started_cluster.postgres_ip, started_cluster.postgres_port)) + + result = query("SELECT dictGet('test_dictionary_custom_query', ('value_1', 'value_2'), toUInt64(1))") + + assert result == '(\'Value_1\',\'Value_2\')\n' + + query("DROP DICTIONARY test_dictionary_custom_query;") + + cursor.execute("DROP TABLE test_table_2;") + cursor.execute("DROP TABLE test_table_1;") + + +def test_postgres_dictionaries_custom_query_partial_load_complex_key(started_cluster): + conn = get_postgres_conn(ip=started_cluster.postgres_ip, database=True, port=started_cluster.postgres_port) + cursor = conn.cursor() + + cursor.execute("CREATE TABLE IF NOT EXISTS test_table_1 (id Integer, key Text, value_1 Text);") + cursor.execute("CREATE TABLE IF NOT EXISTS test_table_2 (id Integer, key Text, value_2 Text);") + cursor.execute("INSERT INTO test_table_1 VALUES (1, 'Key', 'Value_1');") + cursor.execute("INSERT INTO test_table_2 VALUES (1, 'Key', 'Value_2');") + + query = node1.query + query(""" + CREATE DICTIONARY test_dictionary_custom_query + ( + id UInt64, + key String, + value_1 String, + value_2 String + ) + PRIMARY KEY id, key + LAYOUT(COMPLEX_KEY_DIRECT()) + SOURCE(PostgreSQL( + DB 'clickhouse' + HOST '{}' + PORT {} + USER 'postgres' + PASSWORD 'mysecretpassword' + QUERY $doc$SELECT id, key, value_1, value_2 FROM test_table_1 INNER JOIN test_table_2 USING (id, key) WHERE {{condition}};$doc$)) + """.format(started_cluster.postgres_ip, started_cluster.postgres_port)) + + result = query("SELECT dictGet('test_dictionary_custom_query', ('value_1', 'value_2'), (toUInt64(1), 'Key'))") + + assert result == '(\'Value_1\',\'Value_2\')\n' + + query("DROP DICTIONARY test_dictionary_custom_query;") + + cursor.execute("DROP TABLE test_table_2;") + cursor.execute("DROP TABLE test_table_1;") + def test_invalidate_query(started_cluster): conn = get_postgres_conn(ip=started_cluster.postgres_ip, database=True, port=started_cluster.postgres_port) cursor = conn.cursor() @@ -158,7 +273,7 @@ def test_dictionary_with_replicas(started_cluster): node1.query("DROP DICTIONARY IF EXISTS dict1") -def test_postgres_scema(started_cluster): +def test_postgres_schema(started_cluster): conn = get_postgres_conn(ip=started_cluster.postgres_ip, port=started_cluster.postgres_port, database=True) cursor = conn.cursor() From 1cf38e046d93df4e892a19c415924de5254b21fd Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Tue, 10 Aug 2021 19:39:16 +0300 Subject: [PATCH 440/599] ExternalQueryBuilder updated exception --- src/Dictionaries/ExternalQueryBuilder.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Dictionaries/ExternalQueryBuilder.cpp b/src/Dictionaries/ExternalQueryBuilder.cpp index 0f7db8a2b31..10c4f67d809 100644 --- a/src/Dictionaries/ExternalQueryBuilder.cpp +++ b/src/Dictionaries/ExternalQueryBuilder.cpp @@ -35,8 +35,8 @@ ExternalQueryBuilder::ExternalQueryBuilder( if (table.empty() && query.empty()) throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Setting `table` or `query` must be non empty"); - if (!query.empty() && !where.empty()) - throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Setting `where` is not supported with `query` parameter"); + if (!query.empty() && (!table.empty() || !where.empty())) + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Setting `table` or `where` cannot be used with `query` parameter"); } From 81b85c30f70822da39a4a2a69fa21780b6843c76 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 11 Aug 2021 11:35:10 +0300 Subject: [PATCH 441/599] Remove streams from LV part 1. --- .../LiveView/LiveViewBlockInputStream.h | 25 ++++++++++--------- src/Storages/LiveView/StorageLiveView.h | 6 ++--- 2 files changed, 16 insertions(+), 15 deletions(-) diff --git a/src/Storages/LiveView/LiveViewBlockInputStream.h b/src/Storages/LiveView/LiveViewBlockInputStream.h index 737e76754c5..af07d8558ad 100644 --- a/src/Storages/LiveView/LiveViewBlockInputStream.h +++ b/src/Storages/LiveView/LiveViewBlockInputStream.h @@ -1,6 +1,7 @@ #pragma once -#include +#include +#include namespace DB @@ -10,19 +11,20 @@ namespace DB * Keeps stream alive by outputting blocks with no rows * based on period specified by the heartbeat interval. */ -class LiveViewBlockInputStream : public IBlockInputStream +class LiveViewSource : public SourceWithProgress { using NonBlockingResult = std::pair; public: - LiveViewBlockInputStream(std::shared_ptr storage_, + LiveViewSource(std::shared_ptr storage_, std::shared_ptr blocks_ptr_, std::shared_ptr blocks_metadata_ptr_, std::shared_ptr active_ptr_, const bool has_limit_, const UInt64 limit_, const UInt64 heartbeat_interval_sec_) - : storage(std::move(storage_)), blocks_ptr(std::move(blocks_ptr_)), + : SourceWithProgress(storage_->getHeader()) + , storage(std::move(storage_)), blocks_ptr(std::move(blocks_ptr_)), blocks_metadata_ptr(std::move(blocks_metadata_ptr_)), active_ptr(std::move(active_ptr_)), has_limit(has_limit_), limit(limit_), @@ -34,17 +36,15 @@ public: String getName() const override { return "LiveViewBlockInputStream"; } - void cancel(bool kill) override + void onCancel() override { if (isCancelled() || storage->shutdown_called) return; - IBlockInputStream::cancel(kill); + std::lock_guard lock(storage->mutex); storage->condition.notify_all(); } - Block getHeader() const override { return storage->getHeader(); } - void refresh() { if (active && blocks && it == end) @@ -74,10 +74,11 @@ public: } protected: - Block readImpl() override + Chunk generate() override { /// try reading - return tryReadImpl(true).first; + auto block = tryReadImpl(true).first; + return Chunk(block.getColumns(), block.rows()); } /** tryRead method attempts to read a block in either blocking @@ -135,7 +136,7 @@ protected: if (!end_of_blocks) { end_of_blocks = true; - return { getHeader(), true }; + return { getPort().getHeader(), true }; } while (true) { @@ -157,7 +158,7 @@ protected: { // heartbeat last_event_timestamp_usec = static_cast(Poco::Timestamp().epochMicroseconds()); - return { getHeader(), true }; + return { getPort().getHeader(), true }; } } } diff --git a/src/Storages/LiveView/StorageLiveView.h b/src/Storages/LiveView/StorageLiveView.h index 23a9c84cb9e..5f66a97090f 100644 --- a/src/Storages/LiveView/StorageLiveView.h +++ b/src/Storages/LiveView/StorageLiveView.h @@ -52,9 +52,9 @@ using Pipes = std::vector; class StorageLiveView final : public shared_ptr_helper, public IStorage, WithContext { friend struct shared_ptr_helper; -friend class LiveViewBlockInputStream; -friend class LiveViewEventsBlockInputStream; -friend class LiveViewBlockOutputStream; +friend class LiveViewSource; +friend class LiveViewEventsSource; +friend class LiveViewSink; public: ~StorageLiveView() override; From 83ebcd45eb4bc5ff9fcc5b6eed3e86950b15e53c Mon Sep 17 00:00:00 2001 From: benbiti Date: Wed, 11 Aug 2021 17:03:07 +0800 Subject: [PATCH 442/599] dup codes calc QueryMemoryLimitExceeded --- src/Common/MemoryTracker.cpp | 3 --- 1 file changed, 3 deletions(-) diff --git a/src/Common/MemoryTracker.cpp b/src/Common/MemoryTracker.cpp index a05fa3b5ad5..0be7ffda958 100644 --- a/src/Common/MemoryTracker.cpp +++ b/src/Common/MemoryTracker.cpp @@ -183,9 +183,6 @@ void MemoryTracker::allocImpl(Int64 size, bool throw_if_memory_exceeded) std::bernoulli_distribution fault(fault_probability); if (unlikely(fault_probability && fault(thread_local_rng)) && memoryTrackerCanThrow(level, true) && throw_if_memory_exceeded) { - ProfileEvents::increment(ProfileEvents::QueryMemoryLimitExceeded); - amount.fetch_sub(size, std::memory_order_relaxed); - /// Prevent recursion. Exception::ctor -> std::string -> new[] -> MemoryTracker::alloc BlockerInThread untrack_lock(VariableContext::Global); From 61c4d94f1e9158f307a10ca54a5eb0927f697cd3 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 11 Aug 2021 11:21:39 +0200 Subject: [PATCH 443/599] Add explicit dependency between roaring and clickhouse_common_io Fixes roaring memory tracker for split builds --- contrib/croaring-cmake/CMakeLists.txt | 9 +++------ src/CMakeLists.txt | 5 +++-- 2 files changed, 6 insertions(+), 8 deletions(-) diff --git a/contrib/croaring-cmake/CMakeLists.txt b/contrib/croaring-cmake/CMakeLists.txt index f0cb378864b..84cdccedbd3 100644 --- a/contrib/croaring-cmake/CMakeLists.txt +++ b/contrib/croaring-cmake/CMakeLists.txt @@ -26,17 +26,14 @@ target_include_directories(roaring SYSTEM BEFORE PUBLIC "${LIBRARY_DIR}/include" target_include_directories(roaring SYSTEM BEFORE PUBLIC "${LIBRARY_DIR}/cpp") # We redirect malloc/free family of functions to different functions that will track memory in ClickHouse. -# It will make this library depend on linking to 'clickhouse_common_io' library that is not done explicitly via 'target_link_libraries'. -# And we check that all libraries dependencies are satisfied and all symbols are resolved if we do build with shared libraries. -# That's why we enable it only in static build. # Also note that we exploit implicit function declarations. -if (USE_STATIC_LIBRARIES) - target_compile_definitions(roaring PRIVATE +target_compile_definitions(roaring PRIVATE -Dmalloc=clickhouse_malloc -Dcalloc=clickhouse_calloc -Drealloc=clickhouse_realloc -Dreallocarray=clickhouse_reallocarray -Dfree=clickhouse_free -Dposix_memalign=clickhouse_posix_memalign) -endif () + +target_link_libraries(roaring PUBLIC clickhouse_common_io) diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 6c10d3e2f2b..796c9eb4d2c 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -299,10 +299,11 @@ target_link_libraries(clickhouse_common_io ${ZLIB_LIBRARIES} pcg_random Poco::Foundation - roaring ) - +# Make dbms depend on roaring instead of clickhouse_common_io so that roaring itself can depend on clickhouse_common_io +# That way we we can redirect malloc/free functions avoiding circular dependencies +dbms_target_link_libraries(PUBLIC roaring) if (USE_RDKAFKA) dbms_target_link_libraries(PRIVATE ${CPPKAFKA_LIBRARY} ${RDKAFKA_LIBRARY}) From 4545cc91fca071e4eaadbca7a027570c04bb7fe1 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 11 Aug 2021 11:26:18 +0200 Subject: [PATCH 444/599] Add a note about split build drawbacks --- docs/en/development/build.md | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/docs/en/development/build.md b/docs/en/development/build.md index 97b477d55a5..be45c1ed5f7 100644 --- a/docs/en/development/build.md +++ b/docs/en/development/build.md @@ -155,6 +155,10 @@ Normally ClickHouse is statically linked into a single static `clickhouse` binar -DUSE_STATIC_LIBRARIES=0 -DSPLIT_SHARED_LIBRARIES=1 -DCLICKHOUSE_SPLIT_BINARY=1 ``` -Note that in this configuration there is no single `clickhouse` binary, and you have to run `clickhouse-server`, `clickhouse-client` etc. +Note that the split build has several drawbacks: +* There is no single `clickhouse` binary, and you have to run `clickhouse-server`, `clickhouse-client`, etc. +* Risk of segfault if you run any of the programs while rebuilding the project. +* You cannot run the integration tests since they only work a single complete binary. +* You can't easily copy the binaries elsewhere. Instead of moving a single binary you'll need to copy all binaries and libraries. [Original article](https://clickhouse.tech/docs/en/development/build/) From 72393d6deb47e3f7adcf65f668d24d2ec4f1156f Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Wed, 11 Aug 2021 13:05:18 +0300 Subject: [PATCH 445/599] Less emotions --- website/README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/website/README.md b/website/README.md index 9f808c6f658..f02deb0ad91 100644 --- a/website/README.md +++ b/website/README.md @@ -23,7 +23,7 @@ virtualenv build ./build.py --skip-multi-page --skip-single-page --skip-amp --skip-pdf --skip-git-log --skip-docs --livereload 8080 ``` -# How to quickly test the ugly annoying broken links in docs +# How to quickly test the broken links in docs ``` ./build.py --skip-multi-page --skip-amp --skip-pdf --skip-blog --skip-git-log --lang en --livereload 8080 From c93868babf2439b6d030af30870e944800b82a8a Mon Sep 17 00:00:00 2001 From: tavplubix Date: Wed, 11 Aug 2021 13:12:24 +0300 Subject: [PATCH 446/599] Update ci-runner.py --- tests/integration/ci-runner.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/ci-runner.py b/tests/integration/ci-runner.py index 7aecf2953f8..a23d7b3485b 100755 --- a/tests/integration/ci-runner.py +++ b/tests/integration/ci-runner.py @@ -419,7 +419,7 @@ class ClickhouseIntegrationTestsRunner: test_cmd = ' '.join([test for test in sorted(test_names)]) parallel_cmd = " --parallel {} ".format(num_workers) if num_workers > 0 else "" - cmd = "cd {}/tests/integration && ./runner --tmpfs {} -t {} {} '-rfEp --run-id={} --color=no --durations=0 {}' | tee {}".format( + cmd = "cd {}/tests/integration && timeout 1h ./runner --tmpfs {} -t {} {} '-rfEp --run-id={} --color=no --durations=0 {}' | tee {}".format( repo_path, image_cmd, test_cmd, parallel_cmd, i, _get_deselect_option(self.should_skip_tests()), info_path) log_basename = test_group_str + "_" + str(i) + ".log" From 1c5d1c6b7a6062dda03daa3c7f59b1835a4b7a42 Mon Sep 17 00:00:00 2001 From: abel-wang Date: Wed, 11 Aug 2021 19:54:53 +0800 Subject: [PATCH 447/599] Remove useless code. --- src/Interpreters/RequiredSourceColumnsVisitor.cpp | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Interpreters/RequiredSourceColumnsVisitor.cpp b/src/Interpreters/RequiredSourceColumnsVisitor.cpp index f6786451c82..2c81969009f 100644 --- a/src/Interpreters/RequiredSourceColumnsVisitor.cpp +++ b/src/Interpreters/RequiredSourceColumnsVisitor.cpp @@ -145,8 +145,6 @@ void RequiredSourceColumnsMatcher::visit(const ASTSelectQuery & select, const AS /// revisit select_expression_list (with children) when all the aliases are set Visitor(data).visit(select.select()); - if (auto with = select.with()) - Visitor(data).visit(with); } void RequiredSourceColumnsMatcher::visit(const ASTIdentifier & node, const ASTPtr &, Data & data) From eaea0a3e40800b9bf0e3b6d3b86ecdfb798f3a0a Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 11 Aug 2021 12:05:38 +0000 Subject: [PATCH 448/599] fix --- src/DataStreams/PostgreSQLBlockInputStream.cpp | 2 +- src/DataStreams/PostgreSQLBlockInputStream.h | 13 ------------- 2 files changed, 1 insertion(+), 14 deletions(-) diff --git a/src/DataStreams/PostgreSQLBlockInputStream.cpp b/src/DataStreams/PostgreSQLBlockInputStream.cpp index 7f8949740df..0b50c453629 100644 --- a/src/DataStreams/PostgreSQLBlockInputStream.cpp +++ b/src/DataStreams/PostgreSQLBlockInputStream.cpp @@ -73,7 +73,7 @@ void PostgreSQLSource::init(const Block & sample_block) template void PostgreSQLSource::onStart() { - if (connection_holder) + if (!tx) tx = std::make_shared(connection_holder->get()); stream = std::make_unique(*tx, pqxx::from_query, std::string_view(query_str)); diff --git a/src/DataStreams/PostgreSQLBlockInputStream.h b/src/DataStreams/PostgreSQLBlockInputStream.h index 008da976619..2736afec7a9 100644 --- a/src/DataStreams/PostgreSQLBlockInputStream.h +++ b/src/DataStreams/PostgreSQLBlockInputStream.h @@ -76,19 +76,6 @@ public: const Block & sample_block_, const UInt64 max_block_size_) : PostgreSQLSource(tx_, query_str_, sample_block_, max_block_size_, false) {} - - Chunk generate() override - { - if (!is_initialized) - { - Base::stream = std::make_unique(*Base::tx, pqxx::from_query, std::string_view(Base::query_str)); - is_initialized = true; - } - - return Base::generate(); - } - - bool is_initialized = false; }; } From aa8ca01bb96317d9f0cdb112985ace57f8169070 Mon Sep 17 00:00:00 2001 From: tavplubix Date: Wed, 11 Aug 2021 15:17:46 +0300 Subject: [PATCH 449/599] Update ci-runner.py --- tests/integration/ci-runner.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/integration/ci-runner.py b/tests/integration/ci-runner.py index a23d7b3485b..12ee436e400 100755 --- a/tests/integration/ci-runner.py +++ b/tests/integration/ci-runner.py @@ -264,7 +264,7 @@ class ClickhouseIntegrationTestsRunner: out_file = "all_tests.txt" out_file_full = "all_tests_full.txt" cmd = "cd {repo_path}/tests/integration && " \ - "timeout 1h ./runner --tmpfs {image_cmd} ' --setup-plan' " \ + "timeout -s 9 1h ./runner --tmpfs {image_cmd} ' --setup-plan' " \ "| tee {out_file_full} | grep '::' | sed 's/ (fixtures used:.*//g' | sed 's/^ *//g' | sed 's/ *$//g' " \ "| grep -v 'SKIPPED' | sort -u > {out_file}".format( repo_path=repo_path, image_cmd=image_cmd, out_file=out_file, out_file_full=out_file_full) @@ -419,7 +419,7 @@ class ClickhouseIntegrationTestsRunner: test_cmd = ' '.join([test for test in sorted(test_names)]) parallel_cmd = " --parallel {} ".format(num_workers) if num_workers > 0 else "" - cmd = "cd {}/tests/integration && timeout 1h ./runner --tmpfs {} -t {} {} '-rfEp --run-id={} --color=no --durations=0 {}' | tee {}".format( + cmd = "cd {}/tests/integration && timeout -s 9 1h ./runner --tmpfs {} -t {} {} '-rfEp --run-id={} --color=no --durations=0 {}' | tee {}".format( repo_path, image_cmd, test_cmd, parallel_cmd, i, _get_deselect_option(self.should_skip_tests()), info_path) log_basename = test_group_str + "_" + str(i) + ".log" From 2117278f88a5f78543fe1c8c005114fbfff8c9c6 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 11 Aug 2021 15:21:54 +0300 Subject: [PATCH 450/599] Another try --- src/Common/DenseHashMap.h | 2 +- src/Common/DenseHashSet.h | 2 +- src/Common/SparseHashMap.h | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Common/DenseHashMap.h b/src/Common/DenseHashMap.h index 7a9d5a68e2d..9054d049e07 100644 --- a/src/Common/DenseHashMap.h +++ b/src/Common/DenseHashMap.h @@ -2,7 +2,7 @@ #include #if defined(ARCADIA_BUILD) -#define HASH_FUN_H ; +#define HASH_FUN_H template struct THash : public std::hash {}; #endif diff --git a/src/Common/DenseHashSet.h b/src/Common/DenseHashSet.h index 004d1c10903..7f28585d9db 100644 --- a/src/Common/DenseHashSet.h +++ b/src/Common/DenseHashSet.h @@ -1,7 +1,7 @@ #pragma once #if defined(ARCADIA_BUILD) -#define HASH_FUN_H ; +#define HASH_FUN_H template struct THash : public std::hash {}; #endif diff --git a/src/Common/SparseHashMap.h b/src/Common/SparseHashMap.h index 651dd22f15d..7464ebf831a 100644 --- a/src/Common/SparseHashMap.h +++ b/src/Common/SparseHashMap.h @@ -1,7 +1,7 @@ #pragma once #if defined(ARCADIA_BUILD) -#define HASH_FUN_H ; +#define HASH_FUN_H template struct THash : public std::hash {}; #endif From 5ba959e12c614c7ff7baa4ff93df62a07b877298 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 11 Aug 2021 12:29:03 +0000 Subject: [PATCH 451/599] Add timeout --- .../integration/test_postgresql_replica_database_engine/test.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/integration/test_postgresql_replica_database_engine/test.py b/tests/integration/test_postgresql_replica_database_engine/test.py index ed26ab82bc7..3763b503b60 100644 --- a/tests/integration/test_postgresql_replica_database_engine/test.py +++ b/tests/integration/test_postgresql_replica_database_engine/test.py @@ -113,6 +113,7 @@ def assert_nested_table_is_created(table_name, materialized_database='test_datab assert(table_name in database_tables) +@pytest.mark.timeout(320) def check_tables_are_synchronized(table_name, order_by='key', postgres_database='postgres_database', materialized_database='test_database'): assert_nested_table_is_created(table_name, materialized_database) From 51d31b9213ab03dc1d6feadeb01aef7c3b950711 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Wed, 11 Aug 2021 15:32:01 +0300 Subject: [PATCH 452/599] Dictionaries refactor --- src/Dictionaries/CacheDictionary.cpp | 24 ++++++++--------- src/Dictionaries/DictionaryHelpers.h | 10 +++++++ ...ckInputStream.cpp => DictionarySource.cpp} | 26 +++++++----------- ...yBlockInputStream.h => DictionarySource.h} | 15 +++-------- ...treamBase.cpp => DictionarySourceBase.cpp} | 2 +- ...putStreamBase.h => DictionarySourceBase.h} | 0 src/Dictionaries/FlatDictionary.cpp | 2 +- src/Dictionaries/HashedDictionary.cpp | 2 +- src/Dictionaries/IPAddressDictionary.cpp | 2 +- src/Dictionaries/PolygonDictionary.cpp | 6 ++--- ...kInputStream.h => RangeDictionarySource.h} | 27 +++++-------------- src/Dictionaries/RangeHashedDictionary.cpp | 6 ++--- src/Dictionaries/RangeHashedDictionary.h | 8 +++--- 13 files changed, 56 insertions(+), 74 deletions(-) rename src/Dictionaries/{DictionaryBlockInputStream.cpp => DictionarySource.cpp} (89%) rename src/Dictionaries/{DictionaryBlockInputStream.h => DictionarySource.h} (86%) rename src/Dictionaries/{DictionaryBlockInputStreamBase.cpp => DictionarySourceBase.cpp} (91%) rename src/Dictionaries/{DictionaryBlockInputStreamBase.h => DictionarySourceBase.h} (100%) rename src/Dictionaries/{RangeDictionaryBlockInputStream.h => RangeDictionarySource.h} (90%) diff --git a/src/Dictionaries/CacheDictionary.cpp b/src/Dictionaries/CacheDictionary.cpp index 4dfe802dd2b..a5f953ccc15 100644 --- a/src/Dictionaries/CacheDictionary.cpp +++ b/src/Dictionaries/CacheDictionary.cpp @@ -10,7 +10,7 @@ #include #include -#include +#include #include #include @@ -18,21 +18,21 @@ namespace ProfileEvents { -extern const Event DictCacheKeysRequested; -extern const Event DictCacheKeysRequestedMiss; -extern const Event DictCacheKeysRequestedFound; -extern const Event DictCacheKeysExpired; -extern const Event DictCacheKeysNotFound; -extern const Event DictCacheKeysHit; -extern const Event DictCacheRequestTimeNs; -extern const Event DictCacheRequests; -extern const Event DictCacheLockWriteNs; -extern const Event DictCacheLockReadNs; + extern const Event DictCacheKeysRequested; + extern const Event DictCacheKeysRequestedMiss; + extern const Event DictCacheKeysRequestedFound; + extern const Event DictCacheKeysExpired; + extern const Event DictCacheKeysNotFound; + extern const Event DictCacheKeysHit; + extern const Event DictCacheRequestTimeNs; + extern const Event DictCacheRequests; + extern const Event DictCacheLockWriteNs; + extern const Event DictCacheLockReadNs; } namespace CurrentMetrics { -extern const Metric DictCacheRequests; + extern const Metric DictCacheRequests; } namespace DB diff --git a/src/Dictionaries/DictionaryHelpers.h b/src/Dictionaries/DictionaryHelpers.h index 5a050d68326..dde41864ddc 100644 --- a/src/Dictionaries/DictionaryHelpers.h +++ b/src/Dictionaries/DictionaryHelpers.h @@ -648,6 +648,16 @@ static const PaddedPODArray & getColumnVectorData( } } +template +static ColumnPtr getColumnFromPODArray(const PaddedPODArray & array) +{ + auto column_vector = ColumnVector::create(); + column_vector->getData().reserve(array.size()); + column_vector->getData().insert(array.begin(), array.end()); + + return column_vector; +} + } diff --git a/src/Dictionaries/DictionaryBlockInputStream.cpp b/src/Dictionaries/DictionarySource.cpp similarity index 89% rename from src/Dictionaries/DictionaryBlockInputStream.cpp rename to src/Dictionaries/DictionarySource.cpp index fedde8bd886..7ba6ea82ca9 100644 --- a/src/Dictionaries/DictionaryBlockInputStream.cpp +++ b/src/Dictionaries/DictionarySource.cpp @@ -1,4 +1,5 @@ -#include "DictionaryBlockInputStream.h" +#include "DictionarySource.h" +#include namespace DB { @@ -12,7 +13,7 @@ DictionarySourceData::DictionarySourceData( std::shared_ptr dictionary_, PaddedPODArray && ids_, const Names & column_names_) : num_rows(ids_.size()) , dictionary(dictionary_) - , column_names(column_names_) + , column_names(column_names_.begin(), column_names_.end()) , ids(std::move(ids_)) , key_type(DictionaryInputStreamKeyType::Id) { @@ -24,7 +25,7 @@ DictionarySourceData::DictionarySourceData( const Names & column_names_) : num_rows(keys.size()) , dictionary(dictionary_) - , column_names(column_names_) + , column_names(column_names_.begin(), column_names_.end()) , key_type(DictionaryInputStreamKeyType::ComplexKey) { const DictionaryStructure & dictionary_structure = dictionary->getStructure(); @@ -39,7 +40,7 @@ DictionarySourceData::DictionarySourceData( GetColumnsFunction && get_view_columns_function_) : num_rows(data_columns_.front()->size()) , dictionary(dictionary_) - , column_names(column_names_) + , column_names(column_names_.begin(), column_names_.end()) , data_columns(data_columns_) , get_key_columns_function(std::move(get_key_columns_function_)) , get_view_columns_function(std::move(get_view_columns_function_)) @@ -102,8 +103,6 @@ Block DictionarySourceData::fillBlock( const DataTypes & types, ColumnsWithTypeAndName && view) const { - std::unordered_set names(column_names.begin(), column_names.end()); - DataTypes data_types = types; ColumnsWithTypeAndName block_columns; @@ -114,13 +113,13 @@ Block DictionarySourceData::fillBlock( data_types.push_back(key.type); for (const auto & column : view) - if (names.find(column.name) != names.end()) + if (column_names.find(column.name) != column_names.end()) block_columns.push_back(column); const DictionaryStructure & structure = dictionary->getStructure(); - ColumnPtr ids_column = getColumnFromIds(ids_to_fill); + ColumnPtr ids_column = getColumnFromPODArray(ids_to_fill); - if (structure.id && names.find(structure.id->name) != names.end()) + if (structure.id && column_names.find(structure.id->name) != column_names.end()) { block_columns.emplace_back(ids_column, std::make_shared(), structure.id->name); } @@ -129,7 +128,7 @@ Block DictionarySourceData::fillBlock( for (const auto & attribute : structure.attributes) { - if (names.find(attribute.name) != names.end()) + if (column_names.find(attribute.name) != column_names.end()) { ColumnPtr column; @@ -159,13 +158,6 @@ Block DictionarySourceData::fillBlock( return Block(block_columns); } -ColumnPtr DictionarySourceData::getColumnFromIds(const PaddedPODArray & ids_to_fill) -{ - auto column_vector = ColumnVector::create(); - column_vector->getData().assign(ids_to_fill); - return column_vector; -} - void DictionarySourceData::fillKeyColumns( const PaddedPODArray & keys, size_t start, diff --git a/src/Dictionaries/DictionaryBlockInputStream.h b/src/Dictionaries/DictionarySource.h similarity index 86% rename from src/Dictionaries/DictionaryBlockInputStream.h rename to src/Dictionaries/DictionarySource.h index c15406487e2..195a3c66484 100644 --- a/src/Dictionaries/DictionaryBlockInputStream.h +++ b/src/Dictionaries/DictionarySource.h @@ -7,19 +7,14 @@ #include #include #include -#include -#include "DictionaryBlockInputStreamBase.h" -#include "DictionaryStructure.h" -#include "IDictionary.h" +#include +#include +#include namespace DB { -/// TODO: Remove this class -/* BlockInputStream implementation for external dictionaries - * read() returns blocks consisting of the in-memory contents of the dictionaries - */ class DictionarySourceData { public: @@ -56,8 +51,6 @@ private: const DataTypes & types, ColumnsWithTypeAndName && view) const; - static ColumnPtr getColumnFromIds(const PaddedPODArray & ids_to_fill); - static void fillKeyColumns( const PaddedPODArray & keys, size_t start, @@ -67,7 +60,7 @@ private: const size_t num_rows; std::shared_ptr dictionary; - Names column_names; + std::unordered_set column_names; PaddedPODArray ids; ColumnsWithTypeAndName key_columns; diff --git a/src/Dictionaries/DictionaryBlockInputStreamBase.cpp b/src/Dictionaries/DictionarySourceBase.cpp similarity index 91% rename from src/Dictionaries/DictionaryBlockInputStreamBase.cpp rename to src/Dictionaries/DictionarySourceBase.cpp index 0eac8edac3d..cc420b33144 100644 --- a/src/Dictionaries/DictionaryBlockInputStreamBase.cpp +++ b/src/Dictionaries/DictionarySourceBase.cpp @@ -1,4 +1,4 @@ -#include "DictionaryBlockInputStreamBase.h" +#include "DictionarySourceBase.h" namespace DB { diff --git a/src/Dictionaries/DictionaryBlockInputStreamBase.h b/src/Dictionaries/DictionarySourceBase.h similarity index 100% rename from src/Dictionaries/DictionaryBlockInputStreamBase.h rename to src/Dictionaries/DictionarySourceBase.h diff --git a/src/Dictionaries/FlatDictionary.cpp b/src/Dictionaries/FlatDictionary.cpp index 58cb5048737..639895ac8ac 100644 --- a/src/Dictionaries/FlatDictionary.cpp +++ b/src/Dictionaries/FlatDictionary.cpp @@ -13,7 +13,7 @@ #include #include -#include +#include #include #include diff --git a/src/Dictionaries/HashedDictionary.cpp b/src/Dictionaries/HashedDictionary.cpp index b50b6a72707..189994dabf4 100644 --- a/src/Dictionaries/HashedDictionary.cpp +++ b/src/Dictionaries/HashedDictionary.cpp @@ -6,7 +6,7 @@ #include #include -#include +#include #include #include diff --git a/src/Dictionaries/IPAddressDictionary.cpp b/src/Dictionaries/IPAddressDictionary.cpp index 380ad460cba..fbe911c1d49 100644 --- a/src/Dictionaries/IPAddressDictionary.cpp +++ b/src/Dictionaries/IPAddressDictionary.cpp @@ -13,7 +13,7 @@ #include #include #include -#include +#include #include #include diff --git a/src/Dictionaries/PolygonDictionary.cpp b/src/Dictionaries/PolygonDictionary.cpp index 39152963ede..f10aa071442 100644 --- a/src/Dictionaries/PolygonDictionary.cpp +++ b/src/Dictionaries/PolygonDictionary.cpp @@ -3,14 +3,14 @@ #include #include -#include "DictionaryBlockInputStream.h" -#include "DictionaryFactory.h" - #include #include #include #include #include +#include +#include + namespace DB { diff --git a/src/Dictionaries/RangeDictionaryBlockInputStream.h b/src/Dictionaries/RangeDictionarySource.h similarity index 90% rename from src/Dictionaries/RangeDictionaryBlockInputStream.h rename to src/Dictionaries/RangeDictionarySource.h index d17687b7164..d4fce32a54f 100644 --- a/src/Dictionaries/RangeDictionaryBlockInputStream.h +++ b/src/Dictionaries/RangeDictionarySource.h @@ -1,14 +1,14 @@ #pragma once +#include +#include #include #include #include -#include -#include -#include -#include "DictionaryBlockInputStreamBase.h" -#include "DictionaryStructure.h" -#include "IDictionary.h" -#include "RangeHashedDictionary.h" +#include +#include +#include +#include +#include namespace DB @@ -31,8 +31,6 @@ public: size_t getNumRows() const { return ids.size(); } private: - template - ColumnPtr getColumnFromPODArray(const PaddedPODArray & array) const; Block fillBlock( const PaddedPODArray & ids_to_fill, @@ -86,17 +84,6 @@ Block RangeDictionarySourceData::getBlock(size_t start, size_t length return fillBlock(block_ids, block_start_dates, block_end_dates); } -template -template -ColumnPtr RangeDictionarySourceData::getColumnFromPODArray(const PaddedPODArray & array) const -{ - auto column_vector = ColumnVector::create(); - column_vector->getData().reserve(array.size()); - column_vector->getData().insert(array.begin(), array.end()); - - return column_vector; -} - template PaddedPODArray RangeDictionarySourceData::makeDateKey( const PaddedPODArray & block_start_dates, const PaddedPODArray & block_end_dates) const diff --git a/src/Dictionaries/RangeHashedDictionary.cpp b/src/Dictionaries/RangeHashedDictionary.cpp index 8b882b5a107..bbd70b51437 100644 --- a/src/Dictionaries/RangeHashedDictionary.cpp +++ b/src/Dictionaries/RangeHashedDictionary.cpp @@ -2,11 +2,11 @@ #include #include #include -#include -#include "DictionaryFactory.h" -#include "RangeDictionaryBlockInputStream.h" #include #include +#include +#include + namespace { diff --git a/src/Dictionaries/RangeHashedDictionary.h b/src/Dictionaries/RangeHashedDictionary.h index 01ee2b3c773..13fa6ad570f 100644 --- a/src/Dictionaries/RangeHashedDictionary.h +++ b/src/Dictionaries/RangeHashedDictionary.h @@ -9,10 +9,10 @@ #include #include #include -#include "DictionaryStructure.h" -#include "IDictionary.h" -#include "IDictionarySource.h" -#include "DictionaryHelpers.h" +#include +#include +#include +#include namespace DB { From 4ebdb5917a1b71e3ad40a8a0ffd7aef05a166fee Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 11 Aug 2021 15:20:36 +0200 Subject: [PATCH 453/599] clickhouse-test: Implement @@SKIP@@ directive in tests --- tests/clickhouse-test | 4 ++++ .../01103_check_cpu_instructions_at_startup.sh | 7 ++++++- tests/queries/0_stateless/01801_s3_cluster.sh | 9 +++++++++ 3 files changed, 19 insertions(+), 1 deletion(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index f6833cfbd09..c627810a550 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -650,6 +650,10 @@ def run_tests_array(all_tests_with_params): status += " - having exception in stdout:\n{}\n".format( '\n'.join(stdout.split('\n')[:100])) status += 'Database: ' + testcase_args.testcase_database + elif '@@SKIP@@' in stdout: + skipped_total += 1 + skip_reason = stdout.replace('@@SKIP@@', '').rstrip("\n") + status += MSG_SKIPPED + f" - {skip_reason}\n" elif reference_file is None: status += MSG_UNKNOWN status += print_test_time(total_time) diff --git a/tests/queries/0_stateless/01103_check_cpu_instructions_at_startup.sh b/tests/queries/0_stateless/01103_check_cpu_instructions_at_startup.sh index 1039f8f7d97..79c531e1a72 100755 --- a/tests/queries/0_stateless/01103_check_cpu_instructions_at_startup.sh +++ b/tests/queries/0_stateless/01103_check_cpu_instructions_at_startup.sh @@ -6,10 +6,15 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # If we run sanitized binary under qemu, it will try to slowly allocate 20 TiB until OOM. # Don't even try to do that. This test should be disabled for sanitizer builds. -${CLICKHOUSE_LOCAL} --query "SELECT max(value LIKE '%sanitize%') FROM system.build_options" | grep -q '1' && echo 'Skip test for sanitizer build' && exit +${CLICKHOUSE_LOCAL} --query "SELECT max(value LIKE '%sanitize%') FROM system.build_options" | grep -q '1' && echo '@@SKIP@@: Sanitizer build' && exit command=$(command -v ${CLICKHOUSE_LOCAL}) +if ! hash qemu-x86_64-static 2>/dev/null; then + echo "@@SKIP@@: No qemu-x86_64-static" + exit 0 +fi + function run_with_cpu() { qemu-x86_64-static -cpu "$@" "$command" --query "SELECT 1" 2>&1 | grep -v -F "warning: TCG doesn't support requested feature" ||: diff --git a/tests/queries/0_stateless/01801_s3_cluster.sh b/tests/queries/0_stateless/01801_s3_cluster.sh index 215d5500be5..460f3856a2f 100755 --- a/tests/queries/0_stateless/01801_s3_cluster.sh +++ b/tests/queries/0_stateless/01801_s3_cluster.sh @@ -7,6 +7,15 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh +if [[ -z $S3_ACCESS_KEY_ID ]]; then + echo "@@SKIP@@: Missing \$S3_ACCESS_KEY_ID" + exit 0 +fi + +if [[ -z $S3_SECRET_ACCESS ]]; then + echo "@@SKIP@@: Missing \$S3_SECRET_ACCESS" + exit 0 +fi ${CLICKHOUSE_CLIENT_BINARY} --send_logs_level="none" -q "SELECT * FROM s3('https://s3.mds.yandex.net/clickhouse-test-reports/*/*/functional_stateless_tests_(ubsan)/test_results.tsv', '$S3_ACCESS_KEY_ID', '$S3_SECRET_ACCESS', 'LineAsString', 'line String') limit 100 FORMAT Null;" ${CLICKHOUSE_CLIENT_BINARY} --send_logs_level="none" -q "SELECT * FROM s3Cluster('test_cluster_two_shards', 'https://s3.mds.yandex.net/clickhouse-test-reports/*/*/functional_stateless_tests_(ubsan)/test_results.tsv', '$S3_ACCESS_KEY_ID', '$S3_SECRET_ACCESS', 'LineAsString', 'line String') limit 100 FORMAT Null;" From af6249c8ece85ebc36527bbd8d6ce97400e26089 Mon Sep 17 00:00:00 2001 From: Mikhail Filimonov Date: Wed, 11 Aug 2021 16:19:45 +0200 Subject: [PATCH 454/599] Less include --- programs/client/QueryFuzzer.h | 1 - src/Coordination/KeeperStorageDispatcher.cpp | 1 - src/DataStreams/ExecutionSpeedLimits.h | 3 ++- src/Databases/DatabaseReplicated.cpp | 1 - src/Dictionaries/SSDCacheDictionaryStorage.h | 1 - src/IO/Progress.h | 2 -- src/Interpreters/CrashLog.cpp | 1 + src/Interpreters/DNSCacheUpdater.h | 2 -- src/Interpreters/QueryPriorities.h | 2 -- src/Interpreters/SystemLog.h | 1 - src/Processors/Formats/IRowInputFormat.h | 2 +- 11 files changed, 4 insertions(+), 13 deletions(-) diff --git a/programs/client/QueryFuzzer.h b/programs/client/QueryFuzzer.h index 19f089c6c4e..09d57f4161f 100644 --- a/programs/client/QueryFuzzer.h +++ b/programs/client/QueryFuzzer.h @@ -7,7 +7,6 @@ #include #include -#include #include #include diff --git a/src/Coordination/KeeperStorageDispatcher.cpp b/src/Coordination/KeeperStorageDispatcher.cpp index e95a6940baa..7c416b38d8b 100644 --- a/src/Coordination/KeeperStorageDispatcher.cpp +++ b/src/Coordination/KeeperStorageDispatcher.cpp @@ -1,6 +1,5 @@ #include #include -#include #include #include #include diff --git a/src/DataStreams/ExecutionSpeedLimits.h b/src/DataStreams/ExecutionSpeedLimits.h index d52dc713c1a..9c86ba2faf4 100644 --- a/src/DataStreams/ExecutionSpeedLimits.h +++ b/src/DataStreams/ExecutionSpeedLimits.h @@ -3,7 +3,8 @@ #include #include #include -#include + +class Stopwatch; namespace DB { diff --git a/src/Databases/DatabaseReplicated.cpp b/src/Databases/DatabaseReplicated.cpp index 26dd8763c40..8e8fb4e2d6d 100644 --- a/src/Databases/DatabaseReplicated.cpp +++ b/src/Databases/DatabaseReplicated.cpp @@ -8,7 +8,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Dictionaries/SSDCacheDictionaryStorage.h b/src/Dictionaries/SSDCacheDictionaryStorage.h index 395328a904d..bdb640c90be 100644 --- a/src/Dictionaries/SSDCacheDictionaryStorage.h +++ b/src/Dictionaries/SSDCacheDictionaryStorage.h @@ -12,7 +12,6 @@ #include #include -#include #include #include #include diff --git a/src/IO/Progress.h b/src/IO/Progress.h index e1253ab8eb8..772131d8cb7 100644 --- a/src/IO/Progress.h +++ b/src/IO/Progress.h @@ -6,8 +6,6 @@ #include #include -#include - namespace DB { diff --git a/src/Interpreters/CrashLog.cpp b/src/Interpreters/CrashLog.cpp index a9da804f1d2..6bc23d6cf62 100644 --- a/src/Interpreters/CrashLog.cpp +++ b/src/Interpreters/CrashLog.cpp @@ -6,6 +6,7 @@ #include #include #include +#include #if !defined(ARCADIA_BUILD) # include diff --git a/src/Interpreters/DNSCacheUpdater.h b/src/Interpreters/DNSCacheUpdater.h index bbbc2ab3d21..5d5486bd012 100644 --- a/src/Interpreters/DNSCacheUpdater.h +++ b/src/Interpreters/DNSCacheUpdater.h @@ -2,8 +2,6 @@ #include #include -#include - namespace DB { diff --git a/src/Interpreters/QueryPriorities.h b/src/Interpreters/QueryPriorities.h index 4a271510537..9e18e7bcff3 100644 --- a/src/Interpreters/QueryPriorities.h +++ b/src/Interpreters/QueryPriorities.h @@ -6,8 +6,6 @@ #include #include #include -#include - namespace CurrentMetrics { diff --git a/src/Interpreters/SystemLog.h b/src/Interpreters/SystemLog.h index b94f3f7d456..176bf60908b 100644 --- a/src/Interpreters/SystemLog.h +++ b/src/Interpreters/SystemLog.h @@ -12,7 +12,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Processors/Formats/IRowInputFormat.h b/src/Processors/Formats/IRowInputFormat.h index 2ca182b7ffe..19a94d41044 100644 --- a/src/Processors/Formats/IRowInputFormat.h +++ b/src/Processors/Formats/IRowInputFormat.h @@ -5,8 +5,8 @@ #include #include #include -#include +class Stopwatch; namespace DB { From fe40990a6d3605d39befdbcc16f27461bf024ba0 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Wed, 11 Aug 2021 19:37:03 +0300 Subject: [PATCH 455/599] Update CHANGELOG.md --- CHANGELOG.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index c7c054a53a8..10ed4dae1c9 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,4 +1,4 @@ -### ClickHouse release v21.8, 2021-08-11 +### ClickHouse release v21.8, 2021-08-12 #### New Features From 07c1a8e26e7f9a2d99a2a0f413f40335c0064380 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Wed, 11 Aug 2021 19:37:33 +0300 Subject: [PATCH 456/599] Aggregation temporary disable compilation without key --- src/Interpreters/Aggregator.cpp | 15 +- .../jit_aggregate_functions_no_key.xml | 284 ------------------ 2 files changed, 8 insertions(+), 291 deletions(-) delete mode 100644 tests/performance/jit_aggregate_functions_no_key.xml diff --git a/src/Interpreters/Aggregator.cpp b/src/Interpreters/Aggregator.cpp index 0b97d403d01..c26eb10e697 100644 --- a/src/Interpreters/Aggregator.cpp +++ b/src/Interpreters/Aggregator.cpp @@ -977,13 +977,14 @@ bool Aggregator::executeOnBlock(Columns columns, UInt64 num_rows, AggregatedData /// For the case when there are no keys (all aggregate into one row). if (result.type == AggregatedDataVariants::Type::without_key) { -#if USE_EMBEDDED_COMPILER - if (compiled_aggregate_functions_holder) - { - executeWithoutKeyImpl(result.without_key, num_rows, aggregate_functions_instructions.data(), result.aggregates_pool); - } - else -#endif + /// TODO: Enable compilation after investigation +// #if USE_EMBEDDED_COMPILER +// if (compiled_aggregate_functions_holder) +// { +// executeWithoutKeyImpl(result.without_key, num_rows, aggregate_functions_instructions.data(), result.aggregates_pool); +// } +// else +// #endif { executeWithoutKeyImpl(result.without_key, num_rows, aggregate_functions_instructions.data(), result.aggregates_pool); } diff --git a/tests/performance/jit_aggregate_functions_no_key.xml b/tests/performance/jit_aggregate_functions_no_key.xml deleted file mode 100644 index 2d8f390059a..00000000000 --- a/tests/performance/jit_aggregate_functions_no_key.xml +++ /dev/null @@ -1,284 +0,0 @@ - - - hits_100m_single - - - - 1 - 0 - - - - CREATE TABLE jit_test_memory ( - key UInt64, - value_1 UInt64, - value_2 UInt64, - value_3 UInt64, - value_4 UInt64, - value_5 UInt64, - predicate UInt8 - ) Engine = Memory - - - - CREATE TABLE jit_test_merge_tree ( - key UInt64, - value_1 UInt64, - value_2 UInt64, - value_3 UInt64, - value_4 UInt64, - value_5 UInt64, - predicate UInt8 - ) Engine = MergeTree - ORDER BY key - - - - CREATE TABLE jit_test_merge_tree_nullable ( - key UInt64, - value_1 Nullable(UInt64), - value_2 Nullable(UInt64), - value_3 Nullable(UInt64), - value_4 Nullable(UInt64), - value_5 Nullable(UInt64), - predicate UInt8 - ) Engine = Memory - - - - CREATE TABLE jit_test_memory_nullable ( - key UInt64, - value_1 Nullable(UInt64), - value_2 Nullable(UInt64), - value_3 Nullable(UInt64), - value_4 Nullable(UInt64), - value_5 Nullable(UInt64), - predicate UInt8 - ) Engine = MergeTree - ORDER BY key - - - - - function - - sum - min - max - avg - any - anyLast - count - groupBitOr - groupBitAnd - groupBitXor - - - - - table - - jit_test_memory - jit_test_merge_tree - jit_test_memory_nullable - jit_test_merge_tree_nullable - - - - - group_scale - - 1000000 - - - - - - INSERT INTO {table} - SELECT - number % 1000000, - number, - number, - number, - number, - number, - if (number % 2 == 0, 1, 0) - FROM - system.numbers_mt - LIMIT 10000000 - - - - SELECT - {function}(value_1), - {function}(value_2), - {function}(value_3) - FROM {table} - FORMAT Null - - - - SELECT - {function}(value_1), - {function}(value_2), - sum(toUInt256(value_3)), - {function}(value_3) - FROM {table} - FORMAT Null - - - - SELECT - {function}If(value_1, predicate), - {function}If(value_2, predicate), - {function}If(value_3, predicate) - FROM {table} - FORMAT Null - - - - SELECT - {function}If(value_1, predicate), - {function}If(value_2, predicate), - sumIf(toUInt256(value_3), predicate), - {function}If(value_3, predicate) - FROM {table} - FORMAT Null - - - - SELECT - {function}(value_1), - {function}(value_2), - {function}(value_3), - {function}(value_4), - {function}(value_5) - FROM {table} - FORMAT Null - - - - SELECT - {function}(value_1), - {function}(value_2), - sum(toUInt256(value_3)), - {function}(value_3), - {function}(value_4), - {function}(value_5) - FROM {table} - FORMAT Null - - - - SELECT - {function}If(value_1, predicate), - {function}If(value_2, predicate), - {function}If(value_3, predicate), - {function}If(value_4, predicate), - {function}If(value_5, predicate) - FROM {table} - FORMAT Null - - - - SELECT - {function}If(value_1, predicate), - {function}If(value_2, predicate), - sumIf(toUInt256(value_3), predicate), - {function}If(value_3, predicate), - {function}If(value_4, predicate), - {function}If(value_5, predicate) - FROM {table} - FORMAT Null - - - - SELECT - {function}(WatchID), - {function}(CounterID), - {function}(ClientIP) - FROM hits_100m_single - FORMAT Null - - - - SELECT - {function}(WatchID), - {function}(CounterID), - sum(toUInt256(ClientIP)), - {function}(ClientIP) - FROM hits_100m_single - FORMAT Null - - - - SELECT - {function}(WatchID), - {function}(CounterID), - {function}(ClientIP), - {function}(IPNetworkID), - {function}(SearchEngineID) - FROM hits_100m_single - FORMAT Null - - - - SELECT - {function}(WatchID), - {function}(CounterID), - sum(toUInt256(ClientIP)), - {function}(ClientIP), - {function}(IPNetworkID), - {function}(SearchEngineID) - FROM hits_100m_single - FORMAT Null - - - - WITH (WatchID % 2 == 0) AS predicate - SELECT - {function}If(WatchID, predicate), - {function}If(CounterID, predicate), - {function}If(ClientIP, predicate) - FROM hits_100m_single - FORMAT Null - - - - WITH (WatchID % 2 == 0) AS predicate - SELECT - {function}If(WatchID, predicate), - {function}If(CounterID, predicate), - sumIf(toUInt256(ClientIP), predicate), - {function}If(ClientIP, predicate) - FROM hits_100m_single - FORMAT Null - - - - WITH (WatchID % 2 == 0) AS predicate - SELECT - {function}If(WatchID, predicate), - {function}If(CounterID, predicate), - {function}If(ClientIP, predicate), - {function}If(IPNetworkID, predicate), - {function}If(SearchEngineID, predicate) - FROM hits_100m_single - FORMAT Null - - - - WITH (WatchID % 2 == 0) AS predicate - SELECT - {function}If(WatchID, predicate), - {function}If(CounterID, predicate), - sumIf(toUInt256(ClientIP), predicate), - {function}If(ClientIP, predicate), - {function}If(IPNetworkID, predicate), - {function}If(SearchEngineID, predicate) - FROM hits_100m_single - FORMAT Null - - - DROP TABLE IF EXISTS {table} - From 188c7371f9789335f4100ff89c31eb9aef668307 Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Wed, 11 Aug 2021 19:41:51 +0300 Subject: [PATCH 457/599] Remove unecessary code --- .../Formats/Impl/ArrowColumnToCHColumn.cpp | 53 ++++--------------- 1 file changed, 11 insertions(+), 42 deletions(-) diff --git a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp index db7d4df7165..2da4d7d298d 100644 --- a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp +++ b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp @@ -31,6 +31,7 @@ #include +/// UINT16 and UINT32 are processed separately, see comments in readColumnFromArrowColumn. #define FOR_ARROW_NUMERIC_TYPES(M) \ M(arrow::Type::UINT8, DB::UInt8) \ M(arrow::Type::INT8, DB::Int8) \ @@ -152,30 +153,6 @@ static ColumnWithTypeAndName readColumnWithBooleanData(std::shared_ptr & arrow_column, const String & column_name) -{ - auto internal_type = std::make_shared(); - auto internal_column = internal_type->createColumn(); - PaddedPODArray & column_data = assert_cast &>(*internal_column).getData(); - column_data.reserve(arrow_column->length()); - - for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i) - { - arrow::UInt16Array & chunk = dynamic_cast(*(arrow_column->chunk(chunk_i))); - - for (size_t value_i = 0, length = static_cast(chunk.length()); value_i < length; ++value_i) - { - UInt16 days_num = static_cast(chunk.Value(value_i)); - if (days_num > DATE_LUT_MAX_DAY_NUM) - throw Exception{ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE, - "Input value {} of a column \"{}\" is greater than max allowed Date value, which is {}", days_num, column_name, DATE_LUT_MAX_DAY_NUM}; - - column_data.emplace_back(days_num); - } - } - return {std::move(internal_column), std::move(internal_type), column_name}; -} - static ColumnWithTypeAndName readColumnWithDate32Data(std::shared_ptr & arrow_column, const String & column_name) { auto internal_type = std::make_shared(); @@ -220,22 +197,6 @@ static ColumnWithTypeAndName readColumnWithDate64Data(std::shared_ptr & arrow_column, const String & column_name) -{ - auto internal_type = std::make_shared(); - auto internal_column = internal_type->createColumn(); - auto & column_data = assert_cast &>(*internal_column).getData(); - column_data.reserve(arrow_column->length()); - - for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i) - { - auto & chunk = dynamic_cast(*(arrow_column->chunk(chunk_i))); - for (size_t value_i = 0, length = static_cast(chunk.length()); value_i < length; ++value_i) - column_data.emplace_back(chunk.Value(value_i)); - } - return {std::move(internal_column), std::move(internal_type), column_name}; -} - static ColumnWithTypeAndName readColumnWithTimestampData(std::shared_ptr & arrow_column, const String & column_name) { auto internal_type = std::make_shared(); @@ -393,9 +354,17 @@ static ColumnWithTypeAndName readColumnFromArrowColumn( // so, read UINT16 as Date and UINT32 as DateTime to perform correct conversion // between Date and DateTime further. case arrow::Type::UINT16: - return readColumnWithDateData(arrow_column, column_name); + { + auto column = readColumnWithNumericData(arrow_column, column_name); + column.type = std::make_shared(); + return column; + } case arrow::Type::UINT32: - return readColumnWithDateTimeData(arrow_column, column_name); + { + auto column = readColumnWithNumericData(arrow_column, column_name); + column.type = std::make_shared(); + return column; + } case arrow::Type::TIMESTAMP: return readColumnWithTimestampData(arrow_column, column_name); #if defined(ARCADIA_BUILD) From f8bdbe7dfd3a0613ef04ee81978ab4d695456382 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Wed, 11 Aug 2021 20:01:18 +0300 Subject: [PATCH 458/599] Update and rename 40_bug-report.md to 85_bug-report.md --- .github/ISSUE_TEMPLATE/{40_bug-report.md => 85_bug-report.md} | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) rename .github/ISSUE_TEMPLATE/{40_bug-report.md => 85_bug-report.md} (94%) diff --git a/.github/ISSUE_TEMPLATE/40_bug-report.md b/.github/ISSUE_TEMPLATE/85_bug-report.md similarity index 94% rename from .github/ISSUE_TEMPLATE/40_bug-report.md rename to .github/ISSUE_TEMPLATE/85_bug-report.md index d62ec578f8d..bd59d17db3f 100644 --- a/.github/ISSUE_TEMPLATE/40_bug-report.md +++ b/.github/ISSUE_TEMPLATE/85_bug-report.md @@ -1,6 +1,6 @@ --- name: Bug report -about: Create a report to help us improve ClickHouse +about: Wrong behaviour (visible to users) in official ClickHouse release. title: '' labels: bug assignees: '' From 2fbfee3514d7918e323890c7882595782becc643 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Wed, 11 Aug 2021 20:17:08 +0300 Subject: [PATCH 459/599] Update docs/ru/sql-reference/functions/string-functions.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/sql-reference/functions/string-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/string-functions.md b/docs/ru/sql-reference/functions/string-functions.md index f60027b6202..0c7f44b4d46 100644 --- a/docs/ru/sql-reference/functions/string-functions.md +++ b/docs/ru/sql-reference/functions/string-functions.md @@ -53,7 +53,7 @@ leftPad('string', 'length'[, 'pad_string']) - `string` — входная строка, которую необходимо заполнить. [String](../data-types/string.md). - `length` — длина результирующей строки. [UInt](../data-types/int-uint.md). Если указанное значение меньше, чем длина входной строки, то входная строка возвращается как есть. -- `pad_string` — строка, используемая для заполнения входной строки. [String](../data-types/string.md). Необязательный параметр. Если не указано, то входная строка заполняется пробелами. +- `pad_string` — строка, используемая для дополнения входной строки. [String](../data-types/string.md). Необязательный параметр. Если не указано, то входная строка дополняется пробелами. **Возвращаемое значение** From 828ee136900bf1ca21b230b03e5bfe84e4ba04f8 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Wed, 11 Aug 2021 20:17:15 +0300 Subject: [PATCH 460/599] Update docs/ru/sql-reference/functions/string-functions.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/sql-reference/functions/string-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/string-functions.md b/docs/ru/sql-reference/functions/string-functions.md index 0c7f44b4d46..627d08e2f10 100644 --- a/docs/ru/sql-reference/functions/string-functions.md +++ b/docs/ru/sql-reference/functions/string-functions.md @@ -41,7 +41,7 @@ toc_title: "Функции для работы со строками" ## leftPad {#leftpad} -Заполняет текущую строку слева пробелами или указанной строкой (несколько раз, если необходимо), пока результирующая строка не достигнет заданной длины. Соответствует MySQL функции `LPAD`. +Дополняет текущую строку слева пробелами или указанной строкой (несколько раз, если необходимо), пока результирующая строка не достигнет заданной длины. Соответствует MySQL функции `LPAD`. **Синтаксис** From 0ee3445345eedf4ab73b9b6e4c7f642937173627 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Wed, 11 Aug 2021 20:17:23 +0300 Subject: [PATCH 461/599] Update docs/ru/sql-reference/functions/string-functions.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/sql-reference/functions/string-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/string-functions.md b/docs/ru/sql-reference/functions/string-functions.md index 627d08e2f10..b37da2c3488 100644 --- a/docs/ru/sql-reference/functions/string-functions.md +++ b/docs/ru/sql-reference/functions/string-functions.md @@ -89,7 +89,7 @@ leftPadUTF8('string','length'[, 'pad_string']) **Параметры** -- `string` — входная строка, которую необходимо заполнить. [String](../data-types/string.md). +- `string` — входная строка, которую необходимо дополнить. [String](../data-types/string.md). - `length` — длина результирующей строки. [UInt](../data-types/int-uint.md). Если указанное значение меньше, чем длина входной строки, то входная строка возвращается как есть. - `pad_string` — строка, используемая для заполнения входной строки. [String](../data-types/string.md). Необязательный параметр. Если не указано, то входная строка заполняется пробелами. From 53792154384b5bdabdf815fa73b3e22e4e1dd659 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Wed, 11 Aug 2021 20:17:31 +0300 Subject: [PATCH 462/599] Update docs/ru/sql-reference/functions/string-functions.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/sql-reference/functions/string-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/string-functions.md b/docs/ru/sql-reference/functions/string-functions.md index b37da2c3488..0b6a72bd03f 100644 --- a/docs/ru/sql-reference/functions/string-functions.md +++ b/docs/ru/sql-reference/functions/string-functions.md @@ -117,7 +117,7 @@ SELECT leftPadUTF8('абвг', 7, '*'), leftPadUTF8('дежз', 7); ## rightPad {#rightpad} -Заполняет текущую строку справа пробелами или указанной строкой (несколько раз, если необходимо), пока результирующая строка не достигнет заданной длины. Соответствует MySQL функции `RPAD`. +Дополняет текущую строку справа пробелами или указанной строкой (несколько раз, если необходимо), пока результирующая строка не достигнет заданной длины. Соответствует MySQL функции `RPAD`. **Синтаксис** From 75fef816b578dd0185fca193152565c1ebea3ddb Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Wed, 11 Aug 2021 20:17:38 +0300 Subject: [PATCH 463/599] Update docs/ru/sql-reference/functions/string-functions.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/sql-reference/functions/string-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/string-functions.md b/docs/ru/sql-reference/functions/string-functions.md index 0b6a72bd03f..55874877b49 100644 --- a/docs/ru/sql-reference/functions/string-functions.md +++ b/docs/ru/sql-reference/functions/string-functions.md @@ -129,7 +129,7 @@ rightPad('string', 'length'[, 'pad_string']) - `string` — входная строка, которую необходимо заполнить. [String](../data-types/string.md). - `length` — длина результирующей строки. [UInt](../data-types/int-uint.md). Если указанное значение меньше, чем длина входной строки, то входная строка возвращается как есть. -- `pad_string` — строка, используемая для заполнения входной строки. [String](../data-types/string.md). Необязательный параметр. Если не указано, то входная строка заполняется пробелами. +- `pad_string` — строка, используемая для дополнения входной строки. [String](../data-types/string.md). Необязательный параметр. Если не указано, то входная строка дополняется пробелами. **Возвращаемое значение** From 4fe8e29484750a261141c84fbd8010def583d046 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Wed, 11 Aug 2021 20:17:44 +0300 Subject: [PATCH 464/599] Update docs/ru/sql-reference/functions/string-functions.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/sql-reference/functions/string-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/string-functions.md b/docs/ru/sql-reference/functions/string-functions.md index 55874877b49..7a438e6165c 100644 --- a/docs/ru/sql-reference/functions/string-functions.md +++ b/docs/ru/sql-reference/functions/string-functions.md @@ -165,7 +165,7 @@ rightPadUTF8('string','length'[, 'pad_string']) **Параметры** -- `string` — входная строка, которую необходимо заполнить. [String](../data-types/string.md). +- `string` — входная строка, которую необходимо дополнить. [String](../data-types/string.md). - `length` — длина результирующей строки. [UInt](../data-types/int-uint.md). Если указанное значение меньше, чем длина входной строки, то входная строка возвращается как есть. - `pad_string` — строка, используемая для заполнения входной строки. [String](../data-types/string.md). Необязательный параметр. Если не указано, то входная строка заполняется пробелами. From c76d6f1e619fa1515f2b27e32219319b4f09732e Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Wed, 11 Aug 2021 20:17:51 +0300 Subject: [PATCH 465/599] Update docs/ru/sql-reference/functions/string-functions.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/sql-reference/functions/string-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/string-functions.md b/docs/ru/sql-reference/functions/string-functions.md index 7a438e6165c..fbe2729489c 100644 --- a/docs/ru/sql-reference/functions/string-functions.md +++ b/docs/ru/sql-reference/functions/string-functions.md @@ -167,7 +167,7 @@ rightPadUTF8('string','length'[, 'pad_string']) - `string` — входная строка, которую необходимо дополнить. [String](../data-types/string.md). - `length` — длина результирующей строки. [UInt](../data-types/int-uint.md). Если указанное значение меньше, чем длина входной строки, то входная строка возвращается как есть. -- `pad_string` — строка, используемая для заполнения входной строки. [String](../data-types/string.md). Необязательный параметр. Если не указано, то входная строка заполняется пробелами. +- `pad_string` — строка, используемая для дополнения входной строки. [String](../data-types/string.md). Необязательный параметр. Если не указано, то входная строка дополняется пробелами. **Возвращаемое значение** From 952ed4b5027f782dc5133e30dd43166c9a128289 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Wed, 11 Aug 2021 20:18:08 +0300 Subject: [PATCH 466/599] Update docs/ru/sql-reference/functions/string-functions.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/sql-reference/functions/string-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/string-functions.md b/docs/ru/sql-reference/functions/string-functions.md index fbe2729489c..694571f00f3 100644 --- a/docs/ru/sql-reference/functions/string-functions.md +++ b/docs/ru/sql-reference/functions/string-functions.md @@ -155,7 +155,7 @@ SELECT rightPad('abc', 7, '*'), rightPad('abc', 7); ## rightPadUTF8 {#rightpadutf8} -Заполняет текущую строку слева пробелами или указанной строкой (несколько раз, если необходимо), пока результирующая строка не достигнет заданной длины. Соответствует MySQL функции `RPAD`. Функция [rightPad](#rightpad) измеряет длину строки в байтах, а функция `rightPadUTF8` — в кодовых точках Unicode. +Дополняет текущую строку слева пробелами или указанной строкой (несколько раз, если необходимо), пока результирующая строка не достигнет заданной длины. Соответствует MySQL функции `RPAD`. В отличие от функции [rightPad](#rightpad), измеряет длину строки не в байтах, а в кодовых точках Unicode. **Синтаксис** From 56fdd4f99af8d537c879f74cc39d802672a6dab1 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Wed, 11 Aug 2021 20:18:33 +0300 Subject: [PATCH 467/599] Update docs/ru/sql-reference/functions/string-functions.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/sql-reference/functions/string-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/string-functions.md b/docs/ru/sql-reference/functions/string-functions.md index 694571f00f3..a70387eaa26 100644 --- a/docs/ru/sql-reference/functions/string-functions.md +++ b/docs/ru/sql-reference/functions/string-functions.md @@ -127,7 +127,7 @@ rightPad('string', 'length'[, 'pad_string']) **Параметры** -- `string` — входная строка, которую необходимо заполнить. [String](../data-types/string.md). +- `string` — входная строка, которую необходимо дополнить. [String](../data-types/string.md). - `length` — длина результирующей строки. [UInt](../data-types/int-uint.md). Если указанное значение меньше, чем длина входной строки, то входная строка возвращается как есть. - `pad_string` — строка, используемая для дополнения входной строки. [String](../data-types/string.md). Необязательный параметр. Если не указано, то входная строка дополняется пробелами. From b1863b4b368f2216efce9a608016dde370cb4240 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Wed, 11 Aug 2021 20:18:41 +0300 Subject: [PATCH 468/599] Update docs/ru/sql-reference/functions/string-functions.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/sql-reference/functions/string-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/string-functions.md b/docs/ru/sql-reference/functions/string-functions.md index a70387eaa26..1745d7bc279 100644 --- a/docs/ru/sql-reference/functions/string-functions.md +++ b/docs/ru/sql-reference/functions/string-functions.md @@ -91,7 +91,7 @@ leftPadUTF8('string','length'[, 'pad_string']) - `string` — входная строка, которую необходимо дополнить. [String](../data-types/string.md). - `length` — длина результирующей строки. [UInt](../data-types/int-uint.md). Если указанное значение меньше, чем длина входной строки, то входная строка возвращается как есть. -- `pad_string` — строка, используемая для заполнения входной строки. [String](../data-types/string.md). Необязательный параметр. Если не указано, то входная строка заполняется пробелами. +- `pad_string` — строка, используемая для дополнения входной строки. [String](../data-types/string.md). Необязательный параметр. Если не указано, то входная строка дополняется пробелами. **Возвращаемое значение** From 7cd9882edef363e0190a63b5f7b3a8872cf07561 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Wed, 11 Aug 2021 20:18:49 +0300 Subject: [PATCH 469/599] Update docs/ru/sql-reference/functions/string-functions.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/sql-reference/functions/string-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/string-functions.md b/docs/ru/sql-reference/functions/string-functions.md index 1745d7bc279..be1fdcdc929 100644 --- a/docs/ru/sql-reference/functions/string-functions.md +++ b/docs/ru/sql-reference/functions/string-functions.md @@ -51,7 +51,7 @@ leftPad('string', 'length'[, 'pad_string']) **Параметры** -- `string` — входная строка, которую необходимо заполнить. [String](../data-types/string.md). +- `string` — входная строка, которую необходимо дополнить. [String](../data-types/string.md). - `length` — длина результирующей строки. [UInt](../data-types/int-uint.md). Если указанное значение меньше, чем длина входной строки, то входная строка возвращается как есть. - `pad_string` — строка, используемая для дополнения входной строки. [String](../data-types/string.md). Необязательный параметр. Если не указано, то входная строка дополняется пробелами. From 99e786049d84dcb9a5032371dc76cfd21c970a0e Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Wed, 11 Aug 2021 20:18:55 +0300 Subject: [PATCH 470/599] Update docs/ru/sql-reference/functions/string-functions.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/sql-reference/functions/string-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/string-functions.md b/docs/ru/sql-reference/functions/string-functions.md index be1fdcdc929..516c5b23834 100644 --- a/docs/ru/sql-reference/functions/string-functions.md +++ b/docs/ru/sql-reference/functions/string-functions.md @@ -79,7 +79,7 @@ SELECT leftPad('abc', 7, '*'), leftPad('def', 7); ## leftPadUTF8 {#leftpadutf8} -Заполняет текущую строку слева пробелами или указанной строкой (несколько раз, если необходимо), пока результирующая строка не достигнет заданной длины. Соответствует MySQL функции `LPAD`. Функция [leftPad](#leftpad) измеряет длину строки в байтах, а функция `leftPadUTF8` — в кодовых точках Unicode. +Дополняет текущую строку слева пробелами или указанной строкой (несколько раз, если необходимо), пока результирующая строка не достигнет заданной длины. Соответствует MySQL функции `LPAD`. В отличие от функции [leftPad](#leftpad), измеряет длину строки не в байтах, а в кодовых точках Unicode. **Синтаксис** From beed4c21a52f753978ed305b54a132012678a062 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 11 Aug 2021 20:28:54 +0300 Subject: [PATCH 471/599] Use processors in Storage::watch --- src/Interpreters/InterpreterWatchQuery.cpp | 9 +- .../Transforms/SquashingChunksTransform.cpp | 27 ++++++ .../Transforms/SquashingChunksTransform.h | 24 ++++++ src/Storages/IStorage.h | 2 +- .../LiveView/LiveViewBlockOutputStream.h | 19 +++-- .../LiveView/LiveViewEventsBlockInputStream.h | 29 +++---- src/Storages/LiveView/StorageLiveView.cpp | 84 +++++++++++++------ src/Storages/LiveView/StorageLiveView.h | 4 +- src/Storages/StorageProxy.h | 2 +- 9 files changed, 139 insertions(+), 61 deletions(-) create mode 100644 src/Processors/Transforms/SquashingChunksTransform.cpp create mode 100644 src/Processors/Transforms/SquashingChunksTransform.h diff --git a/src/Interpreters/InterpreterWatchQuery.cpp b/src/Interpreters/InterpreterWatchQuery.cpp index edf0f37c00e..ee96045bbc4 100644 --- a/src/Interpreters/InterpreterWatchQuery.cpp +++ b/src/Interpreters/InterpreterWatchQuery.cpp @@ -71,10 +71,9 @@ BlockIO InterpreterWatchQuery::execute() QueryProcessingStage::Enum from_stage = QueryProcessingStage::FetchColumns; /// Watch storage - streams = storage->watch(required_columns, query_info, getContext(), from_stage, max_block_size, max_streams); + auto pipe = storage->watch(required_columns, query_info, getContext(), from_stage, max_block_size, max_streams); /// Constraints on the result, the quota on the result, and also callback for progress. - if (IBlockInputStream * stream = dynamic_cast(streams[0].get())) { StreamLocalLimits limits; limits.mode = LimitsMode::LIMITS_CURRENT; //-V1048 @@ -82,11 +81,11 @@ BlockIO InterpreterWatchQuery::execute() limits.size_limits.max_bytes = settings.max_result_bytes; limits.size_limits.overflow_mode = settings.result_overflow_mode; - stream->setLimits(limits); - stream->setQuota(getContext()->getQuota()); + pipe.setLimits(limits); + pipe.setQuota(getContext()->getQuota()); } - res.in = streams[0]; + res.pipeline.init(std::move(pipe)); return res; } diff --git a/src/Processors/Transforms/SquashingChunksTransform.cpp b/src/Processors/Transforms/SquashingChunksTransform.cpp new file mode 100644 index 00000000000..398ce9eb9fb --- /dev/null +++ b/src/Processors/Transforms/SquashingChunksTransform.cpp @@ -0,0 +1,27 @@ +#include + +namespace DB +{ + +SquashingChunksTransform::SquashingChunksTransform( + const Block & header, size_t min_block_size_rows, size_t min_block_size_bytes, bool reserve_memory) + : IAccumulatingTransform(header, header) + , squashing(min_block_size_rows, min_block_size_bytes, reserve_memory) +{ +} + +void SquashingChunksTransform::consume(Chunk chunk) +{ + if (auto block = squashing.add(getInputPort().getHeader().cloneWithColumns(chunk.detachColumns()))) + { + setReadyChunk(Chunk(block.getColumns(), block.rows())); + } +} + +Chunk SquashingChunksTransform::generate() +{ + auto block = squashing.add({}); + return Chunk(block.getColumns(), block.rows()); +} + +} diff --git a/src/Processors/Transforms/SquashingChunksTransform.h b/src/Processors/Transforms/SquashingChunksTransform.h new file mode 100644 index 00000000000..bcacf5abcda --- /dev/null +++ b/src/Processors/Transforms/SquashingChunksTransform.h @@ -0,0 +1,24 @@ +#pragma once +#include +#include + +namespace DB +{ + +class SquashingChunksTransform : public IAccumulatingTransform +{ +public: + explicit SquashingChunksTransform( + const Block & header, size_t min_block_size_rows, size_t min_block_size_bytes, bool reserve_memory = false); + + String getName() const override { return "SquashingTransform"; } + +protected: + void consume(Chunk chunk) override; + Chunk generate() override; + +private: + SquashingTransform squashing; +}; + +} diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index 2180f92df98..85bfbfb1f84 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -264,7 +264,7 @@ public: * * It is guaranteed that the structure of the table will not change over the lifetime of the returned streams (that is, there will not be ALTER, RENAME and DROP). */ - virtual BlockInputStreams watch( + virtual Pipe watch( const Names & /*column_names*/, const SelectQueryInfo & /*query_info*/, ContextPtr /*context*/, diff --git a/src/Storages/LiveView/LiveViewBlockOutputStream.h b/src/Storages/LiveView/LiveViewBlockOutputStream.h index 6b8a5a2cb9e..6101607c21a 100644 --- a/src/Storages/LiveView/LiveViewBlockOutputStream.h +++ b/src/Storages/LiveView/LiveViewBlockOutputStream.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include #include #include @@ -9,19 +9,21 @@ namespace DB { -class LiveViewBlockOutputStream : public IBlockOutputStream +class LiveViewSink : public SinkToStorage { public: - explicit LiveViewBlockOutputStream(StorageLiveView & storage_) : storage(storage_) {} + explicit LiveViewSink(StorageLiveView & storage_) : SinkToStorage(storage_.getHeader()), storage(storage_) {} - void writePrefix() override + String getName() const override { return "LiveViewSink"; } + + void onStart() override { new_blocks = std::make_shared(); new_blocks_metadata = std::make_shared(); new_hash = std::make_shared(); } - void writeSuffix() override + void onFinish() override { UInt128 key; String key_str; @@ -65,14 +67,13 @@ public: new_hash.reset(); } - void write(const Block & block) override + void consume(Chunk chunk) override { - new_blocks->push_back(block); + auto block = getPort().getHeader().cloneWithColumns(chunk.detachColumns()); block.updateHash(*new_hash); + new_blocks->push_back(std::move(block)); } - Block getHeader() const override { return storage.getHeader(); } - private: using SipHashPtr = std::shared_ptr; diff --git a/src/Storages/LiveView/LiveViewEventsBlockInputStream.h b/src/Storages/LiveView/LiveViewEventsBlockInputStream.h index dc6848ec20c..daf9edfef95 100644 --- a/src/Storages/LiveView/LiveViewEventsBlockInputStream.h +++ b/src/Storages/LiveView/LiveViewEventsBlockInputStream.h @@ -16,7 +16,7 @@ limitations under the License. */ #include #include #include -#include +#include #include @@ -27,7 +27,7 @@ namespace DB * Keeps stream alive by outputting blocks with no rows * based on period specified by the heartbeat interval. */ -class LiveViewEventsBlockInputStream : public IBlockInputStream +class LiveViewEventsSource : public SourceWithProgress { using NonBlockingResult = std::pair; @@ -35,13 +35,14 @@ using NonBlockingResult = std::pair; public: /// length default -2 because we want LIMIT to specify number of updates so that LIMIT 1 waits for 1 update /// and LIMIT 0 just returns data without waiting for any updates - LiveViewEventsBlockInputStream(std::shared_ptr storage_, + LiveViewEventsSource(std::shared_ptr storage_, std::shared_ptr blocks_ptr_, std::shared_ptr blocks_metadata_ptr_, std::shared_ptr active_ptr_, const bool has_limit_, const UInt64 limit_, const UInt64 heartbeat_interval_sec_) - : storage(std::move(storage_)), blocks_ptr(std::move(blocks_ptr_)), + : SourceWithProgress({ColumnWithTypeAndName(ColumnUInt64::create(), std::make_shared(), "version")}), + storage(std::move(storage_)), blocks_ptr(std::move(blocks_ptr_)), blocks_metadata_ptr(std::move(blocks_metadata_ptr_)), active_ptr(std::move(active_ptr_)), has_limit(has_limit_), limit(limit_), @@ -51,22 +52,17 @@ public: active = active_ptr.lock(); } - String getName() const override { return "LiveViewEventsBlockInputStream"; } + String getName() const override { return "LiveViewEventsSource"; } - void cancel(bool kill) override + void onCancel() override { if (isCancelled() || storage->shutdown_called) return; - IBlockInputStream::cancel(kill); + std::lock_guard lock(storage->mutex); storage->condition.notify_all(); } - Block getHeader() const override - { - return {ColumnWithTypeAndName(ColumnUInt64::create(), std::make_shared(), "version")}; - } - void refresh() { if (active && blocks && it == end) @@ -109,10 +105,11 @@ public: return res; } protected: - Block readImpl() override + Chunk generate() override { /// try reading - return tryReadImpl(true).first; + auto block = tryReadImpl(true).first; + return Chunk(block.getColumns(), block.rows()); } /** tryRead method attempts to read a block in either blocking @@ -170,7 +167,7 @@ protected: if (!end_of_blocks) { end_of_blocks = true; - return { getHeader(), true }; + return { getPort().getHeader(), true }; } while (true) { @@ -192,7 +189,7 @@ protected: { // repeat the event block as a heartbeat last_event_timestamp_usec = static_cast(timestamp.epochMicroseconds()); - return { getHeader(), true }; + return { getPort().getHeader(), true }; } } } diff --git a/src/Storages/LiveView/StorageLiveView.cpp b/src/Storages/LiveView/StorageLiveView.cpp index 5f5ce8a4a37..5ebb6cd2450 100644 --- a/src/Storages/LiveView/StorageLiveView.cpp +++ b/src/Storages/LiveView/StorageLiveView.cpp @@ -15,10 +15,10 @@ limitations under the License. */ #include #include #include -#include #include -#include -#include +#include +#include +#include #include #include #include @@ -110,15 +110,22 @@ MergeableBlocksPtr StorageLiveView::collectMergeableBlocks(ContextPtr local_cont InterpreterSelectQuery interpreter(mergeable_query->clone(), local_context, SelectQueryOptions(QueryProcessingStage::WithMergeableState), Names()); - auto view_mergeable_stream = std::make_shared(interpreter.execute().getInputStream()); + auto io = interpreter.execute(); + io.pipeline.addSimpleTransform([&](const Block & cur_header) + { + return std::make_shared(cur_header); + }); - while (Block this_block = view_mergeable_stream->read()) + PullingPipelineExecutor executor(io.pipeline); + Block this_block; + + while (executor.pull(this_block)) base_blocks->push_back(this_block); new_blocks->push_back(base_blocks); new_mergeable_blocks->blocks = new_blocks; - new_mergeable_blocks->sample_block = view_mergeable_stream->getHeader(); + new_mergeable_blocks->sample_block = io.pipeline.getHeader(); return new_mergeable_blocks; } @@ -133,7 +140,7 @@ Pipes StorageLiveView::blocksToPipes(BlocksPtrs blocks, Block & sample_block) } /// Complete query using input streams from mergeable blocks -BlockInputStreamPtr StorageLiveView::completeQuery(Pipes pipes) +QueryPipeline StorageLiveView::completeQuery(Pipes pipes) { //FIXME it's dangerous to create Context on stack auto block_context = Context::createCopy(getContext()); @@ -149,16 +156,24 @@ BlockInputStreamPtr StorageLiveView::completeQuery(Pipes pipes) block_context->addExternalTable(getBlocksTableName(), TemporaryTableHolder(getContext(), creator)); InterpreterSelectQuery select(getInnerBlocksQuery(), block_context, StoragePtr(), nullptr, SelectQueryOptions(QueryProcessingStage::Complete)); - BlockInputStreamPtr data = std::make_shared(select.execute().getInputStream()); + auto io = select.execute(); + io.pipeline.addSimpleTransform([&](const Block & cur_header) + { + return std::make_shared(cur_header); + }); /// Squashing is needed here because the view query can generate a lot of blocks /// even when only one block is inserted into the parent table (e.g. if the query is a GROUP BY /// and two-level aggregation is triggered). - data = std::make_shared( - data, getContext()->getSettingsRef().min_insert_block_size_rows, - getContext()->getSettingsRef().min_insert_block_size_bytes); + io.pipeline.addSimpleTransform([&](const Block & cur_header) + { + return std::make_shared( + cur_header, + getContext()->getSettingsRef().min_insert_block_size_rows, + getContext()->getSettingsRef().min_insert_block_size_bytes); + }); - return data; + return std::move(io.pipeline); } void StorageLiveView::writeIntoLiveView( @@ -166,7 +181,7 @@ void StorageLiveView::writeIntoLiveView( const Block & block, ContextPtr local_context) { - BlockOutputStreamPtr output = std::make_shared(live_view); + auto output = std::make_shared(live_view); /// Check if live view has any readers if not /// just reset blocks to empty and do nothing else @@ -220,10 +235,16 @@ void StorageLiveView::writeIntoLiveView( InterpreterSelectQuery select_block(mergeable_query, local_context, blocks_storage.getTable(), blocks_storage.getTable()->getInMemoryMetadataPtr(), QueryProcessingStage::WithMergeableState); - auto data_mergeable_stream = std::make_shared( - select_block.execute().getInputStream()); + auto io = select_block.execute(); + io.pipeline.addSimpleTransform([&](const Block & cur_header) + { + return std::make_shared(cur_header); + }); - while (Block this_block = data_mergeable_stream->read()) + PullingPipelineExecutor executor(io.pipeline); + Block this_block; + + while (executor.pull(this_block)) new_mergeable_blocks->push_back(this_block); if (new_mergeable_blocks->empty()) @@ -238,8 +259,15 @@ void StorageLiveView::writeIntoLiveView( } } - BlockInputStreamPtr data = live_view.completeQuery(std::move(from)); - copyData(*data, *output); + auto pipeline = live_view.completeQuery(std::move(from)); + pipeline.resize(1); + pipeline.setSinks([&](const Block &, Pipe::StreamType) + { + return std::move(output); + }); + + auto executor = pipeline.execute(); + executor->execute(pipeline.getNumThreads()); } @@ -351,9 +379,11 @@ bool StorageLiveView::getNewBlocks() /// inserted data to be duplicated auto new_mergeable_blocks = collectMergeableBlocks(live_view_context); Pipes from = blocksToPipes(new_mergeable_blocks->blocks, new_mergeable_blocks->sample_block); - BlockInputStreamPtr data = completeQuery(std::move(from)); + auto pipeline = completeQuery(std::move(from)); - while (Block block = data->read()) + PullingPipelineExecutor executor(pipeline); + Block block; + while (executor.pull(block)) { /// calculate hash before virtual column is added block.updateHash(hash); @@ -521,7 +551,7 @@ Pipe StorageLiveView::read( return Pipe(std::make_shared(blocks_ptr, getHeader())); } -BlockInputStreams StorageLiveView::watch( +Pipe StorageLiveView::watch( const Names & /*column_names*/, const SelectQueryInfo & query_info, ContextPtr local_context, @@ -533,7 +563,7 @@ BlockInputStreams StorageLiveView::watch( bool has_limit = false; UInt64 limit = 0; - BlockInputStreamPtr reader; + Pipe reader; if (query.limit_length) { @@ -542,15 +572,15 @@ BlockInputStreams StorageLiveView::watch( } if (query.is_watch_events) - reader = std::make_shared( + reader = Pipe(std::make_shared( std::static_pointer_cast(shared_from_this()), blocks_ptr, blocks_metadata_ptr, active_ptr, has_limit, limit, - local_context->getSettingsRef().live_view_heartbeat_interval.totalSeconds()); + local_context->getSettingsRef().live_view_heartbeat_interval.totalSeconds())); else - reader = std::make_shared( + reader = Pipe(std::make_shared( std::static_pointer_cast(shared_from_this()), blocks_ptr, blocks_metadata_ptr, active_ptr, has_limit, limit, - local_context->getSettingsRef().live_view_heartbeat_interval.totalSeconds()); + local_context->getSettingsRef().live_view_heartbeat_interval.totalSeconds())); { std::lock_guard lock(mutex); @@ -563,7 +593,7 @@ BlockInputStreams StorageLiveView::watch( } processed_stage = QueryProcessingStage::Complete; - return { reader }; + return reader; } NamesAndTypesList StorageLiveView::getVirtuals() const diff --git a/src/Storages/LiveView/StorageLiveView.h b/src/Storages/LiveView/StorageLiveView.h index 5f66a97090f..15afc642989 100644 --- a/src/Storages/LiveView/StorageLiveView.h +++ b/src/Storages/LiveView/StorageLiveView.h @@ -153,7 +153,7 @@ public: size_t max_block_size, unsigned num_streams) override; - BlockInputStreams watch( + Pipe watch( const Names & column_names, const SelectQueryInfo & query_info, ContextPtr context, @@ -167,7 +167,7 @@ public: /// Collect mergeable blocks and their sample. Must be called holding mutex MergeableBlocksPtr collectMergeableBlocks(ContextPtr context); /// Complete query using input streams from mergeable blocks - BlockInputStreamPtr completeQuery(Pipes pipes); + QueryPipeline completeQuery(Pipes pipes); void setMergeableBlocks(MergeableBlocksPtr blocks) { mergeable_blocks = blocks; } std::shared_ptr getActivePtr() { return active_ptr; } diff --git a/src/Storages/StorageProxy.h b/src/Storages/StorageProxy.h index 521a2b8d642..c81ef6febdc 100644 --- a/src/Storages/StorageProxy.h +++ b/src/Storages/StorageProxy.h @@ -40,7 +40,7 @@ public: return getNested()->getQueryProcessingStage(context, to_stage, getNested()->getInMemoryMetadataPtr(), info); } - BlockInputStreams watch( + Pipe watch( const Names & column_names, const SelectQueryInfo & query_info, ContextPtr context, From dbf4d14b42b6468191a4bc91cfd71feae9ce8545 Mon Sep 17 00:00:00 2001 From: Braulio Valdivielso Date: Wed, 11 Aug 2021 18:33:13 +0100 Subject: [PATCH 472/599] remove unused extern declaration --- src/Functions/tupleToNameValuePairs.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Functions/tupleToNameValuePairs.cpp b/src/Functions/tupleToNameValuePairs.cpp index 05938955313..8478d742b3f 100644 --- a/src/Functions/tupleToNameValuePairs.cpp +++ b/src/Functions/tupleToNameValuePairs.cpp @@ -17,7 +17,6 @@ namespace DB namespace ErrorCodes { extern const int ILLEGAL_TYPE_OF_ARGUMENT; - extern const int ILLEGAL_INDEX; } namespace From 94babf1c8b2153944c6fb2ca10f41a40260ad6f7 Mon Sep 17 00:00:00 2001 From: Braulio Valdivielso Date: Wed, 11 Aug 2021 18:38:27 +0100 Subject: [PATCH 473/599] rename tests --- ...02008_tuple_to_name_value_pairs.reference} | 0 ...ql => 02008_tuple_to_name_value_pairs.sql} | 20 +++++++++---------- 2 files changed, 10 insertions(+), 10 deletions(-) rename tests/queries/0_stateless/{02007_tuple_to_name_value_pairs.reference => 02008_tuple_to_name_value_pairs.reference} (100%) rename tests/queries/0_stateless/{02007_tuple_to_name_value_pairs.sql => 02008_tuple_to_name_value_pairs.sql} (59%) diff --git a/tests/queries/0_stateless/02007_tuple_to_name_value_pairs.reference b/tests/queries/0_stateless/02008_tuple_to_name_value_pairs.reference similarity index 100% rename from tests/queries/0_stateless/02007_tuple_to_name_value_pairs.reference rename to tests/queries/0_stateless/02008_tuple_to_name_value_pairs.reference diff --git a/tests/queries/0_stateless/02007_tuple_to_name_value_pairs.sql b/tests/queries/0_stateless/02008_tuple_to_name_value_pairs.sql similarity index 59% rename from tests/queries/0_stateless/02007_tuple_to_name_value_pairs.sql rename to tests/queries/0_stateless/02008_tuple_to_name_value_pairs.sql index e80df4ebf28..9204975b579 100644 --- a/tests/queries/0_stateless/02007_tuple_to_name_value_pairs.sql +++ b/tests/queries/0_stateless/02008_tuple_to_name_value_pairs.sql @@ -1,24 +1,24 @@ SELECT tupleToNameValuePairs(tuple(1, 2, 3)); -DROP TABLE IF EXISTS test02007; -CREATE TABLE test02007 ( +DROP TABLE IF EXISTS test02008; +CREATE TABLE test02008 ( col Tuple( a Tuple(key1 int, key2 int), b Tuple(key1 int, key3 int) ) ) ENGINE=Memory(); -INSERT INTO test02007 VALUES (tuple(tuple(1, 2), tuple(3, 4))); -INSERT INTO test02007 VALUES (tuple(tuple(5, 6), tuple(7, 8))); -SELECT tupleToNameValuePairs(col) FROM test02007 ORDER BY col; +INSERT INTO test02008 VALUES (tuple(tuple(1, 2), tuple(3, 4))); +INSERT INTO test02008 VALUES (tuple(tuple(5, 6), tuple(7, 8))); +SELECT tupleToNameValuePairs(col) FROM test02008 ORDER BY col; -DROP TABLE IF EXISTS test02007; -CREATE TABLE test02007 ( +DROP TABLE IF EXISTS test02008; +CREATE TABLE test02008 ( col Tuple(CPU double, Memory double, Disk double) ) ENGINE=Memory(); -INSERT INTO test02007 VALUES (tuple(3.3, 5.5, 6.6)); -SELECT untuple(arrayJoin(tupleToNameValuePairs(col))) from test02007; +INSERT INTO test02008 VALUES (tuple(3.3, 5.5, 6.6)); +SELECT untuple(arrayJoin(tupleToNameValuePairs(col))) from test02008; -DROP TABLE IF EXISTS test02007; +DROP TABLE IF EXISTS test02008; SELECT tupleToNameValuePairs(tuple(1, 1.3)); -- { serverError 43; } SELECT tupleToNameValuePairs(tuple(1, [1,2])); -- { serverError 43; } SELECT tupleToNameValuePairs(tuple(1, 'a')); -- { serverError 43; } From 79290f8e2e2918a619edd28c1b9d5c3f07c03cb1 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 11 Aug 2021 20:49:19 +0300 Subject: [PATCH 474/599] Another try --- src/Common/DenseHashMap.h | 2 +- src/Common/DenseHashSet.h | 2 +- src/Common/SparseHashMap.h | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Common/DenseHashMap.h b/src/Common/DenseHashMap.h index 9054d049e07..26b9224528b 100644 --- a/src/Common/DenseHashMap.h +++ b/src/Common/DenseHashMap.h @@ -4,7 +4,7 @@ #if defined(ARCADIA_BUILD) #define HASH_FUN_H template -struct THash : public std::hash {}; +struct THash; #endif #include diff --git a/src/Common/DenseHashSet.h b/src/Common/DenseHashSet.h index 7f28585d9db..5b30d88a70c 100644 --- a/src/Common/DenseHashSet.h +++ b/src/Common/DenseHashSet.h @@ -3,7 +3,7 @@ #if defined(ARCADIA_BUILD) #define HASH_FUN_H template -struct THash : public std::hash {}; +struct THash; #endif #include diff --git a/src/Common/SparseHashMap.h b/src/Common/SparseHashMap.h index 7464ebf831a..403042c11a9 100644 --- a/src/Common/SparseHashMap.h +++ b/src/Common/SparseHashMap.h @@ -3,7 +3,7 @@ #if defined(ARCADIA_BUILD) #define HASH_FUN_H template -struct THash : public std::hash {}; +struct THash; #endif #include From 739ffe97b658bbcdc5d08b50c48e9cc9fb5540ad Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 11 Aug 2021 21:07:18 +0300 Subject: [PATCH 475/599] Fix ya.make. --- src/Common/ya.make | 2 ++ src/Functions/ya.make | 1 + 2 files changed, 3 insertions(+) diff --git a/src/Common/ya.make b/src/Common/ya.make index 60dfd5f6bee..82962123e56 100644 --- a/src/Common/ya.make +++ b/src/Common/ya.make @@ -102,6 +102,7 @@ SRCS( ZooKeeper/ZooKeeperNodeCache.cpp checkStackSize.cpp clearPasswordFromCommandLine.cpp + clickhouse_malloc.cpp createHardLink.cpp escapeForFileName.cpp filesystemHelpers.cpp @@ -116,6 +117,7 @@ SRCS( hex.cpp isLocalAddress.cpp malloc.cpp + memory.cpp new_delete.cpp parseAddress.cpp parseGlobs.cpp diff --git a/src/Functions/ya.make b/src/Functions/ya.make index 2b9b3d94313..71822924ee3 100644 --- a/src/Functions/ya.make +++ b/src/Functions/ya.make @@ -312,6 +312,7 @@ SRCS( hasToken.cpp hasTokenCaseInsensitive.cpp hostName.cpp + hyperscanRegexpChecker.cpp hypot.cpp identity.cpp if.cpp From 7555fc19fd47d2067e55b0b3aeb1498869226284 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 11 Aug 2021 18:49:25 +0000 Subject: [PATCH 476/599] Revert "Move some templates" This reverts commit 7f247becca1e483489ff46d4af28b3437693f1c9. --- src/Functions/FunctionsConversion.cpp | 1 - src/Functions/FunctionsConversion.h | 17 +++++++++++++++++ src/Functions/cast_overload_fwd.h | 24 ------------------------ src/Interpreters/ActionsDAG.cpp | 1 - src/Interpreters/castColumn.cpp | 1 - src/Storages/MergeTree/KeyCondition.cpp | 1 - 6 files changed, 17 insertions(+), 28 deletions(-) delete mode 100644 src/Functions/cast_overload_fwd.h diff --git a/src/Functions/FunctionsConversion.cpp b/src/Functions/FunctionsConversion.cpp index ae115a6e90d..cdbd32b189c 100644 --- a/src/Functions/FunctionsConversion.cpp +++ b/src/Functions/FunctionsConversion.cpp @@ -1,6 +1,5 @@ #include #include -#include namespace DB diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index 941a2fae0ab..362249b3b27 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -3485,4 +3485,21 @@ private: }; +struct CastOverloadName +{ + static constexpr auto cast_name = "CAST"; + static constexpr auto accurate_cast_name = "accurateCast"; + static constexpr auto accurate_cast_or_null_name = "accurateCastOrNull"; +}; + +struct CastInternalOverloadName +{ + static constexpr auto cast_name = "_CAST"; + static constexpr auto accurate_cast_name = "accurate_Cast"; + static constexpr auto accurate_cast_or_null_name = "accurate_CastOrNull"; +}; + +template using CastOverloadResolver = CastOverloadResolverImpl; +template using CastInternalOverloadResolver = CastOverloadResolverImpl; + } diff --git a/src/Functions/cast_overload_fwd.h b/src/Functions/cast_overload_fwd.h deleted file mode 100644 index 1264d01b5d9..00000000000 --- a/src/Functions/cast_overload_fwd.h +++ /dev/null @@ -1,24 +0,0 @@ -#pragma once -#include - -namespace DB -{ - -struct CastOverloadName -{ - static constexpr auto cast_name = "CAST"; - static constexpr auto accurate_cast_name = "accurateCast"; - static constexpr auto accurate_cast_or_null_name = "accurateCastOrNull"; -}; - -struct CastInternalOverloadName -{ - static constexpr auto cast_name = "_CAST"; - static constexpr auto accurate_cast_name = "accurate_Cast"; - static constexpr auto accurate_cast_or_null_name = "accurate_CastOrNull"; -}; - -template using CastOverloadResolver = CastOverloadResolverImpl; -template using CastInternalOverloadResolver = CastOverloadResolverImpl; - -} diff --git a/src/Interpreters/ActionsDAG.cpp b/src/Interpreters/ActionsDAG.cpp index 664b0d61a4f..a42d6053e9a 100644 --- a/src/Interpreters/ActionsDAG.cpp +++ b/src/Interpreters/ActionsDAG.cpp @@ -7,7 +7,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Interpreters/castColumn.cpp b/src/Interpreters/castColumn.cpp index 4c3c564ce58..3356d37ba7f 100644 --- a/src/Interpreters/castColumn.cpp +++ b/src/Interpreters/castColumn.cpp @@ -1,7 +1,6 @@ #include #include -#include namespace DB { diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index 751ac9fd9d9..e6dfdb859b8 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -11,7 +11,6 @@ #include #include #include -#include #include #include #include From dcc6f597dcae1511d9644c6a346972ddad5c7993 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 11 Aug 2021 19:09:51 +0000 Subject: [PATCH 477/599] Move templates from FunctionsConvertion --- src/Functions/CastOverloadResolver.cpp | 19 +++ src/Functions/CastOverloadResolver.h | 117 ++++++++++++++++++ .../DateOrDateTimeFunctionsConvertion.cpp | 0 src/Functions/FunctionsConversion.cpp | 9 +- src/Functions/FunctionsConversion.h | 110 ---------------- src/Interpreters/ActionsDAG.cpp | 1 + src/Interpreters/castColumn.cpp | 1 + src/Storages/MergeTree/KeyCondition.cpp | 1 + 8 files changed, 142 insertions(+), 116 deletions(-) create mode 100644 src/Functions/CastOverloadResolver.cpp create mode 100644 src/Functions/CastOverloadResolver.h create mode 100644 src/Functions/DateOrDateTimeFunctionsConvertion.cpp diff --git a/src/Functions/CastOverloadResolver.cpp b/src/Functions/CastOverloadResolver.cpp new file mode 100644 index 00000000000..fd6fecc37d6 --- /dev/null +++ b/src/Functions/CastOverloadResolver.cpp @@ -0,0 +1,19 @@ +#include +#include + + +namespace DB +{ + +void registerCastOverloadResolvers(FunctionFactory & factory) +{ + factory.registerFunction>(FunctionFactory::CaseInsensitive); + factory.registerFunction>(); + factory.registerFunction>(); + + factory.registerFunction>(FunctionFactory::CaseInsensitive); + factory.registerFunction>(); + factory.registerFunction>(); +} + +} diff --git a/src/Functions/CastOverloadResolver.h b/src/Functions/CastOverloadResolver.h new file mode 100644 index 00000000000..c5f6489fc3c --- /dev/null +++ b/src/Functions/CastOverloadResolver.h @@ -0,0 +1,117 @@ +#pragma once +#include + + +namespace DB +{ + +/* + * CastInternal does not preserve nullability of the data type, + * i.e. CastInternal(toNullable(toInt8(1)) as Int32) will be Int32(1). + * + * Cast preserves nullability according to setting `cast_keep_nullable`, + * i.e. Cast(toNullable(toInt8(1)) as Int32) will be Nullable(Int32(1)) if `cast_keep_nullable` == 1. +**/ +template +class CastOverloadResolverImpl : public IFunctionOverloadResolver +{ +public: + using MonotonicityForRange = FunctionCastBase::MonotonicityForRange; + using Diagnostic = FunctionCastBase::Diagnostic; + + static constexpr auto name = cast_type == CastType::accurate + ? CastName::accurate_cast_name + : (cast_type == CastType::accurateOrNull ? CastName::accurate_cast_or_null_name : CastName::cast_name); + + String getName() const override { return name; } + + size_t getNumberOfArguments() const override { return 2; } + + ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; } + + explicit CastOverloadResolverImpl(std::optional diagnostic_, bool keep_nullable_) + : diagnostic(std::move(diagnostic_)), keep_nullable(keep_nullable_) + { + } + + static FunctionOverloadResolverPtr create(ContextPtr context) + { + if constexpr (internal) + return createImpl(); + return createImpl({}, context->getSettingsRef().cast_keep_nullable); + } + + static FunctionOverloadResolverPtr createImpl(std::optional diagnostic = {}, bool keep_nullable = false) + { + assert(!internal || !keep_nullable); + return std::make_unique(std::move(diagnostic), keep_nullable); + } + +protected: + + FunctionBasePtr buildImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & return_type) const override + { + DataTypes data_types(arguments.size()); + + for (size_t i = 0; i < arguments.size(); ++i) + data_types[i] = arguments[i].type; + + auto monotonicity = MonotonicityHelper::getMonotonicityInformation(arguments.front().type, return_type.get()); + return std::make_unique>(name, std::move(monotonicity), data_types, return_type, diagnostic, cast_type); + } + + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override + { + const auto & column = arguments.back().column; + if (!column) + throw Exception("Second argument to " + getName() + " must be a constant string describing type." + " Instead there is non-constant column of type " + arguments.back().type->getName(), + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + const auto * type_col = checkAndGetColumnConst(column.get()); + if (!type_col) + throw Exception("Second argument to " + getName() + " must be a constant string describing type." + " Instead there is a column with the following structure: " + column->dumpStructure(), + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + DataTypePtr type = DataTypeFactory::instance().get(type_col->getValue()); + + if constexpr (cast_type == CastType::accurateOrNull) + return makeNullable(type); + + if constexpr (internal) + return type; + + if (keep_nullable && arguments.front().type->isNullable() && type->canBeInsideNullable()) + return makeNullable(type); + + return type; + } + + bool useDefaultImplementationForNulls() const override { return false; } + bool useDefaultImplementationForLowCardinalityColumns() const override { return false; } + +private: + std::optional diagnostic; + bool keep_nullable; +}; + + +struct CastOverloadName +{ + static constexpr auto cast_name = "CAST"; + static constexpr auto accurate_cast_name = "accurateCast"; + static constexpr auto accurate_cast_or_null_name = "accurateCastOrNull"; +}; + +struct CastInternalOverloadName +{ + static constexpr auto cast_name = "_CAST"; + static constexpr auto accurate_cast_name = "accurate_Cast"; + static constexpr auto accurate_cast_or_null_name = "accurate_CastOrNull"; +}; + +template using CastOverloadResolver = CastOverloadResolverImpl; +template using CastInternalOverloadResolver = CastOverloadResolverImpl; + +} diff --git a/src/Functions/DateOrDateTimeFunctionsConvertion.cpp b/src/Functions/DateOrDateTimeFunctionsConvertion.cpp new file mode 100644 index 00000000000..e69de29bb2d diff --git a/src/Functions/FunctionsConversion.cpp b/src/Functions/FunctionsConversion.cpp index cdbd32b189c..f32d5df8a21 100644 --- a/src/Functions/FunctionsConversion.cpp +++ b/src/Functions/FunctionsConversion.cpp @@ -7,6 +7,8 @@ namespace DB void registerFunctionFixedString(FunctionFactory & factory); +void registerCastOverloadResolvers(FunctionFactory & factory); + void registerFunctionsConversion(FunctionFactory & factory) { factory.registerFunction(); @@ -43,12 +45,7 @@ void registerFunctionsConversion(FunctionFactory & factory) factory.registerFunction(); - factory.registerFunction>(FunctionFactory::CaseInsensitive); - factory.registerFunction>(); - factory.registerFunction>(); - factory.registerFunction>(FunctionFactory::CaseInsensitive); - factory.registerFunction>(); - factory.registerFunction>(); + registerCastOverloadResolvers(factory); factory.registerFunction(); factory.registerFunction(); diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index 362249b3b27..e57998e4a72 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -3392,114 +3392,4 @@ public: } }; - -/* - * CastInternal does not preserve nullability of the data type, - * i.e. CastInternal(toNullable(toInt8(1)) as Int32) will be Int32(1). - * - * Cast preserves nullability according to setting `cast_keep_nullable`, - * i.e. Cast(toNullable(toInt8(1)) as Int32) will be Nullable(Int32(1)) if `cast_keep_nullable` == 1. -**/ -template -class CastOverloadResolverImpl : public IFunctionOverloadResolver -{ -public: - using MonotonicityForRange = FunctionCastBase::MonotonicityForRange; - using Diagnostic = FunctionCastBase::Diagnostic; - - static constexpr auto name = cast_type == CastType::accurate - ? CastName::accurate_cast_name - : (cast_type == CastType::accurateOrNull ? CastName::accurate_cast_or_null_name : CastName::cast_name); - - String getName() const override { return name; } - - size_t getNumberOfArguments() const override { return 2; } - - ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; } - - explicit CastOverloadResolverImpl(std::optional diagnostic_, bool keep_nullable_) - : diagnostic(std::move(diagnostic_)), keep_nullable(keep_nullable_) - { - } - - static FunctionOverloadResolverPtr create(ContextPtr context) - { - if constexpr (internal) - return createImpl(); - return createImpl({}, context->getSettingsRef().cast_keep_nullable); - } - - static FunctionOverloadResolverPtr createImpl(std::optional diagnostic = {}, bool keep_nullable = false) - { - assert(!internal || !keep_nullable); - return std::make_unique(std::move(diagnostic), keep_nullable); - } - -protected: - - FunctionBasePtr buildImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & return_type) const override - { - DataTypes data_types(arguments.size()); - - for (size_t i = 0; i < arguments.size(); ++i) - data_types[i] = arguments[i].type; - - auto monotonicity = MonotonicityHelper::getMonotonicityInformation(arguments.front().type, return_type.get()); - return std::make_unique>(name, std::move(monotonicity), data_types, return_type, diagnostic, cast_type); - } - - DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override - { - const auto & column = arguments.back().column; - if (!column) - throw Exception("Second argument to " + getName() + " must be a constant string describing type." - " Instead there is non-constant column of type " + arguments.back().type->getName(), - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - const auto * type_col = checkAndGetColumnConst(column.get()); - if (!type_col) - throw Exception("Second argument to " + getName() + " must be a constant string describing type." - " Instead there is a column with the following structure: " + column->dumpStructure(), - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - DataTypePtr type = DataTypeFactory::instance().get(type_col->getValue()); - - if constexpr (cast_type == CastType::accurateOrNull) - return makeNullable(type); - - if constexpr (internal) - return type; - - if (keep_nullable && arguments.front().type->isNullable() && type->canBeInsideNullable()) - return makeNullable(type); - - return type; - } - - bool useDefaultImplementationForNulls() const override { return false; } - bool useDefaultImplementationForLowCardinalityColumns() const override { return false; } - -private: - std::optional diagnostic; - bool keep_nullable; -}; - - -struct CastOverloadName -{ - static constexpr auto cast_name = "CAST"; - static constexpr auto accurate_cast_name = "accurateCast"; - static constexpr auto accurate_cast_or_null_name = "accurateCastOrNull"; -}; - -struct CastInternalOverloadName -{ - static constexpr auto cast_name = "_CAST"; - static constexpr auto accurate_cast_name = "accurate_Cast"; - static constexpr auto accurate_cast_or_null_name = "accurate_CastOrNull"; -}; - -template using CastOverloadResolver = CastOverloadResolverImpl; -template using CastInternalOverloadResolver = CastOverloadResolverImpl; - } diff --git a/src/Interpreters/ActionsDAG.cpp b/src/Interpreters/ActionsDAG.cpp index a42d6053e9a..e1f1d498367 100644 --- a/src/Interpreters/ActionsDAG.cpp +++ b/src/Interpreters/ActionsDAG.cpp @@ -7,6 +7,7 @@ #include #include #include +#include #include #include #include diff --git a/src/Interpreters/castColumn.cpp b/src/Interpreters/castColumn.cpp index 3356d37ba7f..fd71e02ee7e 100644 --- a/src/Interpreters/castColumn.cpp +++ b/src/Interpreters/castColumn.cpp @@ -1,6 +1,7 @@ #include #include +#include namespace DB { diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index e6dfdb859b8..18ca00ebf0d 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -10,6 +10,7 @@ #include #include #include +#include #include #include #include From 124a87684f635443058cc4631f19bb5700e8341a Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Wed, 11 Aug 2021 19:44:34 +0300 Subject: [PATCH 478/599] Removed some data streams --- src/Common/ya.make | 2 + src/Core/ExternalResultDescription.h | 2 +- ...BlockInputStream.cpp => MongoDBSource.cpp} | 3 +- ...goDBBlockInputStream.h => MongoDBSource.h} | 0 ...ckInputStream.cpp => PostgreSQLSource.cpp} | 2 +- ...LBlockInputStream.h => PostgreSQLSource.h} | 0 ...eBlockInputStream.cpp => SQLiteSource.cpp} | 48 +++++++------------ ...QLiteBlockInputStream.h => SQLiteSource.h} | 18 +++---- src/DataStreams/ya.make | 4 +- src/Databases/MySQL/DatabaseMySQL.cpp | 2 +- .../MySQL/FetchTablesColumnsList.cpp | 2 +- src/Databases/MySQL/MaterializeMetadata.cpp | 2 +- .../MySQL/MaterializedMySQLSyncThread.cpp | 2 +- .../CassandraDictionarySource.cpp | 6 +-- ...ockInputStream.cpp => CassandraSource.cpp} | 2 +- ...raBlockInputStream.h => CassandraSource.h} | 0 src/Dictionaries/MongoDBDictionarySource.cpp | 2 +- src/Dictionaries/MySQLDictionarySource.h | 2 +- .../PostgreSQLDictionarySource.cpp | 2 +- src/Dictionaries/ya.make | 2 +- ...QLBlockInputStream.cpp => MySQLSource.cpp} | 2 +- ...{MySQLBlockInputStream.h => MySQLSource.h} | 2 +- src/Formats/ya.make | 2 +- src/Functions/ya.make | 1 + .../PostgreSQLReplicationHandler.cpp | 2 +- src/Storages/StorageMongoDB.cpp | 2 +- src/Storages/StorageMySQL.cpp | 2 +- src/Storages/StoragePostgreSQL.cpp | 2 +- src/Storages/StorageSQLite.cpp | 5 +- src/TableFunctions/TableFunctionMySQL.cpp | 2 +- 30 files changed, 55 insertions(+), 70 deletions(-) rename src/DataStreams/{MongoDBBlockInputStream.cpp => MongoDBSource.cpp} (99%) rename src/DataStreams/{MongoDBBlockInputStream.h => MongoDBSource.h} (100%) rename src/DataStreams/{PostgreSQLBlockInputStream.cpp => PostgreSQLSource.cpp} (99%) rename src/DataStreams/{PostgreSQLBlockInputStream.h => PostgreSQLSource.h} (100%) rename src/DataStreams/{SQLiteBlockInputStream.cpp => SQLiteSource.cpp} (81%) rename src/DataStreams/{SQLiteBlockInputStream.h => SQLiteSource.h} (63%) rename src/Dictionaries/{CassandraBlockInputStream.cpp => CassandraSource.cpp} (99%) rename src/Dictionaries/{CassandraBlockInputStream.h => CassandraSource.h} (100%) rename src/Formats/{MySQLBlockInputStream.cpp => MySQLSource.cpp} (99%) rename src/Formats/{MySQLBlockInputStream.h => MySQLSource.h} (96%) diff --git a/src/Common/ya.make b/src/Common/ya.make index 60dfd5f6bee..82962123e56 100644 --- a/src/Common/ya.make +++ b/src/Common/ya.make @@ -102,6 +102,7 @@ SRCS( ZooKeeper/ZooKeeperNodeCache.cpp checkStackSize.cpp clearPasswordFromCommandLine.cpp + clickhouse_malloc.cpp createHardLink.cpp escapeForFileName.cpp filesystemHelpers.cpp @@ -116,6 +117,7 @@ SRCS( hex.cpp isLocalAddress.cpp malloc.cpp + memory.cpp new_delete.cpp parseAddress.cpp parseGlobs.cpp diff --git a/src/Core/ExternalResultDescription.h b/src/Core/ExternalResultDescription.h index 78c054e805f..a9ffe8b2ed2 100644 --- a/src/Core/ExternalResultDescription.h +++ b/src/Core/ExternalResultDescription.h @@ -6,7 +6,7 @@ namespace DB { -/** Common part for implementation of MySQLBlockInputStream, MongoDBBlockInputStream and others. +/** Common part for implementation of MySQLSource, MongoDBSource and others. */ struct ExternalResultDescription { diff --git a/src/DataStreams/MongoDBBlockInputStream.cpp b/src/DataStreams/MongoDBSource.cpp similarity index 99% rename from src/DataStreams/MongoDBBlockInputStream.cpp rename to src/DataStreams/MongoDBSource.cpp index a0a8e3e40a5..c00d214249a 100644 --- a/src/DataStreams/MongoDBBlockInputStream.cpp +++ b/src/DataStreams/MongoDBSource.cpp @@ -1,3 +1,5 @@ +#include "MongoDBSource.h" + #include #include @@ -15,7 +17,6 @@ #include #include #include -#include #include #include #include diff --git a/src/DataStreams/MongoDBBlockInputStream.h b/src/DataStreams/MongoDBSource.h similarity index 100% rename from src/DataStreams/MongoDBBlockInputStream.h rename to src/DataStreams/MongoDBSource.h diff --git a/src/DataStreams/PostgreSQLBlockInputStream.cpp b/src/DataStreams/PostgreSQLSource.cpp similarity index 99% rename from src/DataStreams/PostgreSQLBlockInputStream.cpp rename to src/DataStreams/PostgreSQLSource.cpp index 0b50c453629..c3bde8c84ad 100644 --- a/src/DataStreams/PostgreSQLBlockInputStream.cpp +++ b/src/DataStreams/PostgreSQLSource.cpp @@ -1,4 +1,4 @@ -#include "PostgreSQLBlockInputStream.h" +#include "PostgreSQLSource.h" #if USE_LIBPQXX #include diff --git a/src/DataStreams/PostgreSQLBlockInputStream.h b/src/DataStreams/PostgreSQLSource.h similarity index 100% rename from src/DataStreams/PostgreSQLBlockInputStream.h rename to src/DataStreams/PostgreSQLSource.h diff --git a/src/DataStreams/SQLiteBlockInputStream.cpp b/src/DataStreams/SQLiteSource.cpp similarity index 81% rename from src/DataStreams/SQLiteBlockInputStream.cpp rename to src/DataStreams/SQLiteSource.cpp index da7645d968d..f4995703a1e 100644 --- a/src/DataStreams/SQLiteBlockInputStream.cpp +++ b/src/DataStreams/SQLiteSource.cpp @@ -1,4 +1,4 @@ -#include "SQLiteBlockInputStream.h" +#include "SQLiteSource.h" #if USE_SQLITE #include @@ -22,37 +22,33 @@ namespace ErrorCodes extern const int SQLITE_ENGINE_ERROR; } -SQLiteBlockInputStream::SQLiteBlockInputStream( +SQLiteSource::SQLiteSource( SQLitePtr sqlite_db_, const String & query_str_, const Block & sample_block, const UInt64 max_block_size_) - : query_str(query_str_) + : SourceWithProgress(sample_block.cloneEmpty()) + , query_str(query_str_) , max_block_size(max_block_size_) , sqlite_db(std::move(sqlite_db_)) { description.init(sample_block); } - -void SQLiteBlockInputStream::readPrefix() -{ - sqlite3_stmt * compiled_stmt = nullptr; - int status = sqlite3_prepare_v2(sqlite_db.get(), query_str.c_str(), query_str.size() + 1, &compiled_stmt, nullptr); - - if (status != SQLITE_OK) - throw Exception(ErrorCodes::SQLITE_ENGINE_ERROR, - "Cannot prepate sqlite statement. Status: {}. Message: {}", - status, sqlite3_errstr(status)); - - compiled_statement = std::unique_ptr(compiled_stmt, StatementDeleter()); -} - - -Block SQLiteBlockInputStream::readImpl() +Chunk SQLiteSource::generate() { if (!compiled_statement) - return Block(); + { + sqlite3_stmt * compiled_stmt = nullptr; + int status = sqlite3_prepare_v2(sqlite_db.get(), query_str.c_str(), query_str.size() + 1, &compiled_stmt, nullptr); + + if (status != SQLITE_OK) + throw Exception(ErrorCodes::SQLITE_ENGINE_ERROR, + "Cannot prepate sqlite statement. Status: {}. Message: {}", + status, sqlite3_errstr(status)); + + compiled_statement = std::unique_ptr(compiled_stmt, StatementDeleter()); + } MutableColumns columns = description.sample_block.cloneEmptyColumns(); size_t num_rows = 0; @@ -104,18 +100,10 @@ Block SQLiteBlockInputStream::readImpl() break; } - return description.sample_block.cloneWithColumns(std::move(columns)); + return Chunk(std::move(columns), num_rows); } - -void SQLiteBlockInputStream::readSuffix() -{ - if (compiled_statement) - compiled_statement.reset(); -} - - -void SQLiteBlockInputStream::insertValue(IColumn & column, const ExternalResultDescription::ValueType type, size_t idx) +void SQLiteSource::insertValue(IColumn & column, ExternalResultDescription::ValueType type, size_t idx) { switch (type) { diff --git a/src/DataStreams/SQLiteBlockInputStream.h b/src/DataStreams/SQLiteSource.h similarity index 63% rename from src/DataStreams/SQLiteBlockInputStream.h rename to src/DataStreams/SQLiteSource.h index 35fc4801b4b..653fdb402e3 100644 --- a/src/DataStreams/SQLiteBlockInputStream.h +++ b/src/DataStreams/SQLiteSource.h @@ -6,29 +6,27 @@ #if USE_SQLITE #include -#include +#include #include // Y_IGNORE namespace DB { -class SQLiteBlockInputStream : public IBlockInputStream +class SQLiteSource : public SourceWithProgress { using SQLitePtr = std::shared_ptr; public: - SQLiteBlockInputStream(SQLitePtr sqlite_db_, + SQLiteSource(SQLitePtr sqlite_db_, const String & query_str_, const Block & sample_block, UInt64 max_block_size_); String getName() const override { return "SQLite"; } - Block getHeader() const override { return description.sample_block.cloneEmpty(); } - private: - void insertDefaultSQLiteValue(IColumn & column, const IColumn & sample_column) + static void insertDefaultSQLiteValue(IColumn & column, const IColumn & sample_column) { column.insertFrom(sample_column, 0); } @@ -40,13 +38,9 @@ private: void operator()(sqlite3_stmt * stmt) { sqlite3_finalize(stmt); } }; - void readPrefix() override; + Chunk generate() override; - Block readImpl() override; - - void readSuffix() override; - - void insertValue(IColumn & column, const ExternalResultDescription::ValueType type, size_t idx); + void insertValue(IColumn & column, ExternalResultDescription::ValueType type, size_t idx); String query_str; UInt64 max_block_size; diff --git a/src/DataStreams/ya.make b/src/DataStreams/ya.make index b1205828a7e..c16db808a5b 100644 --- a/src/DataStreams/ya.make +++ b/src/DataStreams/ya.make @@ -29,7 +29,7 @@ SRCS( ITTLAlgorithm.cpp InternalTextLogsRowOutputStream.cpp MaterializingBlockInputStream.cpp - MongoDBBlockInputStream.cpp + MongoDBSource.cpp NativeBlockInputStream.cpp NativeBlockOutputStream.cpp PushingToViewsBlockOutputStream.cpp @@ -37,7 +37,7 @@ SRCS( RemoteBlockOutputStream.cpp RemoteQueryExecutor.cpp RemoteQueryExecutorReadContext.cpp - SQLiteBlockInputStream.cpp + SQLiteSource.cpp SizeLimits.cpp SquashingBlockInputStream.cpp SquashingBlockOutputStream.cpp diff --git a/src/Databases/MySQL/DatabaseMySQL.cpp b/src/Databases/MySQL/DatabaseMySQL.cpp index d4acd2af85e..858255e730a 100644 --- a/src/Databases/MySQL/DatabaseMySQL.cpp +++ b/src/Databases/MySQL/DatabaseMySQL.cpp @@ -11,7 +11,7 @@ # include # include # include -# include +# include # include # include # include diff --git a/src/Databases/MySQL/FetchTablesColumnsList.cpp b/src/Databases/MySQL/FetchTablesColumnsList.cpp index 353bcd877ee..c67dcefb433 100644 --- a/src/Databases/MySQL/FetchTablesColumnsList.cpp +++ b/src/Databases/MySQL/FetchTablesColumnsList.cpp @@ -10,7 +10,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Databases/MySQL/MaterializeMetadata.cpp b/src/Databases/MySQL/MaterializeMetadata.cpp index 9f5100991aa..f684797c675 100644 --- a/src/Databases/MySQL/MaterializeMetadata.cpp +++ b/src/Databases/MySQL/MaterializeMetadata.cpp @@ -5,7 +5,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp b/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp index 5175e9d0467..53495aa3cb1 100644 --- a/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp +++ b/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp @@ -16,7 +16,7 @@ # include # include # include -# include +# include # include # include # include diff --git a/src/Dictionaries/CassandraDictionarySource.cpp b/src/Dictionaries/CassandraDictionarySource.cpp index 8b31b4d6fa2..0dea38ecf73 100644 --- a/src/Dictionaries/CassandraDictionarySource.cpp +++ b/src/Dictionaries/CassandraDictionarySource.cpp @@ -36,10 +36,10 @@ void registerDictionarySourceCassandra(DictionarySourceFactory & factory) #if USE_CASSANDRA -#include -#include -#include "CassandraBlockInputStream.h" #include +#include +#include +#include namespace DB { diff --git a/src/Dictionaries/CassandraBlockInputStream.cpp b/src/Dictionaries/CassandraSource.cpp similarity index 99% rename from src/Dictionaries/CassandraBlockInputStream.cpp rename to src/Dictionaries/CassandraSource.cpp index 384717e2ba2..1ebacdb2c2f 100644 --- a/src/Dictionaries/CassandraBlockInputStream.cpp +++ b/src/Dictionaries/CassandraSource.cpp @@ -10,7 +10,7 @@ #include #include #include -#include "CassandraBlockInputStream.h" +#include "CassandraSource.h" namespace DB diff --git a/src/Dictionaries/CassandraBlockInputStream.h b/src/Dictionaries/CassandraSource.h similarity index 100% rename from src/Dictionaries/CassandraBlockInputStream.h rename to src/Dictionaries/CassandraSource.h diff --git a/src/Dictionaries/MongoDBDictionarySource.cpp b/src/Dictionaries/MongoDBDictionarySource.cpp index a3c5119ade1..23ea9bc00e2 100644 --- a/src/Dictionaries/MongoDBDictionarySource.cpp +++ b/src/Dictionaries/MongoDBDictionarySource.cpp @@ -50,7 +50,7 @@ void registerDictionarySourceMongoDB(DictionarySourceFactory & factory) // Poco/MongoDB/BSONWriter.h:54: void writeCString(const std::string & value); // src/IO/WriteHelpers.h:146 #define writeCString(s, buf) #include -#include +#include namespace DB diff --git a/src/Dictionaries/MySQLDictionarySource.h b/src/Dictionaries/MySQLDictionarySource.h index 49ddc924a86..1c8c1ed4e33 100644 --- a/src/Dictionaries/MySQLDictionarySource.h +++ b/src/Dictionaries/MySQLDictionarySource.h @@ -12,7 +12,7 @@ # include "DictionaryStructure.h" # include "ExternalQueryBuilder.h" # include "IDictionarySource.h" -# include +# include namespace Poco { diff --git a/src/Dictionaries/PostgreSQLDictionarySource.cpp b/src/Dictionaries/PostgreSQLDictionarySource.cpp index f226b7a9165..2a0173c1615 100644 --- a/src/Dictionaries/PostgreSQLDictionarySource.cpp +++ b/src/Dictionaries/PostgreSQLDictionarySource.cpp @@ -7,7 +7,7 @@ #if USE_LIBPQXX #include #include -#include +#include #include "readInvalidateQuery.h" #include #endif diff --git a/src/Dictionaries/ya.make b/src/Dictionaries/ya.make index 36152fe439a..2cff8447574 100644 --- a/src/Dictionaries/ya.make +++ b/src/Dictionaries/ya.make @@ -22,9 +22,9 @@ NO_COMPILER_WARNINGS() SRCS( CacheDictionary.cpp CacheDictionaryUpdateQueue.cpp - CassandraBlockInputStream.cpp CassandraDictionarySource.cpp CassandraHelpers.cpp + CassandraSource.cpp ClickHouseDictionarySource.cpp DictionaryBlockInputStream.cpp DictionaryBlockInputStreamBase.cpp diff --git a/src/Formats/MySQLBlockInputStream.cpp b/src/Formats/MySQLSource.cpp similarity index 99% rename from src/Formats/MySQLBlockInputStream.cpp rename to src/Formats/MySQLSource.cpp index 401d85f3d6b..2d305a29df6 100644 --- a/src/Formats/MySQLBlockInputStream.cpp +++ b/src/Formats/MySQLSource.cpp @@ -19,7 +19,7 @@ #include #include #include -#include "MySQLBlockInputStream.h" +#include "MySQLSource.h" namespace DB diff --git a/src/Formats/MySQLBlockInputStream.h b/src/Formats/MySQLSource.h similarity index 96% rename from src/Formats/MySQLBlockInputStream.h rename to src/Formats/MySQLSource.h index 9c33b4404ae..5938cb4b57f 100644 --- a/src/Formats/MySQLBlockInputStream.h +++ b/src/Formats/MySQLSource.h @@ -58,7 +58,7 @@ protected: ExternalResultDescription description; }; -/// Like MySQLBlockInputStream, but allocates connection only when reading is starting. +/// Like MySQLSource, but allocates connection only when reading is starting. /// It allows to create a lot of stream objects without occupation of all connection pool. /// Also makes attempts to reconnect in case of connection failures. class MySQLWithFailoverSource final : public MySQLSource diff --git a/src/Formats/ya.make b/src/Formats/ya.make index 476e13f9a4f..90184350359 100644 --- a/src/Formats/ya.make +++ b/src/Formats/ya.make @@ -14,7 +14,7 @@ SRCS( FormatFactory.cpp FormatSchemaInfo.cpp JSONEachRowUtils.cpp - MySQLBlockInputStream.cpp + MySQLSource.cpp NativeFormat.cpp NullFormat.cpp ParsedTemplateFormatString.cpp diff --git a/src/Functions/ya.make b/src/Functions/ya.make index 2b9b3d94313..71822924ee3 100644 --- a/src/Functions/ya.make +++ b/src/Functions/ya.make @@ -312,6 +312,7 @@ SRCS( hasToken.cpp hasTokenCaseInsensitive.cpp hostName.cpp + hyperscanRegexpChecker.cpp hypot.cpp identity.cpp if.cpp diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp index b812c6d2923..3477397adb7 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp @@ -1,6 +1,6 @@ #include "PostgreSQLReplicationHandler.h" -#include +#include #include #include #include diff --git a/src/Storages/StorageMongoDB.cpp b/src/Storages/StorageMongoDB.cpp index a973efd7277..3bdef7fd295 100644 --- a/src/Storages/StorageMongoDB.cpp +++ b/src/Storages/StorageMongoDB.cpp @@ -15,7 +15,7 @@ #include #include #include -#include +#include namespace DB { diff --git a/src/Storages/StorageMySQL.cpp b/src/Storages/StorageMySQL.cpp index 431fda530f4..79bb1f59cc7 100644 --- a/src/Storages/StorageMySQL.cpp +++ b/src/Storages/StorageMySQL.cpp @@ -4,7 +4,7 @@ #include #include -#include +#include #include #include #include diff --git a/src/Storages/StoragePostgreSQL.cpp b/src/Storages/StoragePostgreSQL.cpp index b71f2415fd8..5f8b81a47cf 100644 --- a/src/Storages/StoragePostgreSQL.cpp +++ b/src/Storages/StoragePostgreSQL.cpp @@ -1,7 +1,7 @@ #include "StoragePostgreSQL.h" #if USE_LIBPQXX -#include +#include #include #include diff --git a/src/Storages/StorageSQLite.cpp b/src/Storages/StorageSQLite.cpp index ba66083fea5..758284e8d50 100644 --- a/src/Storages/StorageSQLite.cpp +++ b/src/Storages/StorageSQLite.cpp @@ -2,7 +2,7 @@ #if USE_SQLITE #include -#include +#include #include #include #include @@ -78,8 +78,7 @@ Pipe StorageSQLite::read( sample_block.insert({column_data.type, column_data.name}); } - return Pipe(std::make_shared( - std::make_shared(sqlite_db, query, sample_block, max_block_size))); + return Pipe(std::make_shared(sqlite_db, query, sample_block, max_block_size)); } diff --git a/src/TableFunctions/TableFunctionMySQL.cpp b/src/TableFunctions/TableFunctionMySQL.cpp index f8e0c41634b..09f9cf8b1f5 100644 --- a/src/TableFunctions/TableFunctionMySQL.cpp +++ b/src/TableFunctions/TableFunctionMySQL.cpp @@ -8,7 +8,7 @@ #include #include #include -#include +#include #include #include #include From 7591c4b6ca09571a8b73f70a7890eb20fa2122ec Mon Sep 17 00:00:00 2001 From: Braulio Valdivielso Date: Thu, 12 Aug 2021 00:09:06 +0100 Subject: [PATCH 479/599] fixup! rename tests --- src/Functions/tupleToNameValuePairs.cpp | 40 ++++++++++++------------- 1 file changed, 20 insertions(+), 20 deletions(-) diff --git a/src/Functions/tupleToNameValuePairs.cpp b/src/Functions/tupleToNameValuePairs.cpp index 8478d742b3f..da09196b12b 100644 --- a/src/Functions/tupleToNameValuePairs.cpp +++ b/src/Functions/tupleToNameValuePairs.cpp @@ -61,36 +61,36 @@ public: "First argument for function {} must be a tuple.", getName()); - const auto & elementTypes = tuple->getElements(); + const auto & element_types = tuple->getElements(); - if (elementTypes.empty()) + if (element_types.empty()) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "The argument tuple for function {} must not be empty.", getName()); - const auto & firstElementType = elementTypes[0]; + const auto & first_element_type = element_types[0]; - bool allValueTypesEqual = std::all_of(elementTypes.begin() + 1, - elementTypes.end(), - [&](const auto &other) - { - return firstElementType->equals(*other); - }); + bool all_value_types_equal = std::all_of(element_types.begin() + 1, + element_types.end(), + [&](const auto &other) + { + return first_element_type->equals(*other); + }); - if (!allValueTypesEqual) + if (!all_value_types_equal) { throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "The argument tuple for function {} must contain just one type.", getName()); } - DataTypePtr tupleNameType = std::make_shared(); - DataTypes itemDataTypes ={tupleNameType, - firstElementType}; + DataTypePtr tuple_name_type = std::make_shared(); + DataTypes item_data_types = {tuple_name_type, + first_element_type}; - auto itemDataType = std::make_shared(itemDataTypes); + auto item_data_type = std::make_shared(item_data_types); - return std::make_shared(itemDataType); + return std::make_shared(item_data_type); } ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/) const override @@ -107,17 +107,17 @@ public: for (size_t col = 0; col < tuple_col_concrete->tupleSize(); ++col) { const std::string & key = tuple->getElementNames()[col]; - const IColumn & valueColumn = tuple_col_concrete->getColumn(col); + const IColumn & value_column = tuple_col_concrete->getColumn(col); - values->insertFrom(valueColumn, row); + values->insertFrom(value_column, row); keys->insertData(key.data(), key.size()); } offsets->insertValue(tuple_col_concrete->tupleSize() * (row + 1)); } - std::vector tupleColumns = { std::move(keys), std::move(values) }; - auto tupleColumn = ColumnTuple::create(std::move(tupleColumns)); - return ColumnArray::create(std::move(tupleColumn), std::move(offsets)); + std::vector tuple_columns = { std::move(keys), std::move(values) }; + auto tuple_column = ColumnTuple::create(std::move(tuple_columns)); + return ColumnArray::create(std::move(tuple_column), std::move(offsets)); } }; From c34be7179ba86423c38626ddcc2575462cb0004b Mon Sep 17 00:00:00 2001 From: Robert Hodges Date: Wed, 11 Aug 2021 21:02:17 -0700 Subject: [PATCH 480/599] Update README.md Added entry for 25 August SF Bay Area ClickHouse community meetup. --- README.md | 3 +++ 1 file changed, 3 insertions(+) diff --git a/README.md b/README.md index 496a6357f44..178547ea523 100644 --- a/README.md +++ b/README.md @@ -13,3 +13,6 @@ ClickHouse® is an open-source column-oriented database management system that a * [Code Browser](https://clickhouse.tech/codebrowser/html_report/ClickHouse/index.html) with syntax highlight and navigation. * [Contacts](https://clickhouse.tech/#contacts) can help to get your questions answered if there are any. * You can also [fill this form](https://clickhouse.tech/#meet) to meet Yandex ClickHouse team in person. + +## Upcoming Events +* [SF Bay Area ClickHouse August Community Meetup (online)](https://www.meetup.com/San-Francisco-Bay-Area-ClickHouse-Meetup/events/279109379/) on 25 August 2021. From 625aa67dad981735f5123a3fe05cc831d7b37926 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 12 Aug 2021 06:34:57 +0000 Subject: [PATCH 481/599] Fix checks --- src/Functions/CastOverloadResolver.h | 6 +++++- .../test_alter_update_cast_keep_nullable/configs/users.xml | 4 +++- 2 files changed, 8 insertions(+), 2 deletions(-) diff --git a/src/Functions/CastOverloadResolver.h b/src/Functions/CastOverloadResolver.h index c5f6489fc3c..ffd5dda4af3 100644 --- a/src/Functions/CastOverloadResolver.h +++ b/src/Functions/CastOverloadResolver.h @@ -1,10 +1,14 @@ #pragma once #include - namespace DB { +namespace ErrorCodes +{ + extern const int ILLEGAL_TYPE_OF_ARGUMENT; +} + /* * CastInternal does not preserve nullability of the data type, * i.e. CastInternal(toNullable(toInt8(1)) as Int32) will be Int32(1). diff --git a/tests/integration/test_alter_update_cast_keep_nullable/configs/users.xml b/tests/integration/test_alter_update_cast_keep_nullable/configs/users.xml index 16b032daacb..aa2f240b831 100644 --- a/tests/integration/test_alter_update_cast_keep_nullable/configs/users.xml +++ b/tests/integration/test_alter_update_cast_keep_nullable/configs/users.xml @@ -1,6 +1,8 @@ - 1 + + 1 + From f034cbedf1ceba49c6115bd1ed84ddd211ed6c68 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Thu, 12 Aug 2021 09:44:15 +0300 Subject: [PATCH 482/599] Update docs/ru/sql-reference/functions/string-functions.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/sql-reference/functions/string-functions.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/ru/sql-reference/functions/string-functions.md b/docs/ru/sql-reference/functions/string-functions.md index 39498fcea9f..16decddb839 100644 --- a/docs/ru/sql-reference/functions/string-functions.md +++ b/docs/ru/sql-reference/functions/string-functions.md @@ -76,9 +76,9 @@ SELECT empty(''); Результат: ```text -┌─empty('')─┐ -│ 1 │ -└───────────┘ +┌─notEmpty('text')─┐ +│ 1 │ +└──────────────────┘ ``` ## length {#length} From f53ef136b1ca8f70b0dc1ad454b9ed68058be640 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Thu, 12 Aug 2021 09:44:31 +0300 Subject: [PATCH 483/599] Update docs/ru/sql-reference/functions/string-functions.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/sql-reference/functions/string-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/string-functions.md b/docs/ru/sql-reference/functions/string-functions.md index 16decddb839..5b166c5bca6 100644 --- a/docs/ru/sql-reference/functions/string-functions.md +++ b/docs/ru/sql-reference/functions/string-functions.md @@ -70,7 +70,7 @@ empty(x) Запрос: ```sql -SELECT empty(''); +SELECT notEmpty('text'); ``` Результат: From 54664ed013f4eb76295c18963940dd9dfda94449 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Thu, 12 Aug 2021 09:44:54 +0300 Subject: [PATCH 484/599] Update docs/en/sql-reference/functions/array-functions.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/en/sql-reference/functions/array-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/array-functions.md b/docs/en/sql-reference/functions/array-functions.md index 38ee2090866..4968d093a19 100644 --- a/docs/en/sql-reference/functions/array-functions.md +++ b/docs/en/sql-reference/functions/array-functions.md @@ -18,7 +18,7 @@ empty([x]) An array is considered empty if it contains all empty elements. !!! note "Note" - Can be optimized by enabling the [optimize_functions_to_subcolumns](../../operations/settings/settings.md#optimize-functions-to-subcolumns) setting. With `optimize_functions_to_subcolumns = 1` the function reads only [size0](../../sql-reference/data-types/array.md#array-size) subcolumn instead of reading and processing the whole array column. The query `SELECT empty(arr) FROM table;` transforms to `SELECT arr.size0 = 0 FROM TABLE;`. + Can be optimized by enabling the [optimize_functions_to_subcolumns](../../operations/settings/settings.md#optimize-functions-to-subcolumns) setting. With `optimize_functions_to_subcolumns = 1` the function reads only [size0](../../sql-reference/data-types/array.md#array-size) subcolumn instead of reading and processing the whole array. The query `SELECT empty(arr) FROM table;` transforms to `SELECT arr.size0 = 0 FROM table;`. The function also works for [strings](string-functions.md#empty) or [UUID](uuid-functions.md#empty). From 3bc91471e43fa6aca7edb36e16061af57a104496 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Thu, 12 Aug 2021 09:45:08 +0300 Subject: [PATCH 485/599] Update docs/en/sql-reference/functions/string-functions.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/en/sql-reference/functions/string-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index 7f3ecb80ee9..57e6a3647ca 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -18,7 +18,7 @@ Checks whether the input string is empty. empty(x) ``` -A string is considered non-empty if it contains at least one byte, even if this is a space or a null byte. The UUID is empty if it contains all zeros (zero UUID). +A string is considered non-empty if it contains at least one byte, even if this is a space or a null byte. The function also works for [arrays](array-functions.md#function-empty) or [UUID](uuid-functions.md#empty). From bc85234efaef9f628adaec7f39a8f402841dc154 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Thu, 12 Aug 2021 09:45:23 +0300 Subject: [PATCH 486/599] Update docs/en/sql-reference/functions/string-functions.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/en/sql-reference/functions/string-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index 57e6a3647ca..778533a2d51 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -58,7 +58,7 @@ Checks whether the input string is non-empty. notEmpty(x) ``` -A string is considered non-empty if it contains at least one byte, even if this is a space or a null byte. The UUID is empty if it contains all zeros (zero UUID). +A string is considered non-empty if it contains at least one byte, even if this is a space or a null byte. The function also works for [arrays](array-functions.md#function-notempty) or [UUID](uuid-functions.md#notempty). From ef6da4183b8e4860d956db0da156359dadb5c526 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Thu, 12 Aug 2021 09:45:31 +0300 Subject: [PATCH 487/599] Update docs/ru/engines/database-engines/materialized-mysql.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/engines/database-engines/materialized-mysql.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/engines/database-engines/materialized-mysql.md b/docs/ru/engines/database-engines/materialized-mysql.md index 40576ec450b..ff15af05397 100644 --- a/docs/ru/engines/database-engines/materialized-mysql.md +++ b/docs/ru/engines/database-engines/materialized-mysql.md @@ -5,7 +5,7 @@ toc_title: "[experimental] MaterializedMySQL" # [экспериментальный] MaterializedMySQL {#materialized-mysql} -**Это экспериментальная функция, которую не следует использовать в продуктивной среде.** +**Это экспериментальный движок, который не следует использовать в продуктивной среде.** Создает базу данных ClickHouse со всеми таблицами, существующими в MySQL, и всеми данными в этих таблицах. From ecc1a882fcc6b74c96fd570839c1d9106982979a Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Thu, 12 Aug 2021 09:46:03 +0300 Subject: [PATCH 488/599] Update docs/ru/engines/database-engines/materialized-mysql.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/engines/database-engines/materialized-mysql.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/engines/database-engines/materialized-mysql.md b/docs/ru/engines/database-engines/materialized-mysql.md index ff15af05397..63cf5039937 100644 --- a/docs/ru/engines/database-engines/materialized-mysql.md +++ b/docs/ru/engines/database-engines/materialized-mysql.md @@ -33,7 +33,7 @@ ENGINE = MaterializedMySQL('host:port', ['database' | database], 'user', 'passwo - `max_bytes_in_buffers` — максимальное количество байтов, которое разрешено кешировать данным в памяти (для базы данных и данных кеша, которые невозможно запросить). При превышении количества строк, данные будут материализованы. Значение по умолчанию: `1 048 576`. - `max_flush_data_time` — максимальное время в миллисекундах, в течение которого разрешено кешировать данные в памяти (для базы данных и данных кеша, которые невозможно запросить). При превышении количества указанного периода, данные будут материализованы. Значение по умолчанию: `1000`. - `max_wait_time_when_mysql_unavailable` — интервал между повторными попытками, если MySQL недоступен. Указывается в миллисекундах. Отрицательное значение отключает повторные попытки. Значение по умолчанию: `1000`. -- `allows_query_when_mysql_lost` — разрешить запрос материализованной таблицы при потере MySQL. Значение по умолчанию: `0` (`false`). +- `allows_query_when_mysql_lost` — признак, разрешен ли запрос к материализованной таблице при потере соединения с MySQL. Значение по умолчанию: `0` (`false`). ```sql CREATE DATABASE mysql ENGINE = MaterializedMySQL('localhost:3306', 'db', 'user', '***') From 77f343267aad45bd874df37c6b9cfdc9997ff4f4 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Thu, 12 Aug 2021 09:46:16 +0300 Subject: [PATCH 489/599] Update docs/en/engines/database-engines/materialized-mysql.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/en/engines/database-engines/materialized-mysql.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/engines/database-engines/materialized-mysql.md b/docs/en/engines/database-engines/materialized-mysql.md index 5fab87d4d8c..0937fe20dd4 100644 --- a/docs/en/engines/database-engines/materialized-mysql.md +++ b/docs/en/engines/database-engines/materialized-mysql.md @@ -45,7 +45,7 @@ CREATE DATABASE mysql ENGINE = MaterializedMySQL('localhost:3306', 'db', 'user', **Settings on MySQL-server Side** -For the correct work of `MaterializeMySQL`, there are few mandatory `MySQL`-side configuration settings that must be set: +For the correct work of `MaterializedMySQL`, there are few mandatory `MySQL`-side configuration settings that must be set: - `default_authentication_plugin = mysql_native_password` since `MaterializeMySQL` can only authorize with this method. - `gtid_mode = on` since GTID based logging is a mandatory for providing correct `MaterializeMySQL` replication. From 3b85c3c8ac9b209a5d25025d86fd6980e96e1f7f Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Thu, 12 Aug 2021 09:50:56 +0300 Subject: [PATCH 490/599] Update docs/ru/engines/database-engines/materialized-mysql.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/engines/database-engines/materialized-mysql.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/engines/database-engines/materialized-mysql.md b/docs/ru/engines/database-engines/materialized-mysql.md index 63cf5039937..d3f764b3067 100644 --- a/docs/ru/engines/database-engines/materialized-mysql.md +++ b/docs/ru/engines/database-engines/materialized-mysql.md @@ -45,7 +45,7 @@ CREATE DATABASE mysql ENGINE = MaterializedMySQL('localhost:3306', 'db', 'user', **Настройки на стороне MySQL-сервера** Для правильной работы `MaterializeMySQL` следует обязательно указать на сервере MySQL следующие параметры конфигурации: -- `default_authentication_plugin = mysql_native_password` — `MaterializeMySQL` может авторизоваться только с помощью этого метода. +- `default_authentication_plugin = mysql_native_password` — `MaterializedMySQL` может авторизоваться только с помощью этого метода. - `gtid_mode = on` — ведение журнала на основе GTID является обязательным для обеспечения правильной репликации. !!! attention "Внимание" From 9230ee317547077aed546c6e9702d8be8c4a7b4f Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Thu, 12 Aug 2021 09:51:05 +0300 Subject: [PATCH 491/599] Update docs/en/engines/database-engines/materialized-mysql.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/en/engines/database-engines/materialized-mysql.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/engines/database-engines/materialized-mysql.md b/docs/en/engines/database-engines/materialized-mysql.md index 0937fe20dd4..72688f056ac 100644 --- a/docs/en/engines/database-engines/materialized-mysql.md +++ b/docs/en/engines/database-engines/materialized-mysql.md @@ -47,7 +47,7 @@ CREATE DATABASE mysql ENGINE = MaterializedMySQL('localhost:3306', 'db', 'user', For the correct work of `MaterializedMySQL`, there are few mandatory `MySQL`-side configuration settings that must be set: -- `default_authentication_plugin = mysql_native_password` since `MaterializeMySQL` can only authorize with this method. +- `default_authentication_plugin = mysql_native_password` since `MaterializedMySQL` can only authorize with this method. - `gtid_mode = on` since GTID based logging is a mandatory for providing correct `MaterializeMySQL` replication. !!! attention "Attention" From 42da5f488dfe46a951b583fb1ad124f8eb83941f Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Thu, 12 Aug 2021 09:51:22 +0300 Subject: [PATCH 492/599] Update docs/en/engines/database-engines/materialized-mysql.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/en/engines/database-engines/materialized-mysql.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/engines/database-engines/materialized-mysql.md b/docs/en/engines/database-engines/materialized-mysql.md index 72688f056ac..d329dff32c5 100644 --- a/docs/en/engines/database-engines/materialized-mysql.md +++ b/docs/en/engines/database-engines/materialized-mysql.md @@ -48,7 +48,7 @@ CREATE DATABASE mysql ENGINE = MaterializedMySQL('localhost:3306', 'db', 'user', For the correct work of `MaterializedMySQL`, there are few mandatory `MySQL`-side configuration settings that must be set: - `default_authentication_plugin = mysql_native_password` since `MaterializedMySQL` can only authorize with this method. -- `gtid_mode = on` since GTID based logging is a mandatory for providing correct `MaterializeMySQL` replication. +- `gtid_mode = on` since GTID based logging is a mandatory for providing correct `MaterializedMySQL` replication. !!! attention "Attention" While turning on `gtid_mode` you should also specify `enforce_gtid_consistency = on`. From 9bd6db00f99bddd96f09cc7a19a1b9efeed4904d Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Thu, 12 Aug 2021 09:51:31 +0300 Subject: [PATCH 493/599] Update docs/ru/sql-reference/functions/string-functions.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/sql-reference/functions/string-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/string-functions.md b/docs/ru/sql-reference/functions/string-functions.md index 5b166c5bca6..48391fe697e 100644 --- a/docs/ru/sql-reference/functions/string-functions.md +++ b/docs/ru/sql-reference/functions/string-functions.md @@ -53,7 +53,7 @@ SELECT notempty('text'); empty(x) ``` -Строка считается непустой, если содержит хотя бы один байт, пусть даже это пробел или нулевой байт. UUID считается пустой, если он содержит только нули (нулевой UUID). +Строка считается непустой, если содержит хотя бы один байт, пусть даже это пробел или нулевой байт. **Параметры** From 902fb83492069c1ad1edf471868306645d388bf4 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Thu, 12 Aug 2021 09:51:39 +0300 Subject: [PATCH 494/599] Update docs/ru/sql-reference/functions/string-functions.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/sql-reference/functions/string-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/string-functions.md b/docs/ru/sql-reference/functions/string-functions.md index 48391fe697e..06edab23325 100644 --- a/docs/ru/sql-reference/functions/string-functions.md +++ b/docs/ru/sql-reference/functions/string-functions.md @@ -57,7 +57,7 @@ empty(x) **Параметры** -- `x` — Входная строка. [Array](../data-types/array.md), [String](../data-types/string.md), [UUID](../data-types/uuid.md). +- `x` — Входная строка. [String](../data-types/string.md). **Возвращаемое значение** From 0798a3343e196c3140c683569dcb6601963c4fb4 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Thu, 12 Aug 2021 09:51:47 +0300 Subject: [PATCH 495/599] Update docs/ru/sql-reference/functions/string-functions.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/sql-reference/functions/string-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/string-functions.md b/docs/ru/sql-reference/functions/string-functions.md index 06edab23325..6f6b6289e28 100644 --- a/docs/ru/sql-reference/functions/string-functions.md +++ b/docs/ru/sql-reference/functions/string-functions.md @@ -50,7 +50,7 @@ SELECT notempty('text'); **Синтаксис** ``` sql -empty(x) +notEmpty(x) ``` Строка считается непустой, если содержит хотя бы один байт, пусть даже это пробел или нулевой байт. From 79f69034270c6b7b72577a524f5ce3f521d6fd80 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Thu, 12 Aug 2021 09:51:54 +0300 Subject: [PATCH 496/599] Update docs/ru/sql-reference/functions/string-functions.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/sql-reference/functions/string-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/string-functions.md b/docs/ru/sql-reference/functions/string-functions.md index 6f6b6289e28..fcf36308703 100644 --- a/docs/ru/sql-reference/functions/string-functions.md +++ b/docs/ru/sql-reference/functions/string-functions.md @@ -19,7 +19,7 @@ empty(x) **Параметры** -- `x` — Входная строка. [Array](../data-types/array.md), [String](../data-types/string.md), [UUID](../data-types/uuid.md). +- `x` — Входная строка. [String](../data-types/string.md). **Возвращаемое значение** From 7a8cc49e93806c9bb6ed052afb988b9711c66482 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Thu, 12 Aug 2021 09:52:06 +0300 Subject: [PATCH 497/599] Update docs/ru/sql-reference/functions/string-functions.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/sql-reference/functions/string-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/string-functions.md b/docs/ru/sql-reference/functions/string-functions.md index fcf36308703..75a4af84297 100644 --- a/docs/ru/sql-reference/functions/string-functions.md +++ b/docs/ru/sql-reference/functions/string-functions.md @@ -15,7 +15,7 @@ toc_title: "Функции для работы со строками" empty(x) ``` -Строка считается непустой, если содержит хотя бы один байт, пусть даже это пробел или нулевой байт. UUID считается пустым, если он содержит только нули (нулевой UUID). +Строка считается непустой, если содержит хотя бы один байт, пусть даже это пробел или нулевой байт. **Параметры** From 31e7e5cb3af63747c06e9dcf736edb6d673a1e6e Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Thu, 12 Aug 2021 09:52:16 +0300 Subject: [PATCH 498/599] Update docs/ru/engines/database-engines/materialized-mysql.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/engines/database-engines/materialized-mysql.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/engines/database-engines/materialized-mysql.md b/docs/ru/engines/database-engines/materialized-mysql.md index d3f764b3067..b82a3dae553 100644 --- a/docs/ru/engines/database-engines/materialized-mysql.md +++ b/docs/ru/engines/database-engines/materialized-mysql.md @@ -44,7 +44,7 @@ CREATE DATABASE mysql ENGINE = MaterializedMySQL('localhost:3306', 'db', 'user', **Настройки на стороне MySQL-сервера** -Для правильной работы `MaterializeMySQL` следует обязательно указать на сервере MySQL следующие параметры конфигурации: +Для правильной работы `MaterializedMySQL` следует обязательно указать на сервере MySQL следующие параметры конфигурации: - `default_authentication_plugin = mysql_native_password` — `MaterializedMySQL` может авторизоваться только с помощью этого метода. - `gtid_mode = on` — ведение журнала на основе GTID является обязательным для обеспечения правильной репликации. From bdc4d9fe75599ac017660cf99a72c7f279c617a1 Mon Sep 17 00:00:00 2001 From: Vasily Nemkov Date: Wed, 11 Aug 2021 18:19:55 +0300 Subject: [PATCH 499/599] Fixed parsing DateTime64 value from string. The time offset undeflowed on positive values. --- base/common/DateLUTImpl.h | 2 +- .../0_stateless/02006_todatetime64_from_string.reference | 1 + tests/queries/0_stateless/02006_todatetime64_from_string.sql | 1 + 3 files changed, 3 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02006_todatetime64_from_string.reference create mode 100644 tests/queries/0_stateless/02006_todatetime64_from_string.sql diff --git a/base/common/DateLUTImpl.h b/base/common/DateLUTImpl.h index 202eb88a361..f2530c13f41 100644 --- a/base/common/DateLUTImpl.h +++ b/base/common/DateLUTImpl.h @@ -955,7 +955,7 @@ public: inline Time makeDateTime(Int16 year, UInt8 month, UInt8 day_of_month, UInt8 hour, UInt8 minute, UInt8 second) const { size_t index = makeLUTIndex(year, month, day_of_month); - UInt32 time_offset = hour * 3600 + minute * 60 + second; + Time time_offset = hour * 3600 + minute * 60 + second; if (time_offset >= lut[index].time_at_offset_change()) time_offset -= lut[index].amount_of_offset_change(); diff --git a/tests/queries/0_stateless/02006_todatetime64_from_string.reference b/tests/queries/0_stateless/02006_todatetime64_from_string.reference new file mode 100644 index 00000000000..e95351fe815 --- /dev/null +++ b/tests/queries/0_stateless/02006_todatetime64_from_string.reference @@ -0,0 +1 @@ +2021-03-21 23:00:00.000 diff --git a/tests/queries/0_stateless/02006_todatetime64_from_string.sql b/tests/queries/0_stateless/02006_todatetime64_from_string.sql new file mode 100644 index 00000000000..fb4ab93f70d --- /dev/null +++ b/tests/queries/0_stateless/02006_todatetime64_from_string.sql @@ -0,0 +1 @@ +SELECT toDateTime64('2021-03-22', 3, 'Asia/Tehran'); \ No newline at end of file From 8d66d0786130d12b3356756a61692d2df7bf70cc Mon Sep 17 00:00:00 2001 From: benbiti Date: Thu, 12 Aug 2021 16:02:25 +0800 Subject: [PATCH 500/599] fix ProfileEvents::CompileFunction --- src/Interpreters/JIT/compileFunction.cpp | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Interpreters/JIT/compileFunction.cpp b/src/Interpreters/JIT/compileFunction.cpp index 5e1d4ca0375..aaf722b505e 100644 --- a/src/Interpreters/JIT/compileFunction.cpp +++ b/src/Interpreters/JIT/compileFunction.cpp @@ -143,8 +143,6 @@ static void compileFunction(llvm::Module & module, const IFunctionBase & functio * } */ - ProfileEvents::increment(ProfileEvents::CompileFunction); - const auto & arg_types = function.getArgumentTypes(); llvm::IRBuilder<> b(module.getContext()); From 014d3c889f4985c340e396dd8b6c383b73c270b8 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 12 Aug 2021 12:02:52 +0300 Subject: [PATCH 501/599] Fix ya.make --- src/Dictionaries/ya.make | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Dictionaries/ya.make b/src/Dictionaries/ya.make index 36152fe439a..ad4ed6e1af7 100644 --- a/src/Dictionaries/ya.make +++ b/src/Dictionaries/ya.make @@ -26,9 +26,9 @@ SRCS( CassandraDictionarySource.cpp CassandraHelpers.cpp ClickHouseDictionarySource.cpp - DictionaryBlockInputStream.cpp - DictionaryBlockInputStreamBase.cpp DictionaryFactory.cpp + DictionarySource.cpp + DictionarySourceBase.cpp DictionarySourceFactory.cpp DictionarySourceHelpers.cpp DictionaryStructure.cpp From 2c01e2789a71c43afe817aefc750cf24a939df5d Mon Sep 17 00:00:00 2001 From: Mikhail Filimonov Date: Mon, 9 Aug 2021 20:52:26 +0200 Subject: [PATCH 502/599] make vdso work again --- base/glibc-compatibility/musl/getauxval.c | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/base/glibc-compatibility/musl/getauxval.c b/base/glibc-compatibility/musl/getauxval.c index a429273fa1a..61a47449db3 100644 --- a/base/glibc-compatibility/musl/getauxval.c +++ b/base/glibc-compatibility/musl/getauxval.c @@ -17,7 +17,11 @@ static size_t __find_auxv(unsigned long type) return (size_t) -1; } -__attribute__((constructor)) static void __auxv_init() +/// __auxv_init should happen BEFORE the first use of getauxval. +/// but getauxval can be used in other init sections (namely in musl/clock_gettime.c::cgt_init), +/// so constructor(0) is needed to prioritize that constructor +/// see also: https://stackoverflow.com/questions/11106875/attribute-constructor-call-order-confusion/11198936 +__attribute__((constructor(0))) static void __auxv_init() { size_t i; for (i = 0; __environ[i]; i++); From 242d8e1861eeb4410c46a2ee525fd17a4038d844 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Thu, 12 Aug 2021 11:05:27 +0200 Subject: [PATCH 503/599] __auxv_init lazy initialization --- base/glibc-compatibility/musl/getauxval.c | 44 ++++++++++++++--------- 1 file changed, 28 insertions(+), 16 deletions(-) diff --git a/base/glibc-compatibility/musl/getauxval.c b/base/glibc-compatibility/musl/getauxval.c index 61a47449db3..db78b89964a 100644 --- a/base/glibc-compatibility/musl/getauxval.c +++ b/base/glibc-compatibility/musl/getauxval.c @@ -1,4 +1,5 @@ #include +#include "atomic.h" #include // __environ #include @@ -17,22 +18,7 @@ static size_t __find_auxv(unsigned long type) return (size_t) -1; } -/// __auxv_init should happen BEFORE the first use of getauxval. -/// but getauxval can be used in other init sections (namely in musl/clock_gettime.c::cgt_init), -/// so constructor(0) is needed to prioritize that constructor -/// see also: https://stackoverflow.com/questions/11106875/attribute-constructor-call-order-confusion/11198936 -__attribute__((constructor(0))) static void __auxv_init() -{ - size_t i; - for (i = 0; __environ[i]; i++); - __auxv = (unsigned long *) (__environ + i + 1); - - size_t secure_idx = __find_auxv(AT_SECURE); - if (secure_idx != ((size_t) -1)) - __auxv_secure = __auxv[secure_idx]; -} - -unsigned long getauxval(unsigned long type) +unsigned long __getauxval(unsigned long type) { if (type == AT_SECURE) return __auxv_secure; @@ -47,3 +33,29 @@ unsigned long getauxval(unsigned long type) errno = ENOENT; return 0; } + +static void * volatile getauxval_func; + +static unsigned long __auxv_init(unsigned long type) +{ + if (!__environ) + return 0; + + size_t i; + for (i = 0; __environ[i]; i++); + __auxv = (unsigned long *) (__environ + i + 1); + + size_t secure_idx = __find_auxv(AT_SECURE); + if (secure_idx != ((size_t) -1)) + __auxv_secure = __auxv[secure_idx]; + + a_cas_p(&getauxval_func, (void *)__auxv_init, (void *)__getauxval); + return __getauxval(type); +} + +static void * volatile getauxval_func = (void *)__auxv_init; + +unsigned long getauxval(unsigned long type) +{ + return ((unsigned long (*)(unsigned long))getauxval_func)(type); +} From 9243e89150ce01c456b37a361e8878d10e3681b2 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Thu, 12 Aug 2021 12:10:37 +0300 Subject: [PATCH 504/599] Update getauxval.c --- base/glibc-compatibility/musl/getauxval.c | 3 +++ 1 file changed, 3 insertions(+) diff --git a/base/glibc-compatibility/musl/getauxval.c b/base/glibc-compatibility/musl/getauxval.c index db78b89964a..d91d787281b 100644 --- a/base/glibc-compatibility/musl/getauxval.c +++ b/base/glibc-compatibility/musl/getauxval.c @@ -49,10 +49,13 @@ static unsigned long __auxv_init(unsigned long type) if (secure_idx != ((size_t) -1)) __auxv_secure = __auxv[secure_idx]; + // Now we've initialized __auxv, next time getauxval() will only call __get_auxval(). a_cas_p(&getauxval_func, (void *)__auxv_init, (void *)__getauxval); + return __getauxval(type); } +// First time getauxval() will call __auxv_init(). static void * volatile getauxval_func = (void *)__auxv_init; unsigned long getauxval(unsigned long type) From 031c15cb0e35b6a5b2bff81442725bb7cb6280bb Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 12 Aug 2021 12:18:20 +0300 Subject: [PATCH 505/599] Add comment. --- src/Common/DenseHashMap.h | 5 +++++ src/Common/DenseHashSet.h | 2 ++ src/Common/SparseHashMap.h | 2 ++ 3 files changed, 9 insertions(+) diff --git a/src/Common/DenseHashMap.h b/src/Common/DenseHashMap.h index 26b9224528b..9ac21c82676 100644 --- a/src/Common/DenseHashMap.h +++ b/src/Common/DenseHashMap.h @@ -1,6 +1,11 @@ #pragma once #include +/// DenseHashMap is a wrapper for google::dense_hash_map. +/// Some hacks are needed to make it work in "Arcadia". +/// "Arcadia" is a proprietary monorepository in Yandex. +/// It uses slightly changed version of sparsehash with a different set of hash functions (which we don't need). +/// Those defines are needed to make it compile. #if defined(ARCADIA_BUILD) #define HASH_FUN_H template diff --git a/src/Common/DenseHashSet.h b/src/Common/DenseHashSet.h index 5b30d88a70c..e8c06f36aa3 100644 --- a/src/Common/DenseHashSet.h +++ b/src/Common/DenseHashSet.h @@ -1,5 +1,7 @@ #pragma once +/// DenseHashSet is a wrapper for google::dense_hash_set. +/// See comment in DenseHashMap.h #if defined(ARCADIA_BUILD) #define HASH_FUN_H template diff --git a/src/Common/SparseHashMap.h b/src/Common/SparseHashMap.h index 403042c11a9..f01fc633d84 100644 --- a/src/Common/SparseHashMap.h +++ b/src/Common/SparseHashMap.h @@ -1,5 +1,7 @@ #pragma once +/// SparseHashMap is a wrapper for google::sparse_hash_map. +/// See comment in DenseHashMap.h #if defined(ARCADIA_BUILD) #define HASH_FUN_H template From f6788fc6600ba6c23ede027050487ac81ef3659b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 12 Aug 2021 11:29:50 +0200 Subject: [PATCH 506/599] Mysql handler: Move format check to the handler --- src/Formats/FormatFactory.cpp | 11 +++-------- src/Server/MySQLHandler.cpp | 5 ++++- .../0_stateless/01176_mysql_client_interactive.expect | 8 ++++++++ 3 files changed, 15 insertions(+), 9 deletions(-) diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index cd55c66fbdf..9b701816a2c 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -5,7 +5,6 @@ #include #include #include -#include #include #include #include @@ -214,8 +213,8 @@ BlockOutputStreamPtr FormatFactory::getOutputStreamParallelIfPossible( bool parallel_formatting = settings.output_format_parallel_formatting; auto format_settings = _format_settings ? *_format_settings : getFormatSettings(context); - if (output_getter && parallel_formatting && getCreators(name).supports_parallel_formatting && !settings.output_format_json_array_of_rows - && !format_settings.mysql_wire.sequence_id) + if (output_getter && parallel_formatting && getCreators(name).supports_parallel_formatting + && !settings.output_format_json_array_of_rows) { auto formatter_creator = [output_getter, sample, callback, format_settings] (WriteBuffer & output) -> OutputFormatPtr @@ -315,7 +314,7 @@ OutputFormatPtr FormatFactory::getOutputFormatParallelIfPossible( const Settings & settings = context->getSettingsRef(); if (settings.output_format_parallel_formatting && getCreators(name).supports_parallel_formatting - && !settings.output_format_json_array_of_rows && !format_settings.mysql_wire.sequence_id) + && !settings.output_format_json_array_of_rows) { auto formatter_creator = [output_getter, sample, callback, format_settings] (WriteBuffer & output) -> OutputFormatPtr @@ -353,10 +352,6 @@ OutputFormatPtr FormatFactory::getOutputFormat( auto format_settings = _format_settings ? *_format_settings : getFormatSettings(context); - /// If we're handling MySQL protocol connection right now then MySQLWire is only allowed output format. - if (format_settings.mysql_wire.sequence_id && (name != "MySQLWire")) - throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "MySQL protocol does not support custom output formats"); - /** TODO: Materialization is needed, because formats can use the functions `IDataType`, * which only work with full columns. */ diff --git a/src/Server/MySQLHandler.cpp b/src/Server/MySQLHandler.cpp index 375f248d939..b05f96bbfe1 100644 --- a/src/Server/MySQLHandler.cpp +++ b/src/Server/MySQLHandler.cpp @@ -57,6 +57,7 @@ namespace ErrorCodes extern const int NOT_IMPLEMENTED; extern const int MYSQL_CLIENT_INSUFFICIENT_CAPABILITIES; extern const int SUPPORT_IS_DISABLED; + extern const int UNSUPPORTED_METHOD; } @@ -352,8 +353,10 @@ void MySQLHandler::comQuery(ReadBuffer & payload) format_settings.mysql_wire.max_packet_size = max_packet_size; format_settings.mysql_wire.sequence_id = &sequence_id; - auto set_result_details = [&with_output](const String &, const String &, const String &, const String &) + auto set_result_details = [&with_output](const String &, const String &, const String &format, const String &) { + if (format != "MySQLWire") + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "MySQL protocol does not support custom output formats"); with_output = true; }; diff --git a/tests/queries/0_stateless/01176_mysql_client_interactive.expect b/tests/queries/0_stateless/01176_mysql_client_interactive.expect index 6f0cea48e76..8f56d047e8e 100755 --- a/tests/queries/0_stateless/01176_mysql_client_interactive.expect +++ b/tests/queries/0_stateless/01176_mysql_client_interactive.expect @@ -34,6 +34,14 @@ expect "ERROR 395 (00000): Code: 395" send -- "select * from system.one format TSV;\r" expect "ERROR 1 (00000): Code: 1" +send -- "select * from system.one format JSON;\r" +expect "ERROR 1 (00000): Code: 1" + +send -- "select * from system.one format MySQLWire;\r" +expect "| dummy |" +expect "| 0 |" +expect "1 row in set" + send -- "select count(number), sum(number) from numbers(10);\r" expect "+---------------+-------------+" expect "| count(number) | sum(number) |" From a451bf6eac84886924f84cf353ab874b9282bdfa Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 12 Aug 2021 11:30:01 +0200 Subject: [PATCH 507/599] Remove unused code --- src/Formats/FormatFactory.cpp | 5 ----- src/Server/MySQLHandler.cpp | 5 ----- 2 files changed, 10 deletions(-) diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index 9b701816a2c..765f298a081 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -18,10 +18,6 @@ #include -#if !defined(ARCADIA_BUILD) -# include -#endif - namespace DB { @@ -31,7 +27,6 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; extern const int FORMAT_IS_NOT_SUITABLE_FOR_INPUT; extern const int FORMAT_IS_NOT_SUITABLE_FOR_OUTPUT; - extern const int UNSUPPORTED_METHOD; } const FormatFactory::Creators & FormatFactory::getCreators(const String & name) const diff --git a/src/Server/MySQLHandler.cpp b/src/Server/MySQLHandler.cpp index b05f96bbfe1..52182257ac9 100644 --- a/src/Server/MySQLHandler.cpp +++ b/src/Server/MySQLHandler.cpp @@ -1,8 +1,6 @@ #include "MySQLHandler.h" #include -#include -#include #include #include #include @@ -10,7 +8,6 @@ #include #include #include -#include #include #include #include @@ -20,7 +17,6 @@ #include #include #include -#include #include #include #include @@ -31,7 +27,6 @@ #endif #if USE_SSL -# include # include # include # include From a547bd6e8a949ea88e9b8a39e27007bfab965fa9 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Thu, 12 Aug 2021 13:25:31 +0300 Subject: [PATCH 508/599] Update tupleToNameValuePairs.cpp --- src/Functions/tupleToNameValuePairs.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/tupleToNameValuePairs.cpp b/src/Functions/tupleToNameValuePairs.cpp index da09196b12b..c3e5f28037b 100644 --- a/src/Functions/tupleToNameValuePairs.cpp +++ b/src/Functions/tupleToNameValuePairs.cpp @@ -97,7 +97,7 @@ public: { const IColumn * tuple_col = arguments[0].column.get(); const DataTypeTuple * tuple = checkAndGetDataType(arguments[0].type.get()); - auto * tuple_col_concrete = assert_cast(tuple_col); + const auto * tuple_col_concrete = assert_cast(tuple_col); auto keys = ColumnString::create(); MutableColumnPtr values = tuple_col_concrete->getColumn(0).cloneEmpty(); From ab3fdfa00e9220ef882e6499adea7e685fc0ebf9 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Thu, 12 Aug 2021 13:27:36 +0300 Subject: [PATCH 509/599] Update 02006_todatetime64_from_string.sql --- tests/queries/0_stateless/02006_todatetime64_from_string.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02006_todatetime64_from_string.sql b/tests/queries/0_stateless/02006_todatetime64_from_string.sql index fb4ab93f70d..305dd49f70e 100644 --- a/tests/queries/0_stateless/02006_todatetime64_from_string.sql +++ b/tests/queries/0_stateless/02006_todatetime64_from_string.sql @@ -1 +1 @@ -SELECT toDateTime64('2021-03-22', 3, 'Asia/Tehran'); \ No newline at end of file +SELECT toDateTime64('2021-03-22', 3, 'Asia/Tehran'); From 88b4200008ded99ded7dd1ef3315423cc0d819d9 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 12 Aug 2021 13:40:54 +0300 Subject: [PATCH 510/599] fix --- tests/integration/ci-runner.py | 26 ++++++++++++++++++++++---- 1 file changed, 22 insertions(+), 4 deletions(-) diff --git a/tests/integration/ci-runner.py b/tests/integration/ci-runner.py index 12ee436e400..bf7549a83c4 100755 --- a/tests/integration/ci-runner.py +++ b/tests/integration/ci-runner.py @@ -125,13 +125,13 @@ def clear_ip_tables_and_restart_daemons(): logging.info("Dump iptables after run %s", subprocess.check_output("iptables -L", shell=True)) try: logging.info("Killing all alive docker containers") - subprocess.check_output("docker kill $(docker ps -q)", shell=True) + subprocess.check_output("timeout -s 9 10m docker kill $(docker ps -q)", shell=True) except subprocess.CalledProcessError as err: logging.info("docker kill excepted: " + str(err)) try: logging.info("Removing all docker containers") - subprocess.check_output("docker rm $(docker ps -a -q) --force", shell=True) + subprocess.check_output("timeout -s 9 10m docker rm $(docker ps -a -q) --force", shell=True) except subprocess.CalledProcessError as err: logging.info("docker rm excepted: " + str(err)) @@ -376,6 +376,24 @@ class ClickhouseIntegrationTestsRunner: res.add(path) return res + def try_run_test_group(self, repo_path, test_group, tests_in_group, num_tries, num_workers): + try: + return self.run_test_group(repo_path, test_group, tests_in_group, num_tries, num_workers) + except Exception as e: + logging.info("Failed to run {}:\n{}".format(str(test_group), str(e))) + counters = { + "ERROR": [], + "PASSED": [], + "FAILED": [], + "SKIPPED": [], + "FLAKY": [], + } + tests_times = defaultdict(float) + for test in tests_in_group: + counters["ERROR"].append(test) + tests_times[test] = 0 + return counters, tests_times, [] + def run_test_group(self, repo_path, test_group, tests_in_group, num_tries, num_workers): counters = { "ERROR": [], @@ -507,7 +525,7 @@ class ClickhouseIntegrationTestsRunner: for i in range(TRIES_COUNT): final_retry += 1 logging.info("Running tests for the %s time", i) - counters, tests_times, log_paths = self.run_test_group(repo_path, "flaky", tests_to_run, 1, 1) + counters, tests_times, log_paths = self.try_run_test_group(repo_path, "flaky", tests_to_run, 1, 1) logs += log_paths if counters["FAILED"]: logging.info("Found failed tests: %s", ' '.join(counters["FAILED"])) @@ -583,7 +601,7 @@ class ClickhouseIntegrationTestsRunner: for group, tests in items_to_run: logging.info("Running test group %s countaining %s tests", group, len(tests)) - group_counters, group_test_times, log_paths = self.run_test_group(repo_path, group, tests, MAX_RETRY, NUM_WORKERS) + group_counters, group_test_times, log_paths = self.try_run_test_group(repo_path, group, tests, MAX_RETRY, NUM_WORKERS) total_tests = 0 for counter, value in group_counters.items(): logging.info("Tests from group %s stats, %s count %s", group, counter, len(value)) From 3d9a4adfbfa33326ba7125f940a9a44749468b20 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Thu, 12 Aug 2021 13:43:06 +0300 Subject: [PATCH 511/599] Better handling case when __environ is null --- base/glibc-compatibility/musl/getauxval.c | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/base/glibc-compatibility/musl/getauxval.c b/base/glibc-compatibility/musl/getauxval.c index d91d787281b..ae7bdd19c72 100644 --- a/base/glibc-compatibility/musl/getauxval.c +++ b/base/glibc-compatibility/musl/getauxval.c @@ -39,8 +39,14 @@ static void * volatile getauxval_func; static unsigned long __auxv_init(unsigned long type) { if (!__environ) + { + // __environ is not initialized yet so we can't initialize __auxv right now. + // This is normally occurred when getauxval() is called from some sanitizer's internal code. + errno = ENOENT; return 0; + } + // Initialize __auxv and __auxv_secure. size_t i; for (i = 0; __environ[i]; i++); __auxv = (unsigned long *) (__environ + i + 1); From 213e6366c2ef381ed0ebdb2e64e7530836ff5f6a Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Thu, 12 Aug 2021 13:44:09 +0300 Subject: [PATCH 512/599] Update getauxval.c --- base/glibc-compatibility/musl/getauxval.c | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/base/glibc-compatibility/musl/getauxval.c b/base/glibc-compatibility/musl/getauxval.c index ae7bdd19c72..dad7aa938d7 100644 --- a/base/glibc-compatibility/musl/getauxval.c +++ b/base/glibc-compatibility/musl/getauxval.c @@ -41,7 +41,7 @@ static unsigned long __auxv_init(unsigned long type) if (!__environ) { // __environ is not initialized yet so we can't initialize __auxv right now. - // This is normally occurred when getauxval() is called from some sanitizer's internal code. + // That's normally occurred only when getauxval() is called from some sanitizer's internal code. errno = ENOENT; return 0; } From 9c066d476e5a60a7488cb93cce8e6229f28925a7 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 12 Aug 2021 13:46:58 +0300 Subject: [PATCH 513/599] Fix tests. --- src/Interpreters/InterpreterInsertQuery.cpp | 1 - src/Processors/ya.make | 1 + ...ewEventsBlockInputStream.h => LiveViewEventsSource.h} | 0 .../{LiveViewBlockOutputStream.h => LiveViewSink.h} | 9 ++++++++- .../{LiveViewBlockInputStream.h => LiveViewSource.h} | 0 src/Storages/LiveView/StorageLiveView.cpp | 5 +++-- 6 files changed, 12 insertions(+), 4 deletions(-) rename src/Storages/LiveView/{LiveViewEventsBlockInputStream.h => LiveViewEventsSource.h} (100%) rename src/Storages/LiveView/{LiveViewBlockOutputStream.h => LiveViewSink.h} (91%) rename src/Storages/LiveView/{LiveViewBlockInputStream.h => LiveViewSource.h} (100%) diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index e5d4d952a0c..3589176f231 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -261,7 +261,6 @@ BlockIO InterpreterInsertQuery::execute() { InterpreterWatchQuery interpreter_watch{ query.watch, getContext() }; res = interpreter_watch.execute(); - res.pipeline.init(Pipe(std::make_shared(std::move(res.in)))); } for (size_t i = 0; i < out_streams_size; i++) diff --git a/src/Processors/ya.make b/src/Processors/ya.make index 543a08caca5..50faac6d97b 100644 --- a/src/Processors/ya.make +++ b/src/Processors/ya.make @@ -165,6 +165,7 @@ SRCS( Transforms/ReverseTransform.cpp Transforms/RollupTransform.cpp Transforms/SortingTransform.cpp + Transforms/SquashingChunksTransform.cpp Transforms/TotalsHavingTransform.cpp Transforms/WindowTransform.cpp Transforms/getSourceFromFromASTInsertQuery.cpp diff --git a/src/Storages/LiveView/LiveViewEventsBlockInputStream.h b/src/Storages/LiveView/LiveViewEventsSource.h similarity index 100% rename from src/Storages/LiveView/LiveViewEventsBlockInputStream.h rename to src/Storages/LiveView/LiveViewEventsSource.h diff --git a/src/Storages/LiveView/LiveViewBlockOutputStream.h b/src/Storages/LiveView/LiveViewSink.h similarity index 91% rename from src/Storages/LiveView/LiveViewBlockOutputStream.h rename to src/Storages/LiveView/LiveViewSink.h index 6101607c21a..433a5554152 100644 --- a/src/Storages/LiveView/LiveViewBlockOutputStream.h +++ b/src/Storages/LiveView/LiveViewSink.h @@ -11,8 +11,15 @@ namespace DB class LiveViewSink : public SinkToStorage { + /// _version column is added manually in sink. + static Block updateHeader(Block block) + { + block.erase("_version"); + return block; + } + public: - explicit LiveViewSink(StorageLiveView & storage_) : SinkToStorage(storage_.getHeader()), storage(storage_) {} + explicit LiveViewSink(StorageLiveView & storage_) : SinkToStorage(updateHeader(storage_.getHeader())), storage(storage_) {} String getName() const override { return "LiveViewSink"; } diff --git a/src/Storages/LiveView/LiveViewBlockInputStream.h b/src/Storages/LiveView/LiveViewSource.h similarity index 100% rename from src/Storages/LiveView/LiveViewBlockInputStream.h rename to src/Storages/LiveView/LiveViewSource.h diff --git a/src/Storages/LiveView/StorageLiveView.cpp b/src/Storages/LiveView/StorageLiveView.cpp index 5ebb6cd2450..a9875da4170 100644 --- a/src/Storages/LiveView/StorageLiveView.cpp +++ b/src/Storages/LiveView/StorageLiveView.cpp @@ -19,6 +19,7 @@ limitations under the License. */ #include #include #include +#include #include #include #include @@ -116,6 +117,8 @@ MergeableBlocksPtr StorageLiveView::collectMergeableBlocks(ContextPtr local_cont return std::make_shared(cur_header); }); + new_mergeable_blocks->sample_block = io.pipeline.getHeader(); + PullingPipelineExecutor executor(io.pipeline); Block this_block; @@ -125,7 +128,6 @@ MergeableBlocksPtr StorageLiveView::collectMergeableBlocks(ContextPtr local_cont new_blocks->push_back(base_blocks); new_mergeable_blocks->blocks = new_blocks; - new_mergeable_blocks->sample_block = io.pipeline.getHeader(); return new_mergeable_blocks; } @@ -154,7 +156,6 @@ QueryPipeline StorageLiveView::completeQuery(Pipes pipes) std::move(pipes), QueryProcessingStage::WithMergeableState); }; block_context->addExternalTable(getBlocksTableName(), TemporaryTableHolder(getContext(), creator)); - InterpreterSelectQuery select(getInnerBlocksQuery(), block_context, StoragePtr(), nullptr, SelectQueryOptions(QueryProcessingStage::Complete)); auto io = select.execute(); io.pipeline.addSimpleTransform([&](const Block & cur_header) From 9b81c075c636d170ba3aba1e26ef10d5161aacb6 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 12 Aug 2021 12:53:32 +0200 Subject: [PATCH 514/599] Use today() to ensure the rollup is tested properly --- .../0_stateless/01236_graphite_mt.reference | 688 +++++++++--------- .../queries/0_stateless/01236_graphite_mt.sql | 46 +- 2 files changed, 371 insertions(+), 363 deletions(-) diff --git a/tests/queries/0_stateless/01236_graphite_mt.reference b/tests/queries/0_stateless/01236_graphite_mt.reference index c68becad53d..a30d2495265 100644 --- a/tests/queries/0_stateless/01236_graphite_mt.reference +++ b/tests/queries/0_stateless/01236_graphite_mt.reference @@ -1,344 +1,344 @@ -1 max_1 2021-08-09 23:50:00 9 1 0 -1 max_1 2021-08-09 23:40:00 19 1 10 -1 max_1 2021-08-09 23:30:00 29 1 20 -1 max_1 2021-08-09 23:20:00 39 1 30 -1 max_1 2021-08-09 23:10:00 49 1 40 -1 max_1 2021-08-09 23:00:00 59 1 50 -1 max_1 2021-08-09 22:50:00 69 1 60 -1 max_1 2021-08-09 22:40:00 79 1 70 -1 max_1 2021-08-09 22:30:00 89 1 80 -1 max_1 2021-08-09 22:20:00 99 1 90 -1 max_1 2021-08-09 22:10:00 109 1 100 -1 max_1 2021-08-09 22:00:00 119 1 110 -1 max_1 2021-08-09 21:50:00 129 1 120 -1 max_1 2021-08-09 21:40:00 139 1 130 -1 max_1 2021-08-09 21:30:00 149 1 140 -1 max_1 2021-08-09 21:20:00 159 1 150 -1 max_1 2021-08-09 21:10:00 169 1 160 -1 max_1 2021-08-09 21:00:00 179 1 170 -1 max_1 2021-08-09 20:50:00 189 1 180 -1 max_1 2021-08-09 20:40:00 199 1 190 -1 max_1 2021-08-09 20:30:00 209 1 200 -1 max_1 2021-08-09 20:20:00 219 1 210 -1 max_1 2021-08-09 20:10:00 229 1 220 -1 max_1 2021-08-09 20:00:00 239 1 230 -1 max_1 2021-08-09 19:50:00 249 1 240 -1 max_1 2021-08-09 19:40:00 259 1 250 -1 max_1 2021-08-09 19:30:00 269 1 260 -1 max_1 2021-08-09 19:20:00 279 1 270 -1 max_1 2021-08-09 19:10:00 289 1 280 -1 max_1 2021-08-09 19:00:00 299 1 290 -1 max_1 2021-08-06 23:20:00 39 1 0 -1 max_1 2021-08-06 21:40:00 139 1 40 -1 max_1 2021-08-06 20:00:00 239 1 140 -1 max_1 2021-08-06 18:20:00 339 1 240 -1 max_1 2021-08-06 16:40:00 439 1 340 -1 max_1 2021-08-06 15:00:00 539 1 440 -1 max_1 2021-08-06 13:20:00 639 1 540 -1 max_1 2021-08-06 11:40:00 739 1 640 -1 max_1 2021-08-06 10:00:00 839 1 740 -1 max_1 2021-08-06 08:20:00 939 1 840 -1 max_1 2021-08-06 06:40:00 1039 1 940 -1 max_1 2021-08-06 05:00:00 1139 1 1040 -1 max_1 2021-08-06 03:20:00 1199 1 1140 -1 max_2 2021-08-09 23:50:00 9 1 0 -1 max_2 2021-08-09 23:40:00 19 1 10 -1 max_2 2021-08-09 23:30:00 29 1 20 -1 max_2 2021-08-09 23:20:00 39 1 30 -1 max_2 2021-08-09 23:10:00 49 1 40 -1 max_2 2021-08-09 23:00:00 59 1 50 -1 max_2 2021-08-09 22:50:00 69 1 60 -1 max_2 2021-08-09 22:40:00 79 1 70 -1 max_2 2021-08-09 22:30:00 89 1 80 -1 max_2 2021-08-09 22:20:00 99 1 90 -1 max_2 2021-08-09 22:10:00 109 1 100 -1 max_2 2021-08-09 22:00:00 119 1 110 -1 max_2 2021-08-09 21:50:00 129 1 120 -1 max_2 2021-08-09 21:40:00 139 1 130 -1 max_2 2021-08-09 21:30:00 149 1 140 -1 max_2 2021-08-09 21:20:00 159 1 150 -1 max_2 2021-08-09 21:10:00 169 1 160 -1 max_2 2021-08-09 21:00:00 179 1 170 -1 max_2 2021-08-09 20:50:00 189 1 180 -1 max_2 2021-08-09 20:40:00 199 1 190 -1 max_2 2021-08-09 20:30:00 209 1 200 -1 max_2 2021-08-09 20:20:00 219 1 210 -1 max_2 2021-08-09 20:10:00 229 1 220 -1 max_2 2021-08-09 20:00:00 239 1 230 -1 max_2 2021-08-09 19:50:00 249 1 240 -1 max_2 2021-08-09 19:40:00 259 1 250 -1 max_2 2021-08-09 19:30:00 269 1 260 -1 max_2 2021-08-09 19:20:00 279 1 270 -1 max_2 2021-08-09 19:10:00 289 1 280 -1 max_2 2021-08-09 19:00:00 299 1 290 -1 max_2 2021-08-06 23:20:00 39 1 0 -1 max_2 2021-08-06 21:40:00 139 1 40 -1 max_2 2021-08-06 20:00:00 239 1 140 -1 max_2 2021-08-06 18:20:00 339 1 240 -1 max_2 2021-08-06 16:40:00 439 1 340 -1 max_2 2021-08-06 15:00:00 539 1 440 -1 max_2 2021-08-06 13:20:00 639 1 540 -1 max_2 2021-08-06 11:40:00 739 1 640 -1 max_2 2021-08-06 10:00:00 839 1 740 -1 max_2 2021-08-06 08:20:00 939 1 840 -1 max_2 2021-08-06 06:40:00 1039 1 940 -1 max_2 2021-08-06 05:00:00 1139 1 1040 -1 max_2 2021-08-06 03:20:00 1199 1 1140 -1 sum_1 2021-08-09 23:50:00 45 1 0 -1 sum_1 2021-08-09 23:40:00 145 1 10 -1 sum_1 2021-08-09 23:30:00 245 1 20 -1 sum_1 2021-08-09 23:20:00 345 1 30 -1 sum_1 2021-08-09 23:10:00 445 1 40 -1 sum_1 2021-08-09 23:00:00 545 1 50 -1 sum_1 2021-08-09 22:50:00 645 1 60 -1 sum_1 2021-08-09 22:40:00 745 1 70 -1 sum_1 2021-08-09 22:30:00 845 1 80 -1 sum_1 2021-08-09 22:20:00 945 1 90 -1 sum_1 2021-08-09 22:10:00 1045 1 100 -1 sum_1 2021-08-09 22:00:00 1145 1 110 -1 sum_1 2021-08-09 21:50:00 1245 1 120 -1 sum_1 2021-08-09 21:40:00 1345 1 130 -1 sum_1 2021-08-09 21:30:00 1445 1 140 -1 sum_1 2021-08-09 21:20:00 1545 1 150 -1 sum_1 2021-08-09 21:10:00 1645 1 160 -1 sum_1 2021-08-09 21:00:00 1745 1 170 -1 sum_1 2021-08-09 20:50:00 1845 1 180 -1 sum_1 2021-08-09 20:40:00 1945 1 190 -1 sum_1 2021-08-09 20:30:00 2045 1 200 -1 sum_1 2021-08-09 20:20:00 2145 1 210 -1 sum_1 2021-08-09 20:10:00 2245 1 220 -1 sum_1 2021-08-09 20:00:00 2345 1 230 -1 sum_1 2021-08-09 19:50:00 2445 1 240 -1 sum_1 2021-08-09 19:40:00 2545 1 250 -1 sum_1 2021-08-09 19:30:00 2645 1 260 -1 sum_1 2021-08-09 19:20:00 2745 1 270 -1 sum_1 2021-08-09 19:10:00 2845 1 280 -1 sum_1 2021-08-09 19:00:00 2945 1 290 -1 sum_1 2021-08-06 23:20:00 780 1 0 -1 sum_1 2021-08-06 21:40:00 8950 1 40 -1 sum_1 2021-08-06 20:00:00 18950 1 140 -1 sum_1 2021-08-06 18:20:00 28950 1 240 -1 sum_1 2021-08-06 16:40:00 38950 1 340 -1 sum_1 2021-08-06 15:00:00 48950 1 440 -1 sum_1 2021-08-06 13:20:00 58950 1 540 -1 sum_1 2021-08-06 11:40:00 68950 1 640 -1 sum_1 2021-08-06 10:00:00 78950 1 740 -1 sum_1 2021-08-06 08:20:00 88950 1 840 -1 sum_1 2021-08-06 06:40:00 98950 1 940 -1 sum_1 2021-08-06 05:00:00 108950 1 1040 -1 sum_1 2021-08-06 03:20:00 70170 1 1140 -1 sum_2 2021-08-09 23:50:00 45 1 0 -1 sum_2 2021-08-09 23:40:00 145 1 10 -1 sum_2 2021-08-09 23:30:00 245 1 20 -1 sum_2 2021-08-09 23:20:00 345 1 30 -1 sum_2 2021-08-09 23:10:00 445 1 40 -1 sum_2 2021-08-09 23:00:00 545 1 50 -1 sum_2 2021-08-09 22:50:00 645 1 60 -1 sum_2 2021-08-09 22:40:00 745 1 70 -1 sum_2 2021-08-09 22:30:00 845 1 80 -1 sum_2 2021-08-09 22:20:00 945 1 90 -1 sum_2 2021-08-09 22:10:00 1045 1 100 -1 sum_2 2021-08-09 22:00:00 1145 1 110 -1 sum_2 2021-08-09 21:50:00 1245 1 120 -1 sum_2 2021-08-09 21:40:00 1345 1 130 -1 sum_2 2021-08-09 21:30:00 1445 1 140 -1 sum_2 2021-08-09 21:20:00 1545 1 150 -1 sum_2 2021-08-09 21:10:00 1645 1 160 -1 sum_2 2021-08-09 21:00:00 1745 1 170 -1 sum_2 2021-08-09 20:50:00 1845 1 180 -1 sum_2 2021-08-09 20:40:00 1945 1 190 -1 sum_2 2021-08-09 20:30:00 2045 1 200 -1 sum_2 2021-08-09 20:20:00 2145 1 210 -1 sum_2 2021-08-09 20:10:00 2245 1 220 -1 sum_2 2021-08-09 20:00:00 2345 1 230 -1 sum_2 2021-08-09 19:50:00 2445 1 240 -1 sum_2 2021-08-09 19:40:00 2545 1 250 -1 sum_2 2021-08-09 19:30:00 2645 1 260 -1 sum_2 2021-08-09 19:20:00 2745 1 270 -1 sum_2 2021-08-09 19:10:00 2845 1 280 -1 sum_2 2021-08-09 19:00:00 2945 1 290 -1 sum_2 2021-08-06 23:20:00 780 1 0 -1 sum_2 2021-08-06 21:40:00 8950 1 40 -1 sum_2 2021-08-06 20:00:00 18950 1 140 -1 sum_2 2021-08-06 18:20:00 28950 1 240 -1 sum_2 2021-08-06 16:40:00 38950 1 340 -1 sum_2 2021-08-06 15:00:00 48950 1 440 -1 sum_2 2021-08-06 13:20:00 58950 1 540 -1 sum_2 2021-08-06 11:40:00 68950 1 640 -1 sum_2 2021-08-06 10:00:00 78950 1 740 -1 sum_2 2021-08-06 08:20:00 88950 1 840 -1 sum_2 2021-08-06 06:40:00 98950 1 940 -1 sum_2 2021-08-06 05:00:00 108950 1 1040 -1 sum_2 2021-08-06 03:20:00 70170 1 1140 -2 max_1 2021-08-09 23:50:00 9 1 0 -2 max_1 2021-08-09 23:40:00 19 1 10 -2 max_1 2021-08-09 23:30:00 29 1 20 -2 max_1 2021-08-09 23:20:00 39 1 30 -2 max_1 2021-08-09 23:10:00 49 1 40 -2 max_1 2021-08-09 23:00:00 59 1 50 -2 max_1 2021-08-09 22:50:00 69 1 60 -2 max_1 2021-08-09 22:40:00 79 1 70 -2 max_1 2021-08-09 22:30:00 89 1 80 -2 max_1 2021-08-09 22:20:00 99 1 90 -2 max_1 2021-08-09 22:10:00 109 1 100 -2 max_1 2021-08-09 22:00:00 119 1 110 -2 max_1 2021-08-09 21:50:00 129 1 120 -2 max_1 2021-08-09 21:40:00 139 1 130 -2 max_1 2021-08-09 21:30:00 149 1 140 -2 max_1 2021-08-09 21:20:00 159 1 150 -2 max_1 2021-08-09 21:10:00 169 1 160 -2 max_1 2021-08-09 21:00:00 179 1 170 -2 max_1 2021-08-09 20:50:00 189 1 180 -2 max_1 2021-08-09 20:40:00 199 1 190 -2 max_1 2021-08-09 20:30:00 209 1 200 -2 max_1 2021-08-09 20:20:00 219 1 210 -2 max_1 2021-08-09 20:10:00 229 1 220 -2 max_1 2021-08-09 20:00:00 239 1 230 -2 max_1 2021-08-09 19:50:00 249 1 240 -2 max_1 2021-08-09 19:40:00 259 1 250 -2 max_1 2021-08-09 19:30:00 269 1 260 -2 max_1 2021-08-09 19:20:00 279 1 270 -2 max_1 2021-08-09 19:10:00 289 1 280 -2 max_1 2021-08-09 19:00:00 299 1 290 -2 max_1 2021-08-06 23:20:00 39 1 0 -2 max_1 2021-08-06 21:40:00 139 1 40 -2 max_1 2021-08-06 20:00:00 239 1 140 -2 max_1 2021-08-06 18:20:00 339 1 240 -2 max_1 2021-08-06 16:40:00 439 1 340 -2 max_1 2021-08-06 15:00:00 539 1 440 -2 max_1 2021-08-06 13:20:00 639 1 540 -2 max_1 2021-08-06 11:40:00 739 1 640 -2 max_1 2021-08-06 10:00:00 839 1 740 -2 max_1 2021-08-06 08:20:00 939 1 840 -2 max_1 2021-08-06 06:40:00 1039 1 940 -2 max_1 2021-08-06 05:00:00 1139 1 1040 -2 max_1 2021-08-06 03:20:00 1199 1 1140 -2 max_2 2021-08-09 23:50:00 9 1 0 -2 max_2 2021-08-09 23:40:00 19 1 10 -2 max_2 2021-08-09 23:30:00 29 1 20 -2 max_2 2021-08-09 23:20:00 39 1 30 -2 max_2 2021-08-09 23:10:00 49 1 40 -2 max_2 2021-08-09 23:00:00 59 1 50 -2 max_2 2021-08-09 22:50:00 69 1 60 -2 max_2 2021-08-09 22:40:00 79 1 70 -2 max_2 2021-08-09 22:30:00 89 1 80 -2 max_2 2021-08-09 22:20:00 99 1 90 -2 max_2 2021-08-09 22:10:00 109 1 100 -2 max_2 2021-08-09 22:00:00 119 1 110 -2 max_2 2021-08-09 21:50:00 129 1 120 -2 max_2 2021-08-09 21:40:00 139 1 130 -2 max_2 2021-08-09 21:30:00 149 1 140 -2 max_2 2021-08-09 21:20:00 159 1 150 -2 max_2 2021-08-09 21:10:00 169 1 160 -2 max_2 2021-08-09 21:00:00 179 1 170 -2 max_2 2021-08-09 20:50:00 189 1 180 -2 max_2 2021-08-09 20:40:00 199 1 190 -2 max_2 2021-08-09 20:30:00 209 1 200 -2 max_2 2021-08-09 20:20:00 219 1 210 -2 max_2 2021-08-09 20:10:00 229 1 220 -2 max_2 2021-08-09 20:00:00 239 1 230 -2 max_2 2021-08-09 19:50:00 249 1 240 -2 max_2 2021-08-09 19:40:00 259 1 250 -2 max_2 2021-08-09 19:30:00 269 1 260 -2 max_2 2021-08-09 19:20:00 279 1 270 -2 max_2 2021-08-09 19:10:00 289 1 280 -2 max_2 2021-08-09 19:00:00 299 1 290 -2 max_2 2021-08-06 23:20:00 39 1 0 -2 max_2 2021-08-06 21:40:00 139 1 40 -2 max_2 2021-08-06 20:00:00 239 1 140 -2 max_2 2021-08-06 18:20:00 339 1 240 -2 max_2 2021-08-06 16:40:00 439 1 340 -2 max_2 2021-08-06 15:00:00 539 1 440 -2 max_2 2021-08-06 13:20:00 639 1 540 -2 max_2 2021-08-06 11:40:00 739 1 640 -2 max_2 2021-08-06 10:00:00 839 1 740 -2 max_2 2021-08-06 08:20:00 939 1 840 -2 max_2 2021-08-06 06:40:00 1039 1 940 -2 max_2 2021-08-06 05:00:00 1139 1 1040 -2 max_2 2021-08-06 03:20:00 1199 1 1140 -2 sum_1 2021-08-09 23:50:00 45 1 0 -2 sum_1 2021-08-09 23:40:00 145 1 10 -2 sum_1 2021-08-09 23:30:00 245 1 20 -2 sum_1 2021-08-09 23:20:00 345 1 30 -2 sum_1 2021-08-09 23:10:00 445 1 40 -2 sum_1 2021-08-09 23:00:00 545 1 50 -2 sum_1 2021-08-09 22:50:00 645 1 60 -2 sum_1 2021-08-09 22:40:00 745 1 70 -2 sum_1 2021-08-09 22:30:00 845 1 80 -2 sum_1 2021-08-09 22:20:00 945 1 90 -2 sum_1 2021-08-09 22:10:00 1045 1 100 -2 sum_1 2021-08-09 22:00:00 1145 1 110 -2 sum_1 2021-08-09 21:50:00 1245 1 120 -2 sum_1 2021-08-09 21:40:00 1345 1 130 -2 sum_1 2021-08-09 21:30:00 1445 1 140 -2 sum_1 2021-08-09 21:20:00 1545 1 150 -2 sum_1 2021-08-09 21:10:00 1645 1 160 -2 sum_1 2021-08-09 21:00:00 1745 1 170 -2 sum_1 2021-08-09 20:50:00 1845 1 180 -2 sum_1 2021-08-09 20:40:00 1945 1 190 -2 sum_1 2021-08-09 20:30:00 2045 1 200 -2 sum_1 2021-08-09 20:20:00 2145 1 210 -2 sum_1 2021-08-09 20:10:00 2245 1 220 -2 sum_1 2021-08-09 20:00:00 2345 1 230 -2 sum_1 2021-08-09 19:50:00 2445 1 240 -2 sum_1 2021-08-09 19:40:00 2545 1 250 -2 sum_1 2021-08-09 19:30:00 2645 1 260 -2 sum_1 2021-08-09 19:20:00 2745 1 270 -2 sum_1 2021-08-09 19:10:00 2845 1 280 -2 sum_1 2021-08-09 19:00:00 2945 1 290 -2 sum_1 2021-08-06 23:20:00 780 1 0 -2 sum_1 2021-08-06 21:40:00 8950 1 40 -2 sum_1 2021-08-06 20:00:00 18950 1 140 -2 sum_1 2021-08-06 18:20:00 28950 1 240 -2 sum_1 2021-08-06 16:40:00 38950 1 340 -2 sum_1 2021-08-06 15:00:00 48950 1 440 -2 sum_1 2021-08-06 13:20:00 58950 1 540 -2 sum_1 2021-08-06 11:40:00 68950 1 640 -2 sum_1 2021-08-06 10:00:00 78950 1 740 -2 sum_1 2021-08-06 08:20:00 88950 1 840 -2 sum_1 2021-08-06 06:40:00 98950 1 940 -2 sum_1 2021-08-06 05:00:00 108950 1 1040 -2 sum_1 2021-08-06 03:20:00 70170 1 1140 -2 sum_2 2021-08-09 23:50:00 45 1 0 -2 sum_2 2021-08-09 23:40:00 145 1 10 -2 sum_2 2021-08-09 23:30:00 245 1 20 -2 sum_2 2021-08-09 23:20:00 345 1 30 -2 sum_2 2021-08-09 23:10:00 445 1 40 -2 sum_2 2021-08-09 23:00:00 545 1 50 -2 sum_2 2021-08-09 22:50:00 645 1 60 -2 sum_2 2021-08-09 22:40:00 745 1 70 -2 sum_2 2021-08-09 22:30:00 845 1 80 -2 sum_2 2021-08-09 22:20:00 945 1 90 -2 sum_2 2021-08-09 22:10:00 1045 1 100 -2 sum_2 2021-08-09 22:00:00 1145 1 110 -2 sum_2 2021-08-09 21:50:00 1245 1 120 -2 sum_2 2021-08-09 21:40:00 1345 1 130 -2 sum_2 2021-08-09 21:30:00 1445 1 140 -2 sum_2 2021-08-09 21:20:00 1545 1 150 -2 sum_2 2021-08-09 21:10:00 1645 1 160 -2 sum_2 2021-08-09 21:00:00 1745 1 170 -2 sum_2 2021-08-09 20:50:00 1845 1 180 -2 sum_2 2021-08-09 20:40:00 1945 1 190 -2 sum_2 2021-08-09 20:30:00 2045 1 200 -2 sum_2 2021-08-09 20:20:00 2145 1 210 -2 sum_2 2021-08-09 20:10:00 2245 1 220 -2 sum_2 2021-08-09 20:00:00 2345 1 230 -2 sum_2 2021-08-09 19:50:00 2445 1 240 -2 sum_2 2021-08-09 19:40:00 2545 1 250 -2 sum_2 2021-08-09 19:30:00 2645 1 260 -2 sum_2 2021-08-09 19:20:00 2745 1 270 -2 sum_2 2021-08-09 19:10:00 2845 1 280 -2 sum_2 2021-08-09 19:00:00 2945 1 290 -2 sum_2 2021-08-06 23:20:00 780 1 0 -2 sum_2 2021-08-06 21:40:00 8950 1 40 -2 sum_2 2021-08-06 20:00:00 18950 1 140 -2 sum_2 2021-08-06 18:20:00 28950 1 240 -2 sum_2 2021-08-06 16:40:00 38950 1 340 -2 sum_2 2021-08-06 15:00:00 48950 1 440 -2 sum_2 2021-08-06 13:20:00 58950 1 540 -2 sum_2 2021-08-06 11:40:00 68950 1 640 -2 sum_2 2021-08-06 10:00:00 78950 1 740 -2 sum_2 2021-08-06 08:20:00 88950 1 840 -2 sum_2 2021-08-06 06:40:00 98950 1 940 -2 sum_2 2021-08-06 05:00:00 108950 1 1040 -2 sum_2 2021-08-06 03:20:00 70170 1 1140 +1 max_1 9 1 0 +1 max_1 19 1 10 +1 max_1 29 1 20 +1 max_1 39 1 30 +1 max_1 49 1 40 +1 max_1 59 1 50 +1 max_1 69 1 60 +1 max_1 79 1 70 +1 max_1 89 1 80 +1 max_1 99 1 90 +1 max_1 109 1 100 +1 max_1 119 1 110 +1 max_1 129 1 120 +1 max_1 139 1 130 +1 max_1 149 1 140 +1 max_1 159 1 150 +1 max_1 169 1 160 +1 max_1 179 1 170 +1 max_1 189 1 180 +1 max_1 199 1 190 +1 max_1 209 1 200 +1 max_1 219 1 210 +1 max_1 229 1 220 +1 max_1 239 1 230 +1 max_1 249 1 240 +1 max_1 259 1 250 +1 max_1 269 1 260 +1 max_1 279 1 270 +1 max_1 289 1 280 +1 max_1 299 1 290 +1 max_1 39 1 0 +1 max_1 139 1 40 +1 max_1 239 1 140 +1 max_1 339 1 240 +1 max_1 439 1 340 +1 max_1 539 1 440 +1 max_1 639 1 540 +1 max_1 739 1 640 +1 max_1 839 1 740 +1 max_1 939 1 840 +1 max_1 1039 1 940 +1 max_1 1139 1 1040 +1 max_1 1199 1 1140 +1 max_2 9 1 0 +1 max_2 19 1 10 +1 max_2 29 1 20 +1 max_2 39 1 30 +1 max_2 49 1 40 +1 max_2 59 1 50 +1 max_2 69 1 60 +1 max_2 79 1 70 +1 max_2 89 1 80 +1 max_2 99 1 90 +1 max_2 109 1 100 +1 max_2 119 1 110 +1 max_2 129 1 120 +1 max_2 139 1 130 +1 max_2 149 1 140 +1 max_2 159 1 150 +1 max_2 169 1 160 +1 max_2 179 1 170 +1 max_2 189 1 180 +1 max_2 199 1 190 +1 max_2 209 1 200 +1 max_2 219 1 210 +1 max_2 229 1 220 +1 max_2 239 1 230 +1 max_2 249 1 240 +1 max_2 259 1 250 +1 max_2 269 1 260 +1 max_2 279 1 270 +1 max_2 289 1 280 +1 max_2 299 1 290 +1 max_2 39 1 0 +1 max_2 139 1 40 +1 max_2 239 1 140 +1 max_2 339 1 240 +1 max_2 439 1 340 +1 max_2 539 1 440 +1 max_2 639 1 540 +1 max_2 739 1 640 +1 max_2 839 1 740 +1 max_2 939 1 840 +1 max_2 1039 1 940 +1 max_2 1139 1 1040 +1 max_2 1199 1 1140 +1 sum_1 45 1 0 +1 sum_1 145 1 10 +1 sum_1 245 1 20 +1 sum_1 345 1 30 +1 sum_1 445 1 40 +1 sum_1 545 1 50 +1 sum_1 645 1 60 +1 sum_1 745 1 70 +1 sum_1 845 1 80 +1 sum_1 945 1 90 +1 sum_1 1045 1 100 +1 sum_1 1145 1 110 +1 sum_1 1245 1 120 +1 sum_1 1345 1 130 +1 sum_1 1445 1 140 +1 sum_1 1545 1 150 +1 sum_1 1645 1 160 +1 sum_1 1745 1 170 +1 sum_1 1845 1 180 +1 sum_1 1945 1 190 +1 sum_1 2045 1 200 +1 sum_1 2145 1 210 +1 sum_1 2245 1 220 +1 sum_1 2345 1 230 +1 sum_1 2445 1 240 +1 sum_1 2545 1 250 +1 sum_1 2645 1 260 +1 sum_1 2745 1 270 +1 sum_1 2845 1 280 +1 sum_1 2945 1 290 +1 sum_1 780 1 0 +1 sum_1 8950 1 40 +1 sum_1 18950 1 140 +1 sum_1 28950 1 240 +1 sum_1 38950 1 340 +1 sum_1 48950 1 440 +1 sum_1 58950 1 540 +1 sum_1 68950 1 640 +1 sum_1 78950 1 740 +1 sum_1 88950 1 840 +1 sum_1 98950 1 940 +1 sum_1 108950 1 1040 +1 sum_1 70170 1 1140 +1 sum_2 45 1 0 +1 sum_2 145 1 10 +1 sum_2 245 1 20 +1 sum_2 345 1 30 +1 sum_2 445 1 40 +1 sum_2 545 1 50 +1 sum_2 645 1 60 +1 sum_2 745 1 70 +1 sum_2 845 1 80 +1 sum_2 945 1 90 +1 sum_2 1045 1 100 +1 sum_2 1145 1 110 +1 sum_2 1245 1 120 +1 sum_2 1345 1 130 +1 sum_2 1445 1 140 +1 sum_2 1545 1 150 +1 sum_2 1645 1 160 +1 sum_2 1745 1 170 +1 sum_2 1845 1 180 +1 sum_2 1945 1 190 +1 sum_2 2045 1 200 +1 sum_2 2145 1 210 +1 sum_2 2245 1 220 +1 sum_2 2345 1 230 +1 sum_2 2445 1 240 +1 sum_2 2545 1 250 +1 sum_2 2645 1 260 +1 sum_2 2745 1 270 +1 sum_2 2845 1 280 +1 sum_2 2945 1 290 +1 sum_2 780 1 0 +1 sum_2 8950 1 40 +1 sum_2 18950 1 140 +1 sum_2 28950 1 240 +1 sum_2 38950 1 340 +1 sum_2 48950 1 440 +1 sum_2 58950 1 540 +1 sum_2 68950 1 640 +1 sum_2 78950 1 740 +1 sum_2 88950 1 840 +1 sum_2 98950 1 940 +1 sum_2 108950 1 1040 +1 sum_2 70170 1 1140 +2 max_1 9 1 0 +2 max_1 19 1 10 +2 max_1 29 1 20 +2 max_1 39 1 30 +2 max_1 49 1 40 +2 max_1 59 1 50 +2 max_1 69 1 60 +2 max_1 79 1 70 +2 max_1 89 1 80 +2 max_1 99 1 90 +2 max_1 109 1 100 +2 max_1 119 1 110 +2 max_1 129 1 120 +2 max_1 139 1 130 +2 max_1 149 1 140 +2 max_1 159 1 150 +2 max_1 169 1 160 +2 max_1 179 1 170 +2 max_1 189 1 180 +2 max_1 199 1 190 +2 max_1 209 1 200 +2 max_1 219 1 210 +2 max_1 229 1 220 +2 max_1 239 1 230 +2 max_1 249 1 240 +2 max_1 259 1 250 +2 max_1 269 1 260 +2 max_1 279 1 270 +2 max_1 289 1 280 +2 max_1 299 1 290 +2 max_1 39 1 0 +2 max_1 139 1 40 +2 max_1 239 1 140 +2 max_1 339 1 240 +2 max_1 439 1 340 +2 max_1 539 1 440 +2 max_1 639 1 540 +2 max_1 739 1 640 +2 max_1 839 1 740 +2 max_1 939 1 840 +2 max_1 1039 1 940 +2 max_1 1139 1 1040 +2 max_1 1199 1 1140 +2 max_2 9 1 0 +2 max_2 19 1 10 +2 max_2 29 1 20 +2 max_2 39 1 30 +2 max_2 49 1 40 +2 max_2 59 1 50 +2 max_2 69 1 60 +2 max_2 79 1 70 +2 max_2 89 1 80 +2 max_2 99 1 90 +2 max_2 109 1 100 +2 max_2 119 1 110 +2 max_2 129 1 120 +2 max_2 139 1 130 +2 max_2 149 1 140 +2 max_2 159 1 150 +2 max_2 169 1 160 +2 max_2 179 1 170 +2 max_2 189 1 180 +2 max_2 199 1 190 +2 max_2 209 1 200 +2 max_2 219 1 210 +2 max_2 229 1 220 +2 max_2 239 1 230 +2 max_2 249 1 240 +2 max_2 259 1 250 +2 max_2 269 1 260 +2 max_2 279 1 270 +2 max_2 289 1 280 +2 max_2 299 1 290 +2 max_2 39 1 0 +2 max_2 139 1 40 +2 max_2 239 1 140 +2 max_2 339 1 240 +2 max_2 439 1 340 +2 max_2 539 1 440 +2 max_2 639 1 540 +2 max_2 739 1 640 +2 max_2 839 1 740 +2 max_2 939 1 840 +2 max_2 1039 1 940 +2 max_2 1139 1 1040 +2 max_2 1199 1 1140 +2 sum_1 45 1 0 +2 sum_1 145 1 10 +2 sum_1 245 1 20 +2 sum_1 345 1 30 +2 sum_1 445 1 40 +2 sum_1 545 1 50 +2 sum_1 645 1 60 +2 sum_1 745 1 70 +2 sum_1 845 1 80 +2 sum_1 945 1 90 +2 sum_1 1045 1 100 +2 sum_1 1145 1 110 +2 sum_1 1245 1 120 +2 sum_1 1345 1 130 +2 sum_1 1445 1 140 +2 sum_1 1545 1 150 +2 sum_1 1645 1 160 +2 sum_1 1745 1 170 +2 sum_1 1845 1 180 +2 sum_1 1945 1 190 +2 sum_1 2045 1 200 +2 sum_1 2145 1 210 +2 sum_1 2245 1 220 +2 sum_1 2345 1 230 +2 sum_1 2445 1 240 +2 sum_1 2545 1 250 +2 sum_1 2645 1 260 +2 sum_1 2745 1 270 +2 sum_1 2845 1 280 +2 sum_1 2945 1 290 +2 sum_1 780 1 0 +2 sum_1 8950 1 40 +2 sum_1 18950 1 140 +2 sum_1 28950 1 240 +2 sum_1 38950 1 340 +2 sum_1 48950 1 440 +2 sum_1 58950 1 540 +2 sum_1 68950 1 640 +2 sum_1 78950 1 740 +2 sum_1 88950 1 840 +2 sum_1 98950 1 940 +2 sum_1 108950 1 1040 +2 sum_1 70170 1 1140 +2 sum_2 45 1 0 +2 sum_2 145 1 10 +2 sum_2 245 1 20 +2 sum_2 345 1 30 +2 sum_2 445 1 40 +2 sum_2 545 1 50 +2 sum_2 645 1 60 +2 sum_2 745 1 70 +2 sum_2 845 1 80 +2 sum_2 945 1 90 +2 sum_2 1045 1 100 +2 sum_2 1145 1 110 +2 sum_2 1245 1 120 +2 sum_2 1345 1 130 +2 sum_2 1445 1 140 +2 sum_2 1545 1 150 +2 sum_2 1645 1 160 +2 sum_2 1745 1 170 +2 sum_2 1845 1 180 +2 sum_2 1945 1 190 +2 sum_2 2045 1 200 +2 sum_2 2145 1 210 +2 sum_2 2245 1 220 +2 sum_2 2345 1 230 +2 sum_2 2445 1 240 +2 sum_2 2545 1 250 +2 sum_2 2645 1 260 +2 sum_2 2745 1 270 +2 sum_2 2845 1 280 +2 sum_2 2945 1 290 +2 sum_2 780 1 0 +2 sum_2 8950 1 40 +2 sum_2 18950 1 140 +2 sum_2 28950 1 240 +2 sum_2 38950 1 340 +2 sum_2 48950 1 440 +2 sum_2 58950 1 540 +2 sum_2 68950 1 640 +2 sum_2 78950 1 740 +2 sum_2 88950 1 840 +2 sum_2 98950 1 940 +2 sum_2 108950 1 1040 +2 sum_2 70170 1 1140 diff --git a/tests/queries/0_stateless/01236_graphite_mt.sql b/tests/queries/0_stateless/01236_graphite_mt.sql index 6f4af186a1c..d5289364d32 100644 --- a/tests/queries/0_stateless/01236_graphite_mt.sql +++ b/tests/queries/0_stateless/01236_graphite_mt.sql @@ -4,28 +4,36 @@ drop table if exists test_graphite; create table test_graphite (key UInt32, Path String, Time DateTime('UTC'), Value Float64, Version UInt32, col UInt64) engine = GraphiteMergeTree('graphite_rollup') order by key settings index_granularity=10; -insert into test_graphite -select 1, 'sum_1', toDateTime('2021-08-10', 'UTC') - number * 60 - 30, number, 1, number from numbers(300) union all -select 2, 'sum_1', toDateTime('2021-08-10', 'UTC') - number * 60 - 30, number, 1, number from numbers(300) union all -select 1, 'sum_2', toDateTime('2021-08-10', 'UTC') - number * 60 - 30, number, 1, number from numbers(300) union all -select 2, 'sum_2', toDateTime('2021-08-10', 'UTC') - number * 60 - 30, number, 1, number from numbers(300) union all -select 1, 'max_1', toDateTime('2021-08-10', 'UTC') - number * 60 - 30, number, 1, number from numbers(300) union all -select 2, 'max_1', toDateTime('2021-08-10', 'UTC') - number * 60 - 30, number, 1, number from numbers(300) union all -select 1, 'max_2', toDateTime('2021-08-10', 'UTC') - number * 60 - 30, number, 1, number from numbers(300) union all -select 2, 'max_2', toDateTime('2021-08-10', 'UTC') - number * 60 - 30, number, 1, number from numbers(300); +SET joined_subquery_requires_alias = 0; -insert into test_graphite -select 1, 'sum_1', toDateTime('2021-08-07', 'UTC') - number * 60 - 30, number, 1, number from numbers(1200) union all -select 2, 'sum_1', toDateTime('2021-08-07', 'UTC') - number * 60 - 30, number, 1, number from numbers(1200) union all -select 1, 'sum_2', toDateTime('2021-08-07', 'UTC') - number * 60 - 30, number, 1, number from numbers(1200) union all -select 2, 'sum_2', toDateTime('2021-08-07', 'UTC') - number * 60 - 30, number, 1, number from numbers(1200) union all -select 1, 'max_1', toDateTime('2021-08-07', 'UTC') - number * 60 - 30, number, 1, number from numbers(1200) union all -select 2, 'max_1', toDateTime('2021-08-07', 'UTC') - number * 60 - 30, number, 1, number from numbers(1200) union all -select 1, 'max_2', toDateTime('2021-08-07', 'UTC') - number * 60 - 30, number, 1, number from numbers(1200) union all -select 2, 'max_2', toDateTime('2021-08-07', 'UTC') - number * 60 - 30, number, 1, number from numbers(1200); +INSERT into test_graphite +WITH dates AS + ( + SELECT today() as today, + today - 3 as older_date + ) + -- Newer than 2 days are kept in windows of 600 seconds + select 1, 'sum_1', toDateTime(today, 'UTC') - number * 60 - 30, number, 1, number from dates, numbers(300) union all + select 2, 'sum_1', toDateTime(today, 'UTC') - number * 60 - 30, number, 1, number from dates, numbers(300) union all + select 1, 'sum_2', toDateTime(today, 'UTC') - number * 60 - 30, number, 1, number from dates, numbers(300) union all + select 2, 'sum_2', toDateTime(today, 'UTC') - number * 60 - 30, number, 1, number from dates, numbers(300) union all + select 1, 'max_1', toDateTime(today, 'UTC') - number * 60 - 30, number, 1, number from dates, numbers(300) union all + select 2, 'max_1', toDateTime(today, 'UTC') - number * 60 - 30, number, 1, number from dates, numbers(300) union all + select 1, 'max_2', toDateTime(today, 'UTC') - number * 60 - 30, number, 1, number from dates, numbers(300) union all + select 2, 'max_2', toDateTime(today, 'UTC') - number * 60 - 30, number, 1, number from dates, numbers(300) union all + + -- Older than 2 days use 6000 second windows + select 1, 'sum_1', toDateTime(older_date, 'UTC') - number * 60 - 30, number, 1, number from dates, numbers(1200) union all + select 2, 'sum_1', toDateTime(older_date, 'UTC') - number * 60 - 30, number, 1, number from dates, numbers(1200) union all + select 1, 'sum_2', toDateTime(older_date, 'UTC') - number * 60 - 30, number, 1, number from dates, numbers(1200) union all + select 2, 'sum_2', toDateTime(older_date, 'UTC') - number * 60 - 30, number, 1, number from dates, numbers(1200) union all + select 1, 'max_1', toDateTime(older_date, 'UTC') - number * 60 - 30, number, 1, number from dates, numbers(1200) union all + select 2, 'max_1', toDateTime(older_date, 'UTC') - number * 60 - 30, number, 1, number from dates, numbers(1200) union all + select 1, 'max_2', toDateTime(older_date, 'UTC') - number * 60 - 30, number, 1, number from dates, numbers(1200) union all + select 2, 'max_2', toDateTime(older_date, 'UTC') - number * 60 - 30, number, 1, number from dates, numbers(1200); optimize table test_graphite final; -select key, Path, Time, Value, Version, col from test_graphite order by key, Path, Time desc; +select key, Path, Value, Version, col from test_graphite order by key, Path, Time desc; drop table test_graphite; From c449cf072a0717d05425746d3f1bf0f7fbd052f1 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Thu, 12 Aug 2021 14:25:42 +0300 Subject: [PATCH 515/599] Update RequiredSourceColumnsVisitor.cpp --- src/Interpreters/RequiredSourceColumnsVisitor.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/RequiredSourceColumnsVisitor.cpp b/src/Interpreters/RequiredSourceColumnsVisitor.cpp index 2c81969009f..21ec94a6917 100644 --- a/src/Interpreters/RequiredSourceColumnsVisitor.cpp +++ b/src/Interpreters/RequiredSourceColumnsVisitor.cpp @@ -123,7 +123,7 @@ void RequiredSourceColumnsMatcher::visit(const ASTSelectQuery & select, const AS data.addColumnAliasIfAny(*node); } - if (auto & with = select.with()) + if (const auto & with = select.with()) { for (auto & node : with->children) { From 735a37765973d522aba4eaf26b43e3869daa7235 Mon Sep 17 00:00:00 2001 From: Vladimir Goncharov Date: Thu, 12 Aug 2021 14:32:35 +0300 Subject: [PATCH 516/599] Update docs/en/engines/table-engines/mergetree-family/mergetree.md Co-authored-by: Alexey Boykov <33257111+mathalex@users.noreply.github.com> --- docs/en/engines/table-engines/mergetree-family/mergetree.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/engines/table-engines/mergetree-family/mergetree.md b/docs/en/engines/table-engines/mergetree-family/mergetree.md index d9ff302a866..4ab44475a8e 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -404,7 +404,7 @@ Projections are stored inside the part directory. It's similar to an index but c #### Query Analysis {#projection-query-analysis} 1. Check if the projection can be used to answer the given query, that is, it generates the same answer as querying the base table. 2. Select the best feasible match, which contains the least granules to read. -3. The query pipeline which uses projections will be different from the one that uses the original parts. if the projection is absent in some parts, we can add the pipeline to "project" it on the fly. +3. The query pipeline which uses projections will be different from the one that uses the original parts. If the projection is absent in some parts, we can add the pipeline to "project" it on the fly. ## Concurrent Data Access {#concurrent-data-access} From 224e0a547a9fad8d6e5158e29fad4db393243a00 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 12 Aug 2021 16:02:17 +0300 Subject: [PATCH 517/599] Fix build. --- src/Storages/LiveView/StorageLiveView.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Storages/LiveView/StorageLiveView.cpp b/src/Storages/LiveView/StorageLiveView.cpp index a9875da4170..69390850ccc 100644 --- a/src/Storages/LiveView/StorageLiveView.cpp +++ b/src/Storages/LiveView/StorageLiveView.cpp @@ -27,9 +27,9 @@ limitations under the License. */ #include #include -#include -#include -#include +#include +#include +#include #include #include From c2f5a9d556197c7f2a655e0ab469f444ab30bc90 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 12 Aug 2021 16:13:53 +0300 Subject: [PATCH 518/599] Fix ya.make --- src/Processors/ya.make | 5 +++++ src/Processors/ya.make.in | 2 +- 2 files changed, 6 insertions(+), 1 deletion(-) diff --git a/src/Processors/ya.make b/src/Processors/ya.make index 891a309b2d8..ad4cc5e4a96 100644 --- a/src/Processors/ya.make +++ b/src/Processors/ya.make @@ -31,8 +31,13 @@ SRCS( Formats/IOutputFormat.cpp Formats/IRowInputFormat.cpp Formats/IRowOutputFormat.cpp + Formats/Impl/ArrowBlockInputFormat.cpp + Formats/Impl/ArrowBlockOutputFormat.cpp + Formats/Impl/ArrowBufferedStreams.cpp + Formats/Impl/ArrowColumnToCHColumn.cpp Formats/Impl/BinaryRowInputFormat.cpp Formats/Impl/BinaryRowOutputFormat.cpp + Formats/Impl/CHColumnToArrowColumn.cpp Formats/Impl/CSVRowInputFormat.cpp Formats/Impl/CSVRowOutputFormat.cpp Formats/Impl/ConstantExpressionTemplate.cpp diff --git a/src/Processors/ya.make.in b/src/Processors/ya.make.in index d4b98acf01c..7160e80bcce 100644 --- a/src/Processors/ya.make.in +++ b/src/Processors/ya.make.in @@ -16,7 +16,7 @@ ADDINCL( CFLAGS(-DUSE_ARROW=1) SRCS( - + ) END() From 36878cd377df1414337a6549daf34d4bee726bdf Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Thu, 12 Aug 2021 16:18:33 +0300 Subject: [PATCH 519/599] Add ru docs for uuid and array functions. --- .../functions/array-functions.md | 3 +- .../functions/array-functions.md | 86 +++++++++++++++++-- .../sql-reference/functions/uuid-functions.md | 84 ++++++++++++++++++ 3 files changed, 164 insertions(+), 9 deletions(-) diff --git a/docs/en/sql-reference/functions/array-functions.md b/docs/en/sql-reference/functions/array-functions.md index 38ee2090866..ddbbab3ecfc 100644 --- a/docs/en/sql-reference/functions/array-functions.md +++ b/docs/en/sql-reference/functions/array-functions.md @@ -18,7 +18,7 @@ empty([x]) An array is considered empty if it contains all empty elements. !!! note "Note" - Can be optimized by enabling the [optimize_functions_to_subcolumns](../../operations/settings/settings.md#optimize-functions-to-subcolumns) setting. With `optimize_functions_to_subcolumns = 1` the function reads only [size0](../../sql-reference/data-types/array.md#array-size) subcolumn instead of reading and processing the whole array column. The query `SELECT empty(arr) FROM table;` transforms to `SELECT arr.size0 = 0 FROM TABLE;`. + Can be optimized by enabling the [optimize_functions_to_subcolumns](../../operations/settings/settings.md#optimize-functions-to-subcolumns) setting. With `optimize_functions_to_subcolumns = 1` the function reads only [size0](../../sql-reference/data-types/array.md#array-size) subcolumn instead of reading and processing the whole array column. The query `SELECT empty(arr) FROM TABLE;` transforms to `SELECT arr.size0 = 0 FROM TABLE;`. The function also works for [strings](string-functions.md#empty) or [UUID](uuid-functions.md#empty). @@ -57,6 +57,7 @@ Checks whether the input array is non-empty. ``` sql notEmpty([x]) ``` + An array is considered non-empty if it contains at least one non-empty element. !!! note "Note" diff --git a/docs/ru/sql-reference/functions/array-functions.md b/docs/ru/sql-reference/functions/array-functions.md index 52fd63864ce..066d37a71f5 100644 --- a/docs/ru/sql-reference/functions/array-functions.md +++ b/docs/ru/sql-reference/functions/array-functions.md @@ -7,19 +7,89 @@ toc_title: "Массивы" ## empty {#function-empty} -Возвращает 1 для пустого массива, и 0 для непустого массива. -Тип результата - UInt8. -Функция также работает для строк. +Проверяет, является ли входной массив пустым. -Функцию можно оптимизировать, если включить настройку [optimize_functions_to_subcolumns](../../operations/settings/settings.md#optimize-functions-to-subcolumns). При `optimize_functions_to_subcolumns = 1` функция читает только подстолбец [size0](../../sql-reference/data-types/array.md#array-size) вместо чтения и обработки всего столбца массива. Запрос `SELECT empty(arr) FROM table` преобразуется к запросу `SELECT arr.size0 = 0 FROM TABLE`. +**Синтаксис** + +``` sql +empty([x]) +``` + +Массив считается пустым, если он содержит все пустые элементы. + +!!! note "Примечание" + Функцию можно оптимизировать, если включить настройку [optimize_functions_to_subcolumns](../../operations/settings/settings.md#optimize-functions-to-subcolumns). При `optimize_functions_to_subcolumns = 1` функция читает только подстолбец [size0](../../sql-reference/data-types/array.md#array-size) вместо чтения и обработки всего столбца массива. Запрос `SELECT empty(arr) FROM TABLE` преобразуется к запросу `SELECT arr.size0 = 0 FROM TABLE`. + +Функция также поддерживает работу с типами [String](string-functions.md#empty) и [UUID](uuid-functions.md#empty). + +**Параметры** + +- `[x]` — массив на входе функции. [Array](../data-types/array.md). + +**Возвращаемое значение** + +- Возвращает `1` для пустого массива или `0` — для непустого массива. + +Тип: [UInt8](../data-types/int-uint.md). + +**Пример** + +Запрос: + +```sql +SELECT empty([]); +``` + +Ответ: + +```text +┌─empty(array())─┐ +│ 1 │ +└────────────────┘ +``` ## notEmpty {#function-notempty} -Возвращает 0 для пустого массива, и 1 для непустого массива. -Тип результата - UInt8. -Функция также работает для строк. +Проверяет, является ли входной массив непустым. -Функцию можно оптимизировать, если включить настройку [optimize_functions_to_subcolumns](../../operations/settings/settings.md#optimize-functions-to-subcolumns). При `optimize_functions_to_subcolumns = 1` функция читает только подстолбец [size0](../../sql-reference/data-types/array.md#array-size) вместо чтения и обработки всего столбца массива. Запрос `SELECT notEmpty(arr) FROM table` преобразуется к запросу `SELECT arr.size0 != 0 FROM TABLE`. +**Синтаксис** + +``` sql +notEmpty([x]) +``` + +Массив считается непустым, если он содержит хотя бы один непустой элемент. + +!!! note "Примечание" + Функцию можно оптимизировать, если включить настройку [optimize_functions_to_subcolumns](../../operations/settings/settings.md#optimize-functions-to-subcolumns). При `optimize_functions_to_subcolumns = 1` функция читает только подстолбец [size0](../../sql-reference/data-types/array.md#array-size) вместо чтения и обработки всего столбца массива. Запрос `SELECT notEmpty(arr) FROM table` преобразуется к запросу `SELECT arr.size0 != 0 FROM TABLE`. + +Функция также поддерживает работу с типами [String](string-functions.md#notempty) и [UUID](uuid-functions.md#notempty). + +**Параметры** + +- `[x]` — массив на входе функции. [Array](../data-types/array.md). + +**Возвращаемое значение** + +- Возвращает `1` для непустого массива или `0` — для пустого массива. + +Тип: [UInt8](../data-types/int-uint.md). + +**Пример** + +Запрос: + +```sql +SELECT notEmpty([1,2]); +``` + +Результат: + +```text +┌─notEmpty([1, 2])─┐ +│ 1 │ +└──────────────────┘ +``` ## length {#array_functions-length} diff --git a/docs/ru/sql-reference/functions/uuid-functions.md b/docs/ru/sql-reference/functions/uuid-functions.md index f0017adbc8b..977446fdf06 100644 --- a/docs/ru/sql-reference/functions/uuid-functions.md +++ b/docs/ru/sql-reference/functions/uuid-functions.md @@ -35,6 +35,90 @@ SELECT * FROM t_uuid └──────────────────────────────────────┘ ``` +## empty {#empty} + +Проверяет, является ли входной UUID пустым. + +**Синтаксис** + +```sql +empty(UUID) +``` + +UUID считается пустым, если он содержит все нули (нулевой UUID). + +Функция также поддерживает работу с типами [Array](array-functions.md#empty) и [String](string-functions.md#empty). + +**Параметры** + +- `x` — UUID на входе функции. [UUID](../data-types/uuid.md). + +**Возвращаемое значение** + +- Возвращает `1` для пустого UUID или `0` — для непустого UUID. + +Тип: [UInt8](../data-types/int-uint.md). + +**Пример** + +Для генерации UUID-значений предназначена функция [generateUUIDv4](#uuid-function-generate). + +Запрос: + +```sql +SELECT empty(generateUUIDv4()); +``` + +Ответ: + +```text +┌─empty(generateUUIDv4())─┐ +│ 0 │ +└─────────────────────────┘ +``` + +## notEmpty {#notempty} + +Проверяет, является ли входной UUID непустым. + +**Синтаксис** + +```sql +notEmpty(UUID) +``` + +UUID считается пустым, если он содержит все нули (нулевой UUID). + +Функция также поддерживает работу с типами [Array](array-functions.md#notempty) и [String](string-functions.md#notempty). + +**Параметры** + +- `x` — UUID на входе функции. [UUID](../data-types/uuid.md). + +**Возвращаемое значение** + +- Возвращает `1` для непустого UUID или `0` — для пустого UUID. + +Тип: [UInt8](../data-types/int-uint.md). + +**Пример** + +Для генерации UUID-значений предназначена функция [generateUUIDv4](#uuid-function-generate). + +Запрос: + +```sql +SELECT notEmpty(generateUUIDv4()); +``` + +Результат: + +```text +┌─notEmpty(generateUUIDv4())─┐ +│ 1 │ +└────────────────────────────┘ +``` + ## toUUID (x) {#touuid-x} Преобразует значение типа String в тип UUID. From a549e29bd4152ac45a2023c739e2e685eb9c7be4 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 12 Aug 2021 14:42:51 +0300 Subject: [PATCH 520/599] Better --- src/Interpreters/InterpreterFactory.cpp | 8 +- .../InterpreterIntersectOrExcept.h | 39 --------- ...InterpreterSelectIntersectExceptQuery.cpp} | 77 ++++++++++------- .../InterpreterSelectIntersectExceptQuery.h | 45 ++++++++++ .../InterpreterSelectWithUnionQuery.cpp | 10 ++- .../SelectIntersectExceptQueryVisitor.cpp | 85 +++++++++++++++++++ .../SelectIntersectExceptQueryVisitor.h | 47 ++++++++++ src/Interpreters/executeQuery.cpp | 15 +++- ....cpp => ASTSelectIntersectExceptQuery.cpp} | 19 +++-- ...cept.h => ASTSelectIntersectExceptQuery.h} | 10 ++- src/Parsers/ParserQueryWithOutput.cpp | 4 +- ...p => ParserSelectIntersectExceptQuery.cpp} | 18 ++-- ...y.h => ParserSelectIntersectExceptQuery.h} | 2 +- src/Parsers/ParserUnionQueryElement.cpp | 5 +- .../QueryPlan/IntersectOrExceptStep.cpp | 13 +-- .../QueryPlan/IntersectOrExceptStep.h | 10 +-- .../Transforms/IntersectOrExceptTransform.cpp | 75 +++++++--------- .../Transforms/IntersectOrExceptTransform.h | 12 +-- ...02004_intersect_except_operators.reference | 25 ++++++ .../02004_intersect_except_operators.sql | 10 +++ 20 files changed, 363 insertions(+), 166 deletions(-) delete mode 100644 src/Interpreters/InterpreterIntersectOrExcept.h rename src/Interpreters/{InterpreterIntersectOrExcept.cpp => InterpreterSelectIntersectExceptQuery.cpp} (68%) create mode 100644 src/Interpreters/InterpreterSelectIntersectExceptQuery.h create mode 100644 src/Interpreters/SelectIntersectExceptQueryVisitor.cpp create mode 100644 src/Interpreters/SelectIntersectExceptQueryVisitor.h rename src/Parsers/{ASTIntersectOrExcept.cpp => ASTSelectIntersectExceptQuery.cpp} (76%) rename src/Parsers/{ASTIntersectOrExcept.h => ASTSelectIntersectExceptQuery.h} (55%) rename src/Parsers/{ParserIntersectOrExceptQuery.cpp => ParserSelectIntersectExceptQuery.cpp} (66%) rename src/Parsers/{ParserIntersectOrExceptQuery.h => ParserSelectIntersectExceptQuery.h} (79%) diff --git a/src/Interpreters/InterpreterFactory.cpp b/src/Interpreters/InterpreterFactory.cpp index e634c072841..e5b381b4d08 100644 --- a/src/Interpreters/InterpreterFactory.cpp +++ b/src/Interpreters/InterpreterFactory.cpp @@ -11,7 +11,7 @@ #include #include #include -#include +#include #include #include #include @@ -49,7 +49,7 @@ #include #include #include -#include +#include #include #include #include @@ -111,9 +111,9 @@ std::unique_ptr InterpreterFactory::get(ASTPtr & query, ContextMut ProfileEvents::increment(ProfileEvents::SelectQuery); return std::make_unique(query, context, options); } - else if (query->as()) + else if (query->as()) { - return std::make_unique(query, context); + return std::make_unique(query, context, options); } else if (query->as()) { diff --git a/src/Interpreters/InterpreterIntersectOrExcept.h b/src/Interpreters/InterpreterIntersectOrExcept.h deleted file mode 100644 index 359be05db8b..00000000000 --- a/src/Interpreters/InterpreterIntersectOrExcept.h +++ /dev/null @@ -1,39 +0,0 @@ -#pragma once - -#include -#include -#include -#include - - -namespace DB -{ - -class Context; -class InterpreterSelectQuery; -class QueryPlan; - -class InterpreterIntersectOrExcept : public IInterpreter -{ -public: - InterpreterIntersectOrExcept(const ASTPtr & query_ptr_, ContextPtr context_); - - BlockIO execute() override; - -private: - String getName() const { return "IntersectOrExcept"; } - - Block getCommonHeader(const Blocks & headers) const; - - std::unique_ptr - buildCurrentChildInterpreter(const ASTPtr & ast_ptr_); - - void buildQueryPlan(QueryPlan & query_plan); - - ContextPtr context; - Block result_header; - std::vector> nested_interpreters; - ASTIntersectOrExcept::Operators operators; -}; - -} diff --git a/src/Interpreters/InterpreterIntersectOrExcept.cpp b/src/Interpreters/InterpreterSelectIntersectExceptQuery.cpp similarity index 68% rename from src/Interpreters/InterpreterIntersectOrExcept.cpp rename to src/Interpreters/InterpreterSelectIntersectExceptQuery.cpp index 52dcb24ed27..9e24dd6e6a0 100644 --- a/src/Interpreters/InterpreterIntersectOrExcept.cpp +++ b/src/Interpreters/InterpreterSelectIntersectExceptQuery.cpp @@ -1,8 +1,8 @@ #include #include -#include +#include #include -#include +#include #include #include #include @@ -19,27 +19,7 @@ namespace ErrorCodes extern const int INTERSECT_OR_EXCEPT_RESULT_STRUCTURES_MISMATCH; } -InterpreterIntersectOrExcept::InterpreterIntersectOrExcept(const ASTPtr & query_ptr, ContextPtr context_) - : context(Context::createCopy(context_)) -{ - ASTIntersectOrExcept * ast = query_ptr->as(); - operators = ast->list_of_operators; - - auto children = ast->list_of_selects->children; - size_t num_children = children.size(); - nested_interpreters.resize(num_children); - - for (size_t i = 0; i < num_children; ++i) - nested_interpreters[i] = buildCurrentChildInterpreter(children.at(i)); - - Blocks headers(num_children); - for (size_t query_num = 0; query_num < num_children; ++query_num) - headers[query_num] = nested_interpreters[query_num]->getSampleBlock(); - - result_header = getCommonHeader(headers); -} - -Block InterpreterIntersectOrExcept::getCommonHeader(const Blocks & headers) const +static Block getCommonHeader(const Blocks & headers) { size_t num_selects = headers.size(); Block common_header = headers.front(); @@ -49,8 +29,8 @@ Block InterpreterIntersectOrExcept::getCommonHeader(const Blocks & headers) cons { if (headers[query_num].columns() != num_columns) throw Exception(ErrorCodes::INTERSECT_OR_EXCEPT_RESULT_STRUCTURES_MISMATCH, - "Different number of columns in {} elements:\n {} \nand\n {}", - getName(), common_header.dumpNames(), headers[query_num].dumpNames()); + "Different number of columns in IntersectExceptQuery elements:\n {} \nand\n {}", + common_header.dumpNames(), headers[query_num].dumpNames()); } std::vector columns(num_selects); @@ -66,16 +46,53 @@ Block InterpreterIntersectOrExcept::getCommonHeader(const Blocks & headers) cons return common_header; } +InterpreterSelectIntersectExceptQuery::InterpreterSelectIntersectExceptQuery( + const ASTPtr & query_ptr_, + ContextPtr context_, + const SelectQueryOptions & options_) + : IInterpreterUnionOrSelectQuery(query_ptr_->clone(), context_, options_) +{ + ASTSelectIntersectExceptQuery * ast = query_ptr->as(); + final_operator = ast->final_operator; + + const auto & children = ast->children[0]->children; + size_t num_children = children.size(); + + /// AST must have been changed by the visitor. + if (final_operator == Operator::UNKNOWN || num_children != 2) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "SelectIntersectExceptyQuery has not been normalized (number of children: {})", + num_children); + + nested_interpreters.resize(num_children); + + for (size_t i = 0; i < num_children; ++i) + nested_interpreters[i] = buildCurrentChildInterpreter(children.at(i)); + + Blocks headers(num_children); + for (size_t query_num = 0; query_num < num_children; ++query_num) + headers[query_num] = nested_interpreters[query_num]->getSampleBlock(); + + result_header = getCommonHeader(headers); +} + std::unique_ptr -InterpreterIntersectOrExcept::buildCurrentChildInterpreter(const ASTPtr & ast_ptr_) +InterpreterSelectIntersectExceptQuery::buildCurrentChildInterpreter(const ASTPtr & ast_ptr_) { if (ast_ptr_->as()) return std::make_unique(ast_ptr_, context, SelectQueryOptions()); - else + + if (ast_ptr_->as()) return std::make_unique(ast_ptr_, context, SelectQueryOptions()); + + if (ast_ptr_->as()) + return std::make_unique(ast_ptr_, context, SelectQueryOptions()); + + // if (ast_ptr_->as()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected query: {}", ast_ptr_->getID()); } -void InterpreterIntersectOrExcept::buildQueryPlan(QueryPlan & query_plan) +void InterpreterSelectIntersectExceptQuery::buildQueryPlan(QueryPlan & query_plan) { size_t num_plans = nested_interpreters.size(); std::vector> plans(num_plans); @@ -101,11 +118,11 @@ void InterpreterIntersectOrExcept::buildQueryPlan(QueryPlan & query_plan) } auto max_threads = context->getSettingsRef().max_threads; - auto step = std::make_unique(std::move(data_streams), operators, max_threads); + auto step = std::make_unique(std::move(data_streams), final_operator, max_threads); query_plan.unitePlans(std::move(step), std::move(plans)); } -BlockIO InterpreterIntersectOrExcept::execute() +BlockIO InterpreterSelectIntersectExceptQuery::execute() { BlockIO res; diff --git a/src/Interpreters/InterpreterSelectIntersectExceptQuery.h b/src/Interpreters/InterpreterSelectIntersectExceptQuery.h new file mode 100644 index 00000000000..9cbde055b0b --- /dev/null +++ b/src/Interpreters/InterpreterSelectIntersectExceptQuery.h @@ -0,0 +1,45 @@ +#pragma once + +#include +#include +#include +#include +#include + + +namespace DB +{ + +class Context; +class InterpreterSelectQuery; +class QueryPlan; + +class InterpreterSelectIntersectExceptQuery : public IInterpreterUnionOrSelectQuery +{ +using Operator = ASTSelectIntersectExceptQuery::Operator; + +public: + InterpreterSelectIntersectExceptQuery( + const ASTPtr & query_ptr_, + ContextPtr context_, + const SelectQueryOptions & options_); + + BlockIO execute() override; + + Block getSampleBlock() { return result_header; } + +private: + static String getName() { return "SelectIntersectExceptQuery"; } + + std::unique_ptr + buildCurrentChildInterpreter(const ASTPtr & ast_ptr_); + + void buildQueryPlan(QueryPlan & query_plan) override; + + void ignoreWithTotals() override {} + + std::vector> nested_interpreters; + Operator final_operator; +}; + +} diff --git a/src/Interpreters/InterpreterSelectWithUnionQuery.cpp b/src/Interpreters/InterpreterSelectWithUnionQuery.cpp index 3cf4a905d38..cd06f51cb12 100644 --- a/src/Interpreters/InterpreterSelectWithUnionQuery.cpp +++ b/src/Interpreters/InterpreterSelectWithUnionQuery.cpp @@ -2,8 +2,10 @@ #include #include #include +#include #include #include +#include #include #include #include @@ -208,8 +210,10 @@ InterpreterSelectWithUnionQuery::buildCurrentChildInterpreter(const ASTPtr & ast { if (ast_ptr_->as()) return std::make_unique(ast_ptr_, context, options, current_required_result_column_names); - else + else if (ast_ptr_->as()) return std::make_unique(ast_ptr_, context, options, current_required_result_column_names); + else + return std::make_unique(ast_ptr_, context, options); } InterpreterSelectWithUnionQuery::~InterpreterSelectWithUnionQuery() = default; @@ -225,10 +229,14 @@ Block InterpreterSelectWithUnionQuery::getSampleBlock(const ASTPtr & query_ptr_, } if (is_subquery) + { return cache[key] = InterpreterSelectWithUnionQuery(query_ptr_, context_, SelectQueryOptions().subquery().analyze()).getSampleBlock(); + } else + { return cache[key] = InterpreterSelectWithUnionQuery(query_ptr_, context_, SelectQueryOptions().analyze()).getSampleBlock(); + } } diff --git a/src/Interpreters/SelectIntersectExceptQueryVisitor.cpp b/src/Interpreters/SelectIntersectExceptQueryVisitor.cpp new file mode 100644 index 00000000000..a404bf3da40 --- /dev/null +++ b/src/Interpreters/SelectIntersectExceptQueryVisitor.cpp @@ -0,0 +1,85 @@ +#include +#include +#include + + +namespace DB +{ + +void SelectIntersectExceptQueryMatcher::visit(ASTPtr & ast, Data & data) +{ + if (auto * select_intersect_except = ast->as()) + { + std::cerr << "\n\nSelectIntersectExceptVisitor BEFORE:\n" << ast->dumpTree() << std::endl; + data.initialize(select_intersect_except); + visit(*select_intersect_except, data); + std::cerr << "\n\nSelectIntersectExceptVisitor AFTER:\n" << ast->dumpTree() << std::endl; + } +} + +void SelectIntersectExceptQueryMatcher::visit(ASTSelectIntersectExceptQuery & ast, Data & data) +{ + /* Example: select 1 intersect select 1 intsect select 1 intersect select 1 intersect select 1; + * + * --SelectIntersectExceptQuery --SelectIntersectExceptQuery + * ---expressionlist ---ExpressionList + * ----SelectQuery ----SelectIntersectExceptQuery + * ----SelectQuery ------ExpressionList + * ----SelectQuery ---> -------SelectIntersectExceptQuery + * ----SelectQuery --------ExpressionList + * ---------SelectQuery + * ---------SelectQuery + * -------SelectQuery + * ----SelectQuery + **/ + + auto & selects = data.reversed_list_of_selects; + + if (selects.empty()) + return; + + const auto left = selects.back(); + selects.pop_back(); + const auto right = selects.back(); + selects.pop_back(); + + auto & operators = data.reversed_list_of_operators; + const auto current_operator = operators.back(); + operators.pop_back(); + + auto list_node = std::make_shared(); + list_node->children = {left, right}; + + if (selects.empty()) + { + ast.final_operator = current_operator; + ast.children = {std::move(list_node)}; + } + else + { + auto select_intersect_except = std::make_shared(); + select_intersect_except->final_operator = {current_operator}; + select_intersect_except->children.emplace_back(std::move(list_node)); + + selects.emplace_back(std::move(select_intersect_except)); + } + + visit(ast, data); +} + +// void SelectIntersectExceptQueryVisitor::visit(ASTSelectWithUnionQuery & ast, Data & data) +// { +// auto & union_modes = ast.list_of_modes; +// ASTs selects; +// auto & select_list = ast.list_of_selects->children; +// +// +// // reverse children list +// std::reverse(selects.begin(), selects.end()); +// +// ast.is_normalized = true; +// ast.union_mode = ASTSelectWithUnionQuery::Mode::ALL; +// +// ast.list_of_selects->children = std::move(selects); +// } +} diff --git a/src/Interpreters/SelectIntersectExceptQueryVisitor.h b/src/Interpreters/SelectIntersectExceptQueryVisitor.h new file mode 100644 index 00000000000..58f3071972f --- /dev/null +++ b/src/Interpreters/SelectIntersectExceptQueryVisitor.h @@ -0,0 +1,47 @@ +#pragma once + +#include + +#include +#include + +#include +#include + + +namespace DB +{ + +class ASTFunction; + +class SelectIntersectExceptQueryMatcher +{ +public: + struct Data + { + Data() = default; + + void initialize(const ASTSelectIntersectExceptQuery * select_intersect_except) + { + reversed_list_of_selects = select_intersect_except->list_of_selects->clone()->children; + reversed_list_of_operators = select_intersect_except->list_of_operators; + + std::reverse(reversed_list_of_selects.begin(), reversed_list_of_selects.end()); + std::reverse(reversed_list_of_operators.begin(), reversed_list_of_operators.end()); + } + + ASTs reversed_list_of_selects; + ASTSelectIntersectExceptQuery::Operators reversed_list_of_operators; + }; + + static bool needChildVisit(const ASTPtr &, const ASTPtr &) { return true; } + + static void visit(ASTPtr & ast, Data &); + static void visit(ASTSelectIntersectExceptQuery &, Data &); + // static void visit(ASTSelectWithUnionQuery &, Data &); +}; + +/// Visit children first. +using SelectIntersectExceptQueryVisitor + = InDepthNodeVisitor; +} diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 1b59f3bc7df..839447a90f7 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -46,6 +46,7 @@ #include #include #include +#include #include #include @@ -490,9 +491,16 @@ static std::tuple executeQueryImpl( ApplyWithGlobalVisitor().visit(ast); } - /// Normalize SelectWithUnionQuery - NormalizeSelectWithUnionQueryVisitor::Data data{context->getSettingsRef().union_default_mode}; - NormalizeSelectWithUnionQueryVisitor{data}.visit(ast); + { + /// Normalize SelectWithUnionQuery + NormalizeSelectWithUnionQueryVisitor::Data data{context->getSettingsRef().union_default_mode}; + NormalizeSelectWithUnionQueryVisitor{data}.visit(ast); + } + + { + SelectIntersectExceptQueryVisitor::Data data; + SelectIntersectExceptQueryVisitor{data}.visit(ast); + } /// Check the limits. checkASTSizeLimits(*ast, settings); @@ -532,6 +540,7 @@ static std::tuple executeQueryImpl( /// reset Input callbacks if query is not INSERT SELECT context->resetInputCallbacks(); + std::cerr << "\n\nAST: " << ast->dumpTree() << std::endl; auto interpreter = InterpreterFactory::get(ast, context, SelectQueryOptions(stage).setInternal(internal)); std::shared_ptr quota; diff --git a/src/Parsers/ASTIntersectOrExcept.cpp b/src/Parsers/ASTSelectIntersectExceptQuery.cpp similarity index 76% rename from src/Parsers/ASTIntersectOrExcept.cpp rename to src/Parsers/ASTSelectIntersectExceptQuery.cpp index 33ffb76c2f7..26fd9353d5b 100644 --- a/src/Parsers/ASTIntersectOrExcept.cpp +++ b/src/Parsers/ASTSelectIntersectExceptQuery.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include @@ -6,20 +6,25 @@ namespace DB { -ASTPtr ASTIntersectOrExcept::clone() const +ASTPtr ASTSelectIntersectExceptQuery::clone() const { - auto res = std::make_shared(*this); - res->children.clear(); + auto res = std::make_shared(*this); + + res->children.clear(); + for (const auto & child : children) + res->children.push_back(child->clone()); + + if (res->list_of_selects) + res->list_of_selects = list_of_selects->clone(); - res->list_of_selects = list_of_selects->clone(); - res->children.push_back(res->list_of_selects); res->list_of_operators = list_of_operators; + res->final_operator = final_operator; cloneOutputOptions(*res); return res; } -void ASTIntersectOrExcept::formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const +void ASTSelectIntersectExceptQuery::formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const { std::string indent_str = settings.one_line ? "" : std::string(4 * frame.indent, ' '); diff --git a/src/Parsers/ASTIntersectOrExcept.h b/src/Parsers/ASTSelectIntersectExceptQuery.h similarity index 55% rename from src/Parsers/ASTIntersectOrExcept.h rename to src/Parsers/ASTSelectIntersectExceptQuery.h index 9adfdedc497..8fc5756e370 100644 --- a/src/Parsers/ASTIntersectOrExcept.h +++ b/src/Parsers/ASTSelectIntersectExceptQuery.h @@ -6,17 +6,20 @@ namespace DB { -class ASTIntersectOrExcept : public ASTQueryWithOutput +class ASTSelectIntersectExceptQuery : public ASTQueryWithOutput { public: - String getID(char) const override { return "IntersectExceptQuery"; } + String getID(char) const override { return "SelectIntersectExceptQuery"; } ASTPtr clone() const override; void formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override; + const char * getQueryKindString() const override { return "SelectIntersectExcept"; } + enum class Operator { + UNKNOWN, INTERSECT, EXCEPT }; @@ -25,6 +28,9 @@ public: ASTPtr list_of_selects; Operators list_of_operators; + + /// Final operator after applying visitor. + Operator final_operator = Operator::UNKNOWN; }; } diff --git a/src/Parsers/ParserQueryWithOutput.cpp b/src/Parsers/ParserQueryWithOutput.cpp index d7d87cac9b9..e2ab8a84cc1 100644 --- a/src/Parsers/ParserQueryWithOutput.cpp +++ b/src/Parsers/ParserQueryWithOutput.cpp @@ -7,7 +7,7 @@ #include #include #include -#include +#include #include #include #include @@ -31,7 +31,6 @@ namespace DB bool ParserQueryWithOutput::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { ParserShowTablesQuery show_tables_p; - ParserIntersectOrExceptQuery intersect_except_p; ParserSelectWithUnionQuery select_p; ParserTablePropertiesQuery table_p; ParserDescribeTableQuery describe_table_p; @@ -55,7 +54,6 @@ bool ParserQueryWithOutput::parseImpl(Pos & pos, ASTPtr & node, Expected & expec bool parsed = explain_p.parse(pos, query, expected) - || intersect_except_p.parse(pos, query, expected) || 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) diff --git a/src/Parsers/ParserIntersectOrExceptQuery.cpp b/src/Parsers/ParserSelectIntersectExceptQuery.cpp similarity index 66% rename from src/Parsers/ParserIntersectOrExceptQuery.cpp rename to src/Parsers/ParserSelectIntersectExceptQuery.cpp index ef6d68f8534..b56598166c6 100644 --- a/src/Parsers/ParserIntersectOrExceptQuery.cpp +++ b/src/Parsers/ParserSelectIntersectExceptQuery.cpp @@ -1,9 +1,9 @@ -#include +#include #include #include #include -#include -#include +#include +#include #include #include @@ -11,18 +11,18 @@ namespace DB { -bool ParserIntersectOrExceptQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) +bool ParserSelectIntersectExceptQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { ParserKeyword intersect_keyword("INTERSECT"); ParserKeyword except_keyword("EXCEPT"); ASTs elements; - ASTIntersectOrExcept::Operators operators; + ASTSelectIntersectExceptQuery::Operators operators; auto parse_element = [&]() -> bool { ASTPtr element; - if (!ParserSelectWithUnionQuery().parse(pos, element, expected) && !ParserSubquery().parse(pos, element, expected)) + if (!ParserSelectQuery().parse(pos, element, expected) && !ParserSubquery().parse(pos, element, expected)) return false; elements.push_back(element); @@ -36,11 +36,11 @@ bool ParserIntersectOrExceptQuery::parseImpl(Pos & pos, ASTPtr & node, Expected if (!except_keyword.ignore(pos)) return false; - operators.emplace_back(ASTIntersectOrExcept::Operator::EXCEPT); + operators.emplace_back(ASTSelectIntersectExceptQuery::Operator::EXCEPT); return true; } - operators.emplace_back(ASTIntersectOrExcept::Operator::INTERSECT); + operators.emplace_back(ASTSelectIntersectExceptQuery::Operator::INTERSECT); return true; }; @@ -56,7 +56,7 @@ bool ParserIntersectOrExceptQuery::parseImpl(Pos & pos, ASTPtr & node, Expected auto list_node = std::make_shared(); list_node->children = std::move(elements); - auto intersect_or_except_ast = std::make_shared(); + auto intersect_or_except_ast = std::make_shared(); node = intersect_or_except_ast; intersect_or_except_ast->list_of_selects = list_node; diff --git a/src/Parsers/ParserIntersectOrExceptQuery.h b/src/Parsers/ParserSelectIntersectExceptQuery.h similarity index 79% rename from src/Parsers/ParserIntersectOrExceptQuery.h rename to src/Parsers/ParserSelectIntersectExceptQuery.h index d8ba82ba053..e01785113a8 100644 --- a/src/Parsers/ParserIntersectOrExceptQuery.h +++ b/src/Parsers/ParserSelectIntersectExceptQuery.h @@ -4,7 +4,7 @@ namespace DB { -class ParserIntersectOrExceptQuery : public IParserBase +class ParserSelectIntersectExceptQuery : public IParserBase { protected: const char * getName() const override { return "INTERSECT or EXCEPT"; } diff --git a/src/Parsers/ParserUnionQueryElement.cpp b/src/Parsers/ParserUnionQueryElement.cpp index efd022e6362..5abbce25930 100644 --- a/src/Parsers/ParserUnionQueryElement.cpp +++ b/src/Parsers/ParserUnionQueryElement.cpp @@ -2,6 +2,7 @@ #include #include #include +#include #include @@ -10,7 +11,9 @@ namespace DB bool ParserUnionQueryElement::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { - if (!ParserSubquery().parse(pos, node, expected) && !ParserSelectQuery().parse(pos, node, expected)) + if (!ParserSubquery().parse(pos, node, expected) + && !ParserSelectIntersectExceptQuery().parse(pos, node, expected) + && !ParserSelectQuery().parse(pos, node, expected)) return false; if (const auto * ast_subquery = node->as()) diff --git a/src/Processors/QueryPlan/IntersectOrExceptStep.cpp b/src/Processors/QueryPlan/IntersectOrExceptStep.cpp index 76f496ba47c..e61afb5ba2a 100644 --- a/src/Processors/QueryPlan/IntersectOrExceptStep.cpp +++ b/src/Processors/QueryPlan/IntersectOrExceptStep.cpp @@ -1,7 +1,8 @@ +#include + #include #include #include -#include #include #include #include @@ -11,22 +12,22 @@ namespace DB { -Block IntersectOrExceptStep::checkHeaders(const DataStreams & input_streams_) const +static Block checkHeaders(const DataStreams & input_streams_) { if (input_streams_.empty()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot perform intersect/except on empty set of query plan steps"); Block res = input_streams_.front().header; for (const auto & stream : input_streams_) - assertBlocksHaveEqualStructure(stream.header, res, "IntersectExceptStep"); + assertBlocksHaveEqualStructure(stream.header, res, "IntersectOrExceptStep"); return res; } IntersectOrExceptStep::IntersectOrExceptStep( - DataStreams input_streams_ , const Operators & operators_ , size_t max_threads_) + DataStreams input_streams_ , Operator operator_ , size_t max_threads_) : header(checkHeaders(input_streams_)) - , operators(operators_) + , current_operator(operator_) , max_threads(max_threads_) { input_streams = std::move(input_streams_); @@ -67,7 +68,7 @@ QueryPipelinePtr IntersectOrExceptStep::updatePipeline(QueryPipelines pipelines, } *pipeline = QueryPipeline::unitePipelines(std::move(pipelines), max_threads); - pipeline->addTransform(std::make_shared(header, operators)); + pipeline->addTransform(std::make_shared(header, current_operator)); processors = collector.detachProcessors(); return pipeline; diff --git a/src/Processors/QueryPlan/IntersectOrExceptStep.h b/src/Processors/QueryPlan/IntersectOrExceptStep.h index 914a7dce197..002f1b1570c 100644 --- a/src/Processors/QueryPlan/IntersectOrExceptStep.h +++ b/src/Processors/QueryPlan/IntersectOrExceptStep.h @@ -1,6 +1,6 @@ #pragma once #include -#include +#include namespace DB @@ -8,11 +8,11 @@ namespace DB class IntersectOrExceptStep : public IQueryPlanStep { -using Operators = ASTIntersectOrExcept::Operators; +using Operator = ASTSelectIntersectExceptQuery::Operator; public: /// max_threads is used to limit the number of threads for result pipeline. - IntersectOrExceptStep(DataStreams input_streams_, const Operators & operators_, size_t max_threads_ = 0); + IntersectOrExceptStep(DataStreams input_streams_, Operator operators_, size_t max_threads_ = 0); String getName() const override { return "IntersectOrExcept"; } @@ -21,10 +21,8 @@ public: void describePipeline(FormatSettings & settings) const override; private: - Block checkHeaders(const DataStreams & input_streams_) const; - Block header; - Operators operators; + Operator current_operator; size_t max_threads; Processors processors; }; diff --git a/src/Processors/Transforms/IntersectOrExceptTransform.cpp b/src/Processors/Transforms/IntersectOrExceptTransform.cpp index 68d5f6a2e5e..b16032bde8e 100644 --- a/src/Processors/Transforms/IntersectOrExceptTransform.cpp +++ b/src/Processors/Transforms/IntersectOrExceptTransform.cpp @@ -4,15 +4,14 @@ namespace DB { -/* - * There are always at least two inputs. Number of operators is always number of inputs minus 1. - * input1 {operator1} input2 {operator2} input3 ... -**/ -IntersectOrExceptTransform::IntersectOrExceptTransform(const Block & header_, const Operators & operators_) - : IProcessor(InputPorts(operators_.size() + 1, header_), {header_}) - , operators(operators_) - , first_input(inputs.begin()) - , second_input(std::next(inputs.begin())) +namespace ErrorCodes +{ + extern const int SET_SIZE_LIMIT_EXCEEDED; +} + +IntersectOrExceptTransform::IntersectOrExceptTransform(const Block & header_, Operator operator_) + : IProcessor(InputPorts(2, header_), {header_}) + , current_operator(operator_) { const Names & columns = header_.getNames(); size_t num_columns = columns.empty() ? header_.columns() : columns.size(); @@ -46,53 +45,33 @@ IntersectOrExceptTransform::Status IntersectOrExceptTransform::prepare() return Status::PortFull; } + if (current_output_chunk) + { + output.push(std::move(current_output_chunk)); + } + if (finished_second_input) { - if (first_input->isFinished() || (use_accumulated_input && !current_input_chunk)) + if (inputs.front().isFinished()) { - std::advance(second_input, 1); - - if (second_input == inputs.end()) - { - if (current_output_chunk) - { - output.push(std::move(current_output_chunk)); - } - - output.finish(); - return Status::Finished; - } - else - { - use_accumulated_input = true; - data.reset(); - finished_second_input = false; - ++current_operator_pos; - } + output.finish(); + return Status::Finished; } } - else if (second_input->isFinished()) + else if (inputs.back().isFinished()) { finished_second_input = true; } if (!has_input) { - if (finished_second_input && use_accumulated_input) - { - current_input_chunk = std::move(current_output_chunk); - } - else - { - InputPort & input = finished_second_input ? *first_input : *second_input; + InputPort & input = finished_second_input ? inputs.front() : inputs.back(); - input.setNeeded(); - if (!input.hasData()) - return Status::NeedData; - - current_input_chunk = input.pull(); - } + input.setNeeded(); + if (!input.hasData()) + return Status::NeedData; + current_input_chunk = input.pull(); has_input = true; } @@ -136,7 +115,7 @@ size_t IntersectOrExceptTransform::buildFilter( for (size_t i = 0; i < rows; ++i) { auto find_result = state.findKey(method.data, i, variants.string_pool); - filter[i] = operators[current_operator_pos] == ASTIntersectOrExcept::Operator::EXCEPT ? !find_result.isFound() : find_result.isFound(); + filter[i] = current_operator == ASTSelectIntersectExceptQuery::Operator::EXCEPT ? !find_result.isFound() : find_result.isFound(); if (filter[i]) ++new_rows_num; } @@ -193,10 +172,11 @@ void IntersectOrExceptTransform::filter(Chunk & chunk) if (data->empty()) data->init(SetVariants::chooseMethod(column_ptrs, key_sizes)); - IColumn::Filter filter(num_rows); - size_t new_rows_num = 0; + + IColumn::Filter filter(num_rows); auto & data_set = *data; + switch (data->type) { case SetVariants::Type::EMPTY: @@ -209,6 +189,9 @@ void IntersectOrExceptTransform::filter(Chunk & chunk) #undef M } + if (!new_rows_num) + return; + for (auto & column : columns) column = column->filter(filter, -1); diff --git a/src/Processors/Transforms/IntersectOrExceptTransform.h b/src/Processors/Transforms/IntersectOrExceptTransform.h index 6d0c3516d5d..da1fa6a119e 100644 --- a/src/Processors/Transforms/IntersectOrExceptTransform.h +++ b/src/Processors/Transforms/IntersectOrExceptTransform.h @@ -3,7 +3,7 @@ #include #include #include -#include +#include namespace DB @@ -11,10 +11,10 @@ namespace DB class IntersectOrExceptTransform : public IProcessor { -using Operators = ASTIntersectOrExcept::Operators; +using Operator = ASTSelectIntersectExceptQuery::Operator; public: - IntersectOrExceptTransform(const Block & header_, const Operators & operators); + IntersectOrExceptTransform(const Block & header_, Operator operators); String getName() const override { return "IntersectOrExcept"; } @@ -24,10 +24,7 @@ protected: void work() override; private: - Operators operators; - InputPorts::iterator first_input; - InputPorts::iterator second_input; - size_t current_operator_pos = 0; + Operator current_operator; ColumnNumbers key_columns_pos; std::optional data; @@ -36,7 +33,6 @@ private: Chunk current_input_chunk; Chunk current_output_chunk; - bool use_accumulated_input = false; bool finished_second_input = false; bool has_input = false; diff --git a/tests/queries/0_stateless/02004_intersect_except_operators.reference b/tests/queries/0_stateless/02004_intersect_except_operators.reference index d17216a5ec4..c3272a5d574 100644 --- a/tests/queries/0_stateless/02004_intersect_except_operators.reference +++ b/tests/queries/0_stateless/02004_intersect_except_operators.reference @@ -32,6 +32,16 @@ select 1 intersect select 1 except select 1; select 1 intersect select 1 except select 2 intersect select 1 except select 3 intersect select 1; 1 select 1 intersect select 1 except select 2 intersect select 1 except select 3 intersect select 2; +select number from numbers(10) except select 5; +0 +1 +2 +3 +4 +6 +7 +8 +9 select number from numbers(100) intersect select number from numbers(20, 60) except select number from numbers(30, 20) except select number from numbers(60, 20); 20 21 @@ -53,3 +63,18 @@ select number from numbers(100) intersect select number from numbers(20, 60) exc 57 58 59 +with (select number from numbers(10) intersect select 5) as a select a * 10; +50 +select count() from (select number from numbers(10) except select 5); +9 +select count() from (select number from numbers(1000000) intersect select number from numbers(200000, 600000)); +600000 +select count() from (select number from numbers(100) intersect select number from numbers(20, 60) except select number from numbers(30, 20) except select number from numbers(60, 20)); +20 +select count() from (select number from numbers(1000000) intersect select number from numbers(200000, 600000) except select number from numbers(300000, 200000) except select number from numbers(600000, 200000)); +200000 +select 1 union all select 1 intersect select 1; +1 +1 +select 1 union all select 1 intersect select 2; +1 diff --git a/tests/queries/0_stateless/02004_intersect_except_operators.sql b/tests/queries/0_stateless/02004_intersect_except_operators.sql index 971aa262070..722670732ac 100644 --- a/tests/queries/0_stateless/02004_intersect_except_operators.sql +++ b/tests/queries/0_stateless/02004_intersect_except_operators.sql @@ -15,4 +15,14 @@ select 1 intersect select 1 except select 1; select 1 intersect select 1 except select 2 intersect select 1 except select 3 intersect select 1; select 1 intersect select 1 except select 2 intersect select 1 except select 3 intersect select 2; +select number from numbers(10) except select 5; select number from numbers(100) intersect select number from numbers(20, 60) except select number from numbers(30, 20) except select number from numbers(60, 20); + +with (select number from numbers(10) intersect select 5) as a select a * 10; +select count() from (select number from numbers(10) except select 5); +select count() from (select number from numbers(1000000) intersect select number from numbers(200000, 600000)); +select count() from (select number from numbers(100) intersect select number from numbers(20, 60) except select number from numbers(30, 20) except select number from numbers(60, 20)); +select count() from (select number from numbers(1000000) intersect select number from numbers(200000, 600000) except select number from numbers(300000, 200000) except select number from numbers(600000, 200000)); + +select 1 union all select 1 intersect select 1; +select 1 union all select 1 intersect select 2; From 74db1eafd5c68c0eb2cbfaa028ebf66d26675678 Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Thu, 12 Aug 2021 16:43:56 +0300 Subject: [PATCH 521/599] Add ru docs for uuid and array functions. --- docs/ru/sql-reference/functions/uuid-functions.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/ru/sql-reference/functions/uuid-functions.md b/docs/ru/sql-reference/functions/uuid-functions.md index 977446fdf06..0d534a2d7ce 100644 --- a/docs/ru/sql-reference/functions/uuid-functions.md +++ b/docs/ru/sql-reference/functions/uuid-functions.md @@ -47,7 +47,7 @@ empty(UUID) UUID считается пустым, если он содержит все нули (нулевой UUID). -Функция также поддерживает работу с типами [Array](array-functions.md#empty) и [String](string-functions.md#empty). +Функция также поддерживает работу с типами [Array](array-functions.md#function-empty) и [String](string-functions.md#empty). **Параметры** @@ -89,7 +89,7 @@ notEmpty(UUID) UUID считается пустым, если он содержит все нули (нулевой UUID). -Функция также поддерживает работу с типами [Array](array-functions.md#notempty) и [String](string-functions.md#notempty). +Функция также поддерживает работу с типами [Array](array-functions.md#function-notempty) и [String](string-functions.md#function-notempty). **Параметры** From 276d757c67676f25534d7113f0283c8505950932 Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Thu, 12 Aug 2021 16:56:00 +0300 Subject: [PATCH 522/599] Add ru docs for uuid and array functions. --- docs/ru/sql-reference/functions/string-functions.md | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/docs/ru/sql-reference/functions/string-functions.md b/docs/ru/sql-reference/functions/string-functions.md index 75a4af84297..5c2b3226219 100644 --- a/docs/ru/sql-reference/functions/string-functions.md +++ b/docs/ru/sql-reference/functions/string-functions.md @@ -17,6 +17,8 @@ empty(x) Строка считается непустой, если содержит хотя бы один байт, пусть даже это пробел или нулевой байт. +Функция также поддерживает работу с типами [Array](array-functions.md#function-empty) и [UUID](uuid-functions.md#empty). + **Параметры** - `x` — Входная строка. [String](../data-types/string.md). @@ -55,6 +57,8 @@ notEmpty(x) Строка считается непустой, если содержит хотя бы один байт, пусть даже это пробел или нулевой байт. +Функция также поддерживает работу с типами [Array](array-functions.md#function-notempty) и [UUID](uuid-functions.md#notempty). + **Параметры** - `x` — Входная строка. [String](../data-types/string.md). From 931bed02b0e2b0110c5188c8b740bd0ccc9188d3 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Thu, 12 Aug 2021 17:47:46 +0300 Subject: [PATCH 523/599] Update Settings.h --- src/Core/Settings.h | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 3c30c6b0b1d..e1bd1d29153 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -437,7 +437,6 @@ class IColumn; M(String, function_implementation, "", "Choose function implementation for specific target or variant (experimental). If empty enable all of them.", 0) \ M(Bool, allow_experimental_geo_types, false, "Allow geo data types such as Point, Ring, Polygon, MultiPolygon", 0) \ M(Bool, data_type_default_nullable, false, "Data types without NULL or NOT NULL will make Nullable", 0) \ - /** Temporarily set to true, to check how tests will feel.*/ \ M(Bool, cast_keep_nullable, false, "CAST operator keep Nullable for result data type", 0) \ M(Bool, alter_partition_verbose_result, false, "Output information about affected parts. Currently works only for FREEZE and ATTACH commands.", 0) \ M(Bool, allow_experimental_database_materialized_mysql, false, "Allow to create database with Engine=MaterializedMySQL(...).", 0) \ From 27cb83f5a74aabbd6f2e62b892068e0e91bf123b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 12 Aug 2021 17:08:39 +0200 Subject: [PATCH 524/599] Get today datetime in UTC --- .../queries/0_stateless/01236_graphite_mt.sql | 36 +++++++++---------- 1 file changed, 18 insertions(+), 18 deletions(-) diff --git a/tests/queries/0_stateless/01236_graphite_mt.sql b/tests/queries/0_stateless/01236_graphite_mt.sql index d5289364d32..ccf7c066e75 100644 --- a/tests/queries/0_stateless/01236_graphite_mt.sql +++ b/tests/queries/0_stateless/01236_graphite_mt.sql @@ -9,28 +9,28 @@ SET joined_subquery_requires_alias = 0; INSERT into test_graphite WITH dates AS ( - SELECT today() as today, - today - 3 as older_date + SELECT toStartOfDay(toDateTime(now('UTC'), 'UTC')) as today, + today - INTERVAL 3 day as older_date ) -- Newer than 2 days are kept in windows of 600 seconds - select 1, 'sum_1', toDateTime(today, 'UTC') - number * 60 - 30, number, 1, number from dates, numbers(300) union all - select 2, 'sum_1', toDateTime(today, 'UTC') - number * 60 - 30, number, 1, number from dates, numbers(300) union all - select 1, 'sum_2', toDateTime(today, 'UTC') - number * 60 - 30, number, 1, number from dates, numbers(300) union all - select 2, 'sum_2', toDateTime(today, 'UTC') - number * 60 - 30, number, 1, number from dates, numbers(300) union all - select 1, 'max_1', toDateTime(today, 'UTC') - number * 60 - 30, number, 1, number from dates, numbers(300) union all - select 2, 'max_1', toDateTime(today, 'UTC') - number * 60 - 30, number, 1, number from dates, numbers(300) union all - select 1, 'max_2', toDateTime(today, 'UTC') - number * 60 - 30, number, 1, number from dates, numbers(300) union all - select 2, 'max_2', toDateTime(today, 'UTC') - number * 60 - 30, number, 1, number from dates, numbers(300) union all + select 1, 'sum_1', today - number * 60 - 30, number, 1, number from dates, numbers(300) union all + select 2, 'sum_1', today - number * 60 - 30, number, 1, number from dates, numbers(300) union all + select 1, 'sum_2', today - number * 60 - 30, number, 1, number from dates, numbers(300) union all + select 2, 'sum_2', today - number * 60 - 30, number, 1, number from dates, numbers(300) union all + select 1, 'max_1', today - number * 60 - 30, number, 1, number from dates, numbers(300) union all + select 2, 'max_1', today - number * 60 - 30, number, 1, number from dates, numbers(300) union all + select 1, 'max_2', today - number * 60 - 30, number, 1, number from dates, numbers(300) union all + select 2, 'max_2', today - number * 60 - 30, number, 1, number from dates, numbers(300) union all -- Older than 2 days use 6000 second windows - select 1, 'sum_1', toDateTime(older_date, 'UTC') - number * 60 - 30, number, 1, number from dates, numbers(1200) union all - select 2, 'sum_1', toDateTime(older_date, 'UTC') - number * 60 - 30, number, 1, number from dates, numbers(1200) union all - select 1, 'sum_2', toDateTime(older_date, 'UTC') - number * 60 - 30, number, 1, number from dates, numbers(1200) union all - select 2, 'sum_2', toDateTime(older_date, 'UTC') - number * 60 - 30, number, 1, number from dates, numbers(1200) union all - select 1, 'max_1', toDateTime(older_date, 'UTC') - number * 60 - 30, number, 1, number from dates, numbers(1200) union all - select 2, 'max_1', toDateTime(older_date, 'UTC') - number * 60 - 30, number, 1, number from dates, numbers(1200) union all - select 1, 'max_2', toDateTime(older_date, 'UTC') - number * 60 - 30, number, 1, number from dates, numbers(1200) union all - select 2, 'max_2', toDateTime(older_date, 'UTC') - number * 60 - 30, number, 1, number from dates, numbers(1200); + select 1, 'sum_1', older_date - number * 60 - 30, number, 1, number from dates, numbers(1200) union all + select 2, 'sum_1', older_date - number * 60 - 30, number, 1, number from dates, numbers(1200) union all + select 1, 'sum_2', older_date - number * 60 - 30, number, 1, number from dates, numbers(1200) union all + select 2, 'sum_2', older_date - number * 60 - 30, number, 1, number from dates, numbers(1200) union all + select 1, 'max_1', older_date - number * 60 - 30, number, 1, number from dates, numbers(1200) union all + select 2, 'max_1', older_date - number * 60 - 30, number, 1, number from dates, numbers(1200) union all + select 1, 'max_2', older_date - number * 60 - 30, number, 1, number from dates, numbers(1200) union all + select 2, 'max_2', older_date - number * 60 - 30, number, 1, number from dates, numbers(1200); optimize table test_graphite final; From ac205ffceb0a0e23480a503c1e45c552a6d8b1e9 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Thu, 12 Aug 2021 19:09:17 +0300 Subject: [PATCH 525/599] Update docs/ru/sql-reference/functions/string-functions.md Co-authored-by: Alexey Boykov <33257111+mathalex@users.noreply.github.com> --- docs/ru/sql-reference/functions/string-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/string-functions.md b/docs/ru/sql-reference/functions/string-functions.md index 516c5b23834..dfe57b7c870 100644 --- a/docs/ru/sql-reference/functions/string-functions.md +++ b/docs/ru/sql-reference/functions/string-functions.md @@ -177,7 +177,7 @@ Type: [String](../data-types/string.md). **Пример** -Результат: +Запрос: ``` sql SELECT rightPadUTF8('абвг', 7, '*'), rightPadUTF8('абвг', 7); From 9707373c11987b939d7df5acfca5d4bdf196cd5e Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Thu, 12 Aug 2021 19:09:27 +0300 Subject: [PATCH 526/599] Update docs/en/sql-reference/functions/string-functions.md Co-authored-by: Alexey Boykov <33257111+mathalex@users.noreply.github.com> --- docs/en/sql-reference/functions/string-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index 038995c5883..9a5ba19aeed 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -55,7 +55,7 @@ leftPad('string', 'length'[, 'pad_string']) **Arguments** -- `string` — Input string, that needs to be padded. [String](../data-types/string.md). +- `string` — Input string that needs to be padded. [String](../data-types/string.md). - `length` — The length of the resulting string. [UInt](../data-types/int-uint.md). If the value is less than the input string length, then the input string is returned as-is. - `pad_string` — The string to pad the input string with. [String](../data-types/string.md). Optional. If not specified, then the input string is padded with spaces. From 8a120e0b1bd4e32bacab79c0aa495177403cd60a Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Thu, 12 Aug 2021 19:09:36 +0300 Subject: [PATCH 527/599] Update docs/en/sql-reference/functions/string-functions.md Co-authored-by: Alexey Boykov <33257111+mathalex@users.noreply.github.com> --- docs/en/sql-reference/functions/string-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index 9a5ba19aeed..396bcdc0bfd 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -131,7 +131,7 @@ rightPad('string', 'length'[, 'pad_string']) **Arguments** -- `string` — Input string, that needs to be padded. [String](../data-types/string.md). +- `string` — Input string that needs to be padded. [String](../data-types/string.md). - `length` — The length of the resulting string. [UInt](../data-types/int-uint.md). If the value is less than the input string length, then the input string is returned as-is. - `pad_string` — The string to pad the input string with. [String](../data-types/string.md). Optional. If not specified, then the input string is padded with spaces. From 72b7737391ac9bf9af9f9893fdc497fe055477eb Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Thu, 12 Aug 2021 19:09:43 +0300 Subject: [PATCH 528/599] Update docs/en/sql-reference/functions/string-functions.md Co-authored-by: Alexey Boykov <33257111+mathalex@users.noreply.github.com> --- docs/en/sql-reference/functions/string-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index 396bcdc0bfd..b29b21abdb9 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -169,7 +169,7 @@ rightPadUTF8('string','length'[, 'pad_string']) **Arguments** -- `string` — Input string, that needs to be padded. [String](../data-types/string.md). +- `string` — Input string that needs to be padded. [String](../data-types/string.md). - `length` — The length of the resulting string. [UInt](../data-types/int-uint.md). If the value is less than the input string length, then the input string is returned as-is. - `pad_string` — The string to pad the input string with. [String](../data-types/string.md). Optional. If not specified, then the input string is padded with spaces. From 18ae8aac30918e7e7fcbc6bff6bc408ea9d4d1d2 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Thu, 12 Aug 2021 19:09:51 +0300 Subject: [PATCH 529/599] Update docs/en/sql-reference/functions/string-functions.md Co-authored-by: Alexey Boykov <33257111+mathalex@users.noreply.github.com> --- docs/en/sql-reference/functions/string-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index b29b21abdb9..77bb14c4f5f 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -93,7 +93,7 @@ leftPadUTF8('string','length'[, 'pad_string']) **Arguments** -- `string` — Input string, that needs to be padded. [String](../data-types/string.md). +- `string` — Input string that needs to be padded. [String](../data-types/string.md). - `length` — The length of the resulting string. [UInt](../data-types/int-uint.md). If the value is less than the input string length, then the input string is returned as-is. - `pad_string` — The string to pad the input string with. [String](../data-types/string.md). Optional. If not specified, then the input string is padded with spaces. From fbd04a545f1e63b80793d2e8020f588d9ebfe778 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 12 Aug 2021 19:22:30 +0300 Subject: [PATCH 530/599] Update version_date.tsv after release 21.8.3.44 --- utils/list-versions/version_date.tsv | 1 + 1 file changed, 1 insertion(+) diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index 151dab728a5..67238318e25 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -1,3 +1,4 @@ +v21.8.3.44-lts 2021-08-12 v21.7.7.47-stable 2021-08-09 v21.7.6.39-stable 2021-08-06 v21.7.5.29-stable 2021-07-28 From ae173e4ea1d785057b4ab8063b84e3969e20810a Mon Sep 17 00:00:00 2001 From: Vladimir Goncharov Date: Thu, 12 Aug 2021 19:46:12 +0300 Subject: [PATCH 531/599] Update docs/ru/sql-reference/statements/alter/projection.md Co-authored-by: Alexey Boykov <33257111+mathalex@users.noreply.github.com> --- docs/ru/sql-reference/statements/alter/projection.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/statements/alter/projection.md b/docs/ru/sql-reference/statements/alter/projection.md index a3d829d21e4..23fd3e7e6de 100644 --- a/docs/ru/sql-reference/statements/alter/projection.md +++ b/docs/ru/sql-reference/statements/alter/projection.md @@ -7,7 +7,7 @@ toc_title: PROJECTION Доступны следующие операции: -- `ALTER TABLE [db].name ADD PROJECTION name AS SELECT [GROUP BY] [ORDER BY]` - добавляет описание проекции в метаданные. +- `ALTER TABLE [db].name ADD PROJECTION name AS SELECT [GROUP BY] [ORDER BY]` — добавляет описание проекции в метаданные. - `ALTER TABLE [db].name DROP PROJECTION name` - удаляет описание проекции из метаданных и удаляет файлы проекции с диска. From 28a2638ed6ea0412625736455a0344fdc3e3c0a2 Mon Sep 17 00:00:00 2001 From: Vladimir Goncharov Date: Thu, 12 Aug 2021 19:46:20 +0300 Subject: [PATCH 532/599] Update docs/ru/sql-reference/statements/alter/projection.md Co-authored-by: Alexey Boykov <33257111+mathalex@users.noreply.github.com> --- docs/ru/sql-reference/statements/alter/projection.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/statements/alter/projection.md b/docs/ru/sql-reference/statements/alter/projection.md index 23fd3e7e6de..57ac3f13fd6 100644 --- a/docs/ru/sql-reference/statements/alter/projection.md +++ b/docs/ru/sql-reference/statements/alter/projection.md @@ -9,7 +9,7 @@ toc_title: PROJECTION - `ALTER TABLE [db].name ADD PROJECTION name AS SELECT [GROUP BY] [ORDER BY]` — добавляет описание проекции в метаданные. -- `ALTER TABLE [db].name DROP PROJECTION name` - удаляет описание проекции из метаданных и удаляет файлы проекции с диска. +- `ALTER TABLE [db].name DROP PROJECTION name` — удаляет описание проекции из метаданных и удаляет файлы проекции с диска. - `ALTER TABLE [db.]table MATERIALIZE PROJECTION name IN PARTITION partition_name` - перестраивает проекцию в указанной партиции. Реализовано как [мутация](../../../sql-reference/statements/alter/index.md#mutations). From c79930986716472cb72882db595817678de5b978 Mon Sep 17 00:00:00 2001 From: Vladimir Goncharov Date: Thu, 12 Aug 2021 19:46:30 +0300 Subject: [PATCH 533/599] Update docs/ru/sql-reference/statements/alter/projection.md Co-authored-by: Alexey Boykov <33257111+mathalex@users.noreply.github.com> --- docs/ru/sql-reference/statements/alter/projection.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/statements/alter/projection.md b/docs/ru/sql-reference/statements/alter/projection.md index 57ac3f13fd6..051b28b085f 100644 --- a/docs/ru/sql-reference/statements/alter/projection.md +++ b/docs/ru/sql-reference/statements/alter/projection.md @@ -11,7 +11,7 @@ toc_title: PROJECTION - `ALTER TABLE [db].name DROP PROJECTION name` — удаляет описание проекции из метаданных и удаляет файлы проекции с диска. -- `ALTER TABLE [db.]table MATERIALIZE PROJECTION name IN PARTITION partition_name` - перестраивает проекцию в указанной партиции. Реализовано как [мутация](../../../sql-reference/statements/alter/index.md#mutations). +- `ALTER TABLE [db.]table MATERIALIZE PROJECTION name IN PARTITION partition_name` — перестраивает проекцию в указанной партиции. Реализовано как [мутация](../../../sql-reference/statements/alter/index.md#mutations). - `ALTER TABLE [db.]table CLEAR PROJECTION name IN PARTITION partition_name` - удаляет файлы проекции с диска без удаления описания. From 1d73f0fe84d10b3f5c95f3f1d810c18ca46f080f Mon Sep 17 00:00:00 2001 From: Vladimir Goncharov Date: Thu, 12 Aug 2021 19:46:35 +0300 Subject: [PATCH 534/599] Update docs/ru/sql-reference/statements/alter/projection.md Co-authored-by: Alexey Boykov <33257111+mathalex@users.noreply.github.com> --- docs/ru/sql-reference/statements/alter/projection.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/statements/alter/projection.md b/docs/ru/sql-reference/statements/alter/projection.md index 051b28b085f..d0ae99aff5d 100644 --- a/docs/ru/sql-reference/statements/alter/projection.md +++ b/docs/ru/sql-reference/statements/alter/projection.md @@ -13,7 +13,7 @@ toc_title: PROJECTION - `ALTER TABLE [db.]table MATERIALIZE PROJECTION name IN PARTITION partition_name` — перестраивает проекцию в указанной партиции. Реализовано как [мутация](../../../sql-reference/statements/alter/index.md#mutations). -- `ALTER TABLE [db.]table CLEAR PROJECTION name IN PARTITION partition_name` - удаляет файлы проекции с диска без удаления описания. +- `ALTER TABLE [db.]table CLEAR PROJECTION name IN PARTITION partition_name` — удаляет файлы проекции с диска без удаления описания. Комманды ADD, DROP и CLEAR - легковесны потому что они только меняют метаданные или удаляют файлы. From 18219e4bf27420c4b227eb332476fc5e5a203fb6 Mon Sep 17 00:00:00 2001 From: Vladimir Goncharov Date: Thu, 12 Aug 2021 19:46:59 +0300 Subject: [PATCH 535/599] Update docs/ru/sql-reference/statements/alter/projection.md Co-authored-by: Alexey Boykov <33257111+mathalex@users.noreply.github.com> --- docs/ru/sql-reference/statements/alter/projection.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/statements/alter/projection.md b/docs/ru/sql-reference/statements/alter/projection.md index d0ae99aff5d..d269746bfd5 100644 --- a/docs/ru/sql-reference/statements/alter/projection.md +++ b/docs/ru/sql-reference/statements/alter/projection.md @@ -15,7 +15,7 @@ toc_title: PROJECTION - `ALTER TABLE [db.]table CLEAR PROJECTION name IN PARTITION partition_name` — удаляет файлы проекции с диска без удаления описания. -Комманды ADD, DROP и CLEAR - легковесны потому что они только меняют метаданные или удаляют файлы. +Комманды ADD, DROP и CLEAR — легковесны, поскольку они только меняют метаданные или удаляют файлы. Также комманды реплицируется, синхронизируя описание проекций в метаданных с помощью ZooKeeper. From 85fffe6dcb98863254d9f46b3f9f37a687d91891 Mon Sep 17 00:00:00 2001 From: Vladimir Goncharov Date: Thu, 12 Aug 2021 19:47:12 +0300 Subject: [PATCH 536/599] Update docs/ru/sql-reference/statements/alter/projection.md Co-authored-by: Alexey Boykov <33257111+mathalex@users.noreply.github.com> --- docs/ru/sql-reference/statements/alter/projection.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/statements/alter/projection.md b/docs/ru/sql-reference/statements/alter/projection.md index d269746bfd5..db116963aa6 100644 --- a/docs/ru/sql-reference/statements/alter/projection.md +++ b/docs/ru/sql-reference/statements/alter/projection.md @@ -17,7 +17,7 @@ toc_title: PROJECTION Комманды ADD, DROP и CLEAR — легковесны, поскольку они только меняют метаданные или удаляют файлы. -Также комманды реплицируется, синхронизируя описание проекций в метаданных с помощью ZooKeeper. +Также команды реплицируются, синхронизируя описания проекций в метаданных с помощью ZooKeeper. !!! note "Note" Манипуляции с проекциями поддерживаются только для таблиц с движком [`*MergeTree`](../../../engines/table-engines/mergetree-family/mergetree.md) (включая [replicated](../../../engines/table-engines/mergetree-family/replication.md) варианты). From b6ff2fbf9bc7c078400a93568051a8791e123b6e Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Thu, 12 Aug 2021 20:20:54 +0300 Subject: [PATCH 537/599] Update developer-instruction.md --- docs/ru/development/developer-instruction.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/development/developer-instruction.md b/docs/ru/development/developer-instruction.md index c8f6e6aba44..fd9ff36e3e6 100644 --- a/docs/ru/development/developer-instruction.md +++ b/docs/ru/development/developer-instruction.md @@ -172,7 +172,7 @@ sudo bash -c "$(wget -O - https://apt.llvm.org/llvm.sh)" ```bash cmake -DUSE_DEBUG_HELPERS=1 -DUSE_STATIC_LIBRARIES=0 -DSPLIT_SHARED_LIBRARIES=1 -DCLICKHOUSE_SPLIT_BINARY=1 .. ``` -При этом надо учесть, что получаемые в результате сборки исполнимые файлы будут динамически слинкованы с библиотеками, в результате фактически станут непереносимыми на другие компьютеры (либо для этого нужно будет предпринять значительно больше усилий по сравнению со статической сборкой). Плюсом же в данном случае является значительно меньшее время сборки и значительно меньшее использование места на жёстком диске при сборке. Для целей разработки, когда планируются только отладочные запуске на том же компьютере, где осуществлялась сборка, это может быть наиболее удобным вариантом. +При этом надо учесть, что получаемые в результате сборки исполнимые файлы будут динамически слинкованы с библиотеками, в результате фактически станут непереносимыми на другие компьютеры (либо для этого нужно будет предпринять значительно больше усилий по сравнению со статической сборкой). Плюсом же в данном случае является значительно меньшее время сборки и значительно меньшее использование места на жёстком диске при сборке. Для целей разработки, когда планируются только отладочные запуски на том же компьютере, где осуществлялась сборка, это может быть наиболее удобным вариантом. Вы можете изменить вариант сборки, выполнив новую команду в директории build. From bf0ccd14de9cd0ea4bb50b73031c844874c47a97 Mon Sep 17 00:00:00 2001 From: Sergei Semin Date: Thu, 12 Aug 2021 23:47:12 +0300 Subject: [PATCH 538/599] specify the advantages of shared build more accurately --- docs/ru/development/developer-instruction.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/ru/development/developer-instruction.md b/docs/ru/development/developer-instruction.md index fd9ff36e3e6..c568db4731f 100644 --- a/docs/ru/development/developer-instruction.md +++ b/docs/ru/development/developer-instruction.md @@ -168,11 +168,11 @@ sudo bash -c "$(wget -O - https://apt.llvm.org/llvm.sh)" cmake -D CMAKE_BUILD_TYPE=Debug .. -Для ещё более быстрой сборки и более экономного расхода места на диске при осуществлении сборки можно также использоавть специальные опции: +В случае использования на разработческой машине старого HDD или SSD, а также при желании использовать меньше места для артефактов сборки можно использовать следующую команду: ```bash cmake -DUSE_DEBUG_HELPERS=1 -DUSE_STATIC_LIBRARIES=0 -DSPLIT_SHARED_LIBRARIES=1 -DCLICKHOUSE_SPLIT_BINARY=1 .. ``` -При этом надо учесть, что получаемые в результате сборки исполнимые файлы будут динамически слинкованы с библиотеками, в результате фактически станут непереносимыми на другие компьютеры (либо для этого нужно будет предпринять значительно больше усилий по сравнению со статической сборкой). Плюсом же в данном случае является значительно меньшее время сборки и значительно меньшее использование места на жёстком диске при сборке. Для целей разработки, когда планируются только отладочные запуски на том же компьютере, где осуществлялась сборка, это может быть наиболее удобным вариантом. +При этом надо учесть, что получаемые в результате сборки исполнимые файлы будут динамически слинкованы с библиотеками, и поэтому фактически станут непереносимыми на другие компьютеры (либо для этого нужно будет предпринять значительно больше усилий по сравнению со статической сборкой). Плюсом же в данном случае является значительно меньшее время сборки (это проявляется не на первой сборке, а на последующих, после внесения изменений в исходный код - тратится меньшее время на линковку по сравнению со статической сборкой) и значительно меньшее использование места на жёстком диске (экономия более, чем в 3 раза по сравнению со статической сборкой). Для целей разработки, когда планируются только отладочные запуски на том же компьютере, где осуществлялась сборка, это может быть наиболее удобным вариантом. Вы можете изменить вариант сборки, выполнив новую команду в директории build. From d9a59370d3774a7738cea3fd32da58a5e4a2240e Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Fri, 13 Aug 2021 00:58:24 +0300 Subject: [PATCH 539/599] Fixed SQLiteSource --- src/DataStreams/SQLiteSource.cpp | 58 ++++++++++--------- src/DataStreams/SQLiteSource.h | 7 +-- src/Dictionaries/RedisDictionarySource.cpp | 2 +- ...isBlockInputStream.cpp => RedisSource.cpp} | 2 +- ...{RedisBlockInputStream.h => RedisSource.h} | 0 src/Dictionaries/ya.make | 6 +- 6 files changed, 39 insertions(+), 36 deletions(-) rename src/Dictionaries/{RedisBlockInputStream.cpp => RedisSource.cpp} (99%) rename src/Dictionaries/{RedisBlockInputStream.h => RedisSource.h} (100%) diff --git a/src/DataStreams/SQLiteSource.cpp b/src/DataStreams/SQLiteSource.cpp index f4995703a1e..d0d8724c2dd 100644 --- a/src/DataStreams/SQLiteSource.cpp +++ b/src/DataStreams/SQLiteSource.cpp @@ -23,32 +23,32 @@ namespace ErrorCodes } SQLiteSource::SQLiteSource( - SQLitePtr sqlite_db_, - const String & query_str_, - const Block & sample_block, - const UInt64 max_block_size_) + SQLitePtr sqlite_db_, + const String & query_str_, + const Block & sample_block, + const UInt64 max_block_size_) : SourceWithProgress(sample_block.cloneEmpty()) , query_str(query_str_) , max_block_size(max_block_size_) , sqlite_db(std::move(sqlite_db_)) { description.init(sample_block); + + sqlite3_stmt * compiled_stmt = nullptr; + int status = sqlite3_prepare_v2(sqlite_db.get(), query_str.c_str(), query_str.size() + 1, &compiled_stmt, nullptr); + + if (status != SQLITE_OK) + throw Exception(ErrorCodes::SQLITE_ENGINE_ERROR, + "Cannot prepate sqlite statement. Status: {}. Message: {}", + status, sqlite3_errstr(status)); + + compiled_statement = std::unique_ptr(compiled_stmt, StatementDeleter()); } Chunk SQLiteSource::generate() { if (!compiled_statement) - { - sqlite3_stmt * compiled_stmt = nullptr; - int status = sqlite3_prepare_v2(sqlite_db.get(), query_str.c_str(), query_str.size() + 1, &compiled_stmt, nullptr); - - if (status != SQLITE_OK) - throw Exception(ErrorCodes::SQLITE_ENGINE_ERROR, - "Cannot prepate sqlite statement. Status: {}. Message: {}", - status, sqlite3_errstr(status)); - - compiled_statement = std::unique_ptr(compiled_stmt, StatementDeleter()); - } + return {}; MutableColumns columns = description.sample_block.cloneEmptyColumns(); size_t num_rows = 0; @@ -69,30 +69,30 @@ Chunk SQLiteSource::generate() else if (status != SQLITE_ROW) { throw Exception(ErrorCodes::SQLITE_ENGINE_ERROR, - "Expected SQLITE_ROW status, but got status {}. Error: {}, Message: {}", - status, sqlite3_errstr(status), sqlite3_errmsg(sqlite_db.get())); + "Expected SQLITE_ROW status, but got status {}. Error: {}, Message: {}", + status, sqlite3_errstr(status), sqlite3_errmsg(sqlite_db.get())); } int column_count = sqlite3_column_count(compiled_statement.get()); - for (const auto idx : collections::range(0, column_count)) - { - const auto & sample = description.sample_block.getByPosition(idx); - if (sqlite3_column_type(compiled_statement.get(), idx) == SQLITE_NULL) + for (int column_index = 0; column_index < column_count; ++column_index) + { + if (sqlite3_column_type(compiled_statement.get(), column_index) == SQLITE_NULL) { - insertDefaultSQLiteValue(*columns[idx], *sample.column); + columns[column_index]->insertDefault(); continue; } - if (description.types[idx].second) + auto & [type, is_nullable] = description.types[column_index]; + if (is_nullable) { - ColumnNullable & column_nullable = assert_cast(*columns[idx]); - insertValue(column_nullable.getNestedColumn(), description.types[idx].first, idx); + ColumnNullable & column_nullable = assert_cast(*columns[column_index]); + insertValue(column_nullable.getNestedColumn(), type, column_index); column_nullable.getNullMapData().emplace_back(0); } else { - insertValue(*columns[idx], description.types[idx].first, idx); + insertValue(*columns[column_index], type, column_index); } } @@ -100,6 +100,12 @@ Chunk SQLiteSource::generate() break; } + if (num_rows == 0) + { + compiled_statement.reset(); + return {}; + } + return Chunk(std::move(columns), num_rows); } diff --git a/src/DataStreams/SQLiteSource.h b/src/DataStreams/SQLiteSource.h index 653fdb402e3..0f8b42c536b 100644 --- a/src/DataStreams/SQLiteSource.h +++ b/src/DataStreams/SQLiteSource.h @@ -13,8 +13,10 @@ namespace DB { + class SQLiteSource : public SourceWithProgress { + using SQLitePtr = std::shared_ptr; public: @@ -26,10 +28,6 @@ public: String getName() const override { return "SQLite"; } private: - static void insertDefaultSQLiteValue(IColumn & column, const IColumn & sample_column) - { - column.insertFrom(sample_column, 0); - } using ValueType = ExternalResultDescription::ValueType; @@ -46,7 +44,6 @@ private: UInt64 max_block_size; ExternalResultDescription description; - SQLitePtr sqlite_db; std::unique_ptr compiled_statement; }; diff --git a/src/Dictionaries/RedisDictionarySource.cpp b/src/Dictionaries/RedisDictionarySource.cpp index bf309dd0e8a..6561a122e9d 100644 --- a/src/Dictionaries/RedisDictionarySource.cpp +++ b/src/Dictionaries/RedisDictionarySource.cpp @@ -31,7 +31,7 @@ void registerDictionarySourceRedis(DictionarySourceFactory & factory) #include -#include "RedisBlockInputStream.h" +#include "RedisSource.h" namespace DB diff --git a/src/Dictionaries/RedisBlockInputStream.cpp b/src/Dictionaries/RedisSource.cpp similarity index 99% rename from src/Dictionaries/RedisBlockInputStream.cpp rename to src/Dictionaries/RedisSource.cpp index c6e2546cf1e..ad5cf8a0977 100644 --- a/src/Dictionaries/RedisBlockInputStream.cpp +++ b/src/Dictionaries/RedisSource.cpp @@ -1,4 +1,4 @@ -#include "RedisBlockInputStream.h" +#include "RedisSource.h" #include #include diff --git a/src/Dictionaries/RedisBlockInputStream.h b/src/Dictionaries/RedisSource.h similarity index 100% rename from src/Dictionaries/RedisBlockInputStream.h rename to src/Dictionaries/RedisSource.h diff --git a/src/Dictionaries/ya.make b/src/Dictionaries/ya.make index 2cff8447574..3f287f8bddc 100644 --- a/src/Dictionaries/ya.make +++ b/src/Dictionaries/ya.make @@ -26,9 +26,9 @@ SRCS( CassandraHelpers.cpp CassandraSource.cpp ClickHouseDictionarySource.cpp - DictionaryBlockInputStream.cpp - DictionaryBlockInputStreamBase.cpp DictionaryFactory.cpp + DictionarySource.cpp + DictionarySourceBase.cpp DictionarySourceFactory.cpp DictionarySourceHelpers.cpp DictionaryStructure.cpp @@ -57,8 +57,8 @@ SRCS( PolygonDictionaryImplementations.cpp PolygonDictionaryUtils.cpp RangeHashedDictionary.cpp - RedisBlockInputStream.cpp RedisDictionarySource.cpp + RedisSource.cpp XDBCDictionarySource.cpp getDictionaryConfigurationFromAST.cpp readInvalidateQuery.cpp From 36db9cf55a4a8675fabe3a944146f81f10224e5e Mon Sep 17 00:00:00 2001 From: Vladimir Goncharov Date: Fri, 13 Aug 2021 00:59:16 +0300 Subject: [PATCH 540/599] Update mergetree.md --- .../mergetree-family/mergetree.md | 18 ++++++++++++++++++ 1 file changed, 18 insertions(+) diff --git a/docs/ru/engines/table-engines/mergetree-family/mergetree.md b/docs/ru/engines/table-engines/mergetree-family/mergetree.md index 4bced6254d1..728c632370c 100644 --- a/docs/ru/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/ru/engines/table-engines/mergetree-family/mergetree.md @@ -375,6 +375,24 @@ INDEX b (u64 * length(str), i32 + f64 * 100, date, str) TYPE set(100) GRANULARIT - `s != 1` - `NOT startsWith(s, 'test')` +### Проекции {#projections} +Проекции похожи на материализованные представления, но определяются на уровне партов. Это обеспечивает гарантии согласованности наряду с автоматическим использованием в запросах. + +#### Запрос {#projection-query} +Запрос проекции - это то, что определяет проекцию. Он имеет следующую грамматику: + +`SELECT [GROUP BY] [ORDER BY]` + +Это неявно выбирает данные из родительской таблицы. + +#### Хранение {#projection-storage} +Проекции хранятся в каталоге парта. Это похоже на хранение индексов, но используется подкаталог, в котором хранится анонимный парт таблицы MergeTree. Таблица создается запросом определения проекции. Если есть конструкция GROUP BY, то базовый механизм хранения становится AggregatedMergeTree, а все агрегатные функции преобразуются в AggregateFunction. Если есть конструкция ORDER BY, таблица MergeTree будет использовать его в качестве выражения первичного ключа. Во время процесса слияния парт проекции будет слит с помощью процедуры слияния ее хранилища. Контрольная сумма парта родительской таблицы будет включать парт проекции. Другие процедуры аналогичны индексам пропуска данных. + +#### Анализ запросов {#projection-query-analysis} +1. Проверить, можно ли использовать проекцию для результата на данный запрос, то есть она даёт тот же результат, что и запрос к базовой таблице. +2. Выбрать наиболее подходящее совпадение, содержащее наименьшее количество гранул для чтения. +3. План запроса, который использует проекции, будет отличаться от того, который использует исходные парты. Если в некоторых партах проекция отсутствует, мы можем расширить план, чтобы «проецировать» на лету. + ## Конкурентный доступ к данным {#concurrent-data-access} Для конкурентного доступа к таблице используется мультиверсионность. То есть, при одновременном чтении и обновлении таблицы, данные будут читаться из набора кусочков, актуального на момент запроса. Длинных блокировок нет. Вставки никак не мешают чтениям. From c83551ef8ecda55ee77f9125caaf686b3714d35d Mon Sep 17 00:00:00 2001 From: Denis Zhuravlev Date: Thu, 12 Aug 2021 21:10:20 -0300 Subject: [PATCH 541/599] enable part_log by default --- programs/server/config.xml | 4 ++-- programs/server/config.yaml.example | 9 +++++---- .../test_config_corresponding_root/configs/config.xml | 5 ++--- .../integration/test_config_xml_full/configs/config.xml | 6 +++--- .../aes_encryption/configs/clickhouse/config.xml | 4 ++-- .../configs/clickhouse/config.xml | 4 ++-- tests/testflows/example/configs/clickhouse/config.xml | 4 ++-- .../configs/clickhouse/config.xml | 4 ++-- tests/testflows/kerberos/configs/clickhouse/config.xml | 4 ++-- .../ldap/authentication/configs/clickhouse/config.xml | 4 ++-- .../configs/clickhouse/config.xml | 4 ++-- .../ldap/role_mapping/configs/clickhouse/config.xml | 4 ++-- tests/testflows/map_type/configs/clickhouse/config.xml | 4 ++-- tests/testflows/rbac/configs/clickhouse/config.xml | 4 ++-- .../window_functions/configs/clickhouse/config.xml | 4 ++-- 15 files changed, 34 insertions(+), 34 deletions(-) diff --git a/programs/server/config.xml b/programs/server/config.xml index 136b982a181..510a5e230f8 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -888,13 +888,13 @@ system part_log
+ toYYYYMM(event_date) 7500
- --> system part_log
+ toYYYYMM(event_date) 7500
- --> - - + @@ -838,13 +838,13 @@ system part_log
+ toYYYYMM(event_date) 7500
- --> system part_log
+ toYYYYMM(event_date) 7500
- --> system part_log
+ toYYYYMM(event_date) 7500
- --> system part_log
+ toYYYYMM(event_date) 7500
- --> system part_log
+ toYYYYMM(event_date) 7500
- --> system part_log
+ toYYYYMM(event_date) 7500
- --> system part_log
+ toYYYYMM(event_date) 7500
- --> system part_log
+ toYYYYMM(event_date) 7500
- --> system part_log
+ toYYYYMM(event_date) 7500
- --> system part_log
+ toYYYYMM(event_date) 7500
- --> system part_log
+ toYYYYMM(event_date) 7500
- --> system part_log
+ toYYYYMM(event_date) 7500
- --> -------SelectIntersectExceptQuery @@ -59,7 +74,8 @@ void SelectIntersectExceptQueryMatcher::visit(ASTSelectIntersectExceptQuery & as { auto select_intersect_except = std::make_shared(); select_intersect_except->final_operator = {current_operator}; - select_intersect_except->children.emplace_back(std::move(list_node)); + select_intersect_except->list_of_selects = std::move(list_node); + select_intersect_except->children.push_back(select_intersect_except->list_of_selects); selects.emplace_back(std::move(select_intersect_except)); } @@ -67,19 +83,88 @@ void SelectIntersectExceptQueryMatcher::visit(ASTSelectIntersectExceptQuery & as visit(ast, data); } -// void SelectIntersectExceptQueryVisitor::visit(ASTSelectWithUnionQuery & ast, Data & data) -// { -// auto & union_modes = ast.list_of_modes; -// ASTs selects; -// auto & select_list = ast.list_of_selects->children; -// -// -// // reverse children list -// std::reverse(selects.begin(), selects.end()); -// -// ast.is_normalized = true; -// ast.union_mode = ASTSelectWithUnionQuery::Mode::ALL; -// -// ast.list_of_selects->children = std::move(selects); -// } +void SelectIntersectExceptQueryMatcher::visit(ASTSelectWithUnionQuery & ast, Data &) +{ + /* Example: select 1 union all select 2 except select 1 except select 2 union distinct select 5; + * + * --SelectWithUnionQuery --SelectIntersectExceptQuery + * ---ExpressionList ---ExpressionList + * ----SelectQuery ----SelectIntersectExceptQuery + * ----SelectQuery -----ExpressionList + * ----SelectQuery (except) ---> ------SelectIntersectExceptQuery + * ----SelectQuery (except) -------ExpressionList + * ----SelectQuery --------SelectWithUnionQuery (select 1 union all select 2) + * --------SelectQuery (select 1) + * ------SelectQuery (select 2) + * -----SelectQuery (select 5) + **/ + + auto & union_modes = ast.list_of_modes; + + if (union_modes.empty()) + return; + + auto selects = std::move(ast.list_of_selects->children); + + if (union_modes.size() + 1 != selects.size()) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Incorrect ASTSelectWithUnionQuery (modes: {}, selects: {})", + union_modes.size(), selects.size()); + + std::reverse(selects.begin(), selects.end()); + + ASTs children = {selects.back()}; + selects.pop_back(); + ASTSelectWithUnionQuery::UnionModes modes; + + for (const auto & mode : union_modes) + { + /// Flatten all previous selects into ASTSelectIntersectQuery + if (mode == ASTSelectWithUnionQuery::Mode::EXCEPT) + { + auto left = std::make_shared(); + left->union_mode = ASTSelectWithUnionQuery::Mode::ALL; + + left->list_of_selects = std::make_shared(); + left->children.push_back(left->list_of_selects); + left->list_of_selects->children = std::move(children); + + left->list_of_modes = std::move(modes); + modes = {}; + + auto right = selects.back(); + selects.pop_back(); + + auto list_node = std::make_shared(); + list_node->children = {left, right}; + + auto select_intersect_except = std::make_shared(); + select_intersect_except->final_operator = {ASTSelectIntersectExceptQuery::Operator::EXCEPT}; + select_intersect_except->children.emplace_back(std::move(list_node)); + select_intersect_except->list_of_selects = std::make_shared(); + select_intersect_except->list_of_selects->children.push_back(select_intersect_except->children[0]); + + children = {select_intersect_except}; + } + else if (!selects.empty()) + { + auto right = selects.back(); + selects.pop_back(); + children.emplace_back(std::move(right)); + modes.push_back(mode); + } + } + + if (!selects.empty()) + { + auto right = selects.back(); + selects.pop_back(); + children.emplace_back(std::move(right)); + } + + ast.union_mode = ASTSelectWithUnionQuery::Mode::ALL; + ast.list_of_selects->children = std::move(children); + ast.list_of_modes = std::move(modes); +} + } diff --git a/src/Interpreters/SelectIntersectExceptQueryVisitor.h b/src/Interpreters/SelectIntersectExceptQueryVisitor.h index 58f3071972f..1dd0694666d 100644 --- a/src/Interpreters/SelectIntersectExceptQueryVisitor.h +++ b/src/Interpreters/SelectIntersectExceptQueryVisitor.h @@ -38,7 +38,7 @@ public: static void visit(ASTPtr & ast, Data &); static void visit(ASTSelectIntersectExceptQuery &, Data &); - // static void visit(ASTSelectWithUnionQuery &, Data &); + static void visit(ASTSelectWithUnionQuery &, Data &); }; /// Visit children first. diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 839447a90f7..4131bac28d1 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -492,14 +492,14 @@ static std::tuple executeQueryImpl( } { - /// Normalize SelectWithUnionQuery - NormalizeSelectWithUnionQueryVisitor::Data data{context->getSettingsRef().union_default_mode}; - NormalizeSelectWithUnionQueryVisitor{data}.visit(ast); + SelectIntersectExceptQueryVisitor::Data data; + SelectIntersectExceptQueryVisitor{data}.visit(ast); } { - SelectIntersectExceptQueryVisitor::Data data; - SelectIntersectExceptQueryVisitor{data}.visit(ast); + /// Normalize SelectWithUnionQuery + NormalizeSelectWithUnionQueryVisitor::Data data{context->getSettingsRef().union_default_mode}; + NormalizeSelectWithUnionQueryVisitor{data}.visit(ast); } /// Check the limits. @@ -540,7 +540,6 @@ static std::tuple executeQueryImpl( /// reset Input callbacks if query is not INSERT SELECT context->resetInputCallbacks(); - std::cerr << "\n\nAST: " << ast->dumpTree() << std::endl; auto interpreter = InterpreterFactory::get(ast, context, SelectQueryOptions(stage).setInternal(internal)); std::shared_ptr quota; diff --git a/src/Parsers/ASTSelectWithUnionQuery.cpp b/src/Parsers/ASTSelectWithUnionQuery.cpp index fa7359574f8..d19e860c16a 100644 --- a/src/Parsers/ASTSelectWithUnionQuery.cpp +++ b/src/Parsers/ASTSelectWithUnionQuery.cpp @@ -8,6 +8,10 @@ namespace DB { +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} ASTPtr ASTSelectWithUnionQuery::clone() const { @@ -28,6 +32,9 @@ ASTPtr ASTSelectWithUnionQuery::clone() const void ASTSelectWithUnionQuery::formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const { + if (!list_of_selects || list_of_selects->children.size() != list_of_modes.size() + 1) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Incorrect ASTSelectWithUnionQuery"); + std::string indent_str = settings.one_line ? "" : std::string(4 * frame.indent, ' '); auto mode_to_str = [&](auto mode) diff --git a/src/Parsers/ASTSelectWithUnionQuery.h b/src/Parsers/ASTSelectWithUnionQuery.h index 0465bdac3a6..2c36bcecf6b 100644 --- a/src/Parsers/ASTSelectWithUnionQuery.h +++ b/src/Parsers/ASTSelectWithUnionQuery.h @@ -22,7 +22,8 @@ public: { Unspecified, ALL, - DISTINCT + DISTINCT, + EXCEPT }; using UnionModes = std::vector; diff --git a/src/Parsers/ExpressionListParsers.cpp b/src/Parsers/ExpressionListParsers.cpp index e22f2c7cded..e75aad8d02f 100644 --- a/src/Parsers/ExpressionListParsers.cpp +++ b/src/Parsers/ExpressionListParsers.cpp @@ -140,7 +140,14 @@ bool ParserUnionList::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) } // SELECT ... UNION SELECT ... else + { union_modes.push_back(ASTSelectWithUnionQuery::Mode::Unspecified); + } + return true; + } + else if (s_except_parser->check(pos, expected)) + { + union_modes.push_back(ASTSelectWithUnionQuery::Mode::EXCEPT); return true; } return false; @@ -1024,4 +1031,3 @@ bool ParserKeyValuePairsList::parseImpl(Pos & pos, ASTPtr & node, Expected & exp } } - diff --git a/src/Parsers/ExpressionListParsers.h b/src/Parsers/ExpressionListParsers.h index ef6a5744603..36f39a50ab3 100644 --- a/src/Parsers/ExpressionListParsers.h +++ b/src/Parsers/ExpressionListParsers.h @@ -79,11 +79,17 @@ private: class ParserUnionList : public IParserBase { public: - ParserUnionList(ParserPtr && elem_parser_, ParserPtr && s_union_parser_, ParserPtr && s_all_parser_, ParserPtr && s_distinct_parser_) + ParserUnionList( + ParserPtr && elem_parser_, + ParserPtr && s_union_parser_, + ParserPtr && s_all_parser_, + ParserPtr && s_distinct_parser_, + ParserPtr && s_except_parser_) : elem_parser(std::move(elem_parser_)) , s_union_parser(std::move(s_union_parser_)) , s_all_parser(std::move(s_all_parser_)) , s_distinct_parser(std::move(s_distinct_parser_)) + , s_except_parser(std::move(s_except_parser_)) { } @@ -120,6 +126,7 @@ private: ParserPtr s_union_parser; ParserPtr s_all_parser; ParserPtr s_distinct_parser; + ParserPtr s_except_parser; ASTSelectWithUnionQuery::UnionModes union_modes; }; diff --git a/src/Parsers/ParserSelectIntersectExceptQuery.cpp b/src/Parsers/ParserSelectIntersectExceptQuery.cpp index b56598166c6..2b4ba9d60e2 100644 --- a/src/Parsers/ParserSelectIntersectExceptQuery.cpp +++ b/src/Parsers/ParserSelectIntersectExceptQuery.cpp @@ -14,7 +14,6 @@ namespace DB bool ParserSelectIntersectExceptQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { ParserKeyword intersect_keyword("INTERSECT"); - ParserKeyword except_keyword("EXCEPT"); ASTs elements; ASTSelectIntersectExceptQuery::Operators operators; @@ -32,13 +31,7 @@ bool ParserSelectIntersectExceptQuery::parseImpl(Pos & pos, ASTPtr & node, Expec auto parse_separator = [&]() -> bool { if (!intersect_keyword.ignore(pos)) - { - if (!except_keyword.ignore(pos)) - return false; - - operators.emplace_back(ASTSelectIntersectExceptQuery::Operator::EXCEPT); - return true; - } + return false; operators.emplace_back(ASTSelectIntersectExceptQuery::Operator::INTERSECT); return true; diff --git a/src/Parsers/ParserSelectWithUnionQuery.cpp b/src/Parsers/ParserSelectWithUnionQuery.cpp index 87e2dab1a47..8c4c183a099 100644 --- a/src/Parsers/ParserSelectWithUnionQuery.cpp +++ b/src/Parsers/ParserSelectWithUnionQuery.cpp @@ -15,7 +15,8 @@ bool ParserSelectWithUnionQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & std::make_unique(), std::make_unique("UNION"), std::make_unique("ALL"), - std::make_unique("DISTINCT")); + std::make_unique("DISTINCT"), + std::make_unique("EXCEPT")); if (!parser.parse(pos, list_node, expected)) return false; diff --git a/src/Processors/QueryPlan/IntersectOrExceptStep.cpp b/src/Processors/QueryPlan/IntersectOrExceptStep.cpp index e61afb5ba2a..e4d04115cff 100644 --- a/src/Processors/QueryPlan/IntersectOrExceptStep.cpp +++ b/src/Processors/QueryPlan/IntersectOrExceptStep.cpp @@ -65,6 +65,8 @@ QueryPipelinePtr IntersectOrExceptStep::updatePipeline(QueryPipelines pipelines, return std::make_shared(cur_header, converting_actions); }); } + + cur_pipeline->addTransform(std::make_shared(header, cur_pipeline->getNumStreams(), 1)); } *pipeline = QueryPipeline::unitePipelines(std::move(pipelines), max_threads); diff --git a/tests/queries/0_stateless/02004_intersect_except_operators.reference b/tests/queries/0_stateless/02004_intersect_except_operators.reference index c3272a5d574..9a9e4e1bf58 100644 --- a/tests/queries/0_stateless/02004_intersect_except_operators.reference +++ b/tests/queries/0_stateless/02004_intersect_except_operators.reference @@ -5,7 +5,7 @@ select 2 intersect select 1; select 1 except select 1; select 2 except select 1; 2 -select number from numbers(5, 5) intersect select number from numbers(20); +select number from numbers(20) intersect select number from numbers(5, 5); 5 6 7 @@ -26,12 +26,15 @@ select number, number+10 from numbers(12) except select number+5, number+15 from select 1 except select 2 intersect select 1; 1 select 1 except select 2 intersect select 2; +1 select 1 intersect select 1 except select 2; 1 select 1 intersect select 1 except select 1; select 1 intersect select 1 except select 2 intersect select 1 except select 3 intersect select 1; 1 select 1 intersect select 1 except select 2 intersect select 1 except select 3 intersect select 2; +1 +select 1 intersect select 1 except select 2 intersect select 1 except select 3 intersect select 2 except select 1; select number from numbers(10) except select 5; 0 1 @@ -71,6 +74,8 @@ select count() from (select number from numbers(1000000) intersect select number 600000 select count() from (select number from numbers(100) intersect select number from numbers(20, 60) except select number from numbers(30, 20) except select number from numbers(60, 20)); 20 +select count() from (select number from numbers(100) intersect select number from numbers(20, 60) except select number from numbers(30, 20) except select number from numbers(60, 20) union all select number from numbers(100, 10)); +30 select count() from (select number from numbers(1000000) intersect select number from numbers(200000, 600000) except select number from numbers(300000, 200000) except select number from numbers(600000, 200000)); 200000 select 1 union all select 1 intersect select 1; @@ -78,3 +83,13 @@ select 1 union all select 1 intersect select 1; 1 select 1 union all select 1 intersect select 2; 1 +select * from (select 1 union all select 2 union all select 3 union all select 4 except select 3 union all select 5) order by 1; +1 +2 +4 +5 +select * from (select 1 union all select 2 union all select 3 union all select 4 intersect select 3 union all select 5) order by 1; +1 +2 +3 +5 diff --git a/tests/queries/0_stateless/02004_intersect_except_operators.sql b/tests/queries/0_stateless/02004_intersect_except_operators.sql index 722670732ac..c88951ef353 100644 --- a/tests/queries/0_stateless/02004_intersect_except_operators.sql +++ b/tests/queries/0_stateless/02004_intersect_except_operators.sql @@ -4,7 +4,7 @@ select 2 intersect select 1; select 1 except select 1; select 2 except select 1; -select number from numbers(5, 5) intersect select number from numbers(20); +select number from numbers(20) intersect select number from numbers(5, 5); select number from numbers(10) except select number from numbers(5); select number, number+10 from numbers(12) except select number+5, number+15 from numbers(10); @@ -14,6 +14,7 @@ select 1 intersect select 1 except select 2; select 1 intersect select 1 except select 1; select 1 intersect select 1 except select 2 intersect select 1 except select 3 intersect select 1; select 1 intersect select 1 except select 2 intersect select 1 except select 3 intersect select 2; +select 1 intersect select 1 except select 2 intersect select 1 except select 3 intersect select 2 except select 1; select number from numbers(10) except select 5; select number from numbers(100) intersect select number from numbers(20, 60) except select number from numbers(30, 20) except select number from numbers(60, 20); @@ -22,7 +23,10 @@ with (select number from numbers(10) intersect select 5) as a select a * 10; select count() from (select number from numbers(10) except select 5); select count() from (select number from numbers(1000000) intersect select number from numbers(200000, 600000)); select count() from (select number from numbers(100) intersect select number from numbers(20, 60) except select number from numbers(30, 20) except select number from numbers(60, 20)); +select count() from (select number from numbers(100) intersect select number from numbers(20, 60) except select number from numbers(30, 20) except select number from numbers(60, 20) union all select number from numbers(100, 10)); select count() from (select number from numbers(1000000) intersect select number from numbers(200000, 600000) except select number from numbers(300000, 200000) except select number from numbers(600000, 200000)); select 1 union all select 1 intersect select 1; select 1 union all select 1 intersect select 2; +select * from (select 1 union all select 2 union all select 3 union all select 4 except select 3 union all select 5) order by 1; +select * from (select 1 union all select 2 union all select 3 union all select 4 intersect select 3 union all select 5) order by 1; From c0f3c0a176f2deead29afcc2ce159f4cf59b6a99 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Fri, 13 Aug 2021 13:09:31 +0300 Subject: [PATCH 545/599] Update docs/ru/sql-reference/functions/array-functions.md Co-authored-by: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> --- docs/ru/sql-reference/functions/array-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/array-functions.md b/docs/ru/sql-reference/functions/array-functions.md index 066d37a71f5..a675f728bce 100644 --- a/docs/ru/sql-reference/functions/array-functions.md +++ b/docs/ru/sql-reference/functions/array-functions.md @@ -58,7 +58,7 @@ SELECT empty([]); notEmpty([x]) ``` -Массив считается непустым, если он содержит хотя бы один непустой элемент. +Массив считается непустым, если он содержит хотя бы один элемент. !!! note "Примечание" Функцию можно оптимизировать, если включить настройку [optimize_functions_to_subcolumns](../../operations/settings/settings.md#optimize-functions-to-subcolumns). При `optimize_functions_to_subcolumns = 1` функция читает только подстолбец [size0](../../sql-reference/data-types/array.md#array-size) вместо чтения и обработки всего столбца массива. Запрос `SELECT notEmpty(arr) FROM table` преобразуется к запросу `SELECT arr.size0 != 0 FROM TABLE`. From 279b5ef954018b432cda394685f199985b638fb7 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Fri, 13 Aug 2021 13:09:48 +0300 Subject: [PATCH 546/599] Update docs/ru/sql-reference/functions/array-functions.md Co-authored-by: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> --- docs/ru/sql-reference/functions/array-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/array-functions.md b/docs/ru/sql-reference/functions/array-functions.md index a675f728bce..b7a301d30a9 100644 --- a/docs/ru/sql-reference/functions/array-functions.md +++ b/docs/ru/sql-reference/functions/array-functions.md @@ -15,7 +15,7 @@ toc_title: "Массивы" empty([x]) ``` -Массив считается пустым, если он содержит все пустые элементы. +Массив считается пустым, если он не содержит ни одного элемента. !!! note "Примечание" Функцию можно оптимизировать, если включить настройку [optimize_functions_to_subcolumns](../../operations/settings/settings.md#optimize-functions-to-subcolumns). При `optimize_functions_to_subcolumns = 1` функция читает только подстолбец [size0](../../sql-reference/data-types/array.md#array-size) вместо чтения и обработки всего столбца массива. Запрос `SELECT empty(arr) FROM TABLE` преобразуется к запросу `SELECT arr.size0 = 0 FROM TABLE`. From 0875d72e5c267604140375e75f0ab92f09541dee Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Fri, 13 Aug 2021 13:13:09 +0300 Subject: [PATCH 547/599] Update docs/en/sql-reference/functions/array-functions.md Co-authored-by: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> --- docs/en/sql-reference/functions/array-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/array-functions.md b/docs/en/sql-reference/functions/array-functions.md index ddbbab3ecfc..89de48a81ad 100644 --- a/docs/en/sql-reference/functions/array-functions.md +++ b/docs/en/sql-reference/functions/array-functions.md @@ -15,7 +15,7 @@ Checks whether the input array is empty. empty([x]) ``` -An array is considered empty if it contains all empty elements. +An array is considered empty if it does not contain any elements. !!! note "Note" Can be optimized by enabling the [optimize_functions_to_subcolumns](../../operations/settings/settings.md#optimize-functions-to-subcolumns) setting. With `optimize_functions_to_subcolumns = 1` the function reads only [size0](../../sql-reference/data-types/array.md#array-size) subcolumn instead of reading and processing the whole array column. The query `SELECT empty(arr) FROM TABLE;` transforms to `SELECT arr.size0 = 0 FROM TABLE;`. From bbcdf03a8c937768f893ba064e075170d00b7c9b Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Fri, 13 Aug 2021 13:13:35 +0300 Subject: [PATCH 548/599] Update docs/en/sql-reference/functions/array-functions.md Co-authored-by: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> --- docs/en/sql-reference/functions/array-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/array-functions.md b/docs/en/sql-reference/functions/array-functions.md index 89de48a81ad..e7918c018db 100644 --- a/docs/en/sql-reference/functions/array-functions.md +++ b/docs/en/sql-reference/functions/array-functions.md @@ -58,7 +58,7 @@ Checks whether the input array is non-empty. notEmpty([x]) ``` -An array is considered non-empty if it contains at least one non-empty element. +An array is considered non-empty if it contains at least one element. !!! note "Note" Can be optimized by enabling the [optimize_functions_to_subcolumns](../../operations/settings/settings.md#optimize-functions-to-subcolumns) setting. With `optimize_functions_to_subcolumns = 1` the function reads only [size0](../../sql-reference/data-types/array.md#array-size) subcolumn instead of reading and processing the whole array column. The query `SELECT notEmpty(arr) FROM table` transforms to `SELECT arr.size0 != 0 FROM TABLE`. From bd3fe4fb419e95a84cf9e8ad70f8fe59c0c9a2c7 Mon Sep 17 00:00:00 2001 From: tavplubix Date: Fri, 13 Aug 2021 13:23:54 +0300 Subject: [PATCH 549/599] Update MemoryTracker.cpp --- src/Common/MemoryTracker.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Common/MemoryTracker.cpp b/src/Common/MemoryTracker.cpp index 0be7ffda958..50ddcb5a9eb 100644 --- a/src/Common/MemoryTracker.cpp +++ b/src/Common/MemoryTracker.cpp @@ -360,7 +360,7 @@ void MemoryTracker::setOrRaiseHardLimit(Int64 value) { /// This is just atomic set to maximum. Int64 old_value = hard_limit.load(std::memory_order_relaxed); - while (old_value < value && !hard_limit.compare_exchange_weak(old_value, value)) + while ((value == 0 || old_value < value) && !hard_limit.compare_exchange_weak(old_value, value)) ; } @@ -368,6 +368,6 @@ void MemoryTracker::setOrRaiseHardLimit(Int64 value) void MemoryTracker::setOrRaiseProfilerLimit(Int64 value) { Int64 old_value = profiler_limit.load(std::memory_order_relaxed); - while (old_value < value && !profiler_limit.compare_exchange_weak(old_value, value)) + while ((value == 0 || old_value < value) && !profiler_limit.compare_exchange_weak(old_value, value)) ; } From 7fba508b002a01559dcbfcbaa5c637a98b291033 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 13 Aug 2021 13:43:37 +0200 Subject: [PATCH 550/599] MV: Improve text logs when doing parallel processing --- .../PushingToViewsBlockOutputStream.cpp | 36 +++++++------------ 1 file changed, 13 insertions(+), 23 deletions(-) diff --git a/src/DataStreams/PushingToViewsBlockOutputStream.cpp b/src/DataStreams/PushingToViewsBlockOutputStream.cpp index aec1209a454..dec5b710f75 100644 --- a/src/DataStreams/PushingToViewsBlockOutputStream.cpp +++ b/src/DataStreams/PushingToViewsBlockOutputStream.cpp @@ -342,19 +342,7 @@ void PushingToViewsBlockOutputStream::writeSuffix() runViewStage(view, stage_step, [&] { process_suffix(view); }); if (view.exception) - { exception_count.fetch_add(1, std::memory_order_relaxed); - } - else - { - LOG_TRACE( - log, - "Pushing (parallel {}) from {} to {} took {} ms.", - max_threads, - storage->getStorageID().getNameForLogs(), - view.table_id.getNameForLogs(), - view.runtime_stats.elapsed_ms); - } }); } pool.wait(); @@ -371,20 +359,22 @@ void PushingToViewsBlockOutputStream::writeSuffix() } runViewStage(view, stage_step, [&] { process_suffix(view); }); if (view.exception) - { exception_happened = true; - } - else - { - LOG_TRACE( - log, - "Pushing (sequentially) from {} to {} took {} ms.", - storage->getStorageID().getNameForLogs(), - view.table_id.getNameForLogs(), - view.runtime_stats.elapsed_ms); - } } } + + for (auto & view : views) + { + if (!view.exception) + LOG_TRACE( + log, + "Pushing ({}) from {} to {} took {} ms.", + max_threads <= 1 ? "sequentially" : ("parallel " + std::to_string(max_threads)), + storage->getStorageID().getNameForLogs(), + view.table_id.getNameForLogs(), + view.runtime_stats.elapsed_ms); + } + if (exception_happened) checkExceptionsInViews(); From 47fb923975b4b827c1296e71a1781edfc5a3e5d3 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 13 Aug 2021 15:07:44 +0300 Subject: [PATCH 551/599] Some fixes, more tests --- .../AggregateFunctionMinMaxAny.h | 12 +----------- .../InterpreterSelectIntersectExceptQuery.cpp | 1 + .../NormalizeSelectWithUnionQueryVisitor.cpp | 3 --- .../SelectIntersectExceptQueryVisitor.cpp | 2 +- src/Parsers/ASTSelectWithUnionQuery.cpp | 3 --- src/Parsers/ExpressionListParsers.cpp | 10 +++++----- .../QueryPlan/IntersectOrExceptStep.cpp | 5 +++++ .../Transforms/IntersectOrExceptTransform.cpp | 5 ----- ...02004_intersect_except_operators.reference | 6 ++++++ .../02004_intersect_except_operators.sql | 2 ++ .../02007_test_any_all_operators.reference | 19 +++++++++++++++++++ .../02007_test_any_all_operators.sql | 12 ++++++++++++ 12 files changed, 52 insertions(+), 28 deletions(-) create mode 100644 tests/queries/0_stateless/02007_test_any_all_operators.reference create mode 100644 tests/queries/0_stateless/02007_test_any_all_operators.sql diff --git a/src/AggregateFunctions/AggregateFunctionMinMaxAny.h b/src/AggregateFunctions/AggregateFunctionMinMaxAny.h index 410f94c7afd..577b8127fd7 100644 --- a/src/AggregateFunctions/AggregateFunctionMinMaxAny.h +++ b/src/AggregateFunctions/AggregateFunctionMinMaxAny.h @@ -1032,17 +1032,7 @@ struct AggregateFunctionSingleValueOrNullData : Data #if USE_EMBEDDED_COMPILER - static constexpr bool is_compilable = Data::is_compilable; - - static void compileChangeIfBetter(llvm::IRBuilderBase & builder, llvm::Value * aggregate_data_ptr, llvm::Value * value_to_check) - { - Data::compileChangeFirstTime(builder, aggregate_data_ptr, value_to_check); - } - - static void compileChangeIfBetterMerge(llvm::IRBuilderBase & builder, llvm::Value * aggregate_data_dst_ptr, llvm::Value * aggregate_data_src_ptr) - { - Data::compileChangeFirstTimeMerge(builder, aggregate_data_dst_ptr, aggregate_data_src_ptr); - } + static constexpr bool is_compilable = false; #endif }; diff --git a/src/Interpreters/InterpreterSelectIntersectExceptQuery.cpp b/src/Interpreters/InterpreterSelectIntersectExceptQuery.cpp index 9e24dd6e6a0..34d7ae5b37f 100644 --- a/src/Interpreters/InterpreterSelectIntersectExceptQuery.cpp +++ b/src/Interpreters/InterpreterSelectIntersectExceptQuery.cpp @@ -17,6 +17,7 @@ namespace DB namespace ErrorCodes { extern const int INTERSECT_OR_EXCEPT_RESULT_STRUCTURES_MISMATCH; + extern const int LOGICAL_ERROR; } static Block getCommonHeader(const Blocks & headers) diff --git a/src/Interpreters/NormalizeSelectWithUnionQueryVisitor.cpp b/src/Interpreters/NormalizeSelectWithUnionQueryVisitor.cpp index bbe1a4e048c..0990667b2a8 100644 --- a/src/Interpreters/NormalizeSelectWithUnionQueryVisitor.cpp +++ b/src/Interpreters/NormalizeSelectWithUnionQueryVisitor.cpp @@ -36,9 +36,6 @@ void NormalizeSelectWithUnionQueryMatcher::visit(ASTSelectWithUnionQuery & ast, ASTs selects; auto & select_list = ast.list_of_selects->children; - if (select_list.size() < 2) - return; - int i; for (i = union_modes.size() - 1; i >= 0; --i) { diff --git a/src/Interpreters/SelectIntersectExceptQueryVisitor.cpp b/src/Interpreters/SelectIntersectExceptQueryVisitor.cpp index 5926d5f6f10..273bc327dc3 100644 --- a/src/Interpreters/SelectIntersectExceptQueryVisitor.cpp +++ b/src/Interpreters/SelectIntersectExceptQueryVisitor.cpp @@ -162,7 +162,7 @@ void SelectIntersectExceptQueryMatcher::visit(ASTSelectWithUnionQuery & ast, Dat children.emplace_back(std::move(right)); } - ast.union_mode = ASTSelectWithUnionQuery::Mode::ALL; + ast.union_mode = ASTSelectWithUnionQuery::Mode::Unspecified; ast.list_of_selects->children = std::move(children); ast.list_of_modes = std::move(modes); } diff --git a/src/Parsers/ASTSelectWithUnionQuery.cpp b/src/Parsers/ASTSelectWithUnionQuery.cpp index d19e860c16a..b882c738c9a 100644 --- a/src/Parsers/ASTSelectWithUnionQuery.cpp +++ b/src/Parsers/ASTSelectWithUnionQuery.cpp @@ -32,9 +32,6 @@ ASTPtr ASTSelectWithUnionQuery::clone() const void ASTSelectWithUnionQuery::formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const { - if (!list_of_selects || list_of_selects->children.size() != list_of_modes.size() + 1) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Incorrect ASTSelectWithUnionQuery"); - std::string indent_str = settings.one_line ? "" : std::string(4 * frame.indent, ' '); auto mode_to_str = [&](auto mode) diff --git a/src/Parsers/ExpressionListParsers.cpp b/src/Parsers/ExpressionListParsers.cpp index e75aad8d02f..33085379abb 100644 --- a/src/Parsers/ExpressionListParsers.cpp +++ b/src/Parsers/ExpressionListParsers.cpp @@ -186,13 +186,13 @@ enum class SubqueryFunctionType ALL }; -static bool modifyAST(String operator_name, std::shared_ptr & function, SubqueryFunctionType type) +static bool modifyAST(const String & operator_name, ASTPtr function, SubqueryFunctionType type) { // = ANY --> IN, != ALL --> NOT IN - if ((operator_name == "equals" && type == SubqueryFunctionType::ANY) - || (operator_name == "notEquals" && type == SubqueryFunctionType::ALL)) + if ((type == SubqueryFunctionType::ANY && operator_name == "equals") + || (type == SubqueryFunctionType::ALL && operator_name == "notEquals")) { - function->name = "in"; + assert_cast(function.get())->name = "in"; if (operator_name == "notEquals") { auto function_not = std::make_shared(); @@ -257,7 +257,7 @@ static bool modifyAST(String operator_name, std::shared_ptr & funct if (operator_name == "equals" || operator_name == "notEquals") { aggregate_function->name = "singleValueOrNull"; - function->name = "in"; + assert_cast(function.get())->name = "in"; if (operator_name == "notEquals") { auto function_not = std::make_shared(); diff --git a/src/Processors/QueryPlan/IntersectOrExceptStep.cpp b/src/Processors/QueryPlan/IntersectOrExceptStep.cpp index e4d04115cff..b75898b815b 100644 --- a/src/Processors/QueryPlan/IntersectOrExceptStep.cpp +++ b/src/Processors/QueryPlan/IntersectOrExceptStep.cpp @@ -12,6 +12,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + static Block checkHeaders(const DataStreams & input_streams_) { if (input_streams_.empty()) diff --git a/src/Processors/Transforms/IntersectOrExceptTransform.cpp b/src/Processors/Transforms/IntersectOrExceptTransform.cpp index b16032bde8e..abfd1a7f0ad 100644 --- a/src/Processors/Transforms/IntersectOrExceptTransform.cpp +++ b/src/Processors/Transforms/IntersectOrExceptTransform.cpp @@ -4,11 +4,6 @@ namespace DB { -namespace ErrorCodes -{ - extern const int SET_SIZE_LIMIT_EXCEEDED; -} - IntersectOrExceptTransform::IntersectOrExceptTransform(const Block & header_, Operator operator_) : IProcessor(InputPorts(2, header_), {header_}) , current_operator(operator_) diff --git a/tests/queries/0_stateless/02004_intersect_except_operators.reference b/tests/queries/0_stateless/02004_intersect_except_operators.reference index 9a9e4e1bf58..a097bd0076f 100644 --- a/tests/queries/0_stateless/02004_intersect_except_operators.reference +++ b/tests/queries/0_stateless/02004_intersect_except_operators.reference @@ -66,6 +66,8 @@ select number from numbers(100) intersect select number from numbers(20, 60) exc 57 58 59 +select * from (select 1 intersect select 1); +1 with (select number from numbers(10) intersect select 5) as a select a * 10; 50 select count() from (select number from numbers(10) except select 5); @@ -93,3 +95,7 @@ select * from (select 1 union all select 2 union all select 3 union all select 4 2 3 5 +select * from (select 1 union all select 2 union all select 3 union all select 4 intersect select 3 union all select 5 except select 1) order by 1; +2 +3 +5 diff --git a/tests/queries/0_stateless/02004_intersect_except_operators.sql b/tests/queries/0_stateless/02004_intersect_except_operators.sql index c88951ef353..4602dec7238 100644 --- a/tests/queries/0_stateless/02004_intersect_except_operators.sql +++ b/tests/queries/0_stateless/02004_intersect_except_operators.sql @@ -19,6 +19,7 @@ select 1 intersect select 1 except select 2 intersect select 1 except select 3 i select number from numbers(10) except select 5; select number from numbers(100) intersect select number from numbers(20, 60) except select number from numbers(30, 20) except select number from numbers(60, 20); +select * from (select 1 intersect select 1); with (select number from numbers(10) intersect select 5) as a select a * 10; select count() from (select number from numbers(10) except select 5); select count() from (select number from numbers(1000000) intersect select number from numbers(200000, 600000)); @@ -30,3 +31,4 @@ select 1 union all select 1 intersect select 1; select 1 union all select 1 intersect select 2; select * from (select 1 union all select 2 union all select 3 union all select 4 except select 3 union all select 5) order by 1; select * from (select 1 union all select 2 union all select 3 union all select 4 intersect select 3 union all select 5) order by 1; +select * from (select 1 union all select 2 union all select 3 union all select 4 intersect select 3 union all select 5 except select 1) order by 1; diff --git a/tests/queries/0_stateless/02007_test_any_all_operators.reference b/tests/queries/0_stateless/02007_test_any_all_operators.reference new file mode 100644 index 00000000000..cd36102cb80 --- /dev/null +++ b/tests/queries/0_stateless/02007_test_any_all_operators.reference @@ -0,0 +1,19 @@ +-- { echo } +select 1 == any (select number from numbers(10)); +1 +select 1 == any (select number from numbers(2, 10)); +0 +select 1 == all (select 1 from numbers(10)); +1 +select 1 == all (select number from numbers(10)); +0 +select number as a from numbers(10) where a == any (select number from numbers(3, 3)); +3 +4 +5 +-- TODO: Incorrect: +select 1 != any (select 1 from numbers(10)); +1 +select 1 != all (select 1 from numbers(10)); +1 +select number as a from numbers(10) where a != any (select number from numbers(3, 3)); diff --git a/tests/queries/0_stateless/02007_test_any_all_operators.sql b/tests/queries/0_stateless/02007_test_any_all_operators.sql new file mode 100644 index 00000000000..08fc929bab9 --- /dev/null +++ b/tests/queries/0_stateless/02007_test_any_all_operators.sql @@ -0,0 +1,12 @@ +-- { echo } +select 1 == any (select number from numbers(10)); +select 1 == any (select number from numbers(2, 10)); +select 1 == all (select 1 from numbers(10)); +select 1 == all (select number from numbers(10)); +select number as a from numbers(10) where a == any (select number from numbers(3, 3)); + +-- TODO: Incorrect: +select 1 != any (select 1 from numbers(10)); +select 1 != all (select 1 from numbers(10)); +select number as a from numbers(10) where a != any (select number from numbers(3, 3)); + From d1b93f8b0cbf263bde534845c244b8d6f8b4b1db Mon Sep 17 00:00:00 2001 From: Vladimir Goncharov Date: Fri, 13 Aug 2021 15:32:13 +0300 Subject: [PATCH 552/599] Update docs/ru/engines/table-engines/mergetree-family/mergetree.md Co-authored-by: Alexey Boykov <33257111+mathalex@users.noreply.github.com> --- docs/ru/engines/table-engines/mergetree-family/mergetree.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/engines/table-engines/mergetree-family/mergetree.md b/docs/ru/engines/table-engines/mergetree-family/mergetree.md index 728c632370c..b2290369b4c 100644 --- a/docs/ru/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/ru/engines/table-engines/mergetree-family/mergetree.md @@ -379,7 +379,7 @@ INDEX b (u64 * length(str), i32 + f64 * 100, date, str) TYPE set(100) GRANULARIT Проекции похожи на материализованные представления, но определяются на уровне партов. Это обеспечивает гарантии согласованности наряду с автоматическим использованием в запросах. #### Запрос {#projection-query} -Запрос проекции - это то, что определяет проекцию. Он имеет следующую грамматику: +Запрос проекции — это то, что определяет проекцию. Он имеет следующую грамматику: `SELECT [GROUP BY] [ORDER BY]` From 942bd534d3279076b26d8e3a5b3969633f56a640 Mon Sep 17 00:00:00 2001 From: Vladimir Goncharov Date: Fri, 13 Aug 2021 15:32:23 +0300 Subject: [PATCH 553/599] Update docs/ru/engines/table-engines/mergetree-family/mergetree.md Co-authored-by: Alexey Boykov <33257111+mathalex@users.noreply.github.com> --- docs/ru/engines/table-engines/mergetree-family/mergetree.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/engines/table-engines/mergetree-family/mergetree.md b/docs/ru/engines/table-engines/mergetree-family/mergetree.md index b2290369b4c..7493c1e5fc0 100644 --- a/docs/ru/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/ru/engines/table-engines/mergetree-family/mergetree.md @@ -383,7 +383,7 @@ INDEX b (u64 * length(str), i32 + f64 * 100, date, str) TYPE set(100) GRANULARIT `SELECT [GROUP BY] [ORDER BY]` -Это неявно выбирает данные из родительской таблицы. +Он неявно выбирает данные из родительской таблицы. #### Хранение {#projection-storage} Проекции хранятся в каталоге парта. Это похоже на хранение индексов, но используется подкаталог, в котором хранится анонимный парт таблицы MergeTree. Таблица создается запросом определения проекции. Если есть конструкция GROUP BY, то базовый механизм хранения становится AggregatedMergeTree, а все агрегатные функции преобразуются в AggregateFunction. Если есть конструкция ORDER BY, таблица MergeTree будет использовать его в качестве выражения первичного ключа. Во время процесса слияния парт проекции будет слит с помощью процедуры слияния ее хранилища. Контрольная сумма парта родительской таблицы будет включать парт проекции. Другие процедуры аналогичны индексам пропуска данных. From ff3afb410ddf93f01b8249170e106ff2bd485ece Mon Sep 17 00:00:00 2001 From: Vladimir Goncharov Date: Fri, 13 Aug 2021 15:33:03 +0300 Subject: [PATCH 554/599] Update docs/ru/engines/table-engines/mergetree-family/mergetree.md Co-authored-by: Alexey Boykov <33257111+mathalex@users.noreply.github.com> --- docs/ru/engines/table-engines/mergetree-family/mergetree.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/engines/table-engines/mergetree-family/mergetree.md b/docs/ru/engines/table-engines/mergetree-family/mergetree.md index 7493c1e5fc0..8b7d9bb736c 100644 --- a/docs/ru/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/ru/engines/table-engines/mergetree-family/mergetree.md @@ -389,7 +389,7 @@ INDEX b (u64 * length(str), i32 + f64 * 100, date, str) TYPE set(100) GRANULARIT Проекции хранятся в каталоге парта. Это похоже на хранение индексов, но используется подкаталог, в котором хранится анонимный парт таблицы MergeTree. Таблица создается запросом определения проекции. Если есть конструкция GROUP BY, то базовый механизм хранения становится AggregatedMergeTree, а все агрегатные функции преобразуются в AggregateFunction. Если есть конструкция ORDER BY, таблица MergeTree будет использовать его в качестве выражения первичного ключа. Во время процесса слияния парт проекции будет слит с помощью процедуры слияния ее хранилища. Контрольная сумма парта родительской таблицы будет включать парт проекции. Другие процедуры аналогичны индексам пропуска данных. #### Анализ запросов {#projection-query-analysis} -1. Проверить, можно ли использовать проекцию для результата на данный запрос, то есть она даёт тот же результат, что и запрос к базовой таблице. +1. Проверить, можно ли использовать проекцию в данном запросе, то есть, что с ней выходит тот же результат, что и с запросом к базовой таблице. 2. Выбрать наиболее подходящее совпадение, содержащее наименьшее количество гранул для чтения. 3. План запроса, который использует проекции, будет отличаться от того, который использует исходные парты. Если в некоторых партах проекция отсутствует, мы можем расширить план, чтобы «проецировать» на лету. From af8fcaf26637cc1f826b3241caaecd0dca3d62b0 Mon Sep 17 00:00:00 2001 From: Vladimir Goncharov Date: Fri, 13 Aug 2021 15:33:52 +0300 Subject: [PATCH 555/599] Update docs/ru/engines/table-engines/mergetree-family/mergetree.md Co-authored-by: Alexey Boykov <33257111+mathalex@users.noreply.github.com> --- docs/ru/engines/table-engines/mergetree-family/mergetree.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/engines/table-engines/mergetree-family/mergetree.md b/docs/ru/engines/table-engines/mergetree-family/mergetree.md index 8b7d9bb736c..db6eb8154ba 100644 --- a/docs/ru/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/ru/engines/table-engines/mergetree-family/mergetree.md @@ -391,7 +391,7 @@ INDEX b (u64 * length(str), i32 + f64 * 100, date, str) TYPE set(100) GRANULARIT #### Анализ запросов {#projection-query-analysis} 1. Проверить, можно ли использовать проекцию в данном запросе, то есть, что с ней выходит тот же результат, что и с запросом к базовой таблице. 2. Выбрать наиболее подходящее совпадение, содержащее наименьшее количество гранул для чтения. -3. План запроса, который использует проекции, будет отличаться от того, который использует исходные парты. Если в некоторых партах проекция отсутствует, мы можем расширить план, чтобы «проецировать» на лету. +3. План запроса, который использует проекции, будет отличаться от того, который использует исходные парты. При отсутствии проекции в некоторых партах можно расширить план, чтобы «проецировать» на лету. ## Конкурентный доступ к данным {#concurrent-data-access} From 1d3c11f6a136cff7751186c6f5831198a3e638d8 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Fri, 13 Aug 2021 15:43:22 +0300 Subject: [PATCH 556/599] Update docs/ru/engines/database-engines/materialized-mysql.md Co-authored-by: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> --- docs/ru/engines/database-engines/materialized-mysql.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/engines/database-engines/materialized-mysql.md b/docs/ru/engines/database-engines/materialized-mysql.md index b82a3dae553..1cd864c01e9 100644 --- a/docs/ru/engines/database-engines/materialized-mysql.md +++ b/docs/ru/engines/database-engines/materialized-mysql.md @@ -5,7 +5,7 @@ toc_title: "[experimental] MaterializedMySQL" # [экспериментальный] MaterializedMySQL {#materialized-mysql} -**Это экспериментальный движок, который не следует использовать в продуктивной среде.** +**Это экспериментальный движок, который не следует использовать в продакшене.** Создает базу данных ClickHouse со всеми таблицами, существующими в MySQL, и всеми данными в этих таблицах. From f20660455cd94be141c5d94831efec624b62ec5d Mon Sep 17 00:00:00 2001 From: Denis Zhuravlev Date: Fri, 13 Aug 2021 10:24:30 -0300 Subject: [PATCH 557/599] fix failded tests --- tests/integration/test_system_flush_logs/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_system_flush_logs/test.py b/tests/integration/test_system_flush_logs/test.py index b69105710fb..a4d70339c09 100644 --- a/tests/integration/test_system_flush_logs/test.py +++ b/tests/integration/test_system_flush_logs/test.py @@ -10,12 +10,12 @@ node = cluster.add_instance('node_default') system_logs = [ # disabled by default - ('system.part_log', 0), ('system.text_log', 0), # enabled by default ('system.query_log', 1), ('system.query_thread_log', 1), + ('system.part_log', 1), ('system.trace_log', 1), ('system.metric_log', 1), ] From 67f5889b91b560dbb74cb9ff564ff996a04fc0ae Mon Sep 17 00:00:00 2001 From: Denis Zhuravlev Date: Fri, 13 Aug 2021 10:25:27 -0300 Subject: [PATCH 558/599] fix failed tests --- .../configs/config_without_standard_part_log.xml | 3 +++ tests/integration/test_part_log_table/test.py | 2 +- 2 files changed, 4 insertions(+), 1 deletion(-) create mode 100644 tests/integration/test_part_log_table/configs/config_without_standard_part_log.xml diff --git a/tests/integration/test_part_log_table/configs/config_without_standard_part_log.xml b/tests/integration/test_part_log_table/configs/config_without_standard_part_log.xml new file mode 100644 index 00000000000..6c718dad1ee --- /dev/null +++ b/tests/integration/test_part_log_table/configs/config_without_standard_part_log.xml @@ -0,0 +1,3 @@ + + + diff --git a/tests/integration/test_part_log_table/test.py b/tests/integration/test_part_log_table/test.py index 63adde432b5..050e8c831c7 100644 --- a/tests/integration/test_part_log_table/test.py +++ b/tests/integration/test_part_log_table/test.py @@ -3,7 +3,7 @@ import pytest from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) -node1 = cluster.add_instance("node1") +node1 = cluster.add_instance("node1", main_configs=["configs/config_without_standard_part_log.xml"]) node2 = cluster.add_instance("node2", main_configs=["configs/config_with_standard_part_log.xml"]) node3 = cluster.add_instance("node3", main_configs=["configs/config_with_non_standard_part_log.xml"]) From 30191a7c32525890fdcc6a6089051fcb412e9585 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 13 Aug 2021 13:57:06 +0000 Subject: [PATCH 559/599] Fix --- src/Dictionaries/XDBCDictionarySource.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Dictionaries/XDBCDictionarySource.cpp b/src/Dictionaries/XDBCDictionarySource.cpp index 80d3df1caab..e79e55910b7 100644 --- a/src/Dictionaries/XDBCDictionarySource.cpp +++ b/src/Dictionaries/XDBCDictionarySource.cpp @@ -222,7 +222,7 @@ Pipe XDBCDictionarySource::loadFromQuery(const Poco::URI & url, const Block & re }; auto read_buf = std::make_unique(url, Poco::Net::HTTPRequest::HTTP_POST, write_body_callback, timeouts); - auto format = FormatFactory::instance().getInput(IXDBCBridgeHelper::DEFAULT_FORMAT, *read_buf, sample_block, getContext(), max_block_size); + auto format = FormatFactory::instance().getInput(IXDBCBridgeHelper::DEFAULT_FORMAT, *read_buf, required_sample_block, getContext(), max_block_size); format->addBuffer(std::move(read_buf)); return Pipe(std::move(format)); From 73cb7d55ec8a9bc5ac4ee9a8cfaf2230915e9276 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 13 Aug 2021 16:18:46 +0200 Subject: [PATCH 560/599] Disable arrayJoin on partition expressions --- src/Storages/MergeTree/MergeTreeData.cpp | 13 +++++++++++++ .../02009_array_join_partition.reference | 0 .../0_stateless/02009_array_join_partition.sql | 4 ++++ 3 files changed, 17 insertions(+) create mode 100644 tests/queries/0_stateless/02009_array_join_partition.reference create mode 100644 tests/queries/0_stateless/02009_array_join_partition.sql diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 4730bf9f47c..ebd961ab428 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -3207,6 +3207,16 @@ Pipe MergeTreeData::alterPartition( return {}; } +void checkPartitionExpressionFunction(const ASTPtr & ast) +{ + if (const auto * func = ast->as()) + if (func->name == "arrayJoin") + throw Exception("The partition expression cannot contain array joins", ErrorCodes::INVALID_PARTITION_VALUE); + for (const auto & child : ast->children) + checkPartitionExpressionFunction(child); +} + + String MergeTreeData::getPartitionIDFromQuery(const ASTPtr & ast, ContextPtr local_context) const { const auto & partition_ast = ast->as(); @@ -3217,6 +3227,9 @@ String MergeTreeData::getPartitionIDFromQuery(const ASTPtr & ast, ContextPtr loc return partition_ast.id; } + if (const auto * partition_function = partition_ast.value->as()) + checkPartitionExpressionFunction(ast); + if (format_version < MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING) { /// Month-partitioning specific - partition ID can be passed in the partition value. diff --git a/tests/queries/0_stateless/02009_array_join_partition.reference b/tests/queries/0_stateless/02009_array_join_partition.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02009_array_join_partition.sql b/tests/queries/0_stateless/02009_array_join_partition.sql new file mode 100644 index 00000000000..a78efe96f66 --- /dev/null +++ b/tests/queries/0_stateless/02009_array_join_partition.sql @@ -0,0 +1,4 @@ +CREATE TABLE table_2009_part (`i` Int64, `d` Date, `s` String) ENGINE = MergeTree PARTITION BY toYYYYMM(d) ORDER BY i; + +ALTER TABLE table_2009_part ATTACH PARTITION tuple(arrayJoin([0, 1])); -- {serverError 248} +ALTER TABLE table_2009_part ATTACH PARTITION tuple(toYYYYMM(toDate([arrayJoin([arrayJoin([arrayJoin([arrayJoin([3, materialize(NULL), arrayJoin([1025, materialize(NULL), materialize(NULL)]), NULL])])]), materialize(NULL)])], NULL))); -- {serverError 248} From c534363abe05b348e0e1a3100d9af80f6f5a0088 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 13 Aug 2021 20:33:10 +0300 Subject: [PATCH 561/599] fix intersecting parts --- .../MergeTree/ReplicatedMergeTreeQueue.cpp | 16 ++++++++++++++++ 1 file changed, 16 insertions(+) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index ea5f7cfc36a..ef276a53df2 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -1012,8 +1012,24 @@ bool ReplicatedMergeTreeQueue::isNotCoveredByFuturePartsImpl(const String & log_ bool ReplicatedMergeTreeQueue::addFuturePartIfNotCoveredByThem(const String & part_name, LogEntry & entry, String & reject_reason) { + /// We have found `part_name` on some replica and are going to fetch it instead of covered `entry->new_part_name`. std::lock_guard lock(state_mutex); + if (virtual_parts.getContainingPart(part_name).empty()) + { + /// We should not fetch any parts that absent in our `virtual_parts` set, + /// because we do not know about such parts according to our replication queue (we know about them from some side-channel). + /// Otherwise, it may break invariants in replication queue reordering, for example: + /// 1. Our queue contains GET_PART all_2_2_0, log contains DROP_RANGE all_2_2_0 and MERGE_PARTS all_1_3_1 + /// 2. We execute GET_PART all_2_2_0, but fetch all_1_3_1 instead + /// (drop_ranges.isAffectedByDropRange(...) is false-negative, because DROP_RANGE all_2_2_0 is not pulled yet). + /// It actually means, that MERGE_PARTS all_1_3_1 is executed too, but it's not even pulled yet. + /// 3. Then we pull log, trying to execute DROP_RANGE all_2_2_0 + /// and reveal that it was incorrectly reordered with MERGE_PARTS all_1_3_1 (drop range intersects merged part). + reject_reason = fmt::format("Log entry for part {} or covering part is not pulled from log to queue yet.", part_name); + return false; + } + /// FIXME get rid of actual_part_name. /// If new covering part jumps over DROP_RANGE we should execute drop range first if (drop_ranges.isAffectedByDropRange(part_name, reject_reason)) From 382fd7d4aca4a5220c9a227beffc6a806bcea031 Mon Sep 17 00:00:00 2001 From: Alexey Date: Thu, 5 Aug 2021 19:55:19 +0000 Subject: [PATCH 562/599] translated parts in MergeTree --- docs/en/engines/table-engines/mergetree-family/mergetree.md | 2 +- docs/ru/engines/table-engines/mergetree-family/mergetree.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/engines/table-engines/mergetree-family/mergetree.md b/docs/en/engines/table-engines/mergetree-family/mergetree.md index 0c900454cd0..1b1313e625c 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -79,7 +79,7 @@ For a description of parameters, see the [CREATE query description](../../../sql - `SAMPLE BY` — An expression for sampling. Optional. - If a sampling expression is used, the primary key must contain it. The result of sampling expression must be unsigned integer. Example: `SAMPLE BY intHash32(UserID) ORDER BY (CounterID, EventDate, intHash32(UserID))`. + If a sampling expression is used, the primary key must contain it. The result of a sampling expression must be an unsigned integer. Example: `SAMPLE BY intHash32(UserID) ORDER BY (CounterID, EventDate, intHash32(UserID))`. - `TTL` — A list of rules specifying storage duration of rows and defining logic of automatic parts movement [between disks and volumes](#table_engine-mergetree-multiple-volumes). Optional. diff --git a/docs/ru/engines/table-engines/mergetree-family/mergetree.md b/docs/ru/engines/table-engines/mergetree-family/mergetree.md index db6eb8154ba..61ed34b686c 100644 --- a/docs/ru/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/ru/engines/table-engines/mergetree-family/mergetree.md @@ -68,7 +68,7 @@ ORDER BY expr - `SAMPLE BY` — выражение для сэмплирования. Необязательный параметр. - Если используется выражение для сэмплирования, то первичный ключ должен содержать его. Пример: `SAMPLE BY intHash32(UserID) ORDER BY (CounterID, EventDate, intHash32(UserID))`. + Если используется выражение для сэмплирования, то первичный ключ должен содержать его. Результат выражения для сэмплирования должен быть беззнаковым целым числом. Пример: `SAMPLE BY intHash32(UserID) ORDER BY (CounterID, EventDate, intHash32(UserID))`. - `TTL` — список правил, определяющих длительности хранения строк, а также задающих правила перемещения частей на определённые тома или диски. Необязательный параметр. From 200d75646a2af8b9aa5c62a18927121eefda2307 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 13 Aug 2021 20:57:40 +0000 Subject: [PATCH 563/599] Fix --- src/Databases/PostgreSQL/DatabasePostgreSQL.cpp | 4 ++-- src/Storages/StorageExternalDistributed.cpp | 2 +- src/Storages/StoragePostgreSQL.cpp | 3 --- src/Storages/StoragePostgreSQL.h | 2 -- src/TableFunctions/TableFunctionPostgreSQL.cpp | 1 - 5 files changed, 3 insertions(+), 9 deletions(-) diff --git a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp index c848c784712..259648f4399 100644 --- a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp +++ b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp @@ -164,7 +164,7 @@ StoragePtr DatabasePostgreSQL::tryGetTable(const String & table_name, ContextPtr } -StoragePtr DatabasePostgreSQL::fetchTable(const String & table_name, ContextPtr local_context, const bool table_checked) const +StoragePtr DatabasePostgreSQL::fetchTable(const String & table_name, ContextPtr, const bool table_checked) const { if (!cache_tables || !cached_tables.count(table_name)) { @@ -179,7 +179,7 @@ StoragePtr DatabasePostgreSQL::fetchTable(const String & table_name, ContextPtr auto storage = StoragePostgreSQL::create( StorageID(database_name, table_name), pool, table_name, - ColumnsDescription{*columns}, ConstraintsDescription{}, String{}, local_context, postgres_schema); + ColumnsDescription{*columns}, ConstraintsDescription{}, String{}, postgres_schema); if (cache_tables) cached_tables[table_name] = storage; diff --git a/src/Storages/StorageExternalDistributed.cpp b/src/Storages/StorageExternalDistributed.cpp index 32b9c7e9245..f20e49fe23a 100644 --- a/src/Storages/StorageExternalDistributed.cpp +++ b/src/Storages/StorageExternalDistributed.cpp @@ -98,7 +98,7 @@ StorageExternalDistributed::StorageExternalDistributed( context->getSettingsRef().postgresql_connection_pool_size, context->getSettingsRef().postgresql_connection_pool_wait_timeout); - shard = StoragePostgreSQL::create(table_id_, std::move(pool), remote_table, columns_, constraints_, String{}, context); + shard = StoragePostgreSQL::create(table_id_, std::move(pool), remote_table, columns_, constraints_, String{}); break; } #endif diff --git a/src/Storages/StoragePostgreSQL.cpp b/src/Storages/StoragePostgreSQL.cpp index 5f8b81a47cf..603a52b2801 100644 --- a/src/Storages/StoragePostgreSQL.cpp +++ b/src/Storages/StoragePostgreSQL.cpp @@ -47,12 +47,10 @@ StoragePostgreSQL::StoragePostgreSQL( const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, const String & comment, - ContextPtr context_, const String & remote_table_schema_) : IStorage(table_id_) , remote_table_name(remote_table_name_) , remote_table_schema(remote_table_schema_) - , global_context(context_) , pool(std::move(pool_)) { StorageInMemoryMetadata storage_metadata; @@ -347,7 +345,6 @@ void registerStoragePostgreSQL(StorageFactory & factory) args.columns, args.constraints, args.comment, - args.getContext(), remote_table_schema); }, { diff --git a/src/Storages/StoragePostgreSQL.h b/src/Storages/StoragePostgreSQL.h index 064fa481f9d..bd5cd317c3d 100644 --- a/src/Storages/StoragePostgreSQL.h +++ b/src/Storages/StoragePostgreSQL.h @@ -27,7 +27,6 @@ public: const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, const String & comment, - ContextPtr context_, const std::string & remote_table_schema_ = ""); String getName() const override { return "PostgreSQL"; } @@ -48,7 +47,6 @@ private: String remote_table_name; String remote_table_schema; - ContextPtr global_context; postgres::PoolWithFailoverPtr pool; }; diff --git a/src/TableFunctions/TableFunctionPostgreSQL.cpp b/src/TableFunctions/TableFunctionPostgreSQL.cpp index ceea29b335b..d701728479b 100644 --- a/src/TableFunctions/TableFunctionPostgreSQL.cpp +++ b/src/TableFunctions/TableFunctionPostgreSQL.cpp @@ -37,7 +37,6 @@ StoragePtr TableFunctionPostgreSQL::executeImpl(const ASTPtr & /*ast_function*/, columns, ConstraintsDescription{}, String{}, - context, remote_table_schema); result->startup(); From e39bef97b50467ea8eef8164f450239eb9709f99 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sat, 14 Aug 2021 02:49:45 +0300 Subject: [PATCH 564/599] Update CHANGELOG.md --- CHANGELOG.md | 32 ++++++++++---------------------- 1 file changed, 10 insertions(+), 22 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 10ed4dae1c9..12fda73fe13 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -2,15 +2,15 @@ #### New Features -* Collect common system metrics (in `system.asynchronous_metrics` and `system.asynchronous_metric_log`) on CPU usage, disk usage, memory usage, IO, network, files, load average, CPU frequencies, thermal sensors, EDAC counters, system uptime; also added metrics about the scheduling jitter and the time spent collecting the metrics. It works similar to `atop` in ClickHouse and allows access to monitoring data even if you have no additional tools installed. Close [#9430](https://github.com/ClickHouse/ClickHouse/issues/9430). [#24416](https://github.com/ClickHouse/ClickHouse/pull/24416) ([Yegor Levankov](https://github.com/elevankoff)). +* Add support for a part of SQL/JSON standard. [#24148](https://github.com/ClickHouse/ClickHouse/pull/24148) ([l1tsolaiki](https://github.com/l1tsolaiki)). +* Collect common system metrics (in `system.asynchronous_metrics` and `system.asynchronous_metric_log`) on CPU usage, disk usage, memory usage, IO, network, files, load average, CPU frequencies, thermal sensors, EDAC counters, system uptime; also added metrics about the scheduling jitter and the time spent collecting the metrics. It works similar to `atop` in ClickHouse and allows access to monitoring data even if you have no additional tools installed. Close [#9430](https://github.com/ClickHouse/ClickHouse/issues/9430). [#24416](https://github.com/ClickHouse/ClickHouse/pull/24416) (([alexey-milovidov](https://github.com/alexey-milovidov)), [Yegor Levankov](https://github.com/elevankoff), [Kseniia Sumarokova](https://github.com/kssenii)). +* Add MaterializedPostgreSQL table engine and database engine. This database engine allows replicating a whole database or any subset of database tables. [#20470](https://github.com/ClickHouse/ClickHouse/pull/20470) ([Kseniia Sumarokova](https://github.com/kssenii)). * Add new functions `leftPad()`, `rightPad()`, `leftPadUTF8()`, `rightPadUTF8()`. [#26075](https://github.com/ClickHouse/ClickHouse/pull/26075) ([Vitaly Baranov](https://github.com/vitlibar)). * Add the `FIRST` keyword to the `ADD INDEX` command to be able to add the index at the beginning of the indices list. [#25904](https://github.com/ClickHouse/ClickHouse/pull/25904) ([xjewer](https://github.com/xjewer)). * Introduce `system.data_skipping_indices` table containing information about existing data skipping indices. Close [#7659](https://github.com/ClickHouse/ClickHouse/issues/7659). [#25693](https://github.com/ClickHouse/ClickHouse/pull/25693) ([Dmitry Novik](https://github.com/novikd)). * Add `bin`/`unbin` functions. [#25609](https://github.com/ClickHouse/ClickHouse/pull/25609) ([zhaoyu](https://github.com/zxc111)). * Support `Map` and `(U)Int128`, `U(Int256) types in `mapAdd` and `mapSubtract` functions. [#25596](https://github.com/ClickHouse/ClickHouse/pull/25596) ([Ildus Kurbangaliev](https://github.com/ildus)). * Support `DISTINCT ON (columns)` expression, close [#25404](https://github.com/ClickHouse/ClickHouse/issues/25404). [#25589](https://github.com/ClickHouse/ClickHouse/pull/25589) ([Zijie Lu](https://github.com/TszKitLo40)). -* Add support for a part of SQLJSON standard. [#24148](https://github.com/ClickHouse/ClickHouse/pull/24148) ([l1tsolaiki](https://github.com/l1tsolaiki)). -* Add MaterializedPostgreSQL table engine and database engine. This database engine allows replicating a whole database or any subset of database tables. [#20470](https://github.com/ClickHouse/ClickHouse/pull/20470) ([Kseniia Sumarokova](https://github.com/kssenii)). * Add an ability to reset a custom setting to default and remove it from the table's metadata. It allows rolling back the change without knowing the system/config's default. Closes [#14449](https://github.com/ClickHouse/ClickHouse/issues/14449). [#17769](https://github.com/ClickHouse/ClickHouse/pull/17769) ([xjewer](https://github.com/xjewer)). * Render pipelines as graphs in Web UI if `EXPLAIN PIPELINE graph = 1` query is submitted. [#26067](https://github.com/ClickHouse/ClickHouse/pull/26067) ([alexey-milovidov](https://github.com/alexey-milovidov)). @@ -21,11 +21,11 @@ #### Improvements -* Use `Map` data type for system logs tables (`system.query_log`, `system.query_thread_log`, `system.processes`, `system.opentelemetry_span_log`). These tables will be auto-created with new data types. Virtual columns are created to support old queries. Closes [#18698](https://github.com/ClickHouse/ClickHouse/issues/18698). [#23934](https://github.com/ClickHouse/ClickHouse/pull/23934), [#25773](https://github.com/ClickHouse/ClickHouse/pull/25773) ([hexiaoting](https://github.com/hexiaoting), [sundy-li](https://github.com/sundy-li)). +* Use `Map` data type for system logs tables (`system.query_log`, `system.query_thread_log`, `system.processes`, `system.opentelemetry_span_log`). These tables will be auto-created with new data types. Virtual columns are created to support old queries. Closes [#18698](https://github.com/ClickHouse/ClickHouse/issues/18698). [#23934](https://github.com/ClickHouse/ClickHouse/pull/23934), [#25773](https://github.com/ClickHouse/ClickHouse/pull/25773) ([hexiaoting](https://github.com/hexiaoting), [sundy-li](https://github.com/sundy-li), [Maksim Kita](https://github.com/kitaisreal)). * For a dictionary with a complex key containing only one attribute, allow not wrapping the key expression in tuple for functions `dictGet`, `dictHas`. [#26130](https://github.com/ClickHouse/ClickHouse/pull/26130) ([Maksim Kita](https://github.com/kitaisreal)). * Implement function `bin`/`hex` from `AggregateFunction` states. [#26094](https://github.com/ClickHouse/ClickHouse/pull/26094) ([zhaoyu](https://github.com/zxc111)). * Support arguments of `UUID` type for `empty` and `notEmpty` functions. `UUID` is empty if it is all zeros (nil UUID). Closes [#3446](https://github.com/ClickHouse/ClickHouse/issues/3446). [#25974](https://github.com/ClickHouse/ClickHouse/pull/25974) ([zhaoyu](https://github.com/zxc111)). -* Fix error with query `SET SQL_SELECT_LIMIT` in MySQL protocol. Closes [#17115](https://github.com/ClickHouse/ClickHouse/issues/17115). [#25972](https://github.com/ClickHouse/ClickHouse/pull/25972) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Add support for `SET SQL_SELECT_LIMIT` in MySQL protocol. Closes [#17115](https://github.com/ClickHouse/ClickHouse/issues/17115). [#25972](https://github.com/ClickHouse/ClickHouse/pull/25972) ([Kseniia Sumarokova](https://github.com/kssenii)). * More instrumentation for network interaction: add counters for recv/send bytes; add gauges for recvs/sends. Added missing documentation. Close [#5897](https://github.com/ClickHouse/ClickHouse/issues/5897). [#25962](https://github.com/ClickHouse/ClickHouse/pull/25962) ([alexey-milovidov](https://github.com/alexey-milovidov)). * Add setting `optimize_move_to_prewhere_if_final`. If query has `FINAL`, the optimization `move_to_prewhere` will be enabled only if both `optimize_move_to_prewhere` and `optimize_move_to_prewhere_if_final` are enabled. Closes [#8684](https://github.com/ClickHouse/ClickHouse/issues/8684). [#25940](https://github.com/ClickHouse/ClickHouse/pull/25940) ([Kseniia Sumarokova](https://github.com/kssenii)). * Allow complex quoted identifiers of JOINed tables. Close [#17861](https://github.com/ClickHouse/ClickHouse/issues/17861). [#25924](https://github.com/ClickHouse/ClickHouse/pull/25924) ([alexey-milovidov](https://github.com/alexey-milovidov)). @@ -37,7 +37,7 @@ * Support for queries with a column named `"null"` (it must be specified in back-ticks or double quotes) and `ON CLUSTER`. Closes [#24035](https://github.com/ClickHouse/ClickHouse/issues/24035). [#25907](https://github.com/ClickHouse/ClickHouse/pull/25907) ([alexey-milovidov](https://github.com/alexey-milovidov)). * Support `LowCardinality`, `Decimal`, and `UUID` for `JSONExtract`. Closes [#24606](https://github.com/ClickHouse/ClickHouse/issues/24606). [#25900](https://github.com/ClickHouse/ClickHouse/pull/25900) ([Kseniia Sumarokova](https://github.com/kssenii)). * Convert history file from `readline` format to `replxx` format. [#25888](https://github.com/ClickHouse/ClickHouse/pull/25888) ([Azat Khuzhin](https://github.com/azat)). -* Fix bug which can lead to intersecting parts after `DROP PART` or background deletion of an empty part. [#25884](https://github.com/ClickHouse/ClickHouse/pull/25884) ([alesapin](https://github.com/alesapin)). +* Fix an issue which can lead to intersecting parts after `DROP PART` or background deletion of an empty part. [#25884](https://github.com/ClickHouse/ClickHouse/pull/25884) ([alesapin](https://github.com/alesapin)). * Better handling of lost parts for `ReplicatedMergeTree` tables. Fixes rare inconsistencies in `ReplicationQueue`. Fixes [#10368](https://github.com/ClickHouse/ClickHouse/issues/10368). [#25820](https://github.com/ClickHouse/ClickHouse/pull/25820) ([alesapin](https://github.com/alesapin)). * Allow starting clickhouse-client with unreadable working directory. [#25817](https://github.com/ClickHouse/ClickHouse/pull/25817) ([ianton-ru](https://github.com/ianton-ru)). * Fix "No available columns" error for `Merge` storage. [#25801](https://github.com/ClickHouse/ClickHouse/pull/25801) ([Azat Khuzhin](https://github.com/azat)). @@ -48,7 +48,7 @@ * Support materialized and aliased columns in JOIN, close [#13274](https://github.com/ClickHouse/ClickHouse/issues/13274). [#25634](https://github.com/ClickHouse/ClickHouse/pull/25634) ([Vladimir C](https://github.com/vdimir)). * Fix possible logical race condition between `ALTER TABLE ... DETACH` and background merges. [#25605](https://github.com/ClickHouse/ClickHouse/pull/25605) ([Azat Khuzhin](https://github.com/azat)). * Make `NetworkReceiveElapsedMicroseconds` metric to correctly include the time spent waiting for data from the client to `INSERT`. Close [#9958](https://github.com/ClickHouse/ClickHouse/issues/9958). [#25602](https://github.com/ClickHouse/ClickHouse/pull/25602) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Support `TRUNCATE TABLE` for StorageS3 and StorageHDFS. Close [#25530](https://github.com/ClickHouse/ClickHouse/issues/25530). [#25550](https://github.com/ClickHouse/ClickHouse/pull/25550) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Support `TRUNCATE TABLE` for S3 and HDFS. Close [#25530](https://github.com/ClickHouse/ClickHouse/issues/25530). [#25550](https://github.com/ClickHouse/ClickHouse/pull/25550) ([Kseniia Sumarokova](https://github.com/kssenii)). * Support for dynamic reloading of config to change number of threads in pool for background jobs execution (merges, mutations, fetches). [#25548](https://github.com/ClickHouse/ClickHouse/pull/25548) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). * Allow extracting of non-string element as string using `JSONExtract`. This is for [#25414](https://github.com/ClickHouse/ClickHouse/issues/25414). [#25452](https://github.com/ClickHouse/ClickHouse/pull/25452) ([Amos Bird](https://github.com/amosbird)). * Support regular expression in `Database` argument for `StorageMerge`. Close [#776](https://github.com/ClickHouse/ClickHouse/issues/776). [#25064](https://github.com/ClickHouse/ClickHouse/pull/25064) ([flynn](https://github.com/ucasfl)). @@ -60,13 +60,13 @@ * Fix incorrect `SET ROLE` in some cases. [#26707](https://github.com/ClickHouse/ClickHouse/pull/26707) ([Vitaly Baranov](https://github.com/vitlibar)). * Fix potential `nullptr` dereference in window functions. Fix [#25276](https://github.com/ClickHouse/ClickHouse/issues/25276). [#26668](https://github.com/ClickHouse/ClickHouse/pull/26668) ([Alexander Kuzmenkov](https://github.com/akuzm)). * Fix incorrect function names of `groupBitmapAnd/Or/Xor`. Fix [#26557](https://github.com/ClickHouse/ClickHouse/pull/26557) ([Amos Bird](https://github.com/amosbird)). -* Fix crash in rabbitmq shutdown in case rabbitmq setup was not started. Closes [#26504](https://github.com/ClickHouse/ClickHouse/issues/26504). [#26529](https://github.com/ClickHouse/ClickHouse/pull/26529) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix crash in RabbitMQ shutdown in case RabbitMQ setup was not started. Closes [#26504](https://github.com/ClickHouse/ClickHouse/issues/26504). [#26529](https://github.com/ClickHouse/ClickHouse/pull/26529) ([Kseniia Sumarokova](https://github.com/kssenii)). * Fix issues with `CREATE DICTIONARY` query if dictionary name or database name was quoted. Closes [#26491](https://github.com/ClickHouse/ClickHouse/issues/26491). [#26508](https://github.com/ClickHouse/ClickHouse/pull/26508) ([Maksim Kita](https://github.com/kitaisreal)). * Fix broken name resolution after rewriting column aliases. Fix [#26432](https://github.com/ClickHouse/ClickHouse/issues/26432). [#26475](https://github.com/ClickHouse/ClickHouse/pull/26475) ([Amos Bird](https://github.com/amosbird)). * Fix infinite non-joined block stream in `partial_merge_join` close [#26325](https://github.com/ClickHouse/ClickHouse/issues/26325). [#26374](https://github.com/ClickHouse/ClickHouse/pull/26374) ([Vladimir C](https://github.com/vdimir)). * Fix possible crash when login as dropped user. Fix [#26073](https://github.com/ClickHouse/ClickHouse/issues/26073). [#26363](https://github.com/ClickHouse/ClickHouse/pull/26363) ([Vitaly Baranov](https://github.com/vitlibar)). * Fix `optimize_distributed_group_by_sharding_key` for multiple columns (leads to incorrect result w/ `optimize_skip_unused_shards=1`/`allow_nondeterministic_optimize_skip_unused_shards=1` and multiple columns in sharding key expression). [#26353](https://github.com/ClickHouse/ClickHouse/pull/26353) ([Azat Khuzhin](https://github.com/azat)). - * `CAST` from `Date` to `DateTime` (or `DateTime64`) was not using the timezone of the `DateTime` type. It can also affect the comparison between `Date` and `DateTime`. Inference of the common type for `Date` and `DateTime` also was not using the corresponding timezone. It affected the results of function `if` and array construction. Closes [#24128](https://github.com/ClickHouse/ClickHouse/issues/24128). [#24129](https://github.com/ClickHouse/ClickHouse/pull/24129) ([Maksim Kita](https://github.com/kitaisreal)). +* `CAST` from `Date` to `DateTime` (or `DateTime64`) was not using the timezone of the `DateTime` type. It can also affect the comparison between `Date` and `DateTime`. Inference of the common type for `Date` and `DateTime` also was not using the corresponding timezone. It affected the results of function `if` and array construction. Closes [#24128](https://github.com/ClickHouse/ClickHouse/issues/24128). [#24129](https://github.com/ClickHouse/ClickHouse/pull/24129) ([Maksim Kita](https://github.com/kitaisreal)). * Fixed rare bug in lost replica recovery that may cause replicas to diverge. [#26321](https://github.com/ClickHouse/ClickHouse/pull/26321) ([tavplubix](https://github.com/tavplubix)). * Fix zstd decompression in case there are escape sequences at the end of internal buffer. Closes [#26013](https://github.com/ClickHouse/ClickHouse/issues/26013). [#26314](https://github.com/ClickHouse/ClickHouse/pull/26314) ([Kseniia Sumarokova](https://github.com/kssenii)). * Fix logical error on join with totals, close [#26017](https://github.com/ClickHouse/ClickHouse/issues/26017). [#26250](https://github.com/ClickHouse/ClickHouse/pull/26250) ([Vladimir C](https://github.com/vdimir)). @@ -75,7 +75,7 @@ * Fix possible crash in `pointInPolygon` if the setting `validate_polygons` is turned off. [#26113](https://github.com/ClickHouse/ClickHouse/pull/26113) ([alexey-milovidov](https://github.com/alexey-milovidov)). * Fix throwing exception when iterate over non-existing remote directory. [#26087](https://github.com/ClickHouse/ClickHouse/pull/26087) ([ianton-ru](https://github.com/ianton-ru)). * Fix rare server crash because of `abort` in ZooKeeper client. Fixes [#25813](https://github.com/ClickHouse/ClickHouse/issues/25813). [#26079](https://github.com/ClickHouse/ClickHouse/pull/26079) ([alesapin](https://github.com/alesapin)). -* Fix wrong thread estimation for right subquery join in some cases. Close [#24075](https://github.com/ClickHouse/ClickHouse/issues/24075). [#26052](https://github.com/ClickHouse/ClickHouse/pull/26052) ([Vladimir C](https://github.com/vdimir)). +* Fix wrong thread count estimation for right subquery join in some cases. Close [#24075](https://github.com/ClickHouse/ClickHouse/issues/24075). [#26052](https://github.com/ClickHouse/ClickHouse/pull/26052) ([Vladimir C](https://github.com/vdimir)). * Fixed incorrect `sequence_id` in MySQL protocol packets that ClickHouse sends on exception during query execution. It might cause MySQL client to reset connection to ClickHouse server. Fixes [#21184](https://github.com/ClickHouse/ClickHouse/issues/21184). [#26051](https://github.com/ClickHouse/ClickHouse/pull/26051) ([tavplubix](https://github.com/tavplubix)). * Fix possible mismatched header when using normal projection with `PREWHERE`. Fix [#26020](https://github.com/ClickHouse/ClickHouse/issues/26020). [#26038](https://github.com/ClickHouse/ClickHouse/pull/26038) ([Amos Bird](https://github.com/amosbird)). * Fix formatting of type `Map` with integer keys to `JSON`. [#25982](https://github.com/ClickHouse/ClickHouse/pull/25982) ([Anton Popov](https://github.com/CurtizJ)). @@ -94,20 +94,8 @@ * Fix `ALTER MODIFY COLUMN` of columns, which participates in TTL expressions. [#25554](https://github.com/ClickHouse/ClickHouse/pull/25554) ([Anton Popov](https://github.com/CurtizJ)). * Fix assertion in `PREWHERE` with non-UInt8 type, close [#19589](https://github.com/ClickHouse/ClickHouse/issues/19589). [#25484](https://github.com/ClickHouse/ClickHouse/pull/25484) ([Vladimir C](https://github.com/vdimir)). * Fix some fuzzed msan crash. Fixes [#22517](https://github.com/ClickHouse/ClickHouse/issues/22517). [#26428](https://github.com/ClickHouse/ClickHouse/pull/26428) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fix empty history file conversion. [#26589](https://github.com/ClickHouse/ClickHouse/pull/26589) ([Azat Khuzhin](https://github.com/azat)). * Update `chown` cmd check in `clickhouse-server` docker entrypoint. It fixes error 'cluster pod restart failed (or timeout)' on kubernetes. [#26545](https://github.com/ClickHouse/ClickHouse/pull/26545) ([Ky Li](https://github.com/Kylinrix)). -#### Build/Testing/Packaging Improvements - -* Disabling TestFlows LDAP module due to test fails. [#26065](https://github.com/ClickHouse/ClickHouse/pull/26065) ([vzakaznikov](https://github.com/vzakaznikov)). -* Enabling all TestFlows modules and fixing some tests. [#26011](https://github.com/ClickHouse/ClickHouse/pull/26011) ([vzakaznikov](https://github.com/vzakaznikov)). -* Add new tests for checking access rights for columns used in filters (`WHERE` / `PREWHERE` / row policy) of the `SELECT` statement after changes in [#24405](https://github.com/ClickHouse/ClickHouse/pull/24405). [#25619](https://github.com/ClickHouse/ClickHouse/pull/25619) ([Vitaly Baranov](https://github.com/vitlibar)). - -#### Other - -* Add `clickhouse-keeper-converter` tool which allows converting zookeeper logs and snapshots into `clickhouse-keeper` snapshot format. [#25428](https://github.com/ClickHouse/ClickHouse/pull/25428) ([alesapin](https://github.com/alesapin)). - - ### ClickHouse release v21.7, 2021-07-09 From c92e1143e3cc332ffdf113b1f067866cf2676ae7 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sat, 14 Aug 2021 02:50:25 +0300 Subject: [PATCH 565/599] Update CHANGELOG.md --- CHANGELOG.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 12fda73fe13..9562703d517 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -2,8 +2,8 @@ #### New Features -* Add support for a part of SQL/JSON standard. [#24148](https://github.com/ClickHouse/ClickHouse/pull/24148) ([l1tsolaiki](https://github.com/l1tsolaiki)). -* Collect common system metrics (in `system.asynchronous_metrics` and `system.asynchronous_metric_log`) on CPU usage, disk usage, memory usage, IO, network, files, load average, CPU frequencies, thermal sensors, EDAC counters, system uptime; also added metrics about the scheduling jitter and the time spent collecting the metrics. It works similar to `atop` in ClickHouse and allows access to monitoring data even if you have no additional tools installed. Close [#9430](https://github.com/ClickHouse/ClickHouse/issues/9430). [#24416](https://github.com/ClickHouse/ClickHouse/pull/24416) (([alexey-milovidov](https://github.com/alexey-milovidov)), [Yegor Levankov](https://github.com/elevankoff), [Kseniia Sumarokova](https://github.com/kssenii)). +* Add support for a part of SQL/JSON standard. [#24148](https://github.com/ClickHouse/ClickHouse/pull/24148) ([l1tsolaiki](https://github.com/l1tsolaiki), [Kseniia Sumarokova](https://github.com/kssenii)). +* Collect common system metrics (in `system.asynchronous_metrics` and `system.asynchronous_metric_log`) on CPU usage, disk usage, memory usage, IO, network, files, load average, CPU frequencies, thermal sensors, EDAC counters, system uptime; also added metrics about the scheduling jitter and the time spent collecting the metrics. It works similar to `atop` in ClickHouse and allows access to monitoring data even if you have no additional tools installed. Close [#9430](https://github.com/ClickHouse/ClickHouse/issues/9430). [#24416](https://github.com/ClickHouse/ClickHouse/pull/24416) (([alexey-milovidov](https://github.com/alexey-milovidov)), [Yegor Levankov](https://github.com/elevankoff)). * Add MaterializedPostgreSQL table engine and database engine. This database engine allows replicating a whole database or any subset of database tables. [#20470](https://github.com/ClickHouse/ClickHouse/pull/20470) ([Kseniia Sumarokova](https://github.com/kssenii)). * Add new functions `leftPad()`, `rightPad()`, `leftPadUTF8()`, `rightPadUTF8()`. [#26075](https://github.com/ClickHouse/ClickHouse/pull/26075) ([Vitaly Baranov](https://github.com/vitlibar)). * Add the `FIRST` keyword to the `ADD INDEX` command to be able to add the index at the beginning of the indices list. [#25904](https://github.com/ClickHouse/ClickHouse/pull/25904) ([xjewer](https://github.com/xjewer)). From 596fa80c15226eb9218b746c21147f34f77fd2d1 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sat, 14 Aug 2021 02:51:01 +0300 Subject: [PATCH 566/599] Update CHANGELOG.md --- CHANGELOG.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 9562703d517..6b0570f0d4a 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -3,7 +3,7 @@ #### New Features * Add support for a part of SQL/JSON standard. [#24148](https://github.com/ClickHouse/ClickHouse/pull/24148) ([l1tsolaiki](https://github.com/l1tsolaiki), [Kseniia Sumarokova](https://github.com/kssenii)). -* Collect common system metrics (in `system.asynchronous_metrics` and `system.asynchronous_metric_log`) on CPU usage, disk usage, memory usage, IO, network, files, load average, CPU frequencies, thermal sensors, EDAC counters, system uptime; also added metrics about the scheduling jitter and the time spent collecting the metrics. It works similar to `atop` in ClickHouse and allows access to monitoring data even if you have no additional tools installed. Close [#9430](https://github.com/ClickHouse/ClickHouse/issues/9430). [#24416](https://github.com/ClickHouse/ClickHouse/pull/24416) (([alexey-milovidov](https://github.com/alexey-milovidov)), [Yegor Levankov](https://github.com/elevankoff)). +* Collect common system metrics (in `system.asynchronous_metrics` and `system.asynchronous_metric_log`) on CPU usage, disk usage, memory usage, IO, network, files, load average, CPU frequencies, thermal sensors, EDAC counters, system uptime; also added metrics about the scheduling jitter and the time spent collecting the metrics. It works similar to `atop` in ClickHouse and allows access to monitoring data even if you have no additional tools installed. Close [#9430](https://github.com/ClickHouse/ClickHouse/issues/9430). [#24416](https://github.com/ClickHouse/ClickHouse/pull/24416) ([alexey-milovidov](https://github.com/alexey-milovidov), [Yegor Levankov](https://github.com/elevankoff)). * Add MaterializedPostgreSQL table engine and database engine. This database engine allows replicating a whole database or any subset of database tables. [#20470](https://github.com/ClickHouse/ClickHouse/pull/20470) ([Kseniia Sumarokova](https://github.com/kssenii)). * Add new functions `leftPad()`, `rightPad()`, `leftPadUTF8()`, `rightPadUTF8()`. [#26075](https://github.com/ClickHouse/ClickHouse/pull/26075) ([Vitaly Baranov](https://github.com/vitlibar)). * Add the `FIRST` keyword to the `ADD INDEX` command to be able to add the index at the beginning of the indices list. [#25904](https://github.com/ClickHouse/ClickHouse/pull/25904) ([xjewer](https://github.com/xjewer)). From 032d75b638a80953ce26810145aa9513af31ad27 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sat, 14 Aug 2021 02:53:36 +0300 Subject: [PATCH 567/599] Update CHANGELOG.md --- CHANGELOG.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 6b0570f0d4a..ca3d21af8e5 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -9,7 +9,7 @@ * Add the `FIRST` keyword to the `ADD INDEX` command to be able to add the index at the beginning of the indices list. [#25904](https://github.com/ClickHouse/ClickHouse/pull/25904) ([xjewer](https://github.com/xjewer)). * Introduce `system.data_skipping_indices` table containing information about existing data skipping indices. Close [#7659](https://github.com/ClickHouse/ClickHouse/issues/7659). [#25693](https://github.com/ClickHouse/ClickHouse/pull/25693) ([Dmitry Novik](https://github.com/novikd)). * Add `bin`/`unbin` functions. [#25609](https://github.com/ClickHouse/ClickHouse/pull/25609) ([zhaoyu](https://github.com/zxc111)). -* Support `Map` and `(U)Int128`, `U(Int256) types in `mapAdd` and `mapSubtract` functions. [#25596](https://github.com/ClickHouse/ClickHouse/pull/25596) ([Ildus Kurbangaliev](https://github.com/ildus)). +* Support `Map` and `UInt128`, `Int128`, `UInt256`, `Int256` types in `mapAdd` and `mapSubtract` functions. [#25596](https://github.com/ClickHouse/ClickHouse/pull/25596) ([Ildus Kurbangaliev](https://github.com/ildus)). * Support `DISTINCT ON (columns)` expression, close [#25404](https://github.com/ClickHouse/ClickHouse/issues/25404). [#25589](https://github.com/ClickHouse/ClickHouse/pull/25589) ([Zijie Lu](https://github.com/TszKitLo40)). * Add an ability to reset a custom setting to default and remove it from the table's metadata. It allows rolling back the change without knowing the system/config's default. Closes [#14449](https://github.com/ClickHouse/ClickHouse/issues/14449). [#17769](https://github.com/ClickHouse/ClickHouse/pull/17769) ([xjewer](https://github.com/xjewer)). * Render pipelines as graphs in Web UI if `EXPLAIN PIPELINE graph = 1` query is submitted. [#26067](https://github.com/ClickHouse/ClickHouse/pull/26067) ([alexey-milovidov](https://github.com/alexey-milovidov)). From 9ea4bea8f9ac83ed9ae4420527d52f9a8ce15f17 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sat, 14 Aug 2021 02:54:05 +0300 Subject: [PATCH 568/599] Update CHANGELOG.md --- CHANGELOG.md | 7 ------- 1 file changed, 7 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index ca3d21af8e5..103d8e40fd9 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1282,13 +1282,6 @@ * PODArray: Avoid call to memcpy with (nullptr, 0) arguments (Fix UBSan report). This fixes [#18525](https://github.com/ClickHouse/ClickHouse/issues/18525). [#18526](https://github.com/ClickHouse/ClickHouse/pull/18526) ([alexey-milovidov](https://github.com/alexey-milovidov)). * Minor improvement for path concatenation of zookeeper paths inside DDLWorker. [#17767](https://github.com/ClickHouse/ClickHouse/pull/17767) ([Bharat Nallan](https://github.com/bharatnc)). * Allow to reload symbols from debug file. This PR also fixes a build-id issue. [#17637](https://github.com/ClickHouse/ClickHouse/pull/17637) ([Amos Bird](https://github.com/amosbird)). -* TestFlows: fixes to LDAP tests that fail due to slow test execution. [#18790](https://github.com/ClickHouse/ClickHouse/pull/18790) ([vzakaznikov](https://github.com/vzakaznikov)). -* TestFlows: Merging requirements for AES encryption functions. Updating aes_encryption tests to use new requirements. Updating TestFlows version to 1.6.72. [#18221](https://github.com/ClickHouse/ClickHouse/pull/18221) ([vzakaznikov](https://github.com/vzakaznikov)). -* TestFlows: Updating TestFlows version to the latest 1.6.72. Re-generating requirements.py. [#18208](https://github.com/ClickHouse/ClickHouse/pull/18208) ([vzakaznikov](https://github.com/vzakaznikov)). -* TestFlows: Updating TestFlows README.md to include "How To Debug Why Test Failed" section. [#17808](https://github.com/ClickHouse/ClickHouse/pull/17808) ([vzakaznikov](https://github.com/vzakaznikov)). -* TestFlows: tests for RBAC [ACCESS MANAGEMENT](https://clickhouse.tech/docs/en/sql-reference/statements/grant/#grant-access-management) privileges. [#17804](https://github.com/ClickHouse/ClickHouse/pull/17804) ([MyroTk](https://github.com/MyroTk)). -* TestFlows: RBAC tests for SHOW, TRUNCATE, KILL, and OPTIMIZE. - Updates to old tests. - Resolved comments from #https://github.com/ClickHouse/ClickHouse/pull/16977. [#17657](https://github.com/ClickHouse/ClickHouse/pull/17657) ([MyroTk](https://github.com/MyroTk)). -* TestFlows: Added RBAC tests for `ATTACH`, `CREATE`, `DROP`, and `DETACH`. [#16977](https://github.com/ClickHouse/ClickHouse/pull/16977) ([MyroTk](https://github.com/MyroTk)). ## [Changelog for 2020](https://github.com/ClickHouse/ClickHouse/blob/master/docs/en/whats-new/changelog/2020.md) From 18ab53488fab117b8c08349b22c914732b9d69eb Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 14 Aug 2021 15:31:55 +0300 Subject: [PATCH 569/599] Better --- .../InterpreterSelectIntersectExceptQuery.cpp | 2 +- .../SelectIntersectExceptQueryVisitor.cpp | 163 +++++++----------- .../SelectIntersectExceptQueryVisitor.h | 18 +- src/Parsers/ASTSelectIntersectExceptQuery.cpp | 42 +---- src/Parsers/ASTSelectIntersectExceptQuery.h | 5 - src/Parsers/ASTSelectWithUnionQuery.h | 3 +- src/Parsers/ExpressionListParsers.cpp | 22 ++- src/Parsers/ExpressionListParsers.h | 19 -- src/Parsers/ParserQueryWithOutput.cpp | 1 - .../ParserSelectIntersectExceptQuery.cpp | 62 ------- .../ParserSelectIntersectExceptQuery.h | 14 -- src/Parsers/ParserSelectWithUnionQuery.cpp | 8 +- src/Parsers/ParserUnionQueryElement.cpp | 2 - ...02004_intersect_except_operators.reference | 23 +++ .../02004_intersect_except_operators.sql | 7 + 15 files changed, 117 insertions(+), 274 deletions(-) delete mode 100644 src/Parsers/ParserSelectIntersectExceptQuery.cpp delete mode 100644 src/Parsers/ParserSelectIntersectExceptQuery.h diff --git a/src/Interpreters/InterpreterSelectIntersectExceptQuery.cpp b/src/Interpreters/InterpreterSelectIntersectExceptQuery.cpp index 34d7ae5b37f..4edd13d08e5 100644 --- a/src/Interpreters/InterpreterSelectIntersectExceptQuery.cpp +++ b/src/Interpreters/InterpreterSelectIntersectExceptQuery.cpp @@ -56,7 +56,7 @@ InterpreterSelectIntersectExceptQuery::InterpreterSelectIntersectExceptQuery( ASTSelectIntersectExceptQuery * ast = query_ptr->as(); final_operator = ast->final_operator; - const auto & children = ast->children[0]->children; + const auto & children = ast->children; size_t num_children = children.size(); /// AST must have been changed by the visitor. diff --git a/src/Interpreters/SelectIntersectExceptQueryVisitor.cpp b/src/Interpreters/SelectIntersectExceptQueryVisitor.cpp index 273bc327dc3..3815fb4ad0f 100644 --- a/src/Interpreters/SelectIntersectExceptQueryVisitor.cpp +++ b/src/Interpreters/SelectIntersectExceptQueryVisitor.cpp @@ -18,87 +18,12 @@ namespace DB void SelectIntersectExceptQueryMatcher::visit(ASTPtr & ast, Data & data) { - if (auto * select_intersect_except = ast->as()) - { - if (select_intersect_except->final_operator != ASTSelectIntersectExceptQuery::Operator::UNKNOWN) - return; - - data.initialize(select_intersect_except); - visit(*select_intersect_except, data); - } - else if (auto * select_union = ast->as()) - { + if (auto * select_union = ast->as()) visit(*select_union, data); - } -} - -void SelectIntersectExceptQueryMatcher::visit(ASTSelectIntersectExceptQuery & ast, Data & data) -{ - /* Example: select 1 intersect select 1 intsect select 1 intersect select 1 intersect select 1; - * - * --SelectIntersectExceptQuery --SelectIntersectExceptQuery - * ---ExpressionList ---ExpressionList - * ----SelectQuery ----SelectIntersectExceptQuery - * ----SelectQuery ------ExpressionList - * ----SelectQuery ---> -------SelectIntersectExceptQuery - * ----SelectQuery --------ExpressionList - * ---------SelectQuery - * ---------SelectQuery - * -------SelectQuery - * ----SelectQuery - **/ - - auto & selects = data.reversed_list_of_selects; - - if (selects.empty()) - return; - - const auto left = selects.back(); - selects.pop_back(); - const auto right = selects.back(); - selects.pop_back(); - - auto & operators = data.reversed_list_of_operators; - const auto current_operator = operators.back(); - operators.pop_back(); - - auto list_node = std::make_shared(); - list_node->children = {left, right}; - - if (selects.empty()) - { - ast.final_operator = current_operator; - ast.children = {std::move(list_node)}; - } - else - { - auto select_intersect_except = std::make_shared(); - select_intersect_except->final_operator = {current_operator}; - select_intersect_except->list_of_selects = std::move(list_node); - select_intersect_except->children.push_back(select_intersect_except->list_of_selects); - - selects.emplace_back(std::move(select_intersect_except)); - } - - visit(ast, data); } void SelectIntersectExceptQueryMatcher::visit(ASTSelectWithUnionQuery & ast, Data &) { - /* Example: select 1 union all select 2 except select 1 except select 2 union distinct select 5; - * - * --SelectWithUnionQuery --SelectIntersectExceptQuery - * ---ExpressionList ---ExpressionList - * ----SelectQuery ----SelectIntersectExceptQuery - * ----SelectQuery -----ExpressionList - * ----SelectQuery (except) ---> ------SelectIntersectExceptQuery - * ----SelectQuery (except) -------ExpressionList - * ----SelectQuery --------SelectWithUnionQuery (select 1 union all select 2) - * --------SelectQuery (select 1) - * ------SelectQuery (select 2) - * -----SelectQuery (select 5) - **/ - auto & union_modes = ast.list_of_modes; if (union_modes.empty()) @@ -107,8 +32,7 @@ void SelectIntersectExceptQueryMatcher::visit(ASTSelectWithUnionQuery & ast, Dat auto selects = std::move(ast.list_of_selects->children); if (union_modes.size() + 1 != selects.size()) - throw Exception(ErrorCodes::LOGICAL_ERROR, - "Incorrect ASTSelectWithUnionQuery (modes: {}, selects: {})", + throw Exception(ErrorCodes::LOGICAL_ERROR, "Incorrect ASTSelectWithUnionQuery (modes: {}, selects: {})", union_modes.size(), selects.size()); std::reverse(selects.begin(), selects.end()); @@ -119,39 +43,70 @@ void SelectIntersectExceptQueryMatcher::visit(ASTSelectWithUnionQuery & ast, Dat for (const auto & mode : union_modes) { - /// Flatten all previous selects into ASTSelectIntersectQuery - if (mode == ASTSelectWithUnionQuery::Mode::EXCEPT) + switch (mode) { - auto left = std::make_shared(); - left->union_mode = ASTSelectWithUnionQuery::Mode::ALL; + case ASTSelectWithUnionQuery::Mode::EXCEPT: + { + auto left = std::make_shared(); + left->union_mode = ASTSelectWithUnionQuery::Mode::ALL; - left->list_of_selects = std::make_shared(); - left->children.push_back(left->list_of_selects); - left->list_of_selects->children = std::move(children); + left->list_of_selects = std::make_shared(); + left->children.push_back(left->list_of_selects); + left->list_of_selects->children = std::move(children); - left->list_of_modes = std::move(modes); - modes = {}; + left->list_of_modes = std::move(modes); + modes = {}; - auto right = selects.back(); - selects.pop_back(); + auto right = selects.back(); + selects.pop_back(); - auto list_node = std::make_shared(); - list_node->children = {left, right}; + auto except_node = std::make_shared(); + except_node->final_operator = ASTSelectIntersectExceptQuery::Operator::EXCEPT; + except_node->children = {left, right}; - auto select_intersect_except = std::make_shared(); - select_intersect_except->final_operator = {ASTSelectIntersectExceptQuery::Operator::EXCEPT}; - select_intersect_except->children.emplace_back(std::move(list_node)); - select_intersect_except->list_of_selects = std::make_shared(); - select_intersect_except->list_of_selects->children.push_back(select_intersect_except->children[0]); + children = {except_node}; + break; + } + case ASTSelectWithUnionQuery::Mode::INTERSECT: + { + bool from_except = false; + const auto * except_ast = typeid_cast(children.back().get()); + if (except_ast && (except_ast->final_operator == ASTSelectIntersectExceptQuery::Operator::EXCEPT)) + from_except = true; - children = {select_intersect_except}; - } - else if (!selects.empty()) - { - auto right = selects.back(); - selects.pop_back(); - children.emplace_back(std::move(right)); - modes.push_back(mode); + ASTPtr left; + if (from_except) + { + left = std::move(children.back()->children[1]); + } + else + { + left = children.back(); + children.pop_back(); + } + + auto right = selects.back(); + selects.pop_back(); + + auto intersect_node = std::make_shared(); + intersect_node->final_operator = ASTSelectIntersectExceptQuery::Operator::INTERSECT; + intersect_node->children = {left, right}; + + if (from_except) + children.back()->children[1] = std::move(intersect_node); + else + children.push_back(std::move(intersect_node)); + + break; + } + default: + { + auto right = selects.back(); + selects.pop_back(); + children.emplace_back(std::move(right)); + modes.push_back(mode); + break; + } } } diff --git a/src/Interpreters/SelectIntersectExceptQueryVisitor.h b/src/Interpreters/SelectIntersectExceptQueryVisitor.h index 1dd0694666d..07a6ad606a1 100644 --- a/src/Interpreters/SelectIntersectExceptQueryVisitor.h +++ b/src/Interpreters/SelectIntersectExceptQueryVisitor.h @@ -17,27 +17,11 @@ class ASTFunction; class SelectIntersectExceptQueryMatcher { public: - struct Data - { - Data() = default; - - void initialize(const ASTSelectIntersectExceptQuery * select_intersect_except) - { - reversed_list_of_selects = select_intersect_except->list_of_selects->clone()->children; - reversed_list_of_operators = select_intersect_except->list_of_operators; - - std::reverse(reversed_list_of_selects.begin(), reversed_list_of_selects.end()); - std::reverse(reversed_list_of_operators.begin(), reversed_list_of_operators.end()); - } - - ASTs reversed_list_of_selects; - ASTSelectIntersectExceptQuery::Operators reversed_list_of_operators; - }; + struct Data {}; static bool needChildVisit(const ASTPtr &, const ASTPtr &) { return true; } static void visit(ASTPtr & ast, Data &); - static void visit(ASTSelectIntersectExceptQuery &, Data &); static void visit(ASTSelectWithUnionQuery &, Data &); }; diff --git a/src/Parsers/ASTSelectIntersectExceptQuery.cpp b/src/Parsers/ASTSelectIntersectExceptQuery.cpp index 26fd9353d5b..9d7a717fa6c 100644 --- a/src/Parsers/ASTSelectIntersectExceptQuery.cpp +++ b/src/Parsers/ASTSelectIntersectExceptQuery.cpp @@ -14,10 +14,6 @@ ASTPtr ASTSelectIntersectExceptQuery::clone() const for (const auto & child : children) res->children.push_back(child->clone()); - if (res->list_of_selects) - res->list_of_selects = list_of_selects->clone(); - - res->list_of_operators = list_of_operators; res->final_operator = final_operator; cloneOutputOptions(*res); @@ -28,44 +24,18 @@ void ASTSelectIntersectExceptQuery::formatQueryImpl(const FormatSettings & setti { std::string indent_str = settings.one_line ? "" : std::string(4 * frame.indent, ' '); - auto operator_to_str = [&](auto current_operator) + for (ASTs::const_iterator it = children.begin(); it != children.end(); ++it) { - if (current_operator == Operator::INTERSECT) - return "INTERSECT"; - else - return "EXCEPT"; - }; - - for (ASTs::const_iterator it = list_of_selects->children.begin(); it != list_of_selects->children.end(); ++it) - { - if (it != list_of_selects->children.begin()) + if (it != children.begin()) { settings.ostr << settings.nl_or_ws << indent_str << (settings.hilite ? hilite_keyword : "") - << operator_to_str(list_of_operators[it - list_of_selects->children.begin() - 1]) + << (final_operator == Operator::INTERSECT ? "INTERSECT" : "EXCEPT") << (settings.hilite ? hilite_none : ""); } - if (auto * node = (*it)->as()) - { - settings.ostr << settings.nl_or_ws << indent_str; - - if (node->list_of_selects->children.size() == 1) - { - (node->list_of_selects->children.at(0))->formatImpl(settings, state, frame); - } - else - { - auto sub_query = std::make_shared(); - sub_query->children.push_back(*it); - sub_query->formatImpl(settings, state, frame); - } - } - else - { - if (it != list_of_selects->children.begin()) - settings.ostr << settings.nl_or_ws; - (*it)->formatImpl(settings, state, frame); - } + if (it != children.begin()) + settings.ostr << settings.nl_or_ws; + (*it)->formatImpl(settings, state, frame); } } diff --git a/src/Parsers/ASTSelectIntersectExceptQuery.h b/src/Parsers/ASTSelectIntersectExceptQuery.h index 8fc5756e370..97a8296ce2c 100644 --- a/src/Parsers/ASTSelectIntersectExceptQuery.h +++ b/src/Parsers/ASTSelectIntersectExceptQuery.h @@ -24,11 +24,6 @@ public: EXCEPT }; - using Operators = std::vector; - - ASTPtr list_of_selects; - Operators list_of_operators; - /// Final operator after applying visitor. Operator final_operator = Operator::UNKNOWN; }; diff --git a/src/Parsers/ASTSelectWithUnionQuery.h b/src/Parsers/ASTSelectWithUnionQuery.h index 2c36bcecf6b..629e9b5d96d 100644 --- a/src/Parsers/ASTSelectWithUnionQuery.h +++ b/src/Parsers/ASTSelectWithUnionQuery.h @@ -23,7 +23,8 @@ public: Unspecified, ALL, DISTINCT, - EXCEPT + EXCEPT, + INTERSECT }; using UnionModes = std::vector; diff --git a/src/Parsers/ExpressionListParsers.cpp b/src/Parsers/ExpressionListParsers.cpp index 33085379abb..ef54c627aab 100644 --- a/src/Parsers/ExpressionListParsers.cpp +++ b/src/Parsers/ExpressionListParsers.cpp @@ -9,6 +9,7 @@ #include #include #include +#include #include @@ -111,12 +112,18 @@ bool ParserList::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) bool ParserUnionList::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { + ParserUnionQueryElement elem_parser; + ParserKeyword s_union_parser("UNION"); + ParserKeyword s_all_parser("ALL"); + ParserKeyword s_distinct_parser("DISTINCT"); + ParserKeyword s_except_parser("EXCEPT"); + ParserKeyword s_intersect_parser("INTERSECT"); ASTs elements; auto parse_element = [&] { ASTPtr element; - if (!elem_parser->parse(pos, element, expected)) + if (!elem_parser.parse(pos, element, expected)) return false; elements.push_back(element); @@ -126,15 +133,15 @@ bool ParserUnionList::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) /// Parse UNION type auto parse_separator = [&] { - if (s_union_parser->ignore(pos, expected)) + if (s_union_parser.ignore(pos, expected)) { // SELECT ... UNION ALL SELECT ... - if (s_all_parser->check(pos, expected)) + if (s_all_parser.check(pos, expected)) { union_modes.push_back(ASTSelectWithUnionQuery::Mode::ALL); } // SELECT ... UNION DISTINCT SELECT ... - else if (s_distinct_parser->check(pos, expected)) + else if (s_distinct_parser.check(pos, expected)) { union_modes.push_back(ASTSelectWithUnionQuery::Mode::DISTINCT); } @@ -145,11 +152,16 @@ bool ParserUnionList::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) } return true; } - else if (s_except_parser->check(pos, expected)) + else if (s_except_parser.check(pos, expected)) { union_modes.push_back(ASTSelectWithUnionQuery::Mode::EXCEPT); return true; } + else if (s_intersect_parser.check(pos, expected)) + { + union_modes.push_back(ASTSelectWithUnionQuery::Mode::INTERSECT); + return true; + } return false; }; diff --git a/src/Parsers/ExpressionListParsers.h b/src/Parsers/ExpressionListParsers.h index 36f39a50ab3..e44cacb313f 100644 --- a/src/Parsers/ExpressionListParsers.h +++ b/src/Parsers/ExpressionListParsers.h @@ -79,20 +79,6 @@ private: class ParserUnionList : public IParserBase { public: - ParserUnionList( - ParserPtr && elem_parser_, - ParserPtr && s_union_parser_, - ParserPtr && s_all_parser_, - ParserPtr && s_distinct_parser_, - ParserPtr && s_except_parser_) - : elem_parser(std::move(elem_parser_)) - , s_union_parser(std::move(s_union_parser_)) - , s_all_parser(std::move(s_all_parser_)) - , s_distinct_parser(std::move(s_distinct_parser_)) - , s_except_parser(std::move(s_except_parser_)) - { - } - template static bool parseUtil(Pos & pos, const ElemFunc & parse_element, const SepFunc & parse_separator) { @@ -122,11 +108,6 @@ protected: const char * getName() const override { return "list of union elements"; } bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; private: - ParserPtr elem_parser; - ParserPtr s_union_parser; - ParserPtr s_all_parser; - ParserPtr s_distinct_parser; - ParserPtr s_except_parser; ASTSelectWithUnionQuery::UnionModes union_modes; }; diff --git a/src/Parsers/ParserQueryWithOutput.cpp b/src/Parsers/ParserQueryWithOutput.cpp index e2ab8a84cc1..4a73952674c 100644 --- a/src/Parsers/ParserQueryWithOutput.cpp +++ b/src/Parsers/ParserQueryWithOutput.cpp @@ -7,7 +7,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Parsers/ParserSelectIntersectExceptQuery.cpp b/src/Parsers/ParserSelectIntersectExceptQuery.cpp deleted file mode 100644 index 2b4ba9d60e2..00000000000 --- a/src/Parsers/ParserSelectIntersectExceptQuery.cpp +++ /dev/null @@ -1,62 +0,0 @@ -#include -#include -#include -#include -#include -#include -#include -#include - - -namespace DB -{ - -bool ParserSelectIntersectExceptQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) -{ - ParserKeyword intersect_keyword("INTERSECT"); - - ASTs elements; - ASTSelectIntersectExceptQuery::Operators operators; - - auto parse_element = [&]() -> bool - { - ASTPtr element; - if (!ParserSelectQuery().parse(pos, element, expected) && !ParserSubquery().parse(pos, element, expected)) - return false; - - elements.push_back(element); - return true; - }; - - auto parse_separator = [&]() -> bool - { - if (!intersect_keyword.ignore(pos)) - return false; - - operators.emplace_back(ASTSelectIntersectExceptQuery::Operator::INTERSECT); - return true; - }; - - if (!ParserUnionList::parseUtil(pos, parse_element, parse_separator)) - return false; - - if (operators.empty() || elements.empty()) - return false; - - if (operators.size() + 1 != elements.size()) - return false; - - auto list_node = std::make_shared(); - list_node->children = std::move(elements); - - auto intersect_or_except_ast = std::make_shared(); - - node = intersect_or_except_ast; - intersect_or_except_ast->list_of_selects = list_node; - intersect_or_except_ast->children.push_back(intersect_or_except_ast->list_of_selects); - intersect_or_except_ast->list_of_operators = operators; - - return true; -} - -} diff --git a/src/Parsers/ParserSelectIntersectExceptQuery.h b/src/Parsers/ParserSelectIntersectExceptQuery.h deleted file mode 100644 index e01785113a8..00000000000 --- a/src/Parsers/ParserSelectIntersectExceptQuery.h +++ /dev/null @@ -1,14 +0,0 @@ -#pragma once -#include - - -namespace DB -{ -class ParserSelectIntersectExceptQuery : public IParserBase -{ -protected: - const char * getName() const override { return "INTERSECT or EXCEPT"; } - bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; -}; - -} diff --git a/src/Parsers/ParserSelectWithUnionQuery.cpp b/src/Parsers/ParserSelectWithUnionQuery.cpp index 8c4c183a099..532a9e20735 100644 --- a/src/Parsers/ParserSelectWithUnionQuery.cpp +++ b/src/Parsers/ParserSelectWithUnionQuery.cpp @@ -10,13 +10,7 @@ namespace DB bool ParserSelectWithUnionQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { ASTPtr list_node; - - ParserUnionList parser( - std::make_unique(), - std::make_unique("UNION"), - std::make_unique("ALL"), - std::make_unique("DISTINCT"), - std::make_unique("EXCEPT")); + ParserUnionList parser; if (!parser.parse(pos, list_node, expected)) return false; diff --git a/src/Parsers/ParserUnionQueryElement.cpp b/src/Parsers/ParserUnionQueryElement.cpp index 5abbce25930..d59a7be2278 100644 --- a/src/Parsers/ParserUnionQueryElement.cpp +++ b/src/Parsers/ParserUnionQueryElement.cpp @@ -2,7 +2,6 @@ #include #include #include -#include #include @@ -12,7 +11,6 @@ namespace DB bool ParserUnionQueryElement::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { if (!ParserSubquery().parse(pos, node, expected) - && !ParserSelectIntersectExceptQuery().parse(pos, node, expected) && !ParserSelectQuery().parse(pos, node, expected)) return false; diff --git a/tests/queries/0_stateless/02004_intersect_except_operators.reference b/tests/queries/0_stateless/02004_intersect_except_operators.reference index a097bd0076f..7f41faaf83a 100644 --- a/tests/queries/0_stateless/02004_intersect_except_operators.reference +++ b/tests/queries/0_stateless/02004_intersect_except_operators.reference @@ -99,3 +99,26 @@ select * from (select 1 union all select 2 union all select 3 union all select 4 2 3 5 +select 1 intersect (select 1 except select 2); +1 +select 1 union all select 2 except (select 2 except select 1 union all select 1) except select 4; +explain syntax select 1 intersect select 1; +SELECT 1 +INTERSECT +SELECT 1 +explain syntax select 1 except select 1; +SELECT 1 +EXCEPT +SELECT 1 +explain syntax select 1 union all select 2 except (select 2 except select 1 union all select 1) except select 4; +SELECT 1 +UNION ALL +SELECT 2 +EXCEPT +SELECT 2 +EXCEPT +SELECT 1 +UNION ALL +SELECT 1 +EXCEPT +SELECT 4 diff --git a/tests/queries/0_stateless/02004_intersect_except_operators.sql b/tests/queries/0_stateless/02004_intersect_except_operators.sql index 4602dec7238..ef0e52da116 100644 --- a/tests/queries/0_stateless/02004_intersect_except_operators.sql +++ b/tests/queries/0_stateless/02004_intersect_except_operators.sql @@ -32,3 +32,10 @@ select 1 union all select 1 intersect select 2; select * from (select 1 union all select 2 union all select 3 union all select 4 except select 3 union all select 5) order by 1; select * from (select 1 union all select 2 union all select 3 union all select 4 intersect select 3 union all select 5) order by 1; select * from (select 1 union all select 2 union all select 3 union all select 4 intersect select 3 union all select 5 except select 1) order by 1; + +select 1 intersect (select 1 except select 2); +select 1 union all select 2 except (select 2 except select 1 union all select 1) except select 4; + +explain syntax select 1 intersect select 1; +explain syntax select 1 except select 1; +explain syntax select 1 union all select 2 except (select 2 except select 1 union all select 1) except select 4; From f125fb3fef9c417c82ebf52a14d4aa4aa1b8e88f Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 14 Aug 2021 20:04:21 +0300 Subject: [PATCH 570/599] Some fixes around any/all --- .../AggregateFunctionMinMaxAny.h | 12 +- .../SelectIntersectExceptQueryVisitor.cpp | 4 + src/Parsers/ASTSelectWithUnionQuery.cpp | 4 - src/Parsers/ExpressionListParsers.cpp | 124 ++++++++---------- .../QueryPlan/IntersectOrExceptStep.h | 2 +- .../Transforms/IntersectOrExceptTransform.h | 2 +- .../02007_test_any_all_operators.reference | 22 +++- .../02007_test_any_all_operators.sql | 13 +- 8 files changed, 101 insertions(+), 82 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionMinMaxAny.h b/src/AggregateFunctions/AggregateFunctionMinMaxAny.h index 577b8127fd7..e5471b8a727 100644 --- a/src/AggregateFunctions/AggregateFunctionMinMaxAny.h +++ b/src/AggregateFunctions/AggregateFunctionMinMaxAny.h @@ -50,6 +50,8 @@ private: T value; public: + static constexpr bool is_nullable = false; + bool has() const { return has_value; @@ -470,6 +472,8 @@ private: char small_data[MAX_SMALL_STRING_SIZE]; /// Including the terminating zero. public: + static constexpr bool is_nullable = false; + bool has() const { return size >= 0; @@ -693,6 +697,8 @@ private: Field value; public: + static constexpr bool is_nullable = false; + bool has() const { return !value.isNull(); @@ -979,6 +985,8 @@ struct AggregateFunctionAnyLastData : Data template struct AggregateFunctionSingleValueOrNullData : Data { + static constexpr bool is_nullable = true; + using Self = AggregateFunctionSingleValueOrNullData; bool first_value = true; @@ -1136,7 +1144,9 @@ public: DataTypePtr getReturnType() const override { auto result_type = this->argument_types.at(0); - return Data::name() == "singleValueOrNull" ? makeNullable(result_type) : result_type; + if constexpr (Data::is_nullable) + return makeNullable(result_type); + return result_type; } void add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena * arena) const override diff --git a/src/Interpreters/SelectIntersectExceptQueryVisitor.cpp b/src/Interpreters/SelectIntersectExceptQueryVisitor.cpp index 3815fb4ad0f..190ec279038 100644 --- a/src/Interpreters/SelectIntersectExceptQueryVisitor.cpp +++ b/src/Interpreters/SelectIntersectExceptQueryVisitor.cpp @@ -5,6 +5,10 @@ namespace DB { +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} /* * Note: there is a difference between intersect and except behaviour. diff --git a/src/Parsers/ASTSelectWithUnionQuery.cpp b/src/Parsers/ASTSelectWithUnionQuery.cpp index b882c738c9a..fa7359574f8 100644 --- a/src/Parsers/ASTSelectWithUnionQuery.cpp +++ b/src/Parsers/ASTSelectWithUnionQuery.cpp @@ -8,10 +8,6 @@ namespace DB { -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} ASTPtr ASTSelectWithUnionQuery::clone() const { diff --git a/src/Parsers/ExpressionListParsers.cpp b/src/Parsers/ExpressionListParsers.cpp index ef54c627aab..69d95422799 100644 --- a/src/Parsers/ExpressionListParsers.cpp +++ b/src/Parsers/ExpressionListParsers.cpp @@ -198,91 +198,83 @@ enum class SubqueryFunctionType ALL }; -static bool modifyAST(const String & operator_name, ASTPtr function, SubqueryFunctionType type) +static bool modifyAST(ASTPtr ast, SubqueryFunctionType type) { - // = ANY --> IN, != ALL --> NOT IN - if ((type == SubqueryFunctionType::ANY && operator_name == "equals") - || (type == SubqueryFunctionType::ALL && operator_name == "notEquals")) + /* Rewrite in AST: + * = ANY --> IN + * != ALL --> NOT IN + * = ALL --> IN (SELECT singleValueOrNull(*) FROM subquery) + * != ANY --> NOT IN (SELECT singleValueOrNull(*) FROM subquery) + **/ + + auto * function = assert_cast(ast.get()); + String operator_name = function->name; + + auto function_equals = operator_name == "equals"; + auto function_not_equals = operator_name == "notEquals"; + + String aggregate_function_name; + if (function_equals || function_not_equals) { - assert_cast(function.get())->name = "in"; if (operator_name == "notEquals") + function->name = "notIn"; + else + function->name = "in"; + + if ((type == SubqueryFunctionType::ANY && function_equals) + || (type == SubqueryFunctionType::ALL && function_not_equals)) { - auto function_not = std::make_shared(); - auto exp_list_not = std::make_shared(); - exp_list_not->children.push_back(function); - function_not->name = "not"; - function_not->children.push_back(exp_list_not); - function_not->arguments = exp_list_not; - function = function_not; + return true; } - return true; + + aggregate_function_name = "singleValueOrNull"; } + else if (operator_name == "greaterOrEquals" || operator_name == "greater") + { + aggregate_function_name = (type == SubqueryFunctionType::ANY ? "min" : "max"); + } + else if (operator_name == "lessOrEquals" || operator_name == "less") + { + aggregate_function_name = (type == SubqueryFunctionType::ANY ? "max" : "min"); + } + else + return false; - // subquery --> (SELECT aggregate_function(*) FROM subquery) - auto aggregate_function = std::make_shared(); - auto aggregate_function_exp_list = std::make_shared(); - aggregate_function_exp_list ->children.push_back(std::make_shared()); - aggregate_function->arguments = aggregate_function_exp_list; - aggregate_function->children.push_back(aggregate_function_exp_list); + /// subquery --> (SELECT aggregate_function(*) FROM subquery) + auto aggregate_function = makeASTFunction(aggregate_function_name, std::make_shared()); + auto subquery_node = function->children[0]->children[1]; - ASTPtr subquery_node = function->children[0]->children[1]; - auto select_query = std::make_shared(); - auto tables_in_select = std::make_shared(); - auto tables_in_select_element = std::make_shared(); auto table_expression = std::make_shared(); - table_expression->subquery = subquery_node; - table_expression->children.push_back(subquery_node); - tables_in_select_element->table_expression = table_expression; - tables_in_select_element->children.push_back(table_expression); - tables_in_select->children.push_back(tables_in_select_element); + table_expression->subquery = std::move(subquery_node); + table_expression->children.push_back(table_expression->subquery); + + auto tables_in_select_element = std::make_shared(); + tables_in_select_element->table_expression = std::move(table_expression); + tables_in_select_element->children.push_back(tables_in_select_element->table_expression); + + auto tables_in_select = std::make_shared(); + tables_in_select->children.push_back(std::move(tables_in_select_element)); + auto select_exp_list = std::make_shared(); select_exp_list->children.push_back(aggregate_function); + + auto select_query = std::make_shared(); select_query->children.push_back(select_exp_list); select_query->children.push_back(tables_in_select); - select_query->setExpression(ASTSelectQuery::Expression::SELECT, std::move(select_exp_list)); - select_query->setExpression(ASTSelectQuery::Expression::TABLES, std::move(tables_in_select)); + + select_query->setExpression(ASTSelectQuery::Expression::SELECT, select_exp_list); + select_query->setExpression(ASTSelectQuery::Expression::TABLES, tables_in_select); auto select_with_union_query = std::make_shared(); - auto list_of_selects = std::make_shared(); - list_of_selects->children.push_back(select_query); - select_with_union_query->list_of_selects = list_of_selects; + select_with_union_query->list_of_selects = std::make_shared(); + select_with_union_query->list_of_selects->children.push_back(std::move(select_query)); select_with_union_query->children.push_back(select_with_union_query->list_of_selects); auto new_subquery = std::make_shared(); new_subquery->children.push_back(select_with_union_query); - function->children[0]->children.pop_back(); - function->children[0]->children.push_back(new_subquery); + ast->children[0]->children.back() = std::move(new_subquery); - if (operator_name == "greaterOrEquals" || operator_name == "greater") - { - aggregate_function->name = type == SubqueryFunctionType::ANY ? "min" : "max"; - return true; - } - if (operator_name == "lessOrEquals" || operator_name == "less") - { - aggregate_function->name = type == SubqueryFunctionType::ANY ? "max" : "min"; - return true; - } - - // = ALL --> IN (SELECT singleValueOrNull(*) FROM subquery) - // != ANY --> NOT IN (SELECT singleValueOrNull(*) FROM subquery) - if (operator_name == "equals" || operator_name == "notEquals") - { - aggregate_function->name = "singleValueOrNull"; - assert_cast(function.get())->name = "in"; - if (operator_name == "notEquals") - { - auto function_not = std::make_shared(); - auto exp_list_not = std::make_shared(); - exp_list_not->children.push_back(function); - function_not->name = "not"; - function_not->children.push_back(exp_list_not); - function_not->arguments = exp_list_not; - function = function_not; - } - return true; - } - return false; + return true; } bool ParserComparisonExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) @@ -346,7 +338,7 @@ bool ParserComparisonExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & exp_list->children.push_back(node); exp_list->children.push_back(elem); - if (subquery_function_type != SubqueryFunctionType::NONE && !modifyAST(function->name, function, subquery_function_type)) + if (subquery_function_type != SubqueryFunctionType::NONE && !modifyAST(function, subquery_function_type)) return false; pos.increaseDepth(); diff --git a/src/Processors/QueryPlan/IntersectOrExceptStep.h b/src/Processors/QueryPlan/IntersectOrExceptStep.h index 002f1b1570c..9e87c921ab2 100644 --- a/src/Processors/QueryPlan/IntersectOrExceptStep.h +++ b/src/Processors/QueryPlan/IntersectOrExceptStep.h @@ -12,7 +12,7 @@ using Operator = ASTSelectIntersectExceptQuery::Operator; public: /// max_threads is used to limit the number of threads for result pipeline. - IntersectOrExceptStep(DataStreams input_streams_, Operator operators_, size_t max_threads_ = 0); + IntersectOrExceptStep(DataStreams input_streams_, Operator operator_, size_t max_threads_ = 0); String getName() const override { return "IntersectOrExcept"; } diff --git a/src/Processors/Transforms/IntersectOrExceptTransform.h b/src/Processors/Transforms/IntersectOrExceptTransform.h index da1fa6a119e..e200bfd6cc5 100644 --- a/src/Processors/Transforms/IntersectOrExceptTransform.h +++ b/src/Processors/Transforms/IntersectOrExceptTransform.h @@ -14,7 +14,7 @@ class IntersectOrExceptTransform : public IProcessor using Operator = ASTSelectIntersectExceptQuery::Operator; public: - IntersectOrExceptTransform(const Block & header_, Operator operators); + IntersectOrExceptTransform(const Block & header_, Operator operator_); String getName() const override { return "IntersectOrExcept"; } diff --git a/tests/queries/0_stateless/02007_test_any_all_operators.reference b/tests/queries/0_stateless/02007_test_any_all_operators.reference index cd36102cb80..ebd7cd8f6ca 100644 --- a/tests/queries/0_stateless/02007_test_any_all_operators.reference +++ b/tests/queries/0_stateless/02007_test_any_all_operators.reference @@ -3,17 +3,29 @@ select 1 == any (select number from numbers(10)); 1 select 1 == any (select number from numbers(2, 10)); 0 +select 1 != all (select 1 from numbers(10)); +0 +select 1 != all (select number from numbers(10)); +0 select 1 == all (select 1 from numbers(10)); 1 select 1 == all (select number from numbers(10)); 0 +select 1 != any (select 1 from numbers(10)); +0 +select 1 != any (select number from numbers(10)); +1 select number as a from numbers(10) where a == any (select number from numbers(3, 3)); 3 4 5 --- TODO: Incorrect: -select 1 != any (select 1 from numbers(10)); +select number as a from numbers(10) where a != any (select 5 from numbers(3, 3)); +0 1 -select 1 != all (select 1 from numbers(10)); -1 -select number as a from numbers(10) where a != any (select number from numbers(3, 3)); +2 +3 +4 +6 +7 +8 +9 diff --git a/tests/queries/0_stateless/02007_test_any_all_operators.sql b/tests/queries/0_stateless/02007_test_any_all_operators.sql index 08fc929bab9..525f7e1fabd 100644 --- a/tests/queries/0_stateless/02007_test_any_all_operators.sql +++ b/tests/queries/0_stateless/02007_test_any_all_operators.sql @@ -1,12 +1,17 @@ -- { echo } select 1 == any (select number from numbers(10)); select 1 == any (select number from numbers(2, 10)); + +select 1 != all (select 1 from numbers(10)); +select 1 != all (select number from numbers(10)); + select 1 == all (select 1 from numbers(10)); select 1 == all (select number from numbers(10)); -select number as a from numbers(10) where a == any (select number from numbers(3, 3)); --- TODO: Incorrect: + select 1 != any (select 1 from numbers(10)); -select 1 != all (select 1 from numbers(10)); -select number as a from numbers(10) where a != any (select number from numbers(3, 3)); +select 1 != any (select number from numbers(10)); + +select number as a from numbers(10) where a == any (select number from numbers(3, 3)); +select number as a from numbers(10) where a != any (select 5 from numbers(3, 3)); From e64c9733085f4a9bdbe1eb350709ac1e88b34aae Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 14 Aug 2021 23:34:47 +0300 Subject: [PATCH 571/599] Add a test for #10735 --- .../02009_mysql_client_empty_result.reference | 4 ++++ .../02009_mysql_client_empty_result.sh | 21 +++++++++++++++++++ 2 files changed, 25 insertions(+) create mode 100644 tests/queries/0_stateless/02009_mysql_client_empty_result.reference create mode 100755 tests/queries/0_stateless/02009_mysql_client_empty_result.sh diff --git a/tests/queries/0_stateless/02009_mysql_client_empty_result.reference b/tests/queries/0_stateless/02009_mysql_client_empty_result.reference new file mode 100644 index 00000000000..f11935a87d3 --- /dev/null +++ b/tests/queries/0_stateless/02009_mysql_client_empty_result.reference @@ -0,0 +1,4 @@ +x +1 +i +3 diff --git a/tests/queries/0_stateless/02009_mysql_client_empty_result.sh b/tests/queries/0_stateless/02009_mysql_client_empty_result.sh new file mode 100755 index 00000000000..fde2f57b4ee --- /dev/null +++ b/tests/queries/0_stateless/02009_mysql_client_empty_result.sh @@ -0,0 +1,21 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +${MYSQL_CLIENT} --batch --execute 'SELECT 1 AS x' +${MYSQL_CLIENT} --batch --execute 'SELECT 1 AS x WHERE 0' + +${MYSQL_CLIENT} <<<" + DROP TABLE IF EXISTS b; + CREATE TABLE b (i UInt8) ENGINE=MergeTree() PRIMARY KEY(i) ORDER BY (i); + INSERT INTO b VALUES (1), (2), (3); +" + +${MYSQL_CLIENT} --batch --execute 'SELECT * FROM default.b WHERE i>=3;' +${MYSQL_CLIENT} --batch --execute 'SELECT * FROM default.b WHERE i>=300;' + +${MYSQL_CLIENT} <<<" + DROP TABLE b; +" From dafb75c6041dadc1f1e0e8a1882fa270e351426a Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sun, 15 Aug 2021 03:31:39 +0300 Subject: [PATCH 572/599] Update 02009_mysql_client_empty_result.sh --- tests/queries/0_stateless/02009_mysql_client_empty_result.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02009_mysql_client_empty_result.sh b/tests/queries/0_stateless/02009_mysql_client_empty_result.sh index fde2f57b4ee..cc1d0aece77 100755 --- a/tests/queries/0_stateless/02009_mysql_client_empty_result.sh +++ b/tests/queries/0_stateless/02009_mysql_client_empty_result.sh @@ -13,8 +13,8 @@ ${MYSQL_CLIENT} <<<" INSERT INTO b VALUES (1), (2), (3); " -${MYSQL_CLIENT} --batch --execute 'SELECT * FROM default.b WHERE i>=3;' -${MYSQL_CLIENT} --batch --execute 'SELECT * FROM default.b WHERE i>=300;' +${MYSQL_CLIENT} --batch --execute 'SELECT * FROM b WHERE i>=3;' +${MYSQL_CLIENT} --batch --execute 'SELECT * FROM b WHERE i>=300;' ${MYSQL_CLIENT} <<<" DROP TABLE b; From ef803f14a1ed067b15c5671f3761d86cf5c5e438 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sun, 15 Aug 2021 05:11:30 +0300 Subject: [PATCH 573/599] Update 85_bug-report.md --- .github/ISSUE_TEMPLATE/85_bug-report.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/ISSUE_TEMPLATE/85_bug-report.md b/.github/ISSUE_TEMPLATE/85_bug-report.md index bd59d17db3f..d78474670ff 100644 --- a/.github/ISSUE_TEMPLATE/85_bug-report.md +++ b/.github/ISSUE_TEMPLATE/85_bug-report.md @@ -2,7 +2,7 @@ name: Bug report about: Wrong behaviour (visible to users) in official ClickHouse release. title: '' -labels: bug +labels: 'potential bug' assignees: '' --- From ab4415a22b17f21517351412652800590a1ff594 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sun, 15 Aug 2021 05:28:38 +0300 Subject: [PATCH 574/599] Update MergeTreeData.cpp --- src/Storages/MergeTree/MergeTreeData.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index ebd961ab428..2892efab12d 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -3227,7 +3227,7 @@ String MergeTreeData::getPartitionIDFromQuery(const ASTPtr & ast, ContextPtr loc return partition_ast.id; } - if (const auto * partition_function = partition_ast.value->as()) + if (partition_ast.value->as()) checkPartitionExpressionFunction(ast); if (format_version < MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING) From f063e44131a048ba2d9af8075f03700fd5ec3e69 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 15 Aug 2021 09:00:28 +0300 Subject: [PATCH 575/599] Whitespaces --- base/mysqlxx/Pool.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/base/mysqlxx/Pool.cpp b/base/mysqlxx/Pool.cpp index 386b4544b78..2f47aa67356 100644 --- a/base/mysqlxx/Pool.cpp +++ b/base/mysqlxx/Pool.cpp @@ -296,7 +296,7 @@ void Pool::initialize() Pool::Connection * Pool::allocConnection(bool dont_throw_if_failed_first_time) { - std::unique_ptr conn_ptr{new Connection}; + std::unique_ptr conn_ptr = std::make_unique(); try { From cf87dacd4e6630a6f6a33afc8c66d409751836f7 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 15 Aug 2021 09:01:04 +0300 Subject: [PATCH 576/599] Disable memory tracking for roaring bitmaps on Mac OS --- contrib/croaring-cmake/CMakeLists.txt | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/contrib/croaring-cmake/CMakeLists.txt b/contrib/croaring-cmake/CMakeLists.txt index 84cdccedbd3..5223027843d 100644 --- a/contrib/croaring-cmake/CMakeLists.txt +++ b/contrib/croaring-cmake/CMakeLists.txt @@ -27,8 +27,10 @@ target_include_directories(roaring SYSTEM BEFORE PUBLIC "${LIBRARY_DIR}/cpp") # We redirect malloc/free family of functions to different functions that will track memory in ClickHouse. # Also note that we exploit implicit function declarations. +# Also it is disabled on Mac OS because it fails). -target_compile_definitions(roaring PRIVATE +if (NOT OS_DARWIN) + target_compile_definitions(roaring PRIVATE -Dmalloc=clickhouse_malloc -Dcalloc=clickhouse_calloc -Drealloc=clickhouse_realloc @@ -36,4 +38,5 @@ target_compile_definitions(roaring PRIVATE -Dfree=clickhouse_free -Dposix_memalign=clickhouse_posix_memalign) -target_link_libraries(roaring PUBLIC clickhouse_common_io) + target_link_libraries(roaring PUBLIC clickhouse_common_io) +endif () From 15eb68d117d89128ee91fe7494f456e72aaa5479 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 15 Aug 2021 09:33:08 +0300 Subject: [PATCH 577/599] Fix bad cast in arrayIndex #26330 --- src/Functions/array/arrayIndex.h | 7 +++++++ .../0_stateless/02010_array_index_bad_cast.reference | 0 tests/queries/0_stateless/02010_array_index_bad_cast.sql | 2 ++ 3 files changed, 9 insertions(+) create mode 100644 tests/queries/0_stateless/02010_array_index_bad_cast.reference create mode 100644 tests/queries/0_stateless/02010_array_index_bad_cast.sql diff --git a/src/Functions/array/arrayIndex.h b/src/Functions/array/arrayIndex.h index a390abc4eaf..d7bbcaf8d46 100644 --- a/src/Functions/array/arrayIndex.h +++ b/src/Functions/array/arrayIndex.h @@ -115,6 +115,13 @@ private: [[maybe_unused]] const NullMap * const null_map_data, [[maybe_unused]] const NullMap * const null_map_item) { + if constexpr (std::is_same_v && std::is_same_v) + { + /// Generic variant is using IColumn::compare function that only allows to compare columns of identical types. + if (typeid(data) != typeid(target)) + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Columns {} and {} cannot be compared", data.getName(), target.getName()); + } + const size_t size = offsets.size(); result.resize(size); diff --git a/tests/queries/0_stateless/02010_array_index_bad_cast.reference b/tests/queries/0_stateless/02010_array_index_bad_cast.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02010_array_index_bad_cast.sql b/tests/queries/0_stateless/02010_array_index_bad_cast.sql new file mode 100644 index 00000000000..19c58bb28a7 --- /dev/null +++ b/tests/queries/0_stateless/02010_array_index_bad_cast.sql @@ -0,0 +1,2 @@ +-- This query throws exception about uncomparable data types (but at least it does not introduce bad cast in code). +SELECT has(materialize(CAST(['2021-07-14'] AS Array(LowCardinality(Nullable(DateTime))))), materialize('2021-07-14'::DateTime64(7))); -- { serverError 44 } From 6ff43a614c6ab313f5d87a395a436bba52899bf2 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 15 Aug 2021 09:52:55 +0300 Subject: [PATCH 578/599] Use only SSE2 in "unbundled" build --- CMakeLists.txt | 5 +++-- docker/packager/packager | 8 +++++++- release | 3 --- 3 files changed, 10 insertions(+), 6 deletions(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index 1727caea766..35c22526816 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -395,9 +395,10 @@ endif () # Turns on all external libs like s3, kafka, ODBC, ... option(ENABLE_LIBRARIES "Enable all external libraries by default" ON) -# We recommend avoiding this mode for production builds because we can't guarantee all needed libraries exist in your -# system. +# We recommend avoiding this mode for production builds because we can't guarantee +# all needed libraries exist in your system. # This mode exists for enthusiastic developers who are searching for trouble. +# The whole idea of using unknown version of libraries from the OS distribution is deeply flawed. # Useful for maintainers of OS packages. option (UNBUNDLED "Use system libraries instead of ones in contrib/" OFF) diff --git a/docker/packager/packager b/docker/packager/packager index c05c85d3e28..857df079281 100755 --- a/docker/packager/packager +++ b/docker/packager/packager @@ -151,8 +151,14 @@ def parse_env_variables(build_type, compiler, sanitizer, package_type, image_typ cmake_flags.append('-DENABLE_TESTS=1') cmake_flags.append('-DUSE_GTEST=1') + # "Unbundled" build is not suitable for any production usage. + # But it is occasionally used by some developers. + # The whole idea of using unknown version of libraries from the OS distribution is deeply flawed. + # We wish these developers good luck. if unbundled: - cmake_flags.append('-DUNBUNDLED=1 -DUSE_INTERNAL_RDKAFKA_LIBRARY=1 -DENABLE_ARROW=0 -DENABLE_AVRO=0 -DENABLE_ORC=0 -DENABLE_PARQUET=0') + # We also disable all CPU features except basic x86_64. + # It is only slightly related to "unbundled" build, but it is a good place to test if code compiled without these instruction sets. + cmake_flags.append('-DUNBUNDLED=1 -DUSE_INTERNAL_RDKAFKA_LIBRARY=1 -DENABLE_ARROW=0 -DENABLE_AVRO=0 -DENABLE_ORC=0 -DENABLE_PARQUET=0 -DENABLE_SSSE3=0 -DENABLE_SSE41=0 -DENABLE_SSE42=0 -DENABLE_PCLMULQDQ=0 -DENABLE_POPCNT=0 -DENABLE_AVX=0 -DENABLE_AVX2=0') if split_binary: cmake_flags.append('-DUSE_STATIC_LIBRARIES=0 -DSPLIT_SHARED_LIBRARIES=1 -DCLICKHOUSE_SPLIT_BINARY=1') diff --git a/release b/release index 9484d79630a..de549595d43 100755 --- a/release +++ b/release @@ -71,9 +71,6 @@ then export DEB_CC=${DEB_CC=clang-10} export DEB_CXX=${DEB_CXX=clang++-10} EXTRAPACKAGES="$EXTRAPACKAGES clang-10 lld-10" -elif [[ $BUILD_TYPE == 'valgrind' ]]; then - MALLOC_OPTS="-DENABLE_TCMALLOC=0 -DENABLE_JEMALLOC=0" - VERSION_POSTFIX+="+valgrind" elif [[ $BUILD_TYPE == 'debug' ]]; then CMAKE_BUILD_TYPE=Debug VERSION_POSTFIX+="+debug" From c06f212bb5d4e8058334dc3b34fefea4174591e5 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sun, 15 Aug 2021 09:55:54 +0300 Subject: [PATCH 579/599] Update packager --- docker/packager/packager | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/packager/packager b/docker/packager/packager index 857df079281..95b7fcd8568 100755 --- a/docker/packager/packager +++ b/docker/packager/packager @@ -157,7 +157,7 @@ def parse_env_variables(build_type, compiler, sanitizer, package_type, image_typ # We wish these developers good luck. if unbundled: # We also disable all CPU features except basic x86_64. - # It is only slightly related to "unbundled" build, but it is a good place to test if code compiled without these instruction sets. + # It is only slightly related to "unbundled" build, but it is a good place to test if code compiles without these instruction sets. cmake_flags.append('-DUNBUNDLED=1 -DUSE_INTERNAL_RDKAFKA_LIBRARY=1 -DENABLE_ARROW=0 -DENABLE_AVRO=0 -DENABLE_ORC=0 -DENABLE_PARQUET=0 -DENABLE_SSSE3=0 -DENABLE_SSE41=0 -DENABLE_SSE42=0 -DENABLE_PCLMULQDQ=0 -DENABLE_POPCNT=0 -DENABLE_AVX=0 -DENABLE_AVX2=0') if split_binary: From ba0ba988bafa027b08541dd9c34286fecc53518b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 15 Aug 2021 11:21:30 +0300 Subject: [PATCH 580/599] Fix build --- contrib/libmetrohash/CMakeLists.txt | 4 ---- 1 file changed, 4 deletions(-) diff --git a/contrib/libmetrohash/CMakeLists.txt b/contrib/libmetrohash/CMakeLists.txt index 9304cb3644c..4ec5a58717d 100644 --- a/contrib/libmetrohash/CMakeLists.txt +++ b/contrib/libmetrohash/CMakeLists.txt @@ -2,9 +2,5 @@ set (SRCS src/metrohash64.cpp src/metrohash128.cpp ) -if (HAVE_SSE42) # Not used. Pretty easy to port. - list (APPEND SRCS src/metrohash128crc.cpp) -endif () - add_library(metrohash ${SRCS}) target_include_directories(metrohash PUBLIC src) From 97d921d4da75105094e869ed7b25730946569eeb Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 15 Aug 2021 11:38:16 +0300 Subject: [PATCH 581/599] Remove trash --- src/Compression/CompressedWriteBuffer.cpp | 4 ---- 1 file changed, 4 deletions(-) diff --git a/src/Compression/CompressedWriteBuffer.cpp b/src/Compression/CompressedWriteBuffer.cpp index 8d146e8de23..1c15dd77f14 100644 --- a/src/Compression/CompressedWriteBuffer.cpp +++ b/src/Compression/CompressedWriteBuffer.cpp @@ -30,10 +30,6 @@ void CompressedWriteBuffer::nextImpl() compressed_buffer.resize(compressed_reserve_size); UInt32 compressed_size = codec->compress(working_buffer.begin(), decompressed_size, compressed_buffer.data()); - // FIXME remove this after fixing msan report in lz4. - // Almost always reproduces on stateless tests, the exact test unknown. - __msan_unpoison(compressed_buffer.data(), compressed_size); - CityHash_v1_0_2::uint128 checksum = CityHash_v1_0_2::CityHash128(compressed_buffer.data(), compressed_size); out.write(reinterpret_cast(&checksum), CHECKSUM_SIZE); out.write(compressed_buffer.data(), compressed_size); From f66e8464f94b41624099d2a71151a52778ed74d8 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sun, 15 Aug 2021 09:55:43 +0300 Subject: [PATCH 582/599] Some final fixes --- .../InterpreterSelectIntersectExceptQuery.cpp | 7 +- .../InterpreterSelectIntersectExceptQuery.h | 5 +- .../SelectIntersectExceptQueryVisitor.cpp | 6 +- src/Parsers/ASTSelectIntersectExceptQuery.cpp | 5 +- src/Parsers/ExpressionListParsers.cpp | 86 +++---------------- src/Parsers/ExpressionListParsers.h | 17 +++- src/Parsers/ParserQueryWithOutput.cpp | 1 + src/Parsers/ParserUnionQueryElement.cpp | 3 +- .../QueryPlan/IntersectOrExceptStep.cpp | 6 +- .../Transforms/IntersectOrExceptTransform.cpp | 1 + ...02004_intersect_except_operators.reference | 5 ++ .../02004_intersect_except_operators.sql | 4 + .../02007_test_any_all_operators.reference | 20 +++++ .../02007_test_any_all_operators.sql | 11 ++- 14 files changed, 83 insertions(+), 94 deletions(-) diff --git a/src/Interpreters/InterpreterSelectIntersectExceptQuery.cpp b/src/Interpreters/InterpreterSelectIntersectExceptQuery.cpp index 4edd13d08e5..9c8dda56b44 100644 --- a/src/Interpreters/InterpreterSelectIntersectExceptQuery.cpp +++ b/src/Interpreters/InterpreterSelectIntersectExceptQuery.cpp @@ -89,7 +89,6 @@ InterpreterSelectIntersectExceptQuery::buildCurrentChildInterpreter(const ASTPtr if (ast_ptr_->as()) return std::make_unique(ast_ptr_, context, SelectQueryOptions()); - // if (ast_ptr_->as()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected query: {}", ast_ptr_->getID()); } @@ -140,4 +139,10 @@ BlockIO InterpreterSelectIntersectExceptQuery::execute() return res; } +void InterpreterSelectIntersectExceptQuery::ignoreWithTotals() +{ + for (auto & interpreter : nested_interpreters) + interpreter->ignoreWithTotals(); +} + } diff --git a/src/Interpreters/InterpreterSelectIntersectExceptQuery.h b/src/Interpreters/InterpreterSelectIntersectExceptQuery.h index 9cbde055b0b..805565e4c51 100644 --- a/src/Interpreters/InterpreterSelectIntersectExceptQuery.h +++ b/src/Interpreters/InterpreterSelectIntersectExceptQuery.h @@ -28,6 +28,8 @@ public: Block getSampleBlock() { return result_header; } + void ignoreWithTotals() override; + private: static String getName() { return "SelectIntersectExceptQuery"; } @@ -36,9 +38,8 @@ private: void buildQueryPlan(QueryPlan & query_plan) override; - void ignoreWithTotals() override {} - std::vector> nested_interpreters; + Operator final_operator; }; diff --git a/src/Interpreters/SelectIntersectExceptQueryVisitor.cpp b/src/Interpreters/SelectIntersectExceptQueryVisitor.cpp index 190ec279038..e26c4371591 100644 --- a/src/Interpreters/SelectIntersectExceptQueryVisitor.cpp +++ b/src/Interpreters/SelectIntersectExceptQueryVisitor.cpp @@ -12,11 +12,11 @@ namespace ErrorCodes /* * Note: there is a difference between intersect and except behaviour. - * `intersect` is supposed to be a part of last SelectQuery, i.e. the sequence with no parenthesis: + * `intersect` is supposed to be a part of the last SelectQuery, i.e. the sequence with no parenthesis: * select 1 union all select 2 except select 1 intersect 2 except select 2 union distinct select 5; * is interpreted as: * select 1 union all select 2 except (select 1 intersect 2) except select 2 union distinct select 5; - * Whereas `except` is applied to all union part like: + * Whereas `except` is applied to all left union part like: * (((select 1 union all select 2) except (select 1 intersect 2)) except select 2) union distinct select 5; **/ @@ -28,7 +28,7 @@ void SelectIntersectExceptQueryMatcher::visit(ASTPtr & ast, Data & data) void SelectIntersectExceptQueryMatcher::visit(ASTSelectWithUnionQuery & ast, Data &) { - auto & union_modes = ast.list_of_modes; + const auto & union_modes = ast.list_of_modes; if (union_modes.empty()) return; diff --git a/src/Parsers/ASTSelectIntersectExceptQuery.cpp b/src/Parsers/ASTSelectIntersectExceptQuery.cpp index 9d7a717fa6c..3b9cb0a2c16 100644 --- a/src/Parsers/ASTSelectIntersectExceptQuery.cpp +++ b/src/Parsers/ASTSelectIntersectExceptQuery.cpp @@ -30,11 +30,10 @@ void ASTSelectIntersectExceptQuery::formatQueryImpl(const FormatSettings & setti { settings.ostr << settings.nl_or_ws << indent_str << (settings.hilite ? hilite_keyword : "") << (final_operator == Operator::INTERSECT ? "INTERSECT" : "EXCEPT") - << (settings.hilite ? hilite_none : ""); + << (settings.hilite ? hilite_none : "") + << settings.nl_or_ws; } - if (it != children.begin()) - settings.ostr << settings.nl_or_ws; (*it)->formatImpl(settings, state, frame); } } diff --git a/src/Parsers/ExpressionListParsers.cpp b/src/Parsers/ExpressionListParsers.cpp index 69d95422799..58f5e766905 100644 --- a/src/Parsers/ExpressionListParsers.cpp +++ b/src/Parsers/ExpressionListParsers.cpp @@ -277,79 +277,6 @@ static bool modifyAST(ASTPtr ast, SubqueryFunctionType type) return true; } -bool ParserComparisonExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) -{ - bool first = true; - - auto current_depth = pos.depth; - while (true) - { - if (first) - { - ASTPtr elem; - if (!elem_parser.parse(pos, elem, expected)) - return false; - - node = elem; - first = false; - } - else - { - /// try to find any of the valid operators - const char ** it; - Expected stub; - for (it = overlapping_operators_to_skip; *it; ++it) - if (ParserKeyword{*it}.checkWithoutMoving(pos, stub)) - break; - - if (*it) - break; - - for (it = operators; *it; it += 2) - if (parseOperator(pos, *it, expected)) - break; - - if (!*it) - break; - - /// the function corresponding to the operator - auto function = std::make_shared(); - - /// function arguments - auto exp_list = std::make_shared(); - - ASTPtr elem; - SubqueryFunctionType subquery_function_type = SubqueryFunctionType::NONE; - if (ParserKeyword("ANY").ignore(pos, expected)) - subquery_function_type = SubqueryFunctionType::ANY; - else if (ParserKeyword("ALL").ignore(pos, expected)) - subquery_function_type = SubqueryFunctionType::ALL; - else if (!elem_parser.parse(pos, elem, expected)) - return false; - - if (subquery_function_type != SubqueryFunctionType::NONE && !ParserSubquery().parse(pos, elem, expected)) - return false; - - /// the first argument of the function is the previous element, the second is the next one - function->name = it[1]; - function->arguments = exp_list; - function->children.push_back(exp_list); - - exp_list->children.push_back(node); - exp_list->children.push_back(elem); - - if (subquery_function_type != SubqueryFunctionType::NONE && !modifyAST(function, subquery_function_type)) - return false; - - pos.increaseDepth(); - node = function; - } - } - - pos.depth = current_depth; - return true; -} - bool ParserLeftAssociativeBinaryOperatorList::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { bool first = true; @@ -393,7 +320,15 @@ bool ParserLeftAssociativeBinaryOperatorList::parseImpl(Pos & pos, ASTPtr & node auto exp_list = std::make_shared(); ASTPtr elem; - if (!(remaining_elem_parser ? remaining_elem_parser : first_elem_parser)->parse(pos, elem, expected)) + SubqueryFunctionType subquery_function_type = SubqueryFunctionType::NONE; + if (allow_any_all_operators && ParserKeyword("ANY").ignore(pos, expected)) + subquery_function_type = SubqueryFunctionType::ANY; + else if (allow_any_all_operators && ParserKeyword("ALL").ignore(pos, expected)) + subquery_function_type = SubqueryFunctionType::ALL; + else if (!(remaining_elem_parser ? remaining_elem_parser : first_elem_parser)->parse(pos, elem, expected)) + return false; + + if (subquery_function_type != SubqueryFunctionType::NONE && !ParserSubquery().parse(pos, elem, expected)) return false; /// the first argument of the function is the previous element, the second is the next one @@ -404,6 +339,9 @@ bool ParserLeftAssociativeBinaryOperatorList::parseImpl(Pos & pos, ASTPtr & node exp_list->children.push_back(node); exp_list->children.push_back(elem); + if (allow_any_all_operators && subquery_function_type != SubqueryFunctionType::NONE && !modifyAST(function, subquery_function_type)) + return false; + /** special exception for the access operator to the element of the array `x[y]`, which * contains the infix part '[' and the suffix ''] '(specified as' [') */ diff --git a/src/Parsers/ExpressionListParsers.h b/src/Parsers/ExpressionListParsers.h index e44cacb313f..17deec4e9e4 100644 --- a/src/Parsers/ExpressionListParsers.h +++ b/src/Parsers/ExpressionListParsers.h @@ -121,6 +121,8 @@ private: Operators_t overlapping_operators_to_skip = { (const char *[]){ nullptr } }; ParserPtr first_elem_parser; ParserPtr remaining_elem_parser; + /// =, !=, <, > ALL (subquery) / ANY (subquery) + bool allow_any_all_operators = false; public: /** `operators_` - allowed operators and their corresponding functions @@ -130,8 +132,10 @@ public: { } - ParserLeftAssociativeBinaryOperatorList(Operators_t operators_, Operators_t overlapping_operators_to_skip_, ParserPtr && first_elem_parser_) - : operators(operators_), overlapping_operators_to_skip(overlapping_operators_to_skip_), first_elem_parser(std::move(first_elem_parser_)) + ParserLeftAssociativeBinaryOperatorList(Operators_t operators_, + Operators_t overlapping_operators_to_skip_, ParserPtr && first_elem_parser_, bool allow_any_all_operators_ = false) + : operators(operators_), overlapping_operators_to_skip(overlapping_operators_to_skip_), + first_elem_parser(std::move(first_elem_parser_)), allow_any_all_operators(allow_any_all_operators_) { } @@ -341,12 +345,16 @@ class ParserComparisonExpression : public IParserBase private: static const char * operators[]; static const char * overlapping_operators_to_skip[]; - ParserBetweenExpression elem_parser; + ParserLeftAssociativeBinaryOperatorList operator_parser {operators, + overlapping_operators_to_skip, std::make_unique(), true}; protected: const char * getName() const override{ return "comparison expression"; } - bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override + { + return operator_parser.parse(pos, node, expected); + } }; /** Parser for nullity checking with IS (NOT) NULL. @@ -355,6 +363,7 @@ class ParserNullityChecking : public IParserBase { private: ParserComparisonExpression elem_parser; + protected: const char * getName() const override { return "nullity checking"; } bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; diff --git a/src/Parsers/ParserQueryWithOutput.cpp b/src/Parsers/ParserQueryWithOutput.cpp index 4a73952674c..82f9f561187 100644 --- a/src/Parsers/ParserQueryWithOutput.cpp +++ b/src/Parsers/ParserQueryWithOutput.cpp @@ -24,6 +24,7 @@ #include #include + namespace DB { diff --git a/src/Parsers/ParserUnionQueryElement.cpp b/src/Parsers/ParserUnionQueryElement.cpp index d59a7be2278..efd022e6362 100644 --- a/src/Parsers/ParserUnionQueryElement.cpp +++ b/src/Parsers/ParserUnionQueryElement.cpp @@ -10,8 +10,7 @@ namespace DB bool ParserUnionQueryElement::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { - if (!ParserSubquery().parse(pos, node, expected) - && !ParserSelectQuery().parse(pos, node, expected)) + if (!ParserSubquery().parse(pos, node, expected) && !ParserSelectQuery().parse(pos, node, expected)) return false; if (const auto * ast_subquery = node->as()) diff --git a/src/Processors/QueryPlan/IntersectOrExceptStep.cpp b/src/Processors/QueryPlan/IntersectOrExceptStep.cpp index b75898b815b..d1bb1eb41e9 100644 --- a/src/Processors/QueryPlan/IntersectOrExceptStep.cpp +++ b/src/Processors/QueryPlan/IntersectOrExceptStep.cpp @@ -36,10 +36,7 @@ IntersectOrExceptStep::IntersectOrExceptStep( , max_threads(max_threads_) { input_streams = std::move(input_streams_); - if (input_streams.size() == 1) - output_stream = input_streams.front(); - else - output_stream = DataStream{.header = header}; + output_stream = DataStream{.header = header}; } QueryPipelinePtr IntersectOrExceptStep::updatePipeline(QueryPipelines pipelines, const BuildQueryPipelineSettings &) @@ -71,6 +68,7 @@ QueryPipelinePtr IntersectOrExceptStep::updatePipeline(QueryPipelines pipelines, }); } + /// For the case of union. cur_pipeline->addTransform(std::make_shared(header, cur_pipeline->getNumStreams(), 1)); } diff --git a/src/Processors/Transforms/IntersectOrExceptTransform.cpp b/src/Processors/Transforms/IntersectOrExceptTransform.cpp index abfd1a7f0ad..3e39123ae4b 100644 --- a/src/Processors/Transforms/IntersectOrExceptTransform.cpp +++ b/src/Processors/Transforms/IntersectOrExceptTransform.cpp @@ -4,6 +4,7 @@ namespace DB { +/// After visitor is applied, ASTSelectIntersectExcept always has two child nodes. IntersectOrExceptTransform::IntersectOrExceptTransform(const Block & header_, Operator operator_) : IProcessor(InputPorts(2, header_), {header_}) , current_operator(operator_) diff --git a/tests/queries/0_stateless/02004_intersect_except_operators.reference b/tests/queries/0_stateless/02004_intersect_except_operators.reference index 7f41faaf83a..03b881f690b 100644 --- a/tests/queries/0_stateless/02004_intersect_except_operators.reference +++ b/tests/queries/0_stateless/02004_intersect_except_operators.reference @@ -70,6 +70,10 @@ select * from (select 1 intersect select 1); 1 with (select number from numbers(10) intersect select 5) as a select a * 10; 50 +with (select 5 except select 1) as a select a except select 5; +with (select number from numbers(10) intersect select 5) as a select a intersect select 1; +with (select number from numbers(10) intersect select 5) as a select a except select 1; +5 select count() from (select number from numbers(10) except select 5); 9 select count() from (select number from numbers(1000000) intersect select number from numbers(200000, 600000)); @@ -102,6 +106,7 @@ select * from (select 1 union all select 2 union all select 3 union all select 4 select 1 intersect (select 1 except select 2); 1 select 1 union all select 2 except (select 2 except select 1 union all select 1) except select 4; +select 1 intersect select count() from (select 1 except select 2 intersect select 2 union all select 1); explain syntax select 1 intersect select 1; SELECT 1 INTERSECT diff --git a/tests/queries/0_stateless/02004_intersect_except_operators.sql b/tests/queries/0_stateless/02004_intersect_except_operators.sql index ef0e52da116..7f08cc0adf2 100644 --- a/tests/queries/0_stateless/02004_intersect_except_operators.sql +++ b/tests/queries/0_stateless/02004_intersect_except_operators.sql @@ -21,6 +21,9 @@ select number from numbers(100) intersect select number from numbers(20, 60) exc select * from (select 1 intersect select 1); with (select number from numbers(10) intersect select 5) as a select a * 10; +with (select 5 except select 1) as a select a except select 5; +with (select number from numbers(10) intersect select 5) as a select a intersect select 1; +with (select number from numbers(10) intersect select 5) as a select a except select 1; select count() from (select number from numbers(10) except select 5); select count() from (select number from numbers(1000000) intersect select number from numbers(200000, 600000)); select count() from (select number from numbers(100) intersect select number from numbers(20, 60) except select number from numbers(30, 20) except select number from numbers(60, 20)); @@ -35,6 +38,7 @@ select * from (select 1 union all select 2 union all select 3 union all select 4 select 1 intersect (select 1 except select 2); select 1 union all select 2 except (select 2 except select 1 union all select 1) except select 4; +select 1 intersect select count() from (select 1 except select 2 intersect select 2 union all select 1); explain syntax select 1 intersect select 1; explain syntax select 1 except select 1; diff --git a/tests/queries/0_stateless/02007_test_any_all_operators.reference b/tests/queries/0_stateless/02007_test_any_all_operators.reference index ebd7cd8f6ca..a232320d15c 100644 --- a/tests/queries/0_stateless/02007_test_any_all_operators.reference +++ b/tests/queries/0_stateless/02007_test_any_all_operators.reference @@ -29,3 +29,23 @@ select number as a from numbers(10) where a != any (select 5 from numbers(3, 3)) 7 8 9 +select 1 < any (select 1 from numbers(10)); +0 +select 1 <= any (select 1 from numbers(10)); +1 +select 1 < any (select number from numbers(10)); +1 +select 1 > any (select number from numbers(10)); +1 +select 1 >= any (select number from numbers(10)); +1 +select 11 > all (select number from numbers(10)); +1 +select 11 <= all (select number from numbers(11)); +0 +select 11 < all (select 11 from numbers(10)); +0 +select 11 > all (select 11 from numbers(10)); +0 +select 11 >= all (select 11 from numbers(10)); +1 diff --git a/tests/queries/0_stateless/02007_test_any_all_operators.sql b/tests/queries/0_stateless/02007_test_any_all_operators.sql index 525f7e1fabd..10d7325afca 100644 --- a/tests/queries/0_stateless/02007_test_any_all_operators.sql +++ b/tests/queries/0_stateless/02007_test_any_all_operators.sql @@ -8,10 +8,19 @@ select 1 != all (select number from numbers(10)); select 1 == all (select 1 from numbers(10)); select 1 == all (select number from numbers(10)); - select 1 != any (select 1 from numbers(10)); select 1 != any (select number from numbers(10)); select number as a from numbers(10) where a == any (select number from numbers(3, 3)); select number as a from numbers(10) where a != any (select 5 from numbers(3, 3)); +select 1 < any (select 1 from numbers(10)); +select 1 <= any (select 1 from numbers(10)); +select 1 < any (select number from numbers(10)); +select 1 > any (select number from numbers(10)); +select 1 >= any (select number from numbers(10)); +select 11 > all (select number from numbers(10)); +select 11 <= all (select number from numbers(11)); +select 11 < all (select 11 from numbers(10)); +select 11 > all (select 11 from numbers(10)); +select 11 >= all (select 11 from numbers(10)); From 9ae92798d109e58e95c3a50d3e50227624d98b9a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 15 Aug 2021 11:44:12 +0300 Subject: [PATCH 583/599] Fix stress test in ~CompressedWriteBuffer --- src/Compression/CompressedWriteBuffer.cpp | 7 +++++++ src/Compression/CompressedWriteBuffer.h | 1 + 2 files changed, 8 insertions(+) diff --git a/src/Compression/CompressedWriteBuffer.cpp b/src/Compression/CompressedWriteBuffer.cpp index 1c15dd77f14..7454e09fda0 100644 --- a/src/Compression/CompressedWriteBuffer.cpp +++ b/src/Compression/CompressedWriteBuffer.cpp @@ -36,6 +36,12 @@ void CompressedWriteBuffer::nextImpl() } +void CompressedWriteBuffer::finalize() +{ + next(); +} + + CompressedWriteBuffer::CompressedWriteBuffer( WriteBuffer & out_, CompressionCodecPtr codec_, @@ -44,6 +50,7 @@ CompressedWriteBuffer::CompressedWriteBuffer( { } + CompressedWriteBuffer::~CompressedWriteBuffer() { /// FIXME move final flush into the caller diff --git a/src/Compression/CompressedWriteBuffer.h b/src/Compression/CompressedWriteBuffer.h index a9612b463a5..2268b7bec50 100644 --- a/src/Compression/CompressedWriteBuffer.h +++ b/src/Compression/CompressedWriteBuffer.h @@ -22,6 +22,7 @@ private: PODArray compressed_buffer; void nextImpl() override; + void finalize() override; public: CompressedWriteBuffer( From e28c9c0ba7cea51ffe5b2d1c41d13bc0832270d1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 15 Aug 2021 11:54:28 +0300 Subject: [PATCH 584/599] Mark tests for DatabaseReplicated as green --- docker/test/stateless/process_functional_tests_result.py | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/docker/test/stateless/process_functional_tests_result.py b/docker/test/stateless/process_functional_tests_result.py index e60424ad4d1..a42b0e68d88 100755 --- a/docker/test/stateless/process_functional_tests_result.py +++ b/docker/test/stateless/process_functional_tests_result.py @@ -105,6 +105,10 @@ def process_result(result_path): description += ", skipped: {}".format(skipped) if unknown != 0: description += ", unknown: {}".format(unknown) + + # Temporary green for tests with DatabaseReplicated: + if 1 == int(os.environ.get('USE_DATABASE_REPLICATED', 0)): + state = "success" else: state = "failure" description = "Output log doesn't exist" From 86694a2bbbb049f74e80608527a39fc1fde9aa22 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sun, 15 Aug 2021 12:04:38 +0300 Subject: [PATCH 585/599] Update CompressedWriteBuffer.h --- src/Compression/CompressedWriteBuffer.h | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Compression/CompressedWriteBuffer.h b/src/Compression/CompressedWriteBuffer.h index 2268b7bec50..57ba679855e 100644 --- a/src/Compression/CompressedWriteBuffer.h +++ b/src/Compression/CompressedWriteBuffer.h @@ -22,7 +22,6 @@ private: PODArray compressed_buffer; void nextImpl() override; - void finalize() override; public: CompressedWriteBuffer( @@ -30,6 +29,8 @@ public: CompressionCodecPtr codec_ = CompressionCodecFactory::instance().getDefaultCodec(), size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE); + void finalize() override; + /// The amount of compressed data size_t getCompressedBytes() { From 5f5470c2cdefdf57e6de9bd49b7312f7bc2a3e50 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Sun, 15 Aug 2021 13:32:56 +0300 Subject: [PATCH 586/599] Removed DenseHashMap, DenseHashSet --- src/Common/DenseHashMap.h | 29 ------ src/Common/DenseHashSet.h | 25 ----- src/Common/SparseHashMap.h | 1 - src/Core/NamesAndTypes.cpp | 12 +-- src/Storages/MergeTree/IMergeTreeReader.cpp | 9 +- src/Storages/MergeTree/IMergeTreeReader.h | 5 +- src/Storages/StorageInMemoryMetadata.cpp | 102 +++++++++++--------- 7 files changed, 72 insertions(+), 111 deletions(-) delete mode 100644 src/Common/DenseHashMap.h delete mode 100644 src/Common/DenseHashSet.h diff --git a/src/Common/DenseHashMap.h b/src/Common/DenseHashMap.h deleted file mode 100644 index 9ac21c82676..00000000000 --- a/src/Common/DenseHashMap.h +++ /dev/null @@ -1,29 +0,0 @@ -#pragma once -#include - -/// DenseHashMap is a wrapper for google::dense_hash_map. -/// Some hacks are needed to make it work in "Arcadia". -/// "Arcadia" is a proprietary monorepository in Yandex. -/// It uses slightly changed version of sparsehash with a different set of hash functions (which we don't need). -/// Those defines are needed to make it compile. -#if defined(ARCADIA_BUILD) -#define HASH_FUN_H -template -struct THash; -#endif - -#include - -#if !defined(ARCADIA_BUILD) - template , - class EqualKey = std::equal_to, - class Alloc = google::libc_allocator_with_realloc>> - using DenseHashMap = google::dense_hash_map; -#else - template , - class EqualKey = std::equal_to, - class Alloc = google::sparsehash::libc_allocator_with_realloc>> - using DenseHashMap = google::sparsehash::dense_hash_map; - - #undef THash -#endif diff --git a/src/Common/DenseHashSet.h b/src/Common/DenseHashSet.h deleted file mode 100644 index e8c06f36aa3..00000000000 --- a/src/Common/DenseHashSet.h +++ /dev/null @@ -1,25 +0,0 @@ -#pragma once - -/// DenseHashSet is a wrapper for google::dense_hash_set. -/// See comment in DenseHashMap.h -#if defined(ARCADIA_BUILD) -#define HASH_FUN_H -template -struct THash; -#endif - -#include - -#if !defined(ARCADIA_BUILD) - template , - class EqualKey = std::equal_to, - class Alloc = google::libc_allocator_with_realloc> - using DenseHashSet = google::dense_hash_set; -#else - template , - class EqualKey = std::equal_to, - class Alloc = google::sparsehash::libc_allocator_with_realloc> - using DenseHashSet = google::sparsehash::dense_hash_set; - - #undef THash -#endif diff --git a/src/Common/SparseHashMap.h b/src/Common/SparseHashMap.h index f01fc633d84..0f86cc13612 100644 --- a/src/Common/SparseHashMap.h +++ b/src/Common/SparseHashMap.h @@ -1,7 +1,6 @@ #pragma once /// SparseHashMap is a wrapper for google::sparse_hash_map. -/// See comment in DenseHashMap.h #if defined(ARCADIA_BUILD) #define HASH_FUN_H template diff --git a/src/Core/NamesAndTypes.cpp b/src/Core/NamesAndTypes.cpp index 54f83fc13fc..b47f5a6823b 100644 --- a/src/Core/NamesAndTypes.cpp +++ b/src/Core/NamesAndTypes.cpp @@ -1,4 +1,5 @@ #include +#include #include #include #include @@ -6,7 +7,6 @@ #include #include #include -#include namespace DB @@ -163,8 +163,7 @@ NamesAndTypesList NamesAndTypesList::filter(const Names & names) const NamesAndTypesList NamesAndTypesList::addTypes(const Names & names) const { /// NOTE: It's better to make a map in `IStorage` than to create it here every time again. - DenseHashMap types; - types.set_empty_key(StringRef()); + HashMapWithSavedHash types; for (const auto & column : *this) types[column.name] = &column.type; @@ -172,10 +171,11 @@ NamesAndTypesList NamesAndTypesList::addTypes(const Names & names) const NamesAndTypesList res; for (const String & name : names) { - auto it = types.find(name); + const auto * it = types.find(name); if (it == types.end()) - throw Exception("No column " + name, ErrorCodes::THERE_IS_NO_COLUMN); - res.emplace_back(name, *it->second); + throw Exception(ErrorCodes::THERE_IS_NO_COLUMN, "No column {}", name); + + res.emplace_back(name, *it->getMapped()); } return res; diff --git a/src/Storages/MergeTree/IMergeTreeReader.cpp b/src/Storages/MergeTree/IMergeTreeReader.cpp index 5378b84a5d0..d659259e1a9 100644 --- a/src/Storages/MergeTree/IMergeTreeReader.cpp +++ b/src/Storages/MergeTree/IMergeTreeReader.cpp @@ -48,7 +48,6 @@ IMergeTreeReader::IMergeTreeReader( part_columns = Nested::collect(part_columns); } - columns_from_part.set_empty_key(StringRef()); for (const auto & column_from_part : part_columns) columns_from_part[column_from_part.name] = &column_from_part.type; } @@ -213,7 +212,7 @@ NameAndTypePair IMergeTreeReader::getColumnFromPart(const NameAndTypePair & requ { auto name_in_storage = required_column.getNameInStorage(); - decltype(columns_from_part.begin()) it; + ColumnsFromPart::ConstLookupResult it; if (alter_conversions.isColumnRenamed(name_in_storage)) { String old_name = alter_conversions.getColumnOldName(name_in_storage); @@ -227,7 +226,7 @@ NameAndTypePair IMergeTreeReader::getColumnFromPart(const NameAndTypePair & requ if (it == columns_from_part.end()) return required_column; - const auto & type = *it->second; + const DataTypePtr & type = *it->getMapped(); if (required_column.isSubcolumn()) { auto subcolumn_name = required_column.getSubcolumnName(); @@ -236,10 +235,10 @@ NameAndTypePair IMergeTreeReader::getColumnFromPart(const NameAndTypePair & requ if (!subcolumn_type) return required_column; - return {String(it->first), subcolumn_name, type, subcolumn_type}; + return {String(it->getKey()), subcolumn_name, type, subcolumn_type}; } - return {String(it->first), type}; + return {String(it->getKey()), type}; } void IMergeTreeReader::performRequiredConversions(Columns & res_columns) diff --git a/src/Storages/MergeTree/IMergeTreeReader.h b/src/Storages/MergeTree/IMergeTreeReader.h index 8d80719efaf..696cc2f105b 100644 --- a/src/Storages/MergeTree/IMergeTreeReader.h +++ b/src/Storages/MergeTree/IMergeTreeReader.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include #include #include @@ -95,7 +95,8 @@ private: /// Actual data type of columns in part - DenseHashMap columns_from_part; + using ColumnsFromPart = HashMapWithSavedHash; + ColumnsFromPart columns_from_part; }; } diff --git a/src/Storages/StorageInMemoryMetadata.cpp b/src/Storages/StorageInMemoryMetadata.cpp index 91f69cdac7d..5183b925141 100644 --- a/src/Storages/StorageInMemoryMetadata.cpp +++ b/src/Storages/StorageInMemoryMetadata.cpp @@ -1,7 +1,7 @@ #include -#include -#include +#include +#include #include #include #include @@ -320,8 +320,7 @@ Block StorageInMemoryMetadata::getSampleBlockForColumns( { Block res; - DenseHashMap virtuals_map; - virtuals_map.set_empty_key(StringRef()); + HashMapWithSavedHash virtuals_map; /// Virtual columns must be appended after ordinary, because user can /// override them. @@ -335,9 +334,9 @@ Block StorageInMemoryMetadata::getSampleBlockForColumns( { res.insert({column->type->createColumn(), column->type, column->name}); } - else if (auto it = virtuals_map.find(name); it != virtuals_map.end()) + else if (auto * it = virtuals_map.find(name); it != virtuals_map.end()) { - const auto & type = *it->second; + const auto & type = *it->getMapped(); res.insert({type->createColumn(), type, name}); } else @@ -470,8 +469,8 @@ bool StorageInMemoryMetadata::hasSelectQuery() const namespace { - using NamesAndTypesMap = DenseHashMap; - using UniqueStrings = DenseHashSet; + using NamesAndTypesMap = HashMapWithSavedHash; + using UniqueStrings = HashSetWithSavedHash; String listOfColumns(const NamesAndTypesList & available_columns) { @@ -488,20 +487,12 @@ namespace NamesAndTypesMap getColumnsMap(const NamesAndTypesList & columns) { NamesAndTypesMap res; - res.set_empty_key(StringRef()); for (const auto & column : columns) res.insert({column.name, column.type.get()}); return res; } - - UniqueStrings initUniqueStrings() - { - UniqueStrings strings; - strings.set_empty_key(StringRef()); - return strings; - } } void StorageInMemoryMetadata::check(const Names & column_names, const NamesAndTypesList & virtuals, const StorageID & storage_id) const @@ -514,11 +505,12 @@ void StorageInMemoryMetadata::check(const Names & column_names, const NamesAndTy } const auto virtuals_map = getColumnsMap(virtuals); - auto unique_names = initUniqueStrings(); + UniqueStrings unique_names; for (const auto & name : column_names) { - bool has_column = getColumns().hasColumnOrSubcolumn(ColumnsDescription::AllPhysical, name) || virtuals_map.count(name); + bool has_column = getColumns().hasColumnOrSubcolumn(ColumnsDescription::AllPhysical, name) + || virtuals_map.find(name) != nullptr; if (!has_column) { @@ -540,23 +532,31 @@ void StorageInMemoryMetadata::check(const NamesAndTypesList & provided_columns) const NamesAndTypesList & available_columns = getColumns().getAllPhysical(); const auto columns_map = getColumnsMap(available_columns); - auto unique_names = initUniqueStrings(); + UniqueStrings unique_names; + for (const NameAndTypePair & column : provided_columns) { - auto it = columns_map.find(column.name); + const auto * it = columns_map.find(column.name); if (columns_map.end() == it) throw Exception( - "There is no column with name " + column.name + ". There are columns: " + listOfColumns(available_columns), - ErrorCodes::NO_SUCH_COLUMN_IN_TABLE); + ErrorCodes::NO_SUCH_COLUMN_IN_TABLE, + "There is no column with name {}. There are columns: {}", + column.name, + listOfColumns(available_columns)); - if (!column.type->equals(*it->second)) + if (!column.type->equals(*it->getMapped())) throw Exception( - "Type mismatch for column " + column.name + ". Column has type " + it->second->getName() + ", got type " - + column.type->getName(), - ErrorCodes::TYPE_MISMATCH); + ErrorCodes::TYPE_MISMATCH, + "Type mismatch for column {}. Column has type {}, got type {}", + column.name, + it->getMapped()->getName(), + column.type->getName()); if (unique_names.end() != unique_names.find(column.name)) - throw Exception("Column " + column.name + " queried more than once", ErrorCodes::COLUMN_QUERIED_MORE_THAN_ONCE); + throw Exception(ErrorCodes::COLUMN_QUERIED_MORE_THAN_ONCE, + "Column {} queried more than once", + column.name); + unique_names.insert(column.name); } } @@ -572,26 +572,38 @@ void StorageInMemoryMetadata::check(const NamesAndTypesList & provided_columns, "Empty list of columns queried. There are columns: " + listOfColumns(available_columns), ErrorCodes::EMPTY_LIST_OF_COLUMNS_QUERIED); - auto unique_names = initUniqueStrings(); + UniqueStrings unique_names; + for (const String & name : column_names) { - auto it = provided_columns_map.find(name); + const auto * it = provided_columns_map.find(name); if (provided_columns_map.end() == it) continue; - auto jt = available_columns_map.find(name); + const auto * jt = available_columns_map.find(name); if (available_columns_map.end() == jt) throw Exception( - "There is no column with name " + name + ". There are columns: " + listOfColumns(available_columns), - ErrorCodes::NO_SUCH_COLUMN_IN_TABLE); + ErrorCodes::NO_SUCH_COLUMN_IN_TABLE, + "There is no column with name {}. There are columns: {}", + name, + listOfColumns(available_columns)); - if (!it->second->equals(*jt->second)) + const auto & provided_column_type = *it->getMapped(); + const auto & available_column_type = *jt->getMapped(); + + if (!provided_column_type.equals(available_column_type)) throw Exception( - "Type mismatch for column " + name + ". Column has type " + jt->second->getName() + ", got type " + it->second->getName(), - ErrorCodes::TYPE_MISMATCH); + ErrorCodes::TYPE_MISMATCH, + "Type mismatch for column {}. Column has type {}, got type {}", + name, + provided_column_type.getName(), + available_column_type.getName()); if (unique_names.end() != unique_names.find(name)) - throw Exception("Column " + name + " queried more than once", ErrorCodes::COLUMN_QUERIED_MORE_THAN_ONCE); + throw Exception(ErrorCodes::COLUMN_QUERIED_MORE_THAN_ONCE, + "Column {} queried more than once", + name); + unique_names.insert(name); } } @@ -612,17 +624,21 @@ void StorageInMemoryMetadata::check(const Block & block, bool need_all) const names_in_block.insert(column.name); - auto it = columns_map.find(column.name); + const auto * it = columns_map.find(column.name); if (columns_map.end() == it) throw Exception( - "There is no column with name " + column.name + ". There are columns: " + listOfColumns(available_columns), - ErrorCodes::NO_SUCH_COLUMN_IN_TABLE); + ErrorCodes::NO_SUCH_COLUMN_IN_TABLE, + "There is no column with name {}. There are columns: {}", + column.name, + listOfColumns(available_columns)); - if (!column.type->equals(*it->second)) + if (!column.type->equals(*it->getMapped())) throw Exception( - "Type mismatch for column " + column.name + ". Column has type " + it->second->getName() + ", got type " - + column.type->getName(), - ErrorCodes::TYPE_MISMATCH); + ErrorCodes::TYPE_MISMATCH, + "Type mismatch for column {}. Column has type {}, got type {}", + column.name, + it->getMapped()->getName(), + column.type->getName()); } if (need_all && names_in_block.size() < columns_map.size()) From 404eac198684bf08273dfc91743c7c9cadb17534 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Sun, 15 Aug 2021 13:52:44 +0300 Subject: [PATCH 587/599] Fixed tests --- ...map_add_map_subtract_on_map_type.reference | 4 +-- ...01925_map_populate_series_on_map.reference | 34 +++++++++---------- 2 files changed, 19 insertions(+), 19 deletions(-) diff --git a/tests/queries/0_stateless/01318_map_add_map_subtract_on_map_type.reference b/tests/queries/0_stateless/01318_map_add_map_subtract_on_map_type.reference index 304f7407cf5..de34b856130 100644 --- a/tests/queries/0_stateless/01318_map_add_map_subtract_on_map_type.reference +++ b/tests/queries/0_stateless/01318_map_add_map_subtract_on_map_type.reference @@ -47,9 +47,9 @@ {'b':2} Map(Enum8(\'a\' = 1, \'b\' = 2), Int64) {'00000000-89ab-cdef-0123-456789abcdef':2} Map(UUID, Int64) {'11111111-89ab-cdef-0123-456789abcdef':4} Map(UUID, Int64) -{1:0,2:0} Map(UInt8,UInt64) +{1:0,2:0} Map(UInt8, UInt64) {1:18446744073709551615,2:18446744073709551615} Map(UInt8, UInt64) -{1:-1,2:-1} Map(UInt8,Int64) +{1:-1,2:-1} Map(UInt8, Int64) {1:-1.0999999761581423,2:0} Map(UInt8, Float64) {1:-1,2:-1} Map(UInt8, Int64) {1:-2,2:-2,3:1} Map(UInt8, Int64) diff --git a/tests/queries/0_stateless/01925_map_populate_series_on_map.reference b/tests/queries/0_stateless/01925_map_populate_series_on_map.reference index 235a227f548..fd3d3b2450d 100644 --- a/tests/queries/0_stateless/01925_map_populate_series_on_map.reference +++ b/tests/queries/0_stateless/01925_map_populate_series_on_map.reference @@ -29,39 +29,39 @@ select mapPopulateSeries(m, n) from map_test; {1:1,2:0,3:0,4:0,5:2,6:0} drop table map_test; select mapPopulateSeries(map(toUInt8(1), toUInt8(1), 2, 1)) as res, toTypeName(res); -{1:1,2:1} Map(UInt8,UInt8) +{1:1,2:1} Map(UInt8, UInt8) select mapPopulateSeries(map(toUInt16(1), toUInt16(1), 2, 1)) as res, toTypeName(res); -{1:1,2:1} Map(UInt16,UInt16) +{1:1,2:1} Map(UInt16, UInt16) select mapPopulateSeries(map(toUInt32(1), toUInt32(1), 2, 1)) as res, toTypeName(res); -{1:1,2:1} Map(UInt32,UInt32) +{1:1,2:1} Map(UInt32, UInt32) select mapPopulateSeries(map(toUInt64(1), toUInt64(1), 2, 1)) as res, toTypeName(res); -{1:1,2:1} Map(UInt64,UInt64) +{1:1,2:1} Map(UInt64, UInt64) select mapPopulateSeries(map(toUInt128(1), toUInt128(1), 2, 1)) as res, toTypeName(res); -{1:1,2:1} Map(UInt128,UInt128) +{1:1,2:1} Map(UInt128, UInt128) select mapPopulateSeries(map(toUInt256(1), toUInt256(1), 2, 1)) as res, toTypeName(res); -{1:1,2:1} Map(UInt256,UInt256) +{1:1,2:1} Map(UInt256, UInt256) select mapPopulateSeries(map(toInt8(1), toInt8(1), 2, 1)) as res, toTypeName(res); -{1:1,2:1} Map(Int16,Int16) +{1:1,2:1} Map(Int16, Int16) select mapPopulateSeries(map(toInt16(1), toInt16(1), 2, 1)) as res, toTypeName(res); -{1:1,2:1} Map(Int16,Int16) +{1:1,2:1} Map(Int16, Int16) select mapPopulateSeries(map(toInt32(1), toInt32(1), 2, 1)) as res, toTypeName(res); -{1:1,2:1} Map(Int32,Int32) +{1:1,2:1} Map(Int32, Int32) select mapPopulateSeries(map(toInt64(1), toInt64(1), 2, 1)) as res, toTypeName(res); -{1:1,2:1} Map(Int64,Int64) +{1:1,2:1} Map(Int64, Int64) select mapPopulateSeries(map(toInt128(1), toInt128(1), 2, 1)) as res, toTypeName(res); -{1:1,2:1} Map(Int128,Int128) +{1:1,2:1} Map(Int128, Int128) select mapPopulateSeries(map(toInt256(1), toInt256(1), 2, 1)) as res, toTypeName(res); -{1:1,2:1} Map(Int256,Int256) +{1:1,2:1} Map(Int256, Int256) select mapPopulateSeries(map(toInt8(-10), toInt8(1), 2, 1)) as res, toTypeName(res); -{-10:1,-9:0,-8:0,-7:0,-6:0,-5:0,-4:0,-3:0,-2:0,-1:0,0:0,1:0,2:1} Map(Int16,Int16) +{-10:1,-9:0,-8:0,-7:0,-6:0,-5:0,-4:0,-3:0,-2:0,-1:0,0:0,1:0,2:1} Map(Int16, Int16) select mapPopulateSeries(map(toInt16(-10), toInt16(1), 2, 1)) as res, toTypeName(res); -{-10:1,-9:0,-8:0,-7:0,-6:0,-5:0,-4:0,-3:0,-2:0,-1:0,0:0,1:0,2:1} Map(Int16,Int16) +{-10:1,-9:0,-8:0,-7:0,-6:0,-5:0,-4:0,-3:0,-2:0,-1:0,0:0,1:0,2:1} Map(Int16, Int16) select mapPopulateSeries(map(toInt32(-10), toInt32(1), 2, 1)) as res, toTypeName(res); -{-10:1,-9:0,-8:0,-7:0,-6:0,-5:0,-4:0,-3:0,-2:0,-1:0,0:0,1:0,2:1} Map(Int32,Int32) +{-10:1,-9:0,-8:0,-7:0,-6:0,-5:0,-4:0,-3:0,-2:0,-1:0,0:0,1:0,2:1} Map(Int32, Int32) select mapPopulateSeries(map(toInt64(-10), toInt64(1), 2, 1)) as res, toTypeName(res); -{-10:1,-9:0,-8:0,-7:0,-6:0,-5:0,-4:0,-3:0,-2:0,-1:0,0:0,1:0,2:1} Map(Int64,Int64) +{-10:1,-9:0,-8:0,-7:0,-6:0,-5:0,-4:0,-3:0,-2:0,-1:0,0:0,1:0,2:1} Map(Int64, Int64) select mapPopulateSeries(map(toInt64(-10), toInt64(1), 2, 1), toInt64(-5)) as res, toTypeName(res); -{-10:1,-9:0,-8:0,-7:0,-6:0,-5:0} Map(Int64,Int64) +{-10:1,-9:0,-8:0,-7:0,-6:0,-5:0} Map(Int64, Int64) select mapPopulateSeries(); -- { serverError 42 } select mapPopulateSeries('asdf'); -- { serverError 43 } select mapPopulateSeries(map('1', 1, '2', 1)) as res, toTypeName(res); -- { serverError 43 } From 178d0f9ba95af192b4ae391f216224233269a6eb Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Tue, 10 Aug 2021 17:06:22 +0300 Subject: [PATCH 588/599] Fix checking GRANT OPTION while executing GRANT with REPLACE OPTION. --- src/Access/AccessRightsElement.h | 3 + src/Interpreters/InterpreterGrantQuery.cpp | 391 +++++++++++------- .../integration/test_grant_and_revoke/test.py | 33 ++ 3 files changed, 277 insertions(+), 150 deletions(-) diff --git a/src/Access/AccessRightsElement.h b/src/Access/AccessRightsElement.h index c76f019bc61..c46a4b54e6e 100644 --- a/src/Access/AccessRightsElement.h +++ b/src/Access/AccessRightsElement.h @@ -122,6 +122,9 @@ struct AccessRightsElement class AccessRightsElements : public std::vector { public: + using Base = std::vector; + using Base::Base; + bool empty() const { return std::all_of(begin(), end(), [](const AccessRightsElement & e) { return e.empty(); }); } bool sameDatabaseAndTable() const diff --git a/src/Interpreters/InterpreterGrantQuery.cpp b/src/Interpreters/InterpreterGrantQuery.cpp index 42c440b4c52..2e7b3a58012 100644 --- a/src/Interpreters/InterpreterGrantQuery.cpp +++ b/src/Interpreters/InterpreterGrantQuery.cpp @@ -22,69 +22,108 @@ namespace ErrorCodes namespace { - template - void updateFromQueryTemplate( - T & grantee, + /// Extracts access rights elements which are going to be granted or revoked from a query. + void collectAccessRightsElementsToGrantOrRevoke( const ASTGrantQuery & query, - const std::vector & roles_to_grant_or_revoke) + AccessRightsElements & elements_to_grant, + AccessRightsElements & elements_to_revoke) { - if (!query.is_revoke) + elements_to_grant.clear(); + elements_to_revoke.clear(); + + if (query.is_revoke) { - if (query.replace_access) - grantee.access = {}; - if (query.replace_granted_roles) - grantee.granted_roles = {}; + /// REVOKE + elements_to_revoke = query.access_rights_elements; } - - - if (!query.access_rights_elements.empty()) + else if (query.replace_access) { - if (query.is_revoke) - grantee.access.revoke(query.access_rights_elements); - else - grantee.access.grant(query.access_rights_elements); + /// GRANT WITH REPLACE OPTION + elements_to_grant = query.access_rights_elements; + elements_to_revoke.emplace_back(AccessType::ALL); } - - if (!roles_to_grant_or_revoke.empty()) + else { - if (query.is_revoke) - { - if (query.admin_option) - grantee.granted_roles.revokeAdminOption(roles_to_grant_or_revoke); - else - grantee.granted_roles.revoke(roles_to_grant_or_revoke); - } - else - { - if (query.admin_option) - grantee.granted_roles.grantWithAdminOption(roles_to_grant_or_revoke); - else - grantee.granted_roles.grant(roles_to_grant_or_revoke); - } + /// GRANT + elements_to_grant = query.access_rights_elements; } } - void updateFromQueryImpl( - IAccessEntity & grantee, + /// Extracts roles which are going to be granted or revoked from a query. + void collectRolesToGrantOrRevoke( + const AccessControlManager & access_control, const ASTGrantQuery & query, - const std::vector & roles_to_grant_or_revoke) + std::vector & roles_to_grant, + RolesOrUsersSet & roles_to_revoke) { - if (auto * user = typeid_cast(&grantee)) - updateFromQueryTemplate(*user, query, roles_to_grant_or_revoke); - else if (auto * role = typeid_cast(&grantee)) - updateFromQueryTemplate(*role, query, roles_to_grant_or_revoke); + roles_to_grant.clear(); + roles_to_revoke.clear(); + + RolesOrUsersSet roles_to_grant_or_revoke; + if (query.roles) + roles_to_grant_or_revoke = RolesOrUsersSet{*query.roles, access_control}; + + if (query.is_revoke) + { + /// REVOKE + roles_to_revoke = std::move(roles_to_grant_or_revoke); + } + else if (query.replace_granted_roles) + { + /// GRANT WITH REPLACE OPTION + roles_to_grant = roles_to_grant_or_revoke.getMatchingIDs(access_control); + roles_to_revoke = RolesOrUsersSet::AllTag{}; + } + else + { + /// GRANT + roles_to_grant = roles_to_grant_or_revoke.getMatchingIDs(access_control); + } } - void checkGranteeIsAllowed(const ContextAccess & access, const UUID & grantee_id, const IAccessEntity & grantee) + /// Extracts roles which are going to be granted or revoked from a query. + void collectRolesToGrantOrRevoke( + const ASTGrantQuery & query, + std::vector & roles_to_grant, + RolesOrUsersSet & roles_to_revoke) { - auto current_user = access.getUser(); + roles_to_grant.clear(); + roles_to_revoke.clear(); + + RolesOrUsersSet roles_to_grant_or_revoke; + if (query.roles) + roles_to_grant_or_revoke = RolesOrUsersSet{*query.roles}; + + if (query.is_revoke) + { + /// REVOKE + roles_to_revoke = std::move(roles_to_grant_or_revoke); + } + else if (query.replace_granted_roles) + { + /// GRANT WITH REPLACE OPTION + roles_to_grant = roles_to_grant_or_revoke.getMatchingIDs(); + roles_to_revoke = RolesOrUsersSet::AllTag{}; + } + else + { + /// GRANT + roles_to_grant = roles_to_grant_or_revoke.getMatchingIDs(); + } + } + + /// Checks if a grantee is allowed for the current user, throws an exception if not. + void checkGranteeIsAllowed(const ContextAccess & current_user_access, const UUID & grantee_id, const IAccessEntity & grantee) + { + auto current_user = current_user_access.getUser(); if (current_user && !current_user->grantees.match(grantee_id)) throw Exception(grantee.outputTypeAndName() + " is not allowed as grantee", ErrorCodes::ACCESS_DENIED); } - void checkGranteesAreAllowed(const AccessControlManager & access_control, const ContextAccess & access, const std::vector & grantee_ids) + /// Checks if grantees are allowed for the current user, throws an exception if not. + void checkGranteesAreAllowed(const AccessControlManager & access_control, const ContextAccess & current_user_access, const std::vector & grantee_ids) { - auto current_user = access.getUser(); + auto current_user = current_user_access.getUser(); if (!current_user || (current_user->grantees == RolesOrUsersSet::AllTag{})) return; @@ -92,36 +131,26 @@ namespace { auto entity = access_control.tryRead(id); if (auto role = typeid_cast(entity)) - checkGranteeIsAllowed(access, id, *role); + checkGranteeIsAllowed(current_user_access, id, *role); else if (auto user = typeid_cast(entity)) - checkGranteeIsAllowed(access, id, *user); + checkGranteeIsAllowed(current_user_access, id, *user); } } + /// Checks if the current user has enough access rights granted with grant option to grant or revoke specified access rights. void checkGrantOption( const AccessControlManager & access_control, - const ContextAccess & access, - const ASTGrantQuery & query, + const ContextAccess & current_user_access, const std::vector & grantees_from_query, - bool & need_check_grantees_are_allowed) + bool & need_check_grantees_are_allowed, + const AccessRightsElements & elements_to_grant, + AccessRightsElements & elements_to_revoke) { - const auto & elements = query.access_rights_elements; - need_check_grantees_are_allowed = true; - if (elements.empty()) - { - /// No access rights to grant or revoke. - need_check_grantees_are_allowed = false; - return; - } + /// Check access rights which are going to be granted. + /// To execute the command GRANT the current user needs to have the access granted with GRANT OPTION. + current_user_access.checkGrantOption(elements_to_grant); - if (!query.is_revoke) - { - /// To execute the command GRANT the current user needs to have the access granted with GRANT OPTION. - access.checkGrantOption(elements); - return; - } - - if (access.hasGrantOption(elements)) + if (current_user_access.hasGrantOption(elements_to_revoke)) { /// Simple case: the current user has the grant option for all the access rights specified for REVOKE. return; @@ -141,69 +170,81 @@ namespace auto entity = access_control.tryRead(id); if (auto role = typeid_cast(entity)) { - checkGranteeIsAllowed(access, id, *role); + checkGranteeIsAllowed(current_user_access, id, *role); all_granted_access.makeUnion(role->access); } else if (auto user = typeid_cast(entity)) { - checkGranteeIsAllowed(access, id, *user); + checkGranteeIsAllowed(current_user_access, id, *user); all_granted_access.makeUnion(user->access); } } need_check_grantees_are_allowed = false; /// already checked - AccessRights required_access; - if (elements[0].is_partial_revoke) - { - AccessRightsElements non_revoke_elements = elements; - std::for_each(non_revoke_elements.begin(), non_revoke_elements.end(), [&](AccessRightsElement & element) { element.is_partial_revoke = false; }); - required_access.grant(non_revoke_elements); - } - else - { - required_access.grant(elements); - } - required_access.makeIntersection(all_granted_access); + if (!elements_to_revoke.empty() && elements_to_revoke[0].is_partial_revoke) + std::for_each(elements_to_revoke.begin(), elements_to_revoke.end(), [&](AccessRightsElement & element) { element.is_partial_revoke = false; }); + AccessRights access_to_revoke; + access_to_revoke.grant(elements_to_revoke); + access_to_revoke.makeIntersection(all_granted_access); - for (auto & required_access_element : required_access.getElements()) + /// Build more accurate list of elements to revoke, now we use an intesection of the initial list of elements to revoke + /// and all the granted access rights to these grantees. + bool grant_option = !elements_to_revoke.empty() && elements_to_revoke[0].grant_option; + elements_to_revoke.clear(); + for (auto & element_to_revoke : access_to_revoke.getElements()) { - if (!required_access_element.is_partial_revoke && (required_access_element.grant_option || !elements[0].grant_option)) - access.checkGrantOption(required_access_element); + if (!element_to_revoke.is_partial_revoke && (element_to_revoke.grant_option || !grant_option)) + elements_to_revoke.emplace_back(std::move(element_to_revoke)); } + + current_user_access.checkGrantOption(elements_to_revoke); } - std::vector getRoleIDsAndCheckAdminOption( + /// Checks if the current user has enough access rights granted with grant option to grant or revoke specified access rights. + /// Also checks if grantees are allowed for the current user. + void checkGrantOptionAndGrantees( const AccessControlManager & access_control, - const ContextAccess & access, - const ASTGrantQuery & query, - const RolesOrUsersSet & roles_from_query, + const ContextAccess & current_user_access, const std::vector & grantees_from_query, - bool & need_check_grantees_are_allowed) + const AccessRightsElements & elements_to_grant, + AccessRightsElements & elements_to_revoke) { - need_check_grantees_are_allowed = true; - if (roles_from_query.empty()) - { - /// No roles to grant or revoke. - need_check_grantees_are_allowed = false; - return {}; - } + bool need_check_grantees_are_allowed = true; + checkGrantOption( + access_control, + current_user_access, + grantees_from_query, + need_check_grantees_are_allowed, + elements_to_grant, + elements_to_revoke); - std::vector matching_ids; - if (!query.is_revoke) - { - /// To execute the command GRANT the current user needs to have the roles granted with ADMIN OPTION. - matching_ids = roles_from_query.getMatchingIDs(access_control); - access.checkAdminOption(matching_ids); - return matching_ids; - } + if (need_check_grantees_are_allowed) + checkGranteesAreAllowed(access_control, current_user_access, grantees_from_query); + } - if (!roles_from_query.all) + /// Checks if the current user has enough roles granted with admin option to grant or revoke specified roles. + void checkAdminOption( + const AccessControlManager & access_control, + const ContextAccess & current_user_access, + const std::vector & grantees_from_query, + bool & need_check_grantees_are_allowed, + const std::vector & roles_to_grant, + RolesOrUsersSet & roles_to_revoke, + bool admin_option) + { + /// Check roles which are going to be granted. + /// To execute the command GRANT the current user needs to have the roles granted with ADMIN OPTION. + current_user_access.checkAdminOption(roles_to_grant); + + /// Check roles which are going to be revoked. + std::vector roles_to_revoke_ids; + if (!roles_to_revoke.all) { - matching_ids = roles_from_query.getMatchingIDs(); - if (access.hasAdminOption(matching_ids)) + roles_to_revoke_ids = roles_to_revoke.getMatchingIDs(); + if (current_user_access.hasAdminOption(roles_to_revoke_ids)) { /// Simple case: the current user has the admin option for all the roles specified for REVOKE. - return matching_ids; + return; } } @@ -221,51 +262,109 @@ namespace auto entity = access_control.tryRead(id); if (auto role = typeid_cast(entity)) { - checkGranteeIsAllowed(access, id, *role); + checkGranteeIsAllowed(current_user_access, id, *role); all_granted_roles.makeUnion(role->granted_roles); } else if (auto user = typeid_cast(entity)) { - checkGranteeIsAllowed(access, id, *user); + checkGranteeIsAllowed(current_user_access, id, *user); all_granted_roles.makeUnion(user->granted_roles); } } + const auto & all_granted_roles_set = admin_option ? all_granted_roles.getGrantedWithAdminOption() : all_granted_roles.getGranted(); need_check_grantees_are_allowed = false; /// already checked - const auto & all_granted_roles_set = query.admin_option ? all_granted_roles.getGrantedWithAdminOption() : all_granted_roles.getGranted(); - if (roles_from_query.all) - boost::range::set_difference(all_granted_roles_set, roles_from_query.except_ids, std::back_inserter(matching_ids)); + if (roles_to_revoke.all) + boost::range::set_difference(all_granted_roles_set, roles_to_revoke.except_ids, std::back_inserter(roles_to_revoke_ids)); else - boost::range::remove_erase_if(matching_ids, [&](const UUID & id) { return !all_granted_roles_set.count(id); }); - access.checkAdminOption(matching_ids); - return matching_ids; + boost::range::remove_erase_if(roles_to_revoke_ids, [&](const UUID & id) { return !all_granted_roles_set.count(id); }); + + roles_to_revoke = roles_to_revoke_ids; + current_user_access.checkAdminOption(roles_to_revoke_ids); } - void checkGrantOptionAndGrantees( + /// Checks if the current user has enough roles granted with admin option to grant or revoke specified roles. + /// Also checks if grantees are allowed for the current user. + void checkAdminOptionAndGrantees( const AccessControlManager & access_control, - const ContextAccess & access, - const ASTGrantQuery & query, - const std::vector & grantees_from_query) + const ContextAccess & current_user_access, + const std::vector & grantees_from_query, + const std::vector & roles_to_grant, + RolesOrUsersSet & roles_to_revoke, + bool admin_option) { bool need_check_grantees_are_allowed = true; - checkGrantOption(access_control, access, query, grantees_from_query, need_check_grantees_are_allowed); + checkAdminOption( + access_control, + current_user_access, + grantees_from_query, + need_check_grantees_are_allowed, + roles_to_grant, + roles_to_revoke, + admin_option); + if (need_check_grantees_are_allowed) - checkGranteesAreAllowed(access_control, access, grantees_from_query); + checkGranteesAreAllowed(access_control, current_user_access, grantees_from_query); } - std::vector getRoleIDsAndCheckAdminOptionAndGrantees( - const AccessControlManager & access_control, - const ContextAccess & access, - const ASTGrantQuery & query, - const RolesOrUsersSet & roles_from_query, - const std::vector & grantees_from_query) + template + void updateGrantedAccessRightsAndRolesTemplate( + T & grantee, + const AccessRightsElements & elements_to_grant, + const AccessRightsElements & elements_to_revoke, + const std::vector & roles_to_grant, + const RolesOrUsersSet & roles_to_revoke, + bool admin_option) { - bool need_check_grantees_are_allowed = true; - auto role_ids = getRoleIDsAndCheckAdminOption( - access_control, access, query, roles_from_query, grantees_from_query, need_check_grantees_are_allowed); - if (need_check_grantees_are_allowed) - checkGranteesAreAllowed(access_control, access, grantees_from_query); - return role_ids; + if (!elements_to_revoke.empty()) + grantee.access.revoke(elements_to_revoke); + + if (!elements_to_grant.empty()) + grantee.access.grant(elements_to_grant); + + if (!roles_to_revoke.empty()) + { + if (admin_option) + grantee.granted_roles.revokeAdminOption(grantee.granted_roles.findGrantedWithAdminOption(roles_to_revoke)); + else + grantee.granted_roles.revoke(grantee.granted_roles.findGranted(roles_to_revoke)); + } + + if (!roles_to_grant.empty()) + { + if (admin_option) + grantee.granted_roles.grantWithAdminOption(roles_to_grant); + else + grantee.granted_roles.grant(roles_to_grant); + } + } + + /// Updates grants of a specified user or role. + void updateGrantedAccessRightsAndRoles( + IAccessEntity & grantee, + const AccessRightsElements & elements_to_grant, + const AccessRightsElements & elements_to_revoke, + const std::vector & roles_to_grant, + const RolesOrUsersSet & roles_to_revoke, + bool admin_option) + { + if (auto * user = typeid_cast(&grantee)) + updateGrantedAccessRightsAndRolesTemplate(*user, elements_to_grant, elements_to_revoke, roles_to_grant, roles_to_revoke, admin_option); + else if (auto * role = typeid_cast(&grantee)) + updateGrantedAccessRightsAndRolesTemplate(*role, elements_to_grant, elements_to_revoke, roles_to_grant, roles_to_revoke, admin_option); + } + + /// Updates grants of a specified user or role. + void updateFromQuery(IAccessEntity & grantee, const ASTGrantQuery & query) + { + AccessRightsElements elements_to_grant, elements_to_revoke; + collectAccessRightsElementsToGrantOrRevoke(query, elements_to_grant, elements_to_revoke); + + std::vector roles_to_grant; + RolesOrUsersSet roles_to_revoke; + collectRolesToGrantOrRevoke(query, roles_to_grant, roles_to_revoke); + + updateGrantedAccessRightsAndRoles(grantee, elements_to_grant, elements_to_revoke, roles_to_grant, roles_to_revoke, query.admin_option); } } @@ -283,16 +382,13 @@ BlockIO InterpreterGrantQuery::execute() throw Exception("A partial revoke should be revoked, not granted", ErrorCodes::LOGICAL_ERROR); auto & access_control = getContext()->getAccessControlManager(); - std::optional roles_set; - if (query.roles) - roles_set = RolesOrUsersSet{*query.roles, access_control}; - std::vector grantees = RolesOrUsersSet{*query.grantees, access_control, getContext()->getUserID()}.getMatchingIDs(access_control); /// Check if the current user has corresponding roles granted with admin option. - std::vector roles; - if (roles_set) - roles = getRoleIDsAndCheckAdminOptionAndGrantees(access_control, *getContext()->getAccess(), query, *roles_set, grantees); + std::vector roles_to_grant; + RolesOrUsersSet roles_to_revoke; + collectRolesToGrantOrRevoke(access_control, query, roles_to_grant, roles_to_revoke); + checkAdminOptionAndGrantees(access_control, *getContext()->getAccess(), grantees, roles_to_grant, roles_to_revoke, query.admin_option); if (!query.cluster.empty()) { @@ -306,14 +402,15 @@ BlockIO InterpreterGrantQuery::execute() query.replaceEmptyDatabase(getContext()->getCurrentDatabase()); /// Check if the current user has corresponding access rights with grant option. - if (!query.access_rights_elements.empty()) - checkGrantOptionAndGrantees(access_control, *getContext()->getAccess(), query, grantees); + AccessRightsElements elements_to_grant, elements_to_revoke; + collectAccessRightsElementsToGrantOrRevoke(query, elements_to_grant, elements_to_revoke); + checkGrantOptionAndGrantees(access_control, *getContext()->getAccess(), grantees, elements_to_grant, elements_to_revoke); /// Update roles and users listed in `grantees`. auto update_func = [&](const AccessEntityPtr & entity) -> AccessEntityPtr { auto clone = entity->clone(); - updateFromQueryImpl(*clone, query, roles); + updateGrantedAccessRightsAndRoles(*clone, elements_to_grant, elements_to_revoke, roles_to_grant, roles_to_revoke, query.admin_option); return clone; }; @@ -325,21 +422,15 @@ BlockIO InterpreterGrantQuery::execute() void InterpreterGrantQuery::updateUserFromQuery(User & user, const ASTGrantQuery & query) { - std::vector roles_to_grant_or_revoke; - if (query.roles) - roles_to_grant_or_revoke = RolesOrUsersSet{*query.roles}.getMatchingIDs(); - updateFromQueryImpl(user, query, roles_to_grant_or_revoke); + updateFromQuery(user, query); } - void InterpreterGrantQuery::updateRoleFromQuery(Role & role, const ASTGrantQuery & query) { - std::vector roles_to_grant_or_revoke; - if (query.roles) - roles_to_grant_or_revoke = RolesOrUsersSet{*query.roles}.getMatchingIDs(); - updateFromQueryImpl(role, query, roles_to_grant_or_revoke); + updateFromQuery(role, query); } + void InterpreterGrantQuery::extendQueryLogElemImpl(QueryLogElement & elem, const ASTPtr & /*ast*/, ContextPtr) const { auto & query = query_ptr->as(); diff --git a/tests/integration/test_grant_and_revoke/test.py b/tests/integration/test_grant_and_revoke/test.py index a63d6f136af..79fe4bf9f41 100644 --- a/tests/integration/test_grant_and_revoke/test.py +++ b/tests/integration/test_grant_and_revoke/test.py @@ -282,3 +282,36 @@ def test_current_database(): instance.query("CREATE TABLE default.table(x UInt32, y UInt32) ENGINE = MergeTree ORDER BY tuple()") assert "Not enough privileges" in instance.query_and_get_error("SELECT * FROM table", user='A') + + +def test_grant_with_replace_option(): + instance.query("CREATE USER A") + instance.query('GRANT SELECT ON test.table TO A') + assert instance.query("SHOW GRANTS FOR A") == TSV(["GRANT SELECT ON test.table TO A"]) + + instance.query('GRANT INSERT ON test.table TO A WITH REPLACE OPTION') + assert instance.query("SHOW GRANTS FOR A") == TSV(["GRANT INSERT ON test.table TO A"]) + + instance.query('GRANT NONE ON *.* TO A WITH REPLACE OPTION') + assert instance.query("SHOW GRANTS FOR A") == TSV([]) + + instance.query('CREATE USER B') + instance.query('GRANT SELECT ON test.table TO B') + assert instance.query("SHOW GRANTS FOR A") == TSV([]) + assert instance.query("SHOW GRANTS FOR B") == TSV(["GRANT SELECT ON test.table TO B"]) + + expected_error = "it's necessary to have grant INSERT ON test.table WITH GRANT OPTION" + assert expected_error in instance.query_and_get_error("GRANT INSERT ON test.table TO B WITH REPLACE OPTION", user='A') + assert instance.query("SHOW GRANTS FOR A") == TSV([]) + assert instance.query("SHOW GRANTS FOR B") == TSV(["GRANT SELECT ON test.table TO B"]) + + instance.query("GRANT INSERT ON test.table TO A WITH GRANT OPTION") + expected_error = "it's necessary to have grant SELECT ON test.table WITH GRANT OPTION" + assert expected_error in instance.query_and_get_error("GRANT INSERT ON test.table TO B WITH REPLACE OPTION", user='A') + assert instance.query("SHOW GRANTS FOR A") == TSV(["GRANT INSERT ON test.table TO A WITH GRANT OPTION"]) + assert instance.query("SHOW GRANTS FOR B") == TSV(["GRANT SELECT ON test.table TO B"]) + + instance.query("GRANT SELECT ON test.table TO A WITH GRANT OPTION") + instance.query("GRANT INSERT ON test.table TO B WITH REPLACE OPTION", user='A') + assert instance.query("SHOW GRANTS FOR A") == TSV(["GRANT SELECT, INSERT ON test.table TO A WITH GRANT OPTION"]) + assert instance.query("SHOW GRANTS FOR B") == TSV(["GRANT INSERT ON test.table TO B"]) From 6c4c3df96e41425185beb0c471a8dde0ce6f25a7 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Sun, 15 Aug 2021 16:22:15 +0300 Subject: [PATCH 589/599] Auto version update to [21.9.1.7770] [54454] --- cmake/autogenerated_versions.txt | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/cmake/autogenerated_versions.txt b/cmake/autogenerated_versions.txt index 18072566d04..2435335f669 100644 --- a/cmake/autogenerated_versions.txt +++ b/cmake/autogenerated_versions.txt @@ -6,7 +6,7 @@ SET(VERSION_REVISION 54454) SET(VERSION_MAJOR 21) SET(VERSION_MINOR 9) SET(VERSION_PATCH 1) -SET(VERSION_GITHASH f48c5af90c2ad51955d1ee3b6b05d006b03e4238) -SET(VERSION_DESCRIBE v21.9.1.1-prestable) -SET(VERSION_STRING 21.9.1.1) +SET(VERSION_GITHASH f063e44131a048ba2d9af8075f03700fd5ec3e69) +SET(VERSION_DESCRIBE v21.9.1.7770-prestable) +SET(VERSION_STRING 21.9.1.7770) # end of autochange From fd500d2176a4623145fc53c3e85c3e989da7f2ce Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Sun, 15 Aug 2021 17:05:28 +0300 Subject: [PATCH 590/599] Backquote default database in CREATE USER --- src/Parsers/ASTDatabaseOrNone.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Parsers/ASTDatabaseOrNone.cpp b/src/Parsers/ASTDatabaseOrNone.cpp index f93322ef00c..0a0c800ff36 100644 --- a/src/Parsers/ASTDatabaseOrNone.cpp +++ b/src/Parsers/ASTDatabaseOrNone.cpp @@ -1,4 +1,5 @@ #include +#include #include namespace DB @@ -10,7 +11,7 @@ void ASTDatabaseOrNone::formatImpl(const FormatSettings & settings, FormatState settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << "NONE" << (settings.hilite ? IAST::hilite_none : ""); return; } - settings.ostr << database_name; + settings.ostr << backQuoteIfNeed(database_name); } } From d39128df9fc148c38f02cea926ede9ad03458528 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 8 Aug 2021 20:08:36 +0300 Subject: [PATCH 591/599] Add module part for client/TestHint --- programs/client/CMakeLists.txt | 1 + programs/client/TestHint.cpp | 80 ++++++++++++++++++++++++++++++++++ programs/client/TestHint.h | 75 ++----------------------------- 3 files changed, 84 insertions(+), 72 deletions(-) create mode 100644 programs/client/TestHint.cpp diff --git a/programs/client/CMakeLists.txt b/programs/client/CMakeLists.txt index 084e1b45911..1de5ea88aee 100644 --- a/programs/client/CMakeLists.txt +++ b/programs/client/CMakeLists.txt @@ -3,6 +3,7 @@ set (CLICKHOUSE_CLIENT_SOURCES ConnectionParameters.cpp QueryFuzzer.cpp Suggest.cpp + TestHint.cpp ) set (CLICKHOUSE_CLIENT_LINK diff --git a/programs/client/TestHint.cpp b/programs/client/TestHint.cpp new file mode 100644 index 00000000000..03e6ce56232 --- /dev/null +++ b/programs/client/TestHint.cpp @@ -0,0 +1,80 @@ +#include "TestHint.h" + +#include +#include +#include +#include + +namespace DB +{ + +TestHint::TestHint(bool enabled_, const String & query_) + : query(query_) +{ + if (!enabled_) + return; + + // Don't parse error hints in leading comments, because it feels weird. + // Leading 'echo' hint is OK. + bool is_leading_hint = true; + + Lexer lexer(query.data(), query.data() + query.size()); + + for (Token token = lexer.nextToken(); !token.isEnd(); token = lexer.nextToken()) + { + if (token.type != TokenType::Comment + && token.type != TokenType::Whitespace) + { + is_leading_hint = false; + } + else if (token.type == TokenType::Comment) + { + String comment(token.begin, token.begin + token.size()); + + if (!comment.empty()) + { + size_t pos_start = comment.find('{', 0); + if (pos_start != String::npos) + { + size_t pos_end = comment.find('}', pos_start); + if (pos_end != String::npos) + { + String hint(comment.begin() + pos_start + 1, comment.begin() + pos_end); + parse(hint, is_leading_hint); + } + } + } + } + } +} + +void TestHint::parse(const String & hint, bool is_leading_hint) +{ + std::stringstream ss; // STYLE_CHECK_ALLOW_STD_STRING_STREAM + ss << hint; + String item; + + while (!ss.eof()) + { + ss >> item; + if (ss.eof()) + break; + + if (!is_leading_hint) + { + if (item == "serverError") + ss >> server_error; + else if (item == "clientError") + ss >> client_error; + } + + if (item == "echo") + echo.emplace(true); + if (item == "echoOn") + echo.emplace(true); + if (item == "echoOff") + echo.emplace(false); + } +} + +} diff --git a/programs/client/TestHint.h b/programs/client/TestHint.h index 100d47f4dd2..fcf50002c16 100644 --- a/programs/client/TestHint.h +++ b/programs/client/TestHint.h @@ -1,11 +1,7 @@ #pragma once -#include -#include -#include +#include #include -#include -#include namespace DB @@ -43,45 +39,7 @@ namespace DB class TestHint { public: - TestHint(bool enabled_, const String & query_) : - query(query_) - { - if (!enabled_) - return; - - // Don't parse error hints in leading comments, because it feels weird. - // Leading 'echo' hint is OK. - bool is_leading_hint = true; - - Lexer lexer(query.data(), query.data() + query.size()); - - for (Token token = lexer.nextToken(); !token.isEnd(); token = lexer.nextToken()) - { - if (token.type != TokenType::Comment - && token.type != TokenType::Whitespace) - { - is_leading_hint = false; - } - else if (token.type == TokenType::Comment) - { - String comment(token.begin, token.begin + token.size()); - - if (!comment.empty()) - { - size_t pos_start = comment.find('{', 0); - if (pos_start != String::npos) - { - size_t pos_end = comment.find('}', pos_start); - if (pos_end != String::npos) - { - String hint(comment.begin() + pos_start + 1, comment.begin() + pos_end); - parse(hint, is_leading_hint); - } - } - } - } - } - } + TestHint(bool enabled_, const String & query_); int serverError() const { return server_error; } int clientError() const { return client_error; } @@ -93,34 +51,7 @@ private: int client_error = 0; std::optional echo; - void parse(const String & hint, bool is_leading_hint) - { - std::stringstream ss; // STYLE_CHECK_ALLOW_STD_STRING_STREAM - ss << hint; - String item; - - while (!ss.eof()) - { - ss >> item; - if (ss.eof()) - break; - - if (!is_leading_hint) - { - if (item == "serverError") - ss >> server_error; - else if (item == "clientError") - ss >> client_error; - } - - if (item == "echo") - echo.emplace(true); - if (item == "echoOn") - echo.emplace(true); - if (item == "echoOff") - echo.emplace(false); - } - } + void parse(const String & hint, bool is_leading_hint); bool allErrorsExpected(int actual_server_error, int actual_client_error) const { From ad4833b4ab116261cfaadd4a6ffef4dca8ff232c Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 8 Aug 2021 20:08:36 +0300 Subject: [PATCH 592/599] Fix endless loop for parsing invalid test hints (lack of ss.fail() check) --- programs/client/TestHint.cpp | 31 +++++++++++++++++++++++++++++-- 1 file changed, 29 insertions(+), 2 deletions(-) diff --git a/programs/client/TestHint.cpp b/programs/client/TestHint.cpp index 03e6ce56232..6149135b597 100644 --- a/programs/client/TestHint.cpp +++ b/programs/client/TestHint.cpp @@ -8,6 +8,33 @@ namespace DB { +namespace ErrorCodes +{ + extern const int CANNOT_PARSE_TEXT; +} + +} + +namespace +{ + +int parseErrorCode(std::stringstream & ss) // STYLE_CHECK_ALLOW_STD_STRING_STREAM +{ + using namespace DB; + + int code; + ss >> code; + if (ss.fail()) + throw Exception(ErrorCodes::CANNOT_PARSE_TEXT, + "Expected integer value for test hint, got: '{}'", ss.str()); + return code; +} + +} + +namespace DB +{ + TestHint::TestHint(bool enabled_, const String & query_) : query(query_) { @@ -63,9 +90,9 @@ void TestHint::parse(const String & hint, bool is_leading_hint) if (!is_leading_hint) { if (item == "serverError") - ss >> server_error; + server_error = parseErrorCode(ss); else if (item == "clientError") - ss >> client_error; + client_error = parseErrorCode(ss); } if (item == "echo") From 098a0b2012cc9467d68185b152b5769223606e84 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 8 Aug 2021 20:08:36 +0300 Subject: [PATCH 593/599] Add ErrorCodes::getErrorCodeByName() --- src/Common/ErrorCodes.cpp | 17 +++++++++++++++++ src/Common/ErrorCodes.h | 6 ++++++ 2 files changed, 23 insertions(+) diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index c15605660ef..fd14bddb64c 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -1,4 +1,5 @@ #include +#include #include /** Previously, these constants were located in one enum. @@ -564,6 +565,7 @@ M(594, BZIP2_STREAM_DECODER_FAILED) \ M(595, BZIP2_STREAM_ENCODER_FAILED) \ M(596, INTERSECT_OR_EXCEPT_RESULT_STRUCTURES_MISMATCH) \ + M(597, NO_SUCH_ERROR_CODE) \ \ M(998, POSTGRESQL_CONNECTION_FAILURE) \ M(999, KEEPER_EXCEPTION) \ @@ -602,6 +604,21 @@ namespace ErrorCodes return error_codes_names.names[error_code]; } + ErrorCode getErrorCodeByName(std::string_view error_name) + { + for (size_t i = 0, end = ErrorCodes::end(); i < end; ++i) + { + std::string_view name = ErrorCodes::getName(i); + + if (name.empty()) + continue; + + if (name == error_name) + return i; + } + throw Exception(NO_SUCH_ERROR_CODE, "No error code with name: '{}'", error_name); + } + ErrorCode end() { return END + 1; } void increment(ErrorCode error_code, bool remote, const std::string & message, const FramePointers & trace) diff --git a/src/Common/ErrorCodes.h b/src/Common/ErrorCodes.h index cefd77df868..be36aa6fd50 100644 --- a/src/Common/ErrorCodes.h +++ b/src/Common/ErrorCodes.h @@ -25,6 +25,12 @@ namespace ErrorCodes /// Get name of error_code by identifier. /// Returns statically allocated string. std::string_view getName(ErrorCode error_code); + /// Get error code value by name. + /// + /// It has O(N) complexity, but this is not major, since it is used only + /// for test hints, and it does not worth to keep another structure for + /// this. + ErrorCode getErrorCodeByName(std::string_view error_name); struct Error { From c5d9676779319b13a55b2843d29015c56fd1abff Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 8 Aug 2021 20:08:36 +0300 Subject: [PATCH 594/599] Accept error code by error name in client test hints --- programs/client/TestHint.cpp | 13 +++++++++++-- programs/client/TestHint.h | 4 ++++ .../02006_client_test_hint_error_name.reference | 0 .../02006_client_test_hint_error_name.sql | 1 + ...06_client_test_hint_no_such_error_name.reference | 1 + .../02006_client_test_hint_no_such_error_name.sh | 7 +++++++ 6 files changed, 24 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/02006_client_test_hint_error_name.reference create mode 100644 tests/queries/0_stateless/02006_client_test_hint_error_name.sql create mode 100644 tests/queries/0_stateless/02006_client_test_hint_no_such_error_name.reference create mode 100755 tests/queries/0_stateless/02006_client_test_hint_no_such_error_name.sh diff --git a/programs/client/TestHint.cpp b/programs/client/TestHint.cpp index 6149135b597..7eb6575792a 100644 --- a/programs/client/TestHint.cpp +++ b/programs/client/TestHint.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include namespace DB @@ -23,10 +24,18 @@ int parseErrorCode(std::stringstream & ss) // STYLE_CHECK_ALLOW_STD_STRING_STREA using namespace DB; int code; + String code_name; + ss >> code; if (ss.fail()) - throw Exception(ErrorCodes::CANNOT_PARSE_TEXT, - "Expected integer value for test hint, got: '{}'", ss.str()); + { + ss.clear(); + ss >> code_name; + if (ss.fail()) + throw Exception(ErrorCodes::CANNOT_PARSE_TEXT, + "Cannot parse test hint '{}'", ss.str()); + return ErrorCodes::getErrorCodeByName(code_name); + } return code; } diff --git a/programs/client/TestHint.h b/programs/client/TestHint.h index fcf50002c16..377637d0db8 100644 --- a/programs/client/TestHint.h +++ b/programs/client/TestHint.h @@ -15,6 +15,10 @@ namespace DB /// /// - "-- { clientError 20 }" -- in case of you are expecting client error. /// +/// - "-- { serverError FUNCTION_THROW_IF_VALUE_IS_NON_ZERO }" -- by error name. +/// +/// - "-- { clientError FUNCTION_THROW_IF_VALUE_IS_NON_ZERO }" -- by error name. +/// /// Remember that the client parse the query first (not the server), so for /// example if you are expecting syntax error, then you should use /// clientError not serverError. diff --git a/tests/queries/0_stateless/02006_client_test_hint_error_name.reference b/tests/queries/0_stateless/02006_client_test_hint_error_name.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02006_client_test_hint_error_name.sql b/tests/queries/0_stateless/02006_client_test_hint_error_name.sql new file mode 100644 index 00000000000..268406a8054 --- /dev/null +++ b/tests/queries/0_stateless/02006_client_test_hint_error_name.sql @@ -0,0 +1 @@ +select throwIf(1); -- { serverError FUNCTION_THROW_IF_VALUE_IS_NON_ZERO } diff --git a/tests/queries/0_stateless/02006_client_test_hint_no_such_error_name.reference b/tests/queries/0_stateless/02006_client_test_hint_no_such_error_name.reference new file mode 100644 index 00000000000..40bea919c93 --- /dev/null +++ b/tests/queries/0_stateless/02006_client_test_hint_no_such_error_name.reference @@ -0,0 +1 @@ +No error code with name: 'FOOBAR'. (NO_SUCH_ERROR_CODE) diff --git a/tests/queries/0_stateless/02006_client_test_hint_no_such_error_name.sh b/tests/queries/0_stateless/02006_client_test_hint_no_such_error_name.sh new file mode 100755 index 00000000000..15ede23a510 --- /dev/null +++ b/tests/queries/0_stateless/02006_client_test_hint_no_such_error_name.sh @@ -0,0 +1,7 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +$CLICKHOUSE_CLIENT --testmode -n -q 'select 1 -- { clientError FOOBAR }' |& grep -o 'No error code with name:.*' From 86e14f192c7addcc694d945712b787c3e7645c5b Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 8 Aug 2021 20:08:36 +0300 Subject: [PATCH 595/599] Convert std::stringstream to ReadBufferFromString for TestHint parsing --- programs/client/TestHint.cpp | 53 ++++++++++++++---------------------- 1 file changed, 21 insertions(+), 32 deletions(-) diff --git a/programs/client/TestHint.cpp b/programs/client/TestHint.cpp index 7eb6575792a..2f3be2a5350 100644 --- a/programs/client/TestHint.cpp +++ b/programs/client/TestHint.cpp @@ -1,42 +1,30 @@ #include "TestHint.h" -#include -#include #include #include +#include +#include #include -namespace DB -{ - -namespace ErrorCodes -{ - extern const int CANNOT_PARSE_TEXT; -} - -} - namespace { -int parseErrorCode(std::stringstream & ss) // STYLE_CHECK_ALLOW_STD_STRING_STREAM +/// Parse error as number or as a string (name of the error code const) +int parseErrorCode(DB::ReadBufferFromString & in) { - using namespace DB; - - int code; + int code = -1; String code_name; - ss >> code; - if (ss.fail()) + auto * pos = in.position(); + tryReadText(code, in); + if (pos != in.position()) { - ss.clear(); - ss >> code_name; - if (ss.fail()) - throw Exception(ErrorCodes::CANNOT_PARSE_TEXT, - "Cannot parse test hint '{}'", ss.str()); - return ErrorCodes::getErrorCodeByName(code_name); + return code; } - return code; + + /// Try parse as string + readStringUntilWhitespace(code_name, in); + return DB::ErrorCodes::getErrorCodeByName(code_name); } } @@ -86,22 +74,23 @@ TestHint::TestHint(bool enabled_, const String & query_) void TestHint::parse(const String & hint, bool is_leading_hint) { - std::stringstream ss; // STYLE_CHECK_ALLOW_STD_STRING_STREAM - ss << hint; + ReadBufferFromString in(hint); String item; - while (!ss.eof()) + while (!in.eof()) { - ss >> item; - if (ss.eof()) + readStringUntilWhitespace(item, in); + if (in.eof()) break; + skipWhitespaceIfAny(in); + if (!is_leading_hint) { if (item == "serverError") - server_error = parseErrorCode(ss); + server_error = parseErrorCode(in); else if (item == "clientError") - client_error = parseErrorCode(ss); + client_error = parseErrorCode(in); } if (item == "echo") From 5c56d3a7344615921b952f388caa779b0f245e22 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 16 Aug 2021 03:01:30 +0300 Subject: [PATCH 596/599] Testflows are broken --- tests/testflows/regression.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/testflows/regression.py b/tests/testflows/regression.py index ba2ea3b111c..bae1c5db90a 100755 --- a/tests/testflows/regression.py +++ b/tests/testflows/regression.py @@ -22,7 +22,7 @@ def regression(self, local, clickhouse_binary_path, stress=None): # run_scenario(pool, tasks, Feature(test=load("ldap.regression", "regression")), args) # run_scenario(pool, tasks, Feature(test=load("rbac.regression", "regression")), args) Feature(test=load("aes_encryption.regression", "regression"), parallel=True, executor=pool)(**args) - Feature(test=load("map_type.regression", "regression"), parallel=True, executor=pool)(**args) + # Feature(test=load("map_type.regression", "regression"), parallel=True, executor=pool)(**args) Feature(test=load("window_functions.regression", "regression"), parallel=True, executor=pool)(**args) Feature(test=load("datetime64_extended_range.regression", "regression"), parallel=True, executor=pool)(**args) Feature(test=load("kerberos.regression", "regression"), parallel=True, executor=pool)(**args) From 9a4a8d39439314d563c67e334333bf9a16fecd08 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 16 Aug 2021 03:16:45 +0300 Subject: [PATCH 597/599] Fix build with clang-13 --- base/glibc-compatibility/CMakeLists.txt | 12 +++--------- cmake/add_warning.cmake | 19 +++++++++++++++++++ 2 files changed, 22 insertions(+), 9 deletions(-) diff --git a/base/glibc-compatibility/CMakeLists.txt b/base/glibc-compatibility/CMakeLists.txt index 8cba91de33f..4fc2a002cd8 100644 --- a/base/glibc-compatibility/CMakeLists.txt +++ b/base/glibc-compatibility/CMakeLists.txt @@ -9,10 +9,6 @@ if (GLIBC_COMPATIBILITY) check_include_file("sys/random.h" HAVE_SYS_RANDOM_H) - if(COMPILER_CLANG) - set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -Wno-builtin-requires-header") - endif() - add_headers_and_sources(glibc_compatibility .) add_headers_and_sources(glibc_compatibility musl) if (ARCH_AARCH64) @@ -35,11 +31,9 @@ if (GLIBC_COMPATIBILITY) add_library(glibc-compatibility STATIC ${glibc_compatibility_sources}) - if (COMPILER_CLANG) - target_compile_options(glibc-compatibility PRIVATE -Wno-unused-command-line-argument) - elseif (COMPILER_GCC) - target_compile_options(glibc-compatibility PRIVATE -Wno-unused-but-set-variable) - endif () + target_no_warning(glibc-compatibility unused-command-line-argument) + target_no_warning(glibc-compatibility unused-but-set-variable) + target_no_warning(glibc-compatibility builtin-requires-header) target_include_directories(glibc-compatibility PRIVATE libcxxabi ${musl_arch_include_dir}) diff --git a/cmake/add_warning.cmake b/cmake/add_warning.cmake index 3a776c98ab6..bc9642c9cc6 100644 --- a/cmake/add_warning.cmake +++ b/cmake/add_warning.cmake @@ -27,3 +27,22 @@ endmacro () macro (no_warning flag) add_warning(no-${flag}) endmacro () + + +# The same but only for specified target. +macro (target_add_warning target flag) + string (REPLACE "-" "_" underscored_flag ${flag}) + string (REPLACE "+" "x" underscored_flag ${underscored_flag}) + + check_cxx_compiler_flag("-W${flag}" SUPPORTS_CXXFLAG_${underscored_flag}) + + if (SUPPORTS_CXXFLAG_${underscored_flag}) + target_compile_options (${target} PRIVATE "-W${flag}") + else () + message (WARNING "Flag -W${flag} is unsupported") + endif () +endmacro () + +macro (target_no_warning target flag) + target_add_warning(${target} no-${flag}) +endmacro () From d417bd32474a4e659159b7025c7333c80f9dc467 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 16 Aug 2021 18:56:14 +0800 Subject: [PATCH 598/599] remove duplicated source files in arrow-cmake --- contrib/arrow-cmake/CMakeLists.txt | 3 --- 1 file changed, 3 deletions(-) diff --git a/contrib/arrow-cmake/CMakeLists.txt b/contrib/arrow-cmake/CMakeLists.txt index 2c72055a3e7..427379dc9b2 100644 --- a/contrib/arrow-cmake/CMakeLists.txt +++ b/contrib/arrow-cmake/CMakeLists.txt @@ -119,12 +119,9 @@ set(ORC_SRCS "${ORC_SOURCE_SRC_DIR}/ColumnWriter.cc" "${ORC_SOURCE_SRC_DIR}/Common.cc" "${ORC_SOURCE_SRC_DIR}/Compression.cc" - "${ORC_SOURCE_SRC_DIR}/Exceptions.cc" "${ORC_SOURCE_SRC_DIR}/Int128.cc" "${ORC_SOURCE_SRC_DIR}/LzoDecompressor.cc" "${ORC_SOURCE_SRC_DIR}/MemoryPool.cc" - "${ORC_SOURCE_SRC_DIR}/OrcFile.cc" - "${ORC_SOURCE_SRC_DIR}/Reader.cc" "${ORC_SOURCE_SRC_DIR}/RLE.cc" "${ORC_SOURCE_SRC_DIR}/RLEv1.cc" "${ORC_SOURCE_SRC_DIR}/RLEv2.cc" From 7ee2dc04a1f67963e7e32db9873031d325a0c2bb Mon Sep 17 00:00:00 2001 From: Alexey Boykov <33257111+mathalex@users.noreply.github.com> Date: Mon, 16 Aug 2021 15:06:44 +0300 Subject: [PATCH 599/599] Update parametric-functions.md typo --- .../sql-reference/aggregate-functions/parametric-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/aggregate-functions/parametric-functions.md b/docs/ru/sql-reference/aggregate-functions/parametric-functions.md index b1eefc3fc16..b3bb611e28c 100644 --- a/docs/ru/sql-reference/aggregate-functions/parametric-functions.md +++ b/docs/ru/sql-reference/aggregate-functions/parametric-functions.md @@ -172,7 +172,7 @@ SELECT sequenceMatch('(?1)(?2)')(time, number = 1, number = 2, number = 4) FROM ## sequenceCount(pattern)(time, cond1, cond2, …) {#function-sequencecount} -Вычисляет количество цепочек событий, соответствующих шаблону. Функция обнаруживает только непересекающиеся цепочки событий. Она начитает искать следующую цепочку только после того, как полностью совпала текущая цепочка событий. +Вычисляет количество цепочек событий, соответствующих шаблону. Функция обнаруживает только непересекающиеся цепочки событий. Она начинает искать следующую цепочку только после того, как полностью совпала текущая цепочка событий. !!! warning "Предупреждение" События, произошедшие в одну и ту же секунду, располагаются в последовательности в неопределенном порядке, что может повлиять на результат работы функции.