Merge branch 'master' into access-control-and-named-collections-by-default

This commit is contained in:
Alexey Milovidov 2023-11-12 00:56:06 +01:00
commit e207b1b14a
70 changed files with 1941 additions and 1374 deletions

View File

@ -1,3 +1,5 @@
# Generates a separate file with debug symbols while stripping it from the main binary.
# This is needed for Debian packages.
macro(clickhouse_split_debug_symbols)
set(oneValueArgs TARGET DESTINATION_DIR BINARY_PATH)

View File

@ -126,6 +126,7 @@ fi
mv ./programs/clickhouse* /output || mv ./programs/*_fuzzer /output
[ -x ./programs/self-extracting/clickhouse ] && mv ./programs/self-extracting/clickhouse /output
[ -x ./programs/self-extracting/clickhouse-stripped ] && mv ./programs/self-extracting/clickhouse-stripped /output
mv ./src/unit_tests_dbms /output ||: # may not exist for some binary builds
mv ./programs/*.dict ./programs/*.options ./programs/*_seed_corpus.zip /output ||: # libFuzzer oss-fuzz compatible infrastructure

View File

@ -15,8 +15,8 @@ CLICKHOUSE_CI_LOGS_USER=${CLICKHOUSE_CI_LOGS_USER:-ci}
# Pre-configured destination cluster, where to export the data
CLICKHOUSE_CI_LOGS_CLUSTER=${CLICKHOUSE_CI_LOGS_CLUSTER:-system_logs_export}
EXTRA_COLUMNS=${EXTRA_COLUMNS:-"pull_request_number UInt32, commit_sha String, check_start_time DateTime('UTC'), check_name String, instance_type String, instance_id String, "}
EXTRA_COLUMNS_EXPRESSION=${EXTRA_COLUMNS_EXPRESSION:-"CAST(0 AS UInt32) AS pull_request_number, '' AS commit_sha, now() AS check_start_time, '' AS check_name, '' AS instance_type, '' AS instance_id"}
EXTRA_COLUMNS=${EXTRA_COLUMNS:-"pull_request_number UInt32, commit_sha String, check_start_time DateTime('UTC'), check_name LowCardinality(String), instance_type LowCardinality(String), instance_id String, INDEX ix_pr (pull_request_number) TYPE set(100), INDEX ix_commit (commit_sha) TYPE set(100), INDEX ix_check_time (check_start_time) TYPE minmax, "}
EXTRA_COLUMNS_EXPRESSION=${EXTRA_COLUMNS_EXPRESSION:-"CAST(0 AS UInt32) AS pull_request_number, '' AS commit_sha, now() AS check_start_time, toLowCardinality('') AS check_name, toLowCardinality('') AS instance_type, '' AS instance_id"}
EXTRA_ORDER_BY_COLUMNS=${EXTRA_ORDER_BY_COLUMNS:-"check_name, "}
function __set_connection_args
@ -127,7 +127,7 @@ function setup_logs_replication
do
# Calculate hash of its structure. Note: 4 is the version of extra columns - increment it if extra columns are changed:
hash=$(clickhouse-client --query "
SELECT sipHash64(4, groupArray((name, type)))
SELECT sipHash64(7, groupArray((name, type)))
FROM (SELECT name, type FROM system.columns
WHERE database = 'system' AND table = '$table'
ORDER BY position)

View File

@ -438,7 +438,7 @@ $ curl -v 'http://localhost:8123/predefined_query'
< X-ClickHouse-Query-Id: 96fe0052-01e6-43ce-b12a-6b7370de6e8a
< X-ClickHouse-Format: Template
< X-ClickHouse-Timezone: Asia/Shanghai
< Keep-Alive: timeout=3
< Keep-Alive: timeout=10
< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334"}
<
# HELP "Query" "Number of executing queries"
@ -603,7 +603,7 @@ $ curl -vv -H 'XXX:xxx' 'http://localhost:8123/hi'
< Connection: Keep-Alive
< Content-Type: text/html; charset=UTF-8
< Transfer-Encoding: chunked
< Keep-Alive: timeout=3
< Keep-Alive: timeout=10
< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334"}
<
* Connection #0 to host localhost left intact
@ -643,7 +643,7 @@ $ curl -v -H 'XXX:xxx' 'http://localhost:8123/get_config_static_handler'
< Connection: Keep-Alive
< Content-Type: text/plain; charset=UTF-8
< Transfer-Encoding: chunked
< Keep-Alive: timeout=3
< Keep-Alive: timeout=10
< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334"}
<
* Connection #0 to host localhost left intact
@ -695,7 +695,7 @@ $ curl -vv -H 'XXX:xxx' 'http://localhost:8123/get_absolute_path_static_handler'
< Connection: Keep-Alive
< Content-Type: text/html; charset=UTF-8
< Transfer-Encoding: chunked
< Keep-Alive: timeout=3
< Keep-Alive: timeout=10
< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334"}
<
<html><body>Absolute Path File</body></html>
@ -714,7 +714,7 @@ $ curl -vv -H 'XXX:xxx' 'http://localhost:8123/get_relative_path_static_handler'
< Connection: Keep-Alive
< Content-Type: text/html; charset=UTF-8
< Transfer-Encoding: chunked
< Keep-Alive: timeout=3
< Keep-Alive: timeout=10
< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334"}
<
<html><body>Relative Path File</body></html>

View File

@ -74,6 +74,7 @@ ClickHouse Inc does **not** maintain the libraries listed below and hasnt don
### Elixir
- [clickhousex](https://github.com/appodeal/clickhousex/)
- [pillar](https://github.com/sofakingworld/pillar)
- [ecto_ch](https://github.com/plausible/ecto_ch)
### Nim
- [nim-clickhouse](https://github.com/leonardoce/nim-clickhouse)
### Haskell

View File

@ -214,7 +214,7 @@ Max consecutive resolving failures before dropping a host from ClickHouse DNS ca
Type: UInt32
Default: 1024
Default: 10
## index_mark_cache_policy

View File

@ -16,7 +16,7 @@ CREATE TABLE IF NOT EXISTS float_vs_decimal
my_decimal Decimal64(3)
)Engine=MergeTree ORDER BY tuple()
INSERT INTO float_vs_decimal SELECT round(canonicalRand(), 3) AS res, res FROM system.numbers LIMIT 1000000; # Generate 1 000 000 random number with 2 decimal places and store them as a float and as a decimal
INSERT INTO float_vs_decimal SELECT round(randCanonical(), 3) AS res, res FROM system.numbers LIMIT 1000000; # Generate 1 000 000 random number with 2 decimal places and store them as a float and as a decimal
SELECT sum(my_float), sum(my_decimal) FROM float_vs_decimal;
> 500279.56300000014 500279.563

View File

@ -366,7 +366,7 @@ $ curl -v 'http://localhost:8123/predefined_query'
< X-ClickHouse-Query-Id: 96fe0052-01e6-43ce-b12a-6b7370de6e8a
< X-ClickHouse-Format: Template
< X-ClickHouse-Timezone: Asia/Shanghai
< Keep-Alive: timeout=3
< Keep-Alive: timeout=10
< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"}
<
# HELP "Query" "Number of executing queries"
@ -529,7 +529,7 @@ $ curl -vv -H 'XXX:xxx' 'http://localhost:8123/hi'
< Connection: Keep-Alive
< Content-Type: text/html; charset=UTF-8
< Transfer-Encoding: chunked
< Keep-Alive: timeout=3
< Keep-Alive: timeout=10
< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334"}
<
* Connection #0 to host localhost left intact
@ -569,7 +569,7 @@ $ curl -v -H 'XXX:xxx' 'http://localhost:8123/get_config_static_handler'
< Connection: Keep-Alive
< Content-Type: text/plain; charset=UTF-8
< Transfer-Encoding: chunked
< Keep-Alive: timeout=3
< Keep-Alive: timeout=10
< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334"}
<
* Connection #0 to host localhost left intact
@ -621,7 +621,7 @@ $ curl -vv -H 'XXX:xxx' 'http://localhost:8123/get_absolute_path_static_handler'
< Connection: Keep-Alive
< Content-Type: text/html; charset=UTF-8
< Transfer-Encoding: chunked
< Keep-Alive: timeout=3
< Keep-Alive: timeout=10
< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334"}
<
<html><body>Absolute Path File</body></html>
@ -640,7 +640,7 @@ $ curl -vv -H 'XXX:xxx' 'http://localhost:8123/get_relative_path_static_handler'
< Connection: Keep-Alive
< Content-Type: text/html; charset=UTF-8
< Transfer-Encoding: chunked
< Keep-Alive: timeout=3
< Keep-Alive: timeout=10
< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334"}
<
<html><body>Relative Path File</body></html>

View File

@ -362,7 +362,7 @@ $ curl -v 'http://localhost:8123/predefined_query'
< X-ClickHouse-Query-Id: 96fe0052-01e6-43ce-b12a-6b7370de6e8a
< X-ClickHouse-Format: Template
< X-ClickHouse-Timezone: Asia/Shanghai
< Keep-Alive: timeout=3
< Keep-Alive: timeout=10
< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334"}
<
# HELP "Query" "Number of executing queries"
@ -520,7 +520,7 @@ $ curl -vv -H 'XXX:xxx' 'http://localhost:8123/hi'
< Connection: Keep-Alive
< Content-Type: text/html; charset=UTF-8
< Transfer-Encoding: chunked
< Keep-Alive: timeout=3
< Keep-Alive: timeout=10
< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334"}
<
* Connection #0 to host localhost left intact
@ -560,7 +560,7 @@ $ curl -v -H 'XXX:xxx' 'http://localhost:8123/get_config_static_handler'
< Connection: Keep-Alive
< Content-Type: text/plain; charset=UTF-8
< Transfer-Encoding: chunked
< Keep-Alive: timeout=3
< Keep-Alive: timeout=10
< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334"}
<
* Connection #0 to host localhost left intact
@ -612,7 +612,7 @@ $ curl -vv -H 'XXX:xxx' 'http://localhost:8123/get_absolute_path_static_handler'
< Connection: Keep-Alive
< Content-Type: text/html; charset=UTF-8
< Transfer-Encoding: chunked
< Keep-Alive: timeout=3
< Keep-Alive: timeout=10
< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334"}
<
<html><body>Absolute Path File</body></html>
@ -631,7 +631,7 @@ $ curl -vv -H 'XXX:xxx' 'http://localhost:8123/get_relative_path_static_handler'
< Connection: Keep-Alive
< Content-Type: text/html; charset=UTF-8
< Transfer-Encoding: chunked
< Keep-Alive: timeout=3
< Keep-Alive: timeout=10
< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334"}
<
<html><body>Relative Path File</body></html>

View File

@ -439,6 +439,13 @@ else()
install (TARGETS clickhouse RUNTIME DESTINATION ${CMAKE_INSTALL_BINDIR} COMPONENT clickhouse)
endif()
# A target to get stripped binary.
# Note: this is different to the above (extract debug symbols to a separate place)
add_custom_target(clickhouse-stripped ALL
COMMAND "${STRIP_PATH}" -o "${CMAKE_CURRENT_BINARY_DIR}/clickhouse-stripped" --strip-debug --remove-section=.comment --remove-section=.note "${CMAKE_CURRENT_BINARY_DIR}/clickhouse"
DEPENDS clickhouse
COMMENT "Stripping clickhouse binary" VERBATIM)
if (ENABLE_TESTS)
set (CLICKHOUSE_UNIT_TESTS_TARGETS unit_tests_dbms)
add_custom_target (clickhouse-tests ALL DEPENDS ${CLICKHOUSE_UNIT_TESTS_TARGETS})

View File

@ -11,8 +11,8 @@ else ()
endif ()
add_custom_target (self-extracting ALL
${CMAKE_COMMAND} -E remove clickhouse
${CMAKE_COMMAND} -E remove clickhouse clickhouse-stripped
COMMAND ${COMPRESSOR} ${DECOMPRESSOR} clickhouse ../clickhouse
DEPENDS clickhouse compressor
COMMAND ${COMPRESSOR} ${DECOMPRESSOR} clickhouse-stripped ../clickhouse-stripped
DEPENDS clickhouse clickhouse-stripped compressor
)

View File

@ -104,15 +104,14 @@
</url_scheme_mappers>
<!-- Add headers to response in options request. OPTIONS method is used in CORS preflight requests. -->
<!-- It is off by default. Next headers are obligate for CORS.-->
<!-- http_options_response>
<http_options_response>
<header>
<name>Access-Control-Allow-Origin</name>
<value>*</value>
</header>
<header>
<name>Access-Control-Allow-Headers</name>
<value>origin, x-requested-with</value>
<value>origin, x-requested-with, x-clickhouse-format, x-clickhouse-user, x-clickhouse-key, Authorization</value>
</header>
<header>
<name>Access-Control-Allow-Methods</name>
@ -122,7 +121,7 @@
<name>Access-Control-Max-Age</name>
<value>86400</value>
</header>
</http_options_response -->
</http_options_response>
<!-- It is the name that will be shown in the clickhouse-client.
By default, anything with "production" will be highlighted in red in query prompt.
@ -245,7 +244,7 @@
<max_connections>4096</max_connections>
<!-- For 'Connection: keep-alive' in HTTP 1.1 -->
<keep_alive_timeout>3</keep_alive_timeout>
<keep_alive_timeout>10</keep_alive_timeout>
<!-- gRPC protocol (see src/Server/grpc_protos/clickhouse_grpc.proto for the API) -->
<!-- <grpc_port>9100</grpc_port> -->

View File

@ -466,7 +466,7 @@ public:
std::vector<DataSet *> data_vec;
data_vec.resize(places.size());
for (unsigned long i = 0; i < data_vec.size(); i++)
for (size_t i = 0; i < data_vec.size(); ++i)
data_vec[i] = &this->data(places[i]).set;
DataSet::parallelizeMergePrepare(data_vec, thread_pool);

View File

@ -1467,9 +1467,15 @@ ProjectionName QueryAnalyzer::calculateFunctionProjectionName(const QueryTreeNod
const ProjectionNames & arguments_projection_names)
{
const auto & function_node_typed = function_node->as<FunctionNode &>();
const auto & function_node_name = function_node_typed.getFunctionName();
bool is_array_function = function_node_name == "array";
bool is_tuple_function = function_node_name == "tuple";
WriteBufferFromOwnString buffer;
buffer << function_node_typed.getFunctionName();
if (!is_array_function && !is_tuple_function)
buffer << function_node_name;
if (!parameters_projection_names.empty())
{
@ -1487,7 +1493,16 @@ ProjectionName QueryAnalyzer::calculateFunctionProjectionName(const QueryTreeNod
buffer << ')';
}
buffer << '(';
char open_bracket = '(';
char close_bracket = ')';
if (is_array_function)
{
open_bracket = '[';
close_bracket = ']';
}
buffer << open_bracket;
size_t function_arguments_projection_names_size = arguments_projection_names.size();
for (size_t i = 0; i < function_arguments_projection_names_size; ++i)
@ -1498,7 +1513,7 @@ ProjectionName QueryAnalyzer::calculateFunctionProjectionName(const QueryTreeNod
buffer << ", ";
}
buffer << ')';
buffer << close_bracket;
return buffer.str();
}

View File

@ -75,7 +75,7 @@ namespace DB
\
M(Bool, disable_internal_dns_cache, false, "Disable internal DNS caching at all.", 0) \
M(Int32, dns_cache_update_period, 15, "Internal DNS cache update period in seconds.", 0) \
M(UInt32, dns_max_consecutive_failures, 1024, "Max DNS resolve failures of a hostname before dropping the hostname from ClickHouse DNS cache.", 0) \
M(UInt32, dns_max_consecutive_failures, 10, "Max DNS resolve failures of a hostname before dropping the hostname from ClickHouse DNS cache.", 0) \
\
M(UInt64, max_table_size_to_drop, 50000000000lu, "If size of a table is greater than this value (in bytes) than table could not be dropped with any DROP query.", 0) \
M(UInt64, max_partition_size_to_drop, 50000000000lu, "Same as max_table_size_to_drop, but for the partitions.", 0) \

View File

@ -227,9 +227,7 @@ private:
struct KeyAttribute final
{
RangeStorageTypeContainer<KeyAttributeContainerType> container;
RangeStorageTypeContainer<InvalidIntervalsContainerType> invalid_intervals_container;
};
void createAttributes();

View File

@ -117,6 +117,9 @@ void DiskObjectStorageMetadata::serialize(WriteBuffer & buf, bool sync) const
if (storage_metadata_write_full_object_key)
write_version = VERSION_FULL_OBJECT_KEY;
if (!inline_data.empty() && write_version < VERSION_INLINE_DATA)
write_version = VERSION_INLINE_DATA;
chassert(write_version >= VERSION_ABSOLUTE_PATHS && write_version <= VERSION_FULL_OBJECT_KEY);
writeIntText(write_version, buf);
writeChar('\n', buf);
@ -153,8 +156,11 @@ void DiskObjectStorageMetadata::serialize(WriteBuffer & buf, bool sync) const
writeBoolText(read_only, buf);
writeChar('\n', buf);
writeEscapedString(inline_data, buf);
writeChar('\n', buf);
if (write_version >= VERSION_INLINE_DATA)
{
writeEscapedString(inline_data, buf);
writeChar('\n', buf);
}
buf.finalize();
if (sync)

View File

@ -19,7 +19,7 @@ private:
static constexpr UInt32 VERSION_INLINE_DATA = 4;
static constexpr UInt32 VERSION_FULL_OBJECT_KEY = 5; /// only for reading data
UInt32 version = VERSION_INLINE_DATA;
UInt32 version = VERSION_READ_ONLY_FLAG;
/// Absolute paths of blobs
ObjectKeysWithMetadata keys_with_meta;

View File

@ -9,6 +9,7 @@ extract_into_parent_list(clickhouse_functions_sources dbms_sources
FunctionHelpers.cpp
extractTimeZoneFromFunctionArguments.cpp
FunctionsLogical.cpp
CastOverloadResolver.cpp
)
extract_into_parent_list(clickhouse_functions_headers dbms_headers
IFunction.h
@ -16,6 +17,7 @@ extract_into_parent_list(clickhouse_functions_headers dbms_headers
FunctionHelpers.h
extractTimeZoneFromFunctionArguments.h
FunctionsLogical.h
CastOverloadResolver.h
)
add_library(clickhouse_functions_obj OBJECT ${clickhouse_functions_headers} ${clickhouse_functions_sources})

View File

@ -1,10 +1,156 @@
#include <Functions/FunctionFactory.h>
#include <Functions/FunctionsConversion.h>
#include <Functions/CastOverloadResolver.h>
#include <Interpreters/parseColumnsListForTableFunction.h>
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
}
/** CastInternal does not preserve nullability of the data type,
* i.e. CastInternal(toNullable(toInt8(1)) as Int32) will be Int32(1).
*
* Cast preserves nullability according to setting `cast_keep_nullable`,
* i.e. Cast(toNullable(toInt8(1)) as Int32) will be Nullable(Int32(1)) if `cast_keep_nullable` == 1.
*/
template <CastType cast_type, bool internal, typename CastName, typename FunctionName>
class CastOverloadResolverImpl : public IFunctionOverloadResolver
{
public:
using MonotonicityForRange = FunctionCastBase::MonotonicityForRange;
static constexpr auto name = cast_type == CastType::accurate
? CastName::accurate_cast_name
: (cast_type == CastType::accurateOrNull ? CastName::accurate_cast_or_null_name : CastName::cast_name);
String getName() const override { return name; }
size_t getNumberOfArguments() const override { return 2; }
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; }
explicit CastOverloadResolverImpl(ContextPtr context_, std::optional<CastDiagnostic> diagnostic_, bool keep_nullable_, const DataTypeValidationSettings & data_type_validation_settings_)
: context(context_)
, diagnostic(std::move(diagnostic_))
, keep_nullable(keep_nullable_)
, data_type_validation_settings(data_type_validation_settings_)
{
}
static FunctionOverloadResolverPtr create(ContextPtr context)
{
const auto & settings_ref = context->getSettingsRef();
if constexpr (internal)
return createImpl(context, {}, false /*keep_nullable*/);
return createImpl(context, {}, settings_ref.cast_keep_nullable, DataTypeValidationSettings(settings_ref));
}
static FunctionOverloadResolverPtr createImpl(ContextPtr context, std::optional<CastDiagnostic> diagnostic = {}, bool keep_nullable = false, const DataTypeValidationSettings & data_type_validation_settings = {})
{
assert(!internal || !keep_nullable);
return std::make_unique<CastOverloadResolverImpl>(context, std::move(diagnostic), keep_nullable, data_type_validation_settings);
}
static FunctionOverloadResolverPtr createImpl(std::optional<CastDiagnostic> diagnostic = {}, bool keep_nullable = false, const DataTypeValidationSettings & data_type_validation_settings = {})
{
assert(!internal || !keep_nullable);
return std::make_unique<CastOverloadResolverImpl>(ContextPtr(), std::move(diagnostic), keep_nullable, data_type_validation_settings);
}
protected:
FunctionBasePtr buildImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & return_type) const override
{
DataTypes data_types(arguments.size());
for (size_t i = 0; i < arguments.size(); ++i)
data_types[i] = arguments[i].type;
auto monotonicity = MonotonicityHelper::getMonotonicityInformation(arguments.front().type, return_type.get());
return std::make_unique<FunctionCast<FunctionName>>(context, name, std::move(monotonicity), data_types, return_type, diagnostic, cast_type);
}
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
{
const auto & column = arguments.back().column;
if (!column)
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Second argument to {} must be a constant string describing type. "
"Instead there is non-constant column of type {}", getName(), arguments.back().type->getName());
const auto * type_col = checkAndGetColumnConst<ColumnString>(column.get());
if (!type_col)
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Second argument to {} must be a constant string describing type. "
"Instead there is a column with the following structure: {}", getName(), column->dumpStructure());
DataTypePtr type = DataTypeFactory::instance().get(type_col->getValue<String>());
validateDataType(type, data_type_validation_settings);
if constexpr (cast_type == CastType::accurateOrNull)
return makeNullable(type);
if constexpr (internal)
return type;
if (keep_nullable && arguments.front().type->isNullable() && type->canBeInsideNullable())
return makeNullable(type);
return type;
}
bool useDefaultImplementationForNulls() const override { return false; }
bool useDefaultImplementationForNothing() const override { return false; }
bool useDefaultImplementationForLowCardinalityColumns() const override { return false; }
private:
ContextPtr context;
std::optional<CastDiagnostic> diagnostic;
bool keep_nullable;
DataTypeValidationSettings data_type_validation_settings;
};
struct CastOverloadName
{
static constexpr auto cast_name = "CAST";
static constexpr auto accurate_cast_name = "accurateCast";
static constexpr auto accurate_cast_or_null_name = "accurateCastOrNull";
};
struct CastInternalOverloadName
{
static constexpr auto cast_name = "_CAST";
static constexpr auto accurate_cast_name = "accurate_Cast";
static constexpr auto accurate_cast_or_null_name = "accurate_CastOrNull";
};
template <CastType cast_type>
using CastOverloadResolver = CastOverloadResolverImpl<cast_type, false, CastOverloadName, CastName>;
template <CastType cast_type>
using CastInternalOverloadResolver = CastOverloadResolverImpl<cast_type, true, CastInternalOverloadName, CastInternalName>;
FunctionOverloadResolverPtr createInternalCastOverloadResolver(CastType type, std::optional<CastDiagnostic> diagnostic)
{
switch (type)
{
case CastType::nonAccurate:
return CastInternalOverloadResolver<CastType::nonAccurate>::createImpl(diagnostic);
case CastType::accurate:
return CastInternalOverloadResolver<CastType::accurate>::createImpl(diagnostic);
case CastType::accurateOrNull:
return CastInternalOverloadResolver<CastType::accurateOrNull>::createImpl(diagnostic);
}
}
REGISTER_FUNCTION(CastOverloadResolvers)
{
factory.registerFunction<CastInternalOverloadResolver<CastType::nonAccurate>>({}, FunctionFactory::CaseInsensitive);

View File

@ -1,138 +1,29 @@
#pragma once
#include <Functions/FunctionsConversion.h>
#include <Interpreters/parseColumnsListForTableFunction.h>
#include <memory>
#include <optional>
#include <Interpreters/Context_fwd.h>
namespace DB
{
namespace ErrorCodes
class IFunctionOverloadResolver;
using FunctionOverloadResolverPtr = std::shared_ptr<IFunctionOverloadResolver>;
enum class CastType
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
}
/** CastInternal does not preserve nullability of the data type,
* i.e. CastInternal(toNullable(toInt8(1)) as Int32) will be Int32(1).
*
* Cast preserves nullability according to setting `cast_keep_nullable`,
* i.e. Cast(toNullable(toInt8(1)) as Int32) will be Nullable(Int32(1)) if `cast_keep_nullable` == 1.
*/
template <CastType cast_type, bool internal, typename CastName, typename FunctionName>
class CastOverloadResolverImpl : public IFunctionOverloadResolver
{
public:
using MonotonicityForRange = FunctionCastBase::MonotonicityForRange;
using Diagnostic = FunctionCastBase::Diagnostic;
static constexpr auto name = cast_type == CastType::accurate
? CastName::accurate_cast_name
: (cast_type == CastType::accurateOrNull ? CastName::accurate_cast_or_null_name : CastName::cast_name);
String getName() const override { return name; }
size_t getNumberOfArguments() const override { return 2; }
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; }
explicit CastOverloadResolverImpl(ContextPtr context_, std::optional<Diagnostic> diagnostic_, bool keep_nullable_, const DataTypeValidationSettings & data_type_validation_settings_)
: context(context_)
, diagnostic(std::move(diagnostic_))
, keep_nullable(keep_nullable_)
, data_type_validation_settings(data_type_validation_settings_)
{
}
static FunctionOverloadResolverPtr create(ContextPtr context)
{
const auto & settings_ref = context->getSettingsRef();
if constexpr (internal)
return createImpl(context, {}, false /*keep_nullable*/);
return createImpl(context, {}, settings_ref.cast_keep_nullable, DataTypeValidationSettings(settings_ref));
}
static FunctionOverloadResolverPtr createImpl(ContextPtr context, std::optional<Diagnostic> diagnostic = {}, bool keep_nullable = false, const DataTypeValidationSettings & data_type_validation_settings = {})
{
assert(!internal || !keep_nullable);
return std::make_unique<CastOverloadResolverImpl>(context, std::move(diagnostic), keep_nullable, data_type_validation_settings);
}
static FunctionOverloadResolverPtr createImpl(std::optional<Diagnostic> diagnostic = {}, bool keep_nullable = false, const DataTypeValidationSettings & data_type_validation_settings = {})
{
assert(!internal || !keep_nullable);
return std::make_unique<CastOverloadResolverImpl>(ContextPtr(), std::move(diagnostic), keep_nullable, data_type_validation_settings);
}
protected:
FunctionBasePtr buildImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & return_type) const override
{
DataTypes data_types(arguments.size());
for (size_t i = 0; i < arguments.size(); ++i)
data_types[i] = arguments[i].type;
auto monotonicity = MonotonicityHelper::getMonotonicityInformation(arguments.front().type, return_type.get());
return std::make_unique<FunctionCast<FunctionName>>(context, name, std::move(monotonicity), data_types, return_type, diagnostic, cast_type);
}
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
{
const auto & column = arguments.back().column;
if (!column)
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Second argument to {} must be a constant string describing type. "
"Instead there is non-constant column of type {}", getName(), arguments.back().type->getName());
const auto * type_col = checkAndGetColumnConst<ColumnString>(column.get());
if (!type_col)
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Second argument to {} must be a constant string describing type. "
"Instead there is a column with the following structure: {}", getName(), column->dumpStructure());
DataTypePtr type = DataTypeFactory::instance().get(type_col->getValue<String>());
validateDataType(type, data_type_validation_settings);
if constexpr (cast_type == CastType::accurateOrNull)
return makeNullable(type);
if constexpr (internal)
return type;
if (keep_nullable && arguments.front().type->isNullable() && type->canBeInsideNullable())
return makeNullable(type);
return type;
}
bool useDefaultImplementationForNulls() const override { return false; }
bool useDefaultImplementationForNothing() const override { return false; }
bool useDefaultImplementationForLowCardinalityColumns() const override { return false; }
private:
ContextPtr context;
std::optional<Diagnostic> diagnostic;
bool keep_nullable;
DataTypeValidationSettings data_type_validation_settings;
nonAccurate,
accurate,
accurateOrNull
};
struct CastOverloadName
struct CastDiagnostic
{
static constexpr auto cast_name = "CAST";
static constexpr auto accurate_cast_name = "accurateCast";
static constexpr auto accurate_cast_or_null_name = "accurateCastOrNull";
std::string column_from;
std::string column_to;
};
struct CastInternalOverloadName
{
static constexpr auto cast_name = "_CAST";
static constexpr auto accurate_cast_name = "accurate_Cast";
static constexpr auto accurate_cast_or_null_name = "accurate_CastOrNull";
};
template <CastType cast_type>
using CastOverloadResolver = CastOverloadResolverImpl<cast_type, false, CastOverloadName, CastName>;
template <CastType cast_type>
using CastInternalOverloadResolver = CastOverloadResolverImpl<cast_type, true, CastInternalOverloadName, CastInternalName>;
FunctionOverloadResolverPtr createInternalCastOverloadResolver(CastType type, std::optional<CastDiagnostic> diagnostic);
}

