Merge branch 'master' of github.com:ClickHouse/ClickHouse into Avogar-patch-6

This commit is contained in:
avogar 2024-07-18 10:32:02 +00:00
commit bd29313a15
224 changed files with 880 additions and 640 deletions

View File

@ -26,7 +26,10 @@ RUN apt-get update \
zstd \
--yes --no-install-recommends \
&& apt-get clean \
&& rm -rf /var/lib/apt/lists/* /var/cache/debconf /tmp/*
&& rm -rf /var/lib/apt/lists/* /var/cache/debconf /tmp/* \
&& groupadd --system --gid 1000 clickhouse \
&& useradd --system --gid 1000 --uid 1000 -m clickhouse
# ^ For some reason, groupadd and useradd are needed for tests with 'expect', but I don't know, why.
COPY requirements.txt /
RUN pip3 install --no-cache-dir -r /requirements.txt

View File

@ -9,7 +9,7 @@ trap 'kill $(jobs -pr) ||:' EXIT
stage=${stage:-}
# Compiler version, normally set by Dockerfile
export LLVM_VERSION=${LLVM_VERSION:-17}
export LLVM_VERSION=${LLVM_VERSION:-18}
# A variable to pass additional flags to CMake.
# Here we explicitly default it to nothing so that bash doesn't complain about

View File

@ -17,6 +17,7 @@ ENV DEBIAN_FRONTEND=noninteractive LLVM_VERSION=18
RUN apt-get update \
&& apt-get install \
sudo \
apt-transport-https \
apt-utils \
ca-certificates \

View File

@ -185,6 +185,7 @@ You can pass parameters to `clickhouse-client` (all parameters have a default va
- `--format, -f` Use the specified default format to output the result.
- `--vertical, -E` If specified, use the [Vertical format](../interfaces/formats.md#vertical) by default to output the result. This is the same as `format=Vertical`. In this format, each value is printed on a separate line, which is helpful when displaying wide tables.
- `--time, -t` If specified, print the query execution time to stderr in non-interactive mode.
- `--memory-usage` If specified, print memory usage to stderr in non-interactive mode]. Possible values: 'none' - do not print memory usage, 'default' - print number of bytes, 'readable' - print memory usage in human-readable format.
- `--stacktrace` If specified, also print the stack trace if an exception occurs.
- `--config-file` The name of the configuration file.
- `--secure` If specified, will connect to server over secure connection (TLS). You might need to configure your CA certificates in the [configuration file](#configuration_files). The available configuration settings are the same as for [server-side TLS configuration](../operations/server-configuration-parameters/settings.md#server_configuration_parameters-openssl).
@ -339,7 +340,7 @@ clickhouse-client clickhouse://some_user%40some_mail.com@localhost:9000
Connect to one of provides hosts: `192.168.1.15`, `192.168.1.25`.
``` bash
clickhouse-client clickhouse://192.168.1.15,192.168.1.25
clickhouse-client clickhouse://192.168.1.15,192.168.1.25
```
### Configuration Files {#configuration_files}
@ -367,7 +368,7 @@ Example of a config file:
```
Or the same config in a YAML format:
```yaml
user: username
password: 'password'

View File

@ -67,6 +67,7 @@ The supported formats are:
| [Prometheus](#prometheus) | ✗ | ✔ |
| [Protobuf](#protobuf) | ✔ | ✔ |
| [ProtobufSingle](#protobufsingle) | ✔ | ✔ |
| [ProtobufList](#protobuflist) | ✔ | ✔ |
| [Avro](#data-format-avro) | ✔ | ✔ |
| [AvroConfluent](#data-format-avro-confluent) | ✔ | ✗ |
| [Parquet](#data-format-parquet) | ✔ | ✔ |
@ -1952,6 +1953,35 @@ SYSTEM DROP FORMAT SCHEMA CACHE FOR Protobuf
Same as [Protobuf](#protobuf) but for storing/parsing single Protobuf message without length delimiters.
## ProtobufList {#protobuflist}
Similar to Protobuf but rows are represented as a sequence of sub-messages contained in a message with fixed name "Envelope".
Usage example:
``` sql
SELECT * FROM test.table FORMAT ProtobufList SETTINGS format_schema = 'schemafile:MessageType'
```
``` bash
cat protobuflist_messages.bin | clickhouse-client --query "INSERT INTO test.table FORMAT ProtobufList SETTINGS format_schema='schemafile:MessageType'"
```
where the file `schemafile.proto` looks like this:
``` capnp
syntax = "proto3";
message Envelope {
message MessageType {
string name = 1;
string surname = 2;
uint32 birthDate = 3;
repeated string phoneNumbers = 4;
};
MessageType row = 1;
};
```
## Avro {#data-format-avro}
[Apache Avro](https://avro.apache.org/) is a row-oriented data serialization framework developed within Apaches Hadoop project.

View File

@ -76,7 +76,7 @@ WHERE macro = 'test';
└───────┴──────────────┘
```
## FQDN
## fqdn
Returns the fully qualified domain name of the ClickHouse server.
@ -86,7 +86,7 @@ Returns the fully qualified domain name of the ClickHouse server.
fqdn();
```
Aliases: `fullHostName`, 'FQDN'.
Aliases: `fullHostName`, `FQDN`.
**Returned value**

View File

@ -423,6 +423,7 @@ void LocalServer::connect()
{
connection_parameters = ConnectionParameters(getClientConfiguration(), "localhost");
/// This is needed for table function input(...).
ReadBuffer * in;
auto table_file = getClientConfiguration().getString("table-file", "-");
if (table_file == "-" || table_file == "stdin")

View File

@ -2731,8 +2731,7 @@ void Server::createInterserverServers(
void Server::stopServers(
std::vector<ProtocolServerAdapter> & servers,
const ServerType & server_type
) const
const ServerType & server_type) const
{
LoggerRawPtr log = &logger();

View File

@ -129,8 +129,7 @@ private:
void stopServers(
std::vector<ProtocolServerAdapter> & servers,
const ServerType & server_type
) const;
const ServerType & server_type) const;
};
}

View File

@ -118,10 +118,10 @@ AggregateFunctionPtr createAggregateFunctionAnalysisOfVariance(const std::string
void registerAggregateFunctionAnalysisOfVariance(AggregateFunctionFactory & factory)
{
AggregateFunctionProperties properties = { .is_order_dependent = false };
factory.registerFunction("analysisOfVariance", {createAggregateFunctionAnalysisOfVariance, properties}, AggregateFunctionFactory::CaseInsensitive);
factory.registerFunction("analysisOfVariance", {createAggregateFunctionAnalysisOfVariance, properties}, AggregateFunctionFactory::Case::Insensitive);
/// This is widely used term
factory.registerAlias("anova", "analysisOfVariance", AggregateFunctionFactory::CaseInsensitive);
factory.registerAlias("anova", "analysisOfVariance", AggregateFunctionFactory::Case::Insensitive);
}
}

View File

@ -361,9 +361,9 @@ void registerAggregateFunctionsAny(AggregateFunctionFactory & factory)
AggregateFunctionProperties default_properties = {.returns_default_when_only_null = false, .is_order_dependent = true};
factory.registerFunction("any", {createAggregateFunctionAny, default_properties});
factory.registerAlias("any_value", "any", AggregateFunctionFactory::CaseInsensitive);
factory.registerAlias("first_value", "any", AggregateFunctionFactory::CaseInsensitive);
factory.registerAlias("any_value", "any", AggregateFunctionFactory::Case::Insensitive);
factory.registerAlias("first_value", "any", AggregateFunctionFactory::Case::Insensitive);
factory.registerFunction("anyLast", {createAggregateFunctionAnyLast, default_properties});
factory.registerAlias("last_value", "anyLast", AggregateFunctionFactory::CaseInsensitive);
factory.registerAlias("last_value", "anyLast", AggregateFunctionFactory::Case::Insensitive);
}
}

View File

@ -221,11 +221,11 @@ void registerAggregateFunctionsAnyRespectNulls(AggregateFunctionFactory & factor
= {.returns_default_when_only_null = false, .is_order_dependent = true, .is_window_function = true};
factory.registerFunction("any_respect_nulls", {createAggregateFunctionAnyRespectNulls, default_properties_for_respect_nulls});
factory.registerAlias("any_value_respect_nulls", "any_respect_nulls", AggregateFunctionFactory::CaseInsensitive);
factory.registerAlias("first_value_respect_nulls", "any_respect_nulls", AggregateFunctionFactory::CaseInsensitive);
factory.registerAlias("any_value_respect_nulls", "any_respect_nulls", AggregateFunctionFactory::Case::Insensitive);
factory.registerAlias("first_value_respect_nulls", "any_respect_nulls", AggregateFunctionFactory::Case::Insensitive);
factory.registerFunction("anyLast_respect_nulls", {createAggregateFunctionAnyLastRespectNulls, default_properties_for_respect_nulls});
factory.registerAlias("last_value_respect_nulls", "anyLast_respect_nulls", AggregateFunctionFactory::CaseInsensitive);
factory.registerAlias("last_value_respect_nulls", "anyLast_respect_nulls", AggregateFunctionFactory::Case::Insensitive);
/// Must happen after registering any and anyLast
factory.registerNullsActionTransformation("any", "any_respect_nulls");

View File

@ -46,6 +46,6 @@ AggregateFunctionPtr createAggregateFunctionAvg(const std::string & name, const
void registerAggregateFunctionAvg(AggregateFunctionFactory & factory)
{
factory.registerFunction("avg", createAggregateFunctionAvg, AggregateFunctionFactory::CaseInsensitive);
factory.registerFunction("avg", createAggregateFunctionAvg, AggregateFunctionFactory::Case::Insensitive);
}
}

View File

@ -234,9 +234,9 @@ void registerAggregateFunctionsBitwise(AggregateFunctionFactory & factory)
factory.registerFunction("groupBitXor", createAggregateFunctionBitwise<AggregateFunctionGroupBitXorData>);
/// Aliases for compatibility with MySQL.
factory.registerAlias("BIT_OR", "groupBitOr", AggregateFunctionFactory::CaseInsensitive);
factory.registerAlias("BIT_AND", "groupBitAnd", AggregateFunctionFactory::CaseInsensitive);
factory.registerAlias("BIT_XOR", "groupBitXor", AggregateFunctionFactory::CaseInsensitive);
factory.registerAlias("BIT_OR", "groupBitOr", AggregateFunctionFactory::Case::Insensitive);
factory.registerAlias("BIT_AND", "groupBitAnd", AggregateFunctionFactory::Case::Insensitive);
factory.registerAlias("BIT_XOR", "groupBitXor", AggregateFunctionFactory::Case::Insensitive);
}
}

View File

@ -9,7 +9,7 @@ template <typename T1, typename T2> using AggregateFunctionCorr = AggregateFunct
void registerAggregateFunctionsStatisticsCorr(AggregateFunctionFactory & factory)
{
factory.registerFunction("corr", createAggregateFunctionStatisticsBinary<AggregateFunctionCorr, StatisticsFunctionKind::corr>, AggregateFunctionFactory::CaseInsensitive);
factory.registerFunction("corr", createAggregateFunctionStatisticsBinary<AggregateFunctionCorr, StatisticsFunctionKind::corr>, AggregateFunctionFactory::Case::Insensitive);
}
}

View File

@ -37,7 +37,7 @@ AggregateFunctionPtr createAggregateFunctionCount(const std::string & name, cons
void registerAggregateFunctionCount(AggregateFunctionFactory & factory)
{
AggregateFunctionProperties properties = { .returns_default_when_only_null = true, .is_order_dependent = false };
factory.registerFunction("count", {createAggregateFunctionCount, properties}, AggregateFunctionFactory::CaseInsensitive);
factory.registerFunction("count", {createAggregateFunctionCount, properties}, AggregateFunctionFactory::Case::Insensitive);
}
}

View File

@ -13,8 +13,8 @@ void registerAggregateFunctionsStatisticsCovar(AggregateFunctionFactory & factor
factory.registerFunction("covarPop", createAggregateFunctionStatisticsBinary<AggregateFunctionCovar, StatisticsFunctionKind::covarPop>);
/// Synonyms for compatibility.
factory.registerAlias("COVAR_SAMP", "covarSamp", AggregateFunctionFactory::CaseInsensitive);
factory.registerAlias("COVAR_POP", "covarPop", AggregateFunctionFactory::CaseInsensitive);
factory.registerAlias("COVAR_SAMP", "covarSamp", AggregateFunctionFactory::Case::Insensitive);
factory.registerAlias("COVAR_POP", "covarPop", AggregateFunctionFactory::Case::Insensitive);
}
}

View File

@ -29,7 +29,7 @@ const String & getAggregateFunctionCanonicalNameIfAny(const String & name)
return AggregateFunctionFactory::instance().getCanonicalNameIfAny(name);
}
void AggregateFunctionFactory::registerFunction(const String & name, Value creator_with_properties, CaseSensitiveness case_sensitiveness)
void AggregateFunctionFactory::registerFunction(const String & name, Value creator_with_properties, Case case_sensitiveness)
{
if (creator_with_properties.creator == nullptr)
throw Exception(ErrorCodes::LOGICAL_ERROR, "AggregateFunctionFactory: "
@ -39,7 +39,7 @@ void AggregateFunctionFactory::registerFunction(const String & name, Value creat
throw Exception(ErrorCodes::LOGICAL_ERROR, "AggregateFunctionFactory: the aggregate function name '{}' is not unique",
name);
if (case_sensitiveness == CaseInsensitive)
if (case_sensitiveness == Case::Insensitive)
{
auto key = Poco::toLower(name);
if (!case_insensitive_aggregate_functions.emplace(key, creator_with_properties).second)

View File

@ -60,7 +60,7 @@ public:
void registerFunction(
const String & name,
Value creator,
CaseSensitiveness case_sensitiveness = CaseSensitive);
Case case_sensitiveness = Case::Sensitive);
/// Register how to transform from one aggregate function to other based on NullsAction
/// Registers them both ways:

View File

@ -840,8 +840,8 @@ void registerAggregateFunctionGroupArray(AggregateFunctionFactory & factory)
AggregateFunctionProperties properties = { .returns_default_when_only_null = false, .is_order_dependent = true };
factory.registerFunction("groupArray", { createAggregateFunctionGroupArray<false>, properties });
factory.registerAlias("array_agg", "groupArray", AggregateFunctionFactory::CaseInsensitive);
factory.registerAliasUnchecked("array_concat_agg", "groupArrayArray", AggregateFunctionFactory::CaseInsensitive);
factory.registerAlias("array_agg", "groupArray", AggregateFunctionFactory::Case::Insensitive);
factory.registerAliasUnchecked("array_concat_agg", "groupArrayArray", AggregateFunctionFactory::Case::Insensitive);
factory.registerFunction("groupArraySample", { createAggregateFunctionGroupArraySample, properties });
factory.registerFunction("groupArrayLast", { createAggregateFunctionGroupArray<true>, properties });
}

View File

@ -277,7 +277,7 @@ void registerAggregateFunctionGroupConcat(AggregateFunctionFactory & factory)
AggregateFunctionProperties properties = { .returns_default_when_only_null = false, .is_order_dependent = true };
factory.registerFunction("groupConcat", { createAggregateFunctionGroupConcat, properties });
factory.registerAlias("group_concat", "groupConcat", AggregateFunctionFactory::CaseInsensitive);
factory.registerAlias("group_concat", "groupConcat", AggregateFunctionFactory::Case::Insensitive);
}
}

View File

@ -350,7 +350,7 @@ AggregateFunctionPtr createAggregateFunctionKolmogorovSmirnovTest(
void registerAggregateFunctionKolmogorovSmirnovTest(AggregateFunctionFactory & factory)
{
factory.registerFunction("kolmogorovSmirnovTest", createAggregateFunctionKolmogorovSmirnovTest, AggregateFunctionFactory::CaseInsensitive);
factory.registerFunction("kolmogorovSmirnovTest", createAggregateFunctionKolmogorovSmirnovTest, AggregateFunctionFactory::Case::Insensitive);
}
}

View File

@ -15,11 +15,11 @@ void registerAggregateFunctionsStatisticsSecondMoment(AggregateFunctionFactory &
factory.registerFunction("stddevPop", createAggregateFunctionStatisticsUnary<AggregateFunctionSecondMoment, StatisticsFunctionKind::stddevPop>);
/// Synonyms for compatibility.
factory.registerAlias("VAR_SAMP", "varSamp", AggregateFunctionFactory::CaseInsensitive);
factory.registerAlias("VAR_POP", "varPop", AggregateFunctionFactory::CaseInsensitive);
factory.registerAlias("STDDEV_SAMP", "stddevSamp", AggregateFunctionFactory::CaseInsensitive);
factory.registerAlias("STDDEV_POP", "stddevPop", AggregateFunctionFactory::CaseInsensitive);
factory.registerAlias("STD", "stddevPop", AggregateFunctionFactory::CaseInsensitive);
factory.registerAlias("VAR_SAMP", "varSamp", AggregateFunctionFactory::Case::Insensitive);
factory.registerAlias("VAR_POP", "varPop", AggregateFunctionFactory::Case::Insensitive);
factory.registerAlias("STDDEV_SAMP", "stddevSamp", AggregateFunctionFactory::Case::Insensitive);
factory.registerAlias("STDDEV_POP", "stddevPop", AggregateFunctionFactory::Case::Insensitive);
factory.registerAlias("STD", "stddevPop", AggregateFunctionFactory::Case::Insensitive);
}
}

View File

@ -72,7 +72,7 @@ AggregateFunctionPtr createAggregateFunctionSum(const std::string & name, const
void registerAggregateFunctionSum(AggregateFunctionFactory & factory)
{
factory.registerFunction("sum", createAggregateFunctionSum<AggregateFunctionSumSimple>, AggregateFunctionFactory::CaseInsensitive);
factory.registerFunction("sum", createAggregateFunctionSum<AggregateFunctionSumSimple>, AggregateFunctionFactory::Case::Insensitive);
factory.registerFunction("sumWithOverflow", createAggregateFunctionSum<AggregateFunctionSumWithOverflow>);
factory.registerFunction("sumKahan", createAggregateFunctionSum<AggregateFunctionSumKahan>);
}

View File

@ -535,9 +535,9 @@ void registerAggregateFunctionTopK(AggregateFunctionFactory & factory)
factory.registerFunction("topK", { createAggregateFunctionTopK<false, false>, properties });
factory.registerFunction("topKWeighted", { createAggregateFunctionTopK<true, false>, properties });
factory.registerFunction("approx_top_k", { createAggregateFunctionTopK<false, true>, properties }, AggregateFunctionFactory::CaseInsensitive);
factory.registerFunction("approx_top_sum", { createAggregateFunctionTopK<true, true>, properties }, AggregateFunctionFactory::CaseInsensitive);
factory.registerAlias("approx_top_count", "approx_top_k", AggregateFunctionFactory::CaseInsensitive);
factory.registerFunction("approx_top_k", { createAggregateFunctionTopK<false, true>, properties }, AggregateFunctionFactory::Case::Insensitive);
factory.registerFunction("approx_top_sum", { createAggregateFunctionTopK<true, true>, properties }, AggregateFunctionFactory::Case::Insensitive);
factory.registerAlias("approx_top_count", "approx_top_k", AggregateFunctionFactory::Case::Insensitive);
}
}

View File

@ -195,8 +195,8 @@ AggregateFunctionPtr createAggregateFunctionMinMax(
void registerAggregateFunctionsMinMax(AggregateFunctionFactory & factory)
{
factory.registerFunction("min", createAggregateFunctionMinMax<true>, AggregateFunctionFactory::CaseInsensitive);
factory.registerFunction("max", createAggregateFunctionMinMax<false>, AggregateFunctionFactory::CaseInsensitive);
factory.registerFunction("min", createAggregateFunctionMinMax<true>, AggregateFunctionFactory::Case::Insensitive);
factory.registerFunction("max", createAggregateFunctionMinMax<false>, AggregateFunctionFactory::Case::Insensitive);
}
}

View File

@ -2070,9 +2070,18 @@ void ClientBase::processParsedSingleQuery(const String & full_query, const Strin
progress_indication.writeFinalProgress();
output_stream << std::endl << std::endl;
}
else if (getClientConfiguration().getBool("print-time-to-stderr", false))
else
{
error_stream << progress_indication.elapsedSeconds() << "\n";
const auto & config = getClientConfiguration();
if (config.getBool("print-time-to-stderr", false))
error_stream << progress_indication.elapsedSeconds() << "\n";
const auto & print_memory_mode = config.getString("print-memory-to-stderr", "");
auto peak_memeory_usage = std::max<Int64>(progress_indication.getMemoryUsage().peak, 0);
if (print_memory_mode == "default")
error_stream << peak_memeory_usage << "\n";
else if (print_memory_mode == "readable")
error_stream << formatReadableSizeWithBinarySuffix(peak_memeory_usage) << "\n";
}
if (!is_interactive && getClientConfiguration().getBool("print-num-processed-rows", false))
@ -3036,6 +3045,7 @@ void ClientBase::init(int argc, char ** argv)
("disable_suggestion,A", "Disable loading suggestion data. Note that suggestion data is loaded asynchronously through a second connection to ClickHouse server. Also it is reasonable to disable suggestion if you want to paste a query with TAB characters. Shorthand option -A is for those who get used to mysql client.")
("wait_for_suggestions_to_load", "Load suggestion data synchonously.")
("time,t", "print query execution time to stderr in non-interactive mode (for benchmarks)")
("memory-usage", po::value<std::string>()->implicit_value("default")->default_value("none"), "print memory usage to stderr in non-interactive mode (for benchmarks). Values: 'none', 'default', 'readable'")
("echo", "in batch mode, print query before execution")
@ -3121,6 +3131,14 @@ void ClientBase::init(int argc, char ** argv)
/// Output execution time to stderr in batch mode.
if (options.count("time"))
getClientConfiguration().setBool("print-time-to-stderr", true);
if (options.count("memory-usage"))
{
const auto & memory_usage_mode = options["memory-usage"].as<std::string>();
if (memory_usage_mode != "none" && memory_usage_mode != "default" && memory_usage_mode != "readable")
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown memory-usage mode: {}", memory_usage_mode);
getClientConfiguration().setString("print-memory-to-stderr", memory_usage_mode);
}
if (options.count("query"))
queries = options["query"].as<std::vector<std::string>>();
if (options.count("query_id"))

View File

@ -54,8 +54,6 @@ public:
struct ReplicaInfo
{
bool collaborate_with_initiator{false};
size_t all_replicas_count{0};
size_t number_of_current_replica{0};
};

View File

@ -175,4 +175,5 @@ private:
ReadBuffer * in;
};
}

View File

@ -142,13 +142,12 @@ void MultiplexedConnections::sendQuery(
modified_settings.group_by_two_level_threshold = 0;
modified_settings.group_by_two_level_threshold_bytes = 0;
}
}
if (replica_info)
{
client_info.collaborate_with_initiator = true;
client_info.count_participating_replicas = replica_info->all_replicas_count;
client_info.number_of_current_replica = replica_info->number_of_current_replica;
}
if (replica_info)
{
client_info.collaborate_with_initiator = true;
client_info.number_of_current_replica = replica_info->number_of_current_replica;
}
/// FIXME: Remove once we will make `allow_experimental_analyzer` obsolete setting.

View File

@ -187,12 +187,9 @@ void * Allocator<clear_memory_, populate>::realloc(void * buf, size_t old_size,
#if USE_GWP_ASAN
if (unlikely(GWPAsan::GuardedAlloc.shouldSample()))
{
auto trace_alloc = CurrentMemoryTracker::alloc(new_size);
if (void * ptr = GWPAsan::GuardedAlloc.allocate(new_size, alignment))
{
auto trace_free = CurrentMemoryTracker::free(old_size);
auto trace_alloc = CurrentMemoryTracker::alloc(new_size);
trace_free.onFree(buf, old_size);
memcpy(ptr, buf, std::min(old_size, new_size));
free(buf, old_size);
trace_alloc.onAlloc(buf, new_size);
@ -209,6 +206,7 @@ void * Allocator<clear_memory_, populate>::realloc(void * buf, size_t old_size,
}
else
{
[[maybe_unused]] auto trace_free = CurrentMemoryTracker::free(old_size);
ProfileEvents::increment(ProfileEvents::GWPAsanAllocateFailed);
}
}
@ -231,13 +229,17 @@ void * Allocator<clear_memory_, populate>::realloc(void * buf, size_t old_size,
if (alignment <= MALLOC_MIN_ALIGNMENT)
{
/// Resize malloc'd memory region with no special alignment requirement.
auto trace_free = CurrentMemoryTracker::free(old_size);
/// Realloc can do 2 possible things:
/// - expand existing memory region
/// - allocate new memory block and free the old one
/// Because we don't know which option will be picked we need to make sure there is enough
/// memory for all options
auto trace_alloc = CurrentMemoryTracker::alloc(new_size);
trace_free.onFree(buf, old_size);
void * new_buf = ::realloc(buf, new_size);
if (nullptr == new_buf)
{
[[maybe_unused]] auto trace_free = CurrentMemoryTracker::free(old_size);
throw DB::ErrnoException(
DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY,
"Allocator: Cannot realloc from {} to {}",
@ -246,6 +248,8 @@ void * Allocator<clear_memory_, populate>::realloc(void * buf, size_t old_size,
}
buf = new_buf;
auto trace_free = CurrentMemoryTracker::free(old_size);
trace_free.onFree(buf, old_size);
trace_alloc.onAlloc(buf, new_size);
if constexpr (clear_memory)

View File

@ -39,16 +39,16 @@ protected:
public:
/// For compatibility with SQL, it's possible to specify that certain function name is case insensitive.
enum CaseSensitiveness
enum Case
{
CaseSensitive,
CaseInsensitive
Sensitive,
Insensitive
};
/** Register additional name for value
* real_name have to be already registered.
*/
void registerAlias(const String & alias_name, const String & real_name, CaseSensitiveness case_sensitiveness = CaseSensitive)
void registerAlias(const String & alias_name, const String & real_name, Case case_sensitiveness = Sensitive)
{
const auto & creator_map = getMap();
const auto & case_insensitive_creator_map = getCaseInsensitiveMap();
@ -66,12 +66,12 @@ public:
}
/// We need sure the real_name exactly exists when call the function directly.
void registerAliasUnchecked(const String & alias_name, const String & real_name, CaseSensitiveness case_sensitiveness = CaseSensitive)
void registerAliasUnchecked(const String & alias_name, const String & real_name, Case case_sensitiveness = Sensitive)
{
String alias_name_lowercase = Poco::toLower(alias_name);
const String factory_name = getFactoryName();
if (case_sensitiveness == CaseInsensitive)
if (case_sensitiveness == Insensitive)
{
if (!case_insensitive_aliases.emplace(alias_name_lowercase, real_name).second)
throw Exception(ErrorCodes::LOGICAL_ERROR, "{}: case insensitive alias name '{}' is not unique", factory_name, alias_name);

View File

@ -72,11 +72,6 @@ public:
/// How much seconds passed since query execution start.
double elapsedSeconds() const { return getElapsedNanoseconds() / 1e9; }
void updateThreadEventData(HostToTimesMap & new_hosts_data);
private:
double getCPUUsage();
struct MemoryUsage
{
UInt64 total = 0;
@ -86,6 +81,11 @@ private:
MemoryUsage getMemoryUsage() const;
void updateThreadEventData(HostToTimesMap & new_hosts_data);
private:
double getCPUUsage();
UInt64 getElapsedNanoseconds() const;
/// This flag controls whether to show the progress bar. We start showing it after

View File

@ -17,7 +17,7 @@ SerializationPtr DataTypeDate::doGetDefaultSerialization() const
void registerDataTypeDate(DataTypeFactory & factory)
{
factory.registerSimpleDataType("Date", [] { return DataTypePtr(std::make_shared<DataTypeDate>()); }, DataTypeFactory::CaseInsensitive);
factory.registerSimpleDataType("Date", [] { return DataTypePtr(std::make_shared<DataTypeDate>()); }, DataTypeFactory::Case::Insensitive);
}
}

View File

@ -24,7 +24,7 @@ Field DataTypeDate32::getDefault() const
void registerDataTypeDate32(DataTypeFactory & factory)
{
factory.registerSimpleDataType(
"Date32", [] { return DataTypePtr(std::make_shared<DataTypeDate32>()); }, DataTypeFactory::CaseInsensitive);
"Date32", [] { return DataTypePtr(std::make_shared<DataTypeDate32>()); }, DataTypeFactory::Case::Insensitive);
}
}

