mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-23 08:02:02 +00:00
Merge
This commit is contained in:
commit
10b42e1b95
@ -15,6 +15,7 @@
|
||||
#include <DB/Columns/ColumnArray.h>
|
||||
|
||||
#include <stats/IntHash.h>
|
||||
#include <statdaemons/ext/range.hpp>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -234,9 +235,16 @@ namespace detail
|
||||
template <typename ResultType>
|
||||
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<size_t>{}, 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<ColumnFloat32 &>(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]);
|
||||
}
|
||||
};
|
||||
|
@ -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<AggregateFunctionSequenceMatchData>
|
||||
class AggregateFunctionSequenceMatch : public IAggregateFunctionHelper<AggregateFunctionSequenceMatchData>
|
||||
{
|
||||
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 &>(data(place)).sort();
|
||||
static_cast<ColumnUInt8 &>(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<ColumnUInt8 &>(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<PatternActionType, std::string> 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 <typename T1, typename T2>
|
||||
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<decltype(action_it), decltype(events_it), decltype(base_it)>;
|
||||
@ -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 &>(data(place)).sort();
|
||||
static_cast<ColumnUInt64 &>(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;
|
||||
}
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -1379,4 +1379,360 @@ public:
|
||||
}
|
||||
};
|
||||
|
||||
namespace
|
||||
{
|
||||
template <typename T1, typename T2> 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<const DataTypeUInt8 *>(first_arg) &&
|
||||
!typeid_cast<const DataTypeUInt16 *>(first_arg) &&
|
||||
!typeid_cast<const DataTypeUInt32 *>(first_arg) &&
|
||||
!typeid_cast<const DataTypeUInt64 *>(first_arg) &&
|
||||
!typeid_cast<const DataTypeInt8 *>(first_arg) &&
|
||||
!typeid_cast<const DataTypeInt16 *>(first_arg) &&
|
||||
!typeid_cast<const DataTypeInt32 *>(first_arg) &&
|
||||
!typeid_cast<const DataTypeInt64 *>(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<const DataTypeUInt8 *>(second_arg) &&
|
||||
!typeid_cast<const DataTypeUInt16 *>(second_arg) &&
|
||||
!typeid_cast<const DataTypeUInt32 *>(second_arg) &&
|
||||
!typeid_cast<const DataTypeUInt64 *>(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<UInt8>(block, arguments, result, value_col) &&
|
||||
!execute<UInt16>(block, arguments, result, value_col) &&
|
||||
!execute<UInt32>(block, arguments, result, value_col) &&
|
||||
!execute<UInt64>(block, arguments, result, value_col) &&
|
||||
!execute<Int8>(block, arguments, result, value_col) &&
|
||||
!execute<Int16>(block, arguments, result, value_col) &&
|
||||
!execute<Int32>(block, arguments, result, value_col) &&
|
||||
!execute<Int64>(block, arguments, result, value_col))
|
||||
throw Exception{
|
||||
"Illegal column " + value_col->getName() + " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN
|
||||
};
|
||||
}
|
||||
|
||||
private:
|
||||
template <typename T> bool execute(
|
||||
Block & block, const ColumnNumbers & arguments, const size_t result,
|
||||
const IColumn * const value_col_untyped)
|
||||
{
|
||||
if (const auto value_col = typeid_cast<const ColumnVector<T> *>(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<const ColumnConst<T> *>(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 <typename ValueColumn> bool execute(
|
||||
Block & block, const ColumnNumbers & arguments, const size_t result,
|
||||
const ValueColumn * const value_col, const IColumn * const pos_col)
|
||||
{
|
||||
return execute<UInt8>(block, arguments, result, value_col, pos_col) ||
|
||||
execute<UInt16>(block, arguments, result, value_col, pos_col) ||
|
||||
execute<UInt32>(block, arguments, result, value_col, pos_col) ||
|
||||
execute<UInt64>(block, arguments, result, value_col, pos_col);
|
||||
}
|
||||
|
||||
template <typename T, typename ValueType> bool execute(
|
||||
Block & block, const ColumnNumbers & arguments, const size_t result,
|
||||
const ColumnVector<ValueType> * const value_col, const IColumn * const pos_col_untyped)
|
||||
{
|
||||
if (const auto pos_col = typeid_cast<const ColumnVector<T> *>(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<UInt8>(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<const ColumnConst<T> *>(pos_col_untyped))
|
||||
{
|
||||
const auto & values = value_col->getData();
|
||||
|
||||
const auto size = value_col->size();
|
||||
const auto out_col = new ColumnVector<UInt8>(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 <typename T, typename ValueType> bool execute(
|
||||
Block & block, const ColumnNumbers & arguments, const size_t result,
|
||||
const ColumnConst<ValueType> * const value_col, const IColumn * const pos_col_untyped)
|
||||
{
|
||||
if (const auto pos_col = typeid_cast<const ColumnVector<T> *>(pos_col_untyped))
|
||||
{
|
||||
const auto & positions = pos_col->getData();
|
||||
|
||||
const auto size = value_col->size();
|
||||
const auto out_col = new ColumnVector<UInt8>(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<const ColumnConst<T> *>(pos_col_untyped))
|
||||
{
|
||||
block.getByPosition(result).column = new ColumnConst<UInt8>{
|
||||
value_col->size(),
|
||||
bitTest(value_col->getData(), pos_col->getData())
|
||||
};
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
return false;
|
||||
}
|
||||
};
|
||||
|
||||
template <typename Impl>
|
||||
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<const DataTypeUInt8 *>(first_arg) && !typeid_cast<const DataTypeUInt16 *>(first_arg) &&
|
||||
!typeid_cast<const DataTypeUInt32 *>(first_arg) && !typeid_cast<const DataTypeUInt64 *>(first_arg) &&
|
||||
!typeid_cast<const DataTypeInt8 *>(first_arg) && !typeid_cast<const DataTypeInt16 *>(first_arg) &&
|
||||
!typeid_cast<const DataTypeInt32 *>(first_arg) && !typeid_cast<const DataTypeInt64 *>(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<const DataTypeUInt8 *>(pos_arg) && !typeid_cast<const DataTypeUInt16 *>(pos_arg) &&
|
||||
!typeid_cast<const DataTypeUInt32 *>(pos_arg) && !typeid_cast<const DataTypeUInt64 *>(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<UInt8>(block, arguments, result, value_col) &&
|
||||
!execute<UInt16>(block, arguments, result, value_col) &&
|
||||
!execute<UInt32>(block, arguments, result, value_col) &&
|
||||
!execute<UInt64>(block, arguments, result, value_col) &&
|
||||
!execute<Int8>(block, arguments, result, value_col) &&
|
||||
!execute<Int16>(block, arguments, result, value_col) &&
|
||||
!execute<Int32>(block, arguments, result, value_col) &&
|
||||
!execute<Int64>(block, arguments, result, value_col))
|
||||
throw Exception{
|
||||
"Illegal column " + value_col->getName() + " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN
|
||||
};
|
||||
}
|
||||
|
||||
private:
|
||||
template <typename T> bool execute(
|
||||
Block & block, const ColumnNumbers & arguments, const size_t result,
|
||||
const IColumn * const value_col_untyped)
|
||||
{
|
||||
if (const auto value_col = typeid_cast<const ColumnVector<T> *>(value_col_untyped))
|
||||
{
|
||||
const auto size = value_col->size();
|
||||
const auto & mask = createMask<T>(size, block, arguments);
|
||||
const auto & val = value_col->getData();
|
||||
|
||||
const auto out_col = new ColumnVector<UInt8>(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<const ColumnConst<T> *>(value_col_untyped))
|
||||
{
|
||||
const auto size = value_col->size();
|
||||
const auto & mask = createMask<T>(size, block, arguments);
|
||||
const auto & val = value_col->getData();
|
||||
|
||||
const auto out_col = new ColumnVector<UInt8>(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 <typename T>
|
||||
PODArray<T> createMask(const std::size_t size, const Block & block, const ColumnNumbers & arguments)
|
||||
{
|
||||
PODArray<T> mask(size, T{});
|
||||
|
||||
for (const auto i : ext::range(1, arguments.size()))
|
||||
addToMask(mask, block.getByPosition(arguments[i]).column.get());
|
||||
|
||||
return mask;
|
||||
}
|
||||
|
||||
template <typename ValueType>
|
||||
void addToMask(PODArray<ValueType> & mask, const IColumn * const pos_col)
|
||||
{
|
||||
if (!addToMaskImpl<UInt8>(mask, pos_col) && !addToMaskImpl<UInt16>(mask, pos_col) &&
|
||||
!addToMaskImpl<UInt32>(mask, pos_col) && !addToMaskImpl<UInt64>(mask, pos_col))
|
||||
throw Exception{
|
||||
"Illegal column " + pos_col->getName() + " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN
|
||||
};
|
||||
}
|
||||
|
||||
template <typename PosType, typename ValueType>
|
||||
bool addToMaskImpl(PODArray<ValueType> & mask, const IColumn * const pos_col_untyped)
|
||||
{
|
||||
if (const auto pos_col = typeid_cast<const ColumnVector<PosType> *>(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<const ColumnConst<PosType> *>(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 <typename T> static UInt8 combine(const T val, const T mask) { return (val & mask) != 0; }
|
||||
};
|
||||
|
||||
struct BitTestAllImpl
|
||||
{
|
||||
static constexpr auto name = "bitTestAll";
|
||||
template <typename T> static UInt8 combine(const T val, const T mask) { return (val & mask) == mask; }
|
||||
};
|
||||
|
||||
using FunctionBitTestAny = FunctionBitTestMany<BitTestAnyImpl>;
|
||||
using FunctionBitTestAll = FunctionBitTestMany<BitTestAllImpl>;
|
||||
|
||||
|
||||
}
|
||||
|
@ -1462,15 +1462,6 @@ private:
|
||||
return true;
|
||||
}
|
||||
|
||||
template <typename T, typename DictionaryType>
|
||||
bool execute(Block & block, const size_t result, const DictionaryType * const dictionary,
|
||||
const IColumn * const id_col_untyped)
|
||||
{
|
||||
|
||||
|
||||
return false;
|
||||
}
|
||||
|
||||
const ExternalDictionaries & dictionaries;
|
||||
};
|
||||
|
||||
|
@ -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",
|
||||
|
@ -15,6 +15,9 @@ void registerFunctionsCoding(FunctionFactory & factory)
|
||||
factory.registerFunction<FunctionHex>();
|
||||
factory.registerFunction<FunctionUnhex>();
|
||||
factory.registerFunction<FunctionBitmaskToArray>();
|
||||
factory.registerFunction<FunctionBitTest>();
|
||||
factory.registerFunction<FunctionBitTestAny>();
|
||||
factory.registerFunction<FunctionBitTestAll>();
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -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<size_t> sum_marks_in_parts(parts.size());
|
||||
size_t sum_marks = 0;
|
||||
|
@ -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
|
@ -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;
|
Loading…
Reference in New Issue
Block a user