View File

@ -0,0 +1,211 @@
#pragma once
#include <Columns/ColumnArray.h>
#include <Columns/ColumnConst.h>
#include <Columns/ColumnFixedString.h>
#include <Columns/ColumnNullable.h>
#include <Columns/ColumnString.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypeNullable.h>
#include <DataTypes/DataTypeString.h>
#include <Functions/FunctionHelpers.h>
#include <Functions/IFunction.h>
#include <Functions/Regexps.h>
#include <Interpreters/Context.h>
#include <IO/WriteHelpers.h>
#include <Interpreters/castColumn.h>
#include <Common/StringUtils/StringUtils.h>
#include <Common/assert_cast.h>
#include <Common/typeid_cast.h>
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_COLUMN;
}
/** Functions that split strings into an array of strings or vice versa.
*
* splitByChar(sep, s[, max_substrings])
* splitByString(sep, s[, max_substrings])
* splitByRegexp(regexp, s[, max_substrings])
*
* splitByWhitespace(s[, max_substrings]) - split the string by whitespace characters
* splitByNonAlpha(s[, max_substrings]) - split the string by whitespace and punctuation characters
*
* extractAll(s, regexp) - select from the string the subsequences corresponding to the regexp.
* - first subpattern, if regexp has subpattern;
* - zero subpattern (the match part, otherwise);
* - otherwise, an empty array
*
* alphaTokens(s[, max_substrings]) - select from the string subsequence `[a-zA-Z]+`.
*
* URL functions are located separately.
*/
/// A function that takes a string, and returns an array of substrings created by some generator.
template <typename Generator>
class FunctionTokens : public IFunction
{
private:
using Pos = const char *;
bool max_substrings_includes_remaining_string;
public:
static constexpr auto name = Generator::name;
static FunctionPtr create(ContextPtr context) { return std::make_shared<FunctionTokens>(context); }
explicit FunctionTokens<Generator>(ContextPtr context)
{
const Settings & settings = context->getSettingsRef();
max_substrings_includes_remaining_string = settings.splitby_max_substrings_includes_remaining_string;
}
String getName() const override { return name; }
bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; }
bool isVariadic() const override { return Generator::isVariadic(); }
size_t getNumberOfArguments() const override { return Generator::getNumberOfArguments(); }
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
{
Generator::checkArguments(*this, arguments);
return std::make_shared<DataTypeArray>(std::make_shared<DataTypeString>());
}
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t /*input_rows_count*/) const override
{
Generator generator;
generator.init(arguments, max_substrings_includes_remaining_string);
const auto & array_argument = arguments[generator.strings_argument_position];
const ColumnString * col_str = checkAndGetColumn<ColumnString>(array_argument.column.get());
const ColumnConst * col_str_const = checkAndGetColumnConstStringOrFixedString(array_argument.column.get());
auto col_res = ColumnArray::create(ColumnString::create());
ColumnString & res_strings = typeid_cast<ColumnString &>(col_res->getData());
ColumnString::Chars & res_strings_chars = res_strings.getChars();
ColumnString::Offsets & res_strings_offsets = res_strings.getOffsets();
ColumnArray::Offsets & res_offsets = col_res->getOffsets();
if (col_str)
{
const ColumnString::Chars & src_chars = col_str->getChars();
const ColumnString::Offsets & src_offsets = col_str->getOffsets();
res_offsets.reserve(src_offsets.size());
res_strings_offsets.reserve(src_offsets.size() * 5); /// Constant 5 - at random.
res_strings_chars.reserve(src_chars.size());
Pos token_begin = nullptr;
Pos token_end = nullptr;
size_t size = src_offsets.size();
ColumnString::Offset current_src_offset = 0;
ColumnArray::Offset current_dst_offset = 0;
ColumnString::Offset current_dst_strings_offset = 0;
for (size_t i = 0; i < size; ++i)
{
Pos pos = reinterpret_cast<Pos>(&src_chars[current_src_offset]);
current_src_offset = src_offsets[i];
Pos end = reinterpret_cast<Pos>(&src_chars[current_src_offset]) - 1;
generator.set(pos, end);
size_t j = 0;
while (generator.get(token_begin, token_end))
{
size_t token_size = token_end - token_begin;
res_strings_chars.resize(res_strings_chars.size() + token_size + 1);
memcpySmallAllowReadWriteOverflow15(&res_strings_chars[current_dst_strings_offset], token_begin, token_size);
res_strings_chars[current_dst_strings_offset + token_size] = 0;
current_dst_strings_offset += token_size + 1;
res_strings_offsets.push_back(current_dst_strings_offset);
++j;
}
current_dst_offset += j;
res_offsets.push_back(current_dst_offset);
}
return col_res;
}
else if (col_str_const)
{
String src = col_str_const->getValue<String>();
Array dst;
generator.set(src.data(), src.data() + src.size());
Pos token_begin = nullptr;
Pos token_end = nullptr;
while (generator.get(token_begin, token_end))
dst.push_back(String(token_begin, token_end - token_begin));
return result_type->createColumnConst(col_str_const->size(), dst);
}
else
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal columns {}, {} of arguments of function {}",
array_argument.column->getName(), array_argument.column->getName(), getName());
}
};
/// Helper functions for implementations
static inline std::optional<size_t> extractMaxSplits(
const ColumnsWithTypeAndName & arguments, size_t max_substrings_argument_position)
{
if (max_substrings_argument_position >= arguments.size())
return std::nullopt;
if (const ColumnConst * column = checkAndGetColumn<ColumnConst>(arguments[max_substrings_argument_position].column.get()))
{
size_t res = column->getUInt(0);
if (res)
return res;
}
return std::nullopt;
}
static inline void checkArgumentsWithSeparatorAndOptionalMaxSubstrings(
const IFunction & func, const ColumnsWithTypeAndName & arguments)
{
FunctionArgumentDescriptors mandatory_args{
{"separator", &isString<IDataType>, isColumnConst, "const String"},
{"s", &isString<IDataType>, nullptr, "String"}
};
FunctionArgumentDescriptors optional_args{
{"max_substrings", &isNativeInteger<IDataType>, isColumnConst, "const Number"},
};
validateFunctionArgumentTypes(func, arguments, mandatory_args, optional_args);
}
static inline void checkArgumentsWithOptionalMaxSubstrings(const IFunction & func, const ColumnsWithTypeAndName & arguments)
{
FunctionArgumentDescriptors mandatory_args{
{"s", &isString<IDataType>, nullptr, "String"},
};
FunctionArgumentDescriptors optional_args{
{"max_substrings", &isNativeInteger<IDataType>, isColumnConst, "const Number"},
};
validateFunctionArgumentTypes(func, arguments, mandatory_args, optional_args);
}
}