View File

@ -15,8 +15,8 @@ void registerDataTypeDomainBool(DataTypeFactory & factory)
std::make_unique<DataTypeCustomFixedName>("Bool"), std::make_unique<SerializationBool>(type->getDefaultSerialization())));
});
factory.registerAlias("bool", "Bool", DataTypeFactory::CaseInsensitive);
factory.registerAlias("boolean", "Bool", DataTypeFactory::CaseInsensitive);
factory.registerAlias("bool", "Bool", DataTypeFactory::Case::Insensitive);
factory.registerAlias("boolean", "Bool", DataTypeFactory::Case::Insensitive);
}
}

View File

@ -318,7 +318,7 @@ void registerDataTypeEnum(DataTypeFactory & factory)
factory.registerDataType("Enum", create);
/// MySQL
factory.registerAlias("ENUM", "Enum", DataTypeFactory::CaseInsensitive);
factory.registerAlias("ENUM", "Enum", DataTypeFactory::Case::Insensitive);
}
}

View File

@ -175,7 +175,7 @@ DataTypePtr DataTypeFactory::getCustom(DataTypeCustomDescPtr customization) cons
}
void DataTypeFactory::registerDataType(const String & family_name, Value creator, CaseSensitiveness case_sensitiveness)
void DataTypeFactory::registerDataType(const String & family_name, Value creator, Case case_sensitiveness)
{
if (creator == nullptr)
throw Exception(ErrorCodes::LOGICAL_ERROR, "DataTypeFactory: the data type family {} has been provided a null constructor", family_name);
@ -189,12 +189,12 @@ void DataTypeFactory::registerDataType(const String & family_name, Value creator
throw Exception(ErrorCodes::LOGICAL_ERROR, "DataTypeFactory: the data type family name '{}' is not unique",
family_name);
if (case_sensitiveness == CaseInsensitive
if (case_sensitiveness == Case::Insensitive
&& !case_insensitive_data_types.emplace(family_name_lowercase, creator).second)
throw Exception(ErrorCodes::LOGICAL_ERROR, "DataTypeFactory: the case insensitive data type family name '{}' is not unique", family_name);
}
void DataTypeFactory::registerSimpleDataType(const String & name, SimpleCreator creator, CaseSensitiveness case_sensitiveness)
void DataTypeFactory::registerSimpleDataType(const String & name, SimpleCreator creator, Case case_sensitiveness)
{
if (creator == nullptr)
throw Exception(ErrorCodes::LOGICAL_ERROR, "DataTypeFactory: the data type {} has been provided a null constructor",
@ -208,7 +208,7 @@ void DataTypeFactory::registerSimpleDataType(const String & name, SimpleCreator
}, case_sensitiveness);
}
void DataTypeFactory::registerDataTypeCustom(const String & family_name, CreatorWithCustom creator, CaseSensitiveness case_sensitiveness)
void DataTypeFactory::registerDataTypeCustom(const String & family_name, CreatorWithCustom creator, Case case_sensitiveness)
{
registerDataType(family_name, [creator](const ASTPtr & ast)
{
@ -219,7 +219,7 @@ void DataTypeFactory::registerDataTypeCustom(const String & family_name, Creator
}, case_sensitiveness);
}
void DataTypeFactory::registerSimpleDataTypeCustom(const String & name, SimpleCreatorWithCustom creator, CaseSensitiveness case_sensitiveness)
void DataTypeFactory::registerSimpleDataTypeCustom(const String & name, SimpleCreatorWithCustom creator, Case case_sensitiveness)
{
registerDataTypeCustom(name, [name, creator](const ASTPtr & ast)
{

View File

@ -41,16 +41,16 @@ public:
DataTypePtr tryGet(const ASTPtr & ast) const;
/// Register a type family by its name.
void registerDataType(const String & family_name, Value creator, CaseSensitiveness case_sensitiveness = CaseSensitive);
void registerDataType(const String & family_name, Value creator, Case case_sensitiveness = Case::Sensitive);
/// Register a simple data type, that have no parameters.
void registerSimpleDataType(const String & name, SimpleCreator creator, CaseSensitiveness case_sensitiveness = CaseSensitive);
void registerSimpleDataType(const String & name, SimpleCreator creator, Case case_sensitiveness = Case::Sensitive);
/// Register a customized type family
void registerDataTypeCustom(const String & family_name, CreatorWithCustom creator, CaseSensitiveness case_sensitiveness = CaseSensitive);
void registerDataTypeCustom(const String & family_name, CreatorWithCustom creator, Case case_sensitiveness = Case::Sensitive);
/// Register a simple customized data type
void registerSimpleDataTypeCustom(const String & name, SimpleCreatorWithCustom creator, CaseSensitiveness case_sensitiveness = CaseSensitive);
void registerSimpleDataTypeCustom(const String & name, SimpleCreatorWithCustom creator, Case case_sensitiveness = Case::Sensitive);
private:
template <bool nullptr_on_error>

View File

@ -64,7 +64,7 @@ void registerDataTypeFixedString(DataTypeFactory & factory)
factory.registerDataType("FixedString", create);
/// Compatibility alias.
factory.registerAlias("BINARY", "FixedString", DataTypeFactory::CaseInsensitive);
factory.registerAlias("BINARY", "FixedString", DataTypeFactory::Case::Insensitive);
}
}

View File

@ -9,9 +9,9 @@ namespace DB
void registerDataTypeIPv4andIPv6(DataTypeFactory & factory)
{
factory.registerSimpleDataType("IPv4", [] { return DataTypePtr(std::make_shared<DataTypeIPv4>()); });
factory.registerAlias("INET4", "IPv4", DataTypeFactory::CaseInsensitive);
factory.registerAlias("INET4", "IPv4", DataTypeFactory::Case::Insensitive);
factory.registerSimpleDataType("IPv6", [] { return DataTypePtr(std::make_shared<DataTypeIPv6>()); });
factory.registerAlias("INET6", "IPv6", DataTypeFactory::CaseInsensitive);
factory.registerAlias("INET6", "IPv6", DataTypeFactory::Case::Insensitive);
}
}

View File

@ -76,7 +76,7 @@ void registerDataTypeObject(DataTypeFactory & factory)
factory.registerDataType("Object", create);
factory.registerSimpleDataType("JSON",
[] { return std::make_shared<DataTypeObject>("JSON", false); },
DataTypeFactory::CaseInsensitive);
DataTypeFactory::Case::Insensitive);
}
}

