mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-12-13 18:02:24 +00:00
Merged with master
This commit is contained in:
commit
d4b0532b78
@ -56,17 +56,19 @@ function watchdog
|
||||
sleep 3600
|
||||
|
||||
echo "Fuzzing run has timed out"
|
||||
killall clickhouse-client ||:
|
||||
for _ in {1..10}
|
||||
do
|
||||
if ! pgrep -f clickhouse-client
|
||||
# Only kill by pid the particular client that runs the fuzzing, or else
|
||||
# we can kill some clickhouse-client processes this script starts later,
|
||||
# e.g. for checking server liveness.
|
||||
if ! kill $fuzzer_pid
|
||||
then
|
||||
break
|
||||
fi
|
||||
sleep 1
|
||||
done
|
||||
|
||||
killall -9 clickhouse-client ||:
|
||||
kill -9 -- $fuzzer_pid ||:
|
||||
}
|
||||
|
||||
function filter_exists
|
||||
@ -85,7 +87,7 @@ function fuzz
|
||||
{
|
||||
# Obtain the list of newly added tests. They will be fuzzed in more extreme way than other tests.
|
||||
# Don't overwrite the NEW_TESTS_OPT so that it can be set from the environment.
|
||||
NEW_TESTS="$(grep -P 'tests/queries/0_stateless/.*\.sql' ci-changed-files.txt | sed -r -e 's!^!ch/!' | sort -R)"
|
||||
NEW_TESTS="$(sed -n 's!\(^tests/queries/0_stateless/.*\.sql\)$!ch/\1!p' ci-changed-files.txt | sort -R)"
|
||||
# ci-changed-files.txt contains also files that has been deleted/renamed, filter them out.
|
||||
NEW_TESTS="$(filter_exists $NEW_TESTS)"
|
||||
if [[ -n "$NEW_TESTS" ]]
|
||||
@ -115,17 +117,49 @@ continue
|
||||
|
||||
gdb -batch -command script.gdb -p "$(pidof clickhouse-server)" &
|
||||
|
||||
fuzzer_exit_code=0
|
||||
# SC2012: Use find instead of ls to better handle non-alphanumeric filenames. They are all alphanumeric.
|
||||
# SC2046: Quote this to prevent word splitting. Actually I need word splitting.
|
||||
# shellcheck disable=SC2012,SC2046
|
||||
clickhouse-client --query-fuzzer-runs=1000 --queries-file $(ls -1 ch/tests/queries/0_stateless/*.sql | sort -R) $NEW_TESTS_OPT \
|
||||
> >(tail -n 100000 > fuzzer.log) \
|
||||
2>&1 \
|
||||
|| fuzzer_exit_code=$?
|
||||
2>&1 &
|
||||
fuzzer_pid=$!
|
||||
echo "Fuzzer pid is $fuzzer_pid"
|
||||
|
||||
# Start a watchdog that should kill the fuzzer on timeout.
|
||||
# The shell won't kill the child sleep when we kill it, so we have to put it
|
||||
# into a separate process group so that we can kill them all.
|
||||
set -m
|
||||
watchdog &
|
||||
watchdog_pid=$!
|
||||
set +m
|
||||
# Check that the watchdog has started.
|
||||
kill -0 $watchdog_pid
|
||||
|
||||
# Wait for the fuzzer to complete.
|
||||
# Note that the 'wait || ...' thing is required so that the script doesn't
|
||||
# exit because of 'set -e' when 'wait' returns nonzero code.
|
||||
fuzzer_exit_code=0
|
||||
wait "$fuzzer_pid" || fuzzer_exit_code=$?
|
||||
echo "Fuzzer exit code is $fuzzer_exit_code"
|
||||
|
||||
kill -- -$watchdog_pid ||:
|
||||
|
||||
# If the server dies, most often the fuzzer returns code 210: connetion
|
||||
# refused, and sometimes also code 32: attempt to read after eof. For
|
||||
# simplicity, check again whether the server is accepting connections, using
|
||||
# clickhouse-client. We don't check for existence of server process, because
|
||||
# the process is still present while the server is terminating and not
|
||||
# accepting the connections anymore.
|
||||
if clickhouse-client --query "select 1 format Null"
|
||||
then
|
||||
server_died=0
|
||||
else
|
||||
echo "Server live check returns $?"
|
||||
server_died=1
|
||||
fi
|
||||
|
||||
# Stop the server.
|
||||
clickhouse-client --query "select elapsed, query from system.processes" ||:
|
||||
killall clickhouse-server ||:
|
||||
for _ in {1..10}
|
||||
@ -137,6 +171,41 @@ continue
|
||||
sleep 1
|
||||
done
|
||||
killall -9 clickhouse-server ||:
|
||||
|
||||
# Debug.
|
||||
date
|
||||
sleep 10
|
||||
jobs
|
||||
pstree -aspgT
|
||||
|
||||
# Make files with status and description we'll show for this check on Github.
|
||||
task_exit_code=$fuzzer_exit_code
|
||||
if [ "$server_died" == 1 ]
|
||||
then
|
||||
# The server has died.
|
||||
task_exit_code=210
|
||||
echo "failure" > status.txt
|
||||
if ! grep -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
|
||||
then
|
||||
echo "Lost connection to server. See the logs." > description.txt
|
||||
fi
|
||||
elif [ "$fuzzer_exit_code" == "143" ] || [ "$fuzzer_exit_code" == "0" ]
|
||||
then
|
||||
# Variants of a normal run:
|
||||
# 0 -- fuzzing ended earlier than timeout.
|
||||
# 143 -- SIGTERM -- the fuzzer was killed by timeout.
|
||||
task_exit_code=0
|
||||
echo "success" > status.txt
|
||||
echo "OK" > description.txt
|
||||
else
|
||||
# The server was alive, but the fuzzer returned some error. Probably this
|
||||
# is a problem in the fuzzer itself. Don't grep the server log in this
|
||||
# case, because we will find a message about normal server termination
|
||||
# (Received signal 15), which is confusing.
|
||||
task_exit_code=$fuzzer_exit_code
|
||||
echo "failure" > status.txt
|
||||
echo "Fuzzer failed ($fuzzer_exit_code). See the logs." > description.txt
|
||||
fi
|
||||
}
|
||||
|
||||
case "$stage" in
|
||||
@ -165,50 +234,7 @@ case "$stage" in
|
||||
time configure
|
||||
;&
|
||||
"fuzz")
|
||||
# Start a watchdog that should kill the fuzzer on timeout.
|
||||
# The shell won't kill the child sleep when we kill it, so we have to put it
|
||||
# into a separate process group so that we can kill them all.
|
||||
set -m
|
||||
watchdog &
|
||||
watchdog_pid=$!
|
||||
set +m
|
||||
# Check that the watchdog has started
|
||||
kill -0 $watchdog_pid
|
||||
|
||||
fuzzer_exit_code=0
|
||||
time fuzz || fuzzer_exit_code=$?
|
||||
kill -- -$watchdog_pid ||:
|
||||
|
||||
# Debug
|
||||
date
|
||||
sleep 10
|
||||
jobs
|
||||
pstree -aspgT
|
||||
|
||||
# Make files with status and description we'll show for this check on Github
|
||||
task_exit_code=$fuzzer_exit_code
|
||||
if [ "$fuzzer_exit_code" == 143 ]
|
||||
then
|
||||
# SIGTERM -- the fuzzer was killed by timeout, which means a normal run.
|
||||
echo "success" > status.txt
|
||||
echo "OK" > description.txt
|
||||
task_exit_code=0
|
||||
elif [ "$fuzzer_exit_code" == 210 ]
|
||||
then
|
||||
# Lost connection to the server. This probably means that the server died
|
||||
# with abort.
|
||||
echo "failure" > status.txt
|
||||
if ! grep -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
|
||||
then
|
||||
echo "Lost connection to server. See the logs." > description.txt
|
||||
fi
|
||||
else
|
||||
# Something different -- maybe the fuzzer itself died? Don't grep the
|
||||
# server log in this case, because we will find a message about normal
|
||||
# server termination (Received signal 15), which is confusing.
|
||||
echo "failure" > status.txt
|
||||
echo "Fuzzer failed ($fuzzer_exit_code). See the logs." > description.txt
|
||||
fi
|
||||
time fuzz
|
||||
;&
|
||||
"report")
|
||||
cat > report.html <<EOF ||:
|
||||
|
@ -3023,7 +3023,6 @@ SET limit = 5;
|
||||
SET offset = 7;
|
||||
SELECT * FROM test LIMIT 10 OFFSET 100;
|
||||
```
|
||||
|
||||
Result:
|
||||
|
||||
``` text
|
||||
@ -3034,4 +3033,36 @@ Result:
|
||||
└─────┘
|
||||
```
|
||||
|
||||
## optimize_fuse_sum_count_avg {#optimize_fuse_sum_count_avg}
|
||||
|
||||
Enables to fuse aggregate functions with identical argument. It rewrites query contains at least two aggregate functions from [sum](../../sql-reference/aggregate-functions/reference/sum.md#agg_function-sum), [count](../../sql-reference/aggregate-functions/reference/count.md#agg_function-count) or [avg](../../sql-reference/aggregate-functions/reference/avg.md#agg_function-avg) with identical argument to [sumCount](../../sql-reference/aggregate-functions/reference/sumcount.md#agg_function-sumCount).
|
||||
|
||||
Possible values:
|
||||
|
||||
- 0 — Functions with identical argument are not fused.
|
||||
- 1 — Functions with identical argument are fused.
|
||||
|
||||
Default value: `0`.
|
||||
|
||||
**Example**
|
||||
|
||||
Query:
|
||||
|
||||
``` sql
|
||||
CREATE TABLE fuse_tbl(a Int8, b Int8) Engine = Log;
|
||||
SET optimize_fuse_sum_count_avg = 1;
|
||||
EXPLAIN SYNTAX SELECT sum(a), sum(b), count(b), avg(b) from fuse_tbl FORMAT TSV;
|
||||
```
|
||||
|
||||
Result:
|
||||
|
||||
``` text
|
||||
SELECT
|
||||
sum(a),
|
||||
sumCount(b).1,
|
||||
sumCount(b).2,
|
||||
(sumCount(b).1) / (sumCount(b).2)
|
||||
FROM fuse_tbl
|
||||
```
|
||||
|
||||
[Original article](https://clickhouse.tech/docs/en/operations/settings/settings/) <!-- hide -->
|
||||
|
@ -1,4 +1,8 @@
|
||||
## rankCorr {#agg_function-rankcorr}
|
||||
---
|
||||
toc_priority: 145
|
||||
---
|
||||
|
||||
# rankCorr {#agg_function-rankcorr}
|
||||
|
||||
Computes a rank correlation coefficient.
|
||||
|
||||
|
@ -0,0 +1,46 @@
|
||||
---
|
||||
toc_priority: 144
|
||||
---
|
||||
|
||||
# sumCount {#agg_function-sumCount}
|
||||
|
||||
Calculates the sum of the numbers and counts the number of rows at the same time.
|
||||
|
||||
**Syntax**
|
||||
|
||||
``` sql
|
||||
sumCount(x)
|
||||
```
|
||||
|
||||
**Arguments**
|
||||
|
||||
- `x` — Input value, must be [Integer](../../../sql-reference/data-types/int-uint.md), [Float](../../../sql-reference/data-types/float.md), or [Decimal](../../../sql-reference/data-types/decimal.md).
|
||||
|
||||
**Returned value**
|
||||
|
||||
- Tuple `(sum, count)`, where `sum` is the sum of numbers and `count` is the number of rows with not-NULL values.
|
||||
|
||||
Type: [Tuple](../../../sql-reference/data-types/tuple.md).
|
||||
|
||||
**Example**
|
||||
|
||||
Query:
|
||||
|
||||
``` sql
|
||||
CREATE TABLE s_table (x Int8) Engine = Log;
|
||||
INSERT INTO s_table SELECT number FROM numbers(0, 20);
|
||||
INSERT INTO s_table VALUES (NULL);
|
||||
SELECT sumCount(x) from s_table;
|
||||
```
|
||||
|
||||
Result:
|
||||
|
||||
``` text
|
||||
┌─sumCount(x)─┐
|
||||
│ (190,20) │
|
||||
└─────────────┘
|
||||
```
|
||||
|
||||
**See also**
|
||||
|
||||
- [optimize_fuse_sum_count_avg](../../../operations/settings/settings.md#optimize_fuse_sum_count_avg) setting.
|
@ -4,7 +4,7 @@ toc_title: External Dictionaries
|
||||
---
|
||||
|
||||
!!! attention "Attention"
|
||||
`dict_name` parameter must be fully qualified for dictionaries created with DDL queries. Eg. `<database>.<dict_name>`.
|
||||
For dictionaries, created with [DDL queries](../../sql-reference/statements/create/dictionary.md), the `dict_name` parameter must be fully specified, like `<database>.<dict_name>`. Otherwise, the current database is used.
|
||||
|
||||
# Functions for Working with External Dictionaries {#ext_dict_functions}
|
||||
|
||||
|
@ -2924,4 +2924,36 @@ SELECT * FROM test LIMIT 10 OFFSET 100;
|
||||
|
||||
Значение по умолчанию: `1800`.
|
||||
|
||||
## optimize_fuse_sum_count_avg {#optimize_fuse_sum_count_avg}
|
||||
|
||||
Позволяет объединить агрегатные функции с одинаковым аргументом. Запрос, содержащий по крайней мере две агрегатные функции: [sum](../../sql-reference/aggregate-functions/reference/sum.md#agg_function-sum), [count](../../sql-reference/aggregate-functions/reference/count.md#agg_function-count) или [avg](../../sql-reference/aggregate-functions/reference/avg.md#agg_function-avg) с одинаковым аргументом, перезаписывается как [sumCount](../../sql-reference/aggregate-functions/reference/sumcount.md#agg_function-sumCount).
|
||||
|
||||
Возможные значения:
|
||||
|
||||
- 0 — функции с одинаковым аргументом не объединяются.
|
||||
- 1 — функции с одинаковым аргументом объединяются.
|
||||
|
||||
Значение по умолчанию: `0`.
|
||||
|
||||
**Пример**
|
||||
|
||||
Запрос:
|
||||
|
||||
``` sql
|
||||
CREATE TABLE fuse_tbl(a Int8, b Int8) Engine = Log;
|
||||
SET optimize_fuse_sum_count_avg = 1;
|
||||
EXPLAIN SYNTAX SELECT sum(a), sum(b), count(b), avg(b) from fuse_tbl FORMAT TSV;
|
||||
```
|
||||
|
||||
Результат:
|
||||
|
||||
``` text
|
||||
SELECT
|
||||
sum(a),
|
||||
sumCount(b).1,
|
||||
sumCount(b).2,
|
||||
(sumCount(b).1) / (sumCount(b).2)
|
||||
FROM fuse_tbl
|
||||
```
|
||||
|
||||
[Оригинальная статья](https://clickhouse.tech/docs/ru/operations/settings/settings/) <!--hide-->
|
||||
|
@ -1,4 +1,8 @@
|
||||
## rankCorr {#agg_function-rankcorr}
|
||||
---
|
||||
toc_priority: 145
|
||||
---
|
||||
|
||||
# rankCorr {#agg_function-rankcorr}
|
||||
|
||||
Вычисляет коэффициент ранговой корреляции.
|
||||
|
||||
|
@ -0,0 +1,46 @@
|
||||
---
|
||||
toc_priority: 144
|
||||
---
|
||||
|
||||
# sumCount {#agg_function-sumCount}
|
||||
|
||||
Вычисляет сумму чисел и одновременно подсчитывает количество строк.
|
||||
|
||||
**Синтаксис**
|
||||
|
||||
``` sql
|
||||
sumCount(x)
|
||||
```
|
||||
|
||||
**Аргументы**
|
||||
|
||||
- `x` — Входное значение типа [Integer](../../../sql-reference/data-types/int-uint.md), [Float](../../../sql-reference/data-types/float.md), или [Decimal](../../../sql-reference/data-types/decimal.md).
|
||||
|
||||
**Возвращаемое значение**
|
||||
|
||||
- Кортеж из элементов `(sum, count)`, где `sum` — это сумма чисел и `count` — количество строк со значениями, отличными от `NULL`.
|
||||
|
||||
Тип: [Tuple](../../../sql-reference/data-types/tuple.md).
|
||||
|
||||
**Пример**
|
||||
|
||||
Запрос:
|
||||
|
||||
``` sql
|
||||
CREATE TABLE s_table (x Nullable(Int8)) Engine = Log;
|
||||
INSERT INTO s_table SELECT number FROM numbers(0, 20);
|
||||
INSERT INTO s_table VALUES (NULL);
|
||||
SELECT sumCount(x) from s_table;
|
||||
```
|
||||
|
||||
Результат:
|
||||
|
||||
``` text
|
||||
┌─sumCount(x)─┐
|
||||
│ (190,20) │
|
||||
└─────────────┘
|
||||
```
|
||||
|
||||
**Смотрите также**
|
||||
|
||||
- Настройка [optimize_fuse_sum_count_avg](../../../operations/settings/settings.md#optimize_fuse_sum_count_avg)
|
@ -3,6 +3,9 @@ toc_priority: 58
|
||||
toc_title: "Функции для работы с внешними словарями"
|
||||
---
|
||||
|
||||
!!! attention "Внимание"
|
||||
Для словарей, созданных с помощью [DDL-запросов](../../sql-reference/statements/create/dictionary.md), в параметре `dict_name` указывается полное имя словаря вместе с базой данных, например: `<database>.<dict_name>`. Если база данных не указана, используется текущая.
|
||||
|
||||
# Функции для работы с внешними словарями {#ext_dict_functions}
|
||||
|
||||
Информацию о подключении и настройке внешних словарей смотрите в разделе [Внешние словари](../../sql-reference/dictionaries/external-dictionaries/external-dicts.md).
|
||||
|
@ -89,6 +89,17 @@ AggregateFunctionPtr AggregateFunctionFactory::get(
|
||||
|
||||
AggregateFunctionPtr nested_function = getImpl(
|
||||
name, nested_types, nested_parameters, out_properties, has_null_arguments);
|
||||
|
||||
// Pure window functions are not real aggregate functions. Applying
|
||||
// combinators doesn't make sense for them, they must handle the
|
||||
// nullability themselves. Another special case is functions from Nothing
|
||||
// that are rewritten to AggregateFunctionNothing, in this case
|
||||
// nested_function is nullptr.
|
||||
if (nested_function && nested_function->asWindowFunction())
|
||||
{
|
||||
return nested_function;
|
||||
}
|
||||
|
||||
return combinator->transformAggregateFunction(nested_function, out_properties, type_without_low_cardinality, parameters);
|
||||
}
|
||||
|
||||
@ -122,7 +133,7 @@ AggregateFunctionPtr AggregateFunctionFactory::getImpl(
|
||||
is_case_insensitive = true;
|
||||
}
|
||||
|
||||
ContextPtr query_context;
|
||||
ContextConstPtr query_context;
|
||||
if (CurrentThread::isInitialized())
|
||||
query_context = CurrentThread::get().getQueryContext();
|
||||
|
||||
|
@ -148,7 +148,7 @@ void LinearModelData::predict(
|
||||
const ColumnsWithTypeAndName & arguments,
|
||||
size_t offset,
|
||||
size_t limit,
|
||||
ContextPtr context) const
|
||||
ContextConstPtr context) const
|
||||
{
|
||||
gradient_computer->predict(container, arguments, offset, limit, weights, bias, context);
|
||||
}
|
||||
@ -455,7 +455,7 @@ void LogisticRegression::predict(
|
||||
size_t limit,
|
||||
const std::vector<Float64> & weights,
|
||||
Float64 bias,
|
||||
ContextPtr /*context*/) const
|
||||
ContextConstPtr /*context*/) const
|
||||
{
|
||||
size_t rows_num = arguments.front().column->size();
|
||||
|
||||
@ -523,7 +523,7 @@ void LinearRegression::predict(
|
||||
size_t limit,
|
||||
const std::vector<Float64> & weights,
|
||||
Float64 bias,
|
||||
ContextPtr /*context*/) const
|
||||
ContextConstPtr /*context*/) const
|
||||
{
|
||||
if (weights.size() + 1 != arguments.size())
|
||||
{
|
||||
|
@ -46,7 +46,7 @@ public:
|
||||
size_t limit,
|
||||
const std::vector<Float64> & weights,
|
||||
Float64 bias,
|
||||
ContextPtr context) const = 0;
|
||||
ContextConstPtr context) const = 0;
|
||||
};
|
||||
|
||||
|
||||
@ -71,7 +71,7 @@ public:
|
||||
size_t limit,
|
||||
const std::vector<Float64> & weights,
|
||||
Float64 bias,
|
||||
ContextPtr context) const override;
|
||||
ContextConstPtr context) const override;
|
||||
};
|
||||
|
||||
|
||||
@ -96,7 +96,7 @@ public:
|
||||
size_t limit,
|
||||
const std::vector<Float64> & weights,
|
||||
Float64 bias,
|
||||
ContextPtr context) const override;
|
||||
ContextConstPtr context) const override;
|
||||
};
|
||||
|
||||
|
||||
@ -266,7 +266,7 @@ public:
|
||||
const ColumnsWithTypeAndName & arguments,
|
||||
size_t offset,
|
||||
size_t limit,
|
||||
ContextPtr context) const;
|
||||
ContextConstPtr context) const;
|
||||
|
||||
void returnWeights(IColumn & to) const;
|
||||
private:
|
||||
@ -367,7 +367,7 @@ public:
|
||||
const ColumnsWithTypeAndName & arguments,
|
||||
size_t offset,
|
||||
size_t limit,
|
||||
ContextPtr context) const override
|
||||
ContextConstPtr context) const override
|
||||
{
|
||||
if (arguments.size() != param_num + 1)
|
||||
throw Exception(
|
||||
|
@ -123,7 +123,7 @@ public:
|
||||
const ColumnsWithTypeAndName & /*arguments*/,
|
||||
size_t /*offset*/,
|
||||
size_t /*limit*/,
|
||||
ContextPtr /*context*/) const
|
||||
ContextConstPtr /*context*/) const
|
||||
{
|
||||
throw Exception("Method predictValues is not supported for " + getName(), ErrorCodes::NOT_IMPLEMENTED);
|
||||
}
|
||||
|
@ -14,7 +14,7 @@ namespace DB
|
||||
|
||||
/// Common base class for XDBC and Library bridge helpers.
|
||||
/// Contains helper methods to check/start bridge sync.
|
||||
class IBridgeHelper: protected WithContext
|
||||
class IBridgeHelper: protected WithConstContext
|
||||
{
|
||||
|
||||
public:
|
||||
@ -27,7 +27,7 @@ public:
|
||||
static const inline std::string PING_METHOD = Poco::Net::HTTPRequest::HTTP_GET;
|
||||
static const inline std::string MAIN_METHOD = Poco::Net::HTTPRequest::HTTP_POST;
|
||||
|
||||
explicit IBridgeHelper(ContextPtr context_) : WithContext(context_) {}
|
||||
explicit IBridgeHelper(ContextConstPtr context_) : WithConstContext(context_) {}
|
||||
virtual ~IBridgeHelper() = default;
|
||||
|
||||
void startBridgeSync() const;
|
||||
|
@ -21,7 +21,7 @@ namespace DB
|
||||
{
|
||||
|
||||
LibraryBridgeHelper::LibraryBridgeHelper(
|
||||
ContextPtr context_,
|
||||
ContextConstPtr context_,
|
||||
const Block & sample_block_,
|
||||
const Field & dictionary_id_)
|
||||
: IBridgeHelper(context_->getGlobalContext())
|
||||
|
@ -17,7 +17,7 @@ class LibraryBridgeHelper : public IBridgeHelper
|
||||
public:
|
||||
static constexpr inline size_t DEFAULT_PORT = 9012;
|
||||
|
||||
LibraryBridgeHelper(ContextPtr context_, const Block & sample_block, const Field & dictionary_id_);
|
||||
LibraryBridgeHelper(ContextConstPtr context_, const Block & sample_block, const Field & dictionary_id_);
|
||||
|
||||
bool initLibrary(const std::string & library_path, std::string library_settings, std::string attributes_names);
|
||||
|
||||
|
@ -62,7 +62,7 @@ public:
|
||||
static constexpr inline auto SCHEMA_ALLOWED_HANDLER = "/schema_allowed";
|
||||
|
||||
XDBCBridgeHelper(
|
||||
ContextPtr context_,
|
||||
ContextConstPtr context_,
|
||||
Poco::Timespan http_timeout_,
|
||||
const std::string & connection_string_)
|
||||
: IXDBCBridgeHelper(context_->getGlobalContext())
|
||||
|
@ -162,7 +162,7 @@ MutableColumnPtr ColumnAggregateFunction::convertToValues(MutableColumnPtr colum
|
||||
return res;
|
||||
}
|
||||
|
||||
MutableColumnPtr ColumnAggregateFunction::predictValues(const ColumnsWithTypeAndName & arguments, ContextPtr context) const
|
||||
MutableColumnPtr ColumnAggregateFunction::predictValues(const ColumnsWithTypeAndName & arguments, ContextConstPtr context) const
|
||||
{
|
||||
MutableColumnPtr res = func->getReturnTypeToPredict()->createColumn();
|
||||
res->reserve(data.size());
|
||||
|
@ -119,7 +119,7 @@ public:
|
||||
const char * getFamilyName() const override { return "AggregateFunction"; }
|
||||
TypeIndex getDataType() const override { return TypeIndex::AggregateFunction; }
|
||||
|
||||
MutableColumnPtr predictValues(const ColumnsWithTypeAndName & arguments, ContextPtr context) const;
|
||||
MutableColumnPtr predictValues(const ColumnsWithTypeAndName & arguments, ContextConstPtr context) const;
|
||||
|
||||
size_t size() const override
|
||||
{
|
||||
|
@ -21,7 +21,15 @@ namespace ErrorCodes
|
||||
FileChecker::FileChecker(DiskPtr disk_, const String & file_info_path_) : disk(std::move(disk_))
|
||||
{
|
||||
setPath(file_info_path_);
|
||||
load();
|
||||
try
|
||||
{
|
||||
load();
|
||||
}
|
||||
catch (DB::Exception & e)
|
||||
{
|
||||
e.addMessage("Error loading file {}", files_info_path);
|
||||
throw;
|
||||
}
|
||||
}
|
||||
|
||||
void FileChecker::setPath(const String & file_info_path_)
|
||||
|
@ -173,7 +173,7 @@ void MemoryTracker::allocImpl(Int64 size, bool throw_if_memory_exceeded)
|
||||
}
|
||||
|
||||
#ifdef MEMORY_TRACKER_DEBUG_CHECKS
|
||||
if (unlikely(_memory_tracker_always_throw_logical_error_on_allocation) && throw_if_memory_exceeded)
|
||||
if (unlikely(_memory_tracker_always_throw_logical_error_on_allocation))
|
||||
{
|
||||
_memory_tracker_always_throw_logical_error_on_allocation = false;
|
||||
throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Memory tracker: allocations not allowed.");
|
||||
|
@ -19,11 +19,11 @@ namespace ErrorCodes
|
||||
{
|
||||
}
|
||||
|
||||
bool decimalCheckComparisonOverflow(ContextPtr context)
|
||||
bool decimalCheckComparisonOverflow(ContextConstPtr context)
|
||||
{
|
||||
return context->getSettingsRef().decimal_check_overflow;
|
||||
}
|
||||
bool decimalCheckArithmeticOverflow(ContextPtr context)
|
||||
bool decimalCheckArithmeticOverflow(ContextConstPtr context)
|
||||
{
|
||||
return context->getSettingsRef().decimal_check_overflow;
|
||||
}
|
||||
|
@ -18,8 +18,8 @@ namespace ErrorCodes
|
||||
extern const int ARGUMENT_OUT_OF_BOUND;
|
||||
}
|
||||
|
||||
bool decimalCheckComparisonOverflow(ContextPtr context);
|
||||
bool decimalCheckArithmeticOverflow(ContextPtr context);
|
||||
bool decimalCheckComparisonOverflow(ContextConstPtr context);
|
||||
bool decimalCheckArithmeticOverflow(ContextConstPtr context);
|
||||
|
||||
inline UInt32 leastDecimalPrecisionFor(TypeIndex int_type)
|
||||
{
|
||||
|
@ -17,7 +17,7 @@ void registerDictionarySourceCassandra(DictionarySourceFactory & factory)
|
||||
[[maybe_unused]] const Poco::Util::AbstractConfiguration & config,
|
||||
[[maybe_unused]] const std::string & config_prefix,
|
||||
[[maybe_unused]] Block & sample_block,
|
||||
ContextPtr /* context */,
|
||||
ContextConstPtr /* context */,
|
||||
const std::string & /* default_database */,
|
||||
bool /*created_from_ddl*/) -> DictionarySourcePtr
|
||||
{
|
||||
|
@ -27,7 +27,7 @@ namespace
|
||||
{
|
||||
constexpr size_t MAX_CONNECTIONS = 16;
|
||||
|
||||
inline UInt16 getPortFromContext(ContextPtr context, bool secure)
|
||||
inline UInt16 getPortFromContext(ContextConstPtr context, bool secure)
|
||||
{
|
||||
return secure ? context->getTCPPortSecure().value_or(0) : context->getTCPPort();
|
||||
}
|
||||
@ -60,7 +60,7 @@ ClickHouseDictionarySource::ClickHouseDictionarySource(
|
||||
const DictionaryStructure & dict_struct_,
|
||||
const Configuration & configuration_,
|
||||
const Block & sample_block_,
|
||||
ContextPtr context_)
|
||||
ContextConstPtr context_)
|
||||
: update_time{std::chrono::system_clock::from_time_t(0)}
|
||||
, dict_struct{dict_struct_}
|
||||
, configuration{configuration_}
|
||||
@ -209,7 +209,7 @@ void registerDictionarySourceClickHouse(DictionarySourceFactory & factory)
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & config_prefix,
|
||||
Block & sample_block,
|
||||
ContextPtr context,
|
||||
ContextConstPtr context,
|
||||
const std::string & default_database [[maybe_unused]],
|
||||
bool /* created_from_ddl */) -> DictionarySourcePtr
|
||||
{
|
||||
|
@ -37,7 +37,7 @@ public:
|
||||
const DictionaryStructure & dict_struct_,
|
||||
const Configuration & configuration_,
|
||||
const Block & sample_block_,
|
||||
ContextPtr context);
|
||||
ContextConstPtr context);
|
||||
|
||||
/// copy-constructor is provided in order to support cloneability
|
||||
ClickHouseDictionarySource(const ClickHouseDictionarySource & other);
|
||||
|
@ -31,7 +31,7 @@ DictionaryPtr DictionaryFactory::create(
|
||||
const std::string & name,
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & config_prefix,
|
||||
ContextPtr context,
|
||||
ContextConstPtr context,
|
||||
bool created_from_ddl) const
|
||||
{
|
||||
Poco::Util::AbstractConfiguration::Keys keys;
|
||||
@ -68,7 +68,7 @@ DictionaryPtr DictionaryFactory::create(
|
||||
layout_type);
|
||||
}
|
||||
|
||||
DictionaryPtr DictionaryFactory::create(const std::string & name, const ASTCreateQuery & ast, ContextPtr context) const
|
||||
DictionaryPtr DictionaryFactory::create(const std::string & name, const ASTCreateQuery & ast, ContextConstPtr context) const
|
||||
{
|
||||
auto configuration = getDictionaryConfigurationFromAST(ast, context);
|
||||
return DictionaryFactory::create(name, *configuration, "dictionary", context, true);
|
||||
|
@ -36,13 +36,13 @@ public:
|
||||
const std::string & name,
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & config_prefix,
|
||||
ContextPtr context,
|
||||
ContextConstPtr context,
|
||||
bool created_from_ddl) const;
|
||||
|
||||
/// Create dictionary from DDL-query
|
||||
DictionaryPtr create(const std::string & name,
|
||||
const ASTCreateQuery & ast,
|
||||
ContextPtr context) const;
|
||||
ContextConstPtr context) const;
|
||||
|
||||
using LayoutCreateFunction = std::function<DictionaryPtr(
|
||||
const std::string & name,
|
||||
@ -50,7 +50,7 @@ public:
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & config_prefix,
|
||||
DictionarySourcePtr source_ptr,
|
||||
ContextPtr context,
|
||||
ContextConstPtr context,
|
||||
bool created_from_ddl)>;
|
||||
|
||||
bool isComplex(const std::string & layout_type) const;
|
||||
|
@ -79,7 +79,7 @@ DictionarySourcePtr DictionarySourceFactory::create(
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & config_prefix,
|
||||
const DictionaryStructure & dict_struct,
|
||||
ContextPtr context,
|
||||
ContextConstPtr context,
|
||||
const std::string & default_database,
|
||||
bool check_config) const
|
||||
{
|
||||
|
@ -35,7 +35,7 @@ public:
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & config_prefix,
|
||||
Block & sample_block,
|
||||
ContextPtr context,
|
||||
ContextConstPtr context,
|
||||
const std::string & default_database,
|
||||
bool check_config)>;
|
||||
|
||||
@ -48,7 +48,7 @@ public:
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & config_prefix,
|
||||
const DictionaryStructure & dict_struct,
|
||||
ContextPtr context,
|
||||
ContextConstPtr context,
|
||||
const std::string & default_database,
|
||||
bool check_config) const;
|
||||
|
||||
|
@ -69,7 +69,7 @@ Block blockForKeys(
|
||||
|
||||
ContextPtr copyContextAndApplySettings(
|
||||
const std::string & config_prefix,
|
||||
ContextPtr context,
|
||||
ContextConstPtr context,
|
||||
const Poco::Util::AbstractConfiguration & config)
|
||||
{
|
||||
auto local_context = Context::createCopy(context);
|
||||
|
@ -39,12 +39,7 @@ Block blockForKeys(
|
||||
/// Used for applying settings to copied context in some register[...]Source functions
|
||||
ContextPtr copyContextAndApplySettings(
|
||||
const std::string & config_prefix,
|
||||
ContextPtr context,
|
||||
const Poco::Util::AbstractConfiguration & config);
|
||||
|
||||
void applySettingsToContext(
|
||||
const std::string & config_prefix,
|
||||
ContextPtr context,
|
||||
ContextConstPtr context,
|
||||
const Poco::Util::AbstractConfiguration & config);
|
||||
|
||||
/** A stream, adds additional columns to each block that it will read from inner stream.
|
||||
|
@ -306,7 +306,7 @@ namespace
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & config_prefix,
|
||||
DictionarySourcePtr source_ptr,
|
||||
ContextPtr /* context */,
|
||||
ContextConstPtr /* context */,
|
||||
bool /* created_from_ddl */)
|
||||
{
|
||||
const auto * layout_name = dictionary_key_type == DictionaryKeyType::simple ? "direct" : "complex_key_direct";
|
||||
|
@ -63,7 +63,7 @@ ExecutableDictionarySource::ExecutableDictionarySource(
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & config_prefix,
|
||||
Block & sample_block_,
|
||||
ContextPtr context_)
|
||||
ContextConstPtr context_)
|
||||
: log(&Poco::Logger::get("ExecutableDictionarySource"))
|
||||
, dict_struct{dict_struct_}
|
||||
, implicit_key{config.getBool(config_prefix + ".implicit_key", false)}
|
||||
@ -140,7 +140,7 @@ namespace
|
||||
{
|
||||
public:
|
||||
BlockInputStreamWithBackgroundThread(
|
||||
ContextPtr context,
|
||||
ContextConstPtr context,
|
||||
const std::string & format,
|
||||
const Block & sample_block,
|
||||
const std::string & command_str,
|
||||
@ -265,7 +265,7 @@ void registerDictionarySourceExecutable(DictionarySourceFactory & factory)
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & config_prefix,
|
||||
Block & sample_block,
|
||||
ContextPtr context,
|
||||
ContextConstPtr context,
|
||||
const std::string & /* default_database */,
|
||||
bool created_from_ddl) -> DictionarySourcePtr
|
||||
{
|
||||
|
@ -20,7 +20,7 @@ public:
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & config_prefix,
|
||||
Block & sample_block_,
|
||||
ContextPtr context_);
|
||||
ContextConstPtr context_);
|
||||
|
||||
ExecutableDictionarySource(const ExecutableDictionarySource & other);
|
||||
ExecutableDictionarySource & operator=(const ExecutableDictionarySource &) = delete;
|
||||
@ -58,7 +58,7 @@ private:
|
||||
const std::string update_field;
|
||||
const std::string format;
|
||||
Block sample_block;
|
||||
ContextPtr context;
|
||||
ContextConstPtr context;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -32,7 +32,7 @@ ExecutablePoolDictionarySource::ExecutablePoolDictionarySource(
|
||||
const DictionaryStructure & dict_struct_,
|
||||
const Configuration & configuration_,
|
||||
Block & sample_block_,
|
||||
ContextPtr context_)
|
||||
ContextConstPtr context_)
|
||||
: log(&Poco::Logger::get("ExecutablePoolDictionarySource"))
|
||||
, dict_struct{dict_struct_}
|
||||
, configuration{configuration_}
|
||||
@ -273,7 +273,7 @@ void registerDictionarySourceExecutablePool(DictionarySourceFactory & factory)
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & config_prefix,
|
||||
Block & sample_block,
|
||||
ContextPtr context,
|
||||
ContextConstPtr context,
|
||||
const std::string & /* default_database */,
|
||||
bool created_from_ddl) -> DictionarySourcePtr
|
||||
{
|
||||
|
@ -42,7 +42,7 @@ public:
|
||||
const DictionaryStructure & dict_struct_,
|
||||
const Configuration & configuration_,
|
||||
Block & sample_block_,
|
||||
ContextPtr context_);
|
||||
ContextConstPtr context_);
|
||||
|
||||
ExecutablePoolDictionarySource(const ExecutablePoolDictionarySource & other);
|
||||
ExecutablePoolDictionarySource & operator=(const ExecutablePoolDictionarySource &) = delete;
|
||||
@ -78,7 +78,7 @@ private:
|
||||
const Configuration configuration;
|
||||
|
||||
Block sample_block;
|
||||
ContextPtr context;
|
||||
ContextConstPtr context;
|
||||
std::shared_ptr<ProcessPool> process_pool;
|
||||
};
|
||||
|
||||
|
@ -28,7 +28,7 @@ namespace ErrorCodes
|
||||
|
||||
FileDictionarySource::FileDictionarySource(
|
||||
const std::string & filepath_, const std::string & format_,
|
||||
Block & sample_block_, ContextPtr context_, bool created_from_ddl)
|
||||
Block & sample_block_, ContextConstPtr context_, bool created_from_ddl)
|
||||
: filepath{filepath_}
|
||||
, format{format_}
|
||||
, sample_block{sample_block_}
|
||||
@ -77,7 +77,7 @@ void registerDictionarySourceFile(DictionarySourceFactory & factory)
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & config_prefix,
|
||||
Block & sample_block,
|
||||
ContextPtr context,
|
||||
ContextConstPtr context,
|
||||
const std::string & /* default_database */,
|
||||
bool created_from_ddl) -> DictionarySourcePtr
|
||||
{
|
||||
|
@ -17,7 +17,7 @@ class FileDictionarySource final : public IDictionarySource
|
||||
{
|
||||
public:
|
||||
FileDictionarySource(const std::string & filepath_, const std::string & format_,
|
||||
Block & sample_block_, ContextPtr context_, bool created_from_ddl);
|
||||
Block & sample_block_, ContextConstPtr context_, bool created_from_ddl);
|
||||
|
||||
FileDictionarySource(const FileDictionarySource & other);
|
||||
|
||||
@ -61,7 +61,7 @@ private:
|
||||
const std::string filepath;
|
||||
const std::string format;
|
||||
Block sample_block;
|
||||
ContextPtr context;
|
||||
ContextConstPtr context;
|
||||
Poco::Timestamp last_modification;
|
||||
};
|
||||
|
||||
|
@ -506,7 +506,7 @@ void registerDictionaryFlat(DictionaryFactory & factory)
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & config_prefix,
|
||||
DictionarySourcePtr source_ptr,
|
||||
ContextPtr /* context */,
|
||||
ContextConstPtr /* context */,
|
||||
bool /* created_from_ddl */) -> DictionaryPtr
|
||||
{
|
||||
if (dict_struct.key)
|
||||
|
@ -31,7 +31,7 @@ HTTPDictionarySource::HTTPDictionarySource(
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & config_prefix,
|
||||
Block & sample_block_,
|
||||
ContextPtr context_,
|
||||
ContextConstPtr context_,
|
||||
bool created_from_ddl)
|
||||
: log(&Poco::Logger::get("HTTPDictionarySource"))
|
||||
, update_time{std::chrono::system_clock::from_time_t(0)}
|
||||
@ -231,7 +231,7 @@ void registerDictionarySourceHTTP(DictionarySourceFactory & factory)
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & config_prefix,
|
||||
Block & sample_block,
|
||||
ContextPtr context,
|
||||
ContextConstPtr context,
|
||||
const std::string & /* default_database */,
|
||||
bool created_from_ddl) -> DictionarySourcePtr {
|
||||
if (dict_struct.has_expressions)
|
||||
|
@ -27,7 +27,7 @@ public:
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & config_prefix,
|
||||
Block & sample_block_,
|
||||
ContextPtr context_,
|
||||
ContextConstPtr context_,
|
||||
bool created_from_ddl);
|
||||
|
||||
HTTPDictionarySource(const HTTPDictionarySource & other);
|
||||
@ -69,7 +69,7 @@ private:
|
||||
std::string update_field;
|
||||
const std::string format;
|
||||
Block sample_block;
|
||||
ContextPtr context;
|
||||
ContextConstPtr context;
|
||||
ConnectionTimeouts timeouts;
|
||||
};
|
||||
|
||||
|
@ -751,13 +751,13 @@ void registerDictionaryHashed(DictionaryFactory & factory)
|
||||
using namespace std::placeholders;
|
||||
|
||||
factory.registerLayout("hashed",
|
||||
[=](auto && a, auto && b, auto && c, auto && d, DictionarySourcePtr e, ContextPtr /* context */, bool /*created_from_ddl*/){ return create_layout(a, b, c, d, std::move(e), DictionaryKeyType::simple, /* sparse = */ false); }, false);
|
||||
[=](auto && a, auto && b, auto && c, auto && d, DictionarySourcePtr e, ContextConstPtr /* context */, bool /*created_from_ddl*/){ return create_layout(a, b, c, d, std::move(e), DictionaryKeyType::simple, /* sparse = */ false); }, false);
|
||||
factory.registerLayout("sparse_hashed",
|
||||
[=](auto && a, auto && b, auto && c, auto && d, DictionarySourcePtr e, ContextPtr /* context */, bool /*created_from_ddl*/){ return create_layout(a, b, c, d, std::move(e), DictionaryKeyType::simple, /* sparse = */ true); }, false);
|
||||
[=](auto && a, auto && b, auto && c, auto && d, DictionarySourcePtr e, ContextConstPtr /* context */, bool /*created_from_ddl*/){ return create_layout(a, b, c, d, std::move(e), DictionaryKeyType::simple, /* sparse = */ true); }, false);
|
||||
factory.registerLayout("complex_key_hashed",
|
||||
[=](auto && a, auto && b, auto && c, auto && d, DictionarySourcePtr e, ContextPtr /* context */, bool /*created_from_ddl*/){ return create_layout(a, b, c, d, std::move(e), DictionaryKeyType::complex, /* sparse = */ false); }, true);
|
||||
[=](auto && a, auto && b, auto && c, auto && d, DictionarySourcePtr e, ContextConstPtr /* context */, bool /*created_from_ddl*/){ return create_layout(a, b, c, d, std::move(e), DictionaryKeyType::complex, /* sparse = */ false); }, true);
|
||||
factory.registerLayout("complex_key_sparse_hashed",
|
||||
[=](auto && a, auto && b, auto && c, auto && d, DictionarySourcePtr e, ContextPtr /* context */, bool /*created_from_ddl*/){ return create_layout(a, b, c, d, std::move(e), DictionaryKeyType::complex, /* sparse = */ true); }, true);
|
||||
[=](auto && a, auto && b, auto && c, auto && d, DictionarySourcePtr e, ContextConstPtr /* context */, bool /*created_from_ddl*/){ return create_layout(a, b, c, d, std::move(e), DictionaryKeyType::complex, /* sparse = */ true); }, true);
|
||||
|
||||
}
|
||||
|
||||
|
@ -935,7 +935,7 @@ void registerDictionaryTrie(DictionaryFactory & factory)
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & config_prefix,
|
||||
DictionarySourcePtr source_ptr,
|
||||
ContextPtr /* context */,
|
||||
ContextConstPtr /* context */,
|
||||
bool /*created_from_ddl*/) -> DictionaryPtr
|
||||
{
|
||||
if (!dict_struct.key || dict_struct.key->size() != 1)
|
||||
|
@ -31,7 +31,7 @@ LibraryDictionarySource::LibraryDictionarySource(
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & config_prefix_,
|
||||
Block & sample_block_,
|
||||
ContextPtr context_,
|
||||
ContextConstPtr context_,
|
||||
bool created_from_ddl)
|
||||
: log(&Poco::Logger::get("LibraryDictionarySource"))
|
||||
, dict_struct{dict_struct_}
|
||||
@ -172,7 +172,7 @@ void registerDictionarySourceLibrary(DictionarySourceFactory & factory)
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & config_prefix,
|
||||
Block & sample_block,
|
||||
ContextPtr context,
|
||||
ContextConstPtr context,
|
||||
const std::string & /* default_database */,
|
||||
bool created_from_ddl) -> DictionarySourcePtr
|
||||
{
|
||||
|
@ -39,7 +39,7 @@ public:
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & config_prefix_,
|
||||
Block & sample_block_,
|
||||
ContextPtr context_,
|
||||
ContextConstPtr context_,
|
||||
bool created_from_ddl);
|
||||
|
||||
LibraryDictionarySource(const LibraryDictionarySource & other);
|
||||
@ -86,7 +86,7 @@ private:
|
||||
const Field dictionary_id;
|
||||
|
||||
Block sample_block;
|
||||
ContextPtr context;
|
||||
ContextConstPtr context;
|
||||
|
||||
LibraryBridgeHelperPtr bridge_helper;
|
||||
ExternalResultDescription description;
|
||||
|
@ -13,7 +13,7 @@ void registerDictionarySourceMongoDB(DictionarySourceFactory & factory)
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & root_config_prefix,
|
||||
Block & sample_block,
|
||||
ContextPtr,
|
||||
ContextConstPtr,
|
||||
const std::string & /* default_database */,
|
||||
bool /* created_from_ddl */)
|
||||
{
|
||||
|
@ -24,7 +24,7 @@ void registerDictionarySourceMysql(DictionarySourceFactory & factory)
|
||||
[[maybe_unused]] const Poco::Util::AbstractConfiguration & config,
|
||||
[[maybe_unused]] const std::string & config_prefix,
|
||||
[[maybe_unused]] Block & sample_block,
|
||||
[[maybe_unused]] ContextPtr context,
|
||||
[[maybe_unused]] ContextConstPtr context,
|
||||
const std::string & /* default_database */,
|
||||
bool /* created_from_ddl */) -> DictionarySourcePtr {
|
||||
#if USE_MYSQL
|
||||
|
@ -167,7 +167,7 @@ DictionaryPtr createLayout(const std::string & ,
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & config_prefix,
|
||||
DictionarySourcePtr source_ptr,
|
||||
ContextPtr /* context */,
|
||||
ContextConstPtr /* context */,
|
||||
bool /*created_from_ddl*/)
|
||||
{
|
||||
const String database = config.getString(config_prefix + ".database", "");
|
||||
|
@ -188,7 +188,7 @@ void registerDictionarySourcePostgreSQL(DictionarySourceFactory & factory)
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & root_config_prefix,
|
||||
Block & sample_block,
|
||||
ContextPtr context,
|
||||
ContextConstPtr context,
|
||||
const std::string & /* default_database */,
|
||||
bool /* created_from_ddl */) -> DictionarySourcePtr
|
||||
{
|
||||
|
@ -637,7 +637,7 @@ void registerDictionaryRangeHashed(DictionaryFactory & factory)
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & config_prefix,
|
||||
DictionarySourcePtr source_ptr,
|
||||
ContextPtr /* context */,
|
||||
ContextConstPtr /* context */,
|
||||
bool /*created_from_ddl*/) -> DictionaryPtr
|
||||
{
|
||||
if (dict_struct.key)
|
||||
|
@ -12,7 +12,7 @@ void registerDictionarySourceRedis(DictionarySourceFactory & factory)
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const String & config_prefix,
|
||||
Block & sample_block,
|
||||
ContextPtr /* context */,
|
||||
ContextConstPtr /* context */,
|
||||
const std::string & /* default_database */,
|
||||
bool /* created_from_ddl */) -> DictionarySourcePtr {
|
||||
return std::make_unique<RedisDictionarySource>(dict_struct, config, config_prefix + ".redis", sample_block);
|
||||
|
@ -37,7 +37,7 @@ namespace
|
||||
const Poco::URI & uri,
|
||||
std::function<void(std::ostream &)> callback,
|
||||
const Block & sample_block,
|
||||
ContextPtr context,
|
||||
ContextConstPtr context,
|
||||
UInt64 max_block_size,
|
||||
const ConnectionTimeouts & timeouts,
|
||||
const String name_)
|
||||
@ -102,7 +102,7 @@ XDBCDictionarySource::XDBCDictionarySource(
|
||||
const Poco::Util::AbstractConfiguration & config_,
|
||||
const std::string & config_prefix_,
|
||||
const Block & sample_block_,
|
||||
ContextPtr context_,
|
||||
ContextConstPtr context_,
|
||||
const BridgeHelperPtr bridge_)
|
||||
: WithContext(context_->getGlobalContext())
|
||||
, log(&Poco::Logger::get(bridge_->getName() + "DictionarySource"))
|
||||
@ -278,7 +278,7 @@ void registerDictionarySourceXDBC(DictionarySourceFactory & factory)
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & config_prefix,
|
||||
Block & sample_block,
|
||||
ContextPtr context,
|
||||
ContextConstPtr context,
|
||||
const std::string & /* default_database */,
|
||||
bool /* check_config */) -> DictionarySourcePtr {
|
||||
#if USE_ODBC
|
||||
@ -305,7 +305,7 @@ void registerDictionarySourceJDBC(DictionarySourceFactory & factory)
|
||||
const Poco::Util::AbstractConfiguration & /* config */,
|
||||
const std::string & /* config_prefix */,
|
||||
Block & /* sample_block */,
|
||||
ContextPtr /* context */,
|
||||
ContextConstPtr /* context */,
|
||||
const std::string & /* default_database */,
|
||||
bool /* created_from_ddl */) -> DictionarySourcePtr {
|
||||
throw Exception(ErrorCodes::SUPPORT_IS_DISABLED,
|
||||
|
@ -31,7 +31,7 @@ public:
|
||||
const Poco::Util::AbstractConfiguration & config_,
|
||||
const std::string & config_prefix_,
|
||||
const Block & sample_block_,
|
||||
ContextPtr context_,
|
||||
ContextConstPtr context_,
|
||||
BridgeHelperPtr bridge);
|
||||
|
||||
/// copy-constructor is provided in order to support cloneability
|
||||
|
@ -397,7 +397,7 @@ void buildConfigurationFromFunctionWithKeyValueArguments(
|
||||
AutoPtr<Document> doc,
|
||||
AutoPtr<Element> root,
|
||||
const ASTExpressionList * ast_expr_list,
|
||||
ContextPtr context)
|
||||
ContextConstPtr context)
|
||||
{
|
||||
const auto & children = ast_expr_list->children;
|
||||
for (size_t i = 0; i != children.size(); ++i)
|
||||
@ -464,7 +464,7 @@ void buildSourceConfiguration(
|
||||
AutoPtr<Element> root,
|
||||
const ASTFunctionWithKeyValueArguments * source,
|
||||
const ASTDictionarySettings * settings,
|
||||
ContextPtr context)
|
||||
ContextConstPtr context)
|
||||
{
|
||||
AutoPtr<Element> outer_element(doc->createElement("source"));
|
||||
root->appendChild(outer_element);
|
||||
@ -525,7 +525,7 @@ void checkPrimaryKey(const NamesToTypeNames & all_attrs, const Names & key_attrs
|
||||
|
||||
|
||||
DictionaryConfigurationPtr
|
||||
getDictionaryConfigurationFromAST(const ASTCreateQuery & query, ContextPtr context, const std::string & database_)
|
||||
getDictionaryConfigurationFromAST(const ASTCreateQuery & query, ContextConstPtr context, const std::string & database_)
|
||||
{
|
||||
checkAST(query);
|
||||
|
||||
|
@ -12,5 +12,5 @@ using DictionaryConfigurationPtr = Poco::AutoPtr<Poco::Util::AbstractConfigurati
|
||||
/// This function is necessary because all loadable objects configuration are Poco::AbstractConfiguration
|
||||
/// Can throw exception if query is ill-formed
|
||||
DictionaryConfigurationPtr
|
||||
getDictionaryConfigurationFromAST(const ASTCreateQuery & query, ContextPtr context, const std::string & database_ = "");
|
||||
getDictionaryConfigurationFromAST(const ASTCreateQuery & query, ContextConstPtr context, const std::string & database_ = "");
|
||||
}
|
||||
|
@ -206,7 +206,7 @@ DictionaryPtr createSSDCacheDictionaryLayout(
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & config_prefix,
|
||||
DictionarySourcePtr source_ptr,
|
||||
ContextPtr context,
|
||||
ContextConstPtr context,
|
||||
bool created_from_ddl)
|
||||
{
|
||||
static_assert(dictionary_key_type != DictionaryKeyType::range, "Range key type is not supported by CacheDictionary");
|
||||
@ -265,7 +265,7 @@ void registerDictionaryCache(DictionaryFactory & factory)
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & config_prefix,
|
||||
DictionarySourcePtr source_ptr,
|
||||
ContextPtr /* context */,
|
||||
ContextConstPtr /* context */,
|
||||
bool /* created_from_ddl */) -> DictionaryPtr
|
||||
{
|
||||
return createCacheDictionaryLayout<DictionaryKeyType::simple>(full_name, dict_struct, config, config_prefix, std::move(source_ptr));
|
||||
@ -278,7 +278,7 @@ void registerDictionaryCache(DictionaryFactory & factory)
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & config_prefix,
|
||||
DictionarySourcePtr source_ptr,
|
||||
ContextPtr /* context */,
|
||||
ContextConstPtr /* context */,
|
||||
bool /* created_from_ddl */) -> DictionaryPtr
|
||||
{
|
||||
return createCacheDictionaryLayout<DictionaryKeyType::complex>(full_name, dict_struct, config, config_prefix, std::move(source_ptr));
|
||||
@ -293,7 +293,7 @@ void registerDictionaryCache(DictionaryFactory & factory)
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & config_prefix,
|
||||
DictionarySourcePtr source_ptr,
|
||||
ContextPtr context,
|
||||
ContextConstPtr context,
|
||||
bool created_from_ddl) -> DictionaryPtr
|
||||
{
|
||||
return createSSDCacheDictionaryLayout<DictionaryKeyType::simple>(full_name, dict_struct, config, config_prefix, std::move(source_ptr), context, created_from_ddl);
|
||||
@ -306,7 +306,7 @@ void registerDictionaryCache(DictionaryFactory & factory)
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & config_prefix,
|
||||
DictionarySourcePtr source_ptr,
|
||||
ContextPtr context,
|
||||
ContextConstPtr context,
|
||||
bool created_from_ddl) -> DictionaryPtr {
|
||||
return createSSDCacheDictionaryLayout<DictionaryKeyType::complex>(full_name, dict_struct, config, config_prefix, std::move(source_ptr), context, created_from_ddl);
|
||||
};
|
||||
|
@ -61,7 +61,7 @@ class FunctionBase64Conversion : public IFunction
|
||||
public:
|
||||
static constexpr auto name = Func::name;
|
||||
|
||||
static FunctionPtr create(ContextPtr)
|
||||
static FunctionPtr create(ContextConstPtr)
|
||||
{
|
||||
return std::make_shared<FunctionBase64Conversion>();
|
||||
}
|
||||
|
@ -514,7 +514,7 @@ class FunctionBinaryArithmetic : public IFunction
|
||||
static constexpr const bool is_multiply = IsOperation<Op>::multiply;
|
||||
static constexpr const bool is_division = IsOperation<Op>::division;
|
||||
|
||||
ContextPtr context;
|
||||
ContextConstPtr context;
|
||||
bool check_decimal_overflow = true;
|
||||
|
||||
template <typename F>
|
||||
@ -597,7 +597,7 @@ class FunctionBinaryArithmetic : public IFunction
|
||||
}
|
||||
|
||||
static FunctionOverloadResolverPtr
|
||||
getFunctionForIntervalArithmetic(const DataTypePtr & type0, const DataTypePtr & type1, ContextPtr context)
|
||||
getFunctionForIntervalArithmetic(const DataTypePtr & type0, const DataTypePtr & type1, ContextConstPtr context)
|
||||
{
|
||||
bool first_is_date_or_datetime = isDateOrDateTime(type0);
|
||||
bool second_is_date_or_datetime = isDateOrDateTime(type1);
|
||||
@ -945,9 +945,9 @@ class FunctionBinaryArithmetic : public IFunction
|
||||
|
||||
public:
|
||||
static constexpr auto name = Name::name;
|
||||
static FunctionPtr create(ContextPtr context) { return std::make_shared<FunctionBinaryArithmetic>(context); }
|
||||
static FunctionPtr create(ContextConstPtr context) { return std::make_shared<FunctionBinaryArithmetic>(context); }
|
||||
|
||||
explicit FunctionBinaryArithmetic(ContextPtr context_)
|
||||
explicit FunctionBinaryArithmetic(ContextConstPtr context_)
|
||||
: context(context_),
|
||||
check_decimal_overflow(decimalCheckArithmeticOverflow(context))
|
||||
{}
|
||||
@ -961,7 +961,7 @@ public:
|
||||
return getReturnTypeImplStatic(arguments, context);
|
||||
}
|
||||
|
||||
static DataTypePtr getReturnTypeImplStatic(const DataTypes & arguments, ContextPtr context)
|
||||
static DataTypePtr getReturnTypeImplStatic(const DataTypes & arguments, ContextConstPtr context)
|
||||
{
|
||||
/// Special case when multiply aggregate function state
|
||||
if (isAggregateMultiply(arguments[0], arguments[1]))
|
||||
@ -1373,7 +1373,7 @@ public:
|
||||
const ColumnWithTypeAndName & left_,
|
||||
const ColumnWithTypeAndName & right_,
|
||||
const DataTypePtr & return_type_,
|
||||
ContextPtr context)
|
||||
ContextConstPtr context)
|
||||
{
|
||||
return std::make_shared<FunctionBinaryArithmeticWithConstants>(left_, right_, return_type_, context);
|
||||
}
|
||||
@ -1382,7 +1382,7 @@ public:
|
||||
const ColumnWithTypeAndName & left_,
|
||||
const ColumnWithTypeAndName & right_,
|
||||
const DataTypePtr & return_type_,
|
||||
ContextPtr context_)
|
||||
ContextConstPtr context_)
|
||||
: Base(context_), left(left_), right(right_), return_type(return_type_)
|
||||
{
|
||||
}
|
||||
@ -1536,12 +1536,12 @@ class BinaryArithmeticOverloadResolver : public IFunctionOverloadResolver
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = Name::name;
|
||||
static FunctionOverloadResolverPtr create(ContextPtr context)
|
||||
static FunctionOverloadResolverPtr create(ContextConstPtr context)
|
||||
{
|
||||
return std::make_unique<BinaryArithmeticOverloadResolver>(context);
|
||||
}
|
||||
|
||||
explicit BinaryArithmeticOverloadResolver(ContextPtr context_) : context(context_) {}
|
||||
explicit BinaryArithmeticOverloadResolver(ContextConstPtr context_) : context(context_) {}
|
||||
|
||||
String getName() const override { return name; }
|
||||
size_t getNumberOfArguments() const override { return 2; }
|
||||
@ -1577,6 +1577,6 @@ public:
|
||||
}
|
||||
|
||||
private:
|
||||
ContextPtr context;
|
||||
ContextConstPtr context;
|
||||
};
|
||||
}
|
||||
|
@ -24,7 +24,7 @@ struct FunctionBitTestMany : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = Name::name;
|
||||
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionBitTestMany>(); }
|
||||
static FunctionPtr create(ContextConstPtr) { return std::make_shared<FunctionBitTestMany>(); }
|
||||
|
||||
String getName() const override { return name; }
|
||||
|
||||
|
@ -24,7 +24,7 @@ class FunctionCustomWeekToSomething : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = Transform::name;
|
||||
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionCustomWeekToSomething>(); }
|
||||
static FunctionPtr create(ContextConstPtr) { return std::make_shared<FunctionCustomWeekToSomething>(); }
|
||||
|
||||
String getName() const override { return name; }
|
||||
|
||||
|
@ -352,7 +352,7 @@ class FunctionDateOrDateTimeAddInterval : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = Transform::name;
|
||||
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionDateOrDateTimeAddInterval>(); }
|
||||
static FunctionPtr create(ContextConstPtr) { return std::make_shared<FunctionDateOrDateTimeAddInterval>(); }
|
||||
|
||||
String getName() const override
|
||||
{
|
||||
|
@ -24,7 +24,7 @@ class FunctionDateOrDateTimeToSomething : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = Transform::name;
|
||||
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionDateOrDateTimeToSomething>(); }
|
||||
static FunctionPtr create(ContextConstPtr) { return std::make_shared<FunctionDateOrDateTimeToSomething>(); }
|
||||
|
||||
String getName() const override
|
||||
{
|
||||
|
@ -12,7 +12,7 @@ class FunctionFQDN : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "FQDN";
|
||||
static FunctionPtr create(ContextPtr)
|
||||
static FunctionPtr create(ContextConstPtr)
|
||||
{
|
||||
return std::make_shared<FunctionFQDN>();
|
||||
}
|
||||
|
@ -52,7 +52,7 @@ void FunctionFactory::registerFunction(const
|
||||
|
||||
FunctionOverloadResolverPtr FunctionFactory::getImpl(
|
||||
const std::string & name,
|
||||
ContextPtr context) const
|
||||
ContextConstPtr context) const
|
||||
{
|
||||
auto res = tryGetImpl(name, context);
|
||||
if (!res)
|
||||
@ -82,14 +82,14 @@ std::vector<std::string> FunctionFactory::getAllNames() const
|
||||
|
||||
FunctionOverloadResolverPtr FunctionFactory::get(
|
||||
const std::string & name,
|
||||
ContextPtr context) const
|
||||
ContextConstPtr context) const
|
||||
{
|
||||
return getImpl(name, context);
|
||||
}
|
||||
|
||||
FunctionOverloadResolverPtr FunctionFactory::tryGetImpl(
|
||||
const std::string & name_param,
|
||||
ContextPtr context) const
|
||||
ContextConstPtr context) const
|
||||
{
|
||||
String name = getAliasToOrName(name_param);
|
||||
FunctionOverloadResolverPtr res;
|
||||
@ -120,7 +120,7 @@ FunctionOverloadResolverPtr FunctionFactory::tryGetImpl(
|
||||
|
||||
FunctionOverloadResolverPtr FunctionFactory::tryGet(
|
||||
const std::string & name,
|
||||
ContextPtr context) const
|
||||
ContextConstPtr context) const
|
||||
{
|
||||
auto impl = tryGetImpl(name, context);
|
||||
return impl ? std::move(impl) : nullptr;
|
||||
|
@ -19,7 +19,7 @@ namespace DB
|
||||
* some dictionaries from Context.
|
||||
*/
|
||||
class FunctionFactory : private boost::noncopyable,
|
||||
public IFactoryWithAliases<std::function<FunctionOverloadResolverPtr(ContextPtr)>>
|
||||
public IFactoryWithAliases<std::function<FunctionOverloadResolverPtr(ContextConstPtr)>>
|
||||
{
|
||||
public:
|
||||
static FunctionFactory & instance();
|
||||
@ -44,14 +44,14 @@ public:
|
||||
std::vector<std::string> getAllNames() const;
|
||||
|
||||
/// Throws an exception if not found.
|
||||
FunctionOverloadResolverPtr get(const std::string & name, ContextPtr context) const;
|
||||
FunctionOverloadResolverPtr get(const std::string & name, ContextConstPtr context) const;
|
||||
|
||||
/// Returns nullptr if not found.
|
||||
FunctionOverloadResolverPtr tryGet(const std::string & name, ContextPtr context) const;
|
||||
FunctionOverloadResolverPtr tryGet(const std::string & name, ContextConstPtr context) const;
|
||||
|
||||
/// The same methods to get developer interface implementation.
|
||||
FunctionOverloadResolverPtr getImpl(const std::string & name, ContextPtr context) const;
|
||||
FunctionOverloadResolverPtr tryGetImpl(const std::string & name, ContextPtr context) const;
|
||||
FunctionOverloadResolverPtr getImpl(const std::string & name, ContextConstPtr context) const;
|
||||
FunctionOverloadResolverPtr tryGetImpl(const std::string & name, ContextConstPtr context) const;
|
||||
|
||||
/// Register a function by its name.
|
||||
/// No locking, you must register all functions before usage of get.
|
||||
@ -67,7 +67,7 @@ private:
|
||||
Functions case_insensitive_functions;
|
||||
|
||||
template <typename Function>
|
||||
static FunctionOverloadResolverPtr adaptFunctionToOverloadResolver(ContextPtr context)
|
||||
static FunctionOverloadResolverPtr adaptFunctionToOverloadResolver(ContextConstPtr context)
|
||||
{
|
||||
return std::make_unique<FunctionToOverloadResolverAdaptor>(Function::create(context));
|
||||
}
|
||||
|
@ -21,12 +21,12 @@ namespace ErrorCodes
|
||||
}
|
||||
|
||||
/// A function to read file as a string.
|
||||
class FunctionFile : public IFunction, WithContext
|
||||
class FunctionFile : public IFunction, WithConstContext
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "file";
|
||||
static FunctionPtr create(ContextPtr context_) { return std::make_shared<FunctionFile>(context_); }
|
||||
explicit FunctionFile(ContextPtr context_) : WithContext(context_) {}
|
||||
static FunctionPtr create(ContextConstPtr context_) { return std::make_shared<FunctionFile>(context_); }
|
||||
explicit FunctionFile(ContextConstPtr context_) : WithConstContext(context_) {}
|
||||
|
||||
String getName() const override { return name; }
|
||||
|
||||
|
@ -36,7 +36,7 @@ ExecutableFunctionPtr FunctionJoinGet<or_null>::prepare(const ColumnsWithTypeAnd
|
||||
}
|
||||
|
||||
static std::pair<std::shared_ptr<StorageJoin>, String>
|
||||
getJoin(const ColumnsWithTypeAndName & arguments, ContextPtr context)
|
||||
getJoin(const ColumnsWithTypeAndName & arguments, ContextConstPtr context)
|
||||
{
|
||||
String join_name;
|
||||
if (const auto * name_col = checkAndGetColumnConst<ColumnString>(arguments[0].column.get()))
|
||||
@ -63,7 +63,7 @@ getJoin(const ColumnsWithTypeAndName & arguments, ContextPtr context)
|
||||
String table_name = join_name.substr(dot);
|
||||
if (table_name.empty())
|
||||
throw Exception("joinGet does not allow empty table name", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
auto table = DatabaseCatalog::instance().getTable({database_name, table_name}, context);
|
||||
auto table = DatabaseCatalog::instance().getTable({database_name, table_name}, std::const_pointer_cast<Context>(context));
|
||||
auto storage_join = std::dynamic_pointer_cast<StorageJoin>(table);
|
||||
if (!storage_join)
|
||||
throw Exception("Table " + join_name + " should have engine StorageJoin", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
@ -74,13 +74,13 @@ private:
|
||||
};
|
||||
|
||||
template <bool or_null>
|
||||
class JoinGetOverloadResolver final : public IFunctionOverloadResolver, WithContext
|
||||
class JoinGetOverloadResolver final : public IFunctionOverloadResolver, WithConstContext
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = or_null ? "joinGetOrNull" : "joinGet";
|
||||
static FunctionOverloadResolverPtr create(ContextPtr context_) { return std::make_unique<JoinGetOverloadResolver>(context_); }
|
||||
static FunctionOverloadResolverPtr create(ContextConstPtr context_) { return std::make_unique<JoinGetOverloadResolver>(context_); }
|
||||
|
||||
explicit JoinGetOverloadResolver(ContextPtr context_) : WithContext(context_) {}
|
||||
explicit JoinGetOverloadResolver(ContextConstPtr context_) : WithConstContext(context_) {}
|
||||
|
||||
String getName() const override { return name; }
|
||||
|
||||
|
@ -28,7 +28,7 @@ class FunctionMathBinaryFloat64 : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = Impl::name;
|
||||
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionMathBinaryFloat64>(); }
|
||||
static FunctionPtr create(ContextConstPtr) { return std::make_shared<FunctionMathBinaryFloat64>(); }
|
||||
static_assert(Impl::rows_per_iteration > 0, "Impl must process at least one row per iteration");
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
@ -13,7 +13,7 @@ class FunctionMathConstFloat64 : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = Impl::name;
|
||||
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionMathConstFloat64>(); }
|
||||
static FunctionPtr create(ContextConstPtr) { return std::make_shared<FunctionMathConstFloat64>(); }
|
||||
|
||||
private:
|
||||
String getName() const override { return name; }
|
||||
|
@ -35,7 +35,7 @@ class FunctionMathUnary : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = Impl::name;
|
||||
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionMathUnary>(); }
|
||||
static FunctionPtr create(ContextConstPtr) { return std::make_shared<FunctionMathUnary>(); }
|
||||
|
||||
private:
|
||||
String getName() const override { return name; }
|
||||
|
@ -23,7 +23,7 @@ class FunctionNumericPredicate : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = Impl::name;
|
||||
static FunctionPtr create(ContextPtr)
|
||||
static FunctionPtr create(ContextConstPtr)
|
||||
{
|
||||
return std::make_shared<FunctionNumericPredicate>();
|
||||
}
|
||||
|
@ -142,7 +142,7 @@ template <typename Name>
|
||||
class FunctionStartsEndsWith : public TargetSpecific::Default::FunctionStartsEndsWith<Name>
|
||||
{
|
||||
public:
|
||||
explicit FunctionStartsEndsWith(ContextPtr context) : selector(context)
|
||||
explicit FunctionStartsEndsWith(ContextConstPtr context) : selector(context)
|
||||
{
|
||||
selector.registerImplementation<TargetArch::Default,
|
||||
TargetSpecific::Default::FunctionStartsEndsWith<Name>>();
|
||||
@ -164,7 +164,7 @@ public:
|
||||
return selector.selectAndExecute(arguments, result_type, input_rows_count);
|
||||
}
|
||||
|
||||
static FunctionPtr create(ContextPtr context)
|
||||
static FunctionPtr create(ContextConstPtr context)
|
||||
{
|
||||
return std::make_shared<FunctionStartsEndsWith<Name>>(context);
|
||||
}
|
||||
|
@ -25,7 +25,7 @@ class FunctionStringOrArrayToT : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = Name::name;
|
||||
static FunctionPtr create(ContextPtr)
|
||||
static FunctionPtr create(ContextConstPtr)
|
||||
{
|
||||
return std::make_shared<FunctionStringOrArrayToT>();
|
||||
}
|
||||
|
@ -23,7 +23,7 @@ class FunctionStringReplace : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = Name::name;
|
||||
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionStringReplace>(); }
|
||||
static FunctionPtr create(ContextConstPtr) { return std::make_shared<FunctionStringReplace>(); }
|
||||
|
||||
String getName() const override { return name; }
|
||||
|
||||
|
@ -23,7 +23,7 @@ class FunctionStringToString : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = Name::name;
|
||||
static FunctionPtr create(ContextPtr)
|
||||
static FunctionPtr create(ContextConstPtr)
|
||||
{
|
||||
return std::make_shared<FunctionStringToString>();
|
||||
}
|
||||
|
@ -111,7 +111,7 @@ class FunctionUnaryArithmetic : public IFunction
|
||||
|
||||
public:
|
||||
static constexpr auto name = Name::name;
|
||||
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionUnaryArithmetic>(); }
|
||||
static FunctionPtr create(ContextConstPtr) { return std::make_shared<FunctionUnaryArithmetic>(); }
|
||||
|
||||
String getName() const override
|
||||
{
|
||||
|
@ -141,7 +141,7 @@ class FunctionEncrypt : public IFunction
|
||||
public:
|
||||
static constexpr OpenSSLDetails::CompatibilityMode compatibility_mode = Impl::compatibility_mode;
|
||||
static constexpr auto name = Impl::name;
|
||||
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionEncrypt>(); }
|
||||
static FunctionPtr create(ContextConstPtr) { return std::make_shared<FunctionEncrypt>(); }
|
||||
|
||||
private:
|
||||
using CipherMode = OpenSSLDetails::CipherMode;
|
||||
@ -416,7 +416,7 @@ class FunctionDecrypt : public IFunction
|
||||
public:
|
||||
static constexpr OpenSSLDetails::CompatibilityMode compatibility_mode = Impl::compatibility_mode;
|
||||
static constexpr auto name = Impl::name;
|
||||
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionDecrypt>(); }
|
||||
static FunctionPtr create(ContextConstPtr) { return std::make_shared<FunctionDecrypt>(); }
|
||||
|
||||
private:
|
||||
using CipherMode = OpenSSLDetails::CipherMode;
|
||||
|
@ -93,7 +93,7 @@ class FunctionBitmapBuildImpl : public IFunction
|
||||
public:
|
||||
static constexpr auto name = Name::name;
|
||||
|
||||
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionBitmapBuildImpl>(); }
|
||||
static FunctionPtr create(ContextConstPtr) { return std::make_shared<FunctionBitmapBuildImpl>(); }
|
||||
|
||||
String getName() const override { return name; }
|
||||
|
||||
@ -221,7 +221,7 @@ class FunctionBitmapToArrayImpl : public IFunction
|
||||
public:
|
||||
static constexpr auto name = Name::name;
|
||||
|
||||
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionBitmapToArrayImpl>(); }
|
||||
static FunctionPtr create(ContextConstPtr) { return std::make_shared<FunctionBitmapToArrayImpl>(); }
|
||||
|
||||
String getName() const override { return name; }
|
||||
|
||||
@ -311,7 +311,7 @@ class FunctionBitmapSubset : public IFunction
|
||||
public:
|
||||
static constexpr auto name = Impl::name;
|
||||
|
||||
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionBitmapSubset<Impl>>(); }
|
||||
static FunctionPtr create(ContextConstPtr) { return std::make_shared<FunctionBitmapSubset<Impl>>(); }
|
||||
|
||||
String getName() const override { return name; }
|
||||
|
||||
@ -469,7 +469,7 @@ class FunctionBitmapTransform : public IFunction
|
||||
public:
|
||||
static constexpr auto name = "bitmapTransform";
|
||||
|
||||
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionBitmapTransform>(); }
|
||||
static FunctionPtr create(ContextConstPtr) { return std::make_shared<FunctionBitmapTransform>(); }
|
||||
|
||||
String getName() const override { return name; }
|
||||
|
||||
@ -635,7 +635,7 @@ class FunctionBitmapSelfCardinalityImpl : public IFunction
|
||||
public:
|
||||
static constexpr auto name = Impl::name;
|
||||
|
||||
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionBitmapSelfCardinalityImpl<Impl>>(); }
|
||||
static FunctionPtr create(ContextConstPtr) { return std::make_shared<FunctionBitmapSelfCardinalityImpl<Impl>>(); }
|
||||
|
||||
String getName() const override { return name; }
|
||||
|
||||
@ -807,7 +807,7 @@ class FunctionBitmapContains : public IFunction
|
||||
public:
|
||||
static constexpr auto name = "bitmapContains";
|
||||
|
||||
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionBitmapContains>(); }
|
||||
static FunctionPtr create(ContextConstPtr) { return std::make_shared<FunctionBitmapContains>(); }
|
||||
|
||||
String getName() const override { return name; }
|
||||
|
||||
@ -911,7 +911,7 @@ class FunctionBitmapCardinality : public IFunction
|
||||
public:
|
||||
static constexpr auto name = Name::name;
|
||||
|
||||
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionBitmapCardinality>(); }
|
||||
static FunctionPtr create(ContextConstPtr) { return std::make_shared<FunctionBitmapCardinality>(); }
|
||||
|
||||
String getName() const override { return name; }
|
||||
|
||||
@ -1054,7 +1054,7 @@ class FunctionBitmap : public IFunction
|
||||
public:
|
||||
static constexpr auto name = Name::name;
|
||||
|
||||
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionBitmap>(); }
|
||||
static FunctionPtr create(ContextConstPtr) { return std::make_shared<FunctionBitmap>(); }
|
||||
|
||||
String getName() const override { return name; }
|
||||
|
||||
|
@ -69,7 +69,7 @@ class FunctionIPv6NumToString : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "IPv6NumToString";
|
||||
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionIPv6NumToString>(); }
|
||||
static FunctionPtr create(ContextConstPtr) { return std::make_shared<FunctionIPv6NumToString>(); }
|
||||
|
||||
String getName() const override { return name; }
|
||||
|
||||
@ -139,7 +139,7 @@ class FunctionCutIPv6 : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "cutIPv6";
|
||||
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionCutIPv6>(); }
|
||||
static FunctionPtr create(ContextConstPtr) { return std::make_shared<FunctionCutIPv6>(); }
|
||||
|
||||
String getName() const override { return name; }
|
||||
|
||||
@ -262,7 +262,7 @@ class FunctionIPv6StringToNum : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "IPv6StringToNum";
|
||||
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionIPv6StringToNum>(); }
|
||||
static FunctionPtr create(ContextConstPtr) { return std::make_shared<FunctionIPv6StringToNum>(); }
|
||||
|
||||
static inline bool tryParseIPv4(const char * pos)
|
||||
{
|
||||
@ -340,7 +340,7 @@ class FunctionIPv4NumToString : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = Name::name;
|
||||
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionIPv4NumToString<mask_tail_octets, Name>>(); }
|
||||
static FunctionPtr create(ContextConstPtr) { return std::make_shared<FunctionIPv4NumToString<mask_tail_octets, Name>>(); }
|
||||
|
||||
String getName() const override
|
||||
{
|
||||
@ -401,7 +401,7 @@ class FunctionIPv4StringToNum : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "IPv4StringToNum";
|
||||
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionIPv4StringToNum>(); }
|
||||
static FunctionPtr create(ContextConstPtr) { return std::make_shared<FunctionIPv4StringToNum>(); }
|
||||
|
||||
String getName() const override
|
||||
{
|
||||
@ -464,7 +464,7 @@ class FunctionIPv4ToIPv6 : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "IPv4ToIPv6";
|
||||
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionIPv4ToIPv6>(); }
|
||||
static FunctionPtr create(ContextConstPtr) { return std::make_shared<FunctionIPv4ToIPv6>(); }
|
||||
|
||||
String getName() const override { return name; }
|
||||
|
||||
@ -519,7 +519,7 @@ class FunctionToIPv4 : public FunctionIPv4StringToNum
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "toIPv4";
|
||||
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionToIPv4>(); }
|
||||
static FunctionPtr create(ContextConstPtr) { return std::make_shared<FunctionToIPv4>(); }
|
||||
|
||||
String getName() const override
|
||||
{
|
||||
@ -542,7 +542,7 @@ class FunctionToIPv6 : public FunctionIPv6StringToNum
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "toIPv6";
|
||||
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionToIPv6>(); }
|
||||
static FunctionPtr create(ContextConstPtr) { return std::make_shared<FunctionToIPv6>(); }
|
||||
|
||||
String getName() const override { return name; }
|
||||
|
||||
@ -560,7 +560,7 @@ class FunctionMACNumToString : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "MACNumToString";
|
||||
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionMACNumToString>(); }
|
||||
static FunctionPtr create(ContextConstPtr) { return std::make_shared<FunctionMACNumToString>(); }
|
||||
|
||||
String getName() const override
|
||||
{
|
||||
@ -690,7 +690,7 @@ class FunctionMACStringTo : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = Impl::name;
|
||||
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionMACStringTo<Impl>>(); }
|
||||
static FunctionPtr create(ContextConstPtr) { return std::make_shared<FunctionMACStringTo<Impl>>(); }
|
||||
|
||||
String getName() const override
|
||||
{
|
||||
@ -753,7 +753,7 @@ class FunctionUUIDNumToString : public IFunction
|
||||
|
||||
public:
|
||||
static constexpr auto name = "UUIDNumToString";
|
||||
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionUUIDNumToString>(); }
|
||||
static FunctionPtr create(ContextConstPtr) { return std::make_shared<FunctionUUIDNumToString>(); }
|
||||
|
||||
String getName() const override
|
||||
{
|
||||
@ -851,7 +851,7 @@ private:
|
||||
|
||||
public:
|
||||
static constexpr auto name = "UUIDStringToNum";
|
||||
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionUUIDStringToNum>(); }
|
||||
static FunctionPtr create(ContextConstPtr) { return std::make_shared<FunctionUUIDStringToNum>(); }
|
||||
|
||||
String getName() const override
|
||||
{
|
||||
@ -955,7 +955,7 @@ class FunctionHex : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "hex";
|
||||
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionHex>(); }
|
||||
static FunctionPtr create(ContextConstPtr) { return std::make_shared<FunctionHex>(); }
|
||||
|
||||
String getName() const override
|
||||
{
|
||||
@ -1238,7 +1238,7 @@ class FunctionUnhex : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "unhex";
|
||||
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionUnhex>(); }
|
||||
static FunctionPtr create(ContextConstPtr) { return std::make_shared<FunctionUnhex>(); }
|
||||
|
||||
String getName() const override
|
||||
{
|
||||
@ -1327,7 +1327,7 @@ class FunctionChar : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "char";
|
||||
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionChar>(); }
|
||||
static FunctionPtr create(ContextConstPtr) { return std::make_shared<FunctionChar>(); }
|
||||
|
||||
String getName() const override
|
||||
{
|
||||
@ -1422,7 +1422,7 @@ class FunctionBitmaskToArray : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "bitmaskToArray";
|
||||
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionBitmaskToArray>(); }
|
||||
static FunctionPtr create(ContextConstPtr) { return std::make_shared<FunctionBitmaskToArray>(); }
|
||||
|
||||
String getName() const override
|
||||
{
|
||||
@ -1508,7 +1508,7 @@ class FunctionToStringCutToZero : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "toStringCutToZero";
|
||||
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionToStringCutToZero>(); }
|
||||
static FunctionPtr create(ContextConstPtr) { return std::make_shared<FunctionToStringCutToZero>(); }
|
||||
|
||||
String getName() const override
|
||||
{
|
||||
@ -1673,7 +1673,7 @@ private:
|
||||
|
||||
public:
|
||||
static constexpr auto name = "IPv6CIDRToRange";
|
||||
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionIPv6CIDRToRange>(); }
|
||||
static FunctionPtr create(ContextConstPtr) { return std::make_shared<FunctionIPv6CIDRToRange>(); }
|
||||
|
||||
String getName() const override { return name; }
|
||||
size_t getNumberOfArguments() const override { return 2; }
|
||||
@ -1787,7 +1787,7 @@ private:
|
||||
|
||||
public:
|
||||
static constexpr auto name = "IPv4CIDRToRange";
|
||||
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionIPv4CIDRToRange>(); }
|
||||
static FunctionPtr create(ContextConstPtr) { return std::make_shared<FunctionIPv4CIDRToRange>(); }
|
||||
|
||||
String getName() const override { return name; }
|
||||
size_t getNumberOfArguments() const override { return 2; }
|
||||
@ -1868,7 +1868,7 @@ class FunctionIsIPv4String : public FunctionIPv4StringToNum
|
||||
public:
|
||||
static constexpr auto name = "isIPv4String";
|
||||
|
||||
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionIsIPv4String>(); }
|
||||
static FunctionPtr create(ContextConstPtr) { return std::make_shared<FunctionIsIPv4String>(); }
|
||||
|
||||
String getName() const override { return name; }
|
||||
|
||||
@ -1914,7 +1914,7 @@ class FunctionIsIPv6String : public FunctionIPv6StringToNum
|
||||
public:
|
||||
static constexpr auto name = "isIPv6String";
|
||||
|
||||
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionIsIPv6String>(); }
|
||||
static FunctionPtr create(ContextConstPtr) { return std::make_shared<FunctionIsIPv6String>(); }
|
||||
|
||||
String getName() const override { return name; }
|
||||
|
||||
|
@ -557,13 +557,13 @@ class FunctionComparison : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = Name::name;
|
||||
static FunctionPtr create(ContextPtr context) { return std::make_shared<FunctionComparison>(context); }
|
||||
static FunctionPtr create(ContextConstPtr context) { return std::make_shared<FunctionComparison>(context); }
|
||||
|
||||
explicit FunctionComparison(ContextPtr context_)
|
||||
explicit FunctionComparison(ContextConstPtr context_)
|
||||
: context(context_), check_decimal_overflow(decimalCheckComparisonOverflow(context)) {}
|
||||
|
||||
private:
|
||||
ContextPtr context;
|
||||
ContextConstPtr context;
|
||||
bool check_decimal_overflow = true;
|
||||
|
||||
template <typename T0, typename T1>
|
||||
|
@ -24,7 +24,7 @@ class FunctionConsistentHashImpl : public IFunction
|
||||
public:
|
||||
static constexpr auto name = Impl::name;
|
||||
|
||||
static FunctionPtr create(ContextPtr)
|
||||
static FunctionPtr create(ContextConstPtr)
|
||||
{
|
||||
return std::make_shared<FunctionConsistentHashImpl<Impl>>();
|
||||
}
|
||||
|
@ -1277,7 +1277,7 @@ public:
|
||||
static constexpr bool to_string_or_fixed_string = std::is_same_v<ToDataType, DataTypeFixedString> ||
|
||||
std::is_same_v<ToDataType, DataTypeString>;
|
||||
|
||||
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionConvert>(); }
|
||||
static FunctionPtr create(ContextConstPtr) { return std::make_shared<FunctionConvert>(); }
|
||||
static FunctionPtr create() { return std::make_shared<FunctionConvert>(); }
|
||||
|
||||
String getName() const override
|
||||
@ -1592,7 +1592,7 @@ public:
|
||||
|
||||
static constexpr bool to_datetime64 = std::is_same_v<ToDataType, DataTypeDateTime64>;
|
||||
|
||||
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionConvertFromString>(); }
|
||||
static FunctionPtr create(ContextConstPtr) { return std::make_shared<FunctionConvertFromString>(); }
|
||||
static FunctionPtr create() { return std::make_shared<FunctionConvertFromString>(); }
|
||||
|
||||
String getName() const override
|
||||
@ -3196,7 +3196,7 @@ public:
|
||||
? accurate_cast_name
|
||||
: (cast_type == CastType::accurateOrNull ? accurate_cast_or_null_name : cast_name);
|
||||
|
||||
static FunctionOverloadResolverPtr create(ContextPtr context)
|
||||
static FunctionOverloadResolverPtr create(ContextConstPtr context)
|
||||
{
|
||||
return createImpl(context->getSettingsRef().cast_keep_nullable);
|
||||
}
|
||||
|
@ -483,7 +483,7 @@ struct NameRegionIn { static constexpr auto name = "regionIn"
|
||||
struct FunctionRegionToCity :
|
||||
public FunctionTransformWithDictionary<UInt32, RegionToCityImpl, RegionsHierarchyGetter, NameRegionToCity>
|
||||
{
|
||||
static FunctionPtr create(ContextPtr context)
|
||||
static FunctionPtr create(ContextConstPtr context)
|
||||
{
|
||||
return std::make_shared<base_type>(context->getEmbeddedDictionaries().getRegionsHierarchies());
|
||||
}
|
||||
@ -492,7 +492,7 @@ struct FunctionRegionToCity :
|
||||
struct FunctionRegionToArea :
|
||||
public FunctionTransformWithDictionary<UInt32, RegionToAreaImpl, RegionsHierarchyGetter, NameRegionToArea>
|
||||
{
|
||||
static FunctionPtr create(ContextPtr context)
|
||||
static FunctionPtr create(ContextConstPtr context)
|
||||
{
|
||||
return std::make_shared<base_type>(context->getEmbeddedDictionaries().getRegionsHierarchies());
|
||||
}
|
||||
@ -501,7 +501,7 @@ struct FunctionRegionToArea :
|
||||
struct FunctionRegionToDistrict :
|
||||
public FunctionTransformWithDictionary<UInt32, RegionToDistrictImpl, RegionsHierarchyGetter, NameRegionToDistrict>
|
||||
{
|
||||
static FunctionPtr create(ContextPtr context)
|
||||
static FunctionPtr create(ContextConstPtr context)
|
||||
{
|
||||
return std::make_shared<base_type>(context->getEmbeddedDictionaries().getRegionsHierarchies());
|
||||
}
|
||||
@ -510,7 +510,7 @@ struct FunctionRegionToDistrict :
|
||||
struct FunctionRegionToCountry :
|
||||
public FunctionTransformWithDictionary<UInt32, RegionToCountryImpl, RegionsHierarchyGetter, NameRegionToCountry>
|
||||
{
|
||||
static FunctionPtr create(ContextPtr context)
|
||||
static FunctionPtr create(ContextConstPtr context)
|
||||
{
|
||||
return std::make_shared<base_type>(context->getEmbeddedDictionaries().getRegionsHierarchies());
|
||||
}
|
||||
@ -519,7 +519,7 @@ struct FunctionRegionToCountry :
|
||||
struct FunctionRegionToContinent :
|
||||
public FunctionTransformWithDictionary<UInt32, RegionToContinentImpl, RegionsHierarchyGetter, NameRegionToContinent>
|
||||
{
|
||||
static FunctionPtr create(ContextPtr context)
|
||||
static FunctionPtr create(ContextConstPtr context)
|
||||
{
|
||||
return std::make_shared<base_type>(context->getEmbeddedDictionaries().getRegionsHierarchies());
|
||||
}
|
||||
@ -528,7 +528,7 @@ struct FunctionRegionToContinent :
|
||||
struct FunctionRegionToTopContinent :
|
||||
public FunctionTransformWithDictionary<UInt32, RegionToTopContinentImpl, RegionsHierarchyGetter, NameRegionToTopContinent>
|
||||
{
|
||||
static FunctionPtr create(ContextPtr context)
|
||||
static FunctionPtr create(ContextConstPtr context)
|
||||
{
|
||||
return std::make_shared<base_type>(context->getEmbeddedDictionaries().getRegionsHierarchies());
|
||||
}
|
||||
@ -537,7 +537,7 @@ struct FunctionRegionToTopContinent :
|
||||
struct FunctionRegionToPopulation :
|
||||
public FunctionTransformWithDictionary<UInt32, RegionToPopulationImpl, RegionsHierarchyGetter, NameRegionToPopulation>
|
||||
{
|
||||
static FunctionPtr create(ContextPtr context)
|
||||
static FunctionPtr create(ContextConstPtr context)
|
||||
{
|
||||
return std::make_shared<base_type>(context->getEmbeddedDictionaries().getRegionsHierarchies());
|
||||
}
|
||||
@ -546,7 +546,7 @@ struct FunctionRegionToPopulation :
|
||||
struct FunctionRegionIn :
|
||||
public FunctionIsInWithDictionary<UInt32, RegionInImpl, RegionsHierarchyGetter, NameRegionIn>
|
||||
{
|
||||
static FunctionPtr create(ContextPtr context)
|
||||
static FunctionPtr create(ContextConstPtr context)
|
||||
{
|
||||
return std::make_shared<base_type>(context->getEmbeddedDictionaries().getRegionsHierarchies());
|
||||
}
|
||||
@ -555,7 +555,7 @@ struct FunctionRegionIn :
|
||||
struct FunctionRegionHierarchy :
|
||||
public FunctionHierarchyWithDictionary<UInt32, RegionHierarchyImpl, RegionsHierarchyGetter, NameRegionHierarchy>
|
||||
{
|
||||
static FunctionPtr create(ContextPtr context)
|
||||
static FunctionPtr create(ContextConstPtr context)
|
||||
{
|
||||
return std::make_shared<base_type>(context->getEmbeddedDictionaries().getRegionsHierarchies());
|
||||
}
|
||||
@ -567,7 +567,7 @@ class FunctionRegionToName : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "regionToName";
|
||||
static FunctionPtr create(ContextPtr context)
|
||||
static FunctionPtr create(ContextConstPtr context)
|
||||
{
|
||||
return std::make_shared<FunctionRegionToName>(context->getEmbeddedDictionaries().getRegionsNames());
|
||||
}
|
||||
|
@ -62,10 +62,10 @@ namespace ErrorCodes
|
||||
*/
|
||||
|
||||
|
||||
class FunctionDictHelper : WithContext
|
||||
class FunctionDictHelper : WithConstContext
|
||||
{
|
||||
public:
|
||||
explicit FunctionDictHelper(ContextPtr context_) : WithContext(context_) {}
|
||||
explicit FunctionDictHelper(ContextConstPtr context_) : WithConstContext(context_) {}
|
||||
|
||||
std::shared_ptr<const IDictionary> getDictionary(const String & dictionary_name)
|
||||
{
|
||||
@ -132,12 +132,12 @@ class FunctionDictHas final : public IFunction
|
||||
public:
|
||||
static constexpr auto name = "dictHas";
|
||||
|
||||
static FunctionPtr create(ContextPtr context)
|
||||
static FunctionPtr create(ContextConstPtr context)
|
||||
{
|
||||
return std::make_shared<FunctionDictHas>(context);
|
||||
}
|
||||
|
||||
explicit FunctionDictHas(ContextPtr context_) : helper(context_) {}
|
||||
explicit FunctionDictHas(ContextConstPtr context_) : helper(context_) {}
|
||||
|
||||
String getName() const override { return name; }
|
||||
|
||||
@ -270,12 +270,12 @@ class FunctionDictGetNoType final : public IFunction
|
||||
public:
|
||||
static constexpr auto name = dictionary_get_function_type == DictionaryGetFunctionType::get ? "dictGet" : "dictGetOrDefault";
|
||||
|
||||
static FunctionPtr create(ContextPtr context)
|
||||
static FunctionPtr create(ContextConstPtr context)
|
||||
{
|
||||
return std::make_shared<FunctionDictGetNoType>(context);
|
||||
}
|
||||
|
||||
explicit FunctionDictGetNoType(ContextPtr context_) : helper(context_) {}
|
||||
explicit FunctionDictGetNoType(ContextConstPtr context_) : helper(context_) {}
|
||||
|
||||
String getName() const override { return name; }
|
||||
|
||||
@ -604,12 +604,12 @@ class FunctionDictGetImpl final : public IFunction
|
||||
public:
|
||||
static constexpr auto name = Name::name;
|
||||
|
||||
static FunctionPtr create(ContextPtr context)
|
||||
static FunctionPtr create(ContextConstPtr context)
|
||||
{
|
||||
return std::make_shared<FunctionDictGetImpl>(context);
|
||||
}
|
||||
|
||||
explicit FunctionDictGetImpl(ContextPtr context_) : impl(context_) {}
|
||||
explicit FunctionDictGetImpl(ContextConstPtr context_) : impl(context_) {}
|
||||
|
||||
String getName() const override { return name; }
|
||||
|
||||
@ -743,12 +743,12 @@ class FunctionDictGetOrNull final : public IFunction
|
||||
public:
|
||||
static constexpr auto name = "dictGetOrNull";
|
||||
|
||||
static FunctionPtr create(ContextPtr context)
|
||||
static FunctionPtr create(ContextConstPtr context)
|
||||
{
|
||||
return std::make_shared<FunctionDictGetOrNull>(context);
|
||||
}
|
||||
|
||||
explicit FunctionDictGetOrNull(ContextPtr context_)
|
||||
explicit FunctionDictGetOrNull(ContextConstPtr context_)
|
||||
: dictionary_get_func_impl(context_)
|
||||
, dictionary_has_func_impl(context_)
|
||||
{}
|
||||
@ -906,12 +906,12 @@ class FunctionDictGetHierarchy final : public IFunction
|
||||
public:
|
||||
static constexpr auto name = "dictGetHierarchy";
|
||||
|
||||
static FunctionPtr create(ContextPtr context)
|
||||
static FunctionPtr create(ContextConstPtr context)
|
||||
{
|
||||
return std::make_shared<FunctionDictGetHierarchy>(context);
|
||||
}
|
||||
|
||||
explicit FunctionDictGetHierarchy(ContextPtr context_) : helper(context_) {}
|
||||
explicit FunctionDictGetHierarchy(ContextConstPtr context_) : helper(context_) {}
|
||||
|
||||
String getName() const override { return name; }
|
||||
|
||||
@ -966,12 +966,12 @@ class FunctionDictIsIn final : public IFunction
|
||||
public:
|
||||
static constexpr auto name = "dictIsIn";
|
||||
|
||||
static FunctionPtr create(ContextPtr context)
|
||||
static FunctionPtr create(ContextConstPtr context)
|
||||
{
|
||||
return std::make_shared<FunctionDictIsIn>(context);
|
||||
}
|
||||
|
||||
explicit FunctionDictIsIn(ContextPtr context_)
|
||||
explicit FunctionDictIsIn(ContextConstPtr context_)
|
||||
: helper(context_) {}
|
||||
|
||||
String getName() const override { return name; }
|
||||
@ -1032,12 +1032,12 @@ class FunctionDictGetChildren final : public IFunction
|
||||
public:
|
||||
static constexpr auto name = "dictGetChildren";
|
||||
|
||||
static FunctionPtr create(ContextPtr context)
|
||||
static FunctionPtr create(ContextConstPtr context)
|
||||
{
|
||||
return std::make_shared<FunctionDictGetChildren>(context);
|
||||
}
|
||||
|
||||
explicit FunctionDictGetChildren(ContextPtr context_)
|
||||
explicit FunctionDictGetChildren(ContextConstPtr context_)
|
||||
: helper(context_) {}
|
||||
|
||||
String getName() const override { return name; }
|
||||
@ -1091,12 +1091,12 @@ class FunctionDictGetDescendants final : public IFunction
|
||||
public:
|
||||
static constexpr auto name = "dictGetDescendants";
|
||||
|
||||
static FunctionPtr create(ContextPtr context)
|
||||
static FunctionPtr create(ContextConstPtr context)
|
||||
{
|
||||
return std::make_shared<FunctionDictGetDescendants>(context);
|
||||
}
|
||||
|
||||
explicit FunctionDictGetDescendants(ContextPtr context_)
|
||||
explicit FunctionDictGetDescendants(ContextConstPtr context_)
|
||||
: helper(context_) {}
|
||||
|
||||
String getName() const override { return name; }
|
||||
|
@ -18,7 +18,7 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
FunctionPtr FunctionModelEvaluate::create(ContextPtr context)
|
||||
FunctionPtr FunctionModelEvaluate::create(ContextConstPtr context)
|
||||
{
|
||||
return std::make_shared<FunctionModelEvaluate>(context->getExternalModelsLoader());
|
||||
}
|
||||
|
@ -17,7 +17,7 @@ class FunctionModelEvaluate final : public IFunction
|
||||
public:
|
||||
static constexpr auto name = "modelEvaluate";
|
||||
|
||||
static FunctionPtr create(ContextPtr context);
|
||||
static FunctionPtr create(ContextConstPtr context);
|
||||
|
||||
explicit FunctionModelEvaluate(const ExternalModelsLoader & models_loader_) : models_loader(models_loader_) {}
|
||||
|
||||
|
@ -535,7 +535,7 @@ class FunctionStringHashFixedString : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = Impl::name;
|
||||
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionStringHashFixedString>(); }
|
||||
static FunctionPtr create(ContextConstPtr) { return std::make_shared<FunctionStringHashFixedString>(); }
|
||||
|
||||
String getName() const override
|
||||
{
|
||||
@ -700,7 +700,7 @@ template <typename Impl, typename Name>
|
||||
class FunctionIntHash : public TargetSpecific::Default::FunctionIntHash<Impl, Name>
|
||||
{
|
||||
public:
|
||||
explicit FunctionIntHash(ContextPtr context) : selector(context)
|
||||
explicit FunctionIntHash(ContextConstPtr context) : selector(context)
|
||||
{
|
||||
selector.registerImplementation<TargetArch::Default,
|
||||
TargetSpecific::Default::FunctionIntHash<Impl, Name>>();
|
||||
@ -718,7 +718,7 @@ public:
|
||||
return selector.selectAndExecute(arguments, result_type, input_rows_count);
|
||||
}
|
||||
|
||||
static FunctionPtr create(ContextPtr context)
|
||||
static FunctionPtr create(ContextConstPtr context)
|
||||
{
|
||||
return std::make_shared<FunctionIntHash>(context);
|
||||
}
|
||||
@ -1077,7 +1077,7 @@ template <typename Impl>
|
||||
class FunctionAnyHash : public TargetSpecific::Default::FunctionAnyHash<Impl>
|
||||
{
|
||||
public:
|
||||
explicit FunctionAnyHash(ContextPtr context) : selector(context)
|
||||
explicit FunctionAnyHash(ContextConstPtr context) : selector(context)
|
||||
{
|
||||
selector.registerImplementation<TargetArch::Default,
|
||||
TargetSpecific::Default::FunctionAnyHash<Impl>>();
|
||||
@ -1095,7 +1095,7 @@ public:
|
||||
return selector.selectAndExecute(arguments, result_type, input_rows_count);
|
||||
}
|
||||
|
||||
static FunctionPtr create(ContextPtr context)
|
||||
static FunctionPtr create(ContextConstPtr context)
|
||||
{
|
||||
return std::make_shared<FunctionAnyHash>(context);
|
||||
}
|
||||
@ -1182,7 +1182,7 @@ class FunctionURLHash : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "URLHash";
|
||||
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionURLHash>(); }
|
||||
static FunctionPtr create(ContextConstPtr) { return std::make_shared<FunctionURLHash>(); }
|
||||
|
||||
String getName() const override { return name; }
|
||||
|
||||
|
@ -270,11 +270,11 @@ private:
|
||||
|
||||
|
||||
template <typename Name, template<typename> typename Impl>
|
||||
class FunctionJSON : public IFunction, WithContext
|
||||
class FunctionJSON : public IFunction, WithConstContext
|
||||
{
|
||||
public:
|
||||
static FunctionPtr create(ContextPtr context_) { return std::make_shared<FunctionJSON>(context_); }
|
||||
FunctionJSON(ContextPtr context_) : WithContext(context_) {}
|
||||
static FunctionPtr create(ContextConstPtr context_) { return std::make_shared<FunctionJSON>(context_); }
|
||||
FunctionJSON(ContextConstPtr context_) : WithConstContext(context_) {}
|
||||
|
||||
static constexpr auto name = Name::name;
|
||||
String getName() const override { return Name::name; }
|
||||
|
@ -139,7 +139,7 @@ class FunctionAnyArityLogical : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = Name::name;
|
||||
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionAnyArityLogical>(); }
|
||||
static FunctionPtr create(ContextConstPtr) { return std::make_shared<FunctionAnyArityLogical>(); }
|
||||
|
||||
public:
|
||||
String getName() const override
|
||||
@ -206,7 +206,7 @@ class FunctionUnaryLogical : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = Name::name;
|
||||
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionUnaryLogical>(); }
|
||||
static FunctionPtr create(ContextConstPtr) { return std::make_shared<FunctionUnaryLogical>(); }
|
||||
|
||||
public:
|
||||
String getName() const override
|
||||
|
@ -34,7 +34,7 @@ class FunctionsMultiStringFuzzySearch : public IFunction
|
||||
|
||||
public:
|
||||
static constexpr auto name = Name::name;
|
||||
static FunctionPtr create(ContextPtr context)
|
||||
static FunctionPtr create(ContextConstPtr context)
|
||||
{
|
||||
if (Impl::is_using_hyperscan && !context->getSettingsRef().allow_hyperscan)
|
||||
throw Exception(
|
||||
|
@ -51,7 +51,7 @@ class FunctionsMultiStringPosition : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = Name::name;
|
||||
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionsMultiStringPosition>(); }
|
||||
static FunctionPtr create(ContextConstPtr) { return std::make_shared<FunctionsMultiStringPosition>(); }
|
||||
|
||||
String getName() const override { return name; }
|
||||
|
||||
|
@ -47,7 +47,7 @@ class FunctionsMultiStringSearch : public IFunction
|
||||
|
||||
public:
|
||||
static constexpr auto name = Name::name;
|
||||
static FunctionPtr create(ContextPtr context)
|
||||
static FunctionPtr create(ContextConstPtr context)
|
||||
{
|
||||
if (Impl::is_using_hyperscan && !context->getSettingsRef().allow_hyperscan)
|
||||
throw Exception(
|
||||
|
@ -91,7 +91,7 @@ template <typename ToType, typename Name>
|
||||
class FunctionRandom : public FunctionRandomImpl<TargetSpecific::Default::RandImpl, ToType, Name>
|
||||
{
|
||||
public:
|
||||
explicit FunctionRandom(ContextPtr context) : selector(context)
|
||||
explicit FunctionRandom(ContextConstPtr context) : selector(context)
|
||||
{
|
||||
selector.registerImplementation<TargetArch::Default,
|
||||
FunctionRandomImpl<TargetSpecific::Default::RandImpl, ToType, Name>>();
|
||||
@ -107,7 +107,7 @@ public:
|
||||
return selector.selectAndExecute(arguments, result_type, input_rows_count);
|
||||
}
|
||||
|
||||
static FunctionPtr create(ContextPtr context)
|
||||
static FunctionPtr create(ContextConstPtr context)
|
||||
{
|
||||
return std::make_shared<FunctionRandom<ToType, Name>>(context);
|
||||
}
|
||||
|
@ -520,7 +520,7 @@ class FunctionRounding : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = Name::name;
|
||||
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionRounding>(); }
|
||||
static FunctionPtr create(ContextConstPtr) { return std::make_shared<FunctionRounding>(); }
|
||||
|
||||
String getName() const override
|
||||
{
|
||||
@ -629,7 +629,7 @@ class FunctionRoundDown : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "roundDown";
|
||||
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionRoundDown>(); }
|
||||
static FunctionPtr create(ContextConstPtr) { return std::make_shared<FunctionRoundDown>(); }
|
||||
|
||||
String getName() const override { return name; }
|
||||
|
||||
|
@ -427,7 +427,7 @@ class FunctionTokens : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = Generator::name;
|
||||
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionTokens>(); }
|
||||
static FunctionPtr create(ContextConstPtr) { return std::make_shared<FunctionTokens>(); }
|
||||
|
||||
String getName() const override
|
||||
{
|
||||
@ -590,7 +590,7 @@ private:
|
||||
|
||||
public:
|
||||
static constexpr auto name = "arrayStringConcat";
|
||||
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionArrayStringConcat>(); }
|
||||
static FunctionPtr create(ContextConstPtr) { return std::make_shared<FunctionArrayStringConcat>(); }
|
||||
|
||||
String getName() const override
|
||||
{
|
||||
|
@ -35,7 +35,7 @@ public:
|
||||
static constexpr size_t max_shingle_size = 25;
|
||||
static constexpr size_t max_num_hashes = 25;
|
||||
|
||||
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionsStringHash>(); }
|
||||
static FunctionPtr create(ContextConstPtr) { return std::make_shared<FunctionsStringHash>(); }
|
||||
|
||||
String getName() const override { return name; }
|
||||
|
||||
|
@ -51,7 +51,7 @@ class FunctionsStringSearch : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = Name::name;
|
||||
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionsStringSearch>(); }
|
||||
static FunctionPtr create(ContextConstPtr) { return std::make_shared<FunctionsStringSearch>(); }
|
||||
|
||||
String getName() const override { return name; }
|
||||
|
||||
|
@ -36,7 +36,7 @@ class FunctionsStringSearchToString : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = Name::name;
|
||||
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionsStringSearchToString>(); }
|
||||
static FunctionPtr create(ContextConstPtr) { return std::make_shared<FunctionsStringSearchToString>(); }
|
||||
|
||||
String getName() const override { return name; }
|
||||
|
||||
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue
Block a user