View File

@ -53,6 +53,7 @@
#include <Functions/toFixedString.h>
#include <Functions/TransformDateTime64.h>
#include <Functions/FunctionsCodingIP.h>
#include <Functions/CastOverloadResolver.h>
#include <DataTypes/DataTypeLowCardinality.h>
#include <Columns/ColumnLowCardinality.h>
#include <Interpreters/Context.h>
@ -3127,14 +3128,8 @@ class ExecutableFunctionCast : public IExecutableFunction
public:
using WrapperType = std::function<ColumnPtr(ColumnsWithTypeAndName &, const DataTypePtr &, const ColumnNullable *, size_t)>;
struct Diagnostic
{
std::string column_from;
std::string column_to;
};
explicit ExecutableFunctionCast(
WrapperType && wrapper_function_, const char * name_, std::optional<Diagnostic> diagnostic_)
WrapperType && wrapper_function_, const char * name_, std::optional<CastDiagnostic> diagnostic_)
: wrapper_function(std::move(wrapper_function_)), name(name_), diagnostic(std::move(diagnostic_)) {}
String getName() const override { return name; }
@ -3170,24 +3165,16 @@ protected:
private:
WrapperType wrapper_function;
const char * name;
std::optional<Diagnostic> diagnostic;
std::optional<CastDiagnostic> diagnostic;
};
struct CastName { static constexpr auto name = "CAST"; };
struct CastInternalName { static constexpr auto name = "_CAST"; };
enum class CastType
{
nonAccurate,
accurate,
accurateOrNull
};
class FunctionCastBase : public IFunctionBase
{
public:
using MonotonicityForRange = std::function<Monotonicity(const IDataType &, const Field &, const Field &)>;
using Diagnostic = ExecutableFunctionCast::Diagnostic;
};
template <typename FunctionName>
@ -3201,7 +3188,7 @@ public:
, MonotonicityForRange && monotonicity_for_range_
, const DataTypes & argument_types_
, const DataTypePtr & return_type_
, std::optional<Diagnostic> diagnostic_
, std::optional<CastDiagnostic> diagnostic_
, CastType cast_type_)
: cast_name(cast_name_), monotonicity_for_range(std::move(monotonicity_for_range_))
, argument_types(argument_types_), return_type(return_type_), diagnostic(std::move(diagnostic_))
@ -3251,7 +3238,7 @@ private:
DataTypes argument_types;
DataTypePtr return_type;
std::optional<Diagnostic> diagnostic;
std::optional<CastDiagnostic> diagnostic;
CastType cast_type;
ContextPtr context;

View File

@ -1,73 +0,0 @@
#include <Functions/FunctionFactory.h>
#include <Functions/FunctionsStringArray.h>
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_COLUMN;
}
template <typename DataType>
std::optional<Int64> extractMaxSplitsImpl(const ColumnWithTypeAndName & argument)
{
const auto * col = checkAndGetColumnConst<ColumnVector<DataType>>(argument.column.get());
if (!col)
return std::nullopt;
auto value = col->template getValue<DataType>();
return static_cast<Int64>(value);
}
std::optional<size_t> extractMaxSplits(const ColumnsWithTypeAndName & arguments, size_t max_substrings_argument_position)
{
if (max_substrings_argument_position >= arguments.size())
return std::nullopt;
std::optional<Int64> max_splits;
if (!((max_splits = extractMaxSplitsImpl<UInt8>(arguments[max_substrings_argument_position])) || (max_splits = extractMaxSplitsImpl<Int8>(arguments[max_substrings_argument_position]))
|| (max_splits = extractMaxSplitsImpl<UInt16>(arguments[max_substrings_argument_position])) || (max_splits = extractMaxSplitsImpl<Int16>(arguments[max_substrings_argument_position]))
|| (max_splits = extractMaxSplitsImpl<UInt32>(arguments[max_substrings_argument_position])) || (max_splits = extractMaxSplitsImpl<Int32>(arguments[max_substrings_argument_position]))
|| (max_splits = extractMaxSplitsImpl<UInt64>(arguments[max_substrings_argument_position])) || (max_splits = extractMaxSplitsImpl<Int64>(arguments[max_substrings_argument_position]))))
throw Exception(
ErrorCodes::ILLEGAL_COLUMN,
"Illegal column {}, which is {}-th argument",
arguments[max_substrings_argument_position].column->getName(),
max_substrings_argument_position + 1);
if (*max_splits <= 0)
return std::nullopt;
return max_splits;
}
DataTypePtr FunctionArrayStringConcat::getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const
{
FunctionArgumentDescriptors mandatory_args{
{"arr", &isArray<IDataType>, nullptr, "Array"},
};
FunctionArgumentDescriptors optional_args{
{"separator", &isString<IDataType>, isColumnConst, "const String"},
};
validateFunctionArgumentTypes(*this, arguments, mandatory_args, optional_args);
return std::make_shared<DataTypeString>();
}
REGISTER_FUNCTION(StringArray)
{
factory.registerFunction<FunctionExtractAll>();
factory.registerFunction<FunctionSplitByAlpha>();
factory.registerAlias("splitByAlpha", FunctionSplitByAlpha::name);
factory.registerFunction<FunctionSplitByNonAlpha>();
factory.registerFunction<FunctionSplitByWhitespace>();
factory.registerFunction<FunctionSplitByChar>();
factory.registerFunction<FunctionSplitByString>();
factory.registerFunction<FunctionSplitByRegexp>();
factory.registerFunction<FunctionArrayStringConcat>();
}
}

View File