View File

@ -62,38 +62,38 @@ void registerDataTypeString(DataTypeFactory & factory)
/// These synonims are added for compatibility.
factory.registerAlias("CHAR", "String", DataTypeFactory::CaseInsensitive);
factory.registerAlias("NCHAR", "String", DataTypeFactory::CaseInsensitive);
factory.registerAlias("CHARACTER", "String", DataTypeFactory::CaseInsensitive);
factory.registerAlias("VARCHAR", "String", DataTypeFactory::CaseInsensitive);
factory.registerAlias("NVARCHAR", "String", DataTypeFactory::CaseInsensitive);
factory.registerAlias("VARCHAR2", "String", DataTypeFactory::CaseInsensitive); /// Oracle
factory.registerAlias("TEXT", "String", DataTypeFactory::CaseInsensitive);
factory.registerAlias("TINYTEXT", "String", DataTypeFactory::CaseInsensitive);
factory.registerAlias("MEDIUMTEXT", "String", DataTypeFactory::CaseInsensitive);
factory.registerAlias("LONGTEXT", "String", DataTypeFactory::CaseInsensitive);
factory.registerAlias("BLOB", "String", DataTypeFactory::CaseInsensitive);
factory.registerAlias("CLOB", "String", DataTypeFactory::CaseInsensitive);
factory.registerAlias("TINYBLOB", "String", DataTypeFactory::CaseInsensitive);
factory.registerAlias("MEDIUMBLOB", "String", DataTypeFactory::CaseInsensitive);
factory.registerAlias("LONGBLOB", "String", DataTypeFactory::CaseInsensitive);
factory.registerAlias("BYTEA", "String", DataTypeFactory::CaseInsensitive); /// PostgreSQL
factory.registerAlias("CHAR", "String", DataTypeFactory::Case::Insensitive);
factory.registerAlias("NCHAR", "String", DataTypeFactory::Case::Insensitive);
factory.registerAlias("CHARACTER", "String", DataTypeFactory::Case::Insensitive);
factory.registerAlias("VARCHAR", "String", DataTypeFactory::Case::Insensitive);
factory.registerAlias("NVARCHAR", "String", DataTypeFactory::Case::Insensitive);
factory.registerAlias("VARCHAR2", "String", DataTypeFactory::Case::Insensitive); /// Oracle
factory.registerAlias("TEXT", "String", DataTypeFactory::Case::Insensitive);
factory.registerAlias("TINYTEXT", "String", DataTypeFactory::Case::Insensitive);
factory.registerAlias("MEDIUMTEXT", "String", DataTypeFactory::Case::Insensitive);
factory.registerAlias("LONGTEXT", "String", DataTypeFactory::Case::Insensitive);
factory.registerAlias("BLOB", "String", DataTypeFactory::Case::Insensitive);
factory.registerAlias("CLOB", "String", DataTypeFactory::Case::Insensitive);
factory.registerAlias("TINYBLOB", "String", DataTypeFactory::Case::Insensitive);
factory.registerAlias("MEDIUMBLOB", "String", DataTypeFactory::Case::Insensitive);
factory.registerAlias("LONGBLOB", "String", DataTypeFactory::Case::Insensitive);
factory.registerAlias("BYTEA", "String", DataTypeFactory::Case::Insensitive); /// PostgreSQL
factory.registerAlias("CHARACTER LARGE OBJECT", "String", DataTypeFactory::CaseInsensitive);
factory.registerAlias("CHARACTER VARYING", "String", DataTypeFactory::CaseInsensitive);
factory.registerAlias("CHAR LARGE OBJECT", "String", DataTypeFactory::CaseInsensitive);
factory.registerAlias("CHAR VARYING", "String", DataTypeFactory::CaseInsensitive);
factory.registerAlias("NATIONAL CHAR", "String", DataTypeFactory::CaseInsensitive);
factory.registerAlias("NATIONAL CHARACTER", "String", DataTypeFactory::CaseInsensitive);
factory.registerAlias("NATIONAL CHARACTER LARGE OBJECT", "String", DataTypeFactory::CaseInsensitive);
factory.registerAlias("NATIONAL CHARACTER VARYING", "String", DataTypeFactory::CaseInsensitive);
factory.registerAlias("NATIONAL CHAR VARYING", "String", DataTypeFactory::CaseInsensitive);
factory.registerAlias("NCHAR VARYING", "String", DataTypeFactory::CaseInsensitive);
factory.registerAlias("NCHAR LARGE OBJECT", "String", DataTypeFactory::CaseInsensitive);
factory.registerAlias("BINARY LARGE OBJECT", "String", DataTypeFactory::CaseInsensitive);
factory.registerAlias("BINARY VARYING", "String", DataTypeFactory::CaseInsensitive);
factory.registerAlias("VARBINARY", "String", DataTypeFactory::CaseInsensitive);
factory.registerAlias("GEOMETRY", "String", DataTypeFactory::CaseInsensitive); //mysql
factory.registerAlias("CHARACTER LARGE OBJECT", "String", DataTypeFactory::Case::Insensitive);
factory.registerAlias("CHARACTER VARYING", "String", DataTypeFactory::Case::Insensitive);
factory.registerAlias("CHAR LARGE OBJECT", "String", DataTypeFactory::Case::Insensitive);
factory.registerAlias("CHAR VARYING", "String", DataTypeFactory::Case::Insensitive);
factory.registerAlias("NATIONAL CHAR", "String", DataTypeFactory::Case::Insensitive);
factory.registerAlias("NATIONAL CHARACTER", "String", DataTypeFactory::Case::Insensitive);
factory.registerAlias("NATIONAL CHARACTER LARGE OBJECT", "String", DataTypeFactory::Case::Insensitive);
factory.registerAlias("NATIONAL CHARACTER VARYING", "String", DataTypeFactory::Case::Insensitive);
factory.registerAlias("NATIONAL CHAR VARYING", "String", DataTypeFactory::Case::Insensitive);
factory.registerAlias("NCHAR VARYING", "String", DataTypeFactory::Case::Insensitive);
factory.registerAlias("NCHAR LARGE OBJECT", "String", DataTypeFactory::Case::Insensitive);
factory.registerAlias("BINARY LARGE OBJECT", "String", DataTypeFactory::Case::Insensitive);
factory.registerAlias("BINARY VARYING", "String", DataTypeFactory::Case::Insensitive);
factory.registerAlias("VARBINARY", "String", DataTypeFactory::Case::Insensitive);
factory.registerAlias("GEOMETRY", "String", DataTypeFactory::Case::Insensitive); //mysql
}
}

