Skip to content

Commit

Permalink
Merge pull request #25394 from ClickHouse/function-bit-position-to-ar…
Browse files Browse the repository at this point in the history
…ray-merge

Merging #23843
  • Loading branch information
kitaisreal authored Jun 17, 2021
2 parents 2c4c268 + ca672b0 commit b34b66c
Show file tree
Hide file tree
Showing 5 changed files with 261 additions and 2 deletions.
52 changes: 50 additions & 2 deletions docs/en/sql-reference/functions/encoding-functions.md
Original file line number Diff line number Diff line change
Expand Up @@ -6,7 +6,7 @@ toc_title: Encoding
# Encoding Functions {#encoding-functions}

## char {#char}

Returns the string with the length as the number of passed arguments and each byte has the value of corresponding argument. Accepts multiple arguments of numeric types. If the value of argument is out of range of UInt8 data type, it is converted to UInt8 with possible rounding and overflow.

**Syntax**
Expand Down Expand Up @@ -156,7 +156,7 @@ Performs the opposite operation of [hex](#hex). It interprets each pair of hexad
If you want to convert the result to a number, you can use the [reverse](../../sql-reference/functions/string-functions.md#reverse) and [reinterpretAs<Type>](../../sql-reference/functions/type-conversion-functions.md#type-conversion-functions) functions.

!!! note "Note"
If `unhex` is invoked from within the `clickhouse-client`, binary strings display using UTF-8.
If `unhex` is invoked from within the `clickhouse-client`, binary strings display using UTF-8.

Alias: `UNHEX`.

Expand Down Expand Up @@ -221,3 +221,51 @@ Accepts an integer. Returns a string containing the list of powers of two that t
## bitmaskToArray(num) {#bitmasktoarraynum}

Accepts an integer. Returns an array of UInt64 numbers containing the list of powers of two that total the source number when summed. Numbers in the array are in ascending order.

## bitPositionsToArray(num) {#bitpositionstoarraynum}

Accepts an integer, argument will be converted to unsigned integer type. Returns an array of UInt64 numbers containing the list of positions of bits that equals 1. Numbers in the array are in ascending order.

**Syntax**

```sql
bitPositionsToArray(arg)
```

**Arguments**

- `arg` — Integer value.Types: [Int/UInt](../../sql-reference/data-types/int-uint.md)

**Returned value**

An array of UInt64 numbers containing the list of positions of bits that equals 1. Numbers in the array are in ascending order.

**Example**

Query:

``` sql
SELECT bitPositionsToArray(toInt8(1)) AS bit_positions;
```

Result:

``` text
┌─bit_positions─┐
│ [0] │
└───────────────┘
```

Query:

``` sql
select bitPositionsToArray(toInt8(-1)) as bit_positions;
```

Result:

``` text
┌─bit_positions─────┐
│ [0,1,2,3,4,5,6,7] │
└───────────────────┘
```
1 change: 1 addition & 0 deletions src/Functions/FunctionsCoding.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@ void registerFunctionsCoding(FunctionFactory & factory)
factory.registerFunction<FunctionUnhex>(FunctionFactory::CaseInsensitive);
factory.registerFunction<FunctionChar>(FunctionFactory::CaseInsensitive);
factory.registerFunction<FunctionBitmaskToArray>();
factory.registerFunction<FunctionBitPositionsToArray>();
factory.registerFunction<FunctionToIPv4>();
factory.registerFunction<FunctionToIPv6>();
factory.registerFunction<FunctionIPv6CIDRToRange>();
Expand Down
111 changes: 111 additions & 0 deletions src/Functions/FunctionsCoding.h
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@
#include <Common/formatIPv6.h>
#include <Common/hex.h>
#include <Common/typeid_cast.h>
#include <Common/BitHelpers.h>

#include <arpa/inet.h>
#include <common/range.h>
Expand Down Expand Up @@ -1506,6 +1507,116 @@ class FunctionBitmaskToArray : public IFunction
}
};

class FunctionBitPositionsToArray : public IFunction
{
public:
static constexpr auto name = "bitPositionsToArray";
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionBitPositionsToArray>(); }

String getName() const override
{
return name;
}

size_t getNumberOfArguments() const override { return 1; }
bool isInjective(const ColumnsWithTypeAndName &) const override { return true; }

DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (!isInteger(arguments[0]))
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Illegal type {} of argument of function {}",
getName(),
arguments[0]->getName());

return std::make_shared<DataTypeArray>(std::make_shared<DataTypeUInt64>());
}

bool useDefaultImplementationForConstants() const override { return true; }

