mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 15:42:02 +00:00
Merge pull request #56540 from slvrtrn/concat-arbitrary-types
Support arguments of arbitary type in function `concat()`
This commit is contained in:
commit
4353507724
@ -429,7 +429,7 @@ SELECT format('{} {}', 'Hello', 'World')
|
||||
|
||||
## concat
|
||||
|
||||
Concatenates the strings listed in the arguments without separator.
|
||||
Concatenates the given arguments.
|
||||
|
||||
**Syntax**
|
||||
|
||||
@ -439,7 +439,9 @@ concat(s1, s2, ...)
|
||||
|
||||
**Arguments**
|
||||
|
||||
Values of type String or FixedString.
|
||||
At least two values of arbitrary type.
|
||||
|
||||
Arguments which are not of types [String](../../sql-reference/data-types/string.md) or [FixedString](../../sql-reference/data-types/fixedstring.md) are converted to strings using their default serialization. As this decreases performance, it is not recommended to use non-String/FixedString arguments.
|
||||
|
||||
**Returned values**
|
||||
|
||||
@ -449,6 +451,8 @@ If any of arguments is `NULL`, the function returns `NULL`.
|
||||
|
||||
**Example**
|
||||
|
||||
Query:
|
||||
|
||||
``` sql
|
||||
SELECT concat('Hello, ', 'World!');
|
||||
```
|
||||
@ -461,6 +465,20 @@ Result:
|
||||
└─────────────────────────────┘
|
||||
```
|
||||
|
||||
Query:
|
||||
|
||||
```sql
|
||||
SELECT concat(42, 144);
|
||||
```
|
||||
|
||||
Result:
|
||||
|
||||
```result
|
||||
┌─concat(42, 144)─┐
|
||||
│ 42144 │
|
||||
└─────────────────┘
|
||||
```
|
||||
|
||||
## concatAssumeInjective
|
||||
|
||||
Like [concat](#concat) but assumes that `concat(s1, s2, ...) → sn` is injective. Can be used for optimization of GROUP BY.
|
||||
@ -526,6 +544,8 @@ Concatenates the given strings with a given separator.
|
||||
concatWithSeparator(sep, expr1, expr2, expr3...)
|
||||
```
|
||||
|
||||
Alias: `concat_ws`
|
||||
|
||||
**Arguments**
|
||||
|
||||
- sep — separator. Const [String](../../sql-reference/data-types/string.md) or [FixedString](../../sql-reference/data-types/fixedstring.md).
|
||||
|
@ -62,7 +62,7 @@ public:
|
||||
return buffer;
|
||||
}
|
||||
|
||||
inline void rowWritten()
|
||||
void rowWritten()
|
||||
{
|
||||
if constexpr (std::is_same_v<ColumnType, ColumnFixedString>)
|
||||
{
|
||||
|
@ -1247,9 +1247,9 @@ struct ConvertImplGenericToString
|
||||
|
||||
FormatSettings format_settings;
|
||||
auto serialization = type.getDefaultSerialization();
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
for (size_t row = 0; row < size; ++row)
|
||||
{
|
||||
serialization->serializeText(col_from, i, write_buffer, format_settings);
|
||||
serialization->serializeText(col_from, row, write_buffer, format_settings);
|
||||
write_helper.rowWritten();
|
||||
}
|
||||
|
||||
|
@ -2,14 +2,13 @@
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <Functions/FunctionsConversion.h>
|
||||
#include <Functions/GatherUtils/Algorithms.h>
|
||||
#include <Functions/GatherUtils/Sinks.h>
|
||||
#include <Functions/GatherUtils/Slices.h>
|
||||
#include <Functions/GatherUtils/Sources.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <base/map.h>
|
||||
#include <base/range.h>
|
||||
|
||||
#include "formatString.h"
|
||||
|
||||
@ -17,9 +16,7 @@ namespace DB
|
||||
{
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
||||
extern const int ILLEGAL_COLUMN;
|
||||
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
||||
}
|
||||
|
||||
using namespace GatherUtils;
|
||||
@ -32,7 +29,7 @@ class ConcatImpl : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = Name::name;
|
||||
explicit ConcatImpl(ContextPtr context_) : context(context_) {}
|
||||
explicit ConcatImpl(ContextPtr context_) : context(context_) { }
|
||||
static FunctionPtr create(ContextPtr context) { return std::make_shared<ConcatImpl>(context); }
|
||||
|
||||
String getName() const override { return name; }
|
||||
@ -56,18 +53,6 @@ public:
|
||||
getName(),
|
||||
arguments.size());
|
||||
|
||||
for (const auto arg_idx : collections::range(0, arguments.size()))
|
||||
{
|
||||
const auto * arg = arguments[arg_idx].get();
|
||||
if (!isStringOrFixedString(arg))
|
||||
throw Exception(
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
||||
"Illegal type {} of argument {} of function {}",
|
||||
arg->getName(),
|
||||
arg_idx + 1,
|
||||
getName());
|
||||
}
|
||||
|
||||
return std::make_shared<DataTypeString>();
|
||||
}
|
||||
|
||||
@ -76,11 +61,10 @@ public:
|
||||
/// Format function is not proven to be faster for two arguments.
|
||||
/// Actually there is overhead of 2 to 5 extra instructions for each string for checking empty strings in FormatImpl.
|
||||
/// Though, benchmarks are really close, for most examples we saw executeBinary is slightly faster (0-3%).
|
||||
/// For 3 and more arguments FormatImpl is much faster (up to 50-60%).
|
||||
/// For 3 and more arguments FormatStringImpl is much faster (up to 50-60%).
|
||||
if (arguments.size() == 2)
|
||||
return executeBinary(arguments, input_rows_count);
|
||||
else
|
||||
return executeFormatImpl(arguments, input_rows_count);
|
||||
return executeFormatImpl(arguments, input_rows_count);
|
||||
}
|
||||
|
||||
private:
|
||||
@ -123,6 +107,7 @@ private:
|
||||
std::vector<const ColumnString::Offsets *> offsets(num_arguments);
|
||||
std::vector<size_t> fixed_string_sizes(num_arguments);
|
||||
std::vector<std::optional<String>> constant_strings(num_arguments);
|
||||
std::vector<ColumnString::MutablePtr> converted_col_ptrs(num_arguments);
|
||||
bool has_column_string = false;
|
||||
bool has_column_fixed_string = false;
|
||||
for (size_t i = 0; i < num_arguments; ++i)
|
||||
@ -145,8 +130,29 @@ private:
|
||||
constant_strings[i] = const_col->getValue<String>();
|
||||
}
|
||||
else
|
||||
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of argument of function {}",
|
||||
column->getName(), getName());
|
||||
{
|
||||
/// A non-String/non-FixedString-type argument: use the default serialization to convert it to String
|
||||
auto full_column = column->convertToFullIfNeeded();
|
||||
auto serialization = arguments[i].type->getDefaultSerialization();
|
||||
auto converted_col_str = ColumnString::create();
|
||||
ColumnStringHelpers::WriteHelper write_helper(*converted_col_str, column->size());
|
||||
auto & write_buffer = write_helper.getWriteBuffer();
|
||||
FormatSettings format_settings;
|
||||
for (size_t row = 0; row < column->size(); ++row)
|
||||
{
|
||||
serialization->serializeText(*full_column, row, write_buffer, format_settings);
|
||||
write_helper.rowWritten();
|
||||
}
|
||||
write_helper.finalize();
|
||||
|
||||
/// Same as the normal `ColumnString` branch
|
||||
has_column_string = true;
|
||||
data[i] = &converted_col_str->getChars();
|
||||
offsets[i] = &converted_col_str->getOffsets();
|
||||
|
||||
/// Keep the pointer alive
|
||||
converted_col_ptrs[i] = std::move(converted_col_str);
|
||||
}
|
||||
}
|
||||
|
||||
String pattern;
|
||||
@ -155,7 +161,7 @@ private:
|
||||
for (size_t i = 0; i < num_arguments; ++i)
|
||||
pattern += "{}";
|
||||
|
||||
FormatImpl::formatExecute(
|
||||
FormatStringImpl::formatExecute(
|
||||
has_column_string,
|
||||
has_column_fixed_string,
|
||||
std::move(pattern),
|
||||
@ -185,14 +191,15 @@ using FunctionConcat = ConcatImpl<NameConcat, false>;
|
||||
using FunctionConcatAssumeInjective = ConcatImpl<NameConcatAssumeInjective, true>;
|
||||
|
||||
|
||||
/// Also works with arrays.
|
||||
/// Works with arrays via `arrayConcat`, maps via `mapConcat`, and tuples via `tupleConcat`.
|
||||
/// Additionally, allows concatenation of arbitrary types that can be cast to string using the corresponding default serialization.
|
||||
class ConcatOverloadResolver : public IFunctionOverloadResolver
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "concat";
|
||||
static FunctionOverloadResolverPtr create(ContextPtr context) { return std::make_unique<ConcatOverloadResolver>(context); }
|
||||
|
||||
explicit ConcatOverloadResolver(ContextPtr context_) : context(context_) {}
|
||||
explicit ConcatOverloadResolver(ContextPtr context_) : context(context_) { }
|
||||
|
||||
String getName() const override { return name; }
|
||||
size_t getNumberOfArguments() const override { return 0; }
|
||||
@ -200,29 +207,26 @@ public:
|
||||
|
||||
FunctionBasePtr buildImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & return_type) const override
|
||||
{
|
||||
if (isArray(arguments.at(0).type))
|
||||
{
|
||||
if (std::ranges::all_of(arguments, [](const auto & elem) { return isArray(elem.type); }))
|
||||
return FunctionFactory::instance().getImpl("arrayConcat", context)->build(arguments);
|
||||
}
|
||||
else if (isMap(arguments.at(0).type))
|
||||
{
|
||||
if (std::ranges::all_of(arguments, [](const auto & elem) { return isMap(elem.type); }))
|
||||
return FunctionFactory::instance().getImpl("mapConcat", context)->build(arguments);
|
||||
}
|
||||
else if (isTuple(arguments.at(0).type))
|
||||
{
|
||||
if (std::ranges::all_of(arguments, [](const auto & elem) { return isTuple(elem.type); }))
|
||||
return FunctionFactory::instance().getImpl("tupleConcat", context)->build(arguments);
|
||||
}
|
||||
else
|
||||
return std::make_unique<FunctionToFunctionBaseAdaptor>(
|
||||
FunctionConcat::create(context), collections::map<DataTypes>(arguments, [](const auto & elem) { return elem.type; }), return_type);
|
||||
return std::make_unique<FunctionToFunctionBaseAdaptor>(
|
||||
FunctionConcat::create(context),
|
||||
collections::map<DataTypes>(arguments, [](const auto & elem) { return elem.type; }),
|
||||
return_type);
|
||||
}
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
if (arguments.size() < 2)
|
||||
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
|
||||
throw Exception(
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
|
||||
"Number of arguments for function {} doesn't match: passed {}, should be at least 2.",
|
||||
getName(), arguments.size());
|
||||
getName(),
|
||||
arguments.size());
|
||||
|
||||
/// We always return Strings from concat, even if arguments were fixed strings.
|
||||
return std::make_shared<DataTypeString>();
|
||||
|
@ -1,5 +1,5 @@
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Columns/ColumnFixedString.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
@ -14,9 +14,9 @@ namespace DB
|
||||
{
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
||||
extern const int ILLEGAL_COLUMN;
|
||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
||||
extern const int ILLEGAL_COLUMN;
|
||||
}
|
||||
|
||||
namespace
|
||||
@ -26,7 +26,7 @@ class ConcatWithSeparatorImpl : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = Name::name;
|
||||
explicit ConcatWithSeparatorImpl(ContextPtr context_) : context(context_) {}
|
||||
explicit ConcatWithSeparatorImpl(ContextPtr context_) : context(context_) { }
|
||||
|
||||
static FunctionPtr create(ContextPtr context) { return std::make_shared<ConcatWithSeparatorImpl>(context); }
|
||||
|
||||
@ -113,8 +113,7 @@ public:
|
||||
else if (const ColumnConst * const_col = checkAndGetColumnConstStringOrFixedString(column.get()))
|
||||
constant_strings[2 * i] = const_col->getValue<String>();
|
||||
else
|
||||
throw Exception(ErrorCodes::ILLEGAL_COLUMN,
|
||||
"Illegal column {} of argument of function {}", column->getName(), getName());
|
||||
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of argument of function {}", column->getName(), getName());
|
||||
}
|
||||
|
||||
String pattern;
|
||||
@ -122,7 +121,7 @@ public:
|
||||
for (size_t i = 0; i < num_args; ++i)
|
||||
pattern += "{}";
|
||||
|
||||
FormatImpl::formatExecute(
|
||||
FormatStringImpl::formatExecute(
|
||||
has_column_string,
|
||||
has_column_fixed_string,
|
||||
std::move(pattern),
|
||||
@ -156,14 +155,14 @@ using FunctionConcatWithSeparatorAssumeInjective = ConcatWithSeparatorImpl<NameC
|
||||
REGISTER_FUNCTION(ConcatWithSeparator)
|
||||
{
|
||||
factory.registerFunction<FunctionConcatWithSeparator>(FunctionDocumentation{
|
||||
.description=R"(
|
||||
.description = R"(
|
||||
Returns the concatenation strings separated by string separator. Syntax: concatWithSeparator(sep, expr1, expr2, expr3...)
|
||||
)",
|
||||
.examples{{"concatWithSeparator", "SELECT concatWithSeparator('a', '1', '2', '3')", ""}},
|
||||
.categories{"String"}});
|
||||
|
||||
factory.registerFunction<FunctionConcatWithSeparatorAssumeInjective>(FunctionDocumentation{
|
||||
.description=R"(
|
||||
.description = R"(
|
||||
Same as concatWithSeparator, the difference is that you need to ensure that concatWithSeparator(sep, expr1, expr2, expr3...) → result is injective, it will be used for optimization of GROUP BY.
|
||||
|
||||
The function is named “injective” if it always returns different result for different values of arguments. In other words: different arguments never yield identical result.
|
||||
@ -171,7 +170,7 @@ The function is named “injective” if it always returns different result for
|
||||
.examples{{"concatWithSeparatorAssumeInjective", "SELECT concatWithSeparatorAssumeInjective('a', '1', '2', '3')", ""}},
|
||||
.categories{"String"}});
|
||||
|
||||
/// Compatibility with Spark:
|
||||
/// Compatibility with Spark and MySQL:
|
||||
factory.registerAlias("concat_ws", "concatWithSeparator", FunctionFactory::CaseInsensitive);
|
||||
}
|
||||
|
||||
|
@ -110,7 +110,7 @@ public:
|
||||
column->getName(), getName());
|
||||
}
|
||||
|
||||
FormatImpl::formatExecute(
|
||||
FormatStringImpl::formatExecute(
|
||||
has_column_string,
|
||||
has_column_fixed_string,
|
||||
std::move(pattern),
|
||||
|
@ -1,24 +1,19 @@
|
||||
#pragma once
|
||||
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <base/types.h>
|
||||
#include <Common/Exception.h>
|
||||
#include <Common/StringUtils/StringUtils.h>
|
||||
#include <Common/format.h>
|
||||
#include <Common/memcpySmall.h>
|
||||
|
||||
#include <base/types.h>
|
||||
|
||||
#include <algorithm>
|
||||
#include <optional>
|
||||
#include <string>
|
||||
#include <utility>
|
||||
#include <vector>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
struct FormatImpl
|
||||
struct FormatStringImpl
|
||||
{
|
||||
static constexpr size_t right_padding = 15;
|
||||
|
||||
|
@ -1 +1,67 @@
|
||||
Hello, world!
|
||||
-- Const string + non-const arbitrary type
|
||||
With 42
|
||||
With 43
|
||||
With 44
|
||||
With 45
|
||||
With 46
|
||||
With 47
|
||||
With 48
|
||||
With 49
|
||||
With 50
|
||||
With 51
|
||||
With 52
|
||||
With 53
|
||||
With 42.42
|
||||
With 43.43
|
||||
With 44
|
||||
With true
|
||||
With false
|
||||
With foo
|
||||
With bar
|
||||
With foo
|
||||
With bar
|
||||
With foo
|
||||
With bar
|
||||
With foo
|
||||
With bar
|
||||
With 42
|
||||
With 42
|
||||
With fae310ca-d52a-4923-9e9b-02bf67f4b009
|
||||
With 2023-11-14
|
||||
With 2123-11-14
|
||||
With 2023-11-14 05:50:12
|
||||
With 2023-11-14 05:50:12.123
|
||||
With hallo
|
||||
With [\'foo\',\'bar\']
|
||||
With {"foo":"bar"}
|
||||
With (42,\'foo\')
|
||||
With {42:\'foo\'}
|
||||
With 122.233.64.201
|
||||
With 2001:1:130f:2:3:9c0:876a:130b
|
||||
With (42,43)
|
||||
With [(0,0),(10,0),(10,10),(0,10)]
|
||||
With [[(20,20),(50,20),(50,50),(20,50)],[(30,30),(50,50),(50,30)]]
|
||||
With [[[(0,0),(10,0),(10,10),(0,10)]],[[(20,20),(50,20),(50,50),(20,50)],[(30,30),(50,50),(50,30)]]]
|
||||
-- SimpleAggregateFunction
|
||||
With 42
|
||||
With 4
|
||||
-- Nested
|
||||
With [\'foo\',\'bar\'][\'qaz\',\'qux\']
|
||||
-- NULL arguments
|
||||
\N
|
||||
\N
|
||||
\N
|
||||
\N
|
||||
\N
|
||||
\N
|
||||
\N
|
||||
-- Various arguments tests
|
||||
Non-const strings
|
||||
Two arguments test
|
||||
Three arguments test
|
||||
3 arguments test with int type
|
||||
42144
|
||||
42144255
|
||||
42144
|
||||
42144255
|
||||
Testing the alias
|
||||
|
@ -1 +1,89 @@
|
||||
SELECT CONCAT('Hello', ', ', 'world!');
|
||||
-- Tags: no-fasttest
|
||||
-- no-fasttest: json type needs rapidjson library, geo types need s2 geometry
|
||||
|
||||
SET allow_experimental_object_type = 1;
|
||||
SET allow_suspicious_low_cardinality_types=1;
|
||||
|
||||
SELECT '-- Const string + non-const arbitrary type';
|
||||
SELECT concat('With ', materialize(42 :: Int8));
|
||||
SELECT concat('With ', materialize(43 :: Int16));
|
||||
SELECT concat('With ', materialize(44 :: Int32));
|
||||
SELECT concat('With ', materialize(45 :: Int64));
|
||||
SELECT concat('With ', materialize(46 :: Int128));
|
||||
SELECT concat('With ', materialize(47 :: Int256));
|
||||
SELECT concat('With ', materialize(48 :: UInt8));
|
||||
SELECT concat('With ', materialize(49 :: UInt16));
|
||||
SELECT concat('With ', materialize(50 :: UInt32));
|
||||
SELECT concat('With ', materialize(51 :: UInt64));
|
||||
SELECT concat('With ', materialize(52 :: UInt128));
|
||||
SELECT concat('With ', materialize(53 :: UInt256));
|
||||
SELECT concat('With ', materialize(42.42 :: Float32));
|
||||
SELECT concat('With ', materialize(43.43 :: Float64));
|
||||
SELECT concat('With ', materialize(44.44 :: Decimal(2)));
|
||||
SELECT concat('With ', materialize(true :: Bool));
|
||||
SELECT concat('With ', materialize(false :: Bool));
|
||||
SELECT concat('With ', materialize('foo' :: String));
|
||||
SELECT concat('With ', materialize('bar' :: FixedString(3)));
|
||||
SELECT concat('With ', materialize('foo' :: Nullable(String)));
|
||||
SELECT concat('With ', materialize('bar' :: Nullable(FixedString(3))));
|
||||
SELECT concat('With ', materialize('foo' :: LowCardinality(String)));
|
||||
SELECT concat('With ', materialize('bar' :: LowCardinality(FixedString(3))));
|
||||
SELECT concat('With ', materialize('foo' :: LowCardinality(Nullable(String))));
|
||||
SELECT concat('With ', materialize('bar' :: LowCardinality(Nullable(FixedString(3)))));
|
||||
SELECT concat('With ', materialize(42 :: LowCardinality(Nullable(UInt32))));
|
||||
SELECT concat('With ', materialize(42 :: LowCardinality(UInt32)));
|
||||
SELECT concat('With ', materialize('fae310ca-d52a-4923-9e9b-02bf67f4b009' :: UUID));
|
||||
SELECT concat('With ', materialize('2023-11-14' :: Date));
|
||||
SELECT concat('With ', materialize('2123-11-14' :: Date32));
|
||||
SELECT concat('With ', materialize('2023-11-14 05:50:12' :: DateTime('Europe/Amsterdam')));
|
||||
SELECT concat('With ', materialize('2023-11-14 05:50:12.123' :: DateTime64(3, 'Europe/Amsterdam')));
|
||||
SELECT concat('With ', materialize('hallo' :: Enum('hallo' = 1)));
|
||||
SELECT concat('With ', materialize(['foo', 'bar'] :: Array(String)));
|
||||
SELECT concat('With ', materialize('{"foo": "bar"}' :: JSON));
|
||||
SELECT concat('With ', materialize((42, 'foo') :: Tuple(Int32, String)));
|
||||
SELECT concat('With ', materialize(map(42, 'foo') :: Map(Int32, String)));
|
||||
SELECT concat('With ', materialize('122.233.64.201' :: IPv4));
|
||||
SELECT concat('With ', materialize('2001:0001:130F:0002:0003:09C0:876A:130B' :: IPv6));
|
||||
SELECT concat('With ', materialize((42, 43) :: Point));
|
||||
SELECT concat('With ', materialize([(0,0),(10,0),(10,10),(0,10)] :: Ring));
|
||||
SELECT concat('With ', materialize([[(20, 20), (50, 20), (50, 50), (20, 50)], [(30, 30), (50, 50), (50, 30)]] :: Polygon));
|
||||
SELECT concat('With ', materialize([[[(0, 0), (10, 0), (10, 10), (0, 10)]], [[(20, 20), (50, 20), (50, 50), (20, 50)],[(30, 30), (50, 50), (50, 30)]]] :: MultiPolygon));
|
||||
|
||||
SELECT '-- SimpleAggregateFunction';
|
||||
DROP TABLE IF EXISTS concat_saf_test;
|
||||
CREATE TABLE concat_saf_test(x SimpleAggregateFunction(max, Int32)) ENGINE=MergeTree ORDER BY tuple();
|
||||
INSERT INTO concat_saf_test VALUES (42);
|
||||
INSERT INTO concat_saf_test SELECT max(number) FROM numbers(5);
|
||||
SELECT concat('With ', x) FROM concat_saf_test ORDER BY x DESC;
|
||||
DROP TABLE concat_saf_test;
|
||||
|
||||
SELECT '-- Nested';
|
||||
DROP TABLE IF EXISTS concat_nested_test;
|
||||
CREATE TABLE concat_nested_test(attrs Nested(k String, v String)) ENGINE = MergeTree ORDER BY tuple();
|
||||
INSERT INTO concat_nested_test VALUES (['foo', 'bar'], ['qaz', 'qux']);
|
||||
SELECT concat('With ', attrs.k, attrs.v) FROM concat_nested_test;
|
||||
DROP TABLE concat_nested_test;
|
||||
|
||||
SELECT '-- NULL arguments';
|
||||
SELECT concat(NULL, NULL);
|
||||
SELECT concat(NULL, materialize(NULL :: Nullable(UInt64)));
|
||||
SELECT concat(materialize(NULL :: Nullable(UInt64)), materialize(NULL :: Nullable(UInt64)));
|
||||
SELECT concat(42, materialize(NULL :: Nullable(UInt64)));
|
||||
SELECT concat('42', materialize(NULL :: Nullable(UInt64)));
|
||||
SELECT concat(42, materialize(NULL :: Nullable(UInt64)), materialize(NULL :: Nullable(UInt64)));
|
||||
SELECT concat('42', materialize(NULL :: Nullable(UInt64)), materialize(NULL :: Nullable(UInt64)));
|
||||
|
||||
SELECT '-- Various arguments tests';
|
||||
SELECT concat(materialize('Non-const'), materialize(' strings'));
|
||||
SELECT concat('Two arguments ', 'test');
|
||||
SELECT concat('Three ', 'arguments', ' test');
|
||||
SELECT concat(materialize(3 :: Int64), ' arguments test', ' with int type');
|
||||
SELECT concat(materialize(42 :: Int32), materialize(144 :: UInt64));
|
||||
SELECT concat(materialize(42 :: Int32), materialize(144 :: UInt64), materialize(255 :: UInt32));
|
||||
SELECT concat(42, 144);
|
||||
SELECT concat(42, 144, 255);
|
||||
|
||||
SELECT CONCAT('Testing the ', 'alias');
|
||||
|
||||
SELECT concat(); -- { serverError 42 }
|
||||
SELECT concat(1); -- { serverError 42 }
|
||||
|
@ -26,7 +26,7 @@ SELECT n, source, inter FROM (
|
||||
# Test INTERPOLATE with incompatible expression - should produce error
|
||||
SELECT n, source, inter FROM (
|
||||
SELECT toFloat32(number % 10) AS n, 'original' AS source, number as inter FROM numbers(10) WHERE number % 3 = 1
|
||||
) ORDER BY n WITH FILL FROM 0 TO 11.51 STEP 0.5 INTERPOLATE (inter AS inter||'inter'); -- { serverError 44 }
|
||||
) ORDER BY n WITH FILL FROM 0 TO 11.51 STEP 0.5 INTERPOLATE (inter AS reverse(inter)); -- { serverError 44 }
|
||||
|
||||
# Test INTERPOLATE with column from WITH FILL expression - should produce error
|
||||
SELECT n, source, inter FROM (
|
||||
|
@ -117,5 +117,5 @@ SELECT arrayMap(x -> concat(concat(concat(concat(concat(toString(id), '___\0____
|
||||
FROM test_table WHERE concat(concat(concat(toString(id), '___\0_______\0____'), toString(id)), concat(toString(id), NULL), toString(id));
|
||||
SELECT '--';
|
||||
--
|
||||
SELECT arrayMap(x -> concat(toString(id), arrayMap(x -> toString(1), [NULL])), [NULL]) FROM test_table; -- { serverError 44 };
|
||||
SELECT arrayMap(x -> splitByChar(toString(id), arrayMap(x -> toString(1), [NULL])), [NULL]) FROM test_table; -- { serverError 44 };
|
||||
DROP TABLE test_table;
|
||||
|
@ -122,7 +122,7 @@ FROM test_table WHERE concat(concat(concat(toString(id), '___\0_______\0____'),
|
||||
|
||||
SELECT '--';
|
||||
|
||||
SELECT arrayMap(x -> concat(toString(id), arrayMap(x -> toString(1), [NULL])), [NULL]) FROM test_table; -- { serverError 44 };
|
||||
SELECT arrayMap(x -> splitByChar(toString(id), arrayMap(x -> toString(1), [NULL])), [NULL]) FROM test_table; -- { serverError 44 };
|
||||
|
||||
DROP TABLE test_table;
|
||||
|
||||
|
@ -8,4 +8,4 @@ SELECT id, value_element, value FROM test_table ARRAY JOIN [[1,2,3]] AS value_el
|
||||
0 [1,2,3] 3
|
||||
SELECT value_element, value FROM test_table ARRAY JOIN [1048577] AS value_element, arrayMap(x -> value_element, ['']) AS value;
|
||||
1048577 [1048577]
|
||||
SELECT arrayFilter(x -> notEmpty(concat(x)), [NULL, NULL]) FROM system.one ARRAY JOIN [1048577] AS elem, arrayMap(x -> concat(x, elem, ''), ['']) AS unused; -- { serverError 44 }
|
||||
SELECT arrayFilter(x -> notEmpty(concat(x)), [NULL, NULL]) FROM system.one ARRAY JOIN [1048577] AS elem, arrayMap(x -> splitByChar(x, elem), ['']) AS unused; -- { serverError 44 }
|
||||
|
@ -17,7 +17,7 @@ SELECT id, value_element, value FROM test_table ARRAY JOIN [[1,2,3]] AS value_el
|
||||
|
||||
SELECT value_element, value FROM test_table ARRAY JOIN [1048577] AS value_element, arrayMap(x -> value_element, ['']) AS value;
|
||||
|
||||
SELECT arrayFilter(x -> notEmpty(concat(x)), [NULL, NULL]) FROM system.one ARRAY JOIN [1048577] AS elem, arrayMap(x -> concat(x, elem, ''), ['']) AS unused; -- { serverError 44 }
|
||||
SELECT arrayFilter(x -> notEmpty(concat(x)), [NULL, NULL]) FROM system.one ARRAY JOIN [1048577] AS elem, arrayMap(x -> splitByChar(x, elem), ['']) AS unused; -- { serverError 44 }
|
||||
|
||||
-- { echoOff }
|
||||
|
||||
|
@ -5,7 +5,7 @@
|
||||
-- Tests the output of SHOW COLUMNS when called through the ClickHouse protocol.
|
||||
|
||||
-- -----------------------------------------------------------------------------------
|
||||
-- Please keep this test in-sync with 02775_show_columns_called_through_mysql.sql
|
||||
-- Please keep this test in-sync with 02775_show_columns_called_from_clickhouse.expect
|
||||
-- -----------------------------------------------------------------------------------
|
||||
|
||||
DROP TABLE IF EXISTS tab;
|
||||
|
@ -6,7 +6,7 @@
|
||||
# Tests the output of SHOW COLUMNS when called through the MySQL protocol.
|
||||
|
||||
# -----------------------------------------------------------------------------------
|
||||
# Please keep this test in-sync with 02775_show_columns_called_through_clickhouse.sql
|
||||
# Please keep this test in-sync with 02775_show_columns_called_from_clickhouse.sql
|
||||
# -----------------------------------------------------------------------------------
|
||||
|
||||
set basedir [file dirname $argv0]
|
||||
|
Loading…
Reference in New Issue
Block a user