diff --git a/dbms/include/DB/AggregateFunctions/AggregateFunctionQuantileTiming.h b/dbms/include/DB/AggregateFunctions/AggregateFunctionQuantileTiming.h index fcc47a9b8dc..86a75c339e0 100644 --- a/dbms/include/DB/AggregateFunctions/AggregateFunctionQuantileTiming.h +++ b/dbms/include/DB/AggregateFunctions/AggregateFunctionQuantileTiming.h @@ -15,6 +15,7 @@ #include #include +#include namespace DB @@ -234,9 +235,16 @@ namespace detail template void getMany(const double * levels, size_t size, ResultType * result) const { - const double * levels_end = levels + size; - const double * level = levels; - UInt64 pos = count * *level; + std::size_t indices[size]; + std::copy(ext::range_iterator{}, ext::make_range_iterator(size), indices); + std::sort(indices, indices + size, [levels] (auto i1, auto i2) { + return levels[i1] < levels[i2]; + }); + + const auto indices_end = indices + size; + auto index = indices; + + UInt64 pos = count * levels[*index]; UInt64 accumulated = 0; @@ -251,15 +259,14 @@ namespace detail if (i < SMALL_THRESHOLD) { - *result = i; + result[*index] = i; - ++level; - ++result; + ++index; - if (level == levels_end) + if (index == indices_end) return; - pos = count * *level; + pos = count * levels[*index]; } } @@ -274,24 +281,22 @@ namespace detail if (i < BIG_SIZE) { - *result = indexInBigToValue(i); + result[*index] = indexInBigToValue(i); - ++level; - ++result; + ++index; - if (level == levels_end) + if (index == indices_end) return; - pos = count * *level; + pos = count * levels[*index]; } } - while (level < levels_end) + while (index < indices_end) { - *result = BIG_THRESHOLD; + result[*index] = BIG_THRESHOLD; - ++level; - ++result; + ++index; } } @@ -466,7 +471,7 @@ public: large = new detail::QuantileTimingLarge; } - + large->merge(detail::QuantileTimingLarge(buf)); } else @@ -712,11 +717,11 @@ public: size_t size = levels.size(); offsets_to.push_back((offsets_to.size() == 0 ? 0 : offsets_to.back()) + size); - + typename ColumnFloat32::Container_t & data_to = static_cast(arr_to.getData()).getData(); size_t old_size = data_to.size(); data_to.resize(data_to.size() + size); - + this->data(place).getManyFloat(&levels[0], size, &data_to[old_size]); } }; diff --git a/dbms/include/DB/AggregateFunctions/AggregateFunctionSequenceMatch.h b/dbms/include/DB/AggregateFunctions/AggregateFunctionSequenceMatch.h index 255028c2d05..aef08825536 100644 --- a/dbms/include/DB/AggregateFunctions/AggregateFunctionSequenceMatch.h +++ b/dbms/include/DB/AggregateFunctions/AggregateFunctionSequenceMatch.h @@ -124,7 +124,7 @@ struct AggregateFunctionSequenceMatchData final /// Max number of iterations to match the pattern against a sequence, exception thrown when exceeded constexpr auto sequence_match_max_iterations = 1000000; -class AggregateFunctionSequenceMatch final : public IAggregateFunctionHelper +class AggregateFunctionSequenceMatch : public IAggregateFunctionHelper { public: static bool sufficientArgs(const std::size_t arg_count) { return arg_count >= 3; } @@ -218,7 +218,14 @@ public: void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const override { const_cast(data(place)).sort(); - static_cast(to).getData().push_back(match(place)); + + const auto & data_ref = data(place); + + const auto events_begin = std::begin(data_ref.eventsList); + const auto events_end = std::end(data_ref.eventsList); + auto events_it = events_begin; + + static_cast(to).getData().push_back(match(events_it, events_end)); } private: @@ -233,21 +240,6 @@ private: TimeGreater }; - static std::string to_string(const PatternActionType type) - { - static const std::map map{ - { PatternActionType::SpecificEvent, "SpecificEvent" }, - { PatternActionType::AnyEvent, "AnyEvent" }, - { PatternActionType::KleeneStar, "KleeneStar" }, - { PatternActionType::TimeLessOrEqual, "TimeLessOrEqual" }, - { PatternActionType::TimeLess, "TimeLess", }, - { PatternActionType::TimeGreaterOrEqual, "TimeGreaterOrEqual" }, - { PatternActionType::TimeGreater, "TimeGreater" } - }; - - return map.find(type)->second; - } - struct PatternAction final { PatternActionType type; @@ -353,18 +345,15 @@ private: this->actions = std::move(actions); } - bool match(const ConstAggregateDataPtr & place) const +protected: + template + bool match(T1 & events_it, const T2 events_end) const { const auto action_begin = std::begin(actions); const auto action_end = std::end(actions); auto action_it = action_begin; - const auto & data_ref = data(place); - const auto events_begin = std::begin(data_ref.eventsList); - const auto events_end = std::end(data_ref.eventsList); - auto events_it = events_begin; - - auto base_it = events_begin; + auto base_it = events_it; /// an iterator to action plus an iterator to row in events list plus timestamp at the start of sequence using backtrack_info = std::tuple; @@ -392,11 +381,6 @@ private: std::size_t i = 0; while (action_it != action_end && events_it != events_end) { -// std::cout << "start_timestamp " << base_it->first << "; "; -// std::cout << "elapsed " << (events_it->first - base_it->first) << "; "; -// std::cout << "action " << (action_it - action_begin) << " { " << to_string(action_it->type) << ' ' << action_it->extra << " }; "; -// std::cout << "symbol " << (events_it - events_begin) << " { " << events_it->first << ' ' << events_it->second.to_ulong() << " }" << std::endl; - if (action_it->type == PatternActionType::SpecificEvent) { if (events_it->second.test(action_it->extra)) @@ -492,9 +476,40 @@ private: return action_it == action_end; } +private: std::string pattern; std::size_t arg_count; PatternActions actions; }; +class AggregateFunctionSequenceCount final : public AggregateFunctionSequenceMatch +{ +public: + String getName() const override { return "sequenceCount"; } + + DataTypePtr getReturnType() const override { return new DataTypeUInt64; } + + void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const override + { + const_cast(data(place)).sort(); + static_cast(to).getData().push_back(count(place)); + } + +private: + UInt64 count(const ConstAggregateDataPtr & place) const + { + const auto & data_ref = data(place); + + const auto events_begin = std::begin(data_ref.eventsList); + const auto events_end = std::end(data_ref.eventsList); + auto events_it = events_begin; + + std::size_t count = 0; + while (events_it != events_end && match(events_it, events_end)) + ++count; + + return count; + } +}; + } diff --git a/dbms/include/DB/Functions/FunctionsCoding.h b/dbms/include/DB/Functions/FunctionsCoding.h index 13cd8131e84..1d297bea127 100644 --- a/dbms/include/DB/Functions/FunctionsCoding.h +++ b/dbms/include/DB/Functions/FunctionsCoding.h @@ -1379,4 +1379,360 @@ public: } }; +namespace +{ + template UInt8 bitTest(const T1 val, const T2 pos) { return (val >> pos) & 1; }; +} + +class FunctionBitTest : public IFunction +{ +public: + static constexpr auto name = "bitTest"; + static IFunction * create(const Context &) { return new FunctionBitTest; } + + String getName() const override { return name; } + + DataTypePtr getReturnType(const DataTypes & arguments) const override + { + if (arguments.size() != 2) + throw Exception{ + "Number of arguments for function " + getName() + " doesn't match: passed " + + toString(arguments.size()) + ", should be 2.", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH + }; + + const auto first_arg = arguments.front().get(); + if (!typeid_cast(first_arg) && + !typeid_cast(first_arg) && + !typeid_cast(first_arg) && + !typeid_cast(first_arg) && + !typeid_cast(first_arg) && + !typeid_cast(first_arg) && + !typeid_cast(first_arg) && + !typeid_cast(first_arg)) + throw Exception{ + "Illegal type " + first_arg->getName() + " of first argument of function " + getName(), + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT + }; + + + const auto second_arg = arguments.back().get(); + if (!typeid_cast(second_arg) && + !typeid_cast(second_arg) && + !typeid_cast(second_arg) && + !typeid_cast(second_arg)) + throw Exception{ + "Illegal type " + second_arg->getName() + " of second argument of function " + getName(), + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT + }; + + return new DataTypeUInt8; + } + + void execute(Block & block, const ColumnNumbers & arguments, const size_t result) override + { + const auto value_col = block.getByPosition(arguments.front()).column.get(); + + if (!execute(block, arguments, result, value_col) && + !execute(block, arguments, result, value_col) && + !execute(block, arguments, result, value_col) && + !execute(block, arguments, result, value_col) && + !execute(block, arguments, result, value_col) && + !execute(block, arguments, result, value_col) && + !execute(block, arguments, result, value_col) && + !execute(block, arguments, result, value_col)) + throw Exception{ + "Illegal column " + value_col->getName() + " of argument of function " + getName(), + ErrorCodes::ILLEGAL_COLUMN + }; + } + +private: + template bool execute( + Block & block, const ColumnNumbers & arguments, const size_t result, + const IColumn * const value_col_untyped) + { + if (const auto value_col = typeid_cast *>(value_col_untyped)) + { + const auto pos_col = block.getByPosition(arguments.back()).column.get(); + + if (!execute(block, arguments, result, value_col, pos_col)) + throw Exception{ + "Illegal column " + pos_col->getName() + " of argument of function " + getName(), + ErrorCodes::ILLEGAL_COLUMN + }; + + return true; + } + else if (const auto value_col = typeid_cast *>(value_col_untyped)) + { + const auto pos_col = block.getByPosition(arguments.back()).column.get(); + + if (!execute(block, arguments, result, value_col, pos_col)) + throw Exception{ + "Illegal column " + pos_col->getName() + " of argument of function " + getName(), + ErrorCodes::ILLEGAL_COLUMN + }; + + return true; + } + + return false; + } + + template bool execute( + Block & block, const ColumnNumbers & arguments, const size_t result, + const ValueColumn * const value_col, const IColumn * const pos_col) + { + return execute(block, arguments, result, value_col, pos_col) || + execute(block, arguments, result, value_col, pos_col) || + execute(block, arguments, result, value_col, pos_col) || + execute(block, arguments, result, value_col, pos_col); + } + + template bool execute( + Block & block, const ColumnNumbers & arguments, const size_t result, + const ColumnVector * const value_col, const IColumn * const pos_col_untyped) + { + if (const auto pos_col = typeid_cast *>(pos_col_untyped)) + { + const auto & values = value_col->getData(); + const auto & positions = pos_col->getData(); + + const auto size = value_col->size(); + const auto out_col = new ColumnVector(size); + ColumnPtr out_col_ptr{out_col}; + block.getByPosition(result).column = out_col_ptr; + + auto & out = out_col->getData(); + + for (const auto i : ext::range(0, size)) + out[i] = bitTest(values[i], positions[i]); + + return true; + } + else if (const auto pos_col = typeid_cast *>(pos_col_untyped)) + { + const auto & values = value_col->getData(); + + const auto size = value_col->size(); + const auto out_col = new ColumnVector(size); + ColumnPtr out_col_ptr{out_col}; + block.getByPosition(result).column = out_col_ptr; + + auto & out = out_col->getData(); + + for (const auto i : ext::range(0, size)) + out[i] = bitTest(values[i], pos_col->getData()); + + return true; + } + + return false; + } + + template bool execute( + Block & block, const ColumnNumbers & arguments, const size_t result, + const ColumnConst * const value_col, const IColumn * const pos_col_untyped) + { + if (const auto pos_col = typeid_cast *>(pos_col_untyped)) + { + const auto & positions = pos_col->getData(); + + const auto size = value_col->size(); + const auto out_col = new ColumnVector(size); + ColumnPtr out_col_ptr{out_col}; + block.getByPosition(result).column = out_col_ptr; + + auto & out = out_col->getData(); + + for (const auto i : ext::range(0, size)) + out[i] = bitTest(value_col->getData(), positions[i]); + + return true; + } + else if (const auto pos_col = typeid_cast *>(pos_col_untyped)) + { + block.getByPosition(result).column = new ColumnConst{ + value_col->size(), + bitTest(value_col->getData(), pos_col->getData()) + }; + + return true; + } + + return false; + } +}; + +template +struct FunctionBitTestMany : public IFunction +{ +public: + static constexpr auto name = Impl::name; + static IFunction * create(const Context &) { return new FunctionBitTestMany; } + + String getName() const override { return name; } + + DataTypePtr getReturnType(const DataTypes & arguments) const override + { + if (arguments.size() < 2) + throw Exception{ + "Number of arguments for function " + getName() + " doesn't match: passed " + + toString(arguments.size()) + ", should be at least 2.", + ErrorCodes::TOO_LESS_ARGUMENTS_FOR_FUNCTION + }; + + const auto first_arg = arguments.front().get(); + if (!typeid_cast(first_arg) && !typeid_cast(first_arg) && + !typeid_cast(first_arg) && !typeid_cast(first_arg) && + !typeid_cast(first_arg) && !typeid_cast(first_arg) && + !typeid_cast(first_arg) && !typeid_cast(first_arg)) + throw Exception{ + "Illegal type " + first_arg->getName() + " of first argument of function " + getName(), + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT + }; + + + for (const auto i : ext::range(1, arguments.size())) + { + const auto pos_arg = arguments[i].get(); + + if (!typeid_cast(pos_arg) && !typeid_cast(pos_arg) && + !typeid_cast(pos_arg) && !typeid_cast(pos_arg)) + throw Exception{ + "Illegal type " + pos_arg->getName() + " of " + toString(i) + " argument of function " + getName(), + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT + }; + } + + return new DataTypeUInt8; + } + + void execute(Block & block, const ColumnNumbers & arguments, const size_t result) override + { + const auto value_col = block.getByPosition(arguments.front()).column.get(); + + if (!execute(block, arguments, result, value_col) && + !execute(block, arguments, result, value_col) && + !execute(block, arguments, result, value_col) && + !execute(block, arguments, result, value_col) && + !execute(block, arguments, result, value_col) && + !execute(block, arguments, result, value_col) && + !execute(block, arguments, result, value_col) && + !execute(block, arguments, result, value_col)) + throw Exception{ + "Illegal column " + value_col->getName() + " of argument of function " + getName(), + ErrorCodes::ILLEGAL_COLUMN + }; + } + +private: + template bool execute( + Block & block, const ColumnNumbers & arguments, const size_t result, + const IColumn * const value_col_untyped) + { + if (const auto value_col = typeid_cast *>(value_col_untyped)) + { + const auto size = value_col->size(); + const auto & mask = createMask(size, block, arguments); + const auto & val = value_col->getData(); + + const auto out_col = new ColumnVector(size); + ColumnPtr out_col_ptr{out_col}; + block.getByPosition(result).column = out_col_ptr; + + auto & out = out_col->getData(); + + for (const auto i : ext::range(0, size)) + out[i] = Impl::combine(val[i], mask[i]); + + return true; + } + else if (const auto value_col = typeid_cast *>(value_col_untyped)) + { + const auto size = value_col->size(); + const auto & mask = createMask(size, block, arguments); + const auto & val = value_col->getData(); + + const auto out_col = new ColumnVector(size); + ColumnPtr out_col_ptr{out_col}; + block.getByPosition(result).column = out_col_ptr; + + auto & out = out_col->getData(); + + for (const auto i : ext::range(0, size)) + out[i] = Impl::combine(val, mask[i]); + + return true; + } + + return false; + } + + template + PODArray createMask(const std::size_t size, const Block & block, const ColumnNumbers & arguments) + { + PODArray mask(size, T{}); + + for (const auto i : ext::range(1, arguments.size())) + addToMask(mask, block.getByPosition(arguments[i]).column.get()); + + return mask; + } + + template + void addToMask(PODArray & mask, const IColumn * const pos_col) + { + if (!addToMaskImpl(mask, pos_col) && !addToMaskImpl(mask, pos_col) && + !addToMaskImpl(mask, pos_col) && !addToMaskImpl(mask, pos_col)) + throw Exception{ + "Illegal column " + pos_col->getName() + " of argument of function " + getName(), + ErrorCodes::ILLEGAL_COLUMN + }; + } + + template + bool addToMaskImpl(PODArray & mask, const IColumn * const pos_col_untyped) + { + if (const auto pos_col = typeid_cast *>(pos_col_untyped)) + { + const auto & pos = pos_col->getData(); + + for (const auto i : ext::range(0, mask.size())) + mask[i] = mask[i] | (1 << pos[i]); + + return true; + } + else if (const auto pos_col = typeid_cast *>(pos_col_untyped)) + { + const auto & pos = pos_col->getData(); + const auto new_mask = 1 << pos; + + for (const auto i : ext::range(0, mask.size())) + mask[i] = mask[i] | new_mask; + + return true; + } + + return false; + } +}; + +struct BitTestAnyImpl +{ + static constexpr auto name = "bitTestAny"; + template static UInt8 combine(const T val, const T mask) { return (val & mask) != 0; } +}; + +struct BitTestAllImpl +{ + static constexpr auto name = "bitTestAll"; + template static UInt8 combine(const T val, const T mask) { return (val & mask) == mask; } +}; + +using FunctionBitTestAny = FunctionBitTestMany; +using FunctionBitTestAll = FunctionBitTestMany; + + } diff --git a/dbms/include/DB/Functions/FunctionsDictionaries.h b/dbms/include/DB/Functions/FunctionsDictionaries.h index cf7eaea60ff..4cb6b449c34 100644 --- a/dbms/include/DB/Functions/FunctionsDictionaries.h +++ b/dbms/include/DB/Functions/FunctionsDictionaries.h @@ -1462,15 +1462,6 @@ private: return true; } - template - bool execute(Block & block, const size_t result, const DictionaryType * const dictionary, - const IColumn * const id_col_untyped) - { - - - return false; - } - const ExternalDictionaries & dictionaries; }; diff --git a/dbms/src/AggregateFunctions/AggregateFunctionFactory.cpp b/dbms/src/AggregateFunctions/AggregateFunctionFactory.cpp index 245530b86d6..c5f503da3f0 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionFactory.cpp +++ b/dbms/src/AggregateFunctions/AggregateFunctionFactory.cpp @@ -562,6 +562,13 @@ AggregateFunctionPtr AggregateFunctionFactory::get(const String & name, const Da return new AggregateFunctionSequenceMatch; } + else if (name == "sequenceCount") + { + if (!AggregateFunctionSequenceCount::sufficientArgs(argument_types.size())) + throw Exception("Incorrect number of arguments for aggregate function " + name, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + return new AggregateFunctionSequenceCount; + } else if (name == "varSamp") { if (argument_types.size() != 1) @@ -743,6 +750,7 @@ const AggregateFunctionFactory::FunctionNames & AggregateFunctionFactory::getFun "quantileDeterministic", "quantilesDeterministic", "sequenceMatch", + "sequenceCount", "varSamp", "varPop", "stddevSamp", diff --git a/dbms/src/Functions/FunctionsCoding.cpp b/dbms/src/Functions/FunctionsCoding.cpp index 5bef94fe93a..d213f0f6249 100644 --- a/dbms/src/Functions/FunctionsCoding.cpp +++ b/dbms/src/Functions/FunctionsCoding.cpp @@ -15,6 +15,9 @@ void registerFunctionsCoding(FunctionFactory & factory) factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); } } diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 53ad4c2e66d..51d278c3e62 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -351,9 +351,6 @@ BlockInputStreams MergeTreeDataSelectExecutor::spreadMarkRangesAmongThreads( const size_t max_marks_to_use_cache = (settings.merge_tree_max_rows_to_use_cache + data.index_granularity - 1) / data.index_granularity; - /// На всякий случай перемешаем куски. - std::random_shuffle(parts.begin(), parts.end()); - /// Посчитаем засечки для каждого куска. std::vector sum_marks_in_parts(parts.size()); size_t sum_marks = 0; diff --git a/dbms/tests/queries/0_stateless/00216_bit_test_function_family.reference b/dbms/tests/queries/0_stateless/00216_bit_test_function_family.reference new file mode 100644 index 00000000000..54ece010c27 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00216_bit_test_function_family.reference @@ -0,0 +1,771 @@ +1 1 1 1 +1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 +1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 +1 0 0 0 0 0 0 0 0 0 0 +1 1 1 0 0 0 0 0 0 0 1 +1 2 2 0 0 0 0 0 0 1 0 +1 3 3 0 0 0 0 0 0 1 1 +1 4 4 0 0 0 0 0 1 0 0 +1 5 5 0 0 0 0 0 1 0 1 +1 6 6 0 0 0 0 0 1 1 0 +1 7 7 0 0 0 0 0 1 1 1 +1 8 8 0 0 0 0 1 0 0 0 +1 9 9 0 0 0 0 1 0 0 1 +1 10 10 0 0 0 0 1 0 1 0 +1 11 11 0 0 0 0 1 0 1 1 +1 12 12 0 0 0 0 1 1 0 0 +1 13 13 0 0 0 0 1 1 0 1 +1 14 14 0 0 0 0 1 1 1 0 +1 15 15 0 0 0 0 1 1 1 1 +1 16 16 0 0 0 1 0 0 0 0 +1 17 17 0 0 0 1 0 0 0 1 +1 18 18 0 0 0 1 0 0 1 0 +1 19 19 0 0 0 1 0 0 1 1 +1 20 20 0 0 0 1 0 1 0 0 +1 21 21 0 0 0 1 0 1 0 1 +1 22 22 0 0 0 1 0 1 1 0 +1 23 23 0 0 0 1 0 1 1 1 +1 24 24 0 0 0 1 1 0 0 0 +1 25 25 0 0 0 1 1 0 0 1 +1 26 26 0 0 0 1 1 0 1 0 +1 27 27 0 0 0 1 1 0 1 1 +1 28 28 0 0 0 1 1 1 0 0 +1 29 29 0 0 0 1 1 1 0 1 +1 30 30 0 0 0 1 1 1 1 0 +1 31 31 0 0 0 1 1 1 1 1 +1 32 32 0 0 1 0 0 0 0 0 +1 33 33 0 0 1 0 0 0 0 1 +1 34 34 0 0 1 0 0 0 1 0 +1 35 35 0 0 1 0 0 0 1 1 +1 36 36 0 0 1 0 0 1 0 0 +1 37 37 0 0 1 0 0 1 0 1 +1 38 38 0 0 1 0 0 1 1 0 +1 39 39 0 0 1 0 0 1 1 1 +1 40 40 0 0 1 0 1 0 0 0 +1 41 41 0 0 1 0 1 0 0 1 +1 42 42 0 0 1 0 1 0 1 0 +1 43 43 0 0 1 0 1 0 1 1 +1 44 44 0 0 1 0 1 1 0 0 +1 45 45 0 0 1 0 1 1 0 1 +1 46 46 0 0 1 0 1 1 1 0 +1 47 47 0 0 1 0 1 1 1 1 +1 48 48 0 0 1 1 0 0 0 0 +1 49 49 0 0 1 1 0 0 0 1 +1 50 50 0 0 1 1 0 0 1 0 +1 51 51 0 0 1 1 0 0 1 1 +1 52 52 0 0 1 1 0 1 0 0 +1 53 53 0 0 1 1 0 1 0 1 +1 54 54 0 0 1 1 0 1 1 0 +1 55 55 0 0 1 1 0 1 1 1 +1 56 56 0 0 1 1 1 0 0 0 +1 57 57 0 0 1 1 1 0 0 1 +1 58 58 0 0 1 1 1 0 1 0 +1 59 59 0 0 1 1 1 0 1 1 +1 60 60 0 0 1 1 1 1 0 0 +1 61 61 0 0 1 1 1 1 0 1 +1 62 62 0 0 1 1 1 1 1 0 +1 63 63 0 0 1 1 1 1 1 1 +1 64 64 0 1 0 0 0 0 0 0 +1 65 65 0 1 0 0 0 0 0 1 +1 66 66 0 1 0 0 0 0 1 0 +1 67 67 0 1 0 0 0 0 1 1 +1 68 68 0 1 0 0 0 1 0 0 +1 69 69 0 1 0 0 0 1 0 1 +1 70 70 0 1 0 0 0 1 1 0 +1 71 71 0 1 0 0 0 1 1 1 +1 72 72 0 1 0 0 1 0 0 0 +1 73 73 0 1 0 0 1 0 0 1 +1 74 74 0 1 0 0 1 0 1 0 +1 75 75 0 1 0 0 1 0 1 1 +1 76 76 0 1 0 0 1 1 0 0 +1 77 77 0 1 0 0 1 1 0 1 +1 78 78 0 1 0 0 1 1 1 0 +1 79 79 0 1 0 0 1 1 1 1 +1 80 80 0 1 0 1 0 0 0 0 +1 81 81 0 1 0 1 0 0 0 1 +1 82 82 0 1 0 1 0 0 1 0 +1 83 83 0 1 0 1 0 0 1 1 +1 84 84 0 1 0 1 0 1 0 0 +1 85 85 0 1 0 1 0 1 0 1 +1 86 86 0 1 0 1 0 1 1 0 +1 87 87 0 1 0 1 0 1 1 1 +1 88 88 0 1 0 1 1 0 0 0 +1 89 89 0 1 0 1 1 0 0 1 +1 90 90 0 1 0 1 1 0 1 0 +1 91 91 0 1 0 1 1 0 1 1 +1 92 92 0 1 0 1 1 1 0 0 +1 93 93 0 1 0 1 1 1 0 1 +1 94 94 0 1 0 1 1 1 1 0 +1 95 95 0 1 0 1 1 1 1 1 +1 96 96 0 1 1 0 0 0 0 0 +1 97 97 0 1 1 0 0 0 0 1 +1 98 98 0 1 1 0 0 0 1 0 +1 99 99 0 1 1 0 0 0 1 1 +1 100 100 0 1 1 0 0 1 0 0 +1 101 101 0 1 1 0 0 1 0 1 +1 102 102 0 1 1 0 0 1 1 0 +1 103 103 0 1 1 0 0 1 1 1 +1 104 104 0 1 1 0 1 0 0 0 +1 105 105 0 1 1 0 1 0 0 1 +1 106 106 0 1 1 0 1 0 1 0 +1 107 107 0 1 1 0 1 0 1 1 +1 108 108 0 1 1 0 1 1 0 0 +1 109 109 0 1 1 0 1 1 0 1 +1 110 110 0 1 1 0 1 1 1 0 +1 111 111 0 1 1 0 1 1 1 1 +1 112 112 0 1 1 1 0 0 0 0 +1 113 113 0 1 1 1 0 0 0 1 +1 114 114 0 1 1 1 0 0 1 0 +1 115 115 0 1 1 1 0 0 1 1 +1 116 116 0 1 1 1 0 1 0 0 +1 117 117 0 1 1 1 0 1 0 1 +1 118 118 0 1 1 1 0 1 1 0 +1 119 119 0 1 1 1 0 1 1 1 +1 120 120 0 1 1 1 1 0 0 0 +1 121 121 0 1 1 1 1 0 0 1 +1 122 122 0 1 1 1 1 0 1 0 +1 123 123 0 1 1 1 1 0 1 1 +1 124 124 0 1 1 1 1 1 0 0 +1 125 125 0 1 1 1 1 1 0 1 +1 126 126 0 1 1 1 1 1 1 0 +1 127 127 0 1 1 1 1 1 1 1 +1 128 128 1 0 0 0 0 0 0 0 +1 129 129 1 0 0 0 0 0 0 1 +1 130 130 1 0 0 0 0 0 1 0 +1 131 131 1 0 0 0 0 0 1 1 +1 132 132 1 0 0 0 0 1 0 0 +1 133 133 1 0 0 0 0 1 0 1 +1 134 134 1 0 0 0 0 1 1 0 +1 135 135 1 0 0 0 0 1 1 1 +1 136 136 1 0 0 0 1 0 0 0 +1 137 137 1 0 0 0 1 0 0 1 +1 138 138 1 0 0 0 1 0 1 0 +1 139 139 1 0 0 0 1 0 1 1 +1 140 140 1 0 0 0 1 1 0 0 +1 141 141 1 0 0 0 1 1 0 1 +1 142 142 1 0 0 0 1 1 1 0 +1 143 143 1 0 0 0 1 1 1 1 +1 144 144 1 0 0 1 0 0 0 0 +1 145 145 1 0 0 1 0 0 0 1 +1 146 146 1 0 0 1 0 0 1 0 +1 147 147 1 0 0 1 0 0 1 1 +1 148 148 1 0 0 1 0 1 0 0 +1 149 149 1 0 0 1 0 1 0 1 +1 150 150 1 0 0 1 0 1 1 0 +1 151 151 1 0 0 1 0 1 1 1 +1 152 152 1 0 0 1 1 0 0 0 +1 153 153 1 0 0 1 1 0 0 1 +1 154 154 1 0 0 1 1 0 1 0 +1 155 155 1 0 0 1 1 0 1 1 +1 156 156 1 0 0 1 1 1 0 0 +1 157 157 1 0 0 1 1 1 0 1 +1 158 158 1 0 0 1 1 1 1 0 +1 159 159 1 0 0 1 1 1 1 1 +1 160 160 1 0 1 0 0 0 0 0 +1 161 161 1 0 1 0 0 0 0 1 +1 162 162 1 0 1 0 0 0 1 0 +1 163 163 1 0 1 0 0 0 1 1 +1 164 164 1 0 1 0 0 1 0 0 +1 165 165 1 0 1 0 0 1 0 1 +1 166 166 1 0 1 0 0 1 1 0 +1 167 167 1 0 1 0 0 1 1 1 +1 168 168 1 0 1 0 1 0 0 0 +1 169 169 1 0 1 0 1 0 0 1 +1 170 170 1 0 1 0 1 0 1 0 +1 171 171 1 0 1 0 1 0 1 1 +1 172 172 1 0 1 0 1 1 0 0 +1 173 173 1 0 1 0 1 1 0 1 +1 174 174 1 0 1 0 1 1 1 0 +1 175 175 1 0 1 0 1 1 1 1 +1 176 176 1 0 1 1 0 0 0 0 +1 177 177 1 0 1 1 0 0 0 1 +1 178 178 1 0 1 1 0 0 1 0 +1 179 179 1 0 1 1 0 0 1 1 +1 180 180 1 0 1 1 0 1 0 0 +1 181 181 1 0 1 1 0 1 0 1 +1 182 182 1 0 1 1 0 1 1 0 +1 183 183 1 0 1 1 0 1 1 1 +1 184 184 1 0 1 1 1 0 0 0 +1 185 185 1 0 1 1 1 0 0 1 +1 186 186 1 0 1 1 1 0 1 0 +1 187 187 1 0 1 1 1 0 1 1 +1 188 188 1 0 1 1 1 1 0 0 +1 189 189 1 0 1 1 1 1 0 1 +1 190 190 1 0 1 1 1 1 1 0 +1 191 191 1 0 1 1 1 1 1 1 +1 192 192 1 1 0 0 0 0 0 0 +1 193 193 1 1 0 0 0 0 0 1 +1 194 194 1 1 0 0 0 0 1 0 +1 195 195 1 1 0 0 0 0 1 1 +1 196 196 1 1 0 0 0 1 0 0 +1 197 197 1 1 0 0 0 1 0 1 +1 198 198 1 1 0 0 0 1 1 0 +1 199 199 1 1 0 0 0 1 1 1 +1 200 200 1 1 0 0 1 0 0 0 +1 201 201 1 1 0 0 1 0 0 1 +1 202 202 1 1 0 0 1 0 1 0 +1 203 203 1 1 0 0 1 0 1 1 +1 204 204 1 1 0 0 1 1 0 0 +1 205 205 1 1 0 0 1 1 0 1 +1 206 206 1 1 0 0 1 1 1 0 +1 207 207 1 1 0 0 1 1 1 1 +1 208 208 1 1 0 1 0 0 0 0 +1 209 209 1 1 0 1 0 0 0 1 +1 210 210 1 1 0 1 0 0 1 0 +1 211 211 1 1 0 1 0 0 1 1 +1 212 212 1 1 0 1 0 1 0 0 +1 213 213 1 1 0 1 0 1 0 1 +1 214 214 1 1 0 1 0 1 1 0 +1 215 215 1 1 0 1 0 1 1 1 +1 216 216 1 1 0 1 1 0 0 0 +1 217 217 1 1 0 1 1 0 0 1 +1 218 218 1 1 0 1 1 0 1 0 +1 219 219 1 1 0 1 1 0 1 1 +1 220 220 1 1 0 1 1 1 0 0 +1 221 221 1 1 0 1 1 1 0 1 +1 222 222 1 1 0 1 1 1 1 0 +1 223 223 1 1 0 1 1 1 1 1 +1 224 224 1 1 1 0 0 0 0 0 +1 225 225 1 1 1 0 0 0 0 1 +1 226 226 1 1 1 0 0 0 1 0 +1 227 227 1 1 1 0 0 0 1 1 +1 228 228 1 1 1 0 0 1 0 0 +1 229 229 1 1 1 0 0 1 0 1 +1 230 230 1 1 1 0 0 1 1 0 +1 231 231 1 1 1 0 0 1 1 1 +1 232 232 1 1 1 0 1 0 0 0 +1 233 233 1 1 1 0 1 0 0 1 +1 234 234 1 1 1 0 1 0 1 0 +1 235 235 1 1 1 0 1 0 1 1 +1 236 236 1 1 1 0 1 1 0 0 +1 237 237 1 1 1 0 1 1 0 1 +1 238 238 1 1 1 0 1 1 1 0 +1 239 239 1 1 1 0 1 1 1 1 +1 240 240 1 1 1 1 0 0 0 0 +1 241 241 1 1 1 1 0 0 0 1 +1 242 242 1 1 1 1 0 0 1 0 +1 243 243 1 1 1 1 0 0 1 1 +1 244 244 1 1 1 1 0 1 0 0 +1 245 245 1 1 1 1 0 1 0 1 +1 246 246 1 1 1 1 0 1 1 0 +1 247 247 1 1 1 1 0 1 1 1 +1 248 248 1 1 1 1 1 0 0 0 +1 249 249 1 1 1 1 1 0 0 1 +1 250 250 1 1 1 1 1 0 1 0 +1 251 251 1 1 1 1 1 0 1 1 +1 252 252 1 1 1 1 1 1 0 0 +1 253 253 1 1 1 1 1 1 0 1 +1 254 254 1 1 1 1 1 1 1 0 +1 255 255 1 1 1 1 1 1 1 1 +1 0 0 0 0 0 0 0 0 0 0 +1 1 1 0 0 0 0 0 0 0 1 +1 2 2 0 0 0 0 0 0 1 0 +1 3 3 0 0 0 0 0 0 1 1 +1 4 4 0 0 0 0 0 1 0 0 +1 5 5 0 0 0 0 0 1 0 1 +1 6 6 0 0 0 0 0 1 1 0 +1 7 7 0 0 0 0 0 1 1 1 +1 8 8 0 0 0 0 1 0 0 0 +1 9 9 0 0 0 0 1 0 0 1 +1 10 10 0 0 0 0 1 0 1 0 +1 11 11 0 0 0 0 1 0 1 1 +1 12 12 0 0 0 0 1 1 0 0 +1 13 13 0 0 0 0 1 1 0 1 +1 14 14 0 0 0 0 1 1 1 0 +1 15 15 0 0 0 0 1 1 1 1 +1 16 16 0 0 0 1 0 0 0 0 +1 17 17 0 0 0 1 0 0 0 1 +1 18 18 0 0 0 1 0 0 1 0 +1 19 19 0 0 0 1 0 0 1 1 +1 20 20 0 0 0 1 0 1 0 0 +1 21 21 0 0 0 1 0 1 0 1 +1 22 22 0 0 0 1 0 1 1 0 +1 23 23 0 0 0 1 0 1 1 1 +1 24 24 0 0 0 1 1 0 0 0 +1 25 25 0 0 0 1 1 0 0 1 +1 26 26 0 0 0 1 1 0 1 0 +1 27 27 0 0 0 1 1 0 1 1 +1 28 28 0 0 0 1 1 1 0 0 +1 29 29 0 0 0 1 1 1 0 1 +1 30 30 0 0 0 1 1 1 1 0 +1 31 31 0 0 0 1 1 1 1 1 +1 32 32 0 0 1 0 0 0 0 0 +1 33 33 0 0 1 0 0 0 0 1 +1 34 34 0 0 1 0 0 0 1 0 +1 35 35 0 0 1 0 0 0 1 1 +1 36 36 0 0 1 0 0 1 0 0 +1 37 37 0 0 1 0 0 1 0 1 +1 38 38 0 0 1 0 0 1 1 0 +1 39 39 0 0 1 0 0 1 1 1 +1 40 40 0 0 1 0 1 0 0 0 +1 41 41 0 0 1 0 1 0 0 1 +1 42 42 0 0 1 0 1 0 1 0 +1 43 43 0 0 1 0 1 0 1 1 +1 44 44 0 0 1 0 1 1 0 0 +1 45 45 0 0 1 0 1 1 0 1 +1 46 46 0 0 1 0 1 1 1 0 +1 47 47 0 0 1 0 1 1 1 1 +1 48 48 0 0 1 1 0 0 0 0 +1 49 49 0 0 1 1 0 0 0 1 +1 50 50 0 0 1 1 0 0 1 0 +1 51 51 0 0 1 1 0 0 1 1 +1 52 52 0 0 1 1 0 1 0 0 +1 53 53 0 0 1 1 0 1 0 1 +1 54 54 0 0 1 1 0 1 1 0 +1 55 55 0 0 1 1 0 1 1 1 +1 56 56 0 0 1 1 1 0 0 0 +1 57 57 0 0 1 1 1 0 0 1 +1 58 58 0 0 1 1 1 0 1 0 +1 59 59 0 0 1 1 1 0 1 1 +1 60 60 0 0 1 1 1 1 0 0 +1 61 61 0 0 1 1 1 1 0 1 +1 62 62 0 0 1 1 1 1 1 0 +1 63 63 0 0 1 1 1 1 1 1 +1 64 64 0 1 0 0 0 0 0 0 +1 65 65 0 1 0 0 0 0 0 1 +1 66 66 0 1 0 0 0 0 1 0 +1 67 67 0 1 0 0 0 0 1 1 +1 68 68 0 1 0 0 0 1 0 0 +1 69 69 0 1 0 0 0 1 0 1 +1 70 70 0 1 0 0 0 1 1 0 +1 71 71 0 1 0 0 0 1 1 1 +1 72 72 0 1 0 0 1 0 0 0 +1 73 73 0 1 0 0 1 0 0 1 +1 74 74 0 1 0 0 1 0 1 0 +1 75 75 0 1 0 0 1 0 1 1 +1 76 76 0 1 0 0 1 1 0 0 +1 77 77 0 1 0 0 1 1 0 1 +1 78 78 0 1 0 0 1 1 1 0 +1 79 79 0 1 0 0 1 1 1 1 +1 80 80 0 1 0 1 0 0 0 0 +1 81 81 0 1 0 1 0 0 0 1 +1 82 82 0 1 0 1 0 0 1 0 +1 83 83 0 1 0 1 0 0 1 1 +1 84 84 0 1 0 1 0 1 0 0 +1 85 85 0 1 0 1 0 1 0 1 +1 86 86 0 1 0 1 0 1 1 0 +1 87 87 0 1 0 1 0 1 1 1 +1 88 88 0 1 0 1 1 0 0 0 +1 89 89 0 1 0 1 1 0 0 1 +1 90 90 0 1 0 1 1 0 1 0 +1 91 91 0 1 0 1 1 0 1 1 +1 92 92 0 1 0 1 1 1 0 0 +1 93 93 0 1 0 1 1 1 0 1 +1 94 94 0 1 0 1 1 1 1 0 +1 95 95 0 1 0 1 1 1 1 1 +1 96 96 0 1 1 0 0 0 0 0 +1 97 97 0 1 1 0 0 0 0 1 +1 98 98 0 1 1 0 0 0 1 0 +1 99 99 0 1 1 0 0 0 1 1 +1 100 100 0 1 1 0 0 1 0 0 +1 101 101 0 1 1 0 0 1 0 1 +1 102 102 0 1 1 0 0 1 1 0 +1 103 103 0 1 1 0 0 1 1 1 +1 104 104 0 1 1 0 1 0 0 0 +1 105 105 0 1 1 0 1 0 0 1 +1 106 106 0 1 1 0 1 0 1 0 +1 107 107 0 1 1 0 1 0 1 1 +1 108 108 0 1 1 0 1 1 0 0 +1 109 109 0 1 1 0 1 1 0 1 +1 110 110 0 1 1 0 1 1 1 0 +1 111 111 0 1 1 0 1 1 1 1 +1 112 112 0 1 1 1 0 0 0 0 +1 113 113 0 1 1 1 0 0 0 1 +1 114 114 0 1 1 1 0 0 1 0 +1 115 115 0 1 1 1 0 0 1 1 +1 116 116 0 1 1 1 0 1 0 0 +1 117 117 0 1 1 1 0 1 0 1 +1 118 118 0 1 1 1 0 1 1 0 +1 119 119 0 1 1 1 0 1 1 1 +1 120 120 0 1 1 1 1 0 0 0 +1 121 121 0 1 1 1 1 0 0 1 +1 122 122 0 1 1 1 1 0 1 0 +1 123 123 0 1 1 1 1 0 1 1 +1 124 124 0 1 1 1 1 1 0 0 +1 125 125 0 1 1 1 1 1 0 1 +1 126 126 0 1 1 1 1 1 1 0 +1 127 127 0 1 1 1 1 1 1 1 +1 128 128 1 0 0 0 0 0 0 0 +1 129 129 1 0 0 0 0 0 0 1 +1 130 130 1 0 0 0 0 0 1 0 +1 131 131 1 0 0 0 0 0 1 1 +1 132 132 1 0 0 0 0 1 0 0 +1 133 133 1 0 0 0 0 1 0 1 +1 134 134 1 0 0 0 0 1 1 0 +1 135 135 1 0 0 0 0 1 1 1 +1 136 136 1 0 0 0 1 0 0 0 +1 137 137 1 0 0 0 1 0 0 1 +1 138 138 1 0 0 0 1 0 1 0 +1 139 139 1 0 0 0 1 0 1 1 +1 140 140 1 0 0 0 1 1 0 0 +1 141 141 1 0 0 0 1 1 0 1 +1 142 142 1 0 0 0 1 1 1 0 +1 143 143 1 0 0 0 1 1 1 1 +1 144 144 1 0 0 1 0 0 0 0 +1 145 145 1 0 0 1 0 0 0 1 +1 146 146 1 0 0 1 0 0 1 0 +1 147 147 1 0 0 1 0 0 1 1 +1 148 148 1 0 0 1 0 1 0 0 +1 149 149 1 0 0 1 0 1 0 1 +1 150 150 1 0 0 1 0 1 1 0 +1 151 151 1 0 0 1 0 1 1 1 +1 152 152 1 0 0 1 1 0 0 0 +1 153 153 1 0 0 1 1 0 0 1 +1 154 154 1 0 0 1 1 0 1 0 +1 155 155 1 0 0 1 1 0 1 1 +1 156 156 1 0 0 1 1 1 0 0 +1 157 157 1 0 0 1 1 1 0 1 +1 158 158 1 0 0 1 1 1 1 0 +1 159 159 1 0 0 1 1 1 1 1 +1 160 160 1 0 1 0 0 0 0 0 +1 161 161 1 0 1 0 0 0 0 1 +1 162 162 1 0 1 0 0 0 1 0 +1 163 163 1 0 1 0 0 0 1 1 +1 164 164 1 0 1 0 0 1 0 0 +1 165 165 1 0 1 0 0 1 0 1 +1 166 166 1 0 1 0 0 1 1 0 +1 167 167 1 0 1 0 0 1 1 1 +1 168 168 1 0 1 0 1 0 0 0 +1 169 169 1 0 1 0 1 0 0 1 +1 170 170 1 0 1 0 1 0 1 0 +1 171 171 1 0 1 0 1 0 1 1 +1 172 172 1 0 1 0 1 1 0 0 +1 173 173 1 0 1 0 1 1 0 1 +1 174 174 1 0 1 0 1 1 1 0 +1 175 175 1 0 1 0 1 1 1 1 +1 176 176 1 0 1 1 0 0 0 0 +1 177 177 1 0 1 1 0 0 0 1 +1 178 178 1 0 1 1 0 0 1 0 +1 179 179 1 0 1 1 0 0 1 1 +1 180 180 1 0 1 1 0 1 0 0 +1 181 181 1 0 1 1 0 1 0 1 +1 182 182 1 0 1 1 0 1 1 0 +1 183 183 1 0 1 1 0 1 1 1 +1 184 184 1 0 1 1 1 0 0 0 +1 185 185 1 0 1 1 1 0 0 1 +1 186 186 1 0 1 1 1 0 1 0 +1 187 187 1 0 1 1 1 0 1 1 +1 188 188 1 0 1 1 1 1 0 0 +1 189 189 1 0 1 1 1 1 0 1 +1 190 190 1 0 1 1 1 1 1 0 +1 191 191 1 0 1 1 1 1 1 1 +1 192 192 1 1 0 0 0 0 0 0 +1 193 193 1 1 0 0 0 0 0 1 +1 194 194 1 1 0 0 0 0 1 0 +1 195 195 1 1 0 0 0 0 1 1 +1 196 196 1 1 0 0 0 1 0 0 +1 197 197 1 1 0 0 0 1 0 1 +1 198 198 1 1 0 0 0 1 1 0 +1 199 199 1 1 0 0 0 1 1 1 +1 200 200 1 1 0 0 1 0 0 0 +1 201 201 1 1 0 0 1 0 0 1 +1 202 202 1 1 0 0 1 0 1 0 +1 203 203 1 1 0 0 1 0 1 1 +1 204 204 1 1 0 0 1 1 0 0 +1 205 205 1 1 0 0 1 1 0 1 +1 206 206 1 1 0 0 1 1 1 0 +1 207 207 1 1 0 0 1 1 1 1 +1 208 208 1 1 0 1 0 0 0 0 +1 209 209 1 1 0 1 0 0 0 1 +1 210 210 1 1 0 1 0 0 1 0 +1 211 211 1 1 0 1 0 0 1 1 +1 212 212 1 1 0 1 0 1 0 0 +1 213 213 1 1 0 1 0 1 0 1 +1 214 214 1 1 0 1 0 1 1 0 +1 215 215 1 1 0 1 0 1 1 1 +1 216 216 1 1 0 1 1 0 0 0 +1 217 217 1 1 0 1 1 0 0 1 +1 218 218 1 1 0 1 1 0 1 0 +1 219 219 1 1 0 1 1 0 1 1 +1 220 220 1 1 0 1 1 1 0 0 +1 221 221 1 1 0 1 1 1 0 1 +1 222 222 1 1 0 1 1 1 1 0 +1 223 223 1 1 0 1 1 1 1 1 +1 224 224 1 1 1 0 0 0 0 0 +1 225 225 1 1 1 0 0 0 0 1 +1 226 226 1 1 1 0 0 0 1 0 +1 227 227 1 1 1 0 0 0 1 1 +1 228 228 1 1 1 0 0 1 0 0 +1 229 229 1 1 1 0 0 1 0 1 +1 230 230 1 1 1 0 0 1 1 0 +1 231 231 1 1 1 0 0 1 1 1 +1 232 232 1 1 1 0 1 0 0 0 +1 233 233 1 1 1 0 1 0 0 1 +1 234 234 1 1 1 0 1 0 1 0 +1 235 235 1 1 1 0 1 0 1 1 +1 236 236 1 1 1 0 1 1 0 0 +1 237 237 1 1 1 0 1 1 0 1 +1 238 238 1 1 1 0 1 1 1 0 +1 239 239 1 1 1 0 1 1 1 1 +1 240 240 1 1 1 1 0 0 0 0 +1 241 241 1 1 1 1 0 0 0 1 +1 242 242 1 1 1 1 0 0 1 0 +1 243 243 1 1 1 1 0 0 1 1 +1 244 244 1 1 1 1 0 1 0 0 +1 245 245 1 1 1 1 0 1 0 1 +1 246 246 1 1 1 1 0 1 1 0 +1 247 247 1 1 1 1 0 1 1 1 +1 248 248 1 1 1 1 1 0 0 0 +1 249 249 1 1 1 1 1 0 0 1 +1 250 250 1 1 1 1 1 0 1 0 +1 251 251 1 1 1 1 1 0 1 1 +1 252 252 1 1 1 1 1 1 0 0 +1 253 253 1 1 1 1 1 1 0 1 +1 254 254 1 1 1 1 1 1 1 0 +1 255 255 1 1 1 1 1 1 1 1 +1 0 0 0 0 0 0 0 0 0 0 +1 1 1 0 0 0 0 0 0 0 1 +1 2 2 0 0 0 0 0 0 1 0 +1 3 3 0 0 0 0 0 0 1 1 +1 4 4 0 0 0 0 0 1 0 0 +1 5 5 0 0 0 0 0 1 0 1 +1 6 6 0 0 0 0 0 1 1 0 +1 7 7 0 0 0 0 0 1 1 1 +1 8 8 0 0 0 0 1 0 0 0 +1 9 9 0 0 0 0 1 0 0 1 +1 10 10 0 0 0 0 1 0 1 0 +1 11 11 0 0 0 0 1 0 1 1 +1 12 12 0 0 0 0 1 1 0 0 +1 13 13 0 0 0 0 1 1 0 1 +1 14 14 0 0 0 0 1 1 1 0 +1 15 15 0 0 0 0 1 1 1 1 +1 16 16 0 0 0 1 0 0 0 0 +1 17 17 0 0 0 1 0 0 0 1 +1 18 18 0 0 0 1 0 0 1 0 +1 19 19 0 0 0 1 0 0 1 1 +1 20 20 0 0 0 1 0 1 0 0 +1 21 21 0 0 0 1 0 1 0 1 +1 22 22 0 0 0 1 0 1 1 0 +1 23 23 0 0 0 1 0 1 1 1 +1 24 24 0 0 0 1 1 0 0 0 +1 25 25 0 0 0 1 1 0 0 1 +1 26 26 0 0 0 1 1 0 1 0 +1 27 27 0 0 0 1 1 0 1 1 +1 28 28 0 0 0 1 1 1 0 0 +1 29 29 0 0 0 1 1 1 0 1 +1 30 30 0 0 0 1 1 1 1 0 +1 31 31 0 0 0 1 1 1 1 1 +1 32 32 0 0 1 0 0 0 0 0 +1 33 33 0 0 1 0 0 0 0 1 +1 34 34 0 0 1 0 0 0 1 0 +1 35 35 0 0 1 0 0 0 1 1 +1 36 36 0 0 1 0 0 1 0 0 +1 37 37 0 0 1 0 0 1 0 1 +1 38 38 0 0 1 0 0 1 1 0 +1 39 39 0 0 1 0 0 1 1 1 +1 40 40 0 0 1 0 1 0 0 0 +1 41 41 0 0 1 0 1 0 0 1 +1 42 42 0 0 1 0 1 0 1 0 +1 43 43 0 0 1 0 1 0 1 1 +1 44 44 0 0 1 0 1 1 0 0 +1 45 45 0 0 1 0 1 1 0 1 +1 46 46 0 0 1 0 1 1 1 0 +1 47 47 0 0 1 0 1 1 1 1 +1 48 48 0 0 1 1 0 0 0 0 +1 49 49 0 0 1 1 0 0 0 1 +1 50 50 0 0 1 1 0 0 1 0 +1 51 51 0 0 1 1 0 0 1 1 +1 52 52 0 0 1 1 0 1 0 0 +1 53 53 0 0 1 1 0 1 0 1 +1 54 54 0 0 1 1 0 1 1 0 +1 55 55 0 0 1 1 0 1 1 1 +1 56 56 0 0 1 1 1 0 0 0 +1 57 57 0 0 1 1 1 0 0 1 +1 58 58 0 0 1 1 1 0 1 0 +1 59 59 0 0 1 1 1 0 1 1 +1 60 60 0 0 1 1 1 1 0 0 +1 61 61 0 0 1 1 1 1 0 1 +1 62 62 0 0 1 1 1 1 1 0 +1 63 63 0 0 1 1 1 1 1 1 +1 64 64 0 1 0 0 0 0 0 0 +1 65 65 0 1 0 0 0 0 0 1 +1 66 66 0 1 0 0 0 0 1 0 +1 67 67 0 1 0 0 0 0 1 1 +1 68 68 0 1 0 0 0 1 0 0 +1 69 69 0 1 0 0 0 1 0 1 +1 70 70 0 1 0 0 0 1 1 0 +1 71 71 0 1 0 0 0 1 1 1 +1 72 72 0 1 0 0 1 0 0 0 +1 73 73 0 1 0 0 1 0 0 1 +1 74 74 0 1 0 0 1 0 1 0 +1 75 75 0 1 0 0 1 0 1 1 +1 76 76 0 1 0 0 1 1 0 0 +1 77 77 0 1 0 0 1 1 0 1 +1 78 78 0 1 0 0 1 1 1 0 +1 79 79 0 1 0 0 1 1 1 1 +1 80 80 0 1 0 1 0 0 0 0 +1 81 81 0 1 0 1 0 0 0 1 +1 82 82 0 1 0 1 0 0 1 0 +1 83 83 0 1 0 1 0 0 1 1 +1 84 84 0 1 0 1 0 1 0 0 +1 85 85 0 1 0 1 0 1 0 1 +1 86 86 0 1 0 1 0 1 1 0 +1 87 87 0 1 0 1 0 1 1 1 +1 88 88 0 1 0 1 1 0 0 0 +1 89 89 0 1 0 1 1 0 0 1 +1 90 90 0 1 0 1 1 0 1 0 +1 91 91 0 1 0 1 1 0 1 1 +1 92 92 0 1 0 1 1 1 0 0 +1 93 93 0 1 0 1 1 1 0 1 +1 94 94 0 1 0 1 1 1 1 0 +1 95 95 0 1 0 1 1 1 1 1 +1 96 96 0 1 1 0 0 0 0 0 +1 97 97 0 1 1 0 0 0 0 1 +1 98 98 0 1 1 0 0 0 1 0 +1 99 99 0 1 1 0 0 0 1 1 +1 100 100 0 1 1 0 0 1 0 0 +1 101 101 0 1 1 0 0 1 0 1 +1 102 102 0 1 1 0 0 1 1 0 +1 103 103 0 1 1 0 0 1 1 1 +1 104 104 0 1 1 0 1 0 0 0 +1 105 105 0 1 1 0 1 0 0 1 +1 106 106 0 1 1 0 1 0 1 0 +1 107 107 0 1 1 0 1 0 1 1 +1 108 108 0 1 1 0 1 1 0 0 +1 109 109 0 1 1 0 1 1 0 1 +1 110 110 0 1 1 0 1 1 1 0 +1 111 111 0 1 1 0 1 1 1 1 +1 112 112 0 1 1 1 0 0 0 0 +1 113 113 0 1 1 1 0 0 0 1 +1 114 114 0 1 1 1 0 0 1 0 +1 115 115 0 1 1 1 0 0 1 1 +1 116 116 0 1 1 1 0 1 0 0 +1 117 117 0 1 1 1 0 1 0 1 +1 118 118 0 1 1 1 0 1 1 0 +1 119 119 0 1 1 1 0 1 1 1 +1 120 120 0 1 1 1 1 0 0 0 +1 121 121 0 1 1 1 1 0 0 1 +1 122 122 0 1 1 1 1 0 1 0 +1 123 123 0 1 1 1 1 0 1 1 +1 124 124 0 1 1 1 1 1 0 0 +1 125 125 0 1 1 1 1 1 0 1 +1 126 126 0 1 1 1 1 1 1 0 +1 127 127 0 1 1 1 1 1 1 1 +1 128 128 1 0 0 0 0 0 0 0 +1 129 129 1 0 0 0 0 0 0 1 +1 130 130 1 0 0 0 0 0 1 0 +1 131 131 1 0 0 0 0 0 1 1 +1 132 132 1 0 0 0 0 1 0 0 +1 133 133 1 0 0 0 0 1 0 1 +1 134 134 1 0 0 0 0 1 1 0 +1 135 135 1 0 0 0 0 1 1 1 +1 136 136 1 0 0 0 1 0 0 0 +1 137 137 1 0 0 0 1 0 0 1 +1 138 138 1 0 0 0 1 0 1 0 +1 139 139 1 0 0 0 1 0 1 1 +1 140 140 1 0 0 0 1 1 0 0 +1 141 141 1 0 0 0 1 1 0 1 +1 142 142 1 0 0 0 1 1 1 0 +1 143 143 1 0 0 0 1 1 1 1 +1 144 144 1 0 0 1 0 0 0 0 +1 145 145 1 0 0 1 0 0 0 1 +1 146 146 1 0 0 1 0 0 1 0 +1 147 147 1 0 0 1 0 0 1 1 +1 148 148 1 0 0 1 0 1 0 0 +1 149 149 1 0 0 1 0 1 0 1 +1 150 150 1 0 0 1 0 1 1 0 +1 151 151 1 0 0 1 0 1 1 1 +1 152 152 1 0 0 1 1 0 0 0 +1 153 153 1 0 0 1 1 0 0 1 +1 154 154 1 0 0 1 1 0 1 0 +1 155 155 1 0 0 1 1 0 1 1 +1 156 156 1 0 0 1 1 1 0 0 +1 157 157 1 0 0 1 1 1 0 1 +1 158 158 1 0 0 1 1 1 1 0 +1 159 159 1 0 0 1 1 1 1 1 +1 160 160 1 0 1 0 0 0 0 0 +1 161 161 1 0 1 0 0 0 0 1 +1 162 162 1 0 1 0 0 0 1 0 +1 163 163 1 0 1 0 0 0 1 1 +1 164 164 1 0 1 0 0 1 0 0 +1 165 165 1 0 1 0 0 1 0 1 +1 166 166 1 0 1 0 0 1 1 0 +1 167 167 1 0 1 0 0 1 1 1 +1 168 168 1 0 1 0 1 0 0 0 +1 169 169 1 0 1 0 1 0 0 1 +1 170 170 1 0 1 0 1 0 1 0 +1 171 171 1 0 1 0 1 0 1 1 +1 172 172 1 0 1 0 1 1 0 0 +1 173 173 1 0 1 0 1 1 0 1 +1 174 174 1 0 1 0 1 1 1 0 +1 175 175 1 0 1 0 1 1 1 1 +1 176 176 1 0 1 1 0 0 0 0 +1 177 177 1 0 1 1 0 0 0 1 +1 178 178 1 0 1 1 0 0 1 0 +1 179 179 1 0 1 1 0 0 1 1 +1 180 180 1 0 1 1 0 1 0 0 +1 181 181 1 0 1 1 0 1 0 1 +1 182 182 1 0 1 1 0 1 1 0 +1 183 183 1 0 1 1 0 1 1 1 +1 184 184 1 0 1 1 1 0 0 0 +1 185 185 1 0 1 1 1 0 0 1 +1 186 186 1 0 1 1 1 0 1 0 +1 187 187 1 0 1 1 1 0 1 1 +1 188 188 1 0 1 1 1 1 0 0 +1 189 189 1 0 1 1 1 1 0 1 +1 190 190 1 0 1 1 1 1 1 0 +1 191 191 1 0 1 1 1 1 1 1 +1 192 192 1 1 0 0 0 0 0 0 +1 193 193 1 1 0 0 0 0 0 1 +1 194 194 1 1 0 0 0 0 1 0 +1 195 195 1 1 0 0 0 0 1 1 +1 196 196 1 1 0 0 0 1 0 0 +1 197 197 1 1 0 0 0 1 0 1 +1 198 198 1 1 0 0 0 1 1 0 +1 199 199 1 1 0 0 0 1 1 1 +1 200 200 1 1 0 0 1 0 0 0 +1 201 201 1 1 0 0 1 0 0 1 +1 202 202 1 1 0 0 1 0 1 0 +1 203 203 1 1 0 0 1 0 1 1 +1 204 204 1 1 0 0 1 1 0 0 +1 205 205 1 1 0 0 1 1 0 1 +1 206 206 1 1 0 0 1 1 1 0 +1 207 207 1 1 0 0 1 1 1 1 +1 208 208 1 1 0 1 0 0 0 0 +1 209 209 1 1 0 1 0 0 0 1 +1 210 210 1 1 0 1 0 0 1 0 +1 211 211 1 1 0 1 0 0 1 1 +1 212 212 1 1 0 1 0 1 0 0 +1 213 213 1 1 0 1 0 1 0 1 +1 214 214 1 1 0 1 0 1 1 0 +1 215 215 1 1 0 1 0 1 1 1 +1 216 216 1 1 0 1 1 0 0 0 +1 217 217 1 1 0 1 1 0 0 1 +1 218 218 1 1 0 1 1 0 1 0 +1 219 219 1 1 0 1 1 0 1 1 +1 220 220 1 1 0 1 1 1 0 0 +1 221 221 1 1 0 1 1 1 0 1 +1 222 222 1 1 0 1 1 1 1 0 +1 223 223 1 1 0 1 1 1 1 1 +1 224 224 1 1 1 0 0 0 0 0 +1 225 225 1 1 1 0 0 0 0 1 +1 226 226 1 1 1 0 0 0 1 0 +1 227 227 1 1 1 0 0 0 1 1 +1 228 228 1 1 1 0 0 1 0 0 +1 229 229 1 1 1 0 0 1 0 1 +1 230 230 1 1 1 0 0 1 1 0 +1 231 231 1 1 1 0 0 1 1 1 +1 232 232 1 1 1 0 1 0 0 0 +1 233 233 1 1 1 0 1 0 0 1 +1 234 234 1 1 1 0 1 0 1 0 +1 235 235 1 1 1 0 1 0 1 1 +1 236 236 1 1 1 0 1 1 0 0 +1 237 237 1 1 1 0 1 1 0 1 +1 238 238 1 1 1 0 1 1 1 0 +1 239 239 1 1 1 0 1 1 1 1 +1 240 240 1 1 1 1 0 0 0 0 +1 241 241 1 1 1 1 0 0 0 1 +1 242 242 1 1 1 1 0 0 1 0 +1 243 243 1 1 1 1 0 0 1 1 +1 244 244 1 1 1 1 0 1 0 0 +1 245 245 1 1 1 1 0 1 0 1 +1 246 246 1 1 1 1 0 1 1 0 +1 247 247 1 1 1 1 0 1 1 1 +1 248 248 1 1 1 1 1 0 0 0 +1 249 249 1 1 1 1 1 0 0 1 +1 250 250 1 1 1 1 1 0 1 0 +1 251 251 1 1 1 1 1 0 1 1 +1 252 252 1 1 1 1 1 1 0 0 +1 253 253 1 1 1 1 1 1 0 1 +1 254 254 1 1 1 1 1 1 1 0 +1 255 255 1 1 1 1 1 1 1 1 diff --git a/dbms/tests/queries/0_stateless/00216_bit_test_function_family.sql b/dbms/tests/queries/0_stateless/00216_bit_test_function_family.sql new file mode 100644 index 00000000000..a8bbc379087 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00216_bit_test_function_family.sql @@ -0,0 +1,115 @@ +select + bitTest(0, 0) = 0, + bitTest(1, 0) = 1, + bitTest(1, 1) = 0, + bitTest(0xff, 7) = 1; + +select + bitTestAll(0, 0) = 0, + bitTestAll(1, 0) = 1, + bitTestAll(1, 1) = 0, + bitTestAll(0xff, 0) = 1, + bitTestAll(0xff, 1) = 1, + bitTestAll(0xff, 2) = 1, + bitTestAll(0xff, 3) = 1, + bitTestAll(0xff, 4) = 1, + bitTestAll(0xff, 5) = 1, + bitTestAll(0xff, 6) = 1, + bitTestAll(0xff, 7) = 1, + bitTestAll(0xff, 0, 1) = 1, + bitTestAll(0xff, 2, 3) = 1, + bitTestAll(0xff, 4, 5) = 1, + bitTestAll(0xff, 6, 7) = 1, + bitTestAll(0xff, 0, 1, 2, 3) = 1, + bitTestAll(0xff, 4, 5, 6, 7) = 1, + bitTestAll(0xff, 0, 1, 2, 3, 4, 5, 6, 7) = 1, + bitTestAll(0x81, 0) = 1, + bitTestAll(0x81, 1) = 0, + bitTestAll(0x81, 2) = 0, + bitTestAll(0x81, 3) = 0, + bitTestAll(0x81, 4) = 0, + bitTestAll(0x81, 5) = 0, + bitTestAll(0x81, 6) = 0, + bitTestAll(0x81, 7) = 1, + bitTestAll(0x81, 0, 1) = 0, + bitTestAll(0x81, 2, 3) = 0, + bitTestAll(0x81, 4, 5) = 0, + bitTestAll(0x81, 6, 7) = 0, + bitTestAll(0x81, 0, 1, 2, 3) = 0, + bitTestAll(0x81, 4, 5, 6, 7) = 0, + bitTestAll(0x81, 0, 1, 2, 3, 4, 5, 6, 7) = 0, + bitTestAll(0x81, 0, 7) = 1; + +select + bitTestAny(0, 0) = 0, + bitTestAny(1, 0) = 1, + bitTestAny(1, 1) = 0, + bitTestAny(0xff, 0) = 1, + bitTestAny(0xff, 1) = 1, + bitTestAny(0xff, 2) = 1, + bitTestAny(0xff, 3) = 1, + bitTestAny(0xff, 4) = 1, + bitTestAny(0xff, 5) = 1, + bitTestAny(0xff, 6) = 1, + bitTestAny(0xff, 7) = 1, + bitTestAny(0xff, 0, 1) = 1, + bitTestAny(0xff, 2, 3) = 1, + bitTestAny(0xff, 4, 5) = 1, + bitTestAny(0xff, 6, 7) = 1, + bitTestAny(0xff, 0, 1, 2, 3) = 1, + bitTestAny(0xff, 4, 5, 6, 7) = 1, + bitTestAny(0xff, 0, 1, 2, 3, 4, 5, 6, 7) = 1, + bitTestAny(0x81, 0) = 1, + bitTestAny(0x81, 1) = 0, + bitTestAny(0x81, 2) = 0, + bitTestAny(0x81, 3) = 0, + bitTestAny(0x81, 4) = 0, + bitTestAny(0x81, 5) = 0, + bitTestAny(0x81, 6) = 0, + bitTestAny(0x81, 7) = 1, + bitTestAny(0x81, 0, 1) = 1, + bitTestAny(0x81, 2, 3) = 0, + bitTestAny(0x81, 4, 5) = 0, + bitTestAny(0x81, 6, 7) = 1, + bitTestAny(0x81, 0, 1, 2, 3) = 1, + bitTestAny(0x81, 4, 5, 6, 7) = 1, + bitTestAny(0x81, 0, 1, 2, 3, 4, 5, 6, 7) = 1; + +select n = n_, + number as n, + bitOr(bitShiftLeft(bitOr(bitShiftLeft(bitOr(bitShiftLeft(bitOr(bitShiftLeft(bitOr(bitShiftLeft(bitOr(bitShiftLeft(bitOr(bitShiftLeft(b7, 1), b6), 1), b5), 1), b4), 1), b3), 1), b2), 1), b1), 1), b0) as n_, + bitTest(n, 7) as b7, + bitTest(n, 6) as b6, + bitTest(n, 5) as b5, + bitTest(n, 4) as b4, + bitTest(n, 3) as b3, + bitTest(n, 2) as b2, + bitTest(n, 1) as b1, + bitTest(n, 0) as b0 +from system.numbers limit 256; + +select n = n_, + number as n, + bitOr(bitShiftLeft(bitOr(bitShiftLeft(bitOr(bitShiftLeft(bitOr(bitShiftLeft(bitOr(bitShiftLeft(bitOr(bitShiftLeft(bitOr(bitShiftLeft(b7, 1), b6), 1), b5), 1), b4), 1), b3), 1), b2), 1), b1), 1), b0) as n_, + bitTestAll(n, 7) as b7, + bitTestAll(n, 6) as b6, + bitTestAll(n, 5) as b5, + bitTestAll(n, 4) as b4, + bitTestAll(n, 3) as b3, + bitTestAll(n, 2) as b2, + bitTestAll(n, 1) as b1, + bitTestAll(n, 0) as b0 +from system.numbers limit 256; + +select n = n_, + number as n, + bitOr(bitShiftLeft(bitOr(bitShiftLeft(bitOr(bitShiftLeft(bitOr(bitShiftLeft(bitOr(bitShiftLeft(bitOr(bitShiftLeft(bitOr(bitShiftLeft(b7, 1), b6), 1), b5), 1), b4), 1), b3), 1), b2), 1), b1), 1), b0) as n_, + bitTestAny(n, 7) as b7, + bitTestAny(n, 6) as b6, + bitTestAny(n, 5) as b5, + bitTestAny(n, 4) as b4, + bitTestAny(n, 3) as b3, + bitTestAny(n, 2) as b2, + bitTestAny(n, 1) as b1, + bitTestAny(n, 0) as b0 +from system.numbers limit 256; diff --git a/libs/libcommon/src/create_revision.sh.cmake b/libs/libcommon/src/create_revision.sh.cmake index a6acfad3f1d..7da01ee5b95 100644 --- a/libs/libcommon/src/create_revision.sh.cmake +++ b/libs/libcommon/src/create_revision.sh.cmake @@ -15,6 +15,12 @@ revision=$(git tag --points-at HEAD 2> /dev/null | tail -1) if [[ "$revision" = "" ]]; then revision=$( ( git describe --tags || echo 1 ) | cut -d "-" -f 1 ) fi + +is_it_github=$( git config --get remote.origin.url | grep 'github' ) +if [[ "$is_it_github" = "" ]]; then + revision=53190 +fi + echo $revision >> "${CMAKE_CURRENT_BINARY_DIR}/src/revision.h"; echo "#endif" >> "${CMAKE_CURRENT_BINARY_DIR}/src/revision.h"