View File

@ -364,15 +364,15 @@ template class DataTypeDecimal<Decimal256>;
void registerDataTypeDecimal(DataTypeFactory & factory)
{
factory.registerDataType("Decimal32", createExact<Decimal32>, DataTypeFactory::CaseInsensitive);
factory.registerDataType("Decimal64", createExact<Decimal64>, DataTypeFactory::CaseInsensitive);
factory.registerDataType("Decimal128", createExact<Decimal128>, DataTypeFactory::CaseInsensitive);
factory.registerDataType("Decimal256", createExact<Decimal256>, DataTypeFactory::CaseInsensitive);
factory.registerDataType("Decimal32", createExact<Decimal32>, DataTypeFactory::Case::Insensitive);
factory.registerDataType("Decimal64", createExact<Decimal64>, DataTypeFactory::Case::Insensitive);
factory.registerDataType("Decimal128", createExact<Decimal128>, DataTypeFactory::Case::Insensitive);
factory.registerDataType("Decimal256", createExact<Decimal256>, DataTypeFactory::Case::Insensitive);
factory.registerDataType("Decimal", create, DataTypeFactory::CaseInsensitive);
factory.registerAlias("DEC", "Decimal", DataTypeFactory::CaseInsensitive);
factory.registerAlias("NUMERIC", "Decimal", DataTypeFactory::CaseInsensitive);
factory.registerAlias("FIXED", "Decimal", DataTypeFactory::CaseInsensitive);
factory.registerDataType("Decimal", create, DataTypeFactory::Case::Insensitive);
factory.registerAlias("DEC", "Decimal", DataTypeFactory::Case::Insensitive);
factory.registerAlias("NUMERIC", "Decimal", DataTypeFactory::Case::Insensitive);
factory.registerAlias("FIXED", "Decimal", DataTypeFactory::Case::Insensitive);
}
}