@ -1,990 +0,0 @@
#pragma once
#include <Columns/ColumnArray.h>
#include <Columns/ColumnConst.h>
#include <Columns/ColumnFixedString.h>
#include <Columns/ColumnNullable.h>
#include <Columns/ColumnString.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypeNullable.h>
#include <DataTypes/DataTypeString.h>
#include <Functions/FunctionHelpers.h>
#include <Functions/FunctionsConversion.h>
#include <Functions/IFunction.h>
#include <Functions/Regexps.h>
#include <IO/WriteHelpers.h>
#include <Interpreters/Context_fwd.h>
#include <Common/StringUtils/StringUtils.h>
#include <Common/assert_cast.h>
#include <Common/typeid_cast.h>
namespace DB
{
namespace ErrorCodes
{
extern const int BAD_ARGUMENTS;
extern const int ILLEGAL_COLUMN;
}
/** Functions that split strings into an array of strings or vice versa.
*
* splitByChar(sep, s[, max_substrings])
* splitByString(sep, s[, max_substrings])
* splitByRegexp(regexp, s[, max_substrings])
*
* splitByWhitespace(s[, max_substrings]) - split the string by whitespace characters
* splitByNonAlpha(s[, max_substrings]) - split the string by whitespace and punctuation characters
*
* extractAll(s, regexp) - select from the string the subsequences corresponding to the regexp.
* - first subpattern, if regexp has subpattern;
* - zero subpattern (the match part, otherwise);
* - otherwise, an empty array
*
* arrayStringConcat(arr)
* arrayStringConcat(arr, delimiter)
* - join an array of strings into one string via a separator.
*
* alphaTokens(s[, max_substrings]) - select from the string subsequence `[a-zA-Z]+`.
*
* URL functions are located separately.
*/
using Pos = const char *;
std::optional<size_t> extractMaxSplits(const ColumnsWithTypeAndName & arguments, size_t max_substrings_argument_position);
/// Substring generators. All of them have a common interface.
class SplitByAlphaImpl
{
private:
Pos pos;
Pos end;
std::optional<size_t> max_splits;
size_t splits;
bool max_substrings_includes_remaining_string;
public:
static constexpr auto name = "alphaTokens";
static String getName() { return name; }
static bool isVariadic() { return true; }
static size_t getNumberOfArguments() { return 0; }
static void checkArguments(const IFunction & func, const ColumnsWithTypeAndName & arguments)
{
FunctionArgumentDescriptors mandatory_args{
{"s", &isString<IDataType>, nullptr, "String"},
};
FunctionArgumentDescriptors optional_args{
{"max_substrings", &isNativeInteger<IDataType>, isColumnConst, "const Number"},
};
validateFunctionArgumentTypes(func, arguments, mandatory_args, optional_args);
}
static constexpr auto strings_argument_position = 0uz;
void init(const ColumnsWithTypeAndName & arguments, bool max_substrings_includes_remaining_string_)
{
max_substrings_includes_remaining_string = max_substrings_includes_remaining_string_;
max_splits = extractMaxSplits(arguments, 1);
}
/// Called for each next string.
void set(Pos pos_, Pos end_)
{
pos = pos_;
end = end_;
splits = 0;
}
/// Get the next token, if any, or return false.
bool get(Pos & token_begin, Pos & token_end)
{
/// Skip garbage
while (pos < end && !isAlphaASCII(*pos))
++pos;
if (pos == end)
return false;
token_begin = pos;
if (max_splits)
{
if (max_substrings_includes_remaining_string)
{
if (splits == *max_splits - 1)
{
token_end = end;
pos = end;
return true;
}
}
else
if (splits == *max_splits)
return false;
}
while (pos < end && isAlphaASCII(*pos))
++pos;
token_end = pos;
++splits;
return true;
}
};
class SplitByNonAlphaImpl
{
private:
Pos pos;
Pos end;
std::optional<size_t> max_splits;
size_t splits;
bool max_substrings_includes_remaining_string;
public:
/// Get the name of the function.
static constexpr auto name = "splitByNonAlpha";
static String getName() { return name; }
static bool isVariadic() { return true; }
static size_t getNumberOfArguments() { return 0; }
static void checkArguments(const IFunction & func, const ColumnsWithTypeAndName & arguments)
{
SplitByAlphaImpl::checkArguments(func, arguments);
}
static constexpr auto strings_argument_position = 0uz;
void init(const ColumnsWithTypeAndName & arguments, bool max_substrings_includes_remaining_string_)
{
max_substrings_includes_remaining_string = max_substrings_includes_remaining_string_;
max_splits = extractMaxSplits(arguments, 1);
}
/// Called for each next string.
void set(Pos pos_, Pos end_)
{
pos = pos_;
end = end_;
splits = 0;
}
/// Get the next token, if any, or return false.
bool get(Pos & token_begin, Pos & token_end)
{
/// Skip garbage
while (pos < end && (isWhitespaceASCII(*pos) || isPunctuationASCII(*pos)))
++pos;
if (pos == end)
return false;
token_begin = pos;
if (max_splits)
{
if (max_substrings_includes_remaining_string)
{
if (splits == *max_splits - 1)
{
token_end = end;
pos = end;
return true;
}
}
else
if (splits == *max_splits)
return false;
}
while (pos < end && !(isWhitespaceASCII(*pos) || isPunctuationASCII(*pos)))
++pos;
token_end = pos;
splits++;
return true;
}
};
class SplitByWhitespaceImpl
{
private:
Pos pos;
Pos end;
std::optional<size_t> max_splits;
size_t splits;
bool max_substrings_includes_remaining_string;
public:
static constexpr auto name = "splitByWhitespace";
static String getName() { return name; }
static bool isVariadic() { return true; }
static size_t getNumberOfArguments() { return 0; }
static void checkArguments(const IFunction & func, const ColumnsWithTypeAndName & arguments)
{
return SplitByNonAlphaImpl::checkArguments(func, arguments);
}
static constexpr auto strings_argument_position = 0uz;
void init(const ColumnsWithTypeAndName & arguments, bool max_substrings_includes_remaining_string_)
{
max_substrings_includes_remaining_string = max_substrings_includes_remaining_string_;
max_splits = extractMaxSplits(arguments, 1);
}
/// Called for each next string.
void set(Pos pos_, Pos end_)
{
pos = pos_;
end = end_;
splits = 0;
}
/// Get the next token, if any, or return false.
bool get(Pos & token_begin, Pos & token_end)
{
/// Skip garbage
while (pos < end && isWhitespaceASCII(*pos))
++pos;
if (pos == end)
return false;
token_begin = pos;
if (max_splits)
{
if (max_substrings_includes_remaining_string)
{
if (splits == *max_splits - 1)
{
token_end = end;
pos = end;
return true;
}
}
else
if (splits == *max_splits)
return false;
}
while (pos < end && !isWhitespaceASCII(*pos))
++pos;
token_end = pos;
splits++;
return true;
}
};
class SplitByCharImpl
{
private:
Pos pos;
Pos end;
char separator;
std::optional<size_t> max_splits;
size_t splits;
bool max_substrings_includes_remaining_string;
public:
static constexpr auto name = "splitByChar";
static String getName() { return name; }
static bool isVariadic() { return true; }
static size_t getNumberOfArguments() { return 0; }
static void checkArguments(const IFunction & func, const ColumnsWithTypeAndName & arguments)
{
FunctionArgumentDescriptors mandatory_args{
{"separator", &isString<IDataType>, isColumnConst, "const String"},
{"s", &isString<IDataType>, nullptr, "String"}
};
FunctionArgumentDescriptors optional_args{
{"max_substrings", &isNativeInteger<IDataType>, isColumnConst, "const Number"},
};
validateFunctionArgumentTypes(func, arguments, mandatory_args, optional_args);
}
static constexpr auto strings_argument_position = 1uz;
void init(const ColumnsWithTypeAndName & arguments, bool max_substrings_includes_remaining_string_)
{
const ColumnConst * col = checkAndGetColumnConstStringOrFixedString(arguments[0].column.get());
if (!col)
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of first argument of function {}. "
"Must be constant string.", arguments[0].column->getName(), getName());
String sep_str = col->getValue<String>();
if (sep_str.size() != 1)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Illegal separator for function {}. Must be exactly one byte.", getName());
separator = sep_str[0];
max_substrings_includes_remaining_string = max_substrings_includes_remaining_string_;
max_splits = extractMaxSplits(arguments, 2);
}
void set(Pos pos_, Pos end_)
{
pos = pos_;
end = end_;
splits = 0;
}
bool get(Pos & token_begin, Pos & token_end)
{
if (!pos)
return false;
token_begin = pos;
if (max_splits)
{
if (max_substrings_includes_remaining_string)
{
if (splits == *max_splits - 1)
{
token_end = end;
pos = nullptr;
return true;
}
}
else
if (splits == *max_splits)
return false;
}
pos = reinterpret_cast<Pos>(memchr(pos, separator, end - pos));
if (pos)
{
token_end = pos;
++pos;
++splits;
}
else
token_end = end;
return true;
}
};
class SplitByStringImpl
{
private:
Pos pos;
Pos end;
String separator;
std::optional<size_t> max_splits;
size_t splits;
bool max_substrings_includes_remaining_string;
public:
static constexpr auto name = "splitByString";
static String getName() { return name; }
static bool isVariadic() { return true; }
static size_t getNumberOfArguments() { return 0; }
static void checkArguments(const IFunction & func, const ColumnsWithTypeAndName & arguments)
{
SplitByCharImpl::checkArguments(func, arguments);
}
static constexpr auto strings_argument_position = 1uz;
void init(const ColumnsWithTypeAndName & arguments, bool max_substrings_includes_remaining_string_)
{
const ColumnConst * col = checkAndGetColumnConstStringOrFixedString(arguments[0].column.get());
if (!col)
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of first argument of function {}. "
"Must be constant string.", arguments[0].column->getName(), getName());
separator = col->getValue<String>();
max_substrings_includes_remaining_string = max_substrings_includes_remaining_string_;
max_splits = extractMaxSplits(arguments, 2);
}
/// Called for each next string.
void set(Pos pos_, Pos end_)
{
pos = pos_;
end = end_;
splits = 0;
}
/// Get the next token, if any, or return false.
bool get(Pos & token_begin, Pos & token_end)
{
if (separator.empty())
{
if (pos == end)
return false;
token_begin = pos;
if (max_splits)
{
if (max_substrings_includes_remaining_string)
{
if (splits == *max_splits - 1)
{
token_end = end;
pos = end;
return true;
}
}
else
if (splits == *max_splits)
return false;
}
pos += 1;
token_end = pos;
++splits;
}
else
{
if (!pos)
return false;
token_begin = pos;
if (max_splits)
{
if (max_substrings_includes_remaining_string)
{
if (splits == *max_splits - 1)
{
token_end = end;
pos = nullptr;
return true;
}
}
else
if (splits == *max_splits)
return false;
}
pos = reinterpret_cast<Pos>(memmem(pos, end - pos, separator.data(), separator.size()));
if (pos)
{
token_end = pos;
pos += separator.size();
++splits;
}
else
token_end = end;
}
return true;
}
};
class SplitByRegexpImpl
{
private:
Regexps::RegexpPtr re;
OptimizedRegularExpression::MatchVec matches;
Pos pos;
Pos end;
std::optional<size_t> max_splits;
size_t splits;
bool max_substrings_includes_remaining_string;
public:
static constexpr auto name = "splitByRegexp";
static String getName() { return name; }
static bool isVariadic() { return true; }
static size_t getNumberOfArguments() { return 0; }
static void checkArguments(const IFunction & func, const ColumnsWithTypeAndName & arguments)
{
SplitByStringImpl::checkArguments(func, arguments);
}
static constexpr auto strings_argument_position = 1uz;
void init(const ColumnsWithTypeAndName & arguments, bool max_substrings_includes_remaining_string_)
{
const ColumnConst * col = checkAndGetColumnConstStringOrFixedString(arguments[0].column.get());
if (!col)
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of first argument of function {}. "
"Must be constant string.", arguments[0].column->getName(), getName());
if (!col->getValue<String>().empty())
re = std::make_shared<OptimizedRegularExpression>(Regexps::createRegexp<false, false, false>(col->getValue<String>()));
max_substrings_includes_remaining_string = max_substrings_includes_remaining_string_;
max_splits = extractMaxSplits(arguments, 2);
}
/// Called for each next string.
void set(Pos pos_, Pos end_)
{
pos = pos_;
end = end_;
splits = 0;
}
/// Get the next token, if any, or return false.
bool get(Pos & token_begin, Pos & token_end)
{
if (!re)
{
if (pos == end)
return false;
token_begin = pos;
if (max_splits)
{
if (max_substrings_includes_remaining_string)
{
if (splits == *max_splits - 1)
{
token_end = end;
pos = end;
return true;
}
}
else
if (splits == *max_splits)
return false;
}
pos += 1;
token_end = pos;
++splits;
}
else
{
if (!pos || pos > end)
return false;
token_begin = pos;
if (max_splits)
{
if (max_substrings_includes_remaining_string)
{
if (splits == *max_splits - 1)
{
token_end = end;
pos = nullptr;
return true;
}
}
else
if (splits == *max_splits)
return false;
}
if (!re->match(pos, end - pos, matches) || !matches[0].length)
{
token_end = end;
pos = end + 1;
}
else
{
token_end = pos + matches[0].offset;
pos = token_end + matches[0].length;
++splits;
}
}
return true;
}
};
class ExtractAllImpl
{
private:
Regexps::RegexpPtr re;
OptimizedRegularExpression::MatchVec matches;
size_t capture;
Pos pos;
Pos end;
public:
static constexpr auto name = "extractAll";
static String getName() { return name; }
static bool isVariadic() { return false; }
static size_t getNumberOfArguments() { return 2; }
static void checkArguments(const IFunction & func, const ColumnsWithTypeAndName & arguments)
{
FunctionArgumentDescriptors mandatory_args{
{"haystack", &isString<IDataType>, nullptr, "String"},
{"pattern", &isString<IDataType>, isColumnConst, "const String"}
};
validateFunctionArgumentTypes(func, arguments, mandatory_args);
}
static constexpr auto strings_argument_position = 0uz;
void init(const ColumnsWithTypeAndName & arguments, bool /*max_substrings_includes_remaining_string*/)
{
const ColumnConst * col = checkAndGetColumnConstStringOrFixedString(arguments[1].column.get());
if (!col)
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of first argument of function {}. "
"Must be constant string.", arguments[1].column->getName(), getName());
re = std::make_shared<OptimizedRegularExpression>(Regexps::createRegexp<false, false, false>(col->getValue<String>()));
capture = re->getNumberOfSubpatterns() > 0 ? 1 : 0;
matches.resize(capture + 1);
}
/// Called for each next string.
void set(Pos pos_, Pos end_)
{
pos = pos_;
end = end_;
}
/// Get the next token, if any, or return false.
bool get(Pos & token_begin, Pos & token_end)
{
if (!pos || pos > end)
return false;
if (!re->match(pos, end - pos, matches) || !matches[0].length)
return false;
if (matches[capture].offset == std::string::npos)
{
/// Empty match.
token_begin = pos;
token_end = pos;
}
else
{
token_begin = pos + matches[capture].offset;
token_end = token_begin + matches[capture].length;
}
pos += matches[0].offset + matches[0].length;
return true;
}
};
/// A function that takes a string, and returns an array of substrings created by some generator.
template <typename Generator>
class FunctionTokens : public IFunction
{
private:
bool max_substrings_includes_remaining_string;
public:
static constexpr auto name = Generator::name;
static FunctionPtr create(ContextPtr context) { return std::make_shared<FunctionTokens>(context); }
explicit FunctionTokens<Generator>(ContextPtr context)
{
const Settings & settings = context->getSettingsRef();
max_substrings_includes_remaining_string = settings.splitby_max_substrings_includes_remaining_string;
}
String getName() const override { return name; }
bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; }
bool isVariadic() const override { return Generator::isVariadic(); }
size_t getNumberOfArguments() const override { return Generator::getNumberOfArguments(); }
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
{
Generator::checkArguments(*this, arguments);
return std::make_shared<DataTypeArray>(std::make_shared<DataTypeString>());
}
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t /*input_rows_count*/) const override
{
Generator generator;
generator.init(arguments, max_substrings_includes_remaining_string);
const auto & array_argument = arguments[generator.strings_argument_position];
const ColumnString * col_str = checkAndGetColumn<ColumnString>(array_argument.column.get());
const ColumnConst * col_str_const = checkAndGetColumnConstStringOrFixedString(array_argument.column.get());
auto col_res = ColumnArray::create(ColumnString::create());
ColumnString & res_strings = typeid_cast<ColumnString &>(col_res->getData());
ColumnString::Chars & res_strings_chars = res_strings.getChars();
ColumnString::Offsets & res_strings_offsets = res_strings.getOffsets();
ColumnArray::Offsets & res_offsets = col_res->getOffsets();
if (col_str)
{
const ColumnString::Chars & src_chars = col_str->getChars();
const ColumnString::Offsets & src_offsets = col_str->getOffsets();
res_offsets.reserve(src_offsets.size());
res_strings_offsets.reserve(src_offsets.size() * 5); /// Constant 5 - at random.
res_strings_chars.reserve(src_chars.size());
Pos token_begin = nullptr;
Pos token_end = nullptr;
size_t size = src_offsets.size();
ColumnString::Offset current_src_offset = 0;
ColumnArray::Offset current_dst_offset = 0;
ColumnString::Offset current_dst_strings_offset = 0;
for (size_t i = 0; i < size; ++i)
{
Pos pos = reinterpret_cast<Pos>(&src_chars[current_src_offset]);
current_src_offset = src_offsets[i];
Pos end = reinterpret_cast<Pos>(&src_chars[current_src_offset]) - 1;
generator.set(pos, end);
size_t j = 0;
while (generator.get(token_begin, token_end))
{
size_t token_size = token_end - token_begin;
res_strings_chars.resize(res_strings_chars.size() + token_size + 1);
memcpySmallAllowReadWriteOverflow15(&res_strings_chars[current_dst_strings_offset], token_begin, token_size);
res_strings_chars[current_dst_strings_offset + token_size] = 0;
current_dst_strings_offset += token_size + 1;
res_strings_offsets.push_back(current_dst_strings_offset);
++j;
}
current_dst_offset += j;
res_offsets.push_back(current_dst_offset);
}
return col_res;
}
else if (col_str_const)
{
String src = col_str_const->getValue<String>();
Array dst;
generator.set(src.data(), src.data() + src.size());
Pos token_begin = nullptr;
Pos token_end = nullptr;
while (generator.get(token_begin, token_end))
dst.push_back(String(token_begin, token_end - token_begin));
return result_type->createColumnConst(col_str_const->size(), dst);
}
else
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal columns {}, {} of arguments of function {}",
array_argument.column->getName(), array_argument.column->getName(), getName());
}
};
/// Joins an array of type serializable to string into one string via a separator.
class FunctionArrayStringConcat : public IFunction
{
private:
static void executeInternal(
const ColumnString::Chars & src_chars,
const ColumnString::Offsets & src_string_offsets,
const ColumnArray::Offsets & src_array_offsets,
const char * delimiter,
const size_t delimiter_size,
ColumnString::Chars & dst_chars,
ColumnString::Offsets & dst_string_offsets,
const char8_t * null_map)
{
size_t size = src_array_offsets.size();
if (!size)
return;
/// With a small margin - as if the separator goes after the last string of the array.
dst_chars.resize(
src_chars.size()
+ delimiter_size * src_string_offsets.size() /// Separators after each string...
+ src_array_offsets.size() /// Zero byte after each joined string
- src_string_offsets.size()); /// The former zero byte after each string of the array
/// There will be as many strings as there were arrays.
dst_string_offsets.resize(src_array_offsets.size());
ColumnArray::Offset current_src_array_offset = 0;
ColumnString::Offset current_dst_string_offset = 0;
/// Loop through the array of strings.
for (size_t i = 0; i < size; ++i)
{
bool first_non_null = true;
/// Loop through the rows within the array. /// NOTE You can do everything in one copy, if the separator has a size of 1.
for (auto next_src_array_offset = src_array_offsets[i]; current_src_array_offset < next_src_array_offset; ++current_src_array_offset)
{
if (null_map && null_map[current_src_array_offset]) [[unlikely]]
continue;
if (!first_non_null)
{
memcpy(&dst_chars[current_dst_string_offset], delimiter, delimiter_size);
current_dst_string_offset += delimiter_size;
}
first_non_null = false;
const auto current_src_string_offset = current_src_array_offset ? src_string_offsets[current_src_array_offset - 1] : 0;
size_t bytes_to_copy = src_string_offsets[current_src_array_offset] - current_src_string_offset - 1;
memcpySmallAllowReadWriteOverflow15(
&dst_chars[current_dst_string_offset], &src_chars[current_src_string_offset], bytes_to_copy);
current_dst_string_offset += bytes_to_copy;
}
dst_chars[current_dst_string_offset] = 0;
++current_dst_string_offset;
dst_string_offsets[i] = current_dst_string_offset;
}
dst_chars.resize(dst_string_offsets.back());
}
static void executeInternal(
const ColumnString & col_string,
const ColumnArray & col_arr,
const String & delimiter,
ColumnString & col_res,
const char8_t * null_map = nullptr)
{
executeInternal(
col_string.getChars(),
col_string.getOffsets(),
col_arr.getOffsets(),
delimiter.data(),
delimiter.size(),
col_res.getChars(),
col_res.getOffsets(),
null_map);
}
static ColumnPtr serializeNestedColumn(const ColumnArray & col_arr, const DataTypePtr & nested_type)
{
if (isString(nested_type))
{
return col_arr.getDataPtr();
}
else if (const ColumnNullable * col_nullable = checkAndGetColumn<ColumnNullable>(col_arr.getData());
col_nullable && isString(col_nullable->getNestedColumn().getDataType()))
{
return col_nullable->getNestedColumnPtr();
}
else
{
ColumnsWithTypeAndName cols;
cols.emplace_back(col_arr.getDataPtr(), nested_type, "tmp");
return ConvertImplGenericToString<ColumnString>::execute(cols, std::make_shared<DataTypeString>(), col_arr.size());
}
}
public:
static constexpr auto name = "arrayStringConcat";
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionArrayStringConcat>(); }
String getName() const override
{
return name;
}
bool isVariadic() const override { return true; }
bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; }
size_t getNumberOfArguments() const override { return 0; }
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override;
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t /*input_rows_count*/) const override
{
String delimiter;
if (arguments.size() == 2)
{
const ColumnConst * col_delim = checkAndGetColumnConstStringOrFixedString(arguments[1].column.get());
if (!col_delim)
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Second argument for function {} must be constant string.", getName());
delimiter = col_delim->getValue<String>();
}
const auto & nested_type = assert_cast<const DataTypeArray &>(*arguments[0].type).getNestedType();
if (const ColumnConst * col_const_arr = checkAndGetColumnConst<ColumnArray>(arguments[0].column.get());
col_const_arr && isString(nested_type))
{
Array src_arr = col_const_arr->getValue<Array>();
String dst_str;
bool first_non_null = true;
for (size_t i = 0, size = src_arr.size(); i < size; ++i)
{
if (src_arr[i].isNull())
continue;
if (!first_non_null)
dst_str += delimiter;
first_non_null = false;
dst_str += src_arr[i].get<const String &>();
}
return result_type->createColumnConst(col_const_arr->size(), dst_str);
}
ColumnPtr src_column = arguments[0].column->convertToFullColumnIfConst();
const ColumnArray & col_arr = assert_cast<const ColumnArray &>(*src_column.get());
ColumnPtr str_subcolumn = serializeNestedColumn(col_arr, nested_type);
const ColumnString & col_string = assert_cast<const ColumnString &>(*str_subcolumn.get());
auto col_res = ColumnString::create();
if (const ColumnNullable * col_nullable = checkAndGetColumn<ColumnNullable>(col_arr.getData()))
executeInternal(col_string, col_arr, delimiter, *col_res, col_nullable->getNullMapData().data());
else
executeInternal(col_string, col_arr, delimiter, *col_res);
return col_res;
}
};
using FunctionSplitByAlpha = FunctionTokens<SplitByAlphaImpl>;
using FunctionSplitByNonAlpha = FunctionTokens<SplitByNonAlphaImpl>;
using FunctionSplitByWhitespace = FunctionTokens<SplitByWhitespaceImpl>;
using FunctionSplitByChar = FunctionTokens<SplitByCharImpl>;
using FunctionSplitByString = FunctionTokens<SplitByStringImpl>;
using FunctionSplitByRegexp = FunctionTokens<SplitByRegexpImpl>;
using FunctionExtractAll = FunctionTokens<ExtractAllImpl>;
}

View File

