mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-15 12:14:18 +00:00
Merge pull request #70846 from faizan2786/master
Functions `base58Encode/Decode`: Support `FixedString` arguments
This commit is contained in:
commit
9d5410e7ea
@ -1096,7 +1096,7 @@ convertCharset(s, from, to)
|
||||
|
||||
## base58Encode
|
||||
|
||||
Encodes a String using [Base58](https://datatracker.ietf.org/doc/html/draft-msporny-base58) in the "Bitcoin" alphabet.
|
||||
Encodes a string using [Base58](https://datatracker.ietf.org/doc/html/draft-msporny-base58) in the "Bitcoin" alphabet.
|
||||
|
||||
**Syntax**
|
||||
|
||||
@ -1110,7 +1110,7 @@ base58Encode(plaintext)
|
||||
|
||||
**Returned value**
|
||||
|
||||
- A string containing the encoded value of the argument. [String](../data-types/string.md).
|
||||
- A string containing the encoded value of the argument. [String](../data-types/string.md) or [FixedString](../data-types/fixedstring.md).
|
||||
|
||||
**Example**
|
||||
|
||||
@ -1128,7 +1128,7 @@ Result:
|
||||
|
||||
## base58Decode
|
||||
|
||||
Accepts a String and decodes it using [Base58](https://datatracker.ietf.org/doc/html/draft-msporny-base58) encoding scheme using "Bitcoin" alphabet.
|
||||
Accepts a string and decodes it using [Base58](https://datatracker.ietf.org/doc/html/draft-msporny-base58) encoding scheme using "Bitcoin" alphabet.
|
||||
|
||||
**Syntax**
|
||||
|
||||
@ -1138,7 +1138,7 @@ base58Decode(encoded)
|
||||
|
||||
**Arguments**
|
||||
|
||||
- `encoded` — [String](../data-types/string.md) column or constant. If the string is not a valid Base58-encoded value, an exception is thrown.
|
||||
- `encoded` — [String](../data-types/string.md) or [FixedString](../data-types/fixedstring.md). If the string is not a valid Base58-encoded value, an exception is thrown.
|
||||
|
||||
**Returned value**
|
||||
|
||||
@ -1170,7 +1170,7 @@ tryBase58Decode(encoded)
|
||||
|
||||
**Parameters**
|
||||
|
||||
- `encoded`: [String](../data-types/string.md) column or constant. If the string is not a valid Base58-encoded value, returns an empty string in case of error.
|
||||
- `encoded`: [String](../data-types/string.md) or [FixedString](../data-types/fixedstring.md). If the string is not a valid Base58-encoded value, returns an empty string in case of error.
|
||||
|
||||
**Returned value**
|
||||
|
||||
|
@ -1,6 +1,8 @@
|
||||
#pragma once
|
||||
|
||||
#include <Columns/ColumnConst.h>
|
||||
#include <Common/MemorySanitizer.h>
|
||||
#include <Columns/ColumnFixedString.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
@ -16,23 +18,20 @@ namespace DB
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int ILLEGAL_COLUMN;
|
||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||
extern const int BAD_ARGUMENTS;
|
||||
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
||||
}
|
||||
|
||||
struct Base58Encode
|
||||
{
|
||||
static constexpr auto name = "base58Encode";
|
||||
|
||||
static void process(const ColumnString & src_column, ColumnString::MutablePtr & dst_column, size_t input_rows_count)
|
||||
static void processString(const ColumnString & src_column, ColumnString::MutablePtr & dst_column, size_t input_rows_count)
|
||||
{
|
||||
auto & dst_data = dst_column->getChars();
|
||||
auto & dst_offsets = dst_column->getOffsets();
|
||||
|
||||
/// Base58 has efficiency of 73% (8/11) [https://monerodocs.org/cryptography/base58/],
|
||||
/// and we take double scale to avoid any reallocation.
|
||||
|
||||
size_t max_result_size = static_cast<size_t>(ceil(2 * src_column.getChars().size() + 1));
|
||||
|
||||
dst_data.resize(max_result_size);
|
||||
@ -61,6 +60,37 @@ struct Base58Encode
|
||||
|
||||
dst_data.resize(current_dst_offset);
|
||||
}
|
||||
|
||||
static void processFixedString(const ColumnFixedString & src_column, ColumnString::MutablePtr & dst_column, size_t input_rows_count)
|
||||
{
|
||||
auto & dst_data = dst_column->getChars();
|
||||
auto & dst_offsets = dst_column->getOffsets();
|
||||
|
||||
/// Base58 has efficiency of 73% (8/11) [https://monerodocs.org/cryptography/base58/],
|
||||
/// and we take double scale to avoid any reallocation.
|
||||
size_t max_result_size = static_cast<size_t>(ceil(2 * src_column.getChars().size() + 1));
|
||||
|
||||
dst_data.resize(max_result_size);
|
||||
dst_offsets.resize(input_rows_count);
|
||||
|
||||
const auto * src = src_column.getChars().data();
|
||||
auto * dst = dst_data.data();
|
||||
|
||||
size_t N = src_column.getN();
|
||||
size_t current_dst_offset = 0;
|
||||
|
||||
for (size_t row = 0; row < input_rows_count; ++row)
|
||||
{
|
||||
size_t encoded_size = encodeBase58(&src[row * N], N, &dst[current_dst_offset]);
|
||||
current_dst_offset += encoded_size;
|
||||
dst[current_dst_offset] = 0;
|
||||
++current_dst_offset;
|
||||
|
||||
dst_offsets[row] = current_dst_offset;
|
||||
}
|
||||
|
||||
dst_data.resize(current_dst_offset);
|
||||
}
|
||||
};
|
||||
|
||||
enum class Base58DecodeErrorHandling : uint8_t
|
||||
@ -74,14 +104,13 @@ struct Base58Decode
|
||||
{
|
||||
static constexpr auto name = Name::name;
|
||||
|
||||
static void process(const ColumnString & src_column, ColumnString::MutablePtr & dst_column, size_t input_rows_count)
|
||||
static void processString(const ColumnString & src_column, ColumnString::MutablePtr & dst_column, size_t input_rows_count)
|
||||
{
|
||||
auto & dst_data = dst_column->getChars();
|
||||
auto & dst_offsets = dst_column->getOffsets();
|
||||
|
||||
/// Base58 has efficiency of 73% (8/11) [https://monerodocs.org/cryptography/base58/],
|
||||
/// and decoded value will be no longer than source.
|
||||
|
||||
size_t max_result_size = src_column.getChars().size() + 1;
|
||||
|
||||
dst_data.resize(max_result_size);
|
||||
@ -118,6 +147,45 @@ struct Base58Decode
|
||||
|
||||
dst_data.resize(current_dst_offset);
|
||||
}
|
||||
|
||||
static void processFixedString(const ColumnFixedString & src_column, ColumnString::MutablePtr & dst_column, size_t input_rows_count)
|
||||
{
|
||||
auto & dst_data = dst_column->getChars();
|
||||
auto & dst_offsets = dst_column->getOffsets();
|
||||
|
||||
/// Base58 has efficiency of 73% (8/11) [https://monerodocs.org/cryptography/base58/],
|
||||
/// and decoded value will be no longer than source.
|
||||
size_t max_result_size = src_column.getChars().size() + 1;
|
||||
|
||||
dst_data.resize(max_result_size);
|
||||
dst_offsets.resize(input_rows_count);
|
||||
|
||||
const auto * src = src_column.getChars().data();
|
||||
auto * dst = dst_data.data();
|
||||
|
||||
size_t N = src_column.getN();
|
||||
size_t current_dst_offset = 0;
|
||||
|
||||
for (size_t row = 0; row < input_rows_count; ++row)
|
||||
{
|
||||
std::optional<size_t> decoded_size = decodeBase58(&src[row * N], N, &dst[current_dst_offset]);
|
||||
if (!decoded_size)
|
||||
{
|
||||
if constexpr (ErrorHandling == Base58DecodeErrorHandling::ThrowException)
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Invalid Base58 value, cannot be decoded");
|
||||
else
|
||||
decoded_size = 0;
|
||||
}
|
||||
|
||||
current_dst_offset += *decoded_size;
|
||||
dst[current_dst_offset] = 0;
|
||||
++current_dst_offset;
|
||||
|
||||
dst_offsets[row] = current_dst_offset;
|
||||
}
|
||||
|
||||
dst_data.resize(current_dst_offset);
|
||||
}
|
||||
};
|
||||
|
||||
template <typename Func>
|
||||
@ -135,14 +203,10 @@ public:
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
|
||||
{
|
||||
if (arguments.size() != 1)
|
||||
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Wrong number of arguments for function {}: 1 expected.", getName());
|
||||
|
||||
if (!isString(arguments[0].type))
|
||||
throw Exception(
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
||||
"Illegal type {} of first argument of function {}. Must be String.",
|
||||
arguments[0].type->getName(), getName());
|
||||
FunctionArgumentDescriptors args{
|
||||
{"arg", static_cast<FunctionArgumentDescriptor::TypeValidator>(&isStringOrFixedString), nullptr, "String or FixedString"}
|
||||
};
|
||||
validateFunctionArguments(*this, arguments, args);
|
||||
|
||||
return std::make_shared<DataTypeString>();
|
||||
}
|
||||
@ -154,19 +218,25 @@ public:
|
||||
|
||||
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override
|
||||
{
|
||||
const ColumnPtr column_string = arguments[0].column;
|
||||
const ColumnString * input = checkAndGetColumn<ColumnString>(column_string.get());
|
||||
if (!input)
|
||||
throw Exception(
|
||||
ErrorCodes::ILLEGAL_COLUMN,
|
||||
"Illegal column {} of first argument of function {}, must be String",
|
||||
arguments[0].column->getName(), getName());
|
||||
const ColumnPtr col = arguments[0].column;
|
||||
|
||||
auto dst_column = ColumnString::create();
|
||||
if (const ColumnString * col_string = checkAndGetColumn<ColumnString>(col.get()))
|
||||
{
|
||||
auto col_res = ColumnString::create();
|
||||
Func::processString(*col_string, col_res, input_rows_count);
|
||||
return col_res;
|
||||
}
|
||||
else if (const ColumnFixedString * col_fixed_string = checkAndGetColumn<ColumnFixedString>(col.get()))
|
||||
{
|
||||
auto col_res = ColumnString::create();
|
||||
Func::processFixedString(*col_fixed_string, col_res, input_rows_count);
|
||||
return col_res;
|
||||
}
|
||||
|
||||
Func::process(*input, dst_column, input_rows_count);
|
||||
|
||||
return dst_column;
|
||||
throw Exception(
|
||||
ErrorCodes::ILLEGAL_COLUMN,
|
||||
"Illegal column {} of first argument of function {}, must be String or FixedString",
|
||||
arguments[0].column->getName(), getName());
|
||||
}
|
||||
};
|
||||
}
|
||||
|
@ -40,3 +40,15 @@ foobar
|
||||
|
||||
1
|
||||
1
|
||||
32YCBjgZhV4AdCWHaCDNu
|
||||
foobar
|
||||
111
|
||||
bG7y
|
||||
bQZu
|
||||
bQbp
|
||||
\0\0\0
|
||||
f\0\0
|
||||
fo\0
|
||||
foo
|
||||
1
|
||||
1
|
||||
|
@ -1,15 +1,25 @@
|
||||
-- Tags: no-fasttest
|
||||
|
||||
SELECT base58Encode('Hold my beer...');
|
||||
|
||||
SELECT base58Encode('Hold my beer...', 'Second arg'); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH }
|
||||
SELECT base58Decode('Hold my beer...'); -- { serverError BAD_ARGUMENTS }
|
||||
|
||||
SELECT base58Decode(encoded) FROM (SELECT base58Encode(val) as encoded FROM (select arrayJoin(['', 'f', 'fo', 'foo', 'foob', 'fooba', 'foobar', 'Hello world!']) val));
|
||||
SELECT tryBase58Decode(encoded) FROM (SELECT base58Encode(val) as encoded FROM (select arrayJoin(['', 'f', 'fo', 'foo', 'foob', 'fooba', 'foobar', 'Hello world!']) val));
|
||||
SELECT base58Decode(encoded) FROM (SELECT base58Encode(val) as encoded FROM (SELECT arrayJoin(['', 'f', 'fo', 'foo', 'foob', 'fooba', 'foobar', 'Hello world!']) val));
|
||||
SELECT tryBase58Decode(encoded) FROM (SELECT base58Encode(val) as encoded FROM (SELECT arrayJoin(['', 'f', 'fo', 'foo', 'foob', 'fooba', 'foobar', 'Hello world!']) val));
|
||||
SELECT tryBase58Decode(val) FROM (SELECT arrayJoin(['Hold my beer', 'Hold another beer', '3csAg9', 'And a wine', 'And another wine', 'And a lemonade', 't1Zv2yaZ', 'And another wine']) val);
|
||||
|
||||
SELECT base58Encode(val) FROM (select arrayJoin(['', 'f', 'fo', 'foo', 'foob', 'fooba', 'foobar']) val);
|
||||
SELECT base58Decode(val) FROM (select arrayJoin(['', '2m', '8o8', 'bQbp', '3csAg9', 'CZJRhmz', 't1Zv2yaZ', '']) val);
|
||||
SELECT base58Encode(val) FROM (SELECT arrayJoin(['', 'f', 'fo', 'foo', 'foob', 'fooba', 'foobar']) val);
|
||||
SELECT base58Decode(val) FROM (SELECT arrayJoin(['', '2m', '8o8', 'bQbp', '3csAg9', 'CZJRhmz', 't1Zv2yaZ', '']) val);
|
||||
|
||||
SELECT base58Encode(base58Decode('1BWutmTvYPwDtmw9abTkS4Ssr8no61spGAvW1X6NDix')) == '1BWutmTvYPwDtmw9abTkS4Ssr8no61spGAvW1X6NDix';
|
||||
select base58Encode('\x00\x0b\xe3\xe1\xeb\xa1\x7a\x47\x3f\x89\xb0\xf7\xe8\xe2\x49\x40\xf2\x0a\xeb\x8e\xbc\xa7\x1a\x88\xfd\xe9\x5d\x4b\x83\xb7\x1a\x09') == '1BWutmTvYPwDtmw9abTkS4Ssr8no61spGAvW1X6NDix';
|
||||
|
||||
SELECT base58Encode(toFixedString('Hold my beer...', 15));
|
||||
SELECT base58Decode(toFixedString('t1Zv2yaZ', 8));
|
||||
|
||||
SELECT base58Encode(val) FROM (SELECT arrayJoin([toFixedString('', 3), toFixedString('f', 3), toFixedString('fo', 3), toFixedString('foo', 3)]) val);
|
||||
SELECT base58Decode(val) FROM (SELECT arrayJoin([toFixedString('111', 3), toFixedString('bG7y', 4), toFixedString('bQZu', 4), toFixedString('bQbp', 4)]) val);
|
||||
|
||||
Select base58Encode(reinterpretAsFixedString(byteSwap(toUInt256('256')))) == '1111111111111111111111111111115R';
|
||||
Select base58Encode(reinterpretAsString(byteSwap(toUInt256('256')))) == '1111111111111111111111111111112'; -- { reinterpretAsString drops the last null byte hence, encoded value is different than the FixedString version above }
|
||||
|
Loading…
Reference in New Issue
Block a user