View File

@ -65,41 +65,41 @@ void registerDataTypeNumbers(DataTypeFactory & factory)
/// These synonyms are added for compatibility.
factory.registerAlias("TINYINT", "Int8", DataTypeFactory::CaseInsensitive);
factory.registerAlias("INT1", "Int8", DataTypeFactory::CaseInsensitive);
factory.registerAlias("BYTE", "Int8", DataTypeFactory::CaseInsensitive);
factory.registerAlias("TINYINT SIGNED", "Int8", DataTypeFactory::CaseInsensitive);
factory.registerAlias("INT1 SIGNED", "Int8", DataTypeFactory::CaseInsensitive);
factory.registerAlias("SMALLINT", "Int16", DataTypeFactory::CaseInsensitive);
factory.registerAlias("SMALLINT SIGNED", "Int16", DataTypeFactory::CaseInsensitive);
factory.registerAlias("INT", "Int32", DataTypeFactory::CaseInsensitive);
factory.registerAlias("INTEGER", "Int32", DataTypeFactory::CaseInsensitive);
factory.registerAlias("MEDIUMINT", "Int32", DataTypeFactory::CaseInsensitive);
factory.registerAlias("MEDIUMINT SIGNED", "Int32", DataTypeFactory::CaseInsensitive);
factory.registerAlias("INT SIGNED", "Int32", DataTypeFactory::CaseInsensitive);
factory.registerAlias("INTEGER SIGNED", "Int32", DataTypeFactory::CaseInsensitive);
factory.registerAlias("BIGINT", "Int64", DataTypeFactory::CaseInsensitive);
factory.registerAlias("SIGNED", "Int64", DataTypeFactory::CaseInsensitive);
factory.registerAlias("BIGINT SIGNED", "Int64", DataTypeFactory::CaseInsensitive);
factory.registerAlias("TIME", "Int64", DataTypeFactory::CaseInsensitive);
factory.registerAlias("TINYINT", "Int8", DataTypeFactory::Case::Insensitive);
factory.registerAlias("INT1", "Int8", DataTypeFactory::Case::Insensitive);
factory.registerAlias("BYTE", "Int8", DataTypeFactory::Case::Insensitive);
factory.registerAlias("TINYINT SIGNED", "Int8", DataTypeFactory::Case::Insensitive);
factory.registerAlias("INT1 SIGNED", "Int8", DataTypeFactory::Case::Insensitive);
factory.registerAlias("SMALLINT", "Int16", DataTypeFactory::Case::Insensitive);
factory.registerAlias("SMALLINT SIGNED", "Int16", DataTypeFactory::Case::Insensitive);
factory.registerAlias("INT", "Int32", DataTypeFactory::Case::Insensitive);
factory.registerAlias("INTEGER", "Int32", DataTypeFactory::Case::Insensitive);
factory.registerAlias("MEDIUMINT", "Int32", DataTypeFactory::Case::Insensitive);
factory.registerAlias("MEDIUMINT SIGNED", "Int32", DataTypeFactory::Case::Insensitive);
factory.registerAlias("INT SIGNED", "Int32", DataTypeFactory::Case::Insensitive);
factory.registerAlias("INTEGER SIGNED", "Int32", DataTypeFactory::Case::Insensitive);
factory.registerAlias("BIGINT", "Int64", DataTypeFactory::Case::Insensitive);
factory.registerAlias("SIGNED", "Int64", DataTypeFactory::Case::Insensitive);
factory.registerAlias("BIGINT SIGNED", "Int64", DataTypeFactory::Case::Insensitive);
factory.registerAlias("TIME", "Int64", DataTypeFactory::Case::Insensitive);
factory.registerAlias("TINYINT UNSIGNED", "UInt8", DataTypeFactory::CaseInsensitive);
factory.registerAlias("INT1 UNSIGNED", "UInt8", DataTypeFactory::CaseInsensitive);
factory.registerAlias("SMALLINT UNSIGNED", "UInt16", DataTypeFactory::CaseInsensitive);
factory.registerAlias("YEAR", "UInt16", DataTypeFactory::CaseInsensitive);
factory.registerAlias("MEDIUMINT UNSIGNED", "UInt32", DataTypeFactory::CaseInsensitive);
factory.registerAlias("INT UNSIGNED", "UInt32", DataTypeFactory::CaseInsensitive);
factory.registerAlias("INTEGER UNSIGNED", "UInt32", DataTypeFactory::CaseInsensitive);
factory.registerAlias("UNSIGNED", "UInt64", DataTypeFactory::CaseInsensitive);
factory.registerAlias("BIGINT UNSIGNED", "UInt64", DataTypeFactory::CaseInsensitive);
factory.registerAlias("BIT", "UInt64", DataTypeFactory::CaseInsensitive);
factory.registerAlias("SET", "UInt64", DataTypeFactory::CaseInsensitive);
factory.registerAlias("TINYINT UNSIGNED", "UInt8", DataTypeFactory::Case::Insensitive);
factory.registerAlias("INT1 UNSIGNED", "UInt8", DataTypeFactory::Case::Insensitive);
factory.registerAlias("SMALLINT UNSIGNED", "UInt16", DataTypeFactory::Case::Insensitive);
factory.registerAlias("YEAR", "UInt16", DataTypeFactory::Case::Insensitive);
factory.registerAlias("MEDIUMINT UNSIGNED", "UInt32", DataTypeFactory::Case::Insensitive);
factory.registerAlias("INT UNSIGNED", "UInt32", DataTypeFactory::Case::Insensitive);
factory.registerAlias("INTEGER UNSIGNED", "UInt32", DataTypeFactory::Case::Insensitive);
factory.registerAlias("UNSIGNED", "UInt64", DataTypeFactory::Case::Insensitive);
factory.registerAlias("BIGINT UNSIGNED", "UInt64", DataTypeFactory::Case::Insensitive);
factory.registerAlias("BIT", "UInt64", DataTypeFactory::Case::Insensitive);
factory.registerAlias("SET", "UInt64", DataTypeFactory::Case::Insensitive);
factory.registerAlias("FLOAT", "Float32", DataTypeFactory::CaseInsensitive);
factory.registerAlias("REAL", "Float32", DataTypeFactory::CaseInsensitive);
factory.registerAlias("SINGLE", "Float32", DataTypeFactory::CaseInsensitive);
factory.registerAlias("DOUBLE", "Float64", DataTypeFactory::CaseInsensitive);
factory.registerAlias("DOUBLE PRECISION", "Float64", DataTypeFactory::CaseInsensitive);
factory.registerAlias("FLOAT", "Float32", DataTypeFactory::Case::Insensitive);
factory.registerAlias("REAL", "Float32", DataTypeFactory::Case::Insensitive);
factory.registerAlias("SINGLE", "Float32", DataTypeFactory::Case::Insensitive);
factory.registerAlias("DOUBLE", "Float64", DataTypeFactory::Case::Insensitive);
factory.registerAlias("DOUBLE PRECISION", "Float64", DataTypeFactory::Case::Insensitive);
}
/// Explicit template instantiations.

View File

@ -108,11 +108,11 @@ static DataTypePtr create64(const ASTPtr & arguments)
void registerDataTypeDateTime(DataTypeFactory & factory)
{
factory.registerDataType("DateTime", create, DataTypeFactory::CaseInsensitive);
factory.registerDataType("DateTime32", create32, DataTypeFactory::CaseInsensitive);
factory.registerDataType("DateTime64", create64, DataTypeFactory::CaseInsensitive);
factory.registerDataType("DateTime", create, DataTypeFactory::Case::Insensitive);
factory.registerDataType("DateTime32", create32, DataTypeFactory::Case::Insensitive);
factory.registerDataType("DateTime64", create64, DataTypeFactory::Case::Insensitive);
factory.registerAlias("TIMESTAMP", "DateTime", DataTypeFactory::CaseInsensitive);
factory.registerAlias("TIMESTAMP", "DateTime", DataTypeFactory::Case::Insensitive);
}
}

View File

@ -6,7 +6,6 @@
#include <Interpreters/executeQuery.h>
#include <Parsers/IAST_fwd.h>
#include <QueryPipeline/BlockIO.h>
#include <Storages/IStorage.h>
#include <Storages/IStorage_fwd.h>
#include <base/types.h>
#include <Common/AsyncLoader.h>

View File

@ -150,9 +150,9 @@ using FunctionCRC64ECMA = FunctionCRC<CRC64ECMAImpl>;
REGISTER_FUNCTION(CRC)
{
factory.registerFunction<FunctionCRC32ZLib>({}, FunctionFactory::CaseInsensitive);
factory.registerFunction<FunctionCRC32IEEE>({}, FunctionFactory::CaseInsensitive);
factory.registerFunction<FunctionCRC64ECMA>({}, FunctionFactory::CaseInsensitive);
factory.registerFunction<FunctionCRC32ZLib>({}, FunctionFactory::Case::Insensitive);
factory.registerFunction<FunctionCRC32IEEE>({}, FunctionFactory::Case::Insensitive);
factory.registerFunction<FunctionCRC64ECMA>({}, FunctionFactory::Case::Insensitive);
}
}

View File