@ -1,9 +1,15 @@
#include <Functions/FunctionFactory.h>
#include <Functions/FunctionsStringArray.h>
#include <Functions/FunctionTokens.h>
namespace DB
{
namespace
{
using Pos = const char *;
class URLPathHierarchyImpl
{
private:
@ -14,7 +20,6 @@ private:
public:
static constexpr auto name = "URLPathHierarchy";
static String getName() { return name; }
static bool isVariadic() { return false; }
static size_t getNumberOfArguments() { return 1; }
@ -95,9 +100,10 @@ public:
};
struct NameURLPathHierarchy { static constexpr auto name = "URLPathHierarchy"; };
using FunctionURLPathHierarchy = FunctionTokens<URLPathHierarchyImpl>;
}
REGISTER_FUNCTION(URLPathHierarchy)
{
factory.registerFunction<FunctionURLPathHierarchy>();

View File

@ -1,9 +1,14 @@
#include <Functions/FunctionFactory.h>
#include <Functions/FunctionsStringArray.h>
#include <Functions/FunctionTokens.h>
namespace DB
{
namespace
{
using Pos = const char *;
class URLHierarchyImpl
{
private:
@ -13,7 +18,6 @@ private:
public:
static constexpr auto name = "URLHierarchy";
static String getName() { return name; }
static bool isVariadic() { return false; }
static size_t getNumberOfArguments() { return 1; }
@ -97,9 +101,10 @@ public:
};
struct NameURLHierarchy { static constexpr auto name = "URLHierarchy"; };
using FunctionURLHierarchy = FunctionTokens<URLHierarchyImpl>;
}
REGISTER_FUNCTION(URLHierarchy)
{
factory.registerFunction<FunctionURLHierarchy>();

View File

@ -1,9 +1,14 @@
#include <Functions/FunctionFactory.h>
#include <Functions/FunctionsStringArray.h>
#include <Functions/FunctionTokens.h>
namespace DB
{
namespace
{
using Pos = const char *;
class ExtractURLParameterNamesImpl
{
private:
@ -13,7 +18,6 @@ private:
public:
static constexpr auto name = "extractURLParameterNames";
static String getName() { return name; }
static bool isVariadic() { return false; }
static size_t getNumberOfArguments() { return 1; }
@ -80,9 +84,10 @@ public:
}
};
struct NameExtractURLParameterNames { static constexpr auto name = "extractURLParameterNames"; };
using FunctionExtractURLParameterNames = FunctionTokens<ExtractURLParameterNamesImpl>;
}
REGISTER_FUNCTION(ExtractURLParameterNames)
{
factory.registerFunction<FunctionExtractURLParameterNames>();

View File

@ -1,9 +1,15 @@
#include <Functions/FunctionFactory.h>
#include <Functions/FunctionsStringArray.h>
#include <Functions/FunctionTokens.h>
namespace DB
{
namespace
{
using Pos = const char *;
class ExtractURLParametersImpl
{
private:
@ -13,7 +19,6 @@ private:
public:
static constexpr auto name = "extractURLParameters";
static String getName() { return name; }
static bool isVariadic() { return false; }
static size_t getNumberOfArguments() { return 1; }
@ -88,9 +93,10 @@ public:
}
};
struct NameExtractURLParameters { static constexpr auto name = "extractURLParameters"; };
using FunctionExtractURLParameters = FunctionTokens<ExtractURLParametersImpl>;
}
REGISTER_FUNCTION(ExtractURLParameters)
{
factory.registerFunction<FunctionExtractURLParameters>();

View File

@ -0,0 +1,104 @@
#include <Functions/FunctionTokens.h>
#include <Functions/FunctionFactory.h>
#include <Common/StringUtils/StringUtils.h>
namespace DB
{
/** Functions that split strings into an array of strings or vice versa.
*
* alphaTokens(s[, max_substrings]) - select from the string subsequence `[a-zA-Z]+`.
*/
namespace
{
using Pos = const char *;
class SplitByAlphaImpl
{
private:
Pos pos;
Pos end;
std::optional<size_t> max_splits;
size_t splits;
bool max_substrings_includes_remaining_string;
public:
static constexpr auto name = "alphaTokens";
static bool isVariadic() { return true; }
static size_t getNumberOfArguments() { return 0; }
static void checkArguments(const IFunction & func, const ColumnsWithTypeAndName & arguments)
{
checkArgumentsWithOptionalMaxSubstrings(func, arguments);
}
static constexpr auto strings_argument_position = 0uz;
void init(const ColumnsWithTypeAndName & arguments, bool max_substrings_includes_remaining_string_)
{
max_substrings_includes_remaining_string = max_substrings_includes_remaining_string_;
max_splits = extractMaxSplits(arguments, 1);
}
/// Called for each next string.
void set(Pos pos_, Pos end_)
{
pos = pos_;
end = end_;
splits = 0;
}
/// Get the next token, if any, or return false.
bool get(Pos & token_begin, Pos & token_end)
{
/// Skip garbage
while (pos < end && !isAlphaASCII(*pos))
++pos;
if (pos == end)
return false;
token_begin = pos;
if (max_splits)
{
if (max_substrings_includes_remaining_string)
{
if (splits == *max_splits - 1)
{
token_end = end;
pos = end;
return true;
}
}
else
if (splits == *max_splits)
return false;
}
while (pos < end && isAlphaASCII(*pos))
++pos;
token_end = pos;
++splits;
return true;
}
};
using FunctionSplitByAlpha = FunctionTokens<SplitByAlphaImpl>;
}
REGISTER_FUNCTION(SplitByAlpha)
{
factory.registerFunction<FunctionSplitByAlpha>();
factory.registerAlias("splitByAlpha", FunctionSplitByAlpha::name);
}
}

View File

@ -4,7 +4,6 @@
#include <Functions/FunctionFactory.h>
#include <Functions/FunctionHelpers.h>
#include <Functions/IFunction.h>
#include <base/range.h>
namespace DB
@ -46,10 +45,10 @@ private:
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (!isString(arguments[0]))
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of argument of function {}", arguments[0]->getName(), getName());
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of the first argument of function {}", arguments[0]->getName(), getName());
if (!isString(arguments[1]))
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of argument of function {}", arguments[1]->getName(), getName());
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of the second argument of function {}", arguments[1]->getName(), getName());
return std::make_shared<DataTypeString>();
}

View File

@ -0,0 +1,202 @@
#include <Columns/ColumnArray.h>
#include <Columns/ColumnConst.h>
#include <Columns/ColumnNullable.h>
#include <Columns/ColumnString.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypeNullable.h>
#include <DataTypes/DataTypeString.h>
#include <Functions/FunctionHelpers.h>
#include <Functions/IFunction.h>
#include <Functions/Regexps.h>
#include <Functions/FunctionFactory.h>
#include <Interpreters/Context.h>
#include <IO/WriteHelpers.h>
#include <Interpreters/castColumn.h>
#include <Common/StringUtils/StringUtils.h>
#include <Common/assert_cast.h>
/** arrayStringConcat(arr)
* arrayStringConcat(arr, delimiter)
* - join an array of strings into one string via a separator.
*/
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_COLUMN;
}
namespace
{
/// Joins an array of type serializable to string into one string via a separator.
class FunctionArrayStringConcat : public IFunction
{
private:
static void executeInternal(
const ColumnString::Chars & src_chars,
const ColumnString::Offsets & src_string_offsets,
const ColumnArray::Offsets & src_array_offsets,
const char * delimiter,
const size_t delimiter_size,
ColumnString::Chars & dst_chars,
ColumnString::Offsets & dst_string_offsets,
const char8_t * null_map)
{
size_t size = src_array_offsets.size();
if (!size)
return;
/// With a small margin - as if the separator goes after the last string of the array.
dst_chars.resize(
src_chars.size()
+ delimiter_size * src_string_offsets.size() /// Separators after each string...
+ src_array_offsets.size() /// Zero byte after each joined string
- src_string_offsets.size()); /// The former zero byte after each string of the array
/// There will be as many strings as there were arrays.
dst_string_offsets.resize(src_array_offsets.size());
ColumnArray::Offset current_src_array_offset = 0;
ColumnString::Offset current_dst_string_offset = 0;
/// Loop through the array of strings.
for (size_t i = 0; i < size; ++i)
{
bool first_non_null = true;
/// Loop through the rows within the array. /// NOTE You can do everything in one copy, if the separator has a size of 1.
for (auto next_src_array_offset = src_array_offsets[i]; current_src_array_offset < next_src_array_offset; ++current_src_array_offset)
{
if (null_map && null_map[current_src_array_offset]) [[unlikely]]
continue;
if (!first_non_null)
{
memcpy(&dst_chars[current_dst_string_offset], delimiter, delimiter_size);
current_dst_string_offset += delimiter_size;
}
first_non_null = false;
const auto current_src_string_offset = current_src_array_offset ? src_string_offsets[current_src_array_offset - 1] : 0;
size_t bytes_to_copy = src_string_offsets[current_src_array_offset] - current_src_string_offset - 1;
memcpySmallAllowReadWriteOverflow15(
&dst_chars[current_dst_string_offset], &src_chars[current_src_string_offset], bytes_to_copy);
current_dst_string_offset += bytes_to_copy;
}
dst_chars[current_dst_string_offset] = 0;
++current_dst_string_offset;
dst_string_offsets[i] = current_dst_string_offset;
}
dst_chars.resize(dst_string_offsets.back());
}
static void executeInternal(
const ColumnString & col_string,
const ColumnArray & col_arr,
const String & delimiter,
ColumnString & col_res,
const char8_t * null_map = nullptr)
{
executeInternal(
col_string.getChars(),
col_string.getOffsets(),
col_arr.getOffsets(),
delimiter.data(),
delimiter.size(),
col_res.getChars(),
col_res.getOffsets(),
null_map);
}
static ColumnPtr serializeNestedColumn(const ColumnArray & col_arr, const DataTypePtr & nested_type)
{
DataTypePtr type = nested_type;
ColumnPtr column = col_arr.getDataPtr();
if (type->isNullable())
{
type = removeNullable(type);
column = assert_cast<const ColumnNullable &>(*column).getNestedColumnPtr();
}
return castColumn({column, type, "tmp"}, std::make_shared<DataTypeString>());
}
public:
static constexpr auto name = "arrayStringConcat";
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionArrayStringConcat>(); }
String getName() const override
{
return name;
}
bool isVariadic() const override { return true; }
bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; }
size_t getNumberOfArguments() const override { return 0; }
bool useDefaultImplementationForConstants() const override { return true; }
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; }
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
{
FunctionArgumentDescriptors mandatory_args
{
{"arr", &isArray<IDataType>, nullptr, "Array"},
};
FunctionArgumentDescriptors optional_args
{
{"separator", &isString<IDataType>, isColumnConst, "const String"},
};
validateFunctionArgumentTypes(*this, arguments, mandatory_args, optional_args);
return std::make_shared<DataTypeString>();
}
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & /*result_type*/, size_t /*input_rows_count*/) const override
{
String delimiter;
if (arguments.size() == 2)
{
const ColumnConst * col_delim = checkAndGetColumnConstStringOrFixedString(arguments[1].column.get());
if (!col_delim)
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Second argument for function {} must be constant string.", getName());
delimiter = col_delim->getValue<String>();
}
const auto & nested_type = assert_cast<const DataTypeArray &>(*arguments[0].type).getNestedType();
const ColumnArray & col_arr = assert_cast<const ColumnArray &>(*arguments[0].column);
ColumnPtr str_subcolumn = serializeNestedColumn(col_arr, nested_type);
const ColumnString & col_string = assert_cast<const ColumnString &>(*str_subcolumn.get());
auto col_res = ColumnString::create();
if (const ColumnNullable * col_nullable = checkAndGetColumn<ColumnNullable>(col_arr.getData()))
executeInternal(col_string, col_arr, delimiter, *col_res, col_nullable->getNullMapData().data());
else
executeInternal(col_string, col_arr, delimiter, *col_res);
return col_res;
}
};
}
REGISTER_FUNCTION(ArrayStringConcat)
{
factory.registerFunction<FunctionArrayStringConcat>();
}
}

View File

@ -0,0 +1,122 @@
#include <Columns/ColumnArray.h>
#include <Columns/ColumnConst.h>
#include <Columns/ColumnString.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypeNullable.h>
#include <DataTypes/DataTypeString.h>
#include <Functions/FunctionHelpers.h>
#include <Functions/FunctionTokens.h>
#include <Functions/FunctionFactory.h>
#include <Functions/Regexps.h>
#include <Interpreters/Context.h>
#include <Common/StringUtils/StringUtils.h>
#include <Common/assert_cast.h>
#include <Common/typeid_cast.h>
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_COLUMN;
}
/** Functions that split strings into an array of strings or vice versa.
*
* extractAll(s, regexp) - select from the string the subsequences corresponding to the regexp.
* - first subpattern, if regexp has subpattern;
* - zero subpattern (the match part, otherwise);
* - otherwise, an empty array
*/
namespace
{
using Pos = const char *;
class ExtractAllImpl
{
private:
Regexps::RegexpPtr re;
OptimizedRegularExpression::MatchVec matches;
size_t capture;
Pos pos;
Pos end;
public:
static constexpr auto name = "extractAll";
static String getName() { return name; }
static bool isVariadic() { return false; }
static size_t getNumberOfArguments() { return 2; }
static void checkArguments(const IFunction & func, const ColumnsWithTypeAndName & arguments)
{
FunctionArgumentDescriptors mandatory_args{
{"haystack", &isString<IDataType>, nullptr, "String"},
{"pattern", &isString<IDataType>, isColumnConst, "const String"}
};
validateFunctionArgumentTypes(func, arguments, mandatory_args);
}
static constexpr auto strings_argument_position = 0uz;
void init(const ColumnsWithTypeAndName & arguments, bool /*max_substrings_includes_remaining_string*/)
{
const ColumnConst * col = checkAndGetColumnConstStringOrFixedString(arguments[1].column.get());
if (!col)
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of first argument of function {}. "
"Must be constant string.", arguments[1].column->getName(), getName());
re = std::make_shared<OptimizedRegularExpression>(Regexps::createRegexp<false, false, false>(col->getValue<String>()));
capture = re->getNumberOfSubpatterns() > 0 ? 1 : 0;
matches.resize(capture + 1);
}
/// Called for each next string.
void set(Pos pos_, Pos end_)
{
pos = pos_;
end = end_;
}
/// Get the next token, if any, or return false.
bool get(Pos & token_begin, Pos & token_end)
{
if (!pos || pos > end)
return false;
if (!re->match(pos, end - pos, matches) || !matches[0].length)
return false;
if (matches[capture].offset == std::string::npos)
{
/// Empty match.
token_begin = pos;
token_end = pos;
}
else
{
token_begin = pos + matches[capture].offset;
token_end = token_begin + matches[capture].length;
}
pos += matches[0].offset + matches[0].length;
return true;
}
};
using FunctionExtractAll = FunctionTokens<ExtractAllImpl>;
}
REGISTER_FUNCTION(ExtractAll)
{
factory.registerFunction<FunctionExtractAll>();
}
}

View File

@ -9,12 +9,14 @@
#include <Functions/DateTimeTransforms.h>
#include <Functions/FunctionFactory.h>
#include <Functions/FunctionHelpers.h>
#include <Functions/FunctionsConversion.h>
#include <Functions/IFunction.h>
#include <Functions/castTypeToEither.h>
#include <Functions/extractTimeZoneFromFunctionArguments.h>
#include <Functions/numLiteralChars.h>
#include <Interpreters/Context.h>
#include <Interpreters/castColumn.h>
#include <IO/WriteHelpers.h>
#include <Common/Concepts.h>
@ -803,18 +805,7 @@ public:
{
if (arguments.size() == 1)
{
if (!castType(arguments[0].type.get(), [&](const auto & type)
{
using FromDataType = std::decay_t<decltype(type)>;
res = ConvertImpl<FromDataType, DataTypeDateTime, Name>::execute(arguments, result_type, input_rows_count);
return true;
}))
{
throw Exception(ErrorCodes::ILLEGAL_COLUMN,
"Illegal column {} of function {}, must be Integer, Date, Date32, DateTime "
"or DateTime64 when arguments size is 1.",
arguments[0].column->getName(), getName());
}
return castColumn(arguments[0], result_type);
}
else
{

View File

@ -7,15 +7,16 @@
#include <Functions/FunctionFactory.h>
#include <Functions/FunctionHelpers.h>
#include <Functions/FunctionsConversion.h>
#include <Functions/IFunction.h>
#include <Functions/castTypeToEither.h>
#include <Functions/numLiteralChars.h>
#include <Interpreters/Context.h>
#include <IO/WriteHelpers.h>
#include <base/types.h>
#include <boost/algorithm/string/case_conv.hpp>
namespace DB
{
namespace ErrorCodes

View File

@ -0,0 +1,122 @@
#include <Columns/ColumnConst.h>
#include <Functions/FunctionHelpers.h>
#include <Functions/FunctionTokens.h>
#include <Functions/FunctionFactory.h>
#include <Common/StringUtils/StringUtils.h>
#include <Common/assert_cast.h>
namespace DB
{
namespace ErrorCodes
{
extern const int BAD_ARGUMENTS;
extern const int ILLEGAL_COLUMN;
}
/** Functions that split strings into an array of strings or vice versa.
*
* splitByChar(sep, s[, max_substrings])
*/
namespace
{
using Pos = const char *;
class SplitByCharImpl
{
private:
Pos pos;
Pos end;
char separator;
std::optional<size_t> max_splits;
size_t splits;
bool max_substrings_includes_remaining_string;
public:
static constexpr auto name = "splitByChar";
static bool isVariadic() { return true; }
static size_t getNumberOfArguments() { return 0; }
static void checkArguments(const IFunction & func, const ColumnsWithTypeAndName & arguments)
{
checkArgumentsWithSeparatorAndOptionalMaxSubstrings(func, arguments);
}
static constexpr auto strings_argument_position = 1uz;
void init(const ColumnsWithTypeAndName & arguments, bool max_substrings_includes_remaining_string_)
{
const ColumnConst * col = checkAndGetColumnConstStringOrFixedString(arguments[0].column.get());
if (!col)
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of first argument of function {}. "
"Must be constant string.", arguments[0].column->getName(), name);
String sep_str = col->getValue<String>();
if (sep_str.size() != 1)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Illegal separator for function {}. Must be exactly one byte.", name);
separator = sep_str[0];
max_substrings_includes_remaining_string = max_substrings_includes_remaining_string_;
max_splits = extractMaxSplits(arguments, 2);
}
void set(Pos pos_, Pos end_)
{
pos = pos_;
end = end_;
splits = 0;
}
bool get(Pos & token_begin, Pos & token_end)
{
if (!pos)
return false;
token_begin = pos;
if (max_splits)
{
if (max_substrings_includes_remaining_string)
{
if (splits == *max_splits - 1)
{
token_end = end;
pos = nullptr;
return true;
}
}
else
if (splits == *max_splits)
return false;
}
pos = reinterpret_cast<Pos>(memchr(pos, separator, end - pos));
if (pos)
{
token_end = pos;
++pos;
++splits;
}
else
token_end = end;
return true;
}
};
using FunctionSplitByChar = FunctionTokens<SplitByCharImpl>;
}
REGISTER_FUNCTION(SplitByChar)
{
factory.registerFunction<FunctionSplitByChar>();
}
}

View File

@ -0,0 +1,113 @@
#include <Columns/ColumnArray.h>
#include <Columns/ColumnConst.h>
#include <Columns/ColumnString.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypeNullable.h>
#include <DataTypes/DataTypeString.h>
#include <Functions/FunctionHelpers.h>
#include <Functions/FunctionTokens.h>
#include <Functions/FunctionFactory.h>
#include <Functions/Regexps.h>
#include <Interpreters/Context.h>
#include <Common/StringUtils/StringUtils.h>
#include <Common/assert_cast.h>
#include <Common/typeid_cast.h>
namespace DB
{
/** Functions that split strings into an array of strings or vice versa.
*
* splitByNonAlpha(s[, max_substrings]) - split the string by whitespace and punctuation characters
*/
namespace
{
using Pos = const char *;
class SplitByNonAlphaImpl
{
private:
Pos pos;
Pos end;
std::optional<size_t> max_splits;
size_t splits;
bool max_substrings_includes_remaining_string;
public:
/// Get the name of the function.
static constexpr auto name = "splitByNonAlpha";
static bool isVariadic() { return true; }
static size_t getNumberOfArguments() { return 0; }
static void checkArguments(const IFunction & func, const ColumnsWithTypeAndName & arguments)
{
checkArgumentsWithOptionalMaxSubstrings(func, arguments);
}
static constexpr auto strings_argument_position = 0uz;
void init(const ColumnsWithTypeAndName & arguments, bool max_substrings_includes_remaining_string_)
{
max_substrings_includes_remaining_string = max_substrings_includes_remaining_string_;
max_splits = extractMaxSplits(arguments, 1);
}
/// Called for each next string.
void set(Pos pos_, Pos end_)
{
pos = pos_;
end = end_;
splits = 0;
}
/// Get the next token, if any, or return false.
bool get(Pos & token_begin, Pos & token_end)
{
/// Skip garbage
while (pos < end && (isWhitespaceASCII(*pos) || isPunctuationASCII(*pos)))
++pos;
if (pos == end)
return false;
token_begin = pos;
if (max_splits)
{
if (max_substrings_includes_remaining_string)
{
if (splits == *max_splits - 1)
{
token_end = end;
pos = end;
return true;
}
}
else
if (splits == *max_splits)
return false;
}
while (pos < end && !(isWhitespaceASCII(*pos) || isPunctuationASCII(*pos)))
++pos;
token_end = pos;
splits++;
return true;
}
};
using FunctionSplitByNonAlpha = FunctionTokens<SplitByNonAlphaImpl>;
}
REGISTER_FUNCTION(SplitByNonAlpha)
{
factory.registerFunction<FunctionSplitByNonAlpha>();
}
}

View File

@ -0,0 +1,156 @@
#include <Columns/ColumnConst.h>
#include <Functions/FunctionHelpers.h>
#include <Functions/FunctionTokens.h>
#include <Functions/FunctionFactory.h>
#include <Functions/Regexps.h>
#include <Common/StringUtils/StringUtils.h>
#include <Common/assert_cast.h>
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_COLUMN;
}
/** Functions that split strings into an array of strings or vice versa.
*
* splitByRegexp(regexp, s[, max_substrings])
*/
namespace
{
using Pos = const char *;
class SplitByRegexpImpl
{
private:
Regexps::RegexpPtr re;
OptimizedRegularExpression::MatchVec matches;
Pos pos;
Pos end;
std::optional<size_t> max_splits;
size_t splits;
bool max_substrings_includes_remaining_string;
public:
static constexpr auto name = "splitByRegexp";
static bool isVariadic() { return true; }
static size_t getNumberOfArguments() { return 0; }
static void checkArguments(const IFunction & func, const ColumnsWithTypeAndName & arguments)
{
checkArgumentsWithSeparatorAndOptionalMaxSubstrings(func, arguments);
}
static constexpr auto strings_argument_position = 1uz;
void init(const ColumnsWithTypeAndName & arguments, bool max_substrings_includes_remaining_string_)
{
const ColumnConst * col = checkAndGetColumnConstStringOrFixedString(arguments[0].column.get());
if (!col)
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of first argument of function {}. "
"Must be constant string.", arguments[0].column->getName(), name);
if (!col->getValue<String>().empty())
re = std::make_shared<OptimizedRegularExpression>(Regexps::createRegexp<false, false, false>(col->getValue<String>()));
max_substrings_includes_remaining_string = max_substrings_includes_remaining_string_;
max_splits = extractMaxSplits(arguments, 2);
}
/// Called for each next string.
void set(Pos pos_, Pos end_)
{
pos = pos_;
end = end_;
splits = 0;
}
/// Get the next token, if any, or return false.
bool get(Pos & token_begin, Pos & token_end)
{
if (!re)
{
if (pos == end)
return false;
token_begin = pos;
if (max_splits)
{
if (max_substrings_includes_remaining_string)
{
if (splits == *max_splits - 1)
{
token_end = end;
pos = end;
return true;
}
}
else
if (splits == *max_splits)
return false;
}
pos += 1;
token_end = pos;
++splits;
}
else
{
if (!pos || pos > end)
return false;
token_begin = pos;
if (max_splits)
{
if (max_substrings_includes_remaining_string)
{
if (splits == *max_splits - 1)
{
token_end = end;
pos = nullptr;
return true;
}
}
else
if (splits == *max_splits)
return false;
}
if (!re->match(pos, end - pos, matches) || !matches[0].length)
{
token_end = end;
pos = end + 1;
}
else
{
token_end = pos + matches[0].offset;
pos = token_end + matches[0].length;
++splits;
}
}
return true;
}
};
using FunctionSplitByRegexp = FunctionTokens<SplitByRegexpImpl>;
}
REGISTER_FUNCTION(SplitByRegexp)
{
factory.registerFunction<FunctionSplitByRegexp>();
}
}

