Merged with master

This commit is contained in:
Vxider 2021-05-29 14:55:12 +00:00
commit d4b0532b78
345 changed files with 1170 additions and 689 deletions

View File

@ -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 ||:

View File

@ -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 -->

View File

@ -1,4 +1,8 @@
## rankCorr {#agg_function-rankcorr}
---
toc_priority: 145
---
# rankCorr {#agg_function-rankcorr}
Computes a rank correlation coefficient.

View File

@ -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.

View File

@ -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}

View File

@ -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-->

View File

@ -1,4 +1,8 @@
## rankCorr {#agg_function-rankcorr}
---
toc_priority: 145
---
# rankCorr {#agg_function-rankcorr}
Вычисляет коэффициент ранговой корреляции.

View File

@ -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)

View File

@ -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).

View File

@ -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();

View File

@ -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())
{

View File

@ -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(

View File

@ -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);
}

View File

@ -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;

View File

@ -21,7 +21,7 @@ namespace DB
{
LibraryBridgeHelper::LibraryBridgeHelper(
ContextPtr context_,
ContextConstPtr context_,
const Block & sample_block_,
const Field & dictionary_id_)
: IBridgeHelper(context_->getGlobalContext())

View File

@ -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);

View File

@ -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())

View File

@ -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());

View File

@ -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
{

View File

@ -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_)

View File

@ -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.");

View File

@ -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;
}

View File

@ -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)
{

View File

@ -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
{

View File

@ -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
{

View File

@ -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);

View File

@ -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);

View File

@ -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;

View File

@ -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
{

View File

@ -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;

View File

@ -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);

View File

@ -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.

View File

@ -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";

View File

@ -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
{

View File

@ -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;
};
}

View File

@ -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
{

View File

@ -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;
};

View File

@ -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
{

View File

@ -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;
};

View File

@ -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)

View File

@ -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)

View File

@ -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;
};

View File

@ -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);
}

View File

@ -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)

View File

@ -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
{

View File

@ -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;

View File

@ -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 */)
{

View File

@ -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

View File

@ -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", "");

View File

@ -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
{

View File

@ -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)

View File

@ -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);

View File

@ -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,

View File

@ -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

View File

@ -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);

View File

@ -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_ = "");
}

View File

@ -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);
};

View File

@ -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>();
}

View File

@ -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;
};
}

View File

@ -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; }

View File

@ -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; }

View File

@ -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
{

View File

@ -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
{

View File

@ -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>();
}

View File

@ -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;

View File

@ -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));
}

View File

@ -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; }

View File

@ -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);

View File

@ -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; }

View File

@ -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; }

View File

@ -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; }

View File

@ -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; }

View File

@ -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>();
}

View File

@ -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);
}

View File

@ -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>();
}

View File

@ -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; }

View File

@ -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>();
}

View File

@ -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
{

View File

@ -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;

View File

@ -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; }

View File

@ -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; }

View File

@ -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>

View File

@ -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>>();
}

View File

@ -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);
}

View File

@ -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());
}

View File

@ -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; }

View File

@ -18,7 +18,7 @@
namespace DB
{
FunctionPtr FunctionModelEvaluate::create(ContextPtr context)
FunctionPtr FunctionModelEvaluate::create(ContextConstPtr context)
{
return std::make_shared<FunctionModelEvaluate>(context->getExternalModelsLoader());
}

View File

@ -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_) {}

View File

@ -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; }

View File

@ -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; }

View File

@ -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

View File

@ -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(

View File

@ -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; }

View File

@ -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(

View File

@ -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);
}

View File

@ -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; }

View File

@ -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
{

View File

@ -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; }

View File

@ -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; }

View File

@ -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