@ -137,10 +137,10 @@ FunctionOverloadResolverPtr createInternalCastOverloadResolver(CastType type, st
REGISTER_FUNCTION(CastOverloadResolvers)
{
factory.registerFunction("_CAST", [](ContextPtr context){ return CastOverloadResolverImpl::create(context, CastType::nonAccurate, true, {}); }, {}, FunctionFactory::CaseInsensitive);
factory.registerFunction("_CAST", [](ContextPtr context){ return CastOverloadResolverImpl::create(context, CastType::nonAccurate, true, {}); }, {}, FunctionFactory::Case::Insensitive);
/// Note: "internal" (not affected by null preserving setting) versions of accurate cast functions are unneeded.
factory.registerFunction("CAST", [](ContextPtr context){ return CastOverloadResolverImpl::create(context, CastType::nonAccurate, false, {}); }, {}, FunctionFactory::CaseInsensitive);
factory.registerFunction("CAST", [](ContextPtr context){ return CastOverloadResolverImpl::create(context, CastType::nonAccurate, false, {}); }, {}, FunctionFactory::Case::Insensitive);
factory.registerFunction("accurateCast", [](ContextPtr context){ return CastOverloadResolverImpl::create(context, CastType::accurate, false, {}); }, {});
factory.registerFunction("accurateCastOrNull", [](ContextPtr context){ return CastOverloadResolverImpl::create(context, CastType::accurateOrNull, false, {}); }, {});
}

View File

@ -116,7 +116,7 @@ private:
REGISTER_FUNCTION(Char)
{
factory.registerFunction<FunctionChar>({}, FunctionFactory::CaseInsensitive);
factory.registerFunction<FunctionChar>({}, FunctionFactory::Case::Insensitive);
}
}

View File

@ -46,7 +46,7 @@ public:
REGISTER_FUNCTION(FQDN)
{
factory.registerFunction<FunctionFQDN>({}, FunctionFactory::CaseInsensitive);
factory.registerFunction<FunctionFQDN>({}, FunctionFactory::Case::Insensitive);
factory.registerAlias("fullHostName", "FQDN");
}

View File

@ -31,7 +31,7 @@ void FunctionFactory::registerFunction(
const std::string & name,
FunctionCreator creator,
FunctionDocumentation doc,
CaseSensitiveness case_sensitiveness)
Case case_sensitiveness)
{
if (!functions.emplace(name, FunctionFactoryData{creator, doc}).second)
throw Exception(ErrorCodes::LOGICAL_ERROR, "FunctionFactory: the function name '{}' is not unique", name);
@ -41,7 +41,7 @@ void FunctionFactory::registerFunction(
throw Exception(ErrorCodes::LOGICAL_ERROR, "FunctionFactory: the function name '{}' is already registered as alias",
name);
if (case_sensitiveness == CaseInsensitive)
if (case_sensitiveness == Case::Insensitive)
{
if (!case_insensitive_functions.emplace(function_name_lowercase, FunctionFactoryData{creator, doc}).second)
throw Exception(ErrorCodes::LOGICAL_ERROR, "FunctionFactory: the case insensitive function name '{}' is not unique",
@ -54,7 +54,7 @@ void FunctionFactory::registerFunction(
const std::string & name,
FunctionSimpleCreator creator,
FunctionDocumentation doc,
CaseSensitiveness case_sensitiveness)
Case case_sensitiveness)
{
registerFunction(name, [my_creator = std::move(creator)](ContextPtr context)
{

View File

@ -30,7 +30,7 @@ public:
static FunctionFactory & instance();
template <typename Function>
void registerFunction(FunctionDocumentation doc = {}, CaseSensitiveness case_sensitiveness = CaseSensitive)
void registerFunction(FunctionDocumentation doc = {}, Case case_sensitiveness = Case::Sensitive)
{
registerFunction<Function>(Function::name, std::move(doc), case_sensitiveness);
}
@ -56,13 +56,13 @@ public:
const std::string & name,
FunctionCreator creator,
FunctionDocumentation doc = {},
CaseSensitiveness case_sensitiveness = CaseSensitive);
Case case_sensitiveness = Case::Sensitive);
void registerFunction(
const std::string & name,
FunctionSimpleCreator creator,
FunctionDocumentation doc = {},
CaseSensitiveness case_sensitiveness = CaseSensitive);
Case case_sensitiveness = Case::Sensitive);
FunctionDocumentation getDocumentation(const std::string & name) const;
@ -79,7 +79,7 @@ private:
String getFactoryName() const override { return "FunctionFactory"; }
template <typename Function>
void registerFunction(const std::string & name, FunctionDocumentation doc = {}, CaseSensitiveness case_sensitiveness = CaseSensitive)
void registerFunction(const std::string & name, FunctionDocumentation doc = {}, Case case_sensitiveness = Case::Sensitive)
{
registerFunction(name, &Function::create, std::move(doc), case_sensitiveness);
}

View File

@ -445,8 +445,7 @@ The function returns a value of type String.
{"with specified seed", "SELECT generateRandomStructure(1, 42)", "c1 UInt128"},
},
.categories{"Random"}
},
FunctionFactory::CaseSensitive);
});
}
}

View File

@ -728,10 +728,10 @@ public:
REGISTER_FUNCTION(BinaryRepr)
{
factory.registerFunction<EncodeToBinaryRepresentation<HexImpl>>({}, FunctionFactory::CaseInsensitive);
factory.registerFunction<DecodeFromBinaryRepresentation<UnhexImpl>>({}, FunctionFactory::CaseInsensitive);
factory.registerFunction<EncodeToBinaryRepresentation<BinImpl>>({}, FunctionFactory::CaseInsensitive);
factory.registerFunction<DecodeFromBinaryRepresentation<UnbinImpl>>({}, FunctionFactory::CaseInsensitive);
factory.registerFunction<EncodeToBinaryRepresentation<HexImpl>>({}, FunctionFactory::Case::Insensitive);
factory.registerFunction<DecodeFromBinaryRepresentation<UnhexImpl>>({}, FunctionFactory::Case::Insensitive);
factory.registerFunction<EncodeToBinaryRepresentation<BinImpl>>({}, FunctionFactory::Case::Insensitive);
factory.registerFunction<DecodeFromBinaryRepresentation<UnbinImpl>>({}, FunctionFactory::Case::Insensitive);
}
}

View File

@ -1169,10 +1169,10 @@ REGISTER_FUNCTION(Coding)
factory.registerFunction<FunctionIPv6StringToNum<IPStringToNumExceptionMode::Null>>();
/// MySQL compatibility aliases:
factory.registerAlias("INET_ATON", FunctionIPv4StringToNum<IPStringToNumExceptionMode::Throw>::name, FunctionFactory::CaseInsensitive);
factory.registerAlias("INET6_NTOA", FunctionIPv6NumToString::name, FunctionFactory::CaseInsensitive);
factory.registerAlias("INET6_ATON", FunctionIPv6StringToNum<IPStringToNumExceptionMode::Throw>::name, FunctionFactory::CaseInsensitive);
factory.registerAlias("INET_NTOA", NameFunctionIPv4NumToString::name, FunctionFactory::CaseInsensitive);
factory.registerAlias("INET_ATON", FunctionIPv4StringToNum<IPStringToNumExceptionMode::Throw>::name, FunctionFactory::Case::Insensitive);
factory.registerAlias("INET6_NTOA", FunctionIPv6NumToString::name, FunctionFactory::Case::Insensitive);
factory.registerAlias("INET6_ATON", FunctionIPv6StringToNum<IPStringToNumExceptionMode::Throw>::name, FunctionFactory::Case::Insensitive);
factory.registerAlias("INET_NTOA", NameFunctionIPv4NumToString::name, FunctionFactory::Case::Insensitive);
}
}

View File

@ -180,8 +180,7 @@ An optional second argument can be passed to specify a timezone for the timestam
{"ulid", "SELECT ULIDStringToDateTime(generateULID())", ""},
{"timezone", "SELECT ULIDStringToDateTime(generateULID(), 'Asia/Istanbul')", ""}},
.categories{"ULID"}
},
FunctionFactory::CaseSensitive);
});
}
}

View File

@ -496,8 +496,8 @@ This function accepts a UUID and returns a FixedString(16) as its binary represe
612f3c40-5d3b-217e-707b-6a546a3d7b29 a/<@];!~p{jTj={) @</a];!~p{jTj={)
)"}},
.categories{"UUID"}},
FunctionFactory::CaseSensitive);
.categories{"UUID"}});
factory.registerFunction<FunctionUUIDv7ToDateTime>(
FunctionDocumentation{
@ -509,8 +509,7 @@ An optional second argument can be passed to specify a timezone for the timestam
.examples{
{"uuid","select UUIDv7ToDateTime(generateUUIDv7())", ""},
{"uuid","select generateUUIDv7() as uuid, UUIDv7ToDateTime(uuid), UUIDv7ToDateTime(uuid, 'America/New_York')", ""}},
.categories{"UUID"}},
FunctionFactory::CaseSensitive);
.categories{"UUID"}});
}
}

View File

@ -5224,7 +5224,7 @@ REGISTER_FUNCTION(Conversion)
/// MySQL compatibility alias. Cannot be registered as alias,
/// because we don't want it to be normalized to toDate in queries,
/// otherwise CREATE DICTIONARY query breaks.
factory.registerFunction("DATE", &FunctionToDate::create, {}, FunctionFactory::CaseInsensitive);
factory.registerFunction("DATE", &FunctionToDate::create, {}, FunctionFactory::Case::Insensitive);
factory.registerFunction<FunctionToDate32>();
factory.registerFunction<FunctionToDateTime>();

View File

@ -1184,7 +1184,7 @@ private:
if (icolumn->size() != vec_to.size())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Argument column '{}' size {} doesn't match result column size {} of function {}",
icolumn->getName(), icolumn->size(), vec_to.size(), getName());
icolumn->getName(), icolumn->size(), vec_to.size(), getName());
if constexpr (Keyed)
if (key_cols.size() != vec_to.size() && key_cols.size() != 1)
@ -1223,6 +1223,9 @@ private:
else executeGeneric<first>(key_cols, icolumn, vec_to);
}
/// Return a fixed random-looking magic number when input is empty.
static constexpr auto filler = 0xe28dbde7fe22e41c;
void executeForArgument(const KeyColumnsType & key_cols, const IDataType * type, const IColumn * column, typename ColumnVector<ToType>::Container & vec_to, bool & is_first) const
{
/// Flattening of tuples.
@ -1231,6 +1234,11 @@ private:
const auto & tuple_columns = tuple->getColumns();
const DataTypes & tuple_types = typeid_cast<const DataTypeTuple &>(*type).getElements();
size_t tuple_size = tuple_columns.size();
if (0 == tuple_size && is_first)
for (auto & hash : vec_to)
hash = static_cast<ToType>(filler);
for (size_t i = 0; i < tuple_size; ++i)
executeForArgument(key_cols, tuple_types[i].get(), tuple_columns[i].get(), vec_to, is_first);
}
@ -1239,6 +1247,11 @@ private:
const auto & tuple_columns = tuple_const->getColumns();
const DataTypes & tuple_types = typeid_cast<const DataTypeTuple &>(*type).getElements();
size_t tuple_size = tuple_columns.size();
if (0 == tuple_size && is_first)
for (auto & hash : vec_to)
hash = static_cast<ToType>(filler);
for (size_t i = 0; i < tuple_size; ++i)
{
auto tmp = ColumnConst::create(tuple_columns[i], column->size());
@ -1300,10 +1313,7 @@ public:
constexpr size_t first_data_argument = Keyed;
if (arguments.size() <= first_data_argument)
{
/// Return a fixed random-looking magic number when input is empty
vec_to.assign(input_rows_count, static_cast<ToType>(0xe28dbde7fe22e41c));
}
vec_to.assign(input_rows_count, static_cast<ToType>(filler));
KeyColumnsType key_cols{};
if constexpr (Keyed)

View File

@ -41,8 +41,7 @@ REGISTER_FUNCTION(Hashing)
.description="Calculates value of XXH3 64-bit hash function. Refer to https://github.com/Cyan4973/xxHash for detailed documentation.",
.examples{{"hash", "SELECT xxh3('ClickHouse')", ""}},
.categories{"Hash"}
},
FunctionFactory::CaseSensitive);
});
factory.registerFunction<FunctionWyHash64>();

