mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 01:25:21 +00:00
Merge branch 'master' into qoega-patch-5
This commit is contained in:
commit
dced847e9b
53
base/base/IPv4andIPv6.h
Normal file
53
base/base/IPv4andIPv6.h
Normal file
@ -0,0 +1,53 @@
|
||||
#pragma once
|
||||
|
||||
#include <base/strong_typedef.h>
|
||||
#include <base/extended_types.h>
|
||||
#include <Common/memcmpSmall.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
using IPv4 = StrongTypedef<UInt32, struct IPv4Tag>;
|
||||
|
||||
struct IPv6 : StrongTypedef<UInt128, struct IPv6Tag>
|
||||
{
|
||||
constexpr IPv6() = default;
|
||||
constexpr explicit IPv6(const UInt128 & x) : StrongTypedef(x) {}
|
||||
constexpr explicit IPv6(UInt128 && x) : StrongTypedef(std::move(x)) {}
|
||||
|
||||
IPv6 & operator=(const UInt128 & rhs) { StrongTypedef::operator=(rhs); return *this; }
|
||||
IPv6 & operator=(UInt128 && rhs) { StrongTypedef::operator=(std::move(rhs)); return *this; }
|
||||
|
||||
bool operator<(const IPv6 & rhs) const
|
||||
{
|
||||
return
|
||||
memcmp16(
|
||||
reinterpret_cast<const unsigned char *>(toUnderType().items),
|
||||
reinterpret_cast<const unsigned char *>(rhs.toUnderType().items)
|
||||
) < 0;
|
||||
}
|
||||
|
||||
bool operator>(const IPv6 & rhs) const
|
||||
{
|
||||
return
|
||||
memcmp16(
|
||||
reinterpret_cast<const unsigned char *>(toUnderType().items),
|
||||
reinterpret_cast<const unsigned char *>(rhs.toUnderType().items)
|
||||
) > 0;
|
||||
}
|
||||
|
||||
bool operator==(const IPv6 & rhs) const
|
||||
{
|
||||
return
|
||||
memcmp16(
|
||||
reinterpret_cast<const unsigned char *>(toUnderType().items),
|
||||
reinterpret_cast<const unsigned char *>(rhs.toUnderType().items)
|
||||
) == 0;
|
||||
}
|
||||
|
||||
bool operator<=(const IPv6 & rhs) const { return !operator>(rhs); }
|
||||
bool operator>=(const IPv6 & rhs) const { return !operator<(rhs); }
|
||||
bool operator!=(const IPv6 & rhs) const { return !operator==(rhs); }
|
||||
};
|
||||
|
||||
}
|
@ -2,6 +2,7 @@
|
||||
|
||||
#include "Decimal.h"
|
||||
#include "UUID.h"
|
||||
#include "IPv4andIPv6.h"
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -35,6 +36,8 @@ TN_MAP(Float32)
|
||||
TN_MAP(Float64)
|
||||
TN_MAP(String)
|
||||
TN_MAP(UUID)
|
||||
TN_MAP(IPv4)
|
||||
TN_MAP(IPv6)
|
||||
TN_MAP(Decimal32)
|
||||
TN_MAP(Decimal64)
|
||||
TN_MAP(Decimal128)
|
||||
|
@ -63,10 +63,6 @@
|
||||
"name": "clickhouse/integration-tests-runner",
|
||||
"dependent": []
|
||||
},
|
||||
"docker/test/testflows/runner": {
|
||||
"name": "clickhouse/testflows-runner",
|
||||
"dependent": []
|
||||
},
|
||||
"docker/test/fasttest": {
|
||||
"name": "clickhouse/fasttest",
|
||||
"dependent": []
|
||||
|
@ -33,7 +33,7 @@ RUN arch=${TARGETARCH:-amd64} \
|
||||
# lts / testing / prestable / etc
|
||||
ARG REPO_CHANNEL="stable"
|
||||
ARG REPOSITORY="https://packages.clickhouse.com/tgz/${REPO_CHANNEL}"
|
||||
ARG VERSION="22.12.1.1752"
|
||||
ARG VERSION="22.12.2.25"
|
||||
ARG PACKAGES="clickhouse-client clickhouse-server clickhouse-common-static"
|
||||
|
||||
# user/group precreated explicitly with fixed uid/gid on purpose.
|
||||
|
@ -21,7 +21,7 @@ RUN sed -i "s|http://archive.ubuntu.com|${apt_archive}|g" /etc/apt/sources.list
|
||||
|
||||
ARG REPO_CHANNEL="stable"
|
||||
ARG REPOSITORY="deb https://packages.clickhouse.com/deb ${REPO_CHANNEL} main"
|
||||
ARG VERSION="22.12.1.1752"
|
||||
ARG VERSION="22.12.2.25"
|
||||
ARG PACKAGES="clickhouse-client clickhouse-server clickhouse-common-static"
|
||||
|
||||
# set non-empty deb_location_url url to create a docker image
|
||||
|
@ -262,14 +262,17 @@ quit
|
||||
if [ "$server_died" == 1 ]
|
||||
then
|
||||
# The server has died.
|
||||
if ! grep --text -ao "Received signal.*\|Logical error.*\|Assertion.*failed\|Failed assertion.*\|.*runtime error: .*\|.*is located.*\|SUMMARY: AddressSanitizer:.*\|SUMMARY: MemorySanitizer:.*\|SUMMARY: ThreadSanitizer:.*\|.*_LIBCPP_ASSERT.*" server.log > description.txt
|
||||
if ! grep -E --text -o 'Received signal.*|Logical error.*|Assertion.*failed|Failed assertion.*|.*runtime error: .*|.*is located.*|(SUMMARY|ERROR): [a-zA-Z]+Sanitizer:.*|.*_LIBCPP_ASSERT.*' server.log > description.txt
|
||||
then
|
||||
echo "Lost connection to server. See the logs." > description.txt
|
||||
fi
|
||||
|
||||
if grep -E --text 'Sanitizer: (out-of-memory|failed to allocate)' description.txt
|
||||
IS_SANITIZED=$(clickhouse-local --query "SELECT value LIKE '%-fsanitize=%' FROM system.build_options WHERE name = 'CXX_FLAGS'")
|
||||
|
||||
if [ "${IS_SANITIZED}" -eq "1" ] && grep -E --text 'Sanitizer: (out-of-memory|out of memory|failed to allocate|Child process was terminated by signal 9)' description.txt
|
||||
then
|
||||
# OOM of sanitizer is not a problem we can handle - treat it as success, but preserve the description.
|
||||
# Why? Because sanitizers have the memory overhead, that is not controllable from inside clickhouse-server.
|
||||
task_exit_code=0
|
||||
echo "success" > status.txt
|
||||
else
|
||||
|
@ -8,6 +8,7 @@ RUN sed -i "s|http://archive.ubuntu.com|$apt_archive|g" /etc/apt/sources.list
|
||||
|
||||
RUN apt-get update \
|
||||
&& env DEBIAN_FRONTEND=noninteractive apt-get install --yes \
|
||||
adduser \
|
||||
ca-certificates \
|
||||
bash \
|
||||
btrfs-progs \
|
||||
|
29
docs/changelogs/v22.12.2.25-stable.md
Normal file
29
docs/changelogs/v22.12.2.25-stable.md
Normal file
@ -0,0 +1,29 @@
|
||||
---
|
||||
sidebar_position: 1
|
||||
sidebar_label: 2023
|
||||
---
|
||||
|
||||
# 2023 Changelog
|
||||
|
||||
### ClickHouse release v22.12.2.25-stable (c790cfd4465) FIXME as compared to v22.12.1.1752-stable (688e488e930)
|
||||
|
||||
#### Build/Testing/Packaging Improvement
|
||||
* Backported in [#44381](https://github.com/ClickHouse/ClickHouse/issues/44381): In rare cases, we don't rebuild binaries, because another task with a similar prefix succeeded. E.g. `binary_darwin` didn't restart because `binary_darwin_aarch64`. [#44311](https://github.com/ClickHouse/ClickHouse/pull/44311) ([Mikhail f. Shiryaev](https://github.com/Felixoid)).
|
||||
* Backported in [#44561](https://github.com/ClickHouse/ClickHouse/issues/44561): Retry the integration tests on compressing errors. [#44529](https://github.com/ClickHouse/ClickHouse/pull/44529) ([Mikhail f. Shiryaev](https://github.com/Felixoid)).
|
||||
|
||||
#### Bug Fix (user-visible misbehavior in official stable or prestable release)
|
||||
|
||||
* Backported in [#44739](https://github.com/ClickHouse/ClickHouse/issues/44739): [#40651](https://github.com/ClickHouse/ClickHouse/issues/40651) [#41404](https://github.com/ClickHouse/ClickHouse/issues/41404). [#42126](https://github.com/ClickHouse/ClickHouse/pull/42126) ([Alexander Gololobov](https://github.com/davenger)).
|
||||
* Backported in [#44764](https://github.com/ClickHouse/ClickHouse/issues/44764): Fix parsing of bad version from compatibility setting. [#44224](https://github.com/ClickHouse/ClickHouse/pull/44224) ([Kruglov Pavel](https://github.com/Avogar)).
|
||||
* Backported in [#44435](https://github.com/ClickHouse/ClickHouse/issues/44435): Fix possible crash in case function `IN` with constant arguments was used as a constant argument together with `LowCardinality`. Fixes [#44221](https://github.com/ClickHouse/ClickHouse/issues/44221). [#44346](https://github.com/ClickHouse/ClickHouse/pull/44346) ([Nikolai Kochetov](https://github.com/KochetovNicolai)).
|
||||
|
||||
#### NOT FOR CHANGELOG / INSIGNIFICANT
|
||||
|
||||
* Bump libdivide (to gain some new optimizations) [#44132](https://github.com/ClickHouse/ClickHouse/pull/44132) ([Azat Khuzhin](https://github.com/azat)).
|
||||
* Add some settings under `compatibility` [#44209](https://github.com/ClickHouse/ClickHouse/pull/44209) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
|
||||
* Fix deadlock in StorageSystemDatabases [#44272](https://github.com/ClickHouse/ClickHouse/pull/44272) ([Alexander Tokmakov](https://github.com/tavplubix)).
|
||||
* Get rid of global Git object [#44273](https://github.com/ClickHouse/ClickHouse/pull/44273) ([Mikhail f. Shiryaev](https://github.com/Felixoid)).
|
||||
* Add check for submodules sanity [#44386](https://github.com/ClickHouse/ClickHouse/pull/44386) ([Mikhail f. Shiryaev](https://github.com/Felixoid)).
|
||||
* Implement a custom central checkout action [#44399](https://github.com/ClickHouse/ClickHouse/pull/44399) ([Mikhail f. Shiryaev](https://github.com/Felixoid)).
|
||||
* Fix crash on delete from materialized view [#44705](https://github.com/ClickHouse/ClickHouse/pull/44705) ([Alexander Gololobov](https://github.com/davenger)).
|
||||
|
@ -154,10 +154,6 @@ Runs [stateful functional tests](tests.md#functional-tests). Treat them in the s
|
||||
Runs [integration tests](tests.md#integration-tests).
|
||||
|
||||
|
||||
## Testflows Check
|
||||
Runs some tests using Testflows test system. See [here](https://github.com/ClickHouse/ClickHouse/tree/master/tests/testflows#running-tests-locally) how to run them locally.
|
||||
|
||||
|
||||
## Stress Test
|
||||
Runs stateless functional tests concurrently from several clients to detect
|
||||
concurrency-related errors. If it fails:
|
||||
|
@ -281,10 +281,6 @@ We also track test coverage but only for functional tests and only for clickhous
|
||||
|
||||
There is automated check for flaky tests. It runs all new tests 100 times (for functional tests) or 10 times (for integration tests). If at least single time the test failed, it is considered flaky.
|
||||
|
||||
## Testflows
|
||||
|
||||
[Testflows](https://testflows.com/) is an enterprise-grade open-source testing framework, which is used to test a subset of ClickHouse.
|
||||
|
||||
## Test Automation {#test-automation}
|
||||
|
||||
We run tests with [GitHub Actions](https://github.com/features/actions).
|
||||
|
@ -19,7 +19,7 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster]
|
||||
(
|
||||
Path String,
|
||||
Time DateTime,
|
||||
Value <Numeric_type>,
|
||||
Value Float64,
|
||||
Version <Numeric_type>
|
||||
...
|
||||
) ENGINE = GraphiteMergeTree(config_section)
|
||||
@ -37,7 +37,7 @@ A table for the Graphite data should have the following columns for the followin
|
||||
|
||||
- Time of measuring the metric. Data type: `DateTime`.
|
||||
|
||||
- Value of the metric. Data type: any numeric.
|
||||
- Value of the metric. Data type: `Float64`.
|
||||
|
||||
- Version of the metric. Data type: any numeric (ClickHouse saves the rows with the highest version or the last written if versions are the same. Other rows are deleted during the merge of data parts).
|
||||
|
||||
@ -65,7 +65,7 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster]
|
||||
EventDate Date,
|
||||
Path String,
|
||||
Time DateTime,
|
||||
Value <Numeric_type>,
|
||||
Value Float64,
|
||||
Version <Numeric_type>
|
||||
...
|
||||
) ENGINE [=] GraphiteMergeTree(date-column [, sampling_expression], (primary, key), index_granularity, config_section)
|
||||
|
@ -86,7 +86,18 @@ When creating a table, the following settings are applied:
|
||||
[join_any_take_last_row](/docs/en/operations/settings/settings.md/#settings-join_any_take_last_row)
|
||||
#### join_use_nulls
|
||||
|
||||
[persistent](/docs/en/operations/settings/settings.md/#persistent)
|
||||
#### persistent
|
||||
|
||||
Disables persistency for the Join and [Set](/docs/en/engines/table-engines/special/set.md) table engines.
|
||||
|
||||
Reduces the I/O overhead. Suitable for scenarios that pursue performance and do not require persistence.
|
||||
|
||||
Possible values:
|
||||
|
||||
- 1 — Enabled.
|
||||
- 0 — Disabled.
|
||||
|
||||
Default value: `1`.
|
||||
|
||||
The `Join`-engine tables can’t be used in `GLOBAL JOIN` operations.
|
||||
|
||||
|
@ -19,5 +19,16 @@ For a rough server restart, the block of data on the disk might be lost or damag
|
||||
|
||||
When creating a table, the following settings are applied:
|
||||
|
||||
- [persistent](../../../operations/settings/settings.md#persistent)
|
||||
#### persistent
|
||||
|
||||
Disables persistency for the Set and [Join](/docs/en/engines/table-engines/special/join.md/#join) table engines.
|
||||
|
||||
Reduces the I/O overhead. Suitable for scenarios that pursue performance and do not require persistence.
|
||||
|
||||
Possible values:
|
||||
|
||||
- 1 — Enabled.
|
||||
- 0 — Disabled.
|
||||
|
||||
Default value: `1`.
|
||||
|
||||
|
@ -2441,19 +2441,6 @@ Result
|
||||
└──────────────────────────┴───────┴───────────────────────────────────────────────────────┘
|
||||
```
|
||||
|
||||
## persistent {#persistent}
|
||||
|
||||
Disables persistency for the [Set](../../engines/table-engines/special/set.md/#set) and [Join](../../engines/table-engines/special/join.md/#join) table engines.
|
||||
|
||||
Reduces the I/O overhead. Suitable for scenarios that pursue performance and do not require persistence.
|
||||
|
||||
Possible values:
|
||||
|
||||
- 1 — Enabled.
|
||||
- 0 — Disabled.
|
||||
|
||||
Default value: `1`.
|
||||
|
||||
## allow_nullable_key {#allow-nullable-key}
|
||||
|
||||
Allows using of the [Nullable](../../sql-reference/data-types/nullable.md/#data_type-nullable)-typed values in a sorting and a primary key for [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md/#table_engines-mergetree) tables.
|
||||
|
@ -11,6 +11,7 @@ Columns:
|
||||
- `table` ([String](../../sql-reference/data-types/string.md)) — Table name.
|
||||
- `name` ([String](../../sql-reference/data-types/string.md)) — Index name.
|
||||
- `type` ([String](../../sql-reference/data-types/string.md)) — Index type.
|
||||
- `type_full` ([String](../../sql-reference/data-types/string.md)) — Index type expression from create statement.
|
||||
- `expr` ([String](../../sql-reference/data-types/string.md)) — Expression for the index calculation.
|
||||
- `granularity` ([UInt64](../../sql-reference/data-types/int-uint.md)) — The number of granules in the block.
|
||||
- `data_compressed_bytes` ([UInt64](../../sql-reference/data-types/int-uint.md)) — The size of compressed data, in bytes.
|
||||
@ -30,6 +31,7 @@ database: default
|
||||
table: user_actions
|
||||
name: clicks_idx
|
||||
type: minmax
|
||||
type_full: minmax
|
||||
expr: clicks
|
||||
granularity: 1
|
||||
data_compressed_bytes: 58
|
||||
@ -42,6 +44,7 @@ database: default
|
||||
table: users
|
||||
name: contacts_null_idx
|
||||
type: minmax
|
||||
type_full: minmax
|
||||
expr: assumeNotNull(contacts_null)
|
||||
granularity: 1
|
||||
data_compressed_bytes: 58
|
||||
|
@ -806,7 +806,8 @@ Aliases: `dateSub`, `DATE_SUB`.
|
||||
|
||||
**Arguments**
|
||||
|
||||
- `unit` — The type of interval to subtract. [String](../../sql-reference/data-types/string.md).
|
||||
- `unit` — The type of interval to subtract. Note: The unit should be unquoted.
|
||||
|
||||
Possible values:
|
||||
|
||||
- `second`
|
||||
|
@ -416,6 +416,7 @@ Functions:
|
||||
- `dictGetDateTime`
|
||||
- `dictGetUUID`
|
||||
- `dictGetString`
|
||||
- `dictGetIPv4`, `dictGetIPv6`
|
||||
|
||||
All these functions have the `OrDefault` modification. For example, `dictGetDateOrDefault`.
|
||||
|
||||
|
@ -110,9 +110,6 @@ git push
|
||||
## 集成测试 {#integration-tests}
|
||||
运行[集成测试](./tests.md#integration-tests).
|
||||
|
||||
## Testflows 检查{#testflows-check}
|
||||
使用Testflows测试系统去运行一些测试, 在[此处](https://github.com/ClickHouse/ClickHouse/tree/master/tests/testflows#running-tests-locally)查看如何在本地运行它们.
|
||||
|
||||
## 压力测试 {#stress-test}
|
||||
从多个客户端并发运行无状态功能测试, 用以检测与并发相关的错误.如果失败:
|
||||
```
|
||||
|
@ -281,10 +281,6 @@ We also track test coverage but only for functional tests and only for clickhous
|
||||
|
||||
There is automated check for flaky tests. It runs all new tests 100 times (for functional tests) or 10 times (for integration tests). If at least single time the test failed, it is considered flaky.
|
||||
|
||||
## Testflows
|
||||
|
||||
[Testflows](https://testflows.com/) is an enterprise-grade open-source testing framework, which is used to test a subset of ClickHouse.
|
||||
|
||||
## Test Automation {#test-automation}
|
||||
|
||||
We run tests with [GitHub Actions](https://github.com/features/actions).
|
||||
|
@ -12,8 +12,6 @@ priority: "optional"
|
||||
|
||||
conflicts:
|
||||
- clickhouse-server
|
||||
depends:
|
||||
- adduser
|
||||
suggests:
|
||||
- clickhouse-keeper-dbg
|
||||
|
||||
|
@ -12,8 +12,6 @@ priority: "optional"
|
||||
|
||||
conflicts:
|
||||
- clickhouse-keeper
|
||||
depends:
|
||||
- adduser
|
||||
replaces:
|
||||
- clickhouse-server-common
|
||||
- clickhouse-server-base
|
||||
|
@ -116,13 +116,9 @@ public:
|
||||
|
||||
static DataTypePtr getKeyType(const DataTypes & types, const AggregateFunctionPtr & nested)
|
||||
{
|
||||
if (types.empty())
|
||||
if (types.size() != 1)
|
||||
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
|
||||
"Aggregate function {}Map requires at least one argument", nested->getName());
|
||||
|
||||
if (types.size() > 1)
|
||||
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
|
||||
"Aggregate function {}Map requires only one map argument", nested->getName());
|
||||
"Aggregate function {}Map requires one map argument, but {} found", nested->getName(), types.size());
|
||||
|
||||
const auto * map_type = checkAndGetDataType<DataTypeMap>(types[0].get());
|
||||
if (!map_type)
|
||||
|
@ -428,10 +428,7 @@ public:
|
||||
}
|
||||
|
||||
bool keepKey(const T & key) const { return static_cast<const Derived &>(*this).keepKey(key); }
|
||||
String getName() const override { return getNameImpl(); }
|
||||
|
||||
private:
|
||||
static String getNameImpl() { return Derived::getNameImpl(); }
|
||||
String getName() const override { return Derived::getNameImpl(); }
|
||||
};
|
||||
|
||||
template <typename T, bool overflow, bool tuple_argument>
|
||||
|
@ -65,9 +65,9 @@ class IAggregateFunction : public std::enable_shared_from_this<IAggregateFunctio
|
||||
{
|
||||
public:
|
||||
IAggregateFunction(const DataTypes & argument_types_, const Array & parameters_, const DataTypePtr & result_type_)
|
||||
: result_type(result_type_)
|
||||
, argument_types(argument_types_)
|
||||
: argument_types(argument_types_)
|
||||
, parameters(parameters_)
|
||||
, result_type(result_type_)
|
||||
{}
|
||||
|
||||
/// Get main function name.
|
||||
@ -409,9 +409,9 @@ public:
|
||||
#endif
|
||||
|
||||
protected:
|
||||
DataTypePtr result_type;
|
||||
DataTypes argument_types;
|
||||
Array parameters;
|
||||
DataTypePtr result_type;
|
||||
};
|
||||
|
||||
|
||||
|
@ -31,18 +31,21 @@ FunctionNode::FunctionNode(String function_name_)
|
||||
children[arguments_child_index] = std::make_shared<ListNode>();
|
||||
}
|
||||
|
||||
ColumnsWithTypeAndName FunctionNode::getArgumentTypes() const
|
||||
ColumnsWithTypeAndName FunctionNode::getArgumentColumns() const
|
||||
{
|
||||
ColumnsWithTypeAndName argument_types;
|
||||
for (const auto & arg : getArguments().getNodes())
|
||||
const auto & arguments = getArguments().getNodes();
|
||||
ColumnsWithTypeAndName argument_columns;
|
||||
argument_columns.reserve(arguments.size());
|
||||
|
||||
for (const auto & arg : arguments)
|
||||
{
|
||||
ColumnWithTypeAndName argument;
|
||||
argument.type = arg->getResultType();
|
||||
if (auto * constant = arg->as<ConstantNode>())
|
||||
argument.column = argument.type->createColumnConst(1, constant->getValue());
|
||||
argument_types.push_back(argument);
|
||||
argument_columns.push_back(std::move(argument));
|
||||
}
|
||||
return argument_types;
|
||||
return argument_columns;
|
||||
}
|
||||
|
||||
void FunctionNode::resolveAsFunction(FunctionBasePtr function_value)
|
||||
|
@ -1,12 +1,14 @@
|
||||
#pragma once
|
||||
|
||||
#include <memory>
|
||||
#include <Core/IResolvedFunction.h>
|
||||
#include <AggregateFunctions/IAggregateFunction.h>
|
||||
#include <Analyzer/ConstantValue.h>
|
||||
#include <Analyzer/IQueryTreeNode.h>
|
||||
#include <Analyzer/ListNode.h>
|
||||
#include <Analyzer/ConstantValue.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Core/ColumnsWithTypeAndName.h>
|
||||
#include <Core/IResolvedFunction.h>
|
||||
#include <Functions/IFunction.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -19,12 +21,6 @@ namespace ErrorCodes
|
||||
class IFunctionOverloadResolver;
|
||||
using FunctionOverloadResolverPtr = std::shared_ptr<IFunctionOverloadResolver>;
|
||||
|
||||
class IFunctionBase;
|
||||
using FunctionBasePtr = std::shared_ptr<const IFunctionBase>;
|
||||
|
||||
class IAggregateFunction;
|
||||
using AggregateFunctionPtr = std::shared_ptr<const IAggregateFunction>;
|
||||
|
||||
/** Function node represents function in query tree.
|
||||
* Function syntax: function_name(parameter_1, ...)(argument_1, ...).
|
||||
* If function does not have parameters its syntax is function_name(argument_1, ...).
|
||||
@ -63,66 +59,36 @@ public:
|
||||
explicit FunctionNode(String function_name_);
|
||||
|
||||
/// Get function name
|
||||
const String & getFunctionName() const
|
||||
{
|
||||
return function_name;
|
||||
}
|
||||
const String & getFunctionName() const { return function_name; }
|
||||
|
||||
/// Get parameters
|
||||
const ListNode & getParameters() const
|
||||
{
|
||||
return children[parameters_child_index]->as<const ListNode &>();
|
||||
}
|
||||
const ListNode & getParameters() const { return children[parameters_child_index]->as<const ListNode &>(); }
|
||||
|
||||
/// Get parameters
|
||||
ListNode & getParameters()
|
||||
{
|
||||
return children[parameters_child_index]->as<ListNode &>();
|
||||
}
|
||||
ListNode & getParameters() { return children[parameters_child_index]->as<ListNode &>(); }
|
||||
|
||||
/// Get parameters node
|
||||
const QueryTreeNodePtr & getParametersNode() const
|
||||
{
|
||||
return children[parameters_child_index];
|
||||
}
|
||||
const QueryTreeNodePtr & getParametersNode() const { return children[parameters_child_index]; }
|
||||
|
||||
/// Get parameters node
|
||||
QueryTreeNodePtr & getParametersNode()
|
||||
{
|
||||
return children[parameters_child_index];
|
||||
}
|
||||
QueryTreeNodePtr & getParametersNode() { return children[parameters_child_index]; }
|
||||
|
||||
/// Get arguments
|
||||
const ListNode & getArguments() const
|
||||
{
|
||||
return children[arguments_child_index]->as<const ListNode &>();
|
||||
}
|
||||
const ListNode & getArguments() const { return children[arguments_child_index]->as<const ListNode &>(); }
|
||||
|
||||
/// Get arguments
|
||||
ListNode & getArguments()
|
||||
{
|
||||
return children[arguments_child_index]->as<ListNode &>();
|
||||
}
|
||||
ListNode & getArguments() { return children[arguments_child_index]->as<ListNode &>(); }
|
||||
|
||||
/// Get arguments node
|
||||
const QueryTreeNodePtr & getArgumentsNode() const
|
||||
{
|
||||
return children[arguments_child_index];
|
||||
}
|
||||
const QueryTreeNodePtr & getArgumentsNode() const { return children[arguments_child_index]; }
|
||||
|
||||
/// Get arguments node
|
||||
QueryTreeNodePtr & getArgumentsNode()
|
||||
{
|
||||
return children[arguments_child_index];
|
||||
}
|
||||
QueryTreeNodePtr & getArgumentsNode() { return children[arguments_child_index]; }
|
||||
|
||||
ColumnsWithTypeAndName getArgumentTypes() const;
|
||||
ColumnsWithTypeAndName getArgumentColumns() const;
|
||||
|
||||
/// Returns true if function node has window, false otherwise
|
||||
bool hasWindow() const
|
||||
{
|
||||
return children[window_child_index] != nullptr;
|
||||
}
|
||||
bool hasWindow() const { return children[window_child_index] != nullptr; }
|
||||
|
||||
/** Get window node.
|
||||
* Valid only for window function node.
|
||||
@ -130,18 +96,12 @@ public:
|
||||
* 1. It can be identifier node if window function is defined as expr OVER window_name.
|
||||
* 2. It can be window node if window function is defined as expr OVER (window_name ...).
|
||||
*/
|
||||
const QueryTreeNodePtr & getWindowNode() const
|
||||
{
|
||||
return children[window_child_index];
|
||||
}
|
||||
const QueryTreeNodePtr & getWindowNode() const { return children[window_child_index]; }
|
||||
|
||||
/** Get window node.
|
||||
* Valid only for window function node.
|
||||
*/
|
||||
QueryTreeNodePtr & getWindowNode()
|
||||
{
|
||||
return children[window_child_index];
|
||||
}
|
||||
QueryTreeNodePtr & getWindowNode() { return children[window_child_index]; }
|
||||
|
||||
/** Get non aggregate function.
|
||||
* If function is not resolved nullptr returned.
|
||||
@ -150,7 +110,7 @@ public:
|
||||
{
|
||||
if (kind != FunctionKind::ORDINARY)
|
||||
return {};
|
||||
return std::reinterpret_pointer_cast<const IFunctionBase>(function);
|
||||
return std::static_pointer_cast<const IFunctionBase>(function);
|
||||
}
|
||||
|
||||
/** Get aggregate function.
|
||||
@ -161,32 +121,20 @@ public:
|
||||
{
|
||||
if (kind == FunctionKind::UNKNOWN || kind == FunctionKind::ORDINARY)
|
||||
return {};
|
||||
return std::reinterpret_pointer_cast<const IAggregateFunction>(function);
|
||||
return std::static_pointer_cast<const IAggregateFunction>(function);
|
||||
}
|
||||
|
||||
/// Is function node resolved
|
||||
bool isResolved() const
|
||||
{
|
||||
return function != nullptr;
|
||||
}
|
||||
bool isResolved() const { return function != nullptr; }
|
||||
|
||||
/// Is function node window function
|
||||
bool isWindowFunction() const
|
||||
{
|
||||
return hasWindow();
|
||||
}
|
||||
bool isWindowFunction() const { return hasWindow(); }
|
||||
|
||||
/// Is function node aggregate function
|
||||
bool isAggregateFunction() const
|
||||
{
|
||||
return kind == FunctionKind::AGGREGATE;
|
||||
}
|
||||
bool isAggregateFunction() const { return kind == FunctionKind::AGGREGATE; }
|
||||
|
||||
/// Is function node ordinary function
|
||||
bool isOrdinaryFunction() const
|
||||
{
|
||||
return kind == FunctionKind::ORDINARY;
|
||||
}
|
||||
bool isOrdinaryFunction() const { return kind == FunctionKind::ORDINARY; }
|
||||
|
||||
/** Resolve function node as non aggregate function.
|
||||
* It is important that function name is updated with resolved function name.
|
||||
@ -208,10 +156,7 @@ public:
|
||||
*/
|
||||
void resolveAsWindowFunction(AggregateFunctionPtr window_function_value);
|
||||
|
||||
QueryTreeNodeType getNodeType() const override
|
||||
{
|
||||
return QueryTreeNodeType::FUNCTION;
|
||||
}
|
||||
QueryTreeNodeType getNodeType() const override { return QueryTreeNodeType::FUNCTION; }
|
||||
|
||||
DataTypePtr getResultType() const override
|
||||
{
|
||||
|
@ -155,7 +155,7 @@ public:
|
||||
inline void resolveOrdinaryFunctionNode(FunctionNode & function_node, const String & function_name) const
|
||||
{
|
||||
auto function = FunctionFactory::instance().get(function_name, context);
|
||||
function_node.resolveAsFunction(function->build(function_node.getArgumentTypes()));
|
||||
function_node.resolveAsFunction(function->build(function_node.getArgumentColumns()));
|
||||
}
|
||||
|
||||
private:
|
||||
|
@ -193,7 +193,7 @@ private:
|
||||
inline void resolveOrdinaryFunctionNode(FunctionNode & function_node, const String & function_name) const
|
||||
{
|
||||
auto function = FunctionFactory::instance().get(function_name, context);
|
||||
function_node.resolveAsFunction(function->build(function_node.getArgumentTypes()));
|
||||
function_node.resolveAsFunction(function->build(function_node.getArgumentColumns()));
|
||||
}
|
||||
|
||||
ContextPtr & context;
|
||||
|
@ -65,7 +65,7 @@ QueryTreeNodePtr createResolvedFunction(const ContextPtr & context, const String
|
||||
|
||||
auto function = FunctionFactory::instance().get(name, context);
|
||||
function_node->getArguments().getNodes() = std::move(arguments);
|
||||
function_node->resolveAsFunction(function->build(function_node->getArgumentTypes()));
|
||||
function_node->resolveAsFunction(function->build(function_node->getArgumentColumns()));
|
||||
return function_node;
|
||||
}
|
||||
|
||||
@ -88,7 +88,7 @@ FunctionNodePtr createResolvedAggregateFunction(const String & name, const Query
|
||||
{ argument->getResultType() },
|
||||
parameters,
|
||||
properties);
|
||||
function_node->resolveAsAggregateFunction(aggregate_function);
|
||||
function_node->resolveAsAggregateFunction(std::move(aggregate_function));
|
||||
|
||||
return function_node;
|
||||
}
|
||||
|
@ -56,7 +56,7 @@ public:
|
||||
|
||||
auto multi_if_function = std::make_shared<FunctionNode>("multiIf");
|
||||
multi_if_function->getArguments().getNodes() = std::move(multi_if_arguments);
|
||||
multi_if_function->resolveAsFunction(multi_if_function_ptr->build(multi_if_function->getArgumentTypes()));
|
||||
multi_if_function->resolveAsFunction(multi_if_function_ptr->build(multi_if_function->getArgumentColumns()));
|
||||
node = std::move(multi_if_function);
|
||||
}
|
||||
|
||||
|
@ -52,7 +52,7 @@ QueryTreeNodePtr createCastFunction(QueryTreeNodePtr from, DataTypePtr result_ty
|
||||
auto function_node = std::make_shared<FunctionNode>("_CAST");
|
||||
function_node->getArguments().getNodes() = std::move(arguments);
|
||||
|
||||
function_node->resolveAsFunction(cast_function->build(function_node->getArgumentTypes()));
|
||||
function_node->resolveAsFunction(cast_function->build(function_node->getArgumentColumns()));
|
||||
|
||||
return function_node;
|
||||
}
|
||||
@ -71,7 +71,7 @@ void changeIfArguments(
|
||||
|
||||
auto if_resolver = FunctionFactory::instance().get("if", context);
|
||||
|
||||
if_node.resolveAsFunction(if_resolver->build(if_node.getArgumentTypes()));
|
||||
if_node.resolveAsFunction(if_resolver->build(if_node.getArgumentColumns()));
|
||||
}
|
||||
|
||||
/// transform(value, array_from, array_to, default_value) will be transformed to transform(value, array_from, _CAST(array_to, Array(Enum...)), _CAST(default_value, Enum...))
|
||||
@ -93,7 +93,7 @@ void changeTransformArguments(
|
||||
|
||||
auto transform_resolver = FunctionFactory::instance().get("transform", context);
|
||||
|
||||
transform_node.resolveAsFunction(transform_resolver->build(transform_node.getArgumentTypes()));
|
||||
transform_node.resolveAsFunction(transform_resolver->build(transform_node.getArgumentColumns()));
|
||||
}
|
||||
|
||||
void wrapIntoToString(FunctionNode & function_node, QueryTreeNodePtr arg, ContextPtr context)
|
||||
@ -102,7 +102,7 @@ void wrapIntoToString(FunctionNode & function_node, QueryTreeNodePtr arg, Contex
|
||||
QueryTreeNodes arguments{ std::move(arg) };
|
||||
function_node.getArguments().getNodes() = std::move(arguments);
|
||||
|
||||
function_node.resolveAsFunction(to_string_function->build(function_node.getArgumentTypes()));
|
||||
function_node.resolveAsFunction(to_string_function->build(function_node.getArgumentColumns()));
|
||||
|
||||
assert(isString(function_node.getResultType()));
|
||||
}
|
||||
|
@ -27,7 +27,7 @@ public:
|
||||
return;
|
||||
|
||||
auto result_type = function_node->getResultType();
|
||||
function_node->resolveAsFunction(if_function_ptr->build(function_node->getArgumentTypes()));
|
||||
function_node->resolveAsFunction(if_function_ptr->build(function_node->getArgumentColumns()));
|
||||
}
|
||||
|
||||
private:
|
||||
|
@ -4333,7 +4333,7 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi
|
||||
? AggregateFunctionFactory::instance().get(function_name + "OrNull", argument_types, parameters, properties)
|
||||
: AggregateFunctionFactory::instance().get(function_name, argument_types, parameters, properties);
|
||||
|
||||
function_node.resolveAsWindowFunction(aggregate_function);
|
||||
function_node.resolveAsWindowFunction(std::move(aggregate_function));
|
||||
|
||||
bool window_node_is_identifier = function_node.getWindowNode()->getNodeType() == QueryTreeNodeType::IDENTIFIER;
|
||||
ProjectionName window_projection_name = resolveWindow(function_node.getWindowNode(), scope);
|
||||
@ -4396,7 +4396,7 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi
|
||||
auto aggregate_function = need_add_or_null
|
||||
? AggregateFunctionFactory::instance().get(function_name + "OrNull", argument_types, parameters, properties)
|
||||
: AggregateFunctionFactory::instance().get(function_name, argument_types, parameters, properties);
|
||||
function_node.resolveAsAggregateFunction(aggregate_function);
|
||||
function_node.resolveAsAggregateFunction(std::move(aggregate_function));
|
||||
return result_projection_names;
|
||||
}
|
||||
|
||||
|
@ -122,7 +122,7 @@ public:
|
||||
auto & not_function_arguments = not_function->getArguments().getNodes();
|
||||
not_function_arguments.push_back(std::move(nested_if_function_arguments_nodes[0]));
|
||||
|
||||
not_function->resolveAsFunction(FunctionFactory::instance().get("not", context)->build(not_function->getArgumentTypes()));
|
||||
not_function->resolveAsFunction(FunctionFactory::instance().get("not", context)->build(not_function->getArgumentColumns()));
|
||||
|
||||
function_node_arguments_nodes[0] = std::move(not_function);
|
||||
function_node_arguments_nodes.resize(1);
|
||||
|
@ -75,7 +75,6 @@ public:
|
||||
function_node->getAggregateFunction()->getParameters(),
|
||||
properties);
|
||||
|
||||
auto function_result_type = function_node->getResultType();
|
||||
function_node->resolveAsAggregateFunction(std::move(aggregate_function));
|
||||
}
|
||||
};
|
||||
|
@ -59,7 +59,7 @@ class ValidationChecker : public InDepthQueryTreeVisitor<ValidationChecker>
|
||||
if (!function->isResolved())
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR,
|
||||
"Function {} is not resolved after running {} pass",
|
||||
function->dumpTree(), pass_name);
|
||||
function->toAST()->formatForErrorMessage(), pass_name);
|
||||
}
|
||||
|
||||
public:
|
||||
|
@ -355,6 +355,8 @@ size_t ColumnUnique<ColumnType>::uniqueInsert(const Field & x)
|
||||
void operator() (const Int128 & x) { res = {reinterpret_cast<const char *>(&x), sizeof(x)}; }
|
||||
void operator() (const Int256 & x) { res = {reinterpret_cast<const char *>(&x), sizeof(x)}; }
|
||||
void operator() (const UUID & x) { res = {reinterpret_cast<const char *>(&x), sizeof(x)}; }
|
||||
void operator() (const IPv4 & x) { res = {reinterpret_cast<const char *>(&x), sizeof(x)}; }
|
||||
void operator() (const IPv6 & x) { res = {reinterpret_cast<const char *>(&x), sizeof(x)}; }
|
||||
void operator() (const Float64 & x) { res = {reinterpret_cast<const char *>(&x), sizeof(x)}; }
|
||||
void operator() (const DecimalField<Decimal32> & x) { res = {reinterpret_cast<const char *>(&x), sizeof(x)}; }
|
||||
void operator() (const DecimalField<Decimal64> & x) { res = {reinterpret_cast<const char *>(&x), sizeof(x)}; }
|
||||
|
@ -942,5 +942,7 @@ template class ColumnVector<Int256>;
|
||||
template class ColumnVector<Float32>;
|
||||
template class ColumnVector<Float64>;
|
||||
template class ColumnVector<UUID>;
|
||||
template class ColumnVector<IPv4>;
|
||||
template class ColumnVector<IPv6>;
|
||||
|
||||
}
|
||||
|
@ -557,5 +557,7 @@ extern template class ColumnVector<Int256>;
|
||||
extern template class ColumnVector<Float32>;
|
||||
extern template class ColumnVector<Float64>;
|
||||
extern template class ColumnVector<UUID>;
|
||||
extern template class ColumnVector<IPv4>;
|
||||
extern template class ColumnVector<IPv6>;
|
||||
|
||||
}
|
||||
|
@ -27,5 +27,7 @@ using ColumnFloat32 = ColumnVector<Float32>;
|
||||
using ColumnFloat64 = ColumnVector<Float64>;
|
||||
|
||||
using ColumnUUID = ColumnVector<UUID>;
|
||||
using ColumnIPv4 = ColumnVector<IPv4>;
|
||||
using ColumnIPv6 = ColumnVector<IPv6>;
|
||||
|
||||
}
|
||||
|
@ -72,6 +72,8 @@ INSTANTIATE(Decimal256)
|
||||
INSTANTIATE(DateTime64)
|
||||
INSTANTIATE(char *)
|
||||
INSTANTIATE(UUID)
|
||||
INSTANTIATE(IPv4)
|
||||
INSTANTIATE(IPv6)
|
||||
|
||||
#undef INSTANTIATE
|
||||
|
||||
|
@ -1,4 +1,5 @@
|
||||
#include <limits>
|
||||
#include <type_traits>
|
||||
#include <typeinfo>
|
||||
#include <vector>
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
@ -14,6 +15,12 @@ static constexpr size_t MAX_ROWS = 10000;
|
||||
static const std::vector<size_t> filter_ratios = {1, 2, 5, 11, 32, 64, 100, 1000};
|
||||
static const size_t K = filter_ratios.size();
|
||||
|
||||
template <typename, typename = void >
|
||||
struct HasUnderlyingType : std::false_type {};
|
||||
|
||||
template <typename T>
|
||||
struct HasUnderlyingType<T, std::void_t<typename T::UnderlyingType>> : std::true_type {};
|
||||
|
||||
template <typename T>
|
||||
static MutableColumnPtr createColumn(size_t n)
|
||||
{
|
||||
@ -21,6 +28,9 @@ static MutableColumnPtr createColumn(size_t n)
|
||||
auto & values = column->getData();
|
||||
|
||||
for (size_t i = 0; i < n; ++i)
|
||||
if constexpr (HasUnderlyingType<T>::value)
|
||||
values.push_back(static_cast<typename T::UnderlyingType>(i));
|
||||
else
|
||||
values.push_back(static_cast<T>(i));
|
||||
|
||||
return column;
|
||||
@ -85,6 +95,8 @@ TEST(ColumnVector, Filter)
|
||||
testFilter<Float32>();
|
||||
testFilter<Float64>();
|
||||
testFilter<UUID>();
|
||||
testFilter<IPv4>();
|
||||
testFilter<IPv6>();
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
|
@ -643,6 +643,8 @@
|
||||
M(672, INVALID_SCHEDULER_NODE) \
|
||||
M(673, RESOURCE_ACCESS_DENIED) \
|
||||
M(674, RESOURCE_NOT_FOUND) \
|
||||
M(675, CANNOT_PARSE_IPV4) \
|
||||
M(676, CANNOT_PARSE_IPV6) \
|
||||
\
|
||||
M(999, KEEPER_EXCEPTION) \
|
||||
M(1000, POCO_EXCEPTION) \
|
||||
|
@ -55,6 +55,8 @@ public:
|
||||
T operator() (const Int64 & x) const { return T(x); }
|
||||
T operator() (const Int128 & x) const { return T(x); }
|
||||
T operator() (const UUID & x) const { return T(x.toUnderType()); }
|
||||
T operator() (const IPv4 & x) const { return T(x.toUnderType()); }
|
||||
T operator() (const IPv6 & x) const { return T(x.toUnderType()); }
|
||||
|
||||
T operator() (const Float64 & x) const
|
||||
{
|
||||
|
@ -37,6 +37,8 @@ String FieldVisitorDump::operator() (const UInt256 & x) const { return formatQuo
|
||||
String FieldVisitorDump::operator() (const Int128 & x) const { return formatQuotedWithPrefix(x, "Int128_"); }
|
||||
String FieldVisitorDump::operator() (const Int256 & x) const { return formatQuotedWithPrefix(x, "Int256_"); }
|
||||
String FieldVisitorDump::operator() (const UUID & x) const { return formatQuotedWithPrefix(x, "UUID_"); }
|
||||
String FieldVisitorDump::operator() (const IPv4 & x) const { return formatQuotedWithPrefix(x, "IPv4_"); }
|
||||
String FieldVisitorDump::operator() (const IPv6 & x) const { return formatQuotedWithPrefix(x, "IPv6_"); }
|
||||
String FieldVisitorDump::operator() (const bool & x) const { return formatQuotedWithPrefix(x, "Bool_"); }
|
||||
|
||||
|
||||
|
@ -17,6 +17,8 @@ public:
|
||||
String operator() (const Int128 & x) const;
|
||||
String operator() (const Int256 & x) const;
|
||||
String operator() (const UUID & x) const;
|
||||
String operator() (const IPv4 & x) const;
|
||||
String operator() (const IPv6 & x) const;
|
||||
String operator() (const Float64 & x) const;
|
||||
String operator() (const String & x) const;
|
||||
String operator() (const Array & x) const;
|
||||
|
@ -49,6 +49,20 @@ void FieldVisitorHash::operator() (const UUID & x) const
|
||||
hash.update(x);
|
||||
}
|
||||
|
||||
void FieldVisitorHash::operator() (const IPv4 & x) const
|
||||
{
|
||||
UInt8 type = Field::Types::IPv4;
|
||||
hash.update(type);
|
||||
hash.update(x);
|
||||
}
|
||||
|
||||
void FieldVisitorHash::operator() (const IPv6 & x) const
|
||||
{
|
||||
UInt8 type = Field::Types::IPv6;
|
||||
hash.update(type);
|
||||
hash.update(x);
|
||||
}
|
||||
|
||||
void FieldVisitorHash::operator() (const Float64 & x) const
|
||||
{
|
||||
UInt8 type = Field::Types::Float64;
|
||||
|
@ -23,6 +23,8 @@ public:
|
||||
void operator() (const Int128 & x) const;
|
||||
void operator() (const Int256 & x) const;
|
||||
void operator() (const UUID & x) const;
|
||||
void operator() (const IPv4 & x) const;
|
||||
void operator() (const IPv6 & x) const;
|
||||
void operator() (const Float64 & x) const;
|
||||
void operator() (const String & x) const;
|
||||
void operator() (const Array & x) const;
|
||||
|
@ -33,6 +33,8 @@ bool FieldVisitorSum::operator() (Tuple &) const { throw Exception("Cannot sum T
|
||||
bool FieldVisitorSum::operator() (Map &) const { throw Exception("Cannot sum Maps", ErrorCodes::LOGICAL_ERROR); }
|
||||
bool FieldVisitorSum::operator() (Object &) const { throw Exception("Cannot sum Objects", ErrorCodes::LOGICAL_ERROR); }
|
||||
bool FieldVisitorSum::operator() (UUID &) const { throw Exception("Cannot sum UUIDs", ErrorCodes::LOGICAL_ERROR); }
|
||||
bool FieldVisitorSum::operator() (IPv4 &) const { throw Exception("Cannot sum IPv4s", ErrorCodes::LOGICAL_ERROR); }
|
||||
bool FieldVisitorSum::operator() (IPv6 &) const { throw Exception("Cannot sum IPv6s", ErrorCodes::LOGICAL_ERROR); }
|
||||
|
||||
bool FieldVisitorSum::operator() (AggregateFunctionStateData &) const
|
||||
{
|
||||
|
@ -28,6 +28,8 @@ public:
|
||||
bool operator() (Map &) const;
|
||||
bool operator() (Object &) const;
|
||||
bool operator() (UUID &) const;
|
||||
bool operator() (IPv4 &) const;
|
||||
bool operator() (IPv6 &) const;
|
||||
bool operator() (AggregateFunctionStateData &) const;
|
||||
bool operator() (bool &) const;
|
||||
|
||||
|
@ -65,6 +65,8 @@ String FieldVisitorToString::operator() (const UInt128 & x) const { return forma
|
||||
String FieldVisitorToString::operator() (const UInt256 & x) const { return formatQuoted(x); }
|
||||
String FieldVisitorToString::operator() (const Int256 & x) const { return formatQuoted(x); }
|
||||
String FieldVisitorToString::operator() (const UUID & x) const { return formatQuoted(x); }
|
||||
String FieldVisitorToString::operator() (const IPv4 & x) const { return formatQuoted(x); }
|
||||
String FieldVisitorToString::operator() (const IPv6 & x) const { return formatQuoted(x); }
|
||||
String FieldVisitorToString::operator() (const AggregateFunctionStateData & x) const { return formatQuoted(x.data); }
|
||||
String FieldVisitorToString::operator() (const bool & x) const { return x ? "true" : "false"; }
|
||||
|
||||
|
@ -17,6 +17,8 @@ public:
|
||||
String operator() (const Int128 & x) const;
|
||||
String operator() (const Int256 & x) const;
|
||||
String operator() (const UUID & x) const;
|
||||
String operator() (const IPv4 & x) const;
|
||||
String operator() (const IPv6 & x) const;
|
||||
String operator() (const Float64 & x) const;
|
||||
String operator() (const String & x) const;
|
||||
String operator() (const Array & x) const;
|
||||
|
@ -16,6 +16,8 @@ void FieldVisitorWriteBinary::operator() (const Int128 & x, WriteBuffer & buf) c
|
||||
void FieldVisitorWriteBinary::operator() (const UInt256 & x, WriteBuffer & buf) const { writeBinary(x, buf); }
|
||||
void FieldVisitorWriteBinary::operator() (const Int256 & x, WriteBuffer & buf) const { writeBinary(x, buf); }
|
||||
void FieldVisitorWriteBinary::operator() (const UUID & x, WriteBuffer & buf) const { writeBinary(x, buf); }
|
||||
void FieldVisitorWriteBinary::operator() (const IPv4 & x, WriteBuffer & buf) const { writeBinary(x, buf); }
|
||||
void FieldVisitorWriteBinary::operator() (const IPv6 & x, WriteBuffer & buf) const { writeBinary(x, buf); }
|
||||
void FieldVisitorWriteBinary::operator() (const DecimalField<Decimal32> & x, WriteBuffer & buf) const { writeBinary(x.getValue(), buf); }
|
||||
void FieldVisitorWriteBinary::operator() (const DecimalField<Decimal64> & x, WriteBuffer & buf) const { writeBinary(x.getValue(), buf); }
|
||||
void FieldVisitorWriteBinary::operator() (const DecimalField<Decimal128> & x, WriteBuffer & buf) const { writeBinary(x.getValue(), buf); }
|
||||
|
@ -16,6 +16,8 @@ public:
|
||||
void operator() (const Int128 & x, WriteBuffer & buf) const;
|
||||
void operator() (const Int256 & x, WriteBuffer & buf) const;
|
||||
void operator() (const UUID & x, WriteBuffer & buf) const;
|
||||
void operator() (const IPv4 & x, WriteBuffer & buf) const;
|
||||
void operator() (const IPv6 & x, WriteBuffer & buf) const;
|
||||
void operator() (const Float64 & x, WriteBuffer & buf) const;
|
||||
void operator() (const String & x, WriteBuffer & buf) const;
|
||||
void operator() (const Array & x, WriteBuffer & buf) const;
|
||||
|
@ -259,7 +259,7 @@ inline size_t DefaultHash64(T key)
|
||||
static_cast<UInt64>(key) ^
|
||||
static_cast<UInt64>(key >> 64));
|
||||
}
|
||||
else if constexpr (std::is_same_v<T, DB::UUID>)
|
||||
else if constexpr (std::is_same_v<T, DB::UUID> || std::is_same_v<T, DB::IPv6>)
|
||||
{
|
||||
return intHash64(
|
||||
static_cast<UInt64>(key.toUnderType()) ^
|
||||
|
@ -9,37 +9,56 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
// To be used in formatIPv4, maps a byte to it's string form prefixed with length (so save strlen call).
|
||||
extern const char one_byte_to_string_lookup_table[256][4] =
|
||||
/** Further we want to generate constexpr array of strings with sizes from sequence of unsigned ints [0..N)
|
||||
* in order to use this arrey for fast conversion of unsigned integers to strings
|
||||
*/
|
||||
namespace detail
|
||||
{
|
||||
{1, '0'}, {1, '1'}, {1, '2'}, {1, '3'}, {1, '4'}, {1, '5'}, {1, '6'}, {1, '7'}, {1, '8'}, {1, '9'},
|
||||
{2, '1', '0'}, {2, '1', '1'}, {2, '1', '2'}, {2, '1', '3'}, {2, '1', '4'}, {2, '1', '5'}, {2, '1', '6'}, {2, '1', '7'}, {2, '1', '8'}, {2, '1', '9'},
|
||||
{2, '2', '0'}, {2, '2', '1'}, {2, '2', '2'}, {2, '2', '3'}, {2, '2', '4'}, {2, '2', '5'}, {2, '2', '6'}, {2, '2', '7'}, {2, '2', '8'}, {2, '2', '9'},
|
||||
{2, '3', '0'}, {2, '3', '1'}, {2, '3', '2'}, {2, '3', '3'}, {2, '3', '4'}, {2, '3', '5'}, {2, '3', '6'}, {2, '3', '7'}, {2, '3', '8'}, {2, '3', '9'},
|
||||
{2, '4', '0'}, {2, '4', '1'}, {2, '4', '2'}, {2, '4', '3'}, {2, '4', '4'}, {2, '4', '5'}, {2, '4', '6'}, {2, '4', '7'}, {2, '4', '8'}, {2, '4', '9'},
|
||||
{2, '5', '0'}, {2, '5', '1'}, {2, '5', '2'}, {2, '5', '3'}, {2, '5', '4'}, {2, '5', '5'}, {2, '5', '6'}, {2, '5', '7'}, {2, '5', '8'}, {2, '5', '9'},
|
||||
{2, '6', '0'}, {2, '6', '1'}, {2, '6', '2'}, {2, '6', '3'}, {2, '6', '4'}, {2, '6', '5'}, {2, '6', '6'}, {2, '6', '7'}, {2, '6', '8'}, {2, '6', '9'},
|
||||
{2, '7', '0'}, {2, '7', '1'}, {2, '7', '2'}, {2, '7', '3'}, {2, '7', '4'}, {2, '7', '5'}, {2, '7', '6'}, {2, '7', '7'}, {2, '7', '8'}, {2, '7', '9'},
|
||||
{2, '8', '0'}, {2, '8', '1'}, {2, '8', '2'}, {2, '8', '3'}, {2, '8', '4'}, {2, '8', '5'}, {2, '8', '6'}, {2, '8', '7'}, {2, '8', '8'}, {2, '8', '9'},
|
||||
{2, '9', '0'}, {2, '9', '1'}, {2, '9', '2'}, {2, '9', '3'}, {2, '9', '4'}, {2, '9', '5'}, {2, '9', '6'}, {2, '9', '7'}, {2, '9', '8'}, {2, '9', '9'},
|
||||
{3, '1', '0', '0'}, {3, '1', '0', '1'}, {3, '1', '0', '2'}, {3, '1', '0', '3'}, {3, '1', '0', '4'}, {3, '1', '0', '5'}, {3, '1', '0', '6'}, {3, '1', '0', '7'}, {3, '1', '0', '8'}, {3, '1', '0', '9'},
|
||||
{3, '1', '1', '0'}, {3, '1', '1', '1'}, {3, '1', '1', '2'}, {3, '1', '1', '3'}, {3, '1', '1', '4'}, {3, '1', '1', '5'}, {3, '1', '1', '6'}, {3, '1', '1', '7'}, {3, '1', '1', '8'}, {3, '1', '1', '9'},
|
||||
{3, '1', '2', '0'}, {3, '1', '2', '1'}, {3, '1', '2', '2'}, {3, '1', '2', '3'}, {3, '1', '2', '4'}, {3, '1', '2', '5'}, {3, '1', '2', '6'}, {3, '1', '2', '7'}, {3, '1', '2', '8'}, {3, '1', '2', '9'},
|
||||
{3, '1', '3', '0'}, {3, '1', '3', '1'}, {3, '1', '3', '2'}, {3, '1', '3', '3'}, {3, '1', '3', '4'}, {3, '1', '3', '5'}, {3, '1', '3', '6'}, {3, '1', '3', '7'}, {3, '1', '3', '8'}, {3, '1', '3', '9'},
|
||||
{3, '1', '4', '0'}, {3, '1', '4', '1'}, {3, '1', '4', '2'}, {3, '1', '4', '3'}, {3, '1', '4', '4'}, {3, '1', '4', '5'}, {3, '1', '4', '6'}, {3, '1', '4', '7'}, {3, '1', '4', '8'}, {3, '1', '4', '9'},
|
||||
{3, '1', '5', '0'}, {3, '1', '5', '1'}, {3, '1', '5', '2'}, {3, '1', '5', '3'}, {3, '1', '5', '4'}, {3, '1', '5', '5'}, {3, '1', '5', '6'}, {3, '1', '5', '7'}, {3, '1', '5', '8'}, {3, '1', '5', '9'},
|
||||
{3, '1', '6', '0'}, {3, '1', '6', '1'}, {3, '1', '6', '2'}, {3, '1', '6', '3'}, {3, '1', '6', '4'}, {3, '1', '6', '5'}, {3, '1', '6', '6'}, {3, '1', '6', '7'}, {3, '1', '6', '8'}, {3, '1', '6', '9'},
|
||||
{3, '1', '7', '0'}, {3, '1', '7', '1'}, {3, '1', '7', '2'}, {3, '1', '7', '3'}, {3, '1', '7', '4'}, {3, '1', '7', '5'}, {3, '1', '7', '6'}, {3, '1', '7', '7'}, {3, '1', '7', '8'}, {3, '1', '7', '9'},
|
||||
{3, '1', '8', '0'}, {3, '1', '8', '1'}, {3, '1', '8', '2'}, {3, '1', '8', '3'}, {3, '1', '8', '4'}, {3, '1', '8', '5'}, {3, '1', '8', '6'}, {3, '1', '8', '7'}, {3, '1', '8', '8'}, {3, '1', '8', '9'},
|
||||
{3, '1', '9', '0'}, {3, '1', '9', '1'}, {3, '1', '9', '2'}, {3, '1', '9', '3'}, {3, '1', '9', '4'}, {3, '1', '9', '5'}, {3, '1', '9', '6'}, {3, '1', '9', '7'}, {3, '1', '9', '8'}, {3, '1', '9', '9'},
|
||||
{3, '2', '0', '0'}, {3, '2', '0', '1'}, {3, '2', '0', '2'}, {3, '2', '0', '3'}, {3, '2', '0', '4'}, {3, '2', '0', '5'}, {3, '2', '0', '6'}, {3, '2', '0', '7'}, {3, '2', '0', '8'}, {3, '2', '0', '9'},
|
||||
{3, '2', '1', '0'}, {3, '2', '1', '1'}, {3, '2', '1', '2'}, {3, '2', '1', '3'}, {3, '2', '1', '4'}, {3, '2', '1', '5'}, {3, '2', '1', '6'}, {3, '2', '1', '7'}, {3, '2', '1', '8'}, {3, '2', '1', '9'},
|
||||
{3, '2', '2', '0'}, {3, '2', '2', '1'}, {3, '2', '2', '2'}, {3, '2', '2', '3'}, {3, '2', '2', '4'}, {3, '2', '2', '5'}, {3, '2', '2', '6'}, {3, '2', '2', '7'}, {3, '2', '2', '8'}, {3, '2', '2', '9'},
|
||||
{3, '2', '3', '0'}, {3, '2', '3', '1'}, {3, '2', '3', '2'}, {3, '2', '3', '3'}, {3, '2', '3', '4'}, {3, '2', '3', '5'}, {3, '2', '3', '6'}, {3, '2', '3', '7'}, {3, '2', '3', '8'}, {3, '2', '3', '9'},
|
||||
{3, '2', '4', '0'}, {3, '2', '4', '1'}, {3, '2', '4', '2'}, {3, '2', '4', '3'}, {3, '2', '4', '4'}, {3, '2', '4', '5'}, {3, '2', '4', '6'}, {3, '2', '4', '7'}, {3, '2', '4', '8'}, {3, '2', '4', '9'},
|
||||
{3, '2', '5', '0'}, {3, '2', '5', '1'}, {3, '2', '5', '2'}, {3, '2', '5', '3'}, {3, '2', '5', '4'}, {3, '2', '5', '5'},
|
||||
template <unsigned... digits>
|
||||
struct ToChars
|
||||
{
|
||||
static const char value[];
|
||||
static const size_t size;
|
||||
};
|
||||
|
||||
template <unsigned... digits>
|
||||
constexpr char ToChars<digits...>::value[] = {('0' + digits)..., 0};
|
||||
|
||||
template <unsigned... digits>
|
||||
constexpr size_t ToChars<digits...>::size = sizeof...(digits);
|
||||
|
||||
template <unsigned rem, unsigned... digits>
|
||||
struct Decompose : Decompose<rem / 10, rem % 10, digits...> {};
|
||||
|
||||
template <unsigned... digits>
|
||||
struct Decompose<0, digits...> : ToChars<digits...> {};
|
||||
|
||||
template <>
|
||||
struct Decompose<0> : ToChars<0> {};
|
||||
|
||||
template <unsigned num>
|
||||
struct NumToString : Decompose<num> {};
|
||||
|
||||
template <class T, T... ints>
|
||||
consteval std::array<std::pair<const char *, size_t>, sizeof...(ints)> str_make_array_impl(std::integer_sequence<T, ints...>)
|
||||
{
|
||||
return std::array<std::pair<const char *, size_t>, sizeof...(ints)> { std::pair<const char *, size_t> {NumToString<ints>::value, NumToString<ints>::size}... };
|
||||
}
|
||||
}
|
||||
|
||||
/** str_make_array<N>() - generates static array of std::pair<const char *, size_t> for numbers [0..N), where:
|
||||
* first - null-terminated string representing number
|
||||
* second - size of the string as would returned by strlen()
|
||||
*/
|
||||
template <size_t N>
|
||||
consteval std::array<std::pair<const char *, size_t>, N> str_make_array()
|
||||
{
|
||||
return detail::str_make_array_impl(std::make_integer_sequence<int, N>{});
|
||||
}
|
||||
|
||||
/// This will generate static array of pair<const char *, size_t> for [0..255] at compile time
|
||||
extern constexpr auto one_byte_to_string_lookup_table = str_make_array<256>();
|
||||
|
||||
/// integer logarithm, return ceil(log(value, base)) (the smallest integer greater or equal than log(value, base)
|
||||
static constexpr UInt32 intLog(const UInt32 value, const UInt32 base, const bool carry)
|
||||
{
|
||||
|
@ -3,6 +3,7 @@
|
||||
#include <base/types.h>
|
||||
#include <cstring>
|
||||
#include <algorithm>
|
||||
#include <type_traits>
|
||||
#include <utility>
|
||||
#include <base/range.h>
|
||||
#include <base/unaligned.h>
|
||||
@ -17,6 +18,7 @@ constexpr size_t IPV6_MAX_TEXT_LENGTH = 45; /// Does not count tail zero byt
|
||||
namespace DB
|
||||
{
|
||||
|
||||
extern const std::array<std::pair<const char *, size_t>, 256> one_byte_to_string_lookup_table;
|
||||
|
||||
/** Rewritten inet_ntop6 from http://svn.apache.org/repos/asf/apr/apr/trunk/network_io/unix/inet_pton.c
|
||||
* performs significantly faster than the reference implementation due to the absence of sprintf calls,
|
||||
@ -30,17 +32,38 @@ void formatIPv6(const unsigned char * src, char *& dst, uint8_t zeroed_tail_byte
|
||||
* which should be long enough.
|
||||
* That is "127.0.0.1" becomes 0x7f000001.
|
||||
*
|
||||
* In case of failure returns false and doesn't modify buffer pointed by `dst`.
|
||||
* In case of failure doesn't modify buffer pointed by `dst`.
|
||||
*
|
||||
* @param src - input string, expected to be non-null and null-terminated right after the IPv4 string value.
|
||||
* WARNING - this function is adapted to work with ReadBuffer, where src is the position reference (ReadBuffer::position())
|
||||
* and eof is the ReadBuffer::eof() - therefore algorithm below does not rely on buffer's continuity.
|
||||
* To parse strings use overloads below.
|
||||
*
|
||||
* @param src - iterator (reference to pointer) over input string - warning - continuity is not guaranteed.
|
||||
* @param eof - function returning true if iterator riched the end - warning - can break iterator's continuity.
|
||||
* @param dst - where to put output bytes, expected to be non-null and at IPV4_BINARY_LENGTH-long.
|
||||
* @return false if parsing failed, true otherwise.
|
||||
* @param first_octet - preparsed first octet
|
||||
* @return - true if parsed successfully, false otherwise.
|
||||
*/
|
||||
inline bool parseIPv4(const char * src, unsigned char * dst)
|
||||
template <typename T, typename EOFfunction>
|
||||
requires (std::is_same<typename std::remove_cv<T>::type, char>::value)
|
||||
inline bool parseIPv4(T * &src, EOFfunction eof, unsigned char * dst, int32_t first_octet = -1)
|
||||
{
|
||||
if (src == nullptr || first_octet > 255)
|
||||
return false;
|
||||
|
||||
UInt32 result = 0;
|
||||
for (int offset = 24; offset >= 0; offset -= 8)
|
||||
int offset = 24;
|
||||
if (first_octet >= 0)
|
||||
{
|
||||
result |= first_octet << offset;
|
||||
offset -= 8;
|
||||
}
|
||||
|
||||
for (; true; offset -= 8, ++src)
|
||||
{
|
||||
if (eof())
|
||||
return false;
|
||||
|
||||
UInt32 value = 0;
|
||||
size_t len = 0;
|
||||
while (isNumericASCII(*src) && len <= 3)
|
||||
@ -48,137 +71,333 @@ inline bool parseIPv4(const char * src, unsigned char * dst)
|
||||
value = value * 10 + (*src - '0');
|
||||
++len;
|
||||
++src;
|
||||
if (eof())
|
||||
break;
|
||||
}
|
||||
if (len == 0 || value > 255 || (offset > 0 && *src != '.'))
|
||||
if (len == 0 || value > 255 || (offset > 0 && (eof() || *src != '.')))
|
||||
return false;
|
||||
result |= value << offset;
|
||||
++src;
|
||||
}
|
||||
if (*(src - 1) != '\0')
|
||||
return false;
|
||||
|
||||
if (offset == 0)
|
||||
break;
|
||||
}
|
||||
|
||||
if constexpr (std::endian::native == std::endian::little)
|
||||
memcpy(dst, &result, sizeof(result));
|
||||
else
|
||||
reverseMemcpy(dst, &result, sizeof(result));
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
/// returns pointer to the right after parsed sequence or null on failed parsing
|
||||
inline const char * parseIPv4(const char * src, const char * end, unsigned char * dst)
|
||||
{
|
||||
if (parseIPv4(src, [&src, end](){ return src == end; }, dst))
|
||||
return src;
|
||||
return nullptr;
|
||||
}
|
||||
|
||||
/// returns true if whole buffer was parsed successfully
|
||||
inline bool parseIPv4whole(const char * src, const char * end, unsigned char * dst)
|
||||
{
|
||||
return parseIPv4(src, end, dst) == end;
|
||||
}
|
||||
|
||||
/// returns pointer to the right after parsed sequence or null on failed parsing
|
||||
inline const char * parseIPv4(const char * src, unsigned char * dst)
|
||||
{
|
||||
if (parseIPv4(src, [](){ return false; }, dst))
|
||||
return src;
|
||||
return nullptr;
|
||||
}
|
||||
|
||||
/// returns true if whole null-terminated string was parsed successfully
|
||||
inline bool parseIPv4whole(const char * src, unsigned char * dst)
|
||||
{
|
||||
const char * end = parseIPv4(src, dst);
|
||||
return end != nullptr && *end == '\0';
|
||||
}
|
||||
|
||||
/** Unsafe (no bounds-checking for src nor dst), optimized version of parsing IPv6 string.
|
||||
*
|
||||
* Slightly altered implementation from http://svn.apache.org/repos/asf/apr/apr/trunk/network_io/unix/inet_pton.c
|
||||
* Parses the input string `src` and stores binary big-endian value into buffer pointed by `dst`,
|
||||
* which should be long enough. In case of failure zeroes
|
||||
* IPV6_BINARY_LENGTH bytes of buffer pointed by `dst`.
|
||||
* which should be long enough. In case of failure zeroes IPV6_BINARY_LENGTH bytes of buffer pointed by `dst`.
|
||||
*
|
||||
* @param src - input string, expected to be non-null and null-terminated right after the IPv6 string value.
|
||||
* WARNING - this function is adapted to work with ReadBuffer, where src is the position reference (ReadBuffer::position())
|
||||
* and eof is the ReadBuffer::eof() - therefore algorithm below does not rely on buffer's continuity.
|
||||
* To parse strings use overloads below.
|
||||
*
|
||||
* @param src - iterator (reference to pointer) over input string - warning - continuity is not guaranteed.
|
||||
* @param eof - function returning true if iterator riched the end - warning - can break iterator's continuity.
|
||||
* @param dst - where to put output bytes, expected to be non-null and at IPV6_BINARY_LENGTH-long.
|
||||
* @return false if parsing failed, true otherwise.
|
||||
* @param first_block - preparsed first block
|
||||
* @return - true if parsed successfully, false otherwise.
|
||||
*/
|
||||
inline bool parseIPv6(const char * src, unsigned char * dst)
|
||||
template <typename T, typename EOFfunction>
|
||||
requires (std::is_same<typename std::remove_cv<T>::type, char>::value)
|
||||
inline bool parseIPv6(T * &src, EOFfunction eof, unsigned char * dst, int32_t first_block = -1)
|
||||
{
|
||||
const auto clear_dst = [dst]()
|
||||
{
|
||||
memset(dst, '\0', IPV6_BINARY_LENGTH);
|
||||
std::memset(dst, '\0', IPV6_BINARY_LENGTH);
|
||||
return false;
|
||||
};
|
||||
|
||||
/// Leading :: requires some special handling.
|
||||
if (src == nullptr || eof())
|
||||
return clear_dst();
|
||||
|
||||
int groups = 0; /// number of parsed groups
|
||||
unsigned char * iter = dst; /// iterator over dst buffer
|
||||
unsigned char * zptr = nullptr; /// pointer into dst buffer array where all-zeroes block ("::") is started
|
||||
|
||||
std::memset(dst, '\0', IPV6_BINARY_LENGTH);
|
||||
|
||||
if (first_block >= 0)
|
||||
{
|
||||
*iter++ = static_cast<unsigned char>((first_block >> 8) & 0xffu);
|
||||
*iter++ = static_cast<unsigned char>(first_block & 0xffu);
|
||||
if (*src == ':')
|
||||
if (*++src != ':')
|
||||
return clear_dst();
|
||||
|
||||
unsigned char tmp[IPV6_BINARY_LENGTH]{};
|
||||
unsigned char * tp = tmp;
|
||||
unsigned char * endp = tp + IPV6_BINARY_LENGTH;
|
||||
const char * curtok = src;
|
||||
bool saw_xdigit = false;
|
||||
UInt32 val{};
|
||||
unsigned char * colonp = nullptr;
|
||||
|
||||
/// Assuming zero-terminated string.
|
||||
while (char ch = *src++)
|
||||
{
|
||||
UInt8 num = unhex(ch);
|
||||
|
||||
if (num != 0xFF)
|
||||
{
|
||||
val <<= 4;
|
||||
val |= num;
|
||||
if (val > 0xffffu)
|
||||
return clear_dst();
|
||||
|
||||
saw_xdigit = true;
|
||||
continue;
|
||||
zptr = iter;
|
||||
++src;
|
||||
}
|
||||
++groups;
|
||||
}
|
||||
|
||||
if (ch == ':')
|
||||
bool group_start = true;
|
||||
|
||||
while (!eof() && groups < 8)
|
||||
{
|
||||
curtok = src;
|
||||
if (!saw_xdigit)
|
||||
if (*src == ':')
|
||||
{
|
||||
if (colonp)
|
||||
++src;
|
||||
if (eof()) /// trailing colon is not allowed
|
||||
return clear_dst();
|
||||
|
||||
colonp = tp;
|
||||
group_start = true;
|
||||
|
||||
if (*src == ':')
|
||||
{
|
||||
if (zptr != nullptr) /// multiple all-zeroes blocks are not allowed
|
||||
return clear_dst();
|
||||
zptr = iter;
|
||||
++src;
|
||||
continue;
|
||||
}
|
||||
if (groups == 0) /// leading colon is not allowed
|
||||
return clear_dst();
|
||||
}
|
||||
|
||||
if (tp + sizeof(UInt16) > endp)
|
||||
if (*src == '.') /// mixed IPv4 parsing
|
||||
{
|
||||
if (groups <= 1 && zptr == nullptr) /// IPv4 block can't be the first
|
||||
return clear_dst();
|
||||
|
||||
*tp++ = static_cast<unsigned char>((val >> 8) & 0xffu);
|
||||
*tp++ = static_cast<unsigned char>(val & 0xffu);
|
||||
saw_xdigit = false;
|
||||
val = 0;
|
||||
continue;
|
||||
}
|
||||
++src;
|
||||
if (eof())
|
||||
return clear_dst();
|
||||
|
||||
if (ch == '.' && (tp + IPV4_BINARY_LENGTH) <= endp)
|
||||
/// last parsed group should be reinterpreted as a decimal value - it's the first octet of IPv4
|
||||
--groups;
|
||||
iter -= 2;
|
||||
|
||||
UInt16 num = 0;
|
||||
for (int i = 0; i < 2; ++i)
|
||||
{
|
||||
if (!parseIPv4(curtok, tp))
|
||||
unsigned char first = (iter[i] >> 4) & 0x0fu;
|
||||
unsigned char second = iter[i] & 0x0fu;
|
||||
if (first > 9 || second > 9)
|
||||
return clear_dst();
|
||||
(num *= 100) += first * 10 + second;
|
||||
}
|
||||
if (num > 255)
|
||||
return clear_dst();
|
||||
|
||||
/// parse IPv4 with known first octet
|
||||
if (!parseIPv4(src, eof, iter, num))
|
||||
return clear_dst();
|
||||
|
||||
if constexpr (std::endian::native == std::endian::little)
|
||||
std::reverse(tp, tp + IPV4_BINARY_LENGTH);
|
||||
std::reverse(iter, iter + IPV4_BINARY_LENGTH);
|
||||
|
||||
tp += IPV4_BINARY_LENGTH;
|
||||
saw_xdigit = false;
|
||||
break; /* '\0' was seen by ipv4_scan(). */
|
||||
iter += 4;
|
||||
groups += 2;
|
||||
break; /// IPv4 block is the last - end of parsing
|
||||
}
|
||||
|
||||
return clear_dst();
|
||||
}
|
||||
if (!group_start) /// end of parsing
|
||||
break;
|
||||
group_start = false;
|
||||
|
||||
if (saw_xdigit)
|
||||
UInt16 val = 0; /// current decoded group
|
||||
int xdigits = 0; /// number of decoded hex digits in current group
|
||||
|
||||
for (; !eof() && xdigits < 4; ++src, ++xdigits)
|
||||
{
|
||||
if (tp + sizeof(UInt16) > endp)
|
||||
UInt8 num = unhex(*src);
|
||||
if (num == 0xFF)
|
||||
break;
|
||||
(val <<= 4) |= num;
|
||||
}
|
||||
|
||||
if (xdigits == 0) /// end of parsing
|
||||
break;
|
||||
|
||||
*iter++ = static_cast<unsigned char>((val >> 8) & 0xffu);
|
||||
*iter++ = static_cast<unsigned char>(val & 0xffu);
|
||||
++groups;
|
||||
}
|
||||
|
||||
/// either all 8 groups or all-zeroes block should be present
|
||||
if (groups < 8 && zptr == nullptr)
|
||||
return clear_dst();
|
||||
|
||||
*tp++ = static_cast<unsigned char>((val >> 8) & 0xffu);
|
||||
*tp++ = static_cast<unsigned char>(val & 0xffu);
|
||||
}
|
||||
|
||||
if (colonp)
|
||||
if (zptr != nullptr) /// process all-zeroes block
|
||||
{
|
||||
/*
|
||||
* Since some memmove()'s erroneously fail to handle
|
||||
* overlapping regions, we'll do the shift by hand.
|
||||
*/
|
||||
const auto n = tp - colonp;
|
||||
|
||||
for (int i = 1; i <= n; ++i)
|
||||
{
|
||||
endp[- i] = colonp[n - i];
|
||||
colonp[n - i] = 0;
|
||||
}
|
||||
tp = endp;
|
||||
size_t msize = iter - zptr;
|
||||
std::memmove(dst + IPV6_BINARY_LENGTH - msize, zptr, msize);
|
||||
std::memset(zptr, '\0', IPV6_BINARY_LENGTH - (iter - dst));
|
||||
}
|
||||
|
||||
if (tp != endp)
|
||||
return clear_dst();
|
||||
|
||||
memcpy(dst, tmp, sizeof(tmp));
|
||||
return true;
|
||||
}
|
||||
|
||||
/// returns pointer to the right after parsed sequence or null on failed parsing
|
||||
inline const char * parseIPv6(const char * src, const char * end, unsigned char * dst)
|
||||
{
|
||||
if (parseIPv6(src, [&src, end](){ return src == end; }, dst))
|
||||
return src;
|
||||
return nullptr;
|
||||
}
|
||||
|
||||
/// returns true if whole buffer was parsed successfully
|
||||
inline bool parseIPv6whole(const char * src, const char * end, unsigned char * dst)
|
||||
{
|
||||
return parseIPv6(src, end, dst) == end;
|
||||
}
|
||||
|
||||
/// returns pointer to the right after parsed sequence or null on failed parsing
|
||||
inline const char * parseIPv6(const char * src, unsigned char * dst)
|
||||
{
|
||||
if (parseIPv6(src, [](){ return false; }, dst))
|
||||
return src;
|
||||
return nullptr;
|
||||
}
|
||||
|
||||
/// returns true if whole null-terminated string was parsed successfully
|
||||
inline bool parseIPv6whole(const char * src, unsigned char * dst)
|
||||
{
|
||||
const char * end = parseIPv6(src, dst);
|
||||
return end != nullptr && *end == '\0';
|
||||
}
|
||||
|
||||
/** Unsafe (no bounds-checking for src nor dst), optimized version of parsing IPv6 string.
|
||||
*
|
||||
* Parses the input string `src` IPv6 or possible IPv4 into IPv6 and stores binary big-endian value into buffer pointed by `dst`,
|
||||
* which should be long enough. In case of failure zeroes IPV6_BINARY_LENGTH bytes of buffer pointed by `dst`.
|
||||
*
|
||||
* WARNING - this function is adapted to work with ReadBuffer, where src is the position reference (ReadBuffer::position())
|
||||
* and eof is the ReadBuffer::eof() - therefore algorithm below does not rely on buffer's continuity.
|
||||
*
|
||||
* @param src - iterator (reference to pointer) over input string - warning - continuity is not guaranteed.
|
||||
* @param eof - function returning true if iterator riched the end - warning - can break iterator's continuity.
|
||||
* @param dst - where to put output bytes, expected to be non-null and at IPV6_BINARY_LENGTH-long.
|
||||
* @return - true if parsed successfully, false otherwise.
|
||||
*/
|
||||
template <typename T, typename EOFfunction>
|
||||
requires (std::is_same<typename std::remove_cv<T>::type, char>::value)
|
||||
inline bool parseIPv6orIPv4(T * &src, EOFfunction eof, unsigned char * dst)
|
||||
{
|
||||
const auto clear_dst = [dst]()
|
||||
{
|
||||
std::memset(dst, '\0', IPV6_BINARY_LENGTH);
|
||||
return false;
|
||||
};
|
||||
|
||||
if (src == nullptr)
|
||||
return clear_dst();
|
||||
|
||||
bool leading_zero = false;
|
||||
uint16_t val = 0;
|
||||
int digits = 0;
|
||||
/// parse up to 4 first digits as hexadecimal
|
||||
for (; !eof() && digits < 4; ++src, ++digits)
|
||||
{
|
||||
if (*src == ':' || *src == '.')
|
||||
break;
|
||||
|
||||
if (digits == 0 && *src == '0')
|
||||
leading_zero = true;
|
||||
|
||||
UInt8 num = unhex(*src);
|
||||
if (num == 0xFF)
|
||||
return clear_dst();
|
||||
(val <<= 4) |= num;
|
||||
}
|
||||
|
||||
if (eof())
|
||||
return clear_dst();
|
||||
|
||||
if (*src == ':') /// IPv6
|
||||
{
|
||||
if (digits == 0) /// leading colon - no preparsed group
|
||||
return parseIPv6(src, eof, dst);
|
||||
++src;
|
||||
return parseIPv6(src, eof, dst, val); /// parse with first preparsed group
|
||||
}
|
||||
|
||||
if (*src == '.') /// IPv4
|
||||
{
|
||||
/// should has some digits
|
||||
if (digits == 0)
|
||||
return clear_dst();
|
||||
/// should not has leading zeroes, should has no more than 3 digits
|
||||
if ((leading_zero && digits > 1) || digits > 3)
|
||||
return clear_dst();
|
||||
|
||||
/// recode first group as decimal
|
||||
UInt16 num = 0;
|
||||
for (int exp = 1; exp < 1000; exp *= 10)
|
||||
{
|
||||
int n = val & 0x0fu;
|
||||
if (n > 9)
|
||||
return clear_dst();
|
||||
num += n * exp;
|
||||
val >>= 4;
|
||||
}
|
||||
if (num > 255)
|
||||
return clear_dst();
|
||||
|
||||
++src;
|
||||
if (!parseIPv4(src, eof, dst, num)) /// try to parse as IPv4 with preparsed first octet
|
||||
return clear_dst();
|
||||
|
||||
/// convert into IPv6
|
||||
if constexpr (std::endian::native == std::endian::little)
|
||||
{
|
||||
dst[15] = dst[0]; dst[0] = 0;
|
||||
dst[14] = dst[1]; dst[1] = 0;
|
||||
dst[13] = dst[2]; dst[2] = 0;
|
||||
dst[12] = dst[3]; dst[3] = 0;
|
||||
}
|
||||
else
|
||||
{
|
||||
dst[15] = dst[3]; dst[3] = 0;
|
||||
dst[14] = dst[2]; dst[2] = 0;
|
||||
dst[13] = dst[1]; dst[1] = 0;
|
||||
dst[12] = dst[0]; dst[0] = 0;
|
||||
}
|
||||
|
||||
dst[11] = 0xff;
|
||||
dst[10] = 0xff;
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
return clear_dst();
|
||||
}
|
||||
|
||||
/** Format 4-byte binary sequesnce as IPv4 text: 'aaa.bbb.ccc.ddd',
|
||||
* expects in out to be in BE-format, that is 0x7f000001 => "127.0.0.1".
|
||||
*
|
||||
@ -198,22 +417,27 @@ inline bool parseIPv6(const char * src, unsigned char * dst)
|
||||
* formatIPv4(&0x7f000001, dst, mask_tail_octets = 1, "0");
|
||||
* > dst == "127.0.0.0"
|
||||
*/
|
||||
inline void formatIPv4(const unsigned char * src, char *& dst, uint8_t mask_tail_octets = 0, const char * mask_string = "xxx")
|
||||
inline void formatIPv4(const unsigned char * src, size_t src_size, char *& dst, uint8_t mask_tail_octets = 0, const char * mask_string = "xxx")
|
||||
{
|
||||
extern const char one_byte_to_string_lookup_table[256][4];
|
||||
|
||||
const size_t mask_length = mask_string ? strlen(mask_string) : 0;
|
||||
const size_t limit = std::min(IPV4_BINARY_LENGTH, IPV4_BINARY_LENGTH - mask_tail_octets);
|
||||
for (size_t octet = 0; octet < limit; ++octet)
|
||||
const size_t padding = std::min(4 - src_size, limit);
|
||||
|
||||
for (size_t octet = 0; octet < padding; ++octet)
|
||||
{
|
||||
*dst++ = '0';
|
||||
*dst++ = '.';
|
||||
}
|
||||
|
||||
for (size_t octet = 4 - src_size; octet < limit; ++octet)
|
||||
{
|
||||
uint8_t value = 0;
|
||||
if constexpr (std::endian::native == std::endian::little)
|
||||
value = static_cast<uint8_t>(src[IPV4_BINARY_LENGTH - octet - 1]);
|
||||
else
|
||||
value = static_cast<uint8_t>(src[octet]);
|
||||
const auto * rep = one_byte_to_string_lookup_table[value];
|
||||
const uint8_t len = rep[0];
|
||||
const char* str = rep + 1;
|
||||
const uint8_t len = one_byte_to_string_lookup_table[value].second;
|
||||
const char* str = one_byte_to_string_lookup_table[value].first;
|
||||
|
||||
memcpy(dst, str, len);
|
||||
dst += len;
|
||||
@ -231,4 +455,9 @@ inline void formatIPv4(const unsigned char * src, char *& dst, uint8_t mask_tail
|
||||
dst[-1] = '\0';
|
||||
}
|
||||
|
||||
inline void formatIPv4(const unsigned char * src, char *& dst, uint8_t mask_tail_octets = 0, const char * mask_string = "xxx")
|
||||
{
|
||||
formatIPv4(src, 4, dst, mask_tail_octets, mask_string);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -18,6 +18,9 @@ namespace DB
|
||||
}
|
||||
}
|
||||
|
||||
template<typename T, typename ... U>
|
||||
concept is_any_of = (std::same_as<T, U> || ...);
|
||||
|
||||
|
||||
/** Checks type by comparing typeid.
|
||||
* The exact match of the type is checked. That is, cast to the ancestor will be unsuccessful.
|
||||
|
@ -51,6 +51,18 @@ inline Field getBinaryValue(UInt8 type, ReadBuffer & buf)
|
||||
readBinary(value, buf);
|
||||
return value;
|
||||
}
|
||||
case Field::Types::IPv4:
|
||||
{
|
||||
IPv4 value;
|
||||
readBinary(value, buf);
|
||||
return value;
|
||||
}
|
||||
case Field::Types::IPv6:
|
||||
{
|
||||
IPv6 value;
|
||||
readBinary(value.toUnderType(), buf);
|
||||
return value;
|
||||
}
|
||||
case Field::Types::Int64:
|
||||
{
|
||||
Int64 value;
|
||||
@ -583,6 +595,8 @@ String fieldTypeToString(Field::Types::Which type)
|
||||
case Field::Types::Which::UInt128: return "UInt128";
|
||||
case Field::Types::Which::UInt256: return "UInt256";
|
||||
case Field::Types::Which::UUID: return "UUID";
|
||||
case Field::Types::Which::IPv4: return "IPv4";
|
||||
case Field::Types::Which::IPv6: return "IPv6";
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -13,6 +13,7 @@
|
||||
#include <Core/Defines.h>
|
||||
#include <Core/DecimalFunctions.h>
|
||||
#include <Core/UUID.h>
|
||||
#include <base/IPv4andIPv6.h>
|
||||
#include <base/DayNum.h>
|
||||
#include <base/strong_typedef.h>
|
||||
#include <base/EnumReflection.h>
|
||||
@ -192,6 +193,8 @@ template <> struct NearestFieldTypeImpl<UInt32> { using Type = UInt64; };
|
||||
|
||||
template <> struct NearestFieldTypeImpl<DayNum> { using Type = UInt64; };
|
||||
template <> struct NearestFieldTypeImpl<UUID> { using Type = UUID; };
|
||||
template <> struct NearestFieldTypeImpl<IPv4> { using Type = IPv4; };
|
||||
template <> struct NearestFieldTypeImpl<IPv6> { using Type = IPv6; };
|
||||
template <> struct NearestFieldTypeImpl<Int16> { using Type = Int64; };
|
||||
template <> struct NearestFieldTypeImpl<Int32> { using Type = Int64; };
|
||||
|
||||
@ -292,6 +295,8 @@ public:
|
||||
UUID = 27,
|
||||
Bool = 28,
|
||||
Object = 29,
|
||||
IPv4 = 30,
|
||||
IPv6 = 31,
|
||||
};
|
||||
};
|
||||
|
||||
@ -468,6 +473,8 @@ public:
|
||||
case Types::Int128: return get<Int128>() < rhs.get<Int128>();
|
||||
case Types::Int256: return get<Int256>() < rhs.get<Int256>();
|
||||
case Types::UUID: return get<UUID>() < rhs.get<UUID>();
|
||||
case Types::IPv4: return get<IPv4>() < rhs.get<IPv4>();
|
||||
case Types::IPv6: return get<IPv6>() < rhs.get<IPv6>();
|
||||
case Types::Float64: return get<Float64>() < rhs.get<Float64>();
|
||||
case Types::String: return get<String>() < rhs.get<String>();
|
||||
case Types::Array: return get<Array>() < rhs.get<Array>();
|
||||
@ -507,6 +514,8 @@ public:
|
||||
case Types::Int128: return get<Int128>() <= rhs.get<Int128>();
|
||||
case Types::Int256: return get<Int256>() <= rhs.get<Int256>();
|
||||
case Types::UUID: return get<UUID>().toUnderType() <= rhs.get<UUID>().toUnderType();
|
||||
case Types::IPv4: return get<IPv4>() <= rhs.get<IPv4>();
|
||||
case Types::IPv6: return get<IPv6>() <= rhs.get<IPv6>();
|
||||
case Types::Float64: return get<Float64>() <= rhs.get<Float64>();
|
||||
case Types::String: return get<String>() <= rhs.get<String>();
|
||||
case Types::Array: return get<Array>() <= rhs.get<Array>();
|
||||
@ -547,6 +556,8 @@ public:
|
||||
return std::bit_cast<UInt64>(get<Float64>()) == std::bit_cast<UInt64>(rhs.get<Float64>());
|
||||
}
|
||||
case Types::UUID: return get<UUID>() == rhs.get<UUID>();
|
||||
case Types::IPv4: return get<IPv4>() == rhs.get<IPv4>();
|
||||
case Types::IPv6: return get<IPv6>() == rhs.get<IPv6>();
|
||||
case Types::String: return get<String>() == rhs.get<String>();
|
||||
case Types::Array: return get<Array>() == rhs.get<Array>();
|
||||
case Types::Tuple: return get<Tuple>() == rhs.get<Tuple>();
|
||||
@ -586,6 +597,8 @@ public:
|
||||
case Types::Int128: return f(field.template get<Int128>());
|
||||
case Types::Int256: return f(field.template get<Int256>());
|
||||
case Types::UUID: return f(field.template get<UUID>());
|
||||
case Types::IPv4: return f(field.template get<IPv4>());
|
||||
case Types::IPv6: return f(field.template get<IPv6>());
|
||||
case Types::Float64: return f(field.template get<Float64>());
|
||||
case Types::String: return f(field.template get<String>());
|
||||
case Types::Array: return f(field.template get<Array>());
|
||||
@ -612,7 +625,7 @@ public:
|
||||
|
||||
private:
|
||||
std::aligned_union_t<DBMS_MIN_FIELD_SIZE - sizeof(Types::Which),
|
||||
Null, UInt64, UInt128, UInt256, Int64, Int128, Int256, UUID, Float64, String, Array, Tuple, Map,
|
||||
Null, UInt64, UInt128, UInt256, Int64, Int128, Int256, UUID, IPv4, IPv6, Float64, String, Array, Tuple, Map,
|
||||
DecimalField<Decimal32>, DecimalField<Decimal64>, DecimalField<Decimal128>, DecimalField<Decimal256>,
|
||||
AggregateFunctionStateData
|
||||
> storage;
|
||||
@ -747,6 +760,8 @@ template <> struct Field::TypeToEnum<Int64> { static constexpr Types::Which va
|
||||
template <> struct Field::TypeToEnum<Int128> { static constexpr Types::Which value = Types::Int128; };
|
||||
template <> struct Field::TypeToEnum<Int256> { static constexpr Types::Which value = Types::Int256; };
|
||||
template <> struct Field::TypeToEnum<UUID> { static constexpr Types::Which value = Types::UUID; };
|
||||
template <> struct Field::TypeToEnum<IPv4> { static constexpr Types::Which value = Types::IPv4; };
|
||||
template <> struct Field::TypeToEnum<IPv6> { static constexpr Types::Which value = Types::IPv6; };
|
||||
template <> struct Field::TypeToEnum<Float64> { static constexpr Types::Which value = Types::Float64; };
|
||||
template <> struct Field::TypeToEnum<String> { static constexpr Types::Which value = Types::String; };
|
||||
template <> struct Field::TypeToEnum<Array> { static constexpr Types::Which value = Types::Array; };
|
||||
@ -769,6 +784,8 @@ template <> struct Field::EnumToType<Field::Types::Int64> { using Type = Int64
|
||||
template <> struct Field::EnumToType<Field::Types::Int128> { using Type = Int128; };
|
||||
template <> struct Field::EnumToType<Field::Types::Int256> { using Type = Int256; };
|
||||
template <> struct Field::EnumToType<Field::Types::UUID> { using Type = UUID; };
|
||||
template <> struct Field::EnumToType<Field::Types::IPv4> { using Type = IPv4; };
|
||||
template <> struct Field::EnumToType<Field::Types::IPv6> { using Type = IPv6; };
|
||||
template <> struct Field::EnumToType<Field::Types::Float64> { using Type = Float64; };
|
||||
template <> struct Field::EnumToType<Field::Types::String> { using Type = String; };
|
||||
template <> struct Field::EnumToType<Field::Types::Array> { using Type = Array; };
|
||||
|
@ -12,6 +12,9 @@ using DataTypes = std::vector<DataTypePtr>;
|
||||
|
||||
struct Array;
|
||||
|
||||
/* Generic class for all functions.
|
||||
* Represents interface for function signature.
|
||||
*/
|
||||
class IResolvedFunction
|
||||
{
|
||||
public:
|
||||
|
@ -20,6 +20,7 @@
|
||||
#include <DataTypes/DataTypeDateTime64.h>
|
||||
#include <DataTypes/DataTypeEnum.h>
|
||||
#include <DataTypes/DataTypeUUID.h>
|
||||
#include <DataTypes/DataTypeIPv4andIPv6.h>
|
||||
#include <Columns/IColumn.h>
|
||||
#include <Columns/ColumnDecimal.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
@ -696,6 +697,8 @@ private:
|
||||
SortingQueueImpl<SpecializedSingleColumnSortCursor<ColumnDecimal<DateTime64>>, strategy>,
|
||||
|
||||
SortingQueueImpl<SpecializedSingleColumnSortCursor<ColumnVector<UUID>>, strategy>,
|
||||
SortingQueueImpl<SpecializedSingleColumnSortCursor<ColumnVector<IPv4>>, strategy>,
|
||||
SortingQueueImpl<SpecializedSingleColumnSortCursor<ColumnVector<IPv6>>, strategy>,
|
||||
|
||||
SortingQueueImpl<SpecializedSingleColumnSortCursor<ColumnString>, strategy>,
|
||||
SortingQueueImpl<SpecializedSingleColumnSortCursor<ColumnFixedString>, strategy>,
|
||||
|
@ -47,6 +47,8 @@ TYPEID_MAP(Int256)
|
||||
TYPEID_MAP(Float32)
|
||||
TYPEID_MAP(Float64)
|
||||
TYPEID_MAP(UUID)
|
||||
TYPEID_MAP(IPv4)
|
||||
TYPEID_MAP(IPv6)
|
||||
|
||||
TYPEID_MAP(Decimal32)
|
||||
TYPEID_MAP(Decimal64)
|
||||
|
@ -8,6 +8,7 @@
|
||||
#include <base/Decimal.h>
|
||||
#include <base/defines.h>
|
||||
#include <base/UUID.h>
|
||||
#include <base/IPv4andIPv6.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -84,6 +85,8 @@ enum class TypeIndex
|
||||
LowCardinality,
|
||||
Map,
|
||||
Object,
|
||||
IPv4,
|
||||
IPv6,
|
||||
};
|
||||
|
||||
|
||||
|
@ -160,6 +160,9 @@ class DataTypeDate32;
|
||||
class DataTypeString;
|
||||
class DataTypeFixedString;
|
||||
class DataTypeUUID;
|
||||
template <typename IPv> class DataTypeIP;
|
||||
using DataTypeIPv4 = DataTypeIP<IPv4>;
|
||||
using DataTypeIPv6 = DataTypeIP<IPv6>;
|
||||
class DataTypeDateTime;
|
||||
class DataTypeDateTime64;
|
||||
template <typename T> class DataTypeEnum;
|
||||
@ -206,6 +209,8 @@ bool callOnIndexAndDataType(TypeIndex number, F && f, ExtraArgs && ... args)
|
||||
case TypeIndex::Enum16: return f(TypePair<DataTypeEnum<Int16>, T>(), std::forward<ExtraArgs>(args)...);
|
||||
|
||||
case TypeIndex::UUID: return f(TypePair<DataTypeUUID, T>(), std::forward<ExtraArgs>(args)...);
|
||||
case TypeIndex::IPv4: return f(TypePair<DataTypeIPv4, T>(), std::forward<ExtraArgs>(args)...);
|
||||
case TypeIndex::IPv6: return f(TypePair<DataTypeIPv6, T>(), std::forward<ExtraArgs>(args)...);
|
||||
|
||||
default:
|
||||
break;
|
||||
|
@ -1,29 +0,0 @@
|
||||
#include <DataTypes/Serializations/SerializationIP.h>
|
||||
#include <DataTypes/DataTypeFactory.h>
|
||||
#include <DataTypes/DataTypeCustom.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
void registerDataTypeDomainIPv4AndIPv6(DataTypeFactory & factory)
|
||||
{
|
||||
factory.registerSimpleDataTypeCustom("IPv4", []
|
||||
{
|
||||
auto type = DataTypeFactory::instance().get("UInt32");
|
||||
return std::make_pair(type, std::make_unique<DataTypeCustomDesc>(
|
||||
std::make_unique<DataTypeCustomFixedName>("IPv4"), std::make_unique<SerializationIPv4>(type->getDefaultSerialization())));
|
||||
});
|
||||
|
||||
factory.registerSimpleDataTypeCustom("IPv6", []
|
||||
{
|
||||
auto type = DataTypeFactory::instance().get("FixedString(16)");
|
||||
return std::make_pair(type, std::make_unique<DataTypeCustomDesc>(
|
||||
std::make_unique<DataTypeCustomFixedName>("IPv6"), std::make_unique<SerializationIPv6>(type->getDefaultSerialization())));
|
||||
});
|
||||
|
||||
/// MySQL, MariaDB
|
||||
factory.registerAlias("INET4", "IPv4", DataTypeFactory::CaseInsensitive);
|
||||
factory.registerAlias("INET6", "IPv6", DataTypeFactory::CaseInsensitive);
|
||||
}
|
||||
|
||||
}
|
@ -208,11 +208,11 @@ DataTypeFactory::DataTypeFactory()
|
||||
registerDataTypeNullable(*this);
|
||||
registerDataTypeNothing(*this);
|
||||
registerDataTypeUUID(*this);
|
||||
registerDataTypeIPv4andIPv6(*this);
|
||||
registerDataTypeAggregateFunction(*this);
|
||||
registerDataTypeNested(*this);
|
||||
registerDataTypeInterval(*this);
|
||||
registerDataTypeLowCardinality(*this);
|
||||
registerDataTypeDomainIPv4AndIPv6(*this);
|
||||
registerDataTypeDomainBool(*this);
|
||||
registerDataTypeDomainSimpleAggregateFunction(*this);
|
||||
registerDataTypeDomainGeo(*this);
|
||||
|
@ -79,11 +79,11 @@ void registerDataTypeMap(DataTypeFactory & factory);
|
||||
void registerDataTypeNullable(DataTypeFactory & factory);
|
||||
void registerDataTypeNothing(DataTypeFactory & factory);
|
||||
void registerDataTypeUUID(DataTypeFactory & factory);
|
||||
void registerDataTypeIPv4andIPv6(DataTypeFactory & factory);
|
||||
void registerDataTypeAggregateFunction(DataTypeFactory & factory);
|
||||
void registerDataTypeNested(DataTypeFactory & factory);
|
||||
void registerDataTypeInterval(DataTypeFactory & factory);
|
||||
void registerDataTypeLowCardinality(DataTypeFactory & factory);
|
||||
void registerDataTypeDomainIPv4AndIPv6(DataTypeFactory & factory);
|
||||
void registerDataTypeDomainBool(DataTypeFactory & factory);
|
||||
void registerDataTypeDomainSimpleAggregateFunction(DataTypeFactory & factory);
|
||||
void registerDataTypeDomainGeo(DataTypeFactory & factory);
|
||||
|
17
src/DataTypes/DataTypeIPv4andIPv6.cpp
Normal file
17
src/DataTypes/DataTypeIPv4andIPv6.cpp
Normal file
@ -0,0 +1,17 @@
|
||||
#include <DataTypes/DataTypeIPv4andIPv6.h>
|
||||
#include <DataTypes/DataTypeFactory.h>
|
||||
#include <DataTypes/Serializations/SerializationIPv4andIPv6.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
void registerDataTypeIPv4andIPv6(DataTypeFactory & factory)
|
||||
{
|
||||
factory.registerSimpleDataType("IPv4", [] { return DataTypePtr(std::make_shared<DataTypeIPv4>()); });
|
||||
factory.registerAlias("INET4", "IPv4", DataTypeFactory::CaseInsensitive);
|
||||
factory.registerSimpleDataType("IPv6", [] { return DataTypePtr(std::make_shared<DataTypeIPv6>()); });
|
||||
factory.registerAlias("INET6", "IPv6", DataTypeFactory::CaseInsensitive);
|
||||
}
|
||||
|
||||
}
|
53
src/DataTypes/DataTypeIPv4andIPv6.h
Normal file
53
src/DataTypes/DataTypeIPv4andIPv6.h
Normal file
@ -0,0 +1,53 @@
|
||||
#pragma once
|
||||
|
||||
#include <DataTypes/IDataType.h>
|
||||
#include <Columns/ColumnVector.h>
|
||||
#include <base/IPv4andIPv6.h>
|
||||
#include <DataTypes/Serializations/SerializationIPv4andIPv6.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
template <typename IPv>
|
||||
class DataTypeIP : public IDataType
|
||||
{
|
||||
public:
|
||||
static constexpr bool is_parametric = false;
|
||||
|
||||
using FieldType = IPv;
|
||||
using ColumnType = ColumnVector<IPv>;
|
||||
static constexpr auto type_id = TypeToTypeIndex<IPv>;
|
||||
|
||||
const char * getFamilyName() const override { return TypeName<IPv>.data(); }
|
||||
TypeIndex getTypeId() const override { return type_id; }
|
||||
|
||||
Field getDefault() const override { return IPv{}; }
|
||||
|
||||
MutableColumnPtr createColumn() const override {return ColumnVector<IPv>::create();}
|
||||
|
||||
bool isParametric() const override { return false; }
|
||||
bool haveSubtypes() const override { return false; }
|
||||
|
||||
bool equals(const IDataType & rhs) const override { return typeid(rhs) == typeid(*this); }
|
||||
|
||||
bool canBeUsedInBitOperations() const override { return true; }
|
||||
bool canBeInsideNullable() const override { return true; }
|
||||
bool canBePromoted() const override { return false; }
|
||||
bool shouldAlignRightInPrettyFormats() const override { return false; }
|
||||
bool textCanContainOnlyValidUTF8() const override { return true; }
|
||||
bool isComparable() const override { return true; }
|
||||
bool isValueUnambiguouslyRepresentedInContiguousMemoryRegion() const override { return true; }
|
||||
bool isValueUnambiguouslyRepresentedInFixedSizeContiguousMemoryRegion() const override { return true; }
|
||||
bool haveMaximumSizeOfValue() const override { return true; }
|
||||
size_t getSizeOfValueInMemory() const override { return sizeof(IPv); }
|
||||
bool isCategorial() const override { return true; }
|
||||
bool canBeInsideLowCardinality() const override { return true; }
|
||||
|
||||
SerializationPtr doGetDefaultSerialization() const override { return std::make_shared<SerializationIP<IPv>>(); }
|
||||
};
|
||||
|
||||
using DataTypeIPv4 = DataTypeIP<IPv4>;
|
||||
using DataTypeIPv6 = DataTypeIP<IPv6>;
|
||||
|
||||
}
|
@ -84,6 +84,10 @@ MutableColumnUniquePtr DataTypeLowCardinality::createColumnUniqueImpl(const IDat
|
||||
return creator(static_cast<ColumnVector<UInt32> *>(nullptr));
|
||||
else if (which.isUUID())
|
||||
return creator(static_cast<ColumnVector<UUID> *>(nullptr));
|
||||
else if (which.isIPv4())
|
||||
return creator(static_cast<ColumnVector<IPv4> *>(nullptr));
|
||||
else if (which.isIPv6())
|
||||
return creator(static_cast<ColumnVector<IPv6> *>(nullptr));
|
||||
else if (which.isInterval())
|
||||
return creator(static_cast<DataTypeInterval::ColumnType *>(nullptr));
|
||||
else if (which.isInt() || which.isUInt() || which.isFloat())
|
||||
|
@ -9,6 +9,7 @@
|
||||
#include <DataTypes/DataTypeNullable.h>
|
||||
#include <DataTypes/DataTypeNothing.h>
|
||||
#include <DataTypes/DataTypeUUID.h>
|
||||
#include <DataTypes/DataTypeIPv4andIPv6.h>
|
||||
#include <DataTypes/getLeastSupertype.h>
|
||||
#include <DataTypes/DataTypeFactory.h>
|
||||
#include <Common/Exception.h>
|
||||
@ -82,6 +83,18 @@ DataTypePtr FieldToDataType<on_error>::operator() (const UUID &) const
|
||||
return std::make_shared<DataTypeUUID>();
|
||||
}
|
||||
|
||||
template <LeastSupertypeOnError on_error>
|
||||
DataTypePtr FieldToDataType<on_error>::operator() (const IPv4 &) const
|
||||
{
|
||||
return std::make_shared<DataTypeIPv4>();
|
||||
}
|
||||
|
||||
template <LeastSupertypeOnError on_error>
|
||||
DataTypePtr FieldToDataType<on_error>::operator() (const IPv6 &) const
|
||||
{
|
||||
return std::make_shared<DataTypeIPv6>();
|
||||
}
|
||||
|
||||
template <LeastSupertypeOnError on_error>
|
||||
DataTypePtr FieldToDataType<on_error>::operator() (const String &) const
|
||||
{
|
||||
|
@ -28,6 +28,8 @@ public:
|
||||
DataTypePtr operator() (const Int64 & x) const;
|
||||
DataTypePtr operator() (const Int128 & x) const;
|
||||
DataTypePtr operator() (const UUID & x) const;
|
||||
DataTypePtr operator() (const IPv4 & x) const;
|
||||
DataTypePtr operator() (const IPv6 & x) const;
|
||||
DataTypePtr operator() (const Float64 & x) const;
|
||||
DataTypePtr operator() (const String & x) const;
|
||||
DataTypePtr operator() (const Array & x) const;
|
||||
|
@ -377,6 +377,8 @@ struct WhichDataType
|
||||
constexpr bool isStringOrFixedString() const { return isString() || isFixedString(); }
|
||||
|
||||
constexpr bool isUUID() const { return idx == TypeIndex::UUID; }
|
||||
constexpr bool isIPv4() const { return idx == TypeIndex::IPv4; }
|
||||
constexpr bool isIPv6() const { return idx == TypeIndex::IPv6; }
|
||||
constexpr bool isArray() const { return idx == TypeIndex::Array; }
|
||||
constexpr bool isTuple() const { return idx == TypeIndex::Tuple; }
|
||||
constexpr bool isMap() const {return idx == TypeIndex::Map; }
|
||||
@ -414,6 +416,8 @@ inline bool isMap(const DataTypePtr & data_type) {return WhichDataType(data_type
|
||||
inline bool isInterval(const DataTypePtr & data_type) {return WhichDataType(data_type).isInterval(); }
|
||||
inline bool isNothing(const DataTypePtr & data_type) { return WhichDataType(data_type).isNothing(); }
|
||||
inline bool isUUID(const DataTypePtr & data_type) { return WhichDataType(data_type).isUUID(); }
|
||||
inline bool isIPv4(const DataTypePtr & data_type) { return WhichDataType(data_type).isIPv4(); }
|
||||
inline bool isIPv6(const DataTypePtr & data_type) { return WhichDataType(data_type).isIPv6(); }
|
||||
|
||||
template <typename T>
|
||||
inline bool isObject(const T & data_type)
|
||||
@ -479,7 +483,7 @@ template <typename T>
|
||||
inline bool isColumnedAsNumber(const T & data_type)
|
||||
{
|
||||
WhichDataType which(data_type);
|
||||
return which.isInt() || which.isUInt() || which.isFloat() || which.isDateOrDate32() || which.isDateTime() || which.isDateTime64() || which.isUUID();
|
||||
return which.isInt() || which.isUInt() || which.isFloat() || which.isDateOrDate32() || which.isDateTime() || which.isDateTime64() || which.isUUID() || which.isIPv4() || which.isIPv6();
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
|
@ -1,109 +0,0 @@
|
||||
#include <DataTypes/Serializations/SerializationIP.h>
|
||||
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <Columns/ColumnFixedString.h>
|
||||
#include <Common/Exception.h>
|
||||
#include <Common/formatIPv6.h>
|
||||
#include <IO/WriteBuffer.h>
|
||||
#include <IO/ReadBuffer.h>
|
||||
#include <Formats/FormatSettings.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int CANNOT_PARSE_DOMAIN_VALUE_FROM_STRING;
|
||||
extern const int ILLEGAL_COLUMN;
|
||||
}
|
||||
|
||||
SerializationIPv4::SerializationIPv4(const SerializationPtr & nested_)
|
||||
: SerializationCustomSimpleText(nested_)
|
||||
{
|
||||
}
|
||||
|
||||
void SerializationIPv4::serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const
|
||||
{
|
||||
const auto * col = checkAndGetColumn<ColumnUInt32>(&column);
|
||||
if (!col)
|
||||
{
|
||||
throw Exception("IPv4 type can only serialize columns of type UInt32." + column.getName(), ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
|
||||
char buffer[IPV4_MAX_TEXT_LENGTH + 1] = {'\0'};
|
||||
char * ptr = buffer;
|
||||
formatIPv4(reinterpret_cast<const unsigned char *>(&col->getData()[row_num]), ptr);
|
||||
|
||||
ostr.write(buffer, strlen(buffer));
|
||||
}
|
||||
|
||||
void SerializationIPv4::deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings, bool whole) const
|
||||
{
|
||||
ColumnUInt32 * col = typeid_cast<ColumnUInt32 *>(&column);
|
||||
if (!col)
|
||||
{
|
||||
throw Exception("IPv4 type can only deserialize columns of type UInt32." + column.getName(), ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
|
||||
char buffer[IPV4_MAX_TEXT_LENGTH + 1] = {'\0'};
|
||||
[[maybe_unused]] size_t read_bytes = istr.read(buffer, sizeof(buffer) - 1);
|
||||
UInt32 ipv4_value = 0;
|
||||
|
||||
bool parse_result = parseIPv4(buffer, reinterpret_cast<unsigned char *>(&ipv4_value));
|
||||
if (!parse_result && !settings.input_format_ipv4_default_on_conversion_error)
|
||||
{
|
||||
throw Exception("Invalid IPv4 value", ErrorCodes::CANNOT_PARSE_DOMAIN_VALUE_FROM_STRING);
|
||||
}
|
||||
|
||||
col->insert(ipv4_value);
|
||||
|
||||
if (whole && !istr.eof())
|
||||
throwUnexpectedDataAfterParsedValue(column, istr, settings, "IPv4");
|
||||
}
|
||||
|
||||
SerializationIPv6::SerializationIPv6(const SerializationPtr & nested_)
|
||||
: SerializationCustomSimpleText(nested_)
|
||||
{
|
||||
}
|
||||
void SerializationIPv6::serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const
|
||||
{
|
||||
const auto * col = checkAndGetColumn<ColumnFixedString>(&column);
|
||||
if (!col)
|
||||
{
|
||||
throw Exception("IPv6 type domain can only serialize columns of type FixedString(16)." + column.getName(), ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
|
||||
char buffer[IPV6_MAX_TEXT_LENGTH + 1] = {'\0'};
|
||||
char * ptr = buffer;
|
||||
formatIPv6(reinterpret_cast<const unsigned char *>(col->getDataAt(row_num).data), ptr);
|
||||
|
||||
ostr.write(buffer, strlen(buffer));
|
||||
}
|
||||
|
||||
void SerializationIPv6::deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings, bool whole) const
|
||||
{
|
||||
ColumnFixedString * col = typeid_cast<ColumnFixedString *>(&column);
|
||||
if (!col)
|
||||
{
|
||||
throw Exception("IPv6 type domain can only deserialize columns of type FixedString(16)." + column.getName(), ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
|
||||
char buffer[IPV6_MAX_TEXT_LENGTH + 1] = {'\0'};
|
||||
[[maybe_unused]] size_t read_bytes = istr.read(buffer, sizeof(buffer) - 1);
|
||||
|
||||
std::string ipv6_value(IPV6_BINARY_LENGTH, '\0');
|
||||
|
||||
bool parse_result = parseIPv6(buffer, reinterpret_cast<unsigned char *>(ipv6_value.data()));
|
||||
if (!parse_result && !settings.input_format_ipv6_default_on_conversion_error)
|
||||
{
|
||||
throw Exception("Invalid IPv6 value", ErrorCodes::CANNOT_PARSE_DOMAIN_VALUE_FROM_STRING);
|
||||
}
|
||||
|
||||
col->insertString(ipv6_value);
|
||||
|
||||
if (whole && !istr.eof())
|
||||
throwUnexpectedDataAfterParsedValue(column, istr, settings, "IPv6");
|
||||
}
|
||||
|
||||
}
|
@ -1,26 +0,0 @@
|
||||
#pragma once
|
||||
|
||||
#include <DataTypes/Serializations/SerializationCustomSimpleText.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class SerializationIPv4 final : public SerializationCustomSimpleText
|
||||
{
|
||||
public:
|
||||
explicit SerializationIPv4(const SerializationPtr & nested_);
|
||||
|
||||
void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
|
||||
void deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings, bool whole) const override;
|
||||
};
|
||||
|
||||
class SerializationIPv6 : public SerializationCustomSimpleText
|
||||
{
|
||||
public:
|
||||
explicit SerializationIPv6(const SerializationPtr & nested_);
|
||||
|
||||
void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
|
||||
void deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings, bool whole) const override;
|
||||
};
|
||||
|
||||
}
|
134
src/DataTypes/Serializations/SerializationIPv4andIPv6.h
Normal file
134
src/DataTypes/Serializations/SerializationIPv4andIPv6.h
Normal file
@ -0,0 +1,134 @@
|
||||
#pragma once
|
||||
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <base/TypeName.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
template <typename IPv>
|
||||
class SerializationIP : public SimpleTextSerialization
|
||||
{
|
||||
public:
|
||||
void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override
|
||||
{
|
||||
writeText(assert_cast<const ColumnVector<IPv> &>(column).getData()[row_num], ostr);
|
||||
}
|
||||
void deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings, bool whole) const override
|
||||
{
|
||||
IPv x;
|
||||
readText(x, istr);
|
||||
|
||||
if (whole && !istr.eof())
|
||||
throwUnexpectedDataAfterParsedValue(column, istr, settings, TypeName<IPv>.data());
|
||||
|
||||
assert_cast<ColumnVector<IPv> &>(column).getData().push_back(x);
|
||||
}
|
||||
void serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override
|
||||
{
|
||||
serializeText(column, row_num, ostr, settings);
|
||||
}
|
||||
void deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override
|
||||
{
|
||||
deserializeText(column, istr, settings, true);
|
||||
}
|
||||
void serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override
|
||||
{
|
||||
writeChar('\'', ostr);
|
||||
serializeText(column, row_num, ostr, settings);
|
||||
writeChar('\'', ostr);
|
||||
}
|
||||
void deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override
|
||||
{
|
||||
IPv x;
|
||||
assertChar('\'', istr);
|
||||
readText(x, istr);
|
||||
assertChar('\'', istr);
|
||||
assert_cast<ColumnVector<IPv> &>(column).getData().push_back(x); /// It's important to do this at the end - for exception safety.
|
||||
}
|
||||
void serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override
|
||||
{
|
||||
writeChar('"', ostr);
|
||||
serializeText(column, row_num, ostr, settings);
|
||||
writeChar('"', ostr);
|
||||
}
|
||||
void deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override
|
||||
{
|
||||
IPv x;
|
||||
assertChar('"', istr);
|
||||
readText(x, istr);
|
||||
/// this code looks weird, but we want to throw specific exception to match original behavior...
|
||||
if (istr.eof())
|
||||
assertChar('"', istr);
|
||||
if (*istr.position() != '"')
|
||||
throwUnexpectedDataAfterParsedValue(column, istr, settings, TypeName<IPv>.data());
|
||||
istr.ignore();
|
||||
|
||||
assert_cast<ColumnVector<IPv> &>(column).getData().push_back(x);
|
||||
}
|
||||
void serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override
|
||||
{
|
||||
writeChar('"', ostr);
|
||||
serializeText(column, row_num, ostr, settings);
|
||||
writeChar('"', ostr);
|
||||
}
|
||||
void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override
|
||||
{
|
||||
IPv value;
|
||||
readCSV(value, istr);
|
||||
|
||||
if (!istr.eof())
|
||||
throwUnexpectedDataAfterParsedValue(column, istr, settings, TypeName<IPv>.data());
|
||||
|
||||
assert_cast<ColumnVector<IPv> &>(column).getData().push_back(value);
|
||||
}
|
||||
|
||||
void serializeBinary(const Field & field, WriteBuffer & ostr, const FormatSettings &) const override
|
||||
{
|
||||
IPv x = field.get<IPv>();
|
||||
writeBinary(x, ostr);
|
||||
}
|
||||
void deserializeBinary(Field & field, ReadBuffer & istr, const FormatSettings &) const override
|
||||
{
|
||||
IPv x;
|
||||
readBinary(x.toUnderType(), istr);
|
||||
field = NearestFieldType<IPv>(x);
|
||||
}
|
||||
void serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override
|
||||
{
|
||||
writeBinary(assert_cast<const ColumnVector<IPv> &>(column).getData()[row_num], ostr);
|
||||
}
|
||||
void deserializeBinary(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override
|
||||
{
|
||||
IPv x;
|
||||
readBinary(x.toUnderType(), istr);
|
||||
assert_cast<ColumnVector<IPv> &>(column).getData().push_back(x);
|
||||
}
|
||||
void serializeBinaryBulk(const IColumn & column, WriteBuffer & ostr, size_t offset, size_t limit) const override
|
||||
{
|
||||
const typename ColumnVector<IPv>::Container & x = typeid_cast<const ColumnVector<IPv> &>(column).getData();
|
||||
|
||||
size_t size = x.size();
|
||||
|
||||
if (limit == 0 || offset + limit > size)
|
||||
limit = size - offset;
|
||||
|
||||
if (limit)
|
||||
ostr.write(reinterpret_cast<const char *>(&x[offset]), sizeof(IPv) * limit);
|
||||
}
|
||||
void deserializeBinaryBulk(IColumn & column, ReadBuffer & istr, size_t limit, double /*avg_value_size_hint*/) const override
|
||||
{
|
||||
typename ColumnVector<IPv>::Container & x = typeid_cast<ColumnVector<IPv> &>(column).getData();
|
||||
size_t initial_size = x.size();
|
||||
x.resize(initial_size + limit);
|
||||
size_t size = istr.readBig(reinterpret_cast<char*>(&x[initial_size]), sizeof(IPv) * limit);
|
||||
x.resize(initial_size + size / sizeof(IPv));
|
||||
}
|
||||
};
|
||||
|
||||
using SerializationIPv4 = SerializationIP<IPv4>;
|
||||
using SerializationIPv6 = SerializationIP<IPv6>;
|
||||
|
||||
}
|
@ -557,6 +557,8 @@ private:
|
||||
ContainerType<Float32>,
|
||||
ContainerType<Float64>,
|
||||
ContainerType<UUID>,
|
||||
ContainerType<IPv4>,
|
||||
ContainerType<IPv6>,
|
||||
ContainerType<StringRef>,
|
||||
ContainerType<Array>,
|
||||
ContainerType<Field>> attribute_container;
|
||||
|
@ -267,6 +267,14 @@ public:
|
||||
{
|
||||
return ColumnType::create(size);
|
||||
}
|
||||
else if constexpr (std::is_same_v<DictionaryAttributeType, IPv4>)
|
||||
{
|
||||
return ColumnType::create(size);
|
||||
}
|
||||
else if constexpr (std::is_same_v<DictionaryAttributeType, IPv6>)
|
||||
{
|
||||
return ColumnType::create(size);
|
||||
}
|
||||
else if constexpr (is_decimal<DictionaryAttributeType>)
|
||||
{
|
||||
auto nested_type = removeNullable(dictionary_attribute.type);
|
||||
|
@ -136,6 +136,8 @@ private:
|
||||
ContainerType<Float32>,
|
||||
ContainerType<Float64>,
|
||||
ContainerType<UUID>,
|
||||
ContainerType<IPv4>,
|
||||
ContainerType<IPv6>,
|
||||
ContainerType<StringRef>,
|
||||
ContainerType<Array>>
|
||||
container;
|
||||
|
@ -156,6 +156,8 @@ private:
|
||||
AttributeContainerType<Float32>,
|
||||
AttributeContainerType<Float64>,
|
||||
AttributeContainerType<UUID>,
|
||||
AttributeContainerType<IPv4>,
|
||||
AttributeContainerType<IPv6>,
|
||||
AttributeContainerType<StringRef>,
|
||||
AttributeContainerType<Array>>
|
||||
container;
|
||||
|
@ -186,6 +186,8 @@ private:
|
||||
CollectionType<Float32>,
|
||||
CollectionType<Float64>,
|
||||
CollectionType<UUID>,
|
||||
CollectionType<IPv4>,
|
||||
CollectionType<IPv6>,
|
||||
CollectionType<StringRef>,
|
||||
CollectionType<Array>>
|
||||
container;
|
||||
|
@ -8,6 +8,7 @@
|
||||
#include <DataTypes/DataTypeFixedString.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <DataTypes/DataTypesDecimal.h>
|
||||
#include <DataTypes/DataTypeIPv4andIPv6.h>
|
||||
#include <Poco/ByteOrder.h>
|
||||
#include <Common/formatIPv6.h>
|
||||
#include <base/itoa.h>
|
||||
@ -27,7 +28,6 @@ namespace ErrorCodes
|
||||
extern const int CANNOT_PARSE_INPUT_ASSERTION_FAILED;
|
||||
extern const int CANNOT_PARSE_NUMBER;
|
||||
extern const int DICTIONARY_IS_EMPTY;
|
||||
extern const int LOGICAL_ERROR;
|
||||
extern const int TYPE_MISMATCH;
|
||||
extern const int UNSUPPORTED_METHOD;
|
||||
}
|
||||
@ -129,12 +129,12 @@ static void validateKeyTypes(const DataTypes & key_types)
|
||||
if (key_types.empty() || key_types.size() > 2)
|
||||
throw Exception(ErrorCodes::TYPE_MISMATCH, "Expected a single IP address or IP with mask");
|
||||
|
||||
const auto * key_ipv4type = typeid_cast<const DataTypeUInt32 *>(key_types[0].get());
|
||||
const auto * key_ipv6type = typeid_cast<const DataTypeFixedString *>(key_types[0].get());
|
||||
TypeIndex type_id = key_types[0]->getTypeId();
|
||||
const auto * key_string = typeid_cast<const DataTypeFixedString *>(key_types[0].get());
|
||||
|
||||
if (key_ipv4type == nullptr && (key_ipv6type == nullptr || key_ipv6type->getN() != 16))
|
||||
if (type_id != TypeIndex::IPv4 && type_id != TypeIndex::UInt32 && type_id != TypeIndex::IPv6 && !(key_string && key_string->getN() == IPV6_BINARY_LENGTH))
|
||||
throw Exception(ErrorCodes::TYPE_MISMATCH,
|
||||
"Key does not match, expected either `IPv4` (`UInt32`) or `IPv6` (`FixedString(16)`)");
|
||||
"Key does not match, expected either IPv4 (or UInt32) or IPv6 (or FixedString(16))");
|
||||
|
||||
if (key_types.size() > 1)
|
||||
{
|
||||
@ -296,30 +296,33 @@ ColumnUInt8::Ptr IPAddressDictionary::hasKeys(const Columns & key_columns, const
|
||||
|
||||
size_t keys_found = 0;
|
||||
|
||||
if (first_column->isNumeric())
|
||||
TypeIndex type_id = first_column->getDataType();
|
||||
|
||||
if (type_id == TypeIndex::IPv4 || type_id == TypeIndex::UInt32)
|
||||
{
|
||||
uint8_t addrv6_buf[IPV6_BINARY_LENGTH];
|
||||
for (const auto i : collections::range(0, rows))
|
||||
{
|
||||
auto addrv4 = static_cast<UInt32>(first_column->get64(i));
|
||||
auto addrv4 = *reinterpret_cast<const UInt32 *>(first_column->getDataAt(i).data);
|
||||
auto found = tryLookupIPv4(addrv4, addrv6_buf);
|
||||
out[i] = (found != ipNotFound());
|
||||
keys_found += out[i];
|
||||
}
|
||||
}
|
||||
else
|
||||
else if (type_id == TypeIndex::IPv6 || type_id == TypeIndex::FixedString)
|
||||
{
|
||||
for (const auto i : collections::range(0, rows))
|
||||
{
|
||||
auto addr = first_column->getDataAt(i);
|
||||
if (unlikely(addr.size != IPV6_BINARY_LENGTH))
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected key to be FixedString(16)");
|
||||
|
||||
if (addr.size != IPV6_BINARY_LENGTH)
|
||||
throw Exception(ErrorCodes::TYPE_MISMATCH, "Expected key FixedString(16)");
|
||||
auto found = tryLookupIPv6(reinterpret_cast<const uint8_t *>(addr.data));
|
||||
out[i] = (found != ipNotFound());
|
||||
keys_found += out[i];
|
||||
}
|
||||
}
|
||||
else
|
||||
throw Exception(ErrorCodes::TYPE_MISMATCH, "Expected key to be IPv4 (or UInt32) or IPv6 (or FixedString(16))");
|
||||
|
||||
query_count.fetch_add(rows, std::memory_order_relaxed);
|
||||
found_count.fetch_add(keys_found, std::memory_order_relaxed);
|
||||
@ -709,13 +712,15 @@ void IPAddressDictionary::getItemsImpl(
|
||||
|
||||
size_t keys_found = 0;
|
||||
|
||||
if (first_column->isNumeric())
|
||||
TypeIndex type_id = first_column->getDataType();
|
||||
|
||||
if (type_id == TypeIndex::IPv4 || type_id == TypeIndex::UInt32)
|
||||
{
|
||||
uint8_t addrv6_buf[IPV6_BINARY_LENGTH];
|
||||
for (const auto i : collections::range(0, rows))
|
||||
{
|
||||
// addrv4 has native endianness
|
||||
auto addrv4 = static_cast<UInt32>(first_column->get64(i));
|
||||
auto addrv4 = *reinterpret_cast<const UInt32 *>(first_column->getDataAt(i).data);
|
||||
auto found = tryLookupIPv4(addrv4, addrv6_buf);
|
||||
if (found != ipNotFound())
|
||||
{
|
||||
@ -726,14 +731,13 @@ void IPAddressDictionary::getItemsImpl(
|
||||
set_value(i, default_value_extractor[i]);
|
||||
}
|
||||
}
|
||||
else
|
||||
else if (type_id == TypeIndex::IPv6 || type_id == TypeIndex::FixedString)
|
||||
{
|
||||
for (const auto i : collections::range(0, rows))
|
||||
{
|
||||
auto addr = first_column->getDataAt(i);
|
||||
if (addr.size != IPV6_BINARY_LENGTH)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected key to be FixedString(16)");
|
||||
|
||||
throw Exception(ErrorCodes::TYPE_MISMATCH, "Expected key to be FixedString(16)");
|
||||
auto found = tryLookupIPv6(reinterpret_cast<const uint8_t *>(addr.data));
|
||||
if (found != ipNotFound())
|
||||
{
|
||||
@ -744,6 +748,8 @@ void IPAddressDictionary::getItemsImpl(
|
||||
set_value(i, default_value_extractor[i]);
|
||||
}
|
||||
}
|
||||
else
|
||||
throw Exception(ErrorCodes::TYPE_MISMATCH, "Expected key to be IPv4 (or UInt32) or IPv6 (or FixedString(16))");
|
||||
|
||||
query_count.fetch_add(rows, std::memory_order_relaxed);
|
||||
found_count.fetch_add(keys_found, std::memory_order_relaxed);
|
||||
|
@ -118,6 +118,8 @@ private:
|
||||
Float32,
|
||||
Float64,
|
||||
UUID,
|
||||
IPv4,
|
||||
IPv6,
|
||||
String,
|
||||
Array>
|
||||
null_values;
|
||||
@ -142,6 +144,8 @@ private:
|
||||
ContainerType<Float32>,
|
||||
ContainerType<Float64>,
|
||||
ContainerType<UUID>,
|
||||
ContainerType<IPv4>,
|
||||
ContainerType<IPv6>,
|
||||
ContainerType<StringRef>,
|
||||
ContainerType<Array>>
|
||||
maps;
|
||||
|
@ -150,6 +150,8 @@ private:
|
||||
AttributeContainerType<Float32>,
|
||||
AttributeContainerType<Float64>,
|
||||
AttributeContainerType<UUID>,
|
||||
AttributeContainerType<IPv4>,
|
||||
AttributeContainerType<IPv6>,
|
||||
AttributeContainerType<StringRef>,
|
||||
AttributeContainerType<Array>>
|
||||
container;
|
||||
@ -189,7 +191,9 @@ private:
|
||||
ContainerType<DateTime64>,
|
||||
ContainerType<Float32>,
|
||||
ContainerType<Float64>,
|
||||
ContainerType<UUID>>;
|
||||
ContainerType<UUID>,
|
||||
ContainerType<IPv4>,
|
||||
ContainerType<IPv6>>;
|
||||
|
||||
struct KeyAttribute final
|
||||
{
|
||||
|
@ -1648,6 +1648,83 @@ namespace
|
||||
String text_buffer;
|
||||
};
|
||||
|
||||
/// Serializes a ColumnVector<IPv6> containing IPv6s to a field of type TYPE_STRING or TYPE_BYTES.
|
||||
class ProtobufSerializerIPv6 : public ProtobufSerializerSingleValue
|
||||
{
|
||||
public:
|
||||
ProtobufSerializerIPv6(
|
||||
std::string_view column_name_,
|
||||
const google::protobuf::FieldDescriptor & field_descriptor_,
|
||||
const ProtobufReaderOrWriter & reader_or_writer_)
|
||||
: ProtobufSerializerSingleValue(column_name_, field_descriptor_, reader_or_writer_)
|
||||
{
|
||||
setFunctions();
|
||||
}
|
||||
|
||||
void writeRow(size_t row_num) override
|
||||
{
|
||||
const auto & column_vector = assert_cast<const ColumnVector<IPv6> &>(*column);
|
||||
write_function(column_vector.getElement(row_num));
|
||||
}
|
||||
|
||||
void readRow(size_t row_num) override
|
||||
{
|
||||
IPv6 value = read_function();
|
||||
auto & column_vector = assert_cast<ColumnVector<IPv6> &>(column->assumeMutableRef());
|
||||
if (row_num < column_vector.size())
|
||||
column_vector.getElement(row_num) = value;
|
||||
else
|
||||
column_vector.insertValue(value);
|
||||
}
|
||||
|
||||
void insertDefaults(size_t row_num) override
|
||||
{
|
||||
auto & column_vector = assert_cast<ColumnVector<IPv6> &>(column->assumeMutableRef());
|
||||
if (row_num < column_vector.size())
|
||||
return;
|
||||
column_vector.insertDefault();
|
||||
}
|
||||
|
||||
void describeTree(WriteBuffer & out, size_t indent) const override
|
||||
{
|
||||
writeIndent(out, indent) << "ProtobufSerializer" << TypeName<IPv6> << ": column " << quoteString(column_name) << " -> field "
|
||||
<< quoteString(field_descriptor.full_name()) << " (" << field_descriptor.type_name() << ")\n";
|
||||
}
|
||||
|
||||
private:
|
||||
void setFunctions()
|
||||
{
|
||||
if ((field_typeid != FieldTypeId::TYPE_STRING) && (field_typeid != FieldTypeId::TYPE_BYTES))
|
||||
incompatibleColumnType(TypeName<IPv6>);
|
||||
|
||||
write_function = [this](IPv6 value)
|
||||
{
|
||||
ipToString(value, text_buffer);
|
||||
writeStr(text_buffer);
|
||||
};
|
||||
|
||||
read_function = [this]() -> IPv6
|
||||
{
|
||||
readStr(text_buffer);
|
||||
return parse<IPv6>(text_buffer);
|
||||
};
|
||||
|
||||
default_function = [this]() -> IPv6 { return parse<IPv6>(field_descriptor.default_value_string()); };
|
||||
}
|
||||
|
||||
static void ipToString(const IPv6 & ip, String & str)
|
||||
{
|
||||
WriteBufferFromString buf{str};
|
||||
writeText(ip, buf);
|
||||
}
|
||||
|
||||
std::function<void(IPv6)> write_function;
|
||||
std::function<IPv6()> read_function;
|
||||
std::function<IPv6()> default_function;
|
||||
String text_buffer;
|
||||
};
|
||||
|
||||
using ProtobufSerializerIPv4 = ProtobufSerializerNumber<UInt32>;
|
||||
|
||||
using ProtobufSerializerInterval = ProtobufSerializerNumber<Int64>;
|
||||
|
||||
@ -3286,6 +3363,8 @@ namespace
|
||||
case TypeIndex::Decimal128: return std::make_unique<ProtobufSerializerDecimal<Decimal128>>(column_name, assert_cast<const DataTypeDecimal<Decimal128> &>(*data_type), field_descriptor, reader_or_writer);
|
||||
case TypeIndex::Decimal256: return std::make_unique<ProtobufSerializerDecimal<Decimal256>>(column_name, assert_cast<const DataTypeDecimal<Decimal256> &>(*data_type), field_descriptor, reader_or_writer);
|
||||
case TypeIndex::UUID: return std::make_unique<ProtobufSerializerUUID>(column_name, field_descriptor, reader_or_writer);
|
||||
case TypeIndex::IPv4: return std::make_unique<ProtobufSerializerIPv4>(column_name, field_descriptor, reader_or_writer);
|
||||
case TypeIndex::IPv6: return std::make_unique<ProtobufSerializerIPv6>(column_name, field_descriptor, reader_or_writer);
|
||||
case TypeIndex::Interval: return std::make_unique<ProtobufSerializerInterval>(column_name, field_descriptor, reader_or_writer);
|
||||
case TypeIndex::AggregateFunction: return std::make_unique<ProtobufSerializerAggregateFunction>(column_name, typeid_cast<std::shared_ptr<const DataTypeAggregateFunction>>(data_type), field_descriptor, reader_or_writer);
|
||||
|
||||
|
@ -33,10 +33,10 @@ public:
|
||||
|
||||
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; }
|
||||
|
||||
explicit CastOverloadResolverImpl(std::optional<Diagnostic> diagnostic_, bool keep_nullable_, bool cast_ipv4_ipv6_default_on_conversion_error_, const DataTypeValidationSettings & data_type_validation_settings_)
|
||||
: diagnostic(std::move(diagnostic_))
|
||||
explicit CastOverloadResolverImpl(ContextPtr context_, std::optional<Diagnostic> diagnostic_, bool keep_nullable_, const DataTypeValidationSettings & data_type_validation_settings_)
|
||||
: context(context_)
|
||||
, diagnostic(std::move(diagnostic_))
|
||||
, keep_nullable(keep_nullable_)
|
||||
, cast_ipv4_ipv6_default_on_conversion_error(cast_ipv4_ipv6_default_on_conversion_error_)
|
||||
, data_type_validation_settings(data_type_validation_settings_)
|
||||
{
|
||||
}
|
||||
@ -46,15 +46,21 @@ public:
|
||||
const auto & settings_ref = context->getSettingsRef();
|
||||
|
||||
if constexpr (internal)
|
||||
return createImpl({}, false /*keep_nullable*/, settings_ref.cast_ipv4_ipv6_default_on_conversion_error);
|
||||
return createImpl(context, {}, false /*keep_nullable*/);
|
||||
|
||||
return createImpl({}, settings_ref.cast_keep_nullable, settings_ref.cast_ipv4_ipv6_default_on_conversion_error, DataTypeValidationSettings(settings_ref));
|
||||
return createImpl(context, {}, settings_ref.cast_keep_nullable, DataTypeValidationSettings(settings_ref));
|
||||
}
|
||||
|
||||
static FunctionOverloadResolverPtr createImpl(std::optional<Diagnostic> diagnostic = {}, bool keep_nullable = false, bool cast_ipv4_ipv6_default_on_conversion_error = false, const DataTypeValidationSettings & data_type_validation_settings = {})
|
||||
static FunctionOverloadResolverPtr createImpl(ContextPtr context, std::optional<Diagnostic> diagnostic = {}, bool keep_nullable = false, const DataTypeValidationSettings & data_type_validation_settings = {})
|
||||
{
|
||||
assert(!internal || !keep_nullable);
|
||||
return std::make_unique<CastOverloadResolverImpl>(std::move(diagnostic), keep_nullable, cast_ipv4_ipv6_default_on_conversion_error, data_type_validation_settings);
|
||||
return std::make_unique<CastOverloadResolverImpl>(context, std::move(diagnostic), keep_nullable, data_type_validation_settings);
|
||||
}
|
||||
|
||||
static FunctionOverloadResolverPtr createImpl(std::optional<Diagnostic> diagnostic = {}, bool keep_nullable = false, const DataTypeValidationSettings & data_type_validation_settings = {})
|
||||
{
|
||||
assert(!internal || !keep_nullable);
|
||||
return std::make_unique<CastOverloadResolverImpl>(ContextPtr(), std::move(diagnostic), keep_nullable, data_type_validation_settings);
|
||||
}
|
||||
|
||||
protected:
|
||||
@ -67,7 +73,7 @@ protected:
|
||||
data_types[i] = arguments[i].type;
|
||||
|
||||
auto monotonicity = MonotonicityHelper::getMonotonicityInformation(arguments.front().type, return_type.get());
|
||||
return std::make_unique<FunctionCast<FunctionName>>(name, std::move(monotonicity), data_types, return_type, diagnostic, cast_type, cast_ipv4_ipv6_default_on_conversion_error);
|
||||
return std::make_unique<FunctionCast<FunctionName>>(context, name, std::move(monotonicity), data_types, return_type, diagnostic, cast_type);
|
||||
}
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
|
||||
@ -104,9 +110,9 @@ protected:
|
||||
bool useDefaultImplementationForLowCardinalityColumns() const override { return false; }
|
||||
|
||||
private:
|
||||
ContextPtr context;
|
||||
std::optional<Diagnostic> diagnostic;
|
||||
bool keep_nullable;
|
||||
bool cast_ipv4_ipv6_default_on_conversion_error;
|
||||
DataTypeValidationSettings data_type_validation_settings;
|
||||
};
|
||||
|
||||
|
@ -257,6 +257,8 @@ public:
|
||||
!which.isFloat() &&
|
||||
!which.isDecimal() &&
|
||||
!which.isUUID() &&
|
||||
!which.isIPv4() &&
|
||||
!which.isIPv6() &&
|
||||
!which.isAggregateFunction())
|
||||
throw Exception("Illegal type " + arguments[0]->getName() + " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
@ -297,7 +299,9 @@ public:
|
||||
tryExecuteDecimal<Decimal32>(column, res_column) ||
|
||||
tryExecuteDecimal<Decimal64>(column, res_column) ||
|
||||
tryExecuteDecimal<Decimal128>(column, res_column) ||
|
||||
tryExecuteUUID(column, res_column))
|
||||
tryExecuteUUID(column, res_column) ||
|
||||
tryExecuteIPv4(column, res_column) ||
|
||||
tryExecuteIPv6(column, res_column))
|
||||
return res_column;
|
||||
|
||||
throw Exception("Illegal column " + arguments[0].column->getName()
|
||||
@ -519,6 +523,88 @@ public:
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
bool tryExecuteIPv6(const IColumn * col, ColumnPtr & col_res) const
|
||||
{
|
||||
const ColumnIPv6 * col_vec = checkAndGetColumn<ColumnIPv6>(col);
|
||||
|
||||
static constexpr size_t MAX_LENGTH = sizeof(IPv6) * word_size + 1; /// Including trailing zero byte.
|
||||
|
||||
if (!col_vec)
|
||||
return false;
|
||||
|
||||
auto col_str = ColumnString::create();
|
||||
ColumnString::Chars & out_vec = col_str->getChars();
|
||||
ColumnString::Offsets & out_offsets = col_str->getOffsets();
|
||||
|
||||
const typename ColumnIPv6::Container & in_vec = col_vec->getData();
|
||||
const IPv6* ip = in_vec.data();
|
||||
|
||||
size_t size = in_vec.size();
|
||||
out_offsets.resize(size);
|
||||
out_vec.resize(size * (word_size+1) + MAX_LENGTH); /// word_size+1 is length of one byte in hex/bin plus zero byte.
|
||||
|
||||
size_t pos = 0;
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
/// Manual exponential growth, so as not to rely on the linear amortized work time of `resize` (no one guarantees it).
|
||||
if (pos + MAX_LENGTH > out_vec.size())
|
||||
out_vec.resize(out_vec.size() * word_size + MAX_LENGTH);
|
||||
|
||||
char * begin = reinterpret_cast<char *>(&out_vec[pos]);
|
||||
char * end = begin;
|
||||
|
||||
Impl::executeOneString(reinterpret_cast<const UInt8 *>(&ip[i].toUnderType().items[0]), reinterpret_cast<const UInt8 *>(&ip[i].toUnderType().items[2]), end);
|
||||
|
||||
pos += end - begin;
|
||||
out_offsets[i] = pos;
|
||||
}
|
||||
out_vec.resize(pos);
|
||||
|
||||
col_res = std::move(col_str);
|
||||
return true;
|
||||
}
|
||||
|
||||
bool tryExecuteIPv4(const IColumn * col, ColumnPtr & col_res) const
|
||||
{
|
||||
const ColumnIPv4 * col_vec = checkAndGetColumn<ColumnIPv4>(col);
|
||||
|
||||
static constexpr size_t MAX_LENGTH = sizeof(IPv4) * word_size + 1; /// Including trailing zero byte.
|
||||
|
||||
if (!col_vec)
|
||||
return false;
|
||||
|
||||
auto col_str = ColumnString::create();
|
||||
ColumnString::Chars & out_vec = col_str->getChars();
|
||||
ColumnString::Offsets & out_offsets = col_str->getOffsets();
|
||||
|
||||
const typename ColumnIPv4::Container & in_vec = col_vec->getData();
|
||||
const IPv4* ip = in_vec.data();
|
||||
|
||||
size_t size = in_vec.size();
|
||||
out_offsets.resize(size);
|
||||
out_vec.resize(size * (word_size+1) + MAX_LENGTH); /// word_size+1 is length of one byte in hex/bin plus zero byte.
|
||||
|
||||
size_t pos = 0;
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
/// Manual exponential growth, so as not to rely on the linear amortized work time of `resize` (no one guarantees it).
|
||||
if (pos + MAX_LENGTH > out_vec.size())
|
||||
out_vec.resize(out_vec.size() * word_size + MAX_LENGTH);
|
||||
|
||||
char * begin = reinterpret_cast<char *>(&out_vec[pos]);
|
||||
char * end = begin;
|
||||
|
||||
Impl::executeOneUIntOrInt(ip[i].toUnderType(), end);
|
||||
|
||||
pos += end - begin;
|
||||
out_offsets[i] = pos;
|
||||
}
|
||||
out_vec.resize(pos);
|
||||
|
||||
col_res = std::move(col_str);
|
||||
return true;
|
||||
}
|
||||
};
|
||||
|
||||
/// Decode number or string from string with binary or hexadecimal representation
|
||||
|
@ -1,3 +1,4 @@
|
||||
#include <functional>
|
||||
#ifdef HAS_RESERVED_IDENTIFIER
|
||||
#pragma clang diagnostic ignored "-Wreserved-identifier"
|
||||
#endif
|
||||
@ -17,6 +18,7 @@
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <DataTypes/DataTypeTuple.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <DataTypes/DataTypeIPv4andIPv6.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <Functions/IFunction.h>
|
||||
@ -44,8 +46,8 @@ namespace ErrorCodes
|
||||
|
||||
/** Encoding functions for network addresses:
|
||||
*
|
||||
* IPv4NumToString (num) - See below.
|
||||
* IPv4StringToNum(string) - Convert, for example, '192.168.0.1' to 3232235521 and vice versa.
|
||||
* IPv6NumToString (num) - See below.
|
||||
* IPv6StringToNum(string) - Convert, for example, '::1' to 1 and vice versa.
|
||||
*/
|
||||
class FunctionIPv6NumToString : public IFunction
|
||||
{
|
||||
@ -61,60 +63,65 @@ public:
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
const auto * ptr = checkAndGetDataType<DataTypeFixedString>(arguments[0].get());
|
||||
if (!ptr || ptr->getN() != IPV6_BINARY_LENGTH)
|
||||
throw Exception("Illegal type " + arguments[0]->getName() +
|
||||
" of argument of function " + getName() +
|
||||
", expected FixedString(" + toString(IPV6_BINARY_LENGTH) + ")",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
const auto * arg_string = checkAndGetDataType<DataTypeFixedString>(arguments[0].get());
|
||||
const auto * arg_ipv6 = checkAndGetDataType<DataTypeIPv6>(arguments[0].get());
|
||||
if (!arg_ipv6 && !(arg_string && arg_string->getN() == IPV6_BINARY_LENGTH))
|
||||
throw Exception(
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
||||
"Illegal type {} of argument of function {}, expected IPv6 or FixedString({})",
|
||||
arguments[0]->getName(), getName(), IPV6_BINARY_LENGTH
|
||||
);
|
||||
|
||||
return std::make_shared<DataTypeString>();
|
||||
}
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/) const override
|
||||
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override
|
||||
{
|
||||
const auto & col_type_name = arguments[0];
|
||||
const ColumnPtr & column = col_type_name.column;
|
||||
|
||||
if (const auto * col_in = checkAndGetColumn<ColumnFixedString>(column.get()))
|
||||
{
|
||||
if (col_in->getN() != IPV6_BINARY_LENGTH)
|
||||
throw Exception("Illegal type " + col_type_name.type->getName() +
|
||||
" of column " + col_in->getName() +
|
||||
" argument of function " + getName() +
|
||||
", expected FixedString(" + toString(IPV6_BINARY_LENGTH) + ")",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
const auto size = col_in->size();
|
||||
const auto & vec_in = col_in->getChars();
|
||||
const ColumnPtr & column = arguments[0].column;
|
||||
const auto * col_ipv6 = checkAndGetColumn<ColumnIPv6>(column.get());
|
||||
const auto * col_string = checkAndGetColumn<ColumnFixedString>(column.get());
|
||||
if (!col_ipv6 && !(col_string && col_string->getN() == IPV6_BINARY_LENGTH))
|
||||
throw Exception(
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
||||
"Illegal column {} of argument of function {}, expected IPv6 or FixedString({})",
|
||||
arguments[0].name, getName(), IPV6_BINARY_LENGTH
|
||||
);
|
||||
|
||||
auto col_res = ColumnString::create();
|
||||
|
||||
ColumnString::Chars & vec_res = col_res->getChars();
|
||||
ColumnString::Offsets & offsets_res = col_res->getOffsets();
|
||||
vec_res.resize(size * (IPV6_MAX_TEXT_LENGTH + 1));
|
||||
offsets_res.resize(size);
|
||||
vec_res.resize(input_rows_count * (IPV6_MAX_TEXT_LENGTH + 1));
|
||||
offsets_res.resize(input_rows_count);
|
||||
|
||||
auto * begin = reinterpret_cast<char *>(vec_res.data());
|
||||
auto * pos = begin;
|
||||
|
||||
for (size_t offset = 0, i = 0; offset < vec_in.size(); offset += IPV6_BINARY_LENGTH, ++i)
|
||||
if (col_ipv6)
|
||||
{
|
||||
formatIPv6(reinterpret_cast<const unsigned char *>(&vec_in[offset]), pos);
|
||||
const auto & vec_in = col_ipv6->getData();
|
||||
|
||||
for (size_t i = 0; i < input_rows_count; ++i)
|
||||
{
|
||||
formatIPv6(reinterpret_cast<const unsigned char *>(&vec_in[i]), pos);
|
||||
offsets_res[i] = pos - begin;
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
const auto & vec_in = col_string->getChars();
|
||||
|
||||
for (size_t i = 0; i < input_rows_count; ++i)
|
||||
{
|
||||
formatIPv6(reinterpret_cast<const unsigned char *>(&vec_in[i * IPV6_BINARY_LENGTH]), pos);
|
||||
offsets_res[i] = pos - begin;
|
||||
}
|
||||
}
|
||||
|
||||
vec_res.resize(pos - begin);
|
||||
|
||||
return col_res;
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal column " + arguments[0].column->getName()
|
||||
+ " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
@ -302,14 +309,14 @@ public:
|
||||
{
|
||||
if (cast_ipv4_ipv6_default_on_conversion_error)
|
||||
{
|
||||
auto result = convertToIPv6<IPStringToNumExceptionMode::Default>(column, null_map);
|
||||
auto result = convertToIPv6<IPStringToNumExceptionMode::Default, ColumnFixedString>(column, null_map);
|
||||
if (null_map && !result->isNullable())
|
||||
return ColumnNullable::create(result, null_map_column);
|
||||
return result;
|
||||
}
|
||||
}
|
||||
|
||||
auto result = convertToIPv6<exception_mode>(column, null_map);
|
||||
auto result = convertToIPv6<exception_mode, ColumnFixedString>(column, null_map);
|
||||
if (null_map && !result->isNullable())
|
||||
return ColumnNullable::create(IColumn::mutate(result), IColumn::mutate(null_map_column));
|
||||
return result;
|
||||
@ -325,6 +332,43 @@ private:
|
||||
template <size_t mask_tail_octets, typename Name>
|
||||
class FunctionIPv4NumToString : public IFunction
|
||||
{
|
||||
private:
|
||||
template <typename ArgType>
|
||||
ColumnPtr executeTyped(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/) const
|
||||
{
|
||||
using ColumnType = ColumnVector<ArgType>;
|
||||
|
||||
const ColumnPtr & column = arguments[0].column;
|
||||
|
||||
if (const ColumnType * col = typeid_cast<const ColumnType *>(column.get()))
|
||||
{
|
||||
const typename ColumnType::Container & vec_in = col->getData();
|
||||
|
||||
auto col_res = ColumnString::create();
|
||||
|
||||
ColumnString::Chars & vec_res = col_res->getChars();
|
||||
ColumnString::Offsets & offsets_res = col_res->getOffsets();
|
||||
|
||||
vec_res.resize(vec_in.size() * (IPV4_MAX_TEXT_LENGTH + 1)); /// the longest value is: 255.255.255.255\0
|
||||
offsets_res.resize(vec_in.size());
|
||||
char * begin = reinterpret_cast<char *>(vec_res.data());
|
||||
char * pos = begin;
|
||||
|
||||
for (size_t i = 0; i < vec_in.size(); ++i)
|
||||
{
|
||||
DB::formatIPv4(reinterpret_cast<const unsigned char*>(&vec_in[i]), sizeof(ArgType), pos, mask_tail_octets, "xxx");
|
||||
offsets_res[i] = pos - begin;
|
||||
}
|
||||
|
||||
vec_res.resize(pos - begin);
|
||||
|
||||
return col_res;
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal column " + arguments[0].column->getName()
|
||||
+ " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
public:
|
||||
static constexpr auto name = Name::name;
|
||||
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionIPv4NumToString<mask_tail_octets, Name>>(); }
|
||||
@ -340,47 +384,36 @@ public:
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
if (!WhichDataType(arguments[0]).isUInt32())
|
||||
throw Exception("Illegal type " + arguments[0]->getName() + " of argument of function " + getName() + ", expected UInt32",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
WhichDataType arg_type(arguments[0]);
|
||||
if (!(arg_type.isIPv4() || arg_type.isUInt8() || arg_type.isUInt16() || arg_type.isUInt32()))
|
||||
throw Exception(
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
||||
"Illegal type {} of first argument of function {}, expected IPv4 or UInt8 or UInt16 or UInt32",
|
||||
arguments[0]->getName(), getName()
|
||||
);
|
||||
|
||||
return std::make_shared<DataTypeString>();
|
||||
}
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/) const override
|
||||
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & ret_type, size_t input_rows_count) const override
|
||||
{
|
||||
const ColumnPtr & column = arguments[0].column;
|
||||
|
||||
if (const ColumnUInt32 * col = typeid_cast<const ColumnUInt32 *>(column.get()))
|
||||
switch (arguments[0].type->getTypeId())
|
||||
{
|
||||
const ColumnUInt32::Container & vec_in = col->getData();
|
||||
|
||||
auto col_res = ColumnString::create();
|
||||
|
||||
ColumnString::Chars & vec_res = col_res->getChars();
|
||||
ColumnString::Offsets & offsets_res = col_res->getOffsets();
|
||||
|
||||
vec_res.resize(vec_in.size() * (IPV4_MAX_TEXT_LENGTH + 1)); /// the longest value is: 255.255.255.255\0
|
||||
offsets_res.resize(vec_in.size());
|
||||
char * begin = reinterpret_cast<char *>(vec_res.data());
|
||||
char * pos = begin;
|
||||
|
||||
for (size_t i = 0; i < vec_in.size(); ++i)
|
||||
{
|
||||
DB::formatIPv4(reinterpret_cast<const unsigned char*>(&vec_in[i]), pos, mask_tail_octets, "xxx");
|
||||
offsets_res[i] = pos - begin;
|
||||
case TypeIndex::IPv4: return executeTyped<IPv4>(arguments, ret_type, input_rows_count);
|
||||
case TypeIndex::UInt8: return executeTyped<UInt8>(arguments, ret_type, input_rows_count);
|
||||
case TypeIndex::UInt16: return executeTyped<UInt16>(arguments, ret_type, input_rows_count);
|
||||
case TypeIndex::UInt32: return executeTyped<UInt32>(arguments, ret_type, input_rows_count);
|
||||
default: break;
|
||||
}
|
||||
|
||||
vec_res.resize(pos - begin);
|
||||
|
||||
return col_res;
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal column " + arguments[0].column->getName()
|
||||
+ " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
throw Exception(
|
||||
ErrorCodes::ILLEGAL_COLUMN,
|
||||
"Illegal column {} of argument of function {}, expected IPv4 or UInt8 or UInt16 or UInt32",
|
||||
arguments[0].column->getName(), getName()
|
||||
);
|
||||
}
|
||||
};
|
||||
|
||||
@ -444,14 +477,14 @@ public:
|
||||
{
|
||||
if (cast_ipv4_ipv6_default_on_conversion_error)
|
||||
{
|
||||
auto result = convertToIPv4<IPStringToNumExceptionMode::Default>(column, null_map);
|
||||
auto result = convertToIPv4<IPStringToNumExceptionMode::Default, ColumnUInt32>(column, null_map);
|
||||
if (null_map && !result->isNullable())
|
||||
return ColumnNullable::create(result, null_map_column);
|
||||
return result;
|
||||
}
|
||||
}
|
||||
|
||||
auto result = convertToIPv4<exception_mode>(column, null_map);
|
||||
auto result = convertToIPv4<exception_mode, ColumnUInt32>(column, null_map);
|
||||
if (null_map && !result->isNullable())
|
||||
return ColumnNullable::create(IColumn::mutate(result), IColumn::mutate(null_map_column));
|
||||
return result;
|
||||
@ -474,13 +507,21 @@ public:
|
||||
bool isInjective(const ColumnsWithTypeAndName &) const override { return true; }
|
||||
bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; }
|
||||
|
||||
/// for backward compatibility IPv4ToIPv6 is overloaded, and result type depends on type of argument -
|
||||
/// if it is UInt32 (presenting IPv4) then result is FixedString(16), if IPv4 - result is IPv6
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
if (!checkAndGetDataType<DataTypeUInt32>(arguments[0].get()))
|
||||
throw Exception("Illegal type " + arguments[0]->getName() +
|
||||
" of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
const auto * dt_uint32 = checkAndGetDataType<DataTypeUInt32>(arguments[0].get());
|
||||
const auto * dt_ipv4 = checkAndGetDataType<DataTypeIPv4>(arguments[0].get());
|
||||
if (!dt_uint32 && !dt_ipv4)
|
||||
throw Exception(
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
||||
"Illegal type {} of argument of function {}", arguments[0]->getName(), getName()
|
||||
);
|
||||
|
||||
if (dt_uint32)
|
||||
return std::make_shared<DataTypeFixedString>(16);
|
||||
return std::make_shared<DataTypeIPv6>();
|
||||
}
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
@ -490,7 +531,22 @@ public:
|
||||
const auto & col_type_name = arguments[0];
|
||||
const ColumnPtr & column = col_type_name.column;
|
||||
|
||||
if (const auto * col_in = typeid_cast<const ColumnUInt32 *>(column.get()))
|
||||
if (const auto * col_in = checkAndGetColumn<ColumnIPv4>(*column))
|
||||
{
|
||||
auto col_res = ColumnIPv6::create();
|
||||
|
||||
auto & vec_res = col_res->getData();
|
||||
vec_res.resize(col_in->size());
|
||||
|
||||
const auto & vec_in = col_in->getData();
|
||||
|
||||
for (size_t i = 0; i < vec_res.size(); ++i)
|
||||
mapIPv4ToIPv6(vec_in[i], reinterpret_cast<UInt8 *>(&vec_res[i].toUnderType()));
|
||||
|
||||
return col_res;
|
||||
}
|
||||
|
||||
if (const auto * col_in = checkAndGetColumn<ColumnUInt32>(*column))
|
||||
{
|
||||
auto col_res = ColumnFixedString::create(IPV6_BINARY_LENGTH);
|
||||
|
||||
@ -504,10 +560,11 @@ public:
|
||||
|
||||
return col_res;
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal column " + arguments[0].column->getName()
|
||||
+ " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
||||
throw Exception(
|
||||
ErrorCodes::ILLEGAL_COLUMN,
|
||||
"Illegal column {} of argument of function {}", arguments[0].column->getName(), getName()
|
||||
);
|
||||
}
|
||||
|
||||
private:
|
||||
@ -522,82 +579,6 @@ private:
|
||||
}
|
||||
};
|
||||
|
||||
template <IPStringToNumExceptionMode exception_mode>
|
||||
class FunctionToIPv4 : public FunctionIPv4StringToNum<exception_mode>
|
||||
{
|
||||
public:
|
||||
using Base = FunctionIPv4StringToNum<exception_mode>;
|
||||
|
||||
static constexpr auto name = exception_mode == IPStringToNumExceptionMode::Throw
|
||||
? "toIPv4"
|
||||
: (exception_mode == IPStringToNumExceptionMode::Default ? "toIPv4OrDefault" : "toIPv4OrNull");
|
||||
|
||||
static FunctionPtr create(ContextPtr context) { return std::make_shared<FunctionToIPv4>(context); }
|
||||
|
||||
explicit FunctionToIPv4(ContextPtr context) : Base(context) { }
|
||||
|
||||
String getName() const override { return name; }
|
||||
|
||||
bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; }
|
||||
|
||||
size_t getNumberOfArguments() const override { return 1; }
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
if (!isString(removeNullable(arguments[0])))
|
||||
{
|
||||
throw Exception(
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of argument of function {}", arguments[0]->getName(), getName());
|
||||
}
|
||||
|
||||
auto result_type = DataTypeFactory::instance().get("IPv4");
|
||||
|
||||
if constexpr (exception_mode == IPStringToNumExceptionMode::Null)
|
||||
{
|
||||
return makeNullable(result_type);
|
||||
}
|
||||
|
||||
return arguments[0]->isNullable() ? makeNullable(result_type) : result_type;
|
||||
}
|
||||
};
|
||||
|
||||
template <IPStringToNumExceptionMode exception_mode>
|
||||
class FunctionToIPv6 : public FunctionIPv6StringToNum<exception_mode>
|
||||
{
|
||||
public:
|
||||
using Base = FunctionIPv6StringToNum<exception_mode>;
|
||||
|
||||
static constexpr auto name = exception_mode == IPStringToNumExceptionMode::Throw
|
||||
? "toIPv6"
|
||||
: (exception_mode == IPStringToNumExceptionMode::Default ? "toIPv6OrDefault" : "toIPv6OrNull");
|
||||
|
||||
static FunctionPtr create(ContextPtr context) { return std::make_shared<FunctionToIPv6>(context); }
|
||||
|
||||
explicit FunctionToIPv6(ContextPtr context) : Base(context) { }
|
||||
|
||||
String getName() const override { return name; }
|
||||
|
||||
bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; }
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
if (!isStringOrFixedString(removeNullable(arguments[0])))
|
||||
{
|
||||
throw Exception(
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of argument of function {}", arguments[0]->getName(), getName());
|
||||
}
|
||||
|
||||
auto result_type = DataTypeFactory::instance().get("IPv6");
|
||||
|
||||
if constexpr (exception_mode == IPStringToNumExceptionMode::Null)
|
||||
{
|
||||
return makeNullable(result_type);
|
||||
}
|
||||
|
||||
return arguments[0]->isNullable() ? makeNullable(result_type) : result_type;
|
||||
}
|
||||
};
|
||||
|
||||
class FunctionMACNumToString : public IFunction
|
||||
{
|
||||
public:
|
||||
@ -800,7 +781,7 @@ private:
|
||||
|
||||
#include <emmintrin.h>
|
||||
|
||||
static inline void applyCIDRMask(const UInt8 * __restrict src, UInt8 * __restrict dst_lower, UInt8 * __restrict dst_upper, UInt8 bits_to_keep)
|
||||
static inline void applyCIDRMask(const char * __restrict src, char * __restrict dst_lower, char * __restrict dst_upper, UInt8 bits_to_keep)
|
||||
{
|
||||
__m128i mask = _mm_loadu_si128(reinterpret_cast<const __m128i *>(getCIDRMaskIPv6(bits_to_keep).data()));
|
||||
__m128i lower = _mm_and_si128(_mm_loadu_si128(reinterpret_cast<const __m128i *>(src)), mask);
|
||||
@ -814,7 +795,7 @@ private:
|
||||
#else
|
||||
|
||||
/// NOTE IPv6 is stored in memory in big endian format that makes some difficulties.
|
||||
static void applyCIDRMask(const UInt8 * __restrict src, UInt8 * __restrict dst_lower, UInt8 * __restrict dst_upper, UInt8 bits_to_keep)
|
||||
static void applyCIDRMask(const char * __restrict src, char * __restrict dst_lower, char * __restrict dst_upper, UInt8 bits_to_keep)
|
||||
{
|
||||
const auto & mask = getCIDRMaskIPv6(bits_to_keep);
|
||||
|
||||
@ -837,20 +818,24 @@ public:
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
const auto * first_argument = checkAndGetDataType<DataTypeFixedString>(arguments[0].get());
|
||||
if (!first_argument || first_argument->getN() != IPV6_BINARY_LENGTH)
|
||||
throw Exception("Illegal type " + arguments[0]->getName() +
|
||||
" of first argument of function " + getName() +
|
||||
", expected FixedString(" + toString(IPV6_BINARY_LENGTH) + ")",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
const auto * ipv6 = checkAndGetDataType<DataTypeIPv6>(arguments[0].get());
|
||||
const auto * str = checkAndGetDataType<DataTypeFixedString>(arguments[0].get());
|
||||
if (!ipv6 && !(str && str->getN() == IPV6_BINARY_LENGTH))
|
||||
throw Exception(
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
||||
"Illegal type {} of first argument of function {}, expected IPv6 or FixedString({})",
|
||||
arguments[0]->getName(), getName(), IPV6_BINARY_LENGTH
|
||||
);
|
||||
|
||||
const DataTypePtr & second_argument = arguments[1];
|
||||
if (!isUInt8(second_argument))
|
||||
throw Exception{"Illegal type " + second_argument->getName()
|
||||
+ " of second argument of function " + getName()
|
||||
+ ", expected UInt8", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
|
||||
throw Exception(
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
||||
"Illegal type {} of second argument of function {}, expected UInt8",
|
||||
second_argument->getName(), getName()
|
||||
);
|
||||
|
||||
DataTypePtr element = DataTypeFactory::instance().get("IPv6");
|
||||
DataTypePtr element = std::make_shared<DataTypeIPv6>();
|
||||
return std::make_shared<DataTypeTuple>(DataTypes{element, element});
|
||||
}
|
||||
|
||||
@ -859,25 +844,6 @@ public:
|
||||
|
||||
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override
|
||||
{
|
||||
const auto & col_type_name_ip = arguments[0];
|
||||
const ColumnPtr & column_ip = col_type_name_ip.column;
|
||||
|
||||
const auto * col_const_ip_in = checkAndGetColumnConst<ColumnFixedString>(column_ip.get());
|
||||
const auto * col_ip_in = checkAndGetColumn<ColumnFixedString>(column_ip.get());
|
||||
|
||||
if (!col_ip_in && !col_const_ip_in)
|
||||
throw Exception("Illegal column " + arguments[0].column->getName()
|
||||
+ " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
||||
if ((col_const_ip_in && col_const_ip_in->getValue<String>().size() != IPV6_BINARY_LENGTH) ||
|
||||
(col_ip_in && col_ip_in->getN() != IPV6_BINARY_LENGTH))
|
||||
throw Exception("Illegal type " + col_type_name_ip.type->getName() +
|
||||
" of column " + column_ip->getName() +
|
||||
" argument of function " + getName() +
|
||||
", expected FixedString(" + toString(IPV6_BINARY_LENGTH) + ")",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
const auto & col_type_name_cidr = arguments[1];
|
||||
const ColumnPtr & column_cidr = col_type_name_cidr.column;
|
||||
|
||||
@ -885,39 +851,57 @@ public:
|
||||
const auto * col_cidr_in = checkAndGetColumn<ColumnUInt8>(column_cidr.get());
|
||||
|
||||
if (!col_const_cidr_in && !col_cidr_in)
|
||||
throw Exception("Illegal column " + arguments[1].column->getName()
|
||||
+ " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
throw Exception(
|
||||
ErrorCodes::ILLEGAL_COLUMN,
|
||||
"Illegal column {} of argument of function {}",
|
||||
arguments[1].column->getName(), getName()
|
||||
);
|
||||
|
||||
auto col_res_lower_range = ColumnFixedString::create(IPV6_BINARY_LENGTH);
|
||||
auto col_res_upper_range = ColumnFixedString::create(IPV6_BINARY_LENGTH);
|
||||
const auto & col_type_name_ip = arguments[0];
|
||||
const ColumnPtr & column_ip = col_type_name_ip.column;
|
||||
|
||||
ColumnString::Chars & vec_res_lower_range = col_res_lower_range->getChars();
|
||||
vec_res_lower_range.resize(input_rows_count * IPV6_BINARY_LENGTH);
|
||||
const auto * col_const_ip_in = checkAndGetColumnConst<ColumnIPv6>(column_ip.get());
|
||||
const auto * col_ip_in = checkAndGetColumn<ColumnIPv6>(column_ip.get());
|
||||
|
||||
ColumnString::Chars & vec_res_upper_range = col_res_upper_range->getChars();
|
||||
vec_res_upper_range.resize(input_rows_count * IPV6_BINARY_LENGTH);
|
||||
const auto * col_const_str_in = checkAndGetColumnConst<ColumnFixedString>(column_ip.get());
|
||||
const auto * col_str_in = checkAndGetColumn<ColumnFixedString>(column_ip.get());
|
||||
|
||||
std::function<const char *(size_t)> get_ip_data;
|
||||
if (col_const_ip_in)
|
||||
get_ip_data = [col_const_ip_in](size_t) { return col_const_ip_in->getDataAt(0).data; };
|
||||
else if (col_const_str_in)
|
||||
get_ip_data = [col_const_str_in](size_t) { return col_const_str_in->getDataAt(0).data; };
|
||||
else if (col_ip_in)
|
||||
get_ip_data = [col_ip_in](size_t i) { return reinterpret_cast<const char *>(&col_ip_in->getData()[i]); };
|
||||
else if (col_str_in)
|
||||
get_ip_data = [col_str_in](size_t i) { return reinterpret_cast<const char *>(&col_str_in->getChars().data()[i * IPV6_BINARY_LENGTH]); };
|
||||
else
|
||||
throw Exception(
|
||||
ErrorCodes::ILLEGAL_COLUMN,
|
||||
"Illegal column {} of argument of function {}",
|
||||
arguments[0].column->getName(), getName()
|
||||
);
|
||||
|
||||
auto col_res_lower_range = ColumnIPv6::create();
|
||||
auto col_res_upper_range = ColumnIPv6::create();
|
||||
|
||||
auto & vec_res_lower_range = col_res_lower_range->getData();
|
||||
vec_res_lower_range.resize(input_rows_count);
|
||||
|
||||
auto & vec_res_upper_range = col_res_upper_range->getData();
|
||||
vec_res_upper_range.resize(input_rows_count);
|
||||
|
||||
static constexpr UInt8 max_cidr_mask = IPV6_BINARY_LENGTH * 8;
|
||||
|
||||
const String col_const_ip_str = col_const_ip_in ? col_const_ip_in->getValue<String>() : "";
|
||||
const UInt8 * col_const_ip_value = col_const_ip_in ? reinterpret_cast<const UInt8 *>(col_const_ip_str.c_str()) : nullptr;
|
||||
|
||||
for (size_t offset = 0; offset < input_rows_count; ++offset)
|
||||
for (size_t i = 0; i < input_rows_count; ++i)
|
||||
{
|
||||
const size_t offset_ipv6 = offset * IPV6_BINARY_LENGTH;
|
||||
|
||||
const UInt8 * ip = col_const_ip_in
|
||||
? col_const_ip_value
|
||||
: &col_ip_in->getChars()[offset_ipv6];
|
||||
|
||||
UInt8 cidr = col_const_cidr_in
|
||||
? col_const_cidr_in->getValue<UInt8>()
|
||||
: col_cidr_in->getData()[offset];
|
||||
: col_cidr_in->getData()[i];
|
||||
|
||||
cidr = std::min(cidr, max_cidr_mask);
|
||||
|
||||
applyCIDRMask(ip, &vec_res_lower_range[offset_ipv6], &vec_res_upper_range[offset_ipv6], cidr);
|
||||
applyCIDRMask(get_ip_data(i), reinterpret_cast<char *>(&vec_res_lower_range[i]), reinterpret_cast<char *>(&vec_res_upper_range[i]), cidr);
|
||||
}
|
||||
|
||||
return ColumnTuple::create(Columns{std::move(col_res_lower_range), std::move(col_res_upper_range)});
|
||||
@ -942,47 +926,15 @@ private:
|
||||
return { lower, upper };
|
||||
}
|
||||
|
||||
public:
|
||||
static constexpr auto name = "IPv4CIDRToRange";
|
||||
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionIPv4CIDRToRange>(); }
|
||||
|
||||
String getName() const override { return name; }
|
||||
size_t getNumberOfArguments() const override { return 2; }
|
||||
bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; }
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
if (!WhichDataType(arguments[0]).isUInt32())
|
||||
throw Exception("Illegal type " + arguments[0]->getName() +
|
||||
" of first argument of function " + getName() +
|
||||
", expected UInt32",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
|
||||
const DataTypePtr & second_argument = arguments[1];
|
||||
if (!isUInt8(second_argument))
|
||||
throw Exception{"Illegal type " + second_argument->getName()
|
||||
+ " of second argument of function " + getName()
|
||||
+ ", expected UInt8", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
|
||||
|
||||
DataTypePtr element = DataTypeFactory::instance().get("IPv4");
|
||||
return std::make_shared<DataTypeTuple>(DataTypes{element, element});
|
||||
}
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
|
||||
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override
|
||||
template <typename ArgType>
|
||||
ColumnPtr executeTyped(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const
|
||||
{
|
||||
using ColumnType = ColumnVector<ArgType>;
|
||||
const auto & col_type_name_ip = arguments[0];
|
||||
const ColumnPtr & column_ip = col_type_name_ip.column;
|
||||
|
||||
const auto * col_const_ip_in = checkAndGetColumnConst<ColumnUInt32>(column_ip.get());
|
||||
const auto * col_ip_in = checkAndGetColumn<ColumnUInt32>(column_ip.get());
|
||||
if (!col_const_ip_in && !col_ip_in)
|
||||
throw Exception("Illegal column " + arguments[0].column->getName()
|
||||
+ " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
const auto * col_const_ip_in = checkAndGetColumnConst<ColumnType>(column_ip.get());
|
||||
const auto * col_ip_in = checkAndGetColumn<ColumnType>(column_ip.get());
|
||||
|
||||
const auto & col_type_name_cidr = arguments[1];
|
||||
const ColumnPtr & column_cidr = col_type_name_cidr.column;
|
||||
@ -990,13 +942,8 @@ public:
|
||||
const auto * col_const_cidr_in = checkAndGetColumnConst<ColumnUInt8>(column_cidr.get());
|
||||
const auto * col_cidr_in = checkAndGetColumn<ColumnUInt8>(column_cidr.get());
|
||||
|
||||
if (!col_const_cidr_in && !col_cidr_in)
|
||||
throw Exception("Illegal column " + arguments[1].column->getName()
|
||||
+ " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
||||
auto col_res_lower_range = ColumnUInt32::create();
|
||||
auto col_res_upper_range = ColumnUInt32::create();
|
||||
auto col_res_lower_range = ColumnIPv4::create();
|
||||
auto col_res_upper_range = ColumnIPv4::create();
|
||||
|
||||
auto & vec_res_lower_range = col_res_lower_range->getData();
|
||||
vec_res_lower_range.resize(input_rows_count);
|
||||
@ -1006,8 +953,8 @@ public:
|
||||
|
||||
for (size_t i = 0; i < input_rows_count; ++i)
|
||||
{
|
||||
UInt32 ip = col_const_ip_in
|
||||
? col_const_ip_in->getValue<UInt32>()
|
||||
ArgType ip = col_const_ip_in
|
||||
? col_const_ip_in->template getValue<ArgType>()
|
||||
: col_ip_in->getData()[i];
|
||||
|
||||
UInt8 cidr = col_const_cidr_in
|
||||
@ -1019,6 +966,64 @@ public:
|
||||
|
||||
return ColumnTuple::create(Columns{std::move(col_res_lower_range), std::move(col_res_upper_range)});
|
||||
}
|
||||
|
||||
public:
|
||||
static constexpr auto name = "IPv4CIDRToRange";
|
||||
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionIPv4CIDRToRange>(); }
|
||||
|
||||
String getName() const override { return name; }
|
||||
size_t getNumberOfArguments() const override { return 2; }
|
||||
bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; }
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
WhichDataType arg_type(arguments[0]);
|
||||
if (!(arg_type.isIPv4() || arg_type.isUInt8() || arg_type.isUInt16() || arg_type.isUInt32()))
|
||||
throw Exception(
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
||||
"Illegal type {} of first argument of function {}, expected IPv4 or UInt8 or UInt16 or UInt32",
|
||||
arguments[0]->getName(), getName()
|
||||
);
|
||||
|
||||
|
||||
const DataTypePtr & second_argument = arguments[1];
|
||||
if (!isUInt8(second_argument))
|
||||
throw Exception(
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
||||
"Illegal type {} of second argument of function {}, expected UInt8",
|
||||
second_argument->getName(), getName()
|
||||
);
|
||||
|
||||
DataTypePtr element = DataTypeFactory::instance().get("IPv4");
|
||||
return std::make_shared<DataTypeTuple>(DataTypes{element, element});
|
||||
}
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
|
||||
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & ret_type, size_t input_rows_count) const override
|
||||
{
|
||||
if (arguments[1].type->getTypeId() != TypeIndex::UInt8)
|
||||
throw Exception(
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
||||
"Illegal type {} of second argument of function {}, expected UInt8", arguments[1].type->getName(), getName()
|
||||
);
|
||||
|
||||
switch (arguments[0].type->getTypeId())
|
||||
{
|
||||
case TypeIndex::IPv4: return executeTyped<IPv4>(arguments, ret_type, input_rows_count);
|
||||
case TypeIndex::UInt8: return executeTyped<UInt8>(arguments, ret_type, input_rows_count);
|
||||
case TypeIndex::UInt16: return executeTyped<UInt16>(arguments, ret_type, input_rows_count);
|
||||
case TypeIndex::UInt32: return executeTyped<UInt32>(arguments, ret_type, input_rows_count);
|
||||
default: break;
|
||||
}
|
||||
|
||||
throw Exception(
|
||||
ErrorCodes::ILLEGAL_COLUMN,
|
||||
"Illegal column {} of argument of function {}, expected IPv4 or UInt8 or UInt16 or UInt32",
|
||||
arguments[0].column->getName(), getName()
|
||||
);
|
||||
}
|
||||
};
|
||||
|
||||
class FunctionIsIPv4String : public IFunction
|
||||
@ -1066,7 +1071,7 @@ public:
|
||||
|
||||
for (size_t i = 0; i < vec_res.size(); ++i)
|
||||
{
|
||||
vec_res[i] = DB::parseIPv4(reinterpret_cast<const char *>(&vec_src[prev_offset]), reinterpret_cast<unsigned char *>(&result));
|
||||
vec_res[i] = DB::parseIPv4whole(reinterpret_cast<const char *>(&vec_src[prev_offset]), reinterpret_cast<unsigned char *>(&result));
|
||||
prev_offset = offsets_src[i];
|
||||
}
|
||||
|
||||
@ -1121,7 +1126,7 @@ public:
|
||||
|
||||
for (size_t i = 0; i < vec_res.size(); ++i)
|
||||
{
|
||||
vec_res[i] = DB::parseIPv6(reinterpret_cast<const char *>(&vec_src[prev_offset]), reinterpret_cast<unsigned char *>(buffer));
|
||||
vec_res[i] = DB::parseIPv6whole(reinterpret_cast<const char *>(&vec_src[prev_offset]), reinterpret_cast<unsigned char *>(buffer));
|
||||
prev_offset = offsets_src[i];
|
||||
}
|
||||
|
||||
@ -1150,18 +1155,11 @@ REGISTER_FUNCTION(Coding)
|
||||
factory.registerFunction<FunctionIPv4StringToNum<IPStringToNumExceptionMode::Throw>>();
|
||||
factory.registerFunction<FunctionIPv4StringToNum<IPStringToNumExceptionMode::Default>>();
|
||||
factory.registerFunction<FunctionIPv4StringToNum<IPStringToNumExceptionMode::Null>>();
|
||||
factory.registerFunction<FunctionToIPv4<IPStringToNumExceptionMode::Throw>>();
|
||||
factory.registerFunction<FunctionToIPv4<IPStringToNumExceptionMode::Default>>();
|
||||
factory.registerFunction<FunctionToIPv4<IPStringToNumExceptionMode::Null>>();
|
||||
|
||||
factory.registerFunction<FunctionIPv6NumToString>();
|
||||
factory.registerFunction<FunctionIPv6StringToNum<IPStringToNumExceptionMode::Throw>>();
|
||||
factory.registerFunction<FunctionIPv6StringToNum<IPStringToNumExceptionMode::Default>>();
|
||||
factory.registerFunction<FunctionIPv6StringToNum<IPStringToNumExceptionMode::Null>>();
|
||||
factory.registerFunction<FunctionToIPv6<IPStringToNumExceptionMode::Throw>>();
|
||||
factory.registerFunction<FunctionToIPv6<IPStringToNumExceptionMode::Default>>();
|
||||
factory.registerFunction<FunctionToIPv6<IPStringToNumExceptionMode::Null>>();
|
||||
|
||||
|
||||
/// MySQL compatibility aliases:
|
||||
factory.registerAlias("INET_ATON", FunctionIPv4StringToNum<IPStringToNumExceptionMode::Throw>::name, FunctionFactory::CaseInsensitive);
|
||||
|
@ -1,5 +1,6 @@
|
||||
#pragma once
|
||||
|
||||
#include <type_traits>
|
||||
#include <Common/formatIPv6.h>
|
||||
|
||||
#include <Columns/ColumnFixedString.h>
|
||||
@ -12,7 +13,8 @@ namespace DB
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int CANNOT_PARSE_DOMAIN_VALUE_FROM_STRING;
|
||||
extern const int CANNOT_PARSE_IPV4;
|
||||
extern const int CANNOT_PARSE_IPV6;
|
||||
extern const int ILLEGAL_COLUMN;
|
||||
}
|
||||
|
||||
@ -25,14 +27,18 @@ enum class IPStringToNumExceptionMode : uint8_t
|
||||
|
||||
static inline bool tryParseIPv4(const char * pos, UInt32 & result_value)
|
||||
{
|
||||
return parseIPv4(pos, reinterpret_cast<unsigned char *>(&result_value));
|
||||
return parseIPv4whole(pos, reinterpret_cast<unsigned char *>(&result_value));
|
||||
}
|
||||
|
||||
namespace detail
|
||||
{
|
||||
template <IPStringToNumExceptionMode exception_mode, typename StringColumnType>
|
||||
template <IPStringToNumExceptionMode exception_mode, typename ToColumn = ColumnIPv6, typename StringColumnType>
|
||||
ColumnPtr convertToIPv6(const StringColumnType & string_column, const PaddedPODArray<UInt8> * null_map = nullptr)
|
||||
{
|
||||
if constexpr (!std::is_same_v<ToColumn, ColumnFixedString> && !std::is_same_v<ToColumn, ColumnIPv6>)
|
||||
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal return column type {}. Expected IPv6 or FixedString", TypeName<typename ToColumn::ValueType>);
|
||||
|
||||
|
||||
size_t column_size = string_column.size();
|
||||
|
||||
ColumnUInt8::MutablePtr col_null_map_to;
|
||||
@ -44,10 +50,73 @@ namespace detail
|
||||
vec_null_map_to = &col_null_map_to->getData();
|
||||
}
|
||||
|
||||
auto col_res = ColumnFixedString::create(IPV6_BINARY_LENGTH);
|
||||
/// This is a special treatment for source column of type FixedString(16)
|
||||
/// to preserve previous behavior when IPv6 was a domain type of FixedString(16)
|
||||
if constexpr (std::is_same_v<StringColumnType, ColumnFixedString>)
|
||||
{
|
||||
if (string_column.getN() == IPV6_BINARY_LENGTH)
|
||||
{
|
||||
if constexpr (std::is_same_v<ToColumn, ColumnFixedString>)
|
||||
{
|
||||
auto col_res = ColumnFixedString::create(string_column);
|
||||
|
||||
if constexpr (exception_mode == IPStringToNumExceptionMode::Null)
|
||||
{
|
||||
col_null_map_to = ColumnUInt8::create(column_size, false);
|
||||
if (null_map)
|
||||
memcpy(col_null_map_to->getData().data(), null_map->data(), column_size);
|
||||
return ColumnNullable::create(std::move(col_res), std::move(col_null_map_to));
|
||||
}
|
||||
|
||||
return col_res;
|
||||
}
|
||||
else
|
||||
{
|
||||
auto col_res = ColumnIPv6::create();
|
||||
auto & vec_res = col_res->getData();
|
||||
|
||||
vec_res.resize(column_size);
|
||||
memcpy(vec_res.data(), string_column.getChars().data(), column_size * IPV6_BINARY_LENGTH);
|
||||
|
||||
if constexpr (exception_mode == IPStringToNumExceptionMode::Null)
|
||||
{
|
||||
col_null_map_to = ColumnUInt8::create(column_size, false);
|
||||
if (null_map)
|
||||
memcpy(col_null_map_to->getData().data(), null_map->data(), column_size);
|
||||
return ColumnNullable::create(std::move(col_res), std::move(col_null_map_to));
|
||||
}
|
||||
|
||||
return col_res;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
auto column_create = []() -> typename ToColumn::MutablePtr
|
||||
{
|
||||
if constexpr (std::is_same_v<ToColumn, ColumnFixedString>)
|
||||
return ColumnFixedString::create(IPV6_BINARY_LENGTH);
|
||||
else
|
||||
return ColumnIPv6::create();
|
||||
};
|
||||
|
||||
auto get_vector = [](auto & col_res, size_t col_size) -> decltype(auto)
|
||||
{
|
||||
if constexpr (std::is_same_v<ToColumn, ColumnFixedString>)
|
||||
{
|
||||
auto & vec_res = col_res->getChars();
|
||||
vec_res.resize(column_size * IPV6_BINARY_LENGTH);
|
||||
vec_res.resize(col_size * IPV6_BINARY_LENGTH);
|
||||
return (vec_res);
|
||||
}
|
||||
else
|
||||
{
|
||||
auto & vec_res = col_res->getData();
|
||||
vec_res.resize(col_size);
|
||||
return (vec_res);
|
||||
}
|
||||
};
|
||||
|
||||
auto col_res = column_create();
|
||||
auto & vec_res = get_vector(col_res, column_size);
|
||||
|
||||
using Chars = typename StringColumnType::Chars;
|
||||
const Chars & vec_src = string_column.getChars();
|
||||
@ -56,6 +125,7 @@ namespace detail
|
||||
char src_ipv4_buf[sizeof("::ffff:") + IPV4_MAX_TEXT_LENGTH + 1] = "::ffff:";
|
||||
|
||||
/// ColumnFixedString contains not null terminated strings. But functions parseIPv6, parseIPv4 expect null terminated string.
|
||||
/// TODO fix this - now parseIPv6/parseIPv4 accept end iterator, so can be parsed in-place
|
||||
std::string fixed_string_buffer;
|
||||
|
||||
if constexpr (std::is_same_v<StringColumnType, ColumnFixedString>)
|
||||
@ -63,7 +133,11 @@ namespace detail
|
||||
fixed_string_buffer.resize(string_column.getN());
|
||||
}
|
||||
|
||||
for (size_t out_offset = 0, i = 0; out_offset < vec_res.size(); out_offset += IPV6_BINARY_LENGTH, ++i)
|
||||
int offset_inc = 1;
|
||||
if constexpr (std::is_same_v<ToColumn, ColumnFixedString>)
|
||||
offset_inc = IPV6_BINARY_LENGTH;
|
||||
|
||||
for (size_t out_offset = 0, i = 0; i < column_size; out_offset += offset_inc, ++i)
|
||||
{
|
||||
size_t src_next_offset = src_offset;
|
||||
|
||||
@ -87,7 +161,7 @@ namespace detail
|
||||
|
||||
if (null_map && (*null_map)[i])
|
||||
{
|
||||
std::fill_n(&vec_res[i], IPV6_BINARY_LENGTH, 0);
|
||||
std::fill_n(&vec_res[out_offset], offset_inc, 0);
|
||||
src_offset = src_next_offset;
|
||||
if constexpr (exception_mode == IPStringToNumExceptionMode::Null)
|
||||
(*vec_null_map_to)[i] = true;
|
||||
@ -107,17 +181,17 @@ namespace detail
|
||||
src_ipv4_buf + std::strlen("::ffff:"),
|
||||
src_value,
|
||||
std::min<UInt64>(src_next_offset - src_offset, IPV4_MAX_TEXT_LENGTH + 1));
|
||||
parse_result = parseIPv6(src_ipv4_buf, res_value);
|
||||
parse_result = parseIPv6whole(src_ipv4_buf, res_value);
|
||||
}
|
||||
else
|
||||
{
|
||||
parse_result = parseIPv6(src_value, res_value);
|
||||
parse_result = parseIPv6whole(src_value, res_value);
|
||||
}
|
||||
|
||||
if (!parse_result)
|
||||
{
|
||||
if constexpr (exception_mode == IPStringToNumExceptionMode::Throw)
|
||||
throw Exception("Invalid IPv6 value", ErrorCodes::CANNOT_PARSE_DOMAIN_VALUE_FROM_STRING);
|
||||
throw Exception("Invalid IPv6 value", ErrorCodes::CANNOT_PARSE_IPV6);
|
||||
else if constexpr (exception_mode == IPStringToNumExceptionMode::Default)
|
||||
vec_res[i] = 0;
|
||||
else if constexpr (exception_mode == IPStringToNumExceptionMode::Null)
|
||||
@ -134,23 +208,16 @@ namespace detail
|
||||
}
|
||||
}
|
||||
|
||||
template <IPStringToNumExceptionMode exception_mode>
|
||||
template <IPStringToNumExceptionMode exception_mode, typename ToColumn = ColumnIPv6>
|
||||
ColumnPtr convertToIPv6(ColumnPtr column, const PaddedPODArray<UInt8> * null_map = nullptr)
|
||||
{
|
||||
size_t column_size = column->size();
|
||||
|
||||
auto col_res = ColumnFixedString::create(IPV6_BINARY_LENGTH);
|
||||
|
||||
auto & vec_res = col_res->getChars();
|
||||
vec_res.resize(column_size * IPV6_BINARY_LENGTH);
|
||||
|
||||
if (const auto * column_input_string = checkAndGetColumn<ColumnString>(column.get()))
|
||||
{
|
||||
return detail::convertToIPv6<exception_mode>(*column_input_string, null_map);
|
||||
return detail::convertToIPv6<exception_mode, ToColumn>(*column_input_string, null_map);
|
||||
}
|
||||
else if (const auto * column_input_fixed_string = checkAndGetColumn<ColumnFixedString>(column.get()))
|
||||
{
|
||||
return detail::convertToIPv6<exception_mode>(*column_input_fixed_string, null_map);
|
||||
return detail::convertToIPv6<exception_mode, ToColumn>(*column_input_fixed_string, null_map);
|
||||
}
|
||||
else
|
||||
{
|
||||
@ -158,7 +225,7 @@ ColumnPtr convertToIPv6(ColumnPtr column, const PaddedPODArray<UInt8> * null_map
|
||||
}
|
||||
}
|
||||
|
||||
template <IPStringToNumExceptionMode exception_mode>
|
||||
template <IPStringToNumExceptionMode exception_mode, typename ToColumn = ColumnIPv4>
|
||||
ColumnPtr convertToIPv4(ColumnPtr column, const PaddedPODArray<UInt8> * null_map = nullptr)
|
||||
{
|
||||
const ColumnString * column_string = checkAndGetColumn<ColumnString>(column.get());
|
||||
@ -179,9 +246,9 @@ ColumnPtr convertToIPv4(ColumnPtr column, const PaddedPODArray<UInt8> * null_map
|
||||
vec_null_map_to = &col_null_map_to->getData();
|
||||
}
|
||||
|
||||
auto col_res = ColumnUInt32::create();
|
||||
auto col_res = ToColumn::create();
|
||||
|
||||
ColumnUInt32::Container & vec_res = col_res->getData();
|
||||
auto & vec_res = col_res->getData();
|
||||
vec_res.resize(column_size);
|
||||
|
||||
const ColumnString::Chars & vec_src = column_string->getChars();
|
||||
@ -205,7 +272,7 @@ ColumnPtr convertToIPv4(ColumnPtr column, const PaddedPODArray<UInt8> * null_map
|
||||
{
|
||||
if constexpr (exception_mode == IPStringToNumExceptionMode::Throw)
|
||||
{
|
||||
throw Exception("Invalid IPv4 value", ErrorCodes::CANNOT_PARSE_DOMAIN_VALUE_FROM_STRING);
|
||||
throw Exception("Invalid IPv4 value", ErrorCodes::CANNOT_PARSE_IPV4);
|
||||
}
|
||||
else if constexpr (exception_mode == IPStringToNumExceptionMode::Default)
|
||||
{
|
||||
|
@ -1154,6 +1154,8 @@ public:
|
||||
/// You can compare the date, datetime, or datatime64 and an enumeration with a constant string.
|
||||
|| ((left.isDate() || left.isDate32() || left.isDateTime() || left.isDateTime64()) && (right.isDate() || right.isDate32() || right.isDateTime() || right.isDateTime64()) && left.idx == right.idx) /// only date vs date, or datetime vs datetime
|
||||
|| (left.isUUID() && right.isUUID())
|
||||
|| (left.isIPv4() && right.isIPv4())
|
||||
|| (left.isIPv6() && right.isIPv6())
|
||||
|| (left.isEnum() && right.isEnum() && arguments[0]->getName() == arguments[1]->getName()) /// only equivalent enum type values can be compared against
|
||||
|| (left_tuple && right_tuple && left_tuple->getElements().size() == right_tuple->getElements().size())
|
||||
|| (arguments[0]->equals(*arguments[1]))))
|
||||
@ -1245,6 +1247,15 @@ public:
|
||||
const bool left_is_float = which_left.isFloat();
|
||||
const bool right_is_float = which_right.isFloat();
|
||||
|
||||
const bool left_is_ipv6 = which_left.isIPv6();
|
||||
const bool right_is_ipv6 = which_right.isIPv6();
|
||||
const bool left_is_fixed_string = which_left.isFixedString();
|
||||
const bool right_is_fixed_string = which_right.isFixedString();
|
||||
size_t fixed_string_size =
|
||||
left_is_fixed_string ?
|
||||
assert_cast<const DataTypeFixedString &>(*left_type).getN() :
|
||||
(right_is_fixed_string ? assert_cast<const DataTypeFixedString &>(*right_type).getN() : 0);
|
||||
|
||||
bool date_and_datetime = (which_left.idx != which_right.idx) && (which_left.isDate() || which_left.isDate32() || which_left.isDateTime() || which_left.isDateTime64())
|
||||
&& (which_right.isDate() || which_right.isDate32() || which_right.isDateTime() || which_right.isDateTime64());
|
||||
|
||||
@ -1287,6 +1298,17 @@ public:
|
||||
{
|
||||
return res;
|
||||
}
|
||||
else if (((left_is_ipv6 && right_is_fixed_string) || (right_is_ipv6 && left_is_fixed_string)) && fixed_string_size == IPV6_BINARY_LENGTH)
|
||||
{
|
||||
/// Special treatment for FixedString(16) as a binary representation of IPv6 -
|
||||
/// CAST is customized for this case
|
||||
ColumnPtr left_column = left_is_ipv6 ?
|
||||
col_with_type_and_name_left.column : castColumn(col_with_type_and_name_left, right_type);
|
||||
ColumnPtr right_column = right_is_ipv6 ?
|
||||
col_with_type_and_name_right.column : castColumn(col_with_type_and_name_right, left_type);
|
||||
|
||||
return executeGenericIdenticalTypes(left_column.get(), right_column.get());
|
||||
}
|
||||
else if ((isColumnedAsDecimal(left_type) || isColumnedAsDecimal(right_type)))
|
||||
{
|
||||
// Comparing Date/Date32 and DateTime64 requires implicit conversion,
|
||||
|
@ -39,6 +39,8 @@ REGISTER_FUNCTION(Conversion)
|
||||
factory.registerFunction<FunctionToDateTime32>();
|
||||
factory.registerFunction<FunctionToDateTime64>();
|
||||
factory.registerFunction<FunctionToUUID>();
|
||||
factory.registerFunction<FunctionToIPv4>();
|
||||
factory.registerFunction<FunctionToIPv6>();
|
||||
factory.registerFunction<FunctionToString>();
|
||||
|
||||
factory.registerFunction<FunctionToUnixTimestamp>();
|
||||
@ -68,6 +70,8 @@ REGISTER_FUNCTION(Conversion)
|
||||
factory.registerFunction<FunctionToDecimal256OrZero>();
|
||||
|
||||
factory.registerFunction<FunctionToUUIDOrZero>();
|
||||
factory.registerFunction<FunctionToIPv4OrZero>();
|
||||
factory.registerFunction<FunctionToIPv6OrZero>();
|
||||
|
||||
factory.registerFunction<FunctionToUInt8OrNull>();
|
||||
factory.registerFunction<FunctionToUInt16OrNull>();
|
||||
@ -94,6 +98,8 @@ REGISTER_FUNCTION(Conversion)
|
||||
factory.registerFunction<FunctionToDecimal256OrNull>();
|
||||
|
||||
factory.registerFunction<FunctionToUUIDOrNull>();
|
||||
factory.registerFunction<FunctionToIPv4OrNull>();
|
||||
factory.registerFunction<FunctionToIPv6OrNull>();
|
||||
|
||||
factory.registerFunction<FunctionParseDateTimeBestEffort>();
|
||||
factory.registerFunction<FunctionParseDateTimeBestEffortOrZero>();
|
||||
|
@ -55,6 +55,7 @@
|
||||
#include <Columns/ColumnLowCardinality.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Common/HashTable/HashMap.h>
|
||||
#include <DataTypes/DataTypeIPv4andIPv6.h>
|
||||
#include <Core/Types.h>
|
||||
|
||||
|
||||
@ -73,6 +74,8 @@ namespace ErrorCodes
|
||||
extern const int CANNOT_PARSE_DATETIME;
|
||||
extern const int CANNOT_PARSE_TEXT;
|
||||
extern const int CANNOT_PARSE_UUID;
|
||||
extern const int CANNOT_PARSE_IPV4;
|
||||
extern const int CANNOT_PARSE_IPV6;
|
||||
extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION;
|
||||
extern const int LOGICAL_ERROR;
|
||||
extern const int TYPE_MISMATCH;
|
||||
@ -120,6 +123,7 @@ struct AccurateOrNullConvertStrategyAdditions
|
||||
|
||||
struct ConvertDefaultBehaviorTag {};
|
||||
struct ConvertReturnNullOnErrorTag {};
|
||||
struct ConvertReturnZeroOnErrorTag {};
|
||||
|
||||
/** Conversion of number types to each other, enums to numbers, dates and datetimes to numbers and back: done by straight assignment.
|
||||
* (Date is represented internally as number of days from some day; DateTime - as unix timestamp)
|
||||
@ -208,6 +212,17 @@ struct ConvertImpl
|
||||
{
|
||||
throw Exception("Conversion between numeric types and UUID is not supported. Probably the passed UUID is unquoted", ErrorCodes::NOT_IMPLEMENTED);
|
||||
}
|
||||
else if constexpr (
|
||||
(std::is_same_v<FromDataType, DataTypeIPv4> != std::is_same_v<ToDataType, DataTypeIPv4>)
|
||||
&& !(is_any_of<FromDataType, DataTypeUInt8, DataTypeUInt16, DataTypeUInt32> || is_any_of<ToDataType, DataTypeUInt32, DataTypeUInt64, DataTypeUInt128, DataTypeUInt256>)
|
||||
)
|
||||
{
|
||||
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Conversion from {} to {} is not supported", TypeName<typename FromDataType::FieldType>, TypeName<typename ToDataType::FieldType>);
|
||||
}
|
||||
else if constexpr (std::is_same_v<FromDataType, DataTypeIPv6> != std::is_same_v<ToDataType, DataTypeIPv6>)
|
||||
{
|
||||
throw Exception("Conversion between numeric types and IPv6 is not supported. Probably the passed IPv6 is unquoted", ErrorCodes::NOT_IMPLEMENTED);
|
||||
}
|
||||
else
|
||||
{
|
||||
if constexpr (IsDataTypeDecimal<FromDataType> || IsDataTypeDecimal<ToDataType>)
|
||||
@ -993,6 +1008,22 @@ inline void parseImpl<DataTypeUUID>(DataTypeUUID::FieldType & x, ReadBuffer & rb
|
||||
x = tmp.toUnderType();
|
||||
}
|
||||
|
||||
template <>
|
||||
inline void parseImpl<DataTypeIPv4>(DataTypeIPv4::FieldType & x, ReadBuffer & rb, const DateLUTImpl *)
|
||||
{
|
||||
IPv4 tmp;
|
||||
readIPv4Text(tmp, rb);
|
||||
x = tmp.toUnderType();
|
||||
}
|
||||
|
||||
template <>
|
||||
inline void parseImpl<DataTypeIPv6>(DataTypeIPv6::FieldType & x, ReadBuffer & rb, const DateLUTImpl *)
|
||||
{
|
||||
IPv6 tmp;
|
||||
readIPv6Text(tmp, rb);
|
||||
x = tmp;
|
||||
}
|
||||
|
||||
template <typename DataType>
|
||||
bool tryParseImpl(typename DataType::FieldType & x, ReadBuffer & rb, const DateLUTImpl *)
|
||||
{
|
||||
@ -1043,6 +1074,28 @@ inline bool tryParseImpl<DataTypeUUID>(DataTypeUUID::FieldType & x, ReadBuffer &
|
||||
return true;
|
||||
}
|
||||
|
||||
template <>
|
||||
inline bool tryParseImpl<DataTypeIPv4>(DataTypeIPv4::FieldType & x, ReadBuffer & rb, const DateLUTImpl *)
|
||||
{
|
||||
IPv4 tmp;
|
||||
if (!tryReadIPv4Text(tmp, rb))
|
||||
return false;
|
||||
|
||||
x = tmp.toUnderType();
|
||||
return true;
|
||||
}
|
||||
|
||||
template <>
|
||||
inline bool tryParseImpl<DataTypeIPv6>(DataTypeIPv6::FieldType & x, ReadBuffer & rb, const DateLUTImpl *)
|
||||
{
|
||||
IPv6 tmp;
|
||||
if (!tryReadIPv6Text(tmp, rb))
|
||||
return false;
|
||||
|
||||
x = tmp;
|
||||
return true;
|
||||
}
|
||||
|
||||
|
||||
/** Throw exception with verbose message when string value is not parsed completely.
|
||||
*/
|
||||
@ -1262,7 +1315,20 @@ struct ConvertThroughParsing
|
||||
}
|
||||
else
|
||||
{
|
||||
/// we want to utilize constexpr condition here, which is not mixable with value comparison
|
||||
do
|
||||
{
|
||||
if constexpr (std::is_same_v<FromDataType, DataTypeFixedString> && std::is_same_v<ToDataType, DataTypeIPv6>)
|
||||
{
|
||||
if (fixed_string_size == IPV6_BINARY_LENGTH)
|
||||
{
|
||||
readBinary(vec_to[i], read_buffer);
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
parseImpl<ToDataType>(vec_to[i], read_buffer, local_time_zone);
|
||||
} while (false);
|
||||
}
|
||||
}
|
||||
|
||||
@ -1317,7 +1383,23 @@ struct ConvertThroughParsing
|
||||
vec_to[i], read_buffer, ToDataType::maxPrecision(), col_to->getScale());
|
||||
}
|
||||
else
|
||||
{
|
||||
/// we want to utilize constexpr condition here, which is not mixable with value comparison
|
||||
do
|
||||
{
|
||||
if constexpr (std::is_same_v<FromDataType, DataTypeFixedString> && std::is_same_v<ToDataType, DataTypeIPv6>)
|
||||
{
|
||||
if (fixed_string_size == IPV6_BINARY_LENGTH)
|
||||
{
|
||||
readBinary(vec_to[i], read_buffer);
|
||||
parsed = true;
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
parsed = tryParseImpl<ToDataType>(vec_to[i], read_buffer, local_time_zone);
|
||||
} while (false);
|
||||
}
|
||||
}
|
||||
|
||||
if (!isAllRead(read_buffer))
|
||||
@ -1370,6 +1452,11 @@ requires (!std::is_same_v<ToDataType, DataTypeFixedString>)
|
||||
struct ConvertImpl<DataTypeFixedString, ToDataType, Name, ConvertReturnNullOnErrorTag>
|
||||
: ConvertThroughParsing<DataTypeFixedString, ToDataType, Name, ConvertFromStringExceptionMode::Null, ConvertFromStringParsingMode::Normal> {};
|
||||
|
||||
template <typename FromDataType, typename ToDataType, typename Name>
|
||||
requires (is_any_of<FromDataType, DataTypeString, DataTypeFixedString> && is_any_of<ToDataType, DataTypeIPv4, DataTypeIPv6>)
|
||||
struct ConvertImpl<FromDataType, ToDataType, Name, ConvertReturnZeroOnErrorTag>
|
||||
: ConvertThroughParsing<FromDataType, ToDataType, Name, ConvertFromStringExceptionMode::Zero, ConvertFromStringParsingMode::Normal> {};
|
||||
|
||||
/// Generic conversion of any type from String. Used for complex types: Array and Tuple or types with custom serialization.
|
||||
template <typename StringColumnType>
|
||||
struct ConvertImplGenericFromString
|
||||
@ -1597,9 +1684,12 @@ public:
|
||||
std::is_same_v<ToDataType, DataTypeDate32> ||
|
||||
std::is_same_v<ToDataType, DataTypeDateTime>;
|
||||
|
||||
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionConvert>(); }
|
||||
static FunctionPtr create(ContextPtr context) { return std::make_shared<FunctionConvert>(context); }
|
||||
static FunctionPtr create() { return std::make_shared<FunctionConvert>(); }
|
||||
|
||||
FunctionConvert() = default;
|
||||
explicit FunctionConvert(ContextPtr context_) : context(context_) {}
|
||||
|
||||
String getName() const override
|
||||
{
|
||||
return name;
|
||||
@ -1763,7 +1853,9 @@ public:
|
||||
|| e.code() == ErrorCodes::CANNOT_PARSE_ESCAPE_SEQUENCE
|
||||
|| e.code() == ErrorCodes::CANNOT_PARSE_DATE
|
||||
|| e.code() == ErrorCodes::CANNOT_PARSE_DATETIME
|
||||
|| e.code() == ErrorCodes::CANNOT_PARSE_UUID)
|
||||
|| e.code() == ErrorCodes::CANNOT_PARSE_UUID
|
||||
|| e.code() == ErrorCodes::CANNOT_PARSE_IPV4
|
||||
|| e.code() == ErrorCodes::CANNOT_PARSE_IPV6)
|
||||
{
|
||||
e.addMessage("Cannot parse "
|
||||
+ result_type->getName() + " from "
|
||||
@ -1785,6 +1877,7 @@ public:
|
||||
}
|
||||
|
||||
private:
|
||||
ContextPtr context;
|
||||
mutable bool checked_return_type = false;
|
||||
mutable bool to_nullable = false;
|
||||
|
||||
@ -1884,12 +1977,19 @@ private:
|
||||
return ConvertImplGenericToString<ColumnString>::execute(arguments, result_type, input_rows_count);
|
||||
}
|
||||
|
||||
bool done;
|
||||
bool done = false;
|
||||
if constexpr (to_string_or_fixed_string)
|
||||
{
|
||||
done = callOnIndexAndDataType<ToDataType>(from_type->getTypeId(), call, ConvertDefaultBehaviorTag{});
|
||||
}
|
||||
else
|
||||
{
|
||||
bool cast_ipv4_ipv6_default_on_conversion_error = false;
|
||||
if constexpr (is_any_of<ToDataType, DataTypeIPv4, DataTypeIPv6>)
|
||||
if (context && (cast_ipv4_ipv6_default_on_conversion_error = context->getSettingsRef().cast_ipv4_ipv6_default_on_conversion_error))
|
||||
done = callOnIndexAndDataType<ToDataType>(from_type->getTypeId(), call, ConvertReturnZeroOnErrorTag{});
|
||||
|
||||
if (!cast_ipv4_ipv6_default_on_conversion_error)
|
||||
{
|
||||
/// We should use ConvertFromStringExceptionMode::Null mode when converting from String (or FixedString)
|
||||
/// to Nullable type, to avoid 'value is too short' error on attempt to parse empty string from NULL values.
|
||||
@ -1898,6 +1998,7 @@ private:
|
||||
else
|
||||
done = callOnIndexAndDataType<ToDataType>(from_type->getTypeId(), call, ConvertDefaultBehaviorTag{});
|
||||
}
|
||||
}
|
||||
|
||||
if (!done)
|
||||
{
|
||||
@ -2357,6 +2458,8 @@ struct NameToInt256 { static constexpr auto name = "toInt256"; };
|
||||
struct NameToFloat32 { static constexpr auto name = "toFloat32"; };
|
||||
struct NameToFloat64 { static constexpr auto name = "toFloat64"; };
|
||||
struct NameToUUID { static constexpr auto name = "toUUID"; };
|
||||
struct NameToIPv4 { static constexpr auto name = "toIPv4"; };
|
||||
struct NameToIPv6 { static constexpr auto name = "toIPv6"; };
|
||||
|
||||
using FunctionToUInt8 = FunctionConvert<DataTypeUInt8, NameToUInt8, ToNumberMonotonicity<UInt8>>;
|
||||
using FunctionToUInt16 = FunctionConvert<DataTypeUInt16, NameToUInt16, ToNumberMonotonicity<UInt16>>;
|
||||
@ -2378,6 +2481,8 @@ using FunctionToDateTime = FunctionConvert<DataTypeDateTime, NameToDateTime, ToD
|
||||
using FunctionToDateTime32 = FunctionConvert<DataTypeDateTime, NameToDateTime32, ToDateTimeMonotonicity>;
|
||||
using FunctionToDateTime64 = FunctionConvert<DataTypeDateTime64, NameToDateTime64, ToDateTimeMonotonicity>;
|
||||
using FunctionToUUID = FunctionConvert<DataTypeUUID, NameToUUID, ToNumberMonotonicity<UInt128>>;
|
||||
using FunctionToIPv4 = FunctionConvert<DataTypeIPv4, NameToIPv4, ToNumberMonotonicity<UInt32>>;
|
||||
using FunctionToIPv6 = FunctionConvert<DataTypeIPv6, NameToIPv6, ToNumberMonotonicity<UInt128>>;
|
||||
using FunctionToString = FunctionConvert<DataTypeString, NameToString, ToStringMonotonicity>;
|
||||
using FunctionToUnixTimestamp = FunctionConvert<DataTypeUInt32, NameToUnixTimestamp, ToNumberMonotonicity<UInt32>>;
|
||||
using FunctionToDecimal32 = FunctionConvert<DataTypeDecimal<Decimal32>, NameToDecimal32, UnknownMonotonicity>;
|
||||
@ -2407,6 +2512,8 @@ template <> struct FunctionTo<DataTypeDate32> { using Type = FunctionToDate32; }
|
||||
template <> struct FunctionTo<DataTypeDateTime> { using Type = FunctionToDateTime; };
|
||||
template <> struct FunctionTo<DataTypeDateTime64> { using Type = FunctionToDateTime64; };
|
||||
template <> struct FunctionTo<DataTypeUUID> { using Type = FunctionToUUID; };
|
||||
template <> struct FunctionTo<DataTypeIPv4> { using Type = FunctionToIPv4; };
|
||||
template <> struct FunctionTo<DataTypeIPv6> { using Type = FunctionToIPv6; };
|
||||
template <> struct FunctionTo<DataTypeString> { using Type = FunctionToString; };
|
||||
template <> struct FunctionTo<DataTypeFixedString> { using Type = FunctionToFixedString; };
|
||||
template <> struct FunctionTo<DataTypeDecimal<Decimal32>> { using Type = FunctionToDecimal32; };
|
||||
@ -2442,6 +2549,8 @@ struct NameToDecimal64OrZero { static constexpr auto name = "toDecimal64OrZero";
|
||||
struct NameToDecimal128OrZero { static constexpr auto name = "toDecimal128OrZero"; };
|
||||
struct NameToDecimal256OrZero { static constexpr auto name = "toDecimal256OrZero"; };
|
||||
struct NameToUUIDOrZero { static constexpr auto name = "toUUIDOrZero"; };
|
||||
struct NameToIPv4OrZero { static constexpr auto name = "toIPv4OrZero"; };
|
||||
struct NameToIPv6OrZero { static constexpr auto name = "toIPv6OrZero"; };
|
||||
|
||||
using FunctionToUInt8OrZero = FunctionConvertFromString<DataTypeUInt8, NameToUInt8OrZero, ConvertFromStringExceptionMode::Zero>;
|
||||
using FunctionToUInt16OrZero = FunctionConvertFromString<DataTypeUInt16, NameToUInt16OrZero, ConvertFromStringExceptionMode::Zero>;
|
||||
@ -2466,6 +2575,8 @@ using FunctionToDecimal64OrZero = FunctionConvertFromString<DataTypeDecimal<Deci
|
||||
using FunctionToDecimal128OrZero = FunctionConvertFromString<DataTypeDecimal<Decimal128>, NameToDecimal128OrZero, ConvertFromStringExceptionMode::Zero>;
|
||||
using FunctionToDecimal256OrZero = FunctionConvertFromString<DataTypeDecimal<Decimal256>, NameToDecimal256OrZero, ConvertFromStringExceptionMode::Zero>;
|
||||
using FunctionToUUIDOrZero = FunctionConvertFromString<DataTypeUUID, NameToUUIDOrZero, ConvertFromStringExceptionMode::Zero>;
|
||||
using FunctionToIPv4OrZero = FunctionConvertFromString<DataTypeIPv4, NameToIPv4OrZero, ConvertFromStringExceptionMode::Zero>;
|
||||
using FunctionToIPv6OrZero = FunctionConvertFromString<DataTypeIPv6, NameToIPv6OrZero, ConvertFromStringExceptionMode::Zero>;
|
||||
|
||||
struct NameToUInt8OrNull { static constexpr auto name = "toUInt8OrNull"; };
|
||||
struct NameToUInt16OrNull { static constexpr auto name = "toUInt16OrNull"; };
|
||||
@ -2490,6 +2601,8 @@ struct NameToDecimal64OrNull { static constexpr auto name = "toDecimal64OrNull";
|
||||
struct NameToDecimal128OrNull { static constexpr auto name = "toDecimal128OrNull"; };
|
||||
struct NameToDecimal256OrNull { static constexpr auto name = "toDecimal256OrNull"; };
|
||||
struct NameToUUIDOrNull { static constexpr auto name = "toUUIDOrNull"; };
|
||||
struct NameToIPv4OrNull { static constexpr auto name = "toIPv4OrNull"; };
|
||||
struct NameToIPv6OrNull { static constexpr auto name = "toIPv6OrNull"; };
|
||||
|
||||
using FunctionToUInt8OrNull = FunctionConvertFromString<DataTypeUInt8, NameToUInt8OrNull, ConvertFromStringExceptionMode::Null>;
|
||||
using FunctionToUInt16OrNull = FunctionConvertFromString<DataTypeUInt16, NameToUInt16OrNull, ConvertFromStringExceptionMode::Null>;
|
||||
@ -2514,6 +2627,8 @@ using FunctionToDecimal64OrNull = FunctionConvertFromString<DataTypeDecimal<Deci
|
||||
using FunctionToDecimal128OrNull = FunctionConvertFromString<DataTypeDecimal<Decimal128>, NameToDecimal128OrNull, ConvertFromStringExceptionMode::Null>;
|
||||
using FunctionToDecimal256OrNull = FunctionConvertFromString<DataTypeDecimal<Decimal256>, NameToDecimal256OrNull, ConvertFromStringExceptionMode::Null>;
|
||||
using FunctionToUUIDOrNull = FunctionConvertFromString<DataTypeUUID, NameToUUIDOrNull, ConvertFromStringExceptionMode::Null>;
|
||||
using FunctionToIPv4OrNull = FunctionConvertFromString<DataTypeIPv4, NameToIPv4OrNull, ConvertFromStringExceptionMode::Null>;
|
||||
using FunctionToIPv6OrNull = FunctionConvertFromString<DataTypeIPv6, NameToIPv6OrNull, ConvertFromStringExceptionMode::Null>;
|
||||
|
||||
struct NameParseDateTimeBestEffort { static constexpr auto name = "parseDateTimeBestEffort"; };
|
||||
struct NameParseDateTimeBestEffortOrZero { static constexpr auto name = "parseDateTimeBestEffortOrZero"; };
|
||||
@ -2642,17 +2757,17 @@ class FunctionCast final : public FunctionCastBase
|
||||
public:
|
||||
using WrapperType = std::function<ColumnPtr(ColumnsWithTypeAndName &, const DataTypePtr &, const ColumnNullable *, size_t)>;
|
||||
|
||||
FunctionCast(const char * cast_name_
|
||||
FunctionCast(ContextPtr context_
|
||||
, const char * cast_name_
|
||||
, MonotonicityForRange && monotonicity_for_range_
|
||||
, const DataTypes & argument_types_
|
||||
, const DataTypePtr & return_type_
|
||||
, std::optional<Diagnostic> diagnostic_
|
||||
, CastType cast_type_
|
||||
, bool cast_ipv4_ipv6_default_on_conversion_error_)
|
||||
, CastType cast_type_)
|
||||
: cast_name(cast_name_), monotonicity_for_range(std::move(monotonicity_for_range_))
|
||||
, argument_types(argument_types_), return_type(return_type_), diagnostic(std::move(diagnostic_))
|
||||
, cast_type(cast_type_)
|
||||
, cast_ipv4_ipv6_default_on_conversion_error(cast_ipv4_ipv6_default_on_conversion_error_)
|
||||
, context(context_)
|
||||
{
|
||||
}
|
||||
|
||||
@ -2699,7 +2814,7 @@ private:
|
||||
|
||||
std::optional<Diagnostic> diagnostic;
|
||||
CastType cast_type;
|
||||
bool cast_ipv4_ipv6_default_on_conversion_error;
|
||||
ContextPtr context;
|
||||
|
||||
static WrapperType createFunctionAdaptor(FunctionPtr function, const DataTypePtr & from_type)
|
||||
{
|
||||
@ -2740,7 +2855,7 @@ private:
|
||||
}
|
||||
else if (!can_apply_accurate_cast)
|
||||
{
|
||||
FunctionPtr function = FunctionTo<ToDataType>::Type::create();
|
||||
FunctionPtr function = FunctionTo<ToDataType>::Type::create(context);
|
||||
return createFunctionAdaptor(function, from_type);
|
||||
}
|
||||
|
||||
@ -3767,7 +3882,9 @@ private:
|
||||
std::is_same_v<ToDataType, DataTypeDate> ||
|
||||
std::is_same_v<ToDataType, DataTypeDate32> ||
|
||||
std::is_same_v<ToDataType, DataTypeDateTime> ||
|
||||
std::is_same_v<ToDataType, DataTypeUUID>)
|
||||
std::is_same_v<ToDataType, DataTypeUUID> ||
|
||||
std::is_same_v<ToDataType, DataTypeIPv4> ||
|
||||
std::is_same_v<ToDataType, DataTypeIPv6>)
|
||||
{
|
||||
ret = createWrapper(from_type, checkAndGetDataType<ToDataType>(to_type.get()), requested_result_is_nullable);
|
||||
return true;
|
||||
@ -3801,9 +3918,11 @@ private:
|
||||
return false;
|
||||
};
|
||||
|
||||
bool cast_ipv4_ipv6_default_on_conversion_error_value = cast_ipv4_ipv6_default_on_conversion_error;
|
||||
bool cast_ipv4_ipv6_default_on_conversion_error_value = context && context->getSettingsRef().cast_ipv4_ipv6_default_on_conversion_error;
|
||||
bool input_format_ipv4_default_on_conversion_error_value = context && context->getSettingsRef().input_format_ipv4_default_on_conversion_error;
|
||||
bool input_format_ipv6_default_on_conversion_error_value = context && context->getSettingsRef().input_format_ipv6_default_on_conversion_error;
|
||||
|
||||
auto make_custom_serialization_wrapper = [&, cast_ipv4_ipv6_default_on_conversion_error_value](const auto & types) -> bool
|
||||
auto make_custom_serialization_wrapper = [&, cast_ipv4_ipv6_default_on_conversion_error_value, input_format_ipv4_default_on_conversion_error_value, input_format_ipv6_default_on_conversion_error_value](const auto & types) -> bool
|
||||
{
|
||||
using Types = std::decay_t<decltype(types)>;
|
||||
using ToDataType = typename Types::RightType;
|
||||
@ -3811,19 +3930,17 @@ private:
|
||||
|
||||
if constexpr (WhichDataType(FromDataType::type_id).isStringOrFixedString())
|
||||
{
|
||||
if (to_type->getCustomSerialization() && to_type->getCustomName())
|
||||
if constexpr (std::is_same_v<ToDataType, DataTypeIPv4>)
|
||||
{
|
||||
if (to_type->getCustomName()->getName() == "IPv4")
|
||||
{
|
||||
ret = [cast_ipv4_ipv6_default_on_conversion_error_value](
|
||||
ret = [cast_ipv4_ipv6_default_on_conversion_error_value, input_format_ipv4_default_on_conversion_error_value, requested_result_is_nullable](
|
||||
ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, const ColumnNullable * column_nullable, size_t)
|
||||
-> ColumnPtr
|
||||
{
|
||||
if (!WhichDataType(result_type).isUInt32())
|
||||
throw Exception(ErrorCodes::TYPE_MISMATCH, "Wrong result type {}. Expected UInt32", result_type->getName());
|
||||
if (!WhichDataType(result_type).isIPv4())
|
||||
throw Exception(ErrorCodes::TYPE_MISMATCH, "Wrong result type {}. Expected IPv4", result_type->getName());
|
||||
|
||||
const auto * null_map = column_nullable ? &column_nullable->getNullMapData() : nullptr;
|
||||
if (cast_ipv4_ipv6_default_on_conversion_error_value)
|
||||
if (cast_ipv4_ipv6_default_on_conversion_error_value || input_format_ipv4_default_on_conversion_error_value || requested_result_is_nullable)
|
||||
return convertToIPv4<IPStringToNumExceptionMode::Default>(arguments[0].column, null_map);
|
||||
else
|
||||
return convertToIPv4<IPStringToNumExceptionMode::Throw>(arguments[0].column, null_map);
|
||||
@ -3832,18 +3949,18 @@ private:
|
||||
return true;
|
||||
}
|
||||
|
||||
if (to_type->getCustomName()->getName() == "IPv6")
|
||||
if constexpr (std::is_same_v<ToDataType, DataTypeIPv6>)
|
||||
{
|
||||
ret = [cast_ipv4_ipv6_default_on_conversion_error_value](
|
||||
ret = [cast_ipv4_ipv6_default_on_conversion_error_value, input_format_ipv6_default_on_conversion_error_value, requested_result_is_nullable](
|
||||
ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, const ColumnNullable * column_nullable, size_t)
|
||||
-> ColumnPtr
|
||||
{
|
||||
if (!WhichDataType(result_type).isFixedString())
|
||||
if (!WhichDataType(result_type).isIPv6())
|
||||
throw Exception(
|
||||
ErrorCodes::TYPE_MISMATCH, "Wrong result type {}. Expected FixedString", result_type->getName());
|
||||
ErrorCodes::TYPE_MISMATCH, "Wrong result type {}. Expected IPv6", result_type->getName());
|
||||
|
||||
const auto * null_map = column_nullable ? &column_nullable->getNullMapData() : nullptr;
|
||||
if (cast_ipv4_ipv6_default_on_conversion_error_value)
|
||||
if (cast_ipv4_ipv6_default_on_conversion_error_value || input_format_ipv6_default_on_conversion_error_value || requested_result_is_nullable)
|
||||
return convertToIPv6<IPStringToNumExceptionMode::Default>(arguments[0].column, null_map);
|
||||
else
|
||||
return convertToIPv6<IPStringToNumExceptionMode::Throw>(arguments[0].column, null_map);
|
||||
@ -3852,6 +3969,8 @@ private:
|
||||
return true;
|
||||
}
|
||||
|
||||
if (to_type->getCustomSerialization() && to_type->getCustomName())
|
||||
{
|
||||
ret = &ConvertImplGenericFromString<typename FromDataType::ColumnType>::execute;
|
||||
return true;
|
||||
}
|
||||
|
@ -7,42 +7,113 @@ namespace DB
|
||||
|
||||
REGISTER_FUNCTION(ExternalDictionaries)
|
||||
{
|
||||
factory.registerFunction<FunctionDictHas>();
|
||||
factory.registerFunction<FunctionDictGetUInt8>();
|
||||
factory.registerFunction<FunctionDictGetUInt16>();
|
||||
factory.registerFunction<FunctionDictGetUInt32>();
|
||||
factory.registerFunction<FunctionDictGetUInt64>();
|
||||
factory.registerFunction<FunctionDictGetInt8>();
|
||||
factory.registerFunction<FunctionDictGetInt16>();
|
||||
factory.registerFunction<FunctionDictGetInt32>();
|
||||
factory.registerFunction<FunctionDictGetInt64>();
|
||||
factory.registerFunction<FunctionDictGetFloat32>();
|
||||
factory.registerFunction<FunctionDictGetFloat64>();
|
||||
factory.registerFunction<FunctionDictGetDate>();
|
||||
factory.registerFunction<FunctionDictGetDateTime>();
|
||||
factory.registerFunction<FunctionDictGetUUID>();
|
||||
factory.registerFunction<FunctionDictGetString>();
|
||||
factory.registerFunction<FunctionDictGetHierarchy>();
|
||||
factory.registerFunction<FunctionDictIsIn>();
|
||||
factory.registerFunction<FunctionDictGetChildrenOverloadResolver>();
|
||||
factory.registerFunction<FunctionDictGetDescendantsOverloadResolver>();
|
||||
factory.registerFunction<FunctionDictGetUInt8OrDefault>();
|
||||
factory.registerFunction<FunctionDictGetUInt16OrDefault>();
|
||||
factory.registerFunction<FunctionDictGetUInt32OrDefault>();
|
||||
factory.registerFunction<FunctionDictGetUInt64OrDefault>();
|
||||
factory.registerFunction<FunctionDictGetInt8OrDefault>();
|
||||
factory.registerFunction<FunctionDictGetInt16OrDefault>();
|
||||
factory.registerFunction<FunctionDictGetInt32OrDefault>();
|
||||
factory.registerFunction<FunctionDictGetInt64OrDefault>();
|
||||
factory.registerFunction<FunctionDictGetFloat32OrDefault>();
|
||||
factory.registerFunction<FunctionDictGetFloat64OrDefault>();
|
||||
factory.registerFunction<FunctionDictGetDateOrDefault>();
|
||||
factory.registerFunction<FunctionDictGetDateTimeOrDefault>();
|
||||
factory.registerFunction<FunctionDictGetUUIDOrDefault>();
|
||||
factory.registerFunction<FunctionDictGetStringOrDefault>();
|
||||
factory.registerFunction<FunctionDictGetNoType<DictionaryGetFunctionType::get>>();
|
||||
factory.registerFunction<FunctionDictGetNoType<DictionaryGetFunctionType::getOrDefault>>();
|
||||
factory.registerFunction<FunctionDictGetOrNull>();
|
||||
const std::string dict_get_description { R"(
|
||||
Retrieves values from a dictionary.
|
||||
|
||||
Accepts 3 parameters:
|
||||
-- name of the dictionary;
|
||||
-- name of the column of the dictionary or tuple of column names;
|
||||
-- key value - expression returning dictionary key-type value or tuple-type value - depending on the dictionary configuration;
|
||||
|
||||
Returned value: value of the dictionary attribute parsed in the {} if key is found, otherwise <null_value> element specified in the dictionary configuration.
|
||||
|
||||
Throws an exception if cannot parse the value of the attribute or the value does not match the attribute data type.
|
||||
)" };
|
||||
|
||||
const std::string dict_get_or_default_description { R"(
|
||||
Retrieves values from a dictionary.
|
||||
|
||||
Accepts 4 parameters:
|
||||
-- name of the dictionary;
|
||||
-- name of the column of the dictionary or tuple of column names;
|
||||
-- key value - expression returning dictionary key-type value or tuple-type value - depending on the dictionary configuration;
|
||||
-- default values returned if the dictionary does not contain a row with the key value;
|
||||
|
||||
Returned value: value of the dictionary attribute parsed in the {} if key is found, otherwise default value.
|
||||
|
||||
Throws an exception if cannot parse the value of the attribute or the value does not match the attribute data type.
|
||||
)" };
|
||||
|
||||
const std::string dict_get_or_null_description { R"(
|
||||
Retrieves values from a dictionary.
|
||||
|
||||
Accepts 3 parameters:
|
||||
-- name of the dictionary;
|
||||
-- name of the column of the dictionary or tuple of column names;
|
||||
-- key value - expression returning dictionary key-type value or tuple-type value - depending on the dictionary configuration;
|
||||
|
||||
Returned value: value of the dictionary attribute parsed in the attribute’s data type if key is found, otherwise NULL.
|
||||
|
||||
Throws an exception if cannot parse the value of the attribute or the value does not match the attribute data type.
|
||||
)" };
|
||||
|
||||
factory.registerFunction<FunctionDictGetNoType<DictionaryGetFunctionType::get>>(Documentation{ fmt::format(fmt::runtime(dict_get_description), "attribute’s data type") });
|
||||
factory.registerFunction<FunctionDictGetNoType<DictionaryGetFunctionType::getOrDefault>>(Documentation{ fmt::format(fmt::runtime(dict_get_or_default_description), "attribute’s data type") });
|
||||
factory.registerFunction<FunctionDictGetOrNull>(Documentation{ dict_get_or_null_description });
|
||||
|
||||
factory.registerFunction<FunctionDictGetUInt8>(Documentation{ fmt::format(fmt::runtime(dict_get_description), "UInt8") });
|
||||
factory.registerFunction<FunctionDictGetUInt16>(Documentation{ fmt::format(fmt::runtime(dict_get_description), "UInt16") });
|
||||
factory.registerFunction<FunctionDictGetUInt32>(Documentation{ fmt::format(fmt::runtime(dict_get_description), "UInt32") });
|
||||
factory.registerFunction<FunctionDictGetUInt64>(Documentation{ fmt::format(fmt::runtime(dict_get_description), "UInt64") });
|
||||
factory.registerFunction<FunctionDictGetInt8>(Documentation{ fmt::format(fmt::runtime(dict_get_description), "Int8") });
|
||||
factory.registerFunction<FunctionDictGetInt16>(Documentation{ fmt::format(fmt::runtime(dict_get_description), "Int16") });
|
||||
factory.registerFunction<FunctionDictGetInt32>(Documentation{ fmt::format(fmt::runtime(dict_get_description), "Int32") });
|
||||
factory.registerFunction<FunctionDictGetInt64>(Documentation{ fmt::format(fmt::runtime(dict_get_description), "Int64") });
|
||||
factory.registerFunction<FunctionDictGetFloat32>(Documentation{ fmt::format(fmt::runtime(dict_get_description), "Float32") });
|
||||
factory.registerFunction<FunctionDictGetFloat64>(Documentation{ fmt::format(fmt::runtime(dict_get_description), "Float64") });
|
||||
factory.registerFunction<FunctionDictGetDate>(Documentation{ fmt::format(fmt::runtime(dict_get_description), "Date") });
|
||||
factory.registerFunction<FunctionDictGetDateTime>(Documentation{ fmt::format(fmt::runtime(dict_get_description), "DateTime") });
|
||||
factory.registerFunction<FunctionDictGetUUID>(Documentation{ fmt::format(fmt::runtime(dict_get_description), "UUID") });
|
||||
factory.registerFunction<FunctionDictGetIPv4>(Documentation{ fmt::format(fmt::runtime(dict_get_description), "IPv4") });
|
||||
factory.registerFunction<FunctionDictGetIPv6>(Documentation{ fmt::format(fmt::runtime(dict_get_description), "IPv6") });
|
||||
factory.registerFunction<FunctionDictGetString>(Documentation{ fmt::format(fmt::runtime(dict_get_description), "String") });
|
||||
|
||||
factory.registerFunction<FunctionDictGetUInt8OrDefault>(Documentation{ fmt::format(fmt::runtime(dict_get_or_default_description), "UInt8") });
|
||||
factory.registerFunction<FunctionDictGetUInt16OrDefault>(Documentation{ fmt::format(fmt::runtime(dict_get_or_default_description), "UInt16") });
|
||||
factory.registerFunction<FunctionDictGetUInt32OrDefault>(Documentation{ fmt::format(fmt::runtime(dict_get_or_default_description), "UInt32") });
|
||||
factory.registerFunction<FunctionDictGetUInt64OrDefault>(Documentation{ fmt::format(fmt::runtime(dict_get_or_default_description), "UInt64") });
|
||||
factory.registerFunction<FunctionDictGetInt8OrDefault>(Documentation{ fmt::format(fmt::runtime(dict_get_or_default_description), "Int8") });
|
||||
factory.registerFunction<FunctionDictGetInt16OrDefault>(Documentation{ fmt::format(fmt::runtime(dict_get_or_default_description), "Int16") });
|
||||
factory.registerFunction<FunctionDictGetInt32OrDefault>(Documentation{ fmt::format(fmt::runtime(dict_get_or_default_description), "Int32") });
|
||||
factory.registerFunction<FunctionDictGetInt64OrDefault>(Documentation{ fmt::format(fmt::runtime(dict_get_or_default_description), "Int64") });
|
||||
factory.registerFunction<FunctionDictGetFloat32OrDefault>(Documentation{ fmt::format(fmt::runtime(dict_get_or_default_description), "Float32") });
|
||||
factory.registerFunction<FunctionDictGetFloat64OrDefault>(Documentation{ fmt::format(fmt::runtime(dict_get_or_default_description), "Float64") });
|
||||
factory.registerFunction<FunctionDictGetDateOrDefault>(Documentation{ fmt::format(fmt::runtime(dict_get_or_default_description), "Date") });
|
||||
factory.registerFunction<FunctionDictGetDateTimeOrDefault>(Documentation{ fmt::format(fmt::runtime(dict_get_or_default_description), "DateTime") });
|
||||
factory.registerFunction<FunctionDictGetUUIDOrDefault>(Documentation{ fmt::format(fmt::runtime(dict_get_or_default_description), "UUID") });
|
||||
factory.registerFunction<FunctionDictGetIPv4OrDefault>(Documentation{ fmt::format(fmt::runtime(dict_get_or_default_description), "IPv4") });
|
||||
factory.registerFunction<FunctionDictGetIPv6OrDefault>(Documentation{ fmt::format(fmt::runtime(dict_get_or_default_description), "IPv6") });
|
||||
factory.registerFunction<FunctionDictGetStringOrDefault>(Documentation{ fmt::format(fmt::runtime(dict_get_or_default_description), "String") });
|
||||
|
||||
factory.registerFunction<FunctionDictHas>(Documentation{ R"(
|
||||
Checks whether a key is present in a dictionary.
|
||||
Accepts 2 parameters: name of the dictionary, key value - expression returning dictionary key-type value or tuple-type value - depending on the dictionary configuration.
|
||||
Returned value: 0 if there is no key, 1 if there is a key, type of UInt8
|
||||
)"});
|
||||
|
||||
factory.registerFunction<FunctionDictGetHierarchy>(Documentation{ R"(
|
||||
Creates an array, containing all the parents of a key in the hierarchical dictionary.
|
||||
Accepts 2 parameters: name of the dictionary, key value - expression returning a UInt64-type value.
|
||||
Returned value: parents for the key, type of Array(UInt64)
|
||||
)"});
|
||||
|
||||
factory.registerFunction<FunctionDictIsIn>(Documentation{ R"(
|
||||
Checks the ancestor of a key through the whole hierarchical chain in the dictionary.
|
||||
Accepts 3 parameters: name of the dictionary, key to be checked - expression returning a UInt64-type value, alleged ancestor of the key - expression returning a UInt64-type.
|
||||
Returned value: 0 if key is not a child of the ancestor, 1 if key is a child of the ancestor or if key is the ancestor, type of UInt8
|
||||
)"});
|
||||
|
||||
factory.registerFunction<FunctionDictGetChildrenOverloadResolver>(Documentation{ R"(
|
||||
Returns first-level children as an array of indexes. It is the inverse transformation for dictGetHierarchy.
|
||||
Accepts 2 parameters: name of the dictionary, key value - expression returning a UInt64-type value.
|
||||
Returned value: first-level descendants for the key, type of Array(UInt64)
|
||||
)"});
|
||||
|
||||
factory.registerFunction<FunctionDictGetDescendantsOverloadResolver>(Documentation{ R"(
|
||||
Returns all descendants as if dictGetChildren function was applied level times recursively.
|
||||
Accepts 3 parameters: name of the dictionary, key value - expression returning a UInt64-type value, level — hierarchy level - If level = 0 returns all descendants to the end - UInt8
|
||||
Returned value: descendants for the key, type of Array(UInt64)
|
||||
)"});
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -8,6 +8,7 @@
|
||||
#include <DataTypes/DataTypeDateTime.h>
|
||||
#include <DataTypes/DataTypeTuple.h>
|
||||
#include <DataTypes/DataTypeUUID.h>
|
||||
#include <DataTypes/DataTypeIPv4andIPv6.h>
|
||||
#include <DataTypes/DataTypeNullable.h>
|
||||
#include <DataTypes/DataTypeLowCardinality.h>
|
||||
|
||||
@ -716,6 +717,8 @@ struct NameDictGetFloat64 { static constexpr auto name = "dictGetFloat64"; };
|
||||
struct NameDictGetDate { static constexpr auto name = "dictGetDate"; };
|
||||
struct NameDictGetDateTime { static constexpr auto name = "dictGetDateTime"; };
|
||||
struct NameDictGetUUID { static constexpr auto name = "dictGetUUID"; };
|
||||
struct NameDictGetIPv4 { static constexpr auto name = "dictGetIPv4"; };
|
||||
struct NameDictGetIPv6 { static constexpr auto name = "dictGetIPv6"; };
|
||||
struct NameDictGetDecimal32 { static constexpr auto name = "dictGetDecimal32"; };
|
||||
struct NameDictGetDecimal64 { static constexpr auto name = "dictGetDecimal64"; };
|
||||
struct NameDictGetDecimal128 { static constexpr auto name = "dictGetDecimal128"; };
|
||||
@ -734,6 +737,8 @@ using FunctionDictGetFloat64 = FunctionDictGet<DataTypeFloat64, NameDictGetFloat
|
||||
using FunctionDictGetDate = FunctionDictGet<DataTypeDate, NameDictGetDate>;
|
||||
using FunctionDictGetDateTime = FunctionDictGet<DataTypeDateTime, NameDictGetDateTime>;
|
||||
using FunctionDictGetUUID = FunctionDictGet<DataTypeUUID, NameDictGetUUID>;
|
||||
using FunctionDictGetIPv4 = FunctionDictGet<DataTypeIPv4, NameDictGetIPv4>;
|
||||
using FunctionDictGetIPv6 = FunctionDictGet<DataTypeIPv6, NameDictGetIPv6>;
|
||||
using FunctionDictGetDecimal32 = FunctionDictGet<DataTypeDecimal<Decimal32>, NameDictGetDecimal32>;
|
||||
using FunctionDictGetDecimal64 = FunctionDictGet<DataTypeDecimal<Decimal64>, NameDictGetDecimal64>;
|
||||
using FunctionDictGetDecimal128 = FunctionDictGet<DataTypeDecimal<Decimal128>, NameDictGetDecimal128>;
|
||||
@ -755,6 +760,8 @@ struct NameDictGetFloat64OrDefault { static constexpr auto name = "dictGetFloat6
|
||||
struct NameDictGetDateOrDefault { static constexpr auto name = "dictGetDateOrDefault"; };
|
||||
struct NameDictGetDateTimeOrDefault { static constexpr auto name = "dictGetDateTimeOrDefault"; };
|
||||
struct NameDictGetUUIDOrDefault { static constexpr auto name = "dictGetUUIDOrDefault"; };
|
||||
struct NameDictGetIPv4OrDefault { static constexpr auto name = "dictGetIPv4OrDefault"; };
|
||||
struct NameDictGetIPv6OrDefault { static constexpr auto name = "dictGetIPv6OrDefault"; };
|
||||
struct NameDictGetDecimal32OrDefault { static constexpr auto name = "dictGetDecimal32OrDefault"; };
|
||||
struct NameDictGetDecimal64OrDefault { static constexpr auto name = "dictGetDecimal64OrDefault"; };
|
||||
struct NameDictGetDecimal128OrDefault { static constexpr auto name = "dictGetDecimal128OrDefault"; };
|
||||
@ -773,6 +780,8 @@ using FunctionDictGetFloat64OrDefault = FunctionDictGetOrDefault<DataTypeFloat64
|
||||
using FunctionDictGetDateOrDefault = FunctionDictGetOrDefault<DataTypeDate, NameDictGetDateOrDefault>;
|
||||
using FunctionDictGetDateTimeOrDefault = FunctionDictGetOrDefault<DataTypeDateTime, NameDictGetDateTimeOrDefault>;
|
||||
using FunctionDictGetUUIDOrDefault = FunctionDictGetOrDefault<DataTypeUUID, NameDictGetUUIDOrDefault>;
|
||||
using FunctionDictGetIPv4OrDefault = FunctionDictGetOrDefault<DataTypeIPv4, NameDictGetIPv4OrDefault>;
|
||||
using FunctionDictGetIPv6OrDefault = FunctionDictGetOrDefault<DataTypeIPv6, NameDictGetIPv6OrDefault>;
|
||||
using FunctionDictGetDecimal32OrDefault = FunctionDictGetOrDefault<DataTypeDecimal<Decimal32>, NameDictGetDecimal32OrDefault>;
|
||||
using FunctionDictGetDecimal64OrDefault = FunctionDictGetOrDefault<DataTypeDecimal<Decimal64>, NameDictGetDecimal64OrDefault>;
|
||||
using FunctionDictGetDecimal128OrDefault = FunctionDictGetOrDefault<DataTypeDecimal<Decimal128>, NameDictGetDecimal128OrDefault>;
|
||||
|
@ -1136,6 +1136,8 @@ private:
|
||||
else if (which.isInt128()) executeBigIntType<Int128, first>(icolumn, vec_to);
|
||||
else if (which.isInt256()) executeBigIntType<Int256, first>(icolumn, vec_to);
|
||||
else if (which.isUUID()) executeBigIntType<UUID, first>(icolumn, vec_to);
|
||||
else if (which.isIPv4()) executeBigIntType<IPv4, first>(icolumn, vec_to);
|
||||
else if (which.isIPv6()) executeBigIntType<IPv6, first>(icolumn, vec_to);
|
||||
else if (which.isEnum8()) executeIntType<Int8, first>(icolumn, vec_to);
|
||||
else if (which.isEnum16()) executeIntType<Int16, first>(icolumn, vec_to);
|
||||
else if (which.isDate()) executeIntType<UInt16, first>(icolumn, vec_to);
|
||||
|
@ -927,7 +927,9 @@ static bool castColumnNumeric(const IColumn * column, F && f)
|
||||
ColumnVector<Int64>,
|
||||
ColumnVector<Int128>,
|
||||
ColumnVector<Int256>,
|
||||
ColumnVector<UUID>
|
||||
ColumnVector<UUID>,
|
||||
ColumnVector<IPv4>,
|
||||
ColumnVector<IPv6>
|
||||
>(column, std::forward<F>(f));
|
||||
}
|
||||
|
||||
|
@ -8,6 +8,7 @@
|
||||
#include <DataTypes/DataTypeDateTime.h>
|
||||
#include <DataTypes/DataTypeDateTime64.h>
|
||||
#include <DataTypes/DataTypeUUID.h>
|
||||
#include <DataTypes/DataTypeIPv4andIPv6.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Columns/ColumnConst.h>
|
||||
#include <Columns/ColumnNullable.h>
|
||||
@ -334,6 +335,8 @@ struct NameToDecimal64OrDefault { static constexpr auto name = "toDecimal64OrDef
|
||||
struct NameToDecimal128OrDefault { static constexpr auto name = "toDecimal128OrDefault"; };
|
||||
struct NameToDecimal256OrDefault { static constexpr auto name = "toDecimal256OrDefault"; };
|
||||
struct NameToUUIDOrDefault { static constexpr auto name = "toUUIDOrDefault"; };
|
||||
struct NameToIPv4OrDefault { static constexpr auto name = "toIPv4OrDefault"; };
|
||||
struct NameToIPv6OrDefault { static constexpr auto name = "toIPv6OrDefault"; };
|
||||
|
||||
using FunctionToUInt8OrDefault = FunctionCastOrDefaultTyped<DataTypeUInt8, NameToUInt8OrDefault>;
|
||||
using FunctionToUInt16OrDefault = FunctionCastOrDefaultTyped<DataTypeUInt16, NameToUInt16OrDefault>;
|
||||
@ -362,6 +365,8 @@ using FunctionToDecimal128OrDefault = FunctionCastOrDefaultTyped<DataTypeDecimal
|
||||
using FunctionToDecimal256OrDefault = FunctionCastOrDefaultTyped<DataTypeDecimal<Decimal256>, NameToDecimal256OrDefault>;
|
||||
|
||||
using FunctionToUUIDOrDefault = FunctionCastOrDefaultTyped<DataTypeUUID, NameToUUIDOrDefault>;
|
||||
using FunctionToIPv4OrDefault = FunctionCastOrDefaultTyped<DataTypeIPv4, NameToIPv4OrDefault>;
|
||||
using FunctionToIPv6OrDefault = FunctionCastOrDefaultTyped<DataTypeIPv6, NameToIPv6OrDefault>;
|
||||
|
||||
REGISTER_FUNCTION(CastOrDefault)
|
||||
{
|
||||
@ -394,6 +399,8 @@ REGISTER_FUNCTION(CastOrDefault)
|
||||
factory.registerFunction<FunctionToDecimal256OrDefault>();
|
||||
|
||||
factory.registerFunction<FunctionToUUIDOrDefault>();
|
||||
factory.registerFunction<FunctionToIPv4OrDefault>();
|
||||
factory.registerFunction<FunctionToIPv6OrDefault>();
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -29,22 +29,17 @@ public:
|
||||
|
||||
explicit IPAddressVariant(std::string_view address_str)
|
||||
{
|
||||
/// IP address parser functions require that the input is
|
||||
/// NULL-terminated so we need to copy it.
|
||||
const auto address_str_copy = std::string(address_str);
|
||||
|
||||
UInt32 v4;
|
||||
if (DB::parseIPv4(address_str_copy.c_str(), reinterpret_cast<unsigned char *>(&v4)))
|
||||
if (DB::parseIPv4whole(address_str.begin(), address_str.end(), reinterpret_cast<unsigned char *>(&v4)))
|
||||
{
|
||||
addr = v4;
|
||||
}
|
||||
else
|
||||
{
|
||||
addr = IPv6AddrType();
|
||||
bool success = DB::parseIPv6(address_str_copy.c_str(), std::get<IPv6AddrType>(addr).data());
|
||||
bool success = DB::parseIPv6whole(address_str.begin(), address_str.end(), std::get<IPv6AddrType>(addr).data());
|
||||
if (!success)
|
||||
throw DB::Exception("Neither IPv4 nor IPv6 address: '" + address_str_copy + "'",
|
||||
DB::ErrorCodes::CANNOT_PARSE_TEXT);
|
||||
throw DB::Exception(DB::ErrorCodes::CANNOT_PARSE_TEXT, "Neither IPv4 nor IPv6 address: '{}'", address_str);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -11,6 +11,7 @@
|
||||
|
||||
#include <type_traits>
|
||||
|
||||
#include <Common/formatIPv6.h>
|
||||
#include <Common/DateLUT.h>
|
||||
#include <Common/LocalDate.h>
|
||||
#include <Common/LocalDateTime.h>
|
||||
@ -21,6 +22,7 @@
|
||||
#include <Core/Types.h>
|
||||
#include <Core/DecimalFunctions.h>
|
||||
#include <Core/UUID.h>
|
||||
#include <base/IPv4andIPv6.h>
|
||||
|
||||
#include <Common/Allocator.h>
|
||||
#include <Common/Exception.h>
|
||||
@ -54,6 +56,8 @@ namespace ErrorCodes
|
||||
extern const int CANNOT_PARSE_BOOL;
|
||||
extern const int CANNOT_PARSE_DATETIME;
|
||||
extern const int CANNOT_PARSE_UUID;
|
||||
extern const int CANNOT_PARSE_IPV4;
|
||||
extern const int CANNOT_PARSE_IPV6;
|
||||
extern const int CANNOT_READ_ARRAY_FROM_TEXT;
|
||||
extern const int CANNOT_PARSE_NUMBER;
|
||||
extern const int INCORRECT_DATA;
|
||||
@ -844,6 +848,49 @@ inline bool tryReadUUIDText(UUID & uuid, ReadBuffer & buf)
|
||||
return readUUIDTextImpl<bool>(uuid, buf);
|
||||
}
|
||||
|
||||
template <typename ReturnType = void>
|
||||
inline ReturnType readIPv4TextImpl(IPv4 & ip, ReadBuffer & buf)
|
||||
{
|
||||
if (parseIPv4(buf.position(), [&buf](){ return buf.eof(); }, reinterpret_cast<unsigned char *>(&ip.toUnderType())))
|
||||
return ReturnType(true);
|
||||
|
||||
if constexpr (std::is_same_v<ReturnType, void>)
|
||||
throw ParsingException(std::string("Cannot parse IPv4 ").append(buf.position(), buf.available()), ErrorCodes::CANNOT_PARSE_IPV4);
|
||||
else
|
||||
return ReturnType(false);
|
||||
}
|
||||
|
||||
inline void readIPv4Text(IPv4 & ip, ReadBuffer & buf)
|
||||
{
|
||||
return readIPv4TextImpl<void>(ip, buf);
|
||||
}
|
||||
|
||||
inline bool tryReadIPv4Text(IPv4 & ip, ReadBuffer & buf)
|
||||
{
|
||||
return readIPv4TextImpl<bool>(ip, buf);
|
||||
}
|
||||
|
||||
template <typename ReturnType = void>
|
||||
inline ReturnType readIPv6TextImpl(IPv6 & ip, ReadBuffer & buf)
|
||||
{
|
||||
if (parseIPv6orIPv4(buf.position(), [&buf](){ return buf.eof(); }, reinterpret_cast<unsigned char *>(ip.toUnderType().items)))
|
||||
return ReturnType(true);
|
||||
|
||||
if constexpr (std::is_same_v<ReturnType, void>)
|
||||
throw ParsingException(std::string("Cannot parse IPv6 ").append(buf.position(), buf.available()), ErrorCodes::CANNOT_PARSE_IPV6);
|
||||
else
|
||||
return ReturnType(false);
|
||||
}
|
||||
|
||||
inline void readIPv6Text(IPv6 & ip, ReadBuffer & buf)
|
||||
{
|
||||
return readIPv6TextImpl<void>(ip, buf);
|
||||
}
|
||||
|
||||
inline bool tryReadIPv6Text(IPv6 & ip, ReadBuffer & buf)
|
||||
{
|
||||
return readIPv6TextImpl<bool>(ip, buf);
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
inline T parse(const char * data, size_t size);
|
||||
@ -1054,8 +1101,10 @@ inline void readBinary(bool & x, ReadBuffer & buf)
|
||||
}
|
||||
|
||||
inline void readBinary(String & x, ReadBuffer & buf) { readStringBinary(x, buf); }
|
||||
inline void readBinary(Int32 & x, ReadBuffer & buf) { readPODBinary(x, buf); }
|
||||
inline void readBinary(Int128 & x, ReadBuffer & buf) { readPODBinary(x, buf); }
|
||||
inline void readBinary(Int256 & x, ReadBuffer & buf) { readPODBinary(x, buf); }
|
||||
inline void readBinary(UInt32 & x, ReadBuffer & buf) { readPODBinary(x, buf); }
|
||||
inline void readBinary(UInt128 & x, ReadBuffer & buf) { readPODBinary(x, buf); }
|
||||
inline void readBinary(UInt256 & x, ReadBuffer & buf) { readPODBinary(x, buf); }
|
||||
inline void readBinary(Decimal32 & x, ReadBuffer & buf) { readPODBinary(x, buf); }
|
||||
@ -1111,6 +1160,8 @@ inline bool tryReadText(is_integer auto & x, ReadBuffer & buf)
|
||||
}
|
||||
|
||||
inline bool tryReadText(UUID & x, ReadBuffer & buf) { return tryReadUUIDText(x, buf); }
|
||||
inline bool tryReadText(IPv4 & x, ReadBuffer & buf) { return tryReadIPv4Text(x, buf); }
|
||||
inline bool tryReadText(IPv6 & x, ReadBuffer & buf) { return tryReadIPv6Text(x, buf); }
|
||||
|
||||
inline void readText(is_floating_point auto & x, ReadBuffer & buf) { readFloatText(x, buf); }
|
||||
|
||||
@ -1119,6 +1170,8 @@ inline void readText(LocalDate & x, ReadBuffer & buf) { readDateText(x, buf); }
|
||||
inline void readText(DayNum & x, ReadBuffer & buf) { readDateText(x, buf); }
|
||||
inline void readText(LocalDateTime & x, ReadBuffer & buf) { readDateTimeText(x, buf); }
|
||||
inline void readText(UUID & x, ReadBuffer & buf) { readUUIDText(x, buf); }
|
||||
inline void readText(IPv4 & x, ReadBuffer & buf) { readIPv4Text(x, buf); }
|
||||
inline void readText(IPv6 & x, ReadBuffer & buf) { readIPv6Text(x, buf); }
|
||||
|
||||
/// Generic methods to read value in text format,
|
||||
/// possibly in single quotes (only for data types that use quotes in VALUES format of INSERT statement in SQL).
|
||||
@ -1149,6 +1202,19 @@ inline void readQuoted(UUID & x, ReadBuffer & buf)
|
||||
assertChar('\'', buf);
|
||||
}
|
||||
|
||||
inline void readQuoted(IPv4 & x, ReadBuffer & buf)
|
||||
{
|
||||
assertChar('\'', buf);
|
||||
readIPv4Text(x, buf);
|
||||
assertChar('\'', buf);
|
||||
}
|
||||
|
||||
inline void readQuoted(IPv6 & x, ReadBuffer & buf)
|
||||
{
|
||||
assertChar('\'', buf);
|
||||
readIPv6Text(x, buf);
|
||||
assertChar('\'', buf);
|
||||
}
|
||||
|
||||
/// Same as above, but in double quotes.
|
||||
template <typename T>
|
||||
@ -1201,6 +1267,8 @@ inline void readCSV(LocalDate & x, ReadBuffer & buf) { readCSVSimple(x, buf); }
|
||||
inline void readCSV(DayNum & x, ReadBuffer & buf) { readCSVSimple(x, buf); }
|
||||
inline void readCSV(LocalDateTime & x, ReadBuffer & buf) { readCSVSimple(x, buf); }
|
||||
inline void readCSV(UUID & x, ReadBuffer & buf) { readCSVSimple(x, buf); }
|
||||
inline void readCSV(IPv4 & x, ReadBuffer & buf) { readCSVSimple(x, buf); }
|
||||
inline void readCSV(IPv6 & x, ReadBuffer & buf) { readCSVSimple(x, buf); }
|
||||
inline void readCSV(UInt128 & x, ReadBuffer & buf) { readCSVSimple(x, buf); }
|
||||
inline void readCSV(Int128 & x, ReadBuffer & buf) { readCSVSimple(x, buf); }
|
||||
inline void readCSV(UInt256 & x, ReadBuffer & buf) { readCSVSimple(x, buf); }
|
||||
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue
Block a user