View File

@ -0,0 +1,148 @@
#include <Columns/ColumnConst.h>
#include <Functions/FunctionHelpers.h>
#include <Functions/FunctionTokens.h>
#include <Functions/FunctionFactory.h>
#include <Common/StringUtils/StringUtils.h>
#include <Common/assert_cast.h>
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_COLUMN;
}
/** Functions that split strings into an array of strings or vice versa.
*
* splitByString(sep, s[, max_substrings])
*/
namespace
{
using Pos = const char *;
class SplitByStringImpl
{
private:
Pos pos;
Pos end;
String separator;
std::optional<size_t> max_splits;
size_t splits;
bool max_substrings_includes_remaining_string;
public:
static constexpr auto name = "splitByString";
static bool isVariadic() { return true; }
static size_t getNumberOfArguments() { return 0; }
static void checkArguments(const IFunction & func, const ColumnsWithTypeAndName & arguments)
{
checkArgumentsWithSeparatorAndOptionalMaxSubstrings(func, arguments);
}
static constexpr auto strings_argument_position = 1uz;
void init(const ColumnsWithTypeAndName & arguments, bool max_substrings_includes_remaining_string_)
{
const ColumnConst * col = checkAndGetColumnConstStringOrFixedString(arguments[0].column.get());
if (!col)
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of first argument of function {}. "
"Must be constant string.", arguments[0].column->getName(), name);
separator = col->getValue<String>();
max_substrings_includes_remaining_string = max_substrings_includes_remaining_string_;
max_splits = extractMaxSplits(arguments, 2);
}
/// Called for each next string.
void set(Pos pos_, Pos end_)
{
pos = pos_;
end = end_;
splits = 0;
}
/// Get the next token, if any, or return false.
bool get(Pos & token_begin, Pos & token_end)
{
if (separator.empty())
{
if (pos == end)
return false;
token_begin = pos;
if (max_splits)
{
if (max_substrings_includes_remaining_string)
{
if (splits == *max_splits - 1)
{
token_end = end;
pos = end;
return true;
}
}
else
if (splits == *max_splits)
return false;
}
pos += 1;
token_end = pos;
++splits;
}
else
{
if (!pos)
return false;
token_begin = pos;
if (max_splits)
{
if (max_substrings_includes_remaining_string)
{
if (splits == *max_splits - 1)
{
token_end = end;
pos = nullptr;
return true;
}
}
else
if (splits == *max_splits)
return false;
}
pos = reinterpret_cast<Pos>(memmem(pos, end - pos, separator.data(), separator.size()));
if (pos)
{
token_end = pos;
pos += separator.size();
++splits;
}
else
token_end = end;
}
return true;
}
};
using FunctionSplitByString = FunctionTokens<SplitByStringImpl>;
}
REGISTER_FUNCTION(SplitByString)
{
factory.registerFunction<FunctionSplitByString>();
}
}

View File

@ -0,0 +1,101 @@
#include <Functions/FunctionTokens.h>
#include <Functions/FunctionFactory.h>
#include <Common/StringUtils/StringUtils.h>
namespace DB
{
/** Functions that split strings into an array of strings or vice versa.
*
* splitByWhitespace(s[, max_substrings]) - split the string by whitespace characters
*/
namespace
{
using Pos = const char *;
class SplitByWhitespaceImpl
{
private:
Pos pos;
Pos end;
std::optional<size_t> max_splits;
size_t splits;
bool max_substrings_includes_remaining_string;
public:
static constexpr auto name = "splitByWhitespace";
static bool isVariadic() { return true; }
static size_t getNumberOfArguments() { return 0; }
static void checkArguments(const IFunction & func, const ColumnsWithTypeAndName & arguments)
{
checkArgumentsWithOptionalMaxSubstrings(func, arguments);
}
static constexpr auto strings_argument_position = 0uz;
void init(const ColumnsWithTypeAndName & arguments, bool max_substrings_includes_remaining_string_)
{
max_substrings_includes_remaining_string = max_substrings_includes_remaining_string_;
max_splits = extractMaxSplits(arguments, 1);
}
/// Called for each next string.
void set(Pos pos_, Pos end_)
{
pos = pos_;
end = end_;
splits = 0;
}
/// Get the next token, if any, or return false.
bool get(Pos & token_begin, Pos & token_end)
{
/// Skip garbage
while (pos < end && isWhitespaceASCII(*pos))
++pos;
if (pos == end)
return false;
token_begin = pos;
if (max_splits)
{
if (max_substrings_includes_remaining_string)
{
if (splits == *max_splits - 1)
{
token_end = end;
pos = end;
return true;
}
}
else
if (splits == *max_splits)
return false;
}
while (pos < end && !isWhitespaceASCII(*pos))
++pos;
token_end = pos;
splits++;
return true;
}
};
using FunctionSplitByWhitespace = FunctionTokens<SplitByWhitespaceImpl>;
}
REGISTER_FUNCTION(SplitByWhitespace)
{
factory.registerFunction<FunctionSplitByWhitespace>();
}
}

View File

@ -1,6 +1,5 @@
#include <Functions/IFunction.h>
#include <Functions/FunctionFactory.h>
#include <Functions/FunctionsConversion.h>
#include <Functions/CastOverloadResolver.h>
#include <DataTypes/DataTypeFactory.h>
#include <DataTypes/DataTypeString.h>
@ -12,6 +11,14 @@ namespace
{
class FunctionToBool : public IFunction
{
private:
ContextPtr context;
static String getReturnTypeName(const DataTypePtr & argument)
{
return argument->isNullable() ? "Nullable(Bool)" : "Bool";
}
public:
static constexpr auto name = "toBool";
@ -32,8 +39,7 @@ namespace
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
auto bool_type = DataTypeFactory::instance().get("Bool");
return arguments[0]->isNullable() ? makeNullable(bool_type) : bool_type;
return DataTypeFactory::instance().get(getReturnTypeName(arguments[0]));
}
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t) const override
@ -42,18 +48,17 @@ namespace
{
arguments[0],
{
DataTypeString().createColumnConst(arguments[0].column->size(), arguments[0].type->isNullable() ? "Nullable(Bool)" : "Bool"),
DataTypeString().createColumnConst(arguments[0].column->size(), getReturnTypeName(arguments[0].type)),
std::make_shared<DataTypeString>(),
""
}
};
FunctionOverloadResolverPtr func_builder_cast = CastInternalOverloadResolver<CastType::nonAccurate>::createImpl();
FunctionOverloadResolverPtr func_builder_cast = createInternalCastOverloadResolver(CastType::nonAccurate, {});
auto func_cast = func_builder_cast->build(cast_args);
return func_cast->execute(cast_args, result_type, arguments[0].column->size());
}
};
}
REGISTER_FUNCTION(ToBool)

View File