View File

@ -29,7 +29,7 @@ REGISTER_FUNCTION(Logical)
factory.registerFunction<FunctionAnd>();
factory.registerFunction<FunctionOr>();
factory.registerFunction<FunctionXor>();
factory.registerFunction<FunctionNot>({}, FunctionFactory::CaseInsensitive); /// Operator NOT(x) can be parsed as a function.
factory.registerFunction<FunctionNot>({}, FunctionFactory::Case::Insensitive); /// Operator NOT(x) can be parsed as a function.
}
namespace ErrorCodes

View File

@ -99,8 +99,8 @@ using FunctionSubDate = FunctionOpDate<SubDate>;
REGISTER_FUNCTION(AddInterval)
{
factory.registerFunction<FunctionAddDate>({}, FunctionFactory::CaseInsensitive);
factory.registerFunction<FunctionSubDate>({}, FunctionFactory::CaseInsensitive);
factory.registerFunction<FunctionAddDate>({}, FunctionFactory::Case::Insensitive);
factory.registerFunction<FunctionSubDate>({}, FunctionFactory::Case::Insensitive);
}
}

View File

@ -7,16 +7,16 @@ namespace DB
REGISTER_FUNCTION(Round)
{
factory.registerFunction<FunctionRound>({}, FunctionFactory::CaseInsensitive);
factory.registerFunction<FunctionRoundBankers>({}, FunctionFactory::CaseSensitive);
factory.registerFunction<FunctionFloor>({}, FunctionFactory::CaseInsensitive);
factory.registerFunction<FunctionCeil>({}, FunctionFactory::CaseInsensitive);
factory.registerFunction<FunctionTrunc>({}, FunctionFactory::CaseInsensitive);
factory.registerFunction<FunctionRound>({}, FunctionFactory::Case::Insensitive);
factory.registerFunction<FunctionRoundBankers>({}, FunctionFactory::Case::Sensitive);
factory.registerFunction<FunctionFloor>({}, FunctionFactory::Case::Insensitive);
factory.registerFunction<FunctionCeil>({}, FunctionFactory::Case::Insensitive);
factory.registerFunction<FunctionTrunc>({}, FunctionFactory::Case::Insensitive);
factory.registerFunction<FunctionRoundDown>();
/// Compatibility aliases.
factory.registerAlias("ceiling", "ceil", FunctionFactory::CaseInsensitive);
factory.registerAlias("truncate", "trunc", FunctionFactory::CaseInsensitive);
factory.registerAlias("ceiling", "ceil", FunctionFactory::Case::Insensitive);
factory.registerAlias("truncate", "trunc", FunctionFactory::Case::Insensitive);
}
}

View File

@ -428,8 +428,7 @@ REGISTER_FUNCTION(HashFixedStrings)
It returns a BLAKE3 hash as a byte array with type FixedString(32).
)",
.examples{{"hash", "SELECT hex(BLAKE3('ABC'))", ""}},
.categories{"Hash"}},
FunctionFactory::CaseSensitive);
.categories{"Hash"}});
# endif
}
#endif

View File

@ -104,7 +104,7 @@ REGISTER_FUNCTION(JSONArrayLength)
.description="Returns the number of elements in the outermost JSON array. The function returns NULL if input JSON string is invalid."});
/// For Spark compatibility.
factory.registerAlias("JSON_ARRAY_LENGTH", "JSONArrayLength", FunctionFactory::CaseInsensitive);
factory.registerAlias("JSON_ARRAY_LENGTH", "JSONArrayLength", FunctionFactory::Case::Insensitive);
}
}

View File

@ -117,8 +117,8 @@ Example:
)",
.examples{
{"typical", "SELECT UTCTimestamp();", ""}},
.categories{"Dates and Times"}}, FunctionFactory::CaseInsensitive);
factory.registerAlias("UTC_timestamp", UTCTimestampOverloadResolver::name, FunctionFactory::CaseInsensitive);
.categories{"Dates and Times"}}, FunctionFactory::Case::Insensitive);
factory.registerAlias("UTC_timestamp", UTCTimestampOverloadResolver::name, FunctionFactory::Case::Insensitive);
}
}

View File

@ -144,8 +144,8 @@ REGISTER_FUNCTION(UTCTimestampTransform)
{
factory.registerFunction<ToUTCTimestampFunction>();
factory.registerFunction<FromUTCTimestampFunction>();
factory.registerAlias("to_utc_timestamp", NameToUTCTimestamp::name, FunctionFactory::CaseInsensitive);
factory.registerAlias("from_utc_timestamp", NameFromUTCTimestamp::name, FunctionFactory::CaseInsensitive);
factory.registerAlias("to_utc_timestamp", NameToUTCTimestamp::name, FunctionFactory::Case::Insensitive);
factory.registerAlias("from_utc_timestamp", NameFromUTCTimestamp::name, FunctionFactory::Case::Insensitive);
}
}

View File

@ -51,7 +51,7 @@ template <> struct FunctionUnaryArithmeticMonotonicity<NameAbs>
REGISTER_FUNCTION(Abs)
{
factory.registerFunction<FunctionAbs>({}, FunctionFactory::CaseInsensitive);
factory.registerFunction<FunctionAbs>({}, FunctionFactory::Case::Insensitive);
}
}

View File

@ -14,7 +14,7 @@ using FunctionAcos = FunctionMathUnary<UnaryFunctionVectorized<AcosName, acos>>;
REGISTER_FUNCTION(Acos)
{
factory.registerFunction<FunctionAcos>({}, FunctionFactory::CaseInsensitive);
factory.registerFunction<FunctionAcos>({}, FunctionFactory::Case::Insensitive);
}
}

View File

@ -123,7 +123,7 @@ private:
REGISTER_FUNCTION(ArrayFlatten)
{
factory.registerFunction<ArrayFlatten>();
factory.registerAlias("flatten", "arrayFlatten", FunctionFactory::CaseInsensitive);
factory.registerAlias("flatten", "arrayFlatten", FunctionFactory::Case::Insensitive);
}
}

View File

@ -196,7 +196,7 @@ It is possible to override the seed to produce stable results:
{"explicit_seed", "SELECT arrayShuffle([1, 2, 3, 4], 41)", ""},
{"materialize", "SELECT arrayShuffle(materialize([1, 2, 3]), 42), arrayShuffle([1, 2, 3], 42) FROM numbers(10)", ""}},
.categories{"Array"}},
FunctionFactory::CaseInsensitive);
FunctionFactory::Case::Insensitive);
factory.registerFunction<FunctionArrayShuffleImpl<FunctionArrayPartialShuffleTraits>>(
FunctionDocumentation{
@ -224,7 +224,7 @@ It is possible to override the seed to produce stable results:
{"materialize",
"SELECT arrayPartialShuffle(materialize([1, 2, 3, 4]), 2, 42), arrayPartialShuffle([1, 2, 3], 2, 42) FROM numbers(10)", ""}},
.categories{"Array"}},
FunctionFactory::CaseInsensitive);
FunctionFactory::Case::Insensitive);
}
}

View File

@ -100,8 +100,8 @@ It is ok to have ASCII NUL bytes in strings, and they will be counted as well.
},
.categories{"String", "Array"}
},
FunctionFactory::CaseInsensitive);
factory.registerAlias("OCTET_LENGTH", "length", FunctionFactory::CaseInsensitive);
FunctionFactory::Case::Insensitive);
factory.registerAlias("OCTET_LENGTH", "length", FunctionFactory::Case::Insensitive);
}
}

View File

@ -90,7 +90,7 @@ If s is empty, the result is 0. If the first character is not an ASCII character
)",
.examples{{"ascii", "SELECT ascii('234')", ""}},
.categories{"String"}
}, FunctionFactory::CaseInsensitive);
}, FunctionFactory::Case::Insensitive);
}
}

View File

@ -41,7 +41,7 @@ For more details, see [https://en.wikipedia.org/wiki/Inverse_trigonometric_funct
{"nan", "SELECT asin(1.1), asin(-2), asin(inf), asin(nan)", ""}},
.categories{"Mathematical", "Trigonometric"}
},
FunctionFactory::CaseInsensitive);
FunctionFactory::Case::Insensitive);
}
}

View File

@ -14,7 +14,7 @@ using FunctionAtan = FunctionMathUnary<UnaryFunctionVectorized<AtanName, atan>>;
REGISTER_FUNCTION(Atan)
{
factory.registerFunction<FunctionAtan>({}, FunctionFactory::CaseInsensitive);
factory.registerFunction<FunctionAtan>({}, FunctionFactory::Case::Insensitive);
}
}

View File

@ -15,7 +15,7 @@ namespace
REGISTER_FUNCTION(Atan2)
{
factory.registerFunction<FunctionAtan2>({}, FunctionFactory::CaseInsensitive);
factory.registerFunction<FunctionAtan2>({}, FunctionFactory::Case::Insensitive);
}
}

View File

@ -17,7 +17,7 @@ REGISTER_FUNCTION(Base64Decode)
factory.registerFunction<FunctionBase64Conversion<Base64Decode<Base64Variant::Normal>>>({description, syntax, arguments, returned_value, examples, categories});
/// MySQL compatibility alias.
factory.registerAlias("FROM_BASE64", "base64Decode", FunctionFactory::CaseInsensitive);
factory.registerAlias("FROM_BASE64", "base64Decode", FunctionFactory::Case::Insensitive);
}
}

View File

@ -17,7 +17,7 @@ REGISTER_FUNCTION(Base64Encode)
factory.registerFunction<FunctionBase64Conversion<Base64Encode<Base64Variant::Normal>>>({description, syntax, arguments, returned_value, examples, categories});
/// MySQL compatibility alias.
factory.registerAlias("TO_BASE64", "base64Encode", FunctionFactory::CaseInsensitive);
factory.registerAlias("TO_BASE64", "base64Encode", FunctionFactory::Case::Insensitive);
}
}