template <typename T>
ColumnPtr executeType(const IColumn * column) const
{
const ColumnVector<T> * col_from = checkAndGetColumn<ColumnVector<T>>(column);
if (!col_from)
return nullptr;

auto result_array_values = ColumnVector<UInt64>::create();
auto result_array_offsets = ColumnArray::ColumnOffsets::create();

auto & result_array_values_data = result_array_values->getData();
auto & result_array_offsets_data = result_array_offsets->getData();

auto & vec_from = col_from->getData();
size_t size = vec_from.size();
result_array_offsets_data.resize(size);
result_array_values_data.reserve(size * 2);

using UnsignedType = make_unsigned_t<T>;

for (size_t row = 0; row < size; ++row)
{
UnsignedType x = static_cast<UnsignedType>(vec_from[row]);

if constexpr (is_big_int_v<UnsignedType>)
{
size_t position = 0;

while (x)
{
if (x & 1)
result_array_values_data.push_back(position);

x >>= 1;
++position;
}
}
else
{
while (x)
{
result_array_values_data.push_back(getTrailingZeroBitsUnsafe(x));
x &= (x - 1);
}
}

result_array_offsets_data[row] = result_array_values_data.size();
}

auto result_column = ColumnArray::create(std::move(result_array_values), std::move(result_array_offsets));

return result_column;
}

ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/) const override
{
const IColumn * in_column = arguments[0].column.get();
ColumnPtr result_column;

if (!((result_column = executeType<UInt8>(in_column))
|| (result_column = executeType<UInt16>(in_column))
|| (result_column = executeType<UInt32>(in_column))
|| (result_column = executeType<UInt32>(in_column))
|| (result_column = executeType<UInt64>(in_column))
|| (result_column = executeType<UInt128>(in_column))
|| (result_column = executeType<UInt256>(in_column))
|| (result_column = executeType<Int8>(in_column))
|| (result_column = executeType<Int16>(in_column))
|| (result_column = executeType<Int32>(in_column))
|| (result_column = executeType<Int64>(in_column))
|| (result_column = executeType<Int128>(in_column))
|| (result_column = executeType<Int256>(in_column))))
{
throw Exception(ErrorCodes::ILLEGAL_COLUMN,
"Illegal column {} of first argument of function {}",
arguments[0].column->getName(),
getName());
}

return result_column;
}
};

class FunctionToStringCutToZero : public IFunction
{
public:
Expand Down
44 changes: 44 additions & 0 deletions tests/queries/0_stateless/01866_bit_positions_to_array.reference
Original file line number Diff line number Diff line change
@@ -0,0 +1,44 @@
Int8
0 []
1 [0]
-1 [0,1,2,3,4,5,6,7]
127 [0,1,2,3,4,5,6]
-128 [7]
Int16
0 []
1 [0]
-1 [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15]
32765 [0,2,3,4,5,6,7,8,9,10,11,12,13,14]
-32768 [15]
Int32
0 []
1 [0]
Int64
0 []
1 [0]
Int128
0 []
1 [0]
Int256
0 []
1 [0]
UInt8
0 []
1 [0]
128 [7]
UInt16
0 []
1 [0]
UInt32
0 []
1 [0]
UInt64
0 []
1 [0]
UInt128
0 []
1 [0]
340282366920938463463374607431768211455 [0]
UInt256
0 []
1 [0]
55 changes: 55 additions & 0 deletions tests/queries/0_stateless/01866_bit_positions_to_array.sql
Original file line number Diff line number Diff line change
@@ -0,0 +1,55 @@
SELECT 'Int8';
SELECT toInt8(0), bitPositionsToArray(toInt8(0));
SELECT toInt8(1), bitPositionsToArray(toInt8(1));
SELECT toInt8(-1), bitPositionsToArray(toInt8(-1));
SELECT toInt8(127), bitPositionsToArray(toInt8(127));
SELECT toInt8(128), bitPositionsToArray(toInt8(128));

SELECT 'Int16';
SELECT toInt16(0), bitPositionsToArray(toInt16(0));
SELECT toInt16(1), bitPositionsToArray(toInt16(1));
SELECT toInt16(-1), bitPositionsToArray(toInt16(-1));
select toInt16(32765), bitPositionsToArray(toInt16(32765));
select toInt16(32768), bitPositionsToArray(toInt16(32768));

SELECT 'Int32';
SELECT toInt32(0), bitPositionsToArray(toInt32(0));
SELECT toInt32(1), bitPositionsToArray(toInt32(1));

SELECT 'Int64';
SELECT toInt64(0), bitPositionsToArray(toInt64(0));
SELECT toInt64(1), bitPositionsToArray(toInt64(1));

SELECT 'Int128';
SELECT toInt128(0), bitPositionsToArray(toInt128(0));
SELECT toInt128(1), bitPositionsToArray(toInt128(1));

SELECT 'Int256';
SELECT toInt256(0), bitPositionsToArray(toInt256(0));
SELECT toInt256(1), bitPositionsToArray(toInt256(1));

SELECT 'UInt8';
SELECT toUInt8(0), bitPositionsToArray(toUInt8(0));
SELECT toUInt8(1), bitPositionsToArray(toUInt8(1));
SELECT toUInt8(128), bitPositionsToArray(toUInt8(128));

SELECT 'UInt16';
SELECT toUInt16(0), bitPositionsToArray(toUInt16(0));
SELECT toUInt16(1), bitPositionsToArray(toUInt16(1));

SELECT 'UInt32';
SELECT toUInt32(0), bitPositionsToArray(toUInt32(0));
SELECT toUInt32(1), bitPositionsToArray(toUInt32(1));

SELECT 'UInt64';
SELECT toUInt64(0), bitPositionsToArray(toUInt64(0));
SELECT toUInt64(1), bitPositionsToArray(toUInt64(1));

SELECT 'UInt128';
SELECT toUInt128(0), bitPositionsToArray(toUInt128(0));
SELECT toUInt128(1), bitPositionsToArray(toUInt128(1));
SELECT toUInt128(-1), bitPositionsToArray(toUInt128(1));

SELECT 'UInt256';
SELECT toUInt256(0), bitPositionsToArray(toUInt256(0));
SELECT toUInt256(1), bitPositionsToArray(toUInt256(1));

0 comments on commit b34b66c

Please sign in to comment.