@ -3,9 +3,9 @@
#include <Analyzer/FunctionNode.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypeString.h>
#include <Columns/ColumnConst.h>
#include <Functions/IFunction.h>
#include <Functions/IFunctionAdaptors.h>
#include <Functions/FunctionsConversion.h>
#include <Functions/materialize.h>
#include <Functions/FunctionsLogical.h>
#include <Functions/CastOverloadResolver.h>
@ -21,6 +21,7 @@
#include <base/sort.h>
#include <Common/JSONBuilder.h>
namespace DB
{
@ -248,7 +249,7 @@ const ActionsDAG::Node & ActionsDAG::addCast(const Node & node_to_cast, const Da
const auto * cast_type_constant_node = &addColumn(std::move(column));
ActionsDAG::NodeRawConstPtrs children = {&node_to_cast, cast_type_constant_node};
FunctionOverloadResolverPtr func_builder_cast = CastInternalOverloadResolver<CastType::nonAccurate>::createImpl();
FunctionOverloadResolverPtr func_builder_cast = createInternalCastOverloadResolver(CastType::nonAccurate, {});
return addFunction(func_builder_cast, std::move(children), result_name);
}
@ -1381,9 +1382,9 @@ ActionsDAGPtr ActionsDAG::makeConvertingActions(
const auto * right_arg = &actions_dag->addColumn(std::move(column));
const auto * left_arg = dst_node;
FunctionCastBase::Diagnostic diagnostic = {dst_node->result_name, res_elem.name};
CastDiagnostic diagnostic = {dst_node->result_name, res_elem.name};
FunctionOverloadResolverPtr func_builder_cast
= CastInternalOverloadResolver<CastType::nonAccurate>::createImpl(std::move(diagnostic));
= createInternalCastOverloadResolver(CastType::nonAccurate, std::move(diagnostic));
NodeRawConstPtrs children = { left_arg, right_arg };
dst_node = &actions_dag->addFunction(func_builder_cast, std::move(children), {});

View File

@ -1,13 +1,8 @@
#include <Common/typeid_cast.h>
#include <Parsers/queryToString.h>
#include <Functions/FunctionsComparison.h>
#include <Functions/FunctionsLogical.h>
#include <IO/WriteHelpers.h>
#include <Interpreters/CrossToInnerJoinVisitor.h>
#include <Interpreters/DatabaseAndTableWithAlias.h>
#include <Interpreters/IdentifierSemantic.h>
#include <Interpreters/misc.h>
#include <Parsers/ASTExpressionList.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTLiteral.h>
@ -15,11 +10,11 @@
#include <Parsers/ASTSubquery.h>
#include <Parsers/ASTTablesInSelectQuery.h>
#include <Parsers/ExpressionListParsers.h>
#include <Parsers/ParserTablesInSelectQuery.h>
#include <Parsers/parseQuery.h>
#include <Common/logger_useful.h>
namespace DB
{
@ -117,7 +112,7 @@ std::map<size_t, std::vector<ASTPtr>> moveExpressionToJoinOn(
std::map<size_t, std::vector<ASTPtr>> asts_to_join_on;
for (const auto & node : splitConjunctionsAst(ast))
{
if (const auto * func = node->as<ASTFunction>(); func && func->name == NameEquals::name)
if (const auto * func = node->as<ASTFunction>(); func && func->name == "equals")
{
if (!func->arguments || func->arguments->children.size() != 2)
return {};
@ -154,7 +149,7 @@ ASTPtr makeOnExpression(const std::vector<ASTPtr> & expressions)
for (const auto & ast : expressions)
arguments.emplace_back(ast->clone());
return makeASTFunction(NameAnd::name, std::move(arguments));
return makeASTFunction("and", std::move(arguments));
}
std::vector<JoinedElement> getTables(const ASTSelectQuery & select)

View File

@ -1,10 +1,9 @@
#include <Interpreters/GatherFunctionQuantileVisitor.h>
#include <AggregateFunctions/AggregateFunctionQuantile.h>
#include <Parsers/ASTFunction.h>
#include <base/types.h>
#include <Common/Exception.h>
namespace DB
{
@ -14,22 +13,23 @@ namespace ErrorCodes
}
/// Mapping from quantile functions for single value to plural
static const std::unordered_map<String, String> quantile_fuse_name_mapping = {
{NameQuantile::name, NameQuantiles::name},
{NameQuantileBFloat16::name, NameQuantilesBFloat16::name},
{NameQuantileBFloat16Weighted::name, NameQuantilesBFloat16Weighted::name},
{NameQuantileDeterministic::name, NameQuantilesDeterministic::name},
{NameQuantileExact::name, NameQuantilesExact::name},
{NameQuantileExactExclusive::name, NameQuantilesExactExclusive::name},
{NameQuantileExactHigh::name, NameQuantilesExactHigh::name},
{NameQuantileExactInclusive::name, NameQuantilesExactInclusive::name},
{NameQuantileExactLow::name, NameQuantilesExactLow::name},
{NameQuantileExactWeighted::name, NameQuantilesExactWeighted::name},
{NameQuantileInterpolatedWeighted::name, NameQuantilesInterpolatedWeighted::name},
{NameQuantileTDigest::name, NameQuantilesTDigest::name},
{NameQuantileTDigestWeighted::name, NameQuantilesTDigestWeighted::name},
{NameQuantileTiming::name, NameQuantilesTiming::name},
{NameQuantileTimingWeighted::name, NameQuantilesTimingWeighted::name},
static const std::unordered_map<String, String> quantile_fuse_name_mapping =
{
{"quantile", "quantiles"},
{"quantileBFloat16", "quantilesBFloat16"},
{"quantileBFloat16Weighted", "quantilesBFloat16Weighted"},
{"quantileDeterministic", "quantilesDeterministic"},
{"quantileExact", "quantilesExact"},
{"quantileExactExclusive", "quantilesExactExclusive"},
{"quantileExactHigh", "quantilesExactHigh"},
{"quantileExactInclusive", "quantilesExactInclusive"},
{"quantileExactLow", "quantilesExactLow"},
{"quantileExactWeighted", "quantilesExactWeighted"},
{"quantileInterpolatedWeighted", "quantilesInterpolatedWeighted"},
{"quantileTDigest", "quantilesTDigest"},
{"quantileTDigestWeighted", "quantilesTDigestWeighted"},
{"quantileTiming", "quantilesTiming"},
{"quantileTimingWeighted", "quantilesTimingWeighted"},
};
String GatherFunctionQuantileData::toFusedNameOrSelf(const String & func_name)
@ -63,9 +63,9 @@ void GatherFunctionQuantileData::FuseQuantileAggregatesData::addFuncNode(ASTPtr
const auto & arguments = func->arguments->children;
bool need_two_args = func->name == NameQuantileDeterministic::name || func->name == NameQuantileExactWeighted::name
|| func->name == NameQuantileInterpolatedWeighted::name || func->name == NameQuantileTimingWeighted::name
|| func->name == NameQuantileTDigestWeighted::name || func->name == NameQuantileBFloat16Weighted::name;
bool need_two_args = func->name == "quantileDeterministic" || func->name == "quantileExactWeighted"
|| func->name == "quantileInterpolatedWeighted" || func->name == "quantileTimingWeighted"
|| func->name == "quantileTDigestWeighted" || func->name == "quantileBFloat16Weighted";
if (arguments.size() != (need_two_args ? 2 : 1))
return;

View File

@ -8,8 +8,7 @@
#include <Core/Block.h>
#include <Storages/ColumnsDescription.h>
#include <Interpreters/ExpressionActions.h>
#include <Functions/IFunctionAdaptors.h>
#include <Functions/materialize.h>
#include <Functions/FunctionFactory.h>
namespace DB

View File

@ -1,13 +1,16 @@
#include <Interpreters/castColumn.h>
#include <Functions/FunctionsConversion.h>
#include <Functions/CastOverloadResolver.h>
#include <Functions/IFunction.h>
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypeNullable.h>
#include <Core/ColumnsWithTypeAndName.h>
#include <Core/Field.h>
namespace DB
{
template <CastType cast_type = CastType::nonAccurate>
static ColumnPtr castColumn(const ColumnWithTypeAndName & arg, const DataTypePtr & type, InternalCastFunctionCache * cache = nullptr)
static ColumnPtr castColumn(CastType cast_type, const ColumnWithTypeAndName & arg, const DataTypePtr & type, InternalCastFunctionCache * cache = nullptr)
{
if (arg.type->equals(*type) && cast_type != CastType::accurateOrNull)
return arg.column;
@ -23,37 +26,34 @@ static ColumnPtr castColumn(const ColumnWithTypeAndName & arg, const DataTypePtr
""
}
};
auto get_cast_func = [&arguments]
auto get_cast_func = [cast_type, &arguments]
{
FunctionOverloadResolverPtr func_builder_cast = CastInternalOverloadResolver<cast_type>::createImpl();
FunctionOverloadResolverPtr func_builder_cast = createInternalCastOverloadResolver(cast_type, {});
return func_builder_cast->build(arguments);
};
FunctionBasePtr func_cast = cache ? cache->getOrSet(cast_type, from_name, to_name, std::move(get_cast_func)) : get_cast_func();
if constexpr (cast_type == CastType::accurateOrNull)
{
if (cast_type == CastType::accurateOrNull)
return func_cast->execute(arguments, makeNullable(type), arg.column->size());
}
else
{
return func_cast->execute(arguments, type, arg.column->size());
}
}
ColumnPtr castColumn(const ColumnWithTypeAndName & arg, const DataTypePtr & type, InternalCastFunctionCache * cache)
{
return castColumn<CastType::nonAccurate>(arg, type, cache);
return castColumn(CastType::nonAccurate, arg, type, cache);
}
ColumnPtr castColumnAccurate(const ColumnWithTypeAndName & arg, const DataTypePtr & type, InternalCastFunctionCache * cache)
{
return castColumn<CastType::accurate>(arg, type, cache);
return castColumn(CastType::accurate, arg, type, cache);
}
ColumnPtr castColumnAccurateOrNull(const ColumnWithTypeAndName & arg, const DataTypePtr & type, InternalCastFunctionCache * cache)
{
return castColumn<CastType::accurateOrNull>(arg, type, cache);
return castColumn(CastType::accurateOrNull, arg, type, cache);
}
}

View File

@ -2,11 +2,15 @@
#include <tuple>
#include <Core/ColumnWithTypeAndName.h>
#include <Functions/FunctionsConversion.h>
#include <Functions/CastOverloadResolver.h>
namespace DB
{
class IFunctionBase;
using FunctionBasePtr = std::shared_ptr<const IFunctionBase>;
struct InternalCastFunctionCache
{
private:
@ -15,7 +19,7 @@ private:
std::map<std::tuple<CastType, String, String>, FunctionBasePtr> impl;
mutable std::mutex mutex;
public:
template<typename Getter>
template <typename Getter>
FunctionBasePtr getOrSet(CastType cast_type, const String & from, const String & to, Getter && getter)
{
std::lock_guard lock{mutex};

View File

@ -1,22 +1,33 @@
#pragma once
#include "config.h"
#if USE_PARQUET || USE_ORC
#include <unordered_map>
#include <Core/Block.h>
#include <DataTypes/DataTypeTuple.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypeMap.h>
#include <DataTypes/DataTypeNullable.h>
#include <DataTypes/NestedUtils.h>
#include <arrow/type.h>
#include <arrow/type_fwd.h>
#include <boost/algorithm/string/case_conv.hpp>
#include <Common/Exception.h>
namespace arrow
{
class Schema;
class DataType;
class Field;
class Schema;
class DataType;
class Field;
}
namespace DB
{
namespace ErrorCodes
{
extern const int THERE_IS_NO_COLUMN;
@ -211,5 +222,7 @@ private:
}
}
};
}
#endif

View File

@ -35,7 +35,6 @@
#include <Formats/ReadSchemaUtils.h>
#include <Formats/FormatFactory.h>
#include <Functions/FunctionsConversion.h>
#include <QueryPipeline/QueryPipeline.h>
#include <QueryPipeline/QueryPipelineBuilder.h>

View File

@ -1,6 +1,7 @@
#include <Storages/KVStorageUtils.h>
#include <Columns/ColumnSet.h>
#include <Columns/ColumnConst.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTIdentifier.h>
@ -12,6 +13,9 @@
#include <Interpreters/convertFieldToType.h>
#include <Interpreters/evaluateConstantExpression.h>
#include <Functions/IFunction.h>
namespace DB
{

View File

@ -1,6 +1,9 @@
#include <Storages/MergeTree/KeyCondition.h>
#include <Storages/MergeTree/BoolMask.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeLowCardinality.h>
#include <DataTypes/DataTypeNullable.h>
#include <DataTypes/DataTypeString.h>
#include <DataTypes/FieldToDataType.h>
#include <DataTypes/getLeastSupertype.h>
#include <DataTypes/Utils.h>
@ -10,7 +13,6 @@
#include <Interpreters/castColumn.h>
#include <Interpreters/misc.h>
#include <Functions/FunctionFactory.h>
#include <Functions/FunctionsConversion.h>
#include <Functions/indexHint.h>
#include <Functions/CastOverloadResolver.h>
#include <Functions/IFunction.h>
@ -18,6 +20,7 @@
#include <Common/MortonUtils.h>
#include <Common/typeid_cast.h>
#include <Columns/ColumnSet.h>
#include <Columns/ColumnConst.h>
#include <Interpreters/convertFieldToType.h>
#include <Interpreters/Set.h>
#include <Parsers/queryToString.h>
@ -1842,7 +1845,7 @@ bool KeyCondition::extractAtomFromTree(const RPNBuilderTreeNode & node, RPNEleme
ColumnsWithTypeAndName arguments{
{nullptr, key_expr_type, ""},
{DataTypeString().createColumnConst(1, common_type_maybe_nullable->getName()), common_type_maybe_nullable, ""}};
FunctionOverloadResolverPtr func_builder_cast = CastInternalOverloadResolver<CastType::nonAccurate>::createImpl();
FunctionOverloadResolverPtr func_builder_cast = createInternalCastOverloadResolver(CastType::nonAccurate, {});
auto func_cast = func_builder_cast->build(arguments);
/// If we know the given range only contains one value, then we treat all functions as positive monotonic.

View File

@ -1,9 +1,12 @@
#include <Functions/CastOverloadResolver.h>
#include <Functions/FunctionsLogical.h>
#include <Functions/IFunctionAdaptors.h>
#include <Storages/SelectQueryInfo.h>
#include <Storages/MergeTree/MergeTreeRangeReader.h>
#include <DataTypes/DataTypeString.h>
#include <Interpreters/ExpressionActions.h>
namespace DB
{
@ -160,7 +163,7 @@ const ActionsDAG::Node & addCast(
const auto * cast_type_constant_node = &dag->addColumn(std::move(column));
ActionsDAG::NodeRawConstPtrs children = {&node_to_cast, cast_type_constant_node};
FunctionOverloadResolverPtr func_builder_cast = CastInternalOverloadResolver<CastType::nonAccurate>::createImpl();
FunctionOverloadResolverPtr func_builder_cast = createInternalCastOverloadResolver(CastType::nonAccurate, {});
return addFunction(dag, func_builder_cast, std::move(children), node_remap);
}

View File

@ -2,8 +2,6 @@
#include <Common/ArenaUtils.h>
#include <Functions/FunctionsConversion.h>
#include <Interpreters/Context.h>
#include <Interpreters/ExpressionAnalyzer.h>
#include <Interpreters/TreeRewriter.h>
@ -34,7 +32,7 @@ PartitionedSink::PartitionedSink(
, sample_block(sample_block_)
{
ASTs arguments(1, partition_by);
ASTPtr partition_by_string = makeASTFunction(FunctionToString::name, std::move(arguments));
ASTPtr partition_by_string = makeASTFunction("toString", std::move(arguments));
auto syntax_result = TreeRewriter(context).analyze(partition_by_string, sample_block.getNamesAndTypesList());
partition_by_expr = ExpressionAnalyzer(partition_by_string, syntax_result, context).getActions(false);

View File

@ -6,8 +6,6 @@
#include <Common/isValidUTF8.h>
#include <Functions/FunctionsConversion.h>
#include <IO/S3Common.h>
#include <IO/S3/Requests.h>
#include <IO/ParallelReadBuffer.h>

View File

@ -5,6 +5,7 @@
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeUUID.h>
#include <DataTypes/DataTypeMap.h>
#include <Columns/ColumnString.h>
#include <Columns/ColumnsNumber.h>
#include <Interpreters/Context.h>

View File

@ -35,7 +35,6 @@
02352_grouby_shadows_arg
02354_annoy
02402_merge_engine_with_view
02426_orc_bug
02428_parameterized_view
02458_use_structure_from_insertion_table
02479_race_condition_between_insert_and_droppin_mv

View File

@ -212,10 +212,17 @@ def upload_master_static_binaries(
elif pr_info.base_ref != "master":
return
s3_path = "/".join((pr_info.base_ref, static_binary_name, "clickhouse"))
binary = build_output_path / "clickhouse"
url = s3_helper.upload_build_file_to_s3(binary, s3_path)
print(f"::notice ::Binary static URL: {url}")
# Full binary with debug info:
s3_path_full = "/".join((pr_info.base_ref, static_binary_name, "clickhouse-full"))
binary_full = build_output_path / "clickhouse"
url_full = s3_helper.upload_build_file_to_s3(binary_full, s3_path_full)
print(f"::notice ::Binary static URL (with debug info): {url_full}")
# Stripped binary without debug info:
s3_path_compact = "/".join((pr_info.base_ref, static_binary_name, "clickhouse"))
binary_compact = build_output_path / "clickhouse-stripped"
url_compact = s3_helper.upload_build_file_to_s3(binary_compact, s3_path_compact)
print(f"::notice ::Binary static URL (compact): {url_compact}")
def main():

View File

@ -297,8 +297,8 @@ class CiLogsCredentials:
return ""
extra_columns = (
f"CAST({pr_info.number} AS UInt32) AS pull_request_number, '{pr_info.sha}' AS commit_sha, "
f"toDateTime('{check_start_time}', 'UTC') AS check_start_time, '{check_name}' AS check_name, "
f"'{get_instance_type()}' AS instance_type, '{get_instance_id()}' AS instance_id"
f"toDateTime('{check_start_time}', 'UTC') AS check_start_time, toLowCardinality('{check_name}') AS check_name, "
f"toLowCardinality('{get_instance_type()}') AS instance_type, '{get_instance_id()}' AS instance_id"
)
return (
f'-e EXTRA_COLUMNS_EXPRESSION="{extra_columns}" '

View File

@ -1,6 +1,6 @@
< Connection: Keep-Alive
< Keep-Alive: timeout=3
< Keep-Alive: timeout=10
< Connection: Keep-Alive
< Keep-Alive: timeout=3
< Keep-Alive: timeout=10
< Connection: Keep-Alive
< Keep-Alive: timeout=3
< Keep-Alive: timeout=10

View File

@ -2,11 +2,11 @@ HTTP/1.1 200 OK
Connection: Keep-Alive
Content-Type: text/tab-separated-values; charset=UTF-8
Transfer-Encoding: chunked
Keep-Alive: timeout=3
Keep-Alive: timeout=10
HTTP/1.1 200 OK
Connection: Keep-Alive
Content-Type: text/tab-separated-values; charset=UTF-8
Transfer-Encoding: chunked
Keep-Alive: timeout=3
Keep-Alive: timeout=10

View File

@ -2,7 +2,7 @@
hello 1 3 world
9
9 (0,1)
key tupleElement(argMax(tuple(v1, v2, v3, v4, v5), v1), \'1\') tupleElement(argMax(tuple(v1, v2, v3, v4, v5), v1), \'2\') tupleElement(argMax(tuple(v1, v2, v3, v4, v5), v1), \'3\') tupleElement(argMax(tuple(v1, v2, v3, v4, v5), v1), \'4\') tupleElement(argMax(tuple(v1, v2, v3, v4, v5), v1), \'5\')
key tupleElement(argMax((v1, v2, v3, v4, v5), v1), \'1\') tupleElement(argMax((v1, v2, v3, v4, v5), v1), \'2\') tupleElement(argMax((v1, v2, v3, v4, v5), v1), \'3\') tupleElement(argMax((v1, v2, v3, v4, v5), v1), \'4\') tupleElement(argMax((v1, v2, v3, v4, v5), v1), \'5\')
1 20 20 10 20 30
2 11 20 10 20 30
3 70 20 10 20 30

View File

@ -18,7 +18,7 @@ system stop distributed sends dist_01555;
insert into dist_01555 values (1)(2);
-- since test_cluster_with_incorrect_pw contains incorrect password ignore error
system flush distributed dist_01555; -- { serverError 516 }
select length(splitByChar('*', data_path)), replaceRegexpOne(data_path, '^.*/([^/]*)/' , '\\1'), extract(last_exception, 'AUTHENTICATION_FAILED'), dateDiff('s', last_exception_time, now()) < 5 from system.distribution_queue where database = currentDatabase() and table = 'dist_01555' format CSV;
select length(splitByChar('*', data_path)), replaceRegexpOne(data_path, '^.*/([^/]*)/' , '\\1'), extract(last_exception, 'AUTHENTICATION_FAILED'), dateDiff('s', last_exception_time, now()) < 3600 from system.distribution_queue where database = currentDatabase() and table = 'dist_01555' format CSV;
drop table dist_01555;
@ -31,7 +31,7 @@ create table dist_01555 (key Int) Engine=Distributed(test_cluster_with_incorrect
insert into dist_01555 values (1)(2);
-- since test_cluster_with_incorrect_pw contains incorrect password ignore error
system flush distributed dist_01555; -- { serverError 516 }
select length(splitByChar('*', data_path)), replaceRegexpOne(data_path, '^.*/([^/]*)/' , '\\1'), extract(last_exception, 'AUTHENTICATION_FAILED'), dateDiff('s', last_exception_time, now()) < 5 from system.distribution_queue where database = currentDatabase() and table = 'dist_01555' format CSV;
select length(splitByChar('*', data_path)), replaceRegexpOne(data_path, '^.*/([^/]*)/' , '\\1'), extract(last_exception, 'AUTHENTICATION_FAILED'), dateDiff('s', last_exception_time, now()) < 3600 from system.distribution_queue where database = currentDatabase() and table = 'dist_01555' format CSV;
drop table dist_01555;

View File

@ -25,11 +25,11 @@ NULL Nullable(Nothing)
(1, 1) Tuple(UInt8, UInt8)
(1,1)
--
array((1, 1)) Array(Tuple(UInt8, UInt8))
[(1, 1)] Array(Tuple(UInt8, UInt8))
[(1,1)]
NULL Nullable(Nothing)
1 UInt8
\'test\' String
[1, 2, 3] Array(UInt8)
array((1, 1), (1, 1)) Array(Tuple(UInt8, UInt8))
[(1, 1), (1, 1)] Array(Tuple(UInt8, UInt8))
\N 1 test [1,2,3] [(1,1),(1,1)]

View File

@ -13,7 +13,7 @@ concat(\'Value_1\', \'Value_2\') String
SELECT '--';
--
DESCRIBE (SELECT cast(tuple(1, 'Value'), 'Tuple (id UInt64, value String)'));
CAST(tuple(1, \'Value\'), \'Tuple (id UInt64, value String)\') Tuple(id UInt64, value String)
CAST((1, \'Value\'), \'Tuple (id UInt64, value String)\') Tuple(id UInt64, value String)
SELECT 'Columns';
Columns
DESCRIBE (SELECT test_table.id, test_table.id, id FROM test_table);
@ -737,3 +737,18 @@ t2.id UInt64
t2.value String
t3.id UInt64
t3.value String
SELECT 'Special functions array, tuple';
Special functions array, tuple
DESCRIBE (SELECT [], array(), [1], array(1), [1, 2], array(1, 2), tuple(1), (1, 2), [[], []], [([], [])], ([], []), ([([], []), ([], [])]));
[] Array(Nothing)
[] Array(Nothing)
[1] Array(UInt8)
[1] Array(UInt8)
[1, 2] Array(UInt8)
[1, 2] Array(UInt8)
(1) Tuple(UInt8)
(1, 2) Tuple(UInt8, UInt8)
[[], []] Array(Array(Nothing))
[([], [])] Array(Tuple(Array(Nothing), Array(Nothing)))
([], []) Tuple(Array(Nothing), Array(Nothing))
[([], []), ([], [])] Array(Tuple(Array(Nothing), Array(Nothing)))

View File

@ -533,6 +533,9 @@ SELECT '--';
DESCRIBE (SELECT id, value, t1.id, t1.value, t2.id, t2.value, t3.id, t3.value
FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 USING (id, value) INNER JOIN test_table_join_3 AS t3 USING (id, value));
SELECT 'Special functions array, tuple';
DESCRIBE (SELECT [], array(), [1], array(1), [1, 2], array(1, 2), tuple(1), (1, 2), [[], []], [([], [])], ([], []), ([([], []), ([], [])]));
-- { echoOff }
DROP TABLE test_table_join_1;

View File

@ -31,7 +31,7 @@ SELECT
FORMAT Vertical;
Row 1:
──────
cutURLParameter('http://bigmir.net/?a=b&c=d', array()): http://bigmir.net/?a=b&c=d
cutURLParameter('http://bigmir.net/?a=b&c=d', []): http://bigmir.net/?a=b&c=d
cutURLParameter('http://bigmir.net/?a=b&c=d', ['a']): http://bigmir.net/?c=d
cutURLParameter('http://bigmir.net/?a=b&c=d', ['a', 'c']): http://bigmir.net/?
cutURLParameter('http://bigmir.net/?a=b&c=d', ['c']): http://bigmir.net/?a=b
@ -44,7 +44,7 @@ cutURLParameter('http://bigmir.net/?a=b&c=d#e&g=h', ['c', 'g']): http:
cutURLParameter('http://bigmir.net/?a=b&c=d#e&g=h', ['e', 'g']): http://bigmir.net/?a=b&c=d#e
cutURLParameter('http://bigmir.net/?a=b&c=d#test?e=f&g=h', ['test', 'e']): http://bigmir.net/?a=b&c=d#test?g=h
cutURLParameter('http://bigmir.net/?a=b&c=d#test?e=f&g=h', ['test', 'g']): http://bigmir.net/?a=b&c=d#test?e=f
cutURLParameter('//bigmir.net/?a=b&c=d', array()): //bigmir.net/?a=b&c=d
cutURLParameter('//bigmir.net/?a=b&c=d', []): //bigmir.net/?a=b&c=d
cutURLParameter('//bigmir.net/?a=b&c=d', ['a']): //bigmir.net/?c=d
cutURLParameter('//bigmir.net/?a=b&c=d', ['a', 'c']): //bigmir.net/?
cutURLParameter('//bigmir.net/?a=b&c=d#e=f', ['a', 'e']): //bigmir.net/?c=d#
@ -89,7 +89,7 @@ SELECT
FORMAT Vertical;
Row 1:
──────
cutURLParameter(materialize('http://bigmir.net/?a=b&c=d'), array()): http://bigmir.net/?a=b&c=d
cutURLParameter(materialize('http://bigmir.net/?a=b&c=d'), []): http://bigmir.net/?a=b&c=d
cutURLParameter(materialize('http://bigmir.net/?a=b&c=d'), ['a']): http://bigmir.net/?c=d
cutURLParameter(materialize('http://bigmir.net/?a=b&c=d'), ['a', 'c']): http://bigmir.net/?
cutURLParameter(materialize('http://bigmir.net/?a=b&c=d'), ['c']): http://bigmir.net/?a=b
@ -102,7 +102,7 @@ cutURLParameter(materialize('http://bigmir.net/?a=b&c=d#e&g=h'), ['c', 'g']):
cutURLParameter(materialize('http://bigmir.net/?a=b&c=d#e&g=h'), ['e', 'g']): http://bigmir.net/?a=b&c=d#e
cutURLParameter(materialize('http://bigmir.net/?a=b&c=d#test?e=f&g=h'), ['test', 'e']): http://bigmir.net/?a=b&c=d#test?g=h
cutURLParameter(materialize('http://bigmir.net/?a=b&c=d#test?e=f&g=h'), ['test', 'g']): http://bigmir.net/?a=b&c=d#test?e=f
cutURLParameter(materialize('//bigmir.net/?a=b&c=d'), array()): //bigmir.net/?a=b&c=d
cutURLParameter(materialize('//bigmir.net/?a=b&c=d'), []): //bigmir.net/?a=b&c=d
cutURLParameter(materialize('//bigmir.net/?a=b&c=d'), ['a']): //bigmir.net/?c=d
cutURLParameter(materialize('//bigmir.net/?a=b&c=d'), ['a', 'c']): //bigmir.net/?
cutURLParameter(materialize('//bigmir.net/?a=b&c=d#e=f'), ['a', 'e']): //bigmir.net/?c=d#

View File

@ -1,6 +1,6 @@
QUERY id: 0
PROJECTION COLUMNS
uniqCombined(tuple(\'\')) UInt64
uniqCombined((\'\')) UInt64
PROJECTION
LIST id: 1, nodes: 1
FUNCTION id: 2, function_name: uniqCombined, function_type: aggregate, result_type: UInt64

View File

@ -5,12 +5,12 @@ tupleElement(CAST(tuple(1), 'Tuple(a Int)'), 1): 1
tupleElement(CAST(tuple('s'), 'Tuple(a String)'), 1): s
Row 1:
──────
tupleElement(CAST(tuple(1), 'Tuple(a Int)'), 'a'): 1
tupleElement(CAST(tuple('s'), 'Tuple(a String)'), 'a'): s
tupleElement(CAST((1), 'Tuple(a Int)'), 'a'): 1
tupleElement(CAST(('s'), 'Tuple(a String)'), 'a'): s
Row 1:
──────
tupleElement(CAST(tuple(1), 'Tuple(a Int)'), 'a'): 1
tupleElement(CAST(tuple(1), 'Tuple(a Int)'), 'a'): 1
tupleElement(CAST((1), 'Tuple(a Int)'), 'a'): 1
tupleElement(CAST((1), 'Tuple(a Int)'), 'a'): 1
-- tuple element alias + untuple() alias
Row 1:
──────
@ -44,12 +44,12 @@ tupleElement(CAST(tuple(1), 'Tuple(Int)'), 1): 1
tupleElement(CAST(tuple('s'), 'Tuple(String)'), 1): s
Row 1:
──────
tupleElement(CAST(tuple(1), 'Tuple(Int)'), '1'): 1
tupleElement(CAST(tuple('s'), 'Tuple(String)'), '1'): s
tupleElement(CAST((1), 'Tuple(Int)'), '1'): 1
tupleElement(CAST(('s'), 'Tuple(String)'), '1'): s
Row 1:
──────
tupleElement(CAST(tuple(1), 'Tuple(Int)'), '1'): 1
tupleElement(CAST(tuple(1), 'Tuple(Int)'), '1'): 1
tupleElement(CAST((1), 'Tuple(Int)'), '1'): 1
tupleElement(CAST((1), 'Tuple(Int)'), '1'): 1
-- tuple() loses the column names (would be good to fix, see #36773)
Row 1:
──────

View File

@ -0,0 +1,31 @@
-- Original issue with max_insert_delayed_streams_for_parallel_write = 1
-- Landing
2022-09-01 12:23:34 42
2023-09-01 12:23:34 42
-- MV
2022-09-01 12:00:00 84
2023-09-01 12:00:00 42
-- Original issue with deduplicate_blocks_in_dependent_materialized_views = 0 AND max_insert_delayed_streams_for_parallel_write > 1
-- Landing
2022-09-01 12:23:34 42
2023-09-01 12:23:34 42
-- MV
2022-09-01 12:00:00 42
-- Original issue with deduplicate_blocks_in_dependent_materialized_views = 1 AND max_insert_delayed_streams_for_parallel_write > 1
-- Landing
2022-09-01 12:23:34 42
2023-09-01 12:23:34 42
-- MV
2022-09-01 12:00:00 42
2023-09-01 12:00:00 42
-- Regression introduced in https://github.com/ClickHouse/ClickHouse/pull/54184
-- Landing (Agg/Replacing)MergeTree
org-1 prod checkout user 1
org-1 prod login account 1
org-1 prod login user 1
org-1 stage login user 1
--- MV
org-1 prod checkout user 1
org-1 prod login account 3
org-1 prod login user 3
org-1 stage login user 1

View File

@ -0,0 +1,205 @@
-- Tags: replica
SET session_timezone = 'UTC';
SELECT '-- Original issue with max_insert_delayed_streams_for_parallel_write = 1';
/*
This is the expected behavior when mv deduplication is set to false.
- 1st insert works for landing and mv tables
- 2nd insert gets first block 20220901 deduplicated and second one inserted in landing table
- 2nd insert gets both blocks inserted in mv table
*/
SET deduplicate_blocks_in_dependent_materialized_views = 0, max_insert_delayed_streams_for_parallel_write = 1;
CREATE TABLE landing
(
time DateTime,
number Int64
)
Engine=ReplicatedReplacingMergeTree('/clickhouse/' || currentDatabase() || '/landing/{shard}/', '{replica}')
PARTITION BY toYYYYMMDD(time)
ORDER BY time;
CREATE MATERIALIZED VIEW mv
ENGINE = ReplicatedSummingMergeTree('/clickhouse/' || currentDatabase() || '/mv/{shard}/', '{replica}')
PARTITION BY toYYYYMMDD(hour) ORDER BY hour
AS SELECT
toStartOfHour(time) AS hour,
sum(number) AS sum_amount
FROM landing
GROUP BY hour;
INSERT INTO landing VALUES ('2022-09-01 12:23:34', 42);
INSERT INTO landing VALUES ('2022-09-01 12:23:34', 42),('2023-09-01 12:23:34', 42);
SELECT '-- Landing';
SELECT * FROM landing FINAL ORDER BY time;
SELECT '-- MV';
SELECT * FROM mv FINAL ORDER BY hour;
DROP TABLE IF EXISTS landing SYNC;
DROP TABLE IF EXISTS mv SYNC;
SELECT '-- Original issue with deduplicate_blocks_in_dependent_materialized_views = 0 AND max_insert_delayed_streams_for_parallel_write > 1';
/*
This is the unexpected behavior due to setting max_insert_delayed_streams_for_parallel_write > 1.
This unexpected behavior was present since version 21.9 or earlier but due to this PR https://github.com/ClickHouse/ClickHouse/pull/34780
when max_insert_delayed_streams_for_parallel_write setting it to 1 by default the issue was mitigated.
This is what happens:
- 1st insert works for landing and mv tables
- 2nd insert gets first block 20220901 deduplicated and second one inserted in landing table
- 2nd insert is not inserting anything in mv table due to a bug computing blocks to be discarded
*/
SET deduplicate_blocks_in_dependent_materialized_views = 0, max_insert_delayed_streams_for_parallel_write = 10;
CREATE TABLE landing
(
time DateTime,
number Int64
)
Engine=ReplicatedReplacingMergeTree('/clickhouse/' || currentDatabase() || '/landing/{shard}/', '{replica}')
PARTITION BY toYYYYMMDD(time)
ORDER BY time;
CREATE MATERIALIZED VIEW mv
ENGINE = ReplicatedSummingMergeTree('/clickhouse/' || currentDatabase() || '/mv/{shard}/', '{replica}')
PARTITION BY toYYYYMMDD(hour) ORDER BY hour
AS SELECT
toStartOfHour(time) AS hour,
sum(number) AS sum_amount
FROM landing
GROUP BY hour;
INSERT INTO landing VALUES ('2022-09-01 12:23:34', 42);
INSERT INTO landing VALUES ('2022-09-01 12:23:34', 42),('2023-09-01 12:23:34', 42);
SELECT '-- Landing';
SELECT * FROM landing FINAL ORDER BY time;
SELECT '-- MV';
SELECT * FROM mv FINAL ORDER BY hour;
SET max_insert_delayed_streams_for_parallel_write = 1;
DROP TABLE IF EXISTS landing SYNC;
DROP TABLE IF EXISTS mv SYNC;
SELECT '-- Original issue with deduplicate_blocks_in_dependent_materialized_views = 1 AND max_insert_delayed_streams_for_parallel_write > 1';
/*
By setting deduplicate_blocks_in_dependent_materialized_views = 1 we can make the code go through a different path getting an expected
behavior again, even with max_insert_delayed_streams_for_parallel_write > 1.
This is what happens now:
- 1st insert works for landing and mv tables
- 2nd insert gets first block 20220901 deduplicated and second one inserted for landing and mv tables
*/
SET deduplicate_blocks_in_dependent_materialized_views = 1, max_insert_delayed_streams_for_parallel_write = 10;
CREATE TABLE landing
(
time DateTime,
number Int64
)
Engine=ReplicatedReplacingMergeTree('/clickhouse/' || currentDatabase() || '/landing/{shard}/', '{replica}')
PARTITION BY toYYYYMMDD(time)
ORDER BY time;
CREATE MATERIALIZED VIEW mv
ENGINE = ReplicatedSummingMergeTree('/clickhouse/' || currentDatabase() || '/mv/{shard}/', '{replica}')
PARTITION BY toYYYYMMDD(hour) ORDER BY hour
AS SELECT
toStartOfHour(time) AS hour,
sum(number) AS sum_amount
FROM landing
GROUP BY hour;
INSERT INTO landing VALUES ('2022-09-01 12:23:34', 42);
INSERT INTO landing VALUES ('2022-09-01 12:23:34', 42),('2023-09-01 12:23:34', 42);
SELECT '-- Landing';
SELECT * FROM landing FINAL ORDER BY time;
SELECT '-- MV';
SELECT * FROM mv FINAL ORDER BY hour;
SET max_insert_delayed_streams_for_parallel_write = 1;
DROP TABLE IF EXISTS landing SYNC;
DROP TABLE IF EXISTS mv SYNC;
SELECT '-- Regression introduced in https://github.com/ClickHouse/ClickHouse/pull/54184';
/*
This is a test to prevent regression introduced in https://github.com/ClickHouse/ClickHouse/pull/54184 from happening again.
The PR was trying to fix the unexpected behavior when deduplicate_blocks_in_dependent_materialized_views = 0 AND
max_insert_delayed_streams_for_parallel_write > 1 but it ended up adding a new regression.
*/
CREATE TABLE landing
(
`time` DateTime,
`pk1` LowCardinality(String),
`pk2` LowCardinality(String),
`pk3` LowCardinality(String),
`pk4` String
)
ENGINE = ReplicatedReplacingMergeTree('/clickhouse/' || currentDatabase() || '/landing/{shard}/', '{replica}')
ORDER BY (pk1, pk2, pk3, pk4);
CREATE TABLE ds
(
`pk1` LowCardinality(String),
`pk2` LowCardinality(String),
`pk3` LowCardinality(String),
`pk4` LowCardinality(String),
`occurences` AggregateFunction(count)
)
ENGINE = ReplicatedAggregatingMergeTree('/clickhouse/' || currentDatabase() || '/ds/{shard}/', '{replica}')
ORDER BY (pk1, pk2, pk3, pk4);
CREATE MATERIALIZED VIEW mv TO ds AS
SELECT
pk1,
pk2,
pk4,
pk3,
countState() AS occurences
FROM landing
GROUP BY pk1, pk2, pk4, pk3;
INSERT INTO landing (time, pk1, pk2, pk4, pk3)
VALUES ('2023-01-01 00:00:00','org-1','prod','login','user'),('2023-01-01 00:00:00','org-1','prod','login','user'),('2023-01-01 00:00:00','org-1','prod','login','user'),('2023-02-01 00:00:00','org-1','stage','login','user'),('2023-02-01 00:00:00','org-1','prod','login','account'),('2023-02-01 00:00:00','org-1','prod','checkout','user'),('2023-03-01 00:00:00','org-1','prod','login','account'),('2023-03-01 00:00:00','org-1','prod','login','account');
SELECT '-- Landing (Agg/Replacing)MergeTree';
SELECT
pk1,
pk2,
pk4,
pk3,
count() as occurences
FROM landing
GROUP BY pk1, pk2, pk4, pk3
ORDER BY pk1, pk2, pk4, pk3;
SELECT '--- MV';
SELECT
pk1,
pk2,
pk4,
pk3,
countMerge(occurences) AS occurences
FROM ds
GROUP BY pk1, pk2, pk4, pk3
ORDER BY pk1, pk2, pk4, pk3;
DROP TABLE IF EXISTS landing SYNC;
DROP TABLE IF EXISTS ds SYNC;
DROP TABLE IF EXISTS mv SYNC;