View File

@ -100,7 +100,7 @@ One use-case of this function is reversing IPv4s:
{"64-bit", "SELECT byteSwap(123294967295)", "18439412204227788800"},
},
.categories{"Mathematical", "Arithmetic"}},
FunctionFactory::CaseInsensitive);
FunctionFactory::Case::Insensitive);
}
}

View File

@ -180,7 +180,7 @@ private:
REGISTER_FUNCTION(Coalesce)
{
factory.registerFunction<FunctionCoalesce>({}, FunctionFactory::CaseInsensitive);
factory.registerFunction<FunctionCoalesce>({}, FunctionFactory::Case::Insensitive);
}
}

View File

@ -240,7 +240,7 @@ private:
REGISTER_FUNCTION(Concat)
{
factory.registerFunction<ConcatOverloadResolver>({}, FunctionFactory::CaseInsensitive);
factory.registerFunction<ConcatOverloadResolver>({}, FunctionFactory::Case::Insensitive);
factory.registerFunction<FunctionConcatAssumeInjective>();
}

View File

@ -193,7 +193,7 @@ The function is named “injective” if it always returns different result for
.categories{"String"}});
/// Compatibility with Spark and MySQL:
factory.registerAlias("concat_ws", "concatWithSeparator", FunctionFactory::CaseInsensitive);
factory.registerAlias("concat_ws", "concatWithSeparator", FunctionFactory::Case::Insensitive);
}
}

View File

@ -33,8 +33,8 @@ public:
REGISTER_FUNCTION(ConnectionId)
{
factory.registerFunction<FunctionConnectionId>({}, FunctionFactory::CaseInsensitive);
factory.registerAlias("connection_id", "connectionID", FunctionFactory::CaseInsensitive);
factory.registerFunction<FunctionConnectionId>({}, FunctionFactory::Case::Insensitive);
factory.registerAlias("connection_id", "connectionID", FunctionFactory::Case::Insensitive);
}
}

View File

@ -13,7 +13,7 @@ using FunctionCos = FunctionMathUnary<UnaryFunctionVectorized<CosName, cos>>;
REGISTER_FUNCTION(Cos)
{
factory.registerFunction<FunctionCos>({}, FunctionFactory::CaseInsensitive);
factory.registerFunction<FunctionCos>({}, FunctionFactory::Case::Insensitive);
}
}

View File

@ -22,8 +22,8 @@ namespace DB
REGISTER_FUNCTION(CountMatches)
{
factory.registerFunction<FunctionCountMatches<FunctionCountMatchesCaseSensitive>>({}, FunctionFactory::CaseSensitive);
factory.registerFunction<FunctionCountMatches<FunctionCountMatchesCaseInsensitive>>({}, FunctionFactory::CaseSensitive);
factory.registerFunction<FunctionCountMatches<FunctionCountMatchesCaseSensitive>>();
factory.registerFunction<FunctionCountMatches<FunctionCountMatchesCaseInsensitive>>();
}
}

View File

@ -19,6 +19,6 @@ using FunctionCountSubstrings = FunctionsStringSearch<CountSubstringsImpl<NameCo
REGISTER_FUNCTION(CountSubstrings)
{
factory.registerFunction<FunctionCountSubstrings>({}, FunctionFactory::CaseInsensitive);
factory.registerFunction<FunctionCountSubstrings>({}, FunctionFactory::Case::Insensitive);
}
}

View File

@ -54,9 +54,9 @@ public:
REGISTER_FUNCTION(CurrentDatabase)
{
factory.registerFunction<FunctionCurrentDatabase>();
factory.registerAlias("DATABASE", FunctionCurrentDatabase::name, FunctionFactory::CaseInsensitive);
factory.registerAlias("SCHEMA", FunctionCurrentDatabase::name, FunctionFactory::CaseInsensitive);
factory.registerAlias("current_database", FunctionCurrentDatabase::name, FunctionFactory::CaseInsensitive);
factory.registerAlias("DATABASE", FunctionCurrentDatabase::name, FunctionFactory::Case::Insensitive);
factory.registerAlias("SCHEMA", FunctionCurrentDatabase::name, FunctionFactory::Case::Insensitive);
factory.registerAlias("current_database", FunctionCurrentDatabase::name, FunctionFactory::Case::Insensitive);
}
}

View File

@ -80,8 +80,8 @@ Requires a boolean parameter, but it is ignored actually. It is required just fo
{"common", "SELECT current_schemas(true);", "['default']"}
}
},
FunctionFactory::CaseInsensitive);
factory.registerAlias("current_schemas", FunctionCurrentSchemas::name, FunctionFactory::CaseInsensitive);
FunctionFactory::Case::Insensitive);
factory.registerAlias("current_schemas", FunctionCurrentSchemas::name, FunctionFactory::Case::Insensitive);
}

View File

@ -54,8 +54,8 @@ public:
REGISTER_FUNCTION(CurrentUser)
{
factory.registerFunction<FunctionCurrentUser>();
factory.registerAlias("user", FunctionCurrentUser::name, FunctionFactory::CaseInsensitive);
factory.registerAlias("current_user", FunctionCurrentUser::name, FunctionFactory::CaseInsensitive);
factory.registerAlias("user", FunctionCurrentUser::name, FunctionFactory::Case::Insensitive);
factory.registerAlias("current_user", FunctionCurrentUser::name, FunctionFactory::Case::Insensitive);
}
}

View File

@ -490,7 +490,7 @@ private:
REGISTER_FUNCTION(DateDiff)
{
factory.registerFunction<FunctionDateDiff<true>>({}, FunctionFactory::CaseInsensitive);
factory.registerFunction<FunctionDateDiff<true>>({}, FunctionFactory::Case::Insensitive);
factory.registerAlias("date_diff", FunctionDateDiff<true>::name);
factory.registerAlias("DATE_DIFF", FunctionDateDiff<true>::name);
factory.registerAlias("timestampDiff", FunctionDateDiff<true>::name);
@ -509,12 +509,12 @@ Example:
)",
.examples{
{"typical", "SELECT timeDiff(UTCTimestamp(), now());", ""}},
.categories{"Dates and Times"}}, FunctionFactory::CaseInsensitive);
.categories{"Dates and Times"}}, FunctionFactory::Case::Insensitive);
}
REGISTER_FUNCTION(Age)
{
factory.registerFunction<FunctionDateDiff<false>>({}, FunctionFactory::CaseInsensitive);
factory.registerFunction<FunctionDateDiff<false>>({}, FunctionFactory::Case::Insensitive);
}
}

View File

@ -354,7 +354,7 @@ private:
REGISTER_FUNCTION(DateName)
{
factory.registerFunction<FunctionDateNameImpl>({}, FunctionFactory::CaseInsensitive);
factory.registerFunction<FunctionDateNameImpl>({}, FunctionFactory::Case::Insensitive);
}
}

View File

@ -178,7 +178,7 @@ REGISTER_FUNCTION(DateTrunc)
factory.registerFunction<FunctionDateTrunc>();
/// Compatibility alias.
factory.registerAlias("DATE_TRUNC", "dateTrunc", FunctionFactory::CaseInsensitive);
factory.registerAlias("DATE_TRUNC", "dateTrunc", FunctionFactory::Case::Insensitive);
}
}

View File

@ -23,7 +23,7 @@ namespace
REGISTER_FUNCTION(Degrees)
{
factory.registerFunction<FunctionDegrees>({}, FunctionFactory::CaseInsensitive);
factory.registerFunction<FunctionDegrees>({}, FunctionFactory::Case::Insensitive);
}
}

View File

@ -36,7 +36,7 @@ using FunctionExp = FunctionMathUnary<UnaryFunctionVectorized<ExpName, exp>>;
REGISTER_FUNCTION(Exp)
{
factory.registerFunction<FunctionExp>({}, FunctionFactory::CaseInsensitive);
factory.registerFunction<FunctionExp>({}, FunctionFactory::Case::Insensitive);
}
}

View File

@ -18,7 +18,7 @@ namespace DB
REGISTER_FUNCTION(ExtractAllGroupsVertical)
{
factory.registerFunction<FunctionExtractAllGroups<VerticalImpl>>();
factory.registerAlias("extractAllGroups", VerticalImpl::Name, FunctionFactory::CaseSensitive);
factory.registerAlias("extractAllGroups", VerticalImpl::Name);
}
}

View File

@ -106,7 +106,7 @@ The factorial of 0 is 1. Likewise, the factorial() function returns 1 for any ne
)",
.examples{{"factorial", "SELECT factorial(10)", ""}},
.categories{"Mathematical"}},
FunctionFactory::CaseInsensitive);
FunctionFactory::Case::Insensitive);
}
}

View File

@ -1834,10 +1834,10 @@ using FunctionFromUnixTimestampInJodaSyntax = FunctionFormatDateTimeImpl<NameFro
REGISTER_FUNCTION(FormatDateTime)
{
factory.registerFunction<FunctionFormatDateTime>();
factory.registerAlias("DATE_FORMAT", FunctionFormatDateTime::name, FunctionFactory::CaseInsensitive);
factory.registerAlias("DATE_FORMAT", FunctionFormatDateTime::name, FunctionFactory::Case::Insensitive);
factory.registerFunction<FunctionFromUnixTimestamp>();
factory.registerAlias("FROM_UNIXTIME", FunctionFromUnixTimestamp::name, FunctionFactory::CaseInsensitive);
factory.registerAlias("FROM_UNIXTIME", FunctionFromUnixTimestamp::name, FunctionFactory::Case::Insensitive);
factory.registerFunction<FunctionFormatDateTimeInJodaSyntax>();
factory.registerFunction<FunctionFromUnixTimestampInJodaSyntax>();

View File

@ -29,8 +29,7 @@ Accepts the size (number of bytes). Returns a rounded size with a suffix (KB, MB
.examples{
{"formatReadableDecimalSize", "SELECT formatReadableDecimalSize(1000)", ""}},
.categories{"OtherFunctions"}
},
FunctionFactory::CaseSensitive);
});
}
}

View File

@ -22,7 +22,7 @@ namespace
REGISTER_FUNCTION(FormatReadableSize)
{
factory.registerFunction<FunctionFormatReadable<Impl>>();
factory.registerAlias("FORMAT_BYTES", Impl::name, FunctionFactory::CaseInsensitive);
factory.registerAlias("FORMAT_BYTES", Impl::name, FunctionFactory::Case::Insensitive);
}
}

Some files were not shown because too many files have changed in this diff Show More