mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-24 16:42:05 +00:00
Merge branch 'master' of github.com:ClickHouse/ClickHouse into feature-quantileBfloat16
This commit is contained in:
commit
1b7fe54f45
@ -173,9 +173,6 @@ def parse_env_variables(build_type, compiler, sanitizer, package_type, image_typ
|
||||
cmake_flags.append('-DUSE_GTEST=1')
|
||||
cmake_flags.append('-DENABLE_TESTS=1')
|
||||
cmake_flags.append('-DENABLE_EXAMPLES=1')
|
||||
cmake_flags.append('-DENABLE_FUZZING=1')
|
||||
# For fuzzing needs
|
||||
cmake_flags.append('-DUSE_YAML_CPP=1')
|
||||
# Don't stop on first error to find more clang-tidy errors in one run.
|
||||
result.append('NINJA_FLAGS=-k0')
|
||||
|
||||
|
@ -303,6 +303,7 @@ function run_tests
|
||||
01683_codec_encrypted # Depends on OpenSSL
|
||||
01776_decrypt_aead_size_check # Depends on OpenSSL
|
||||
01811_filter_by_null # Depends on OpenSSL
|
||||
02012_sha512_fixedstring # Depends on OpenSSL
|
||||
01281_unsucceeded_insert_select_queries_counter
|
||||
01292_create_user
|
||||
01294_lazy_database_concurrent
|
||||
|
@ -255,7 +255,7 @@ windowFunnel(window, [mode, [mode, ... ]])(timestamp, cond1, cond2, ..., condN)
|
||||
|
||||
- `window` — Length of the sliding window, it is the time interval between the first and the last condition. The unit of `window` depends on the `timestamp` itself and varies. Determined using the expression `timestamp of cond1 <= timestamp of cond2 <= ... <= timestamp of condN <= timestamp of cond1 + window`.
|
||||
- `mode` — It is an optional argument. One or more modes can be set.
|
||||
- `'strict'` — If same condition holds for sequence of events then such non-unique events would be skipped.
|
||||
- `'strict_deduplication'` — If the same condition holds for the sequence of events, then such repeating event interrupts further processing.
|
||||
- `'strict_order'` — Don't allow interventions of other events. E.g. in the case of `A->B->D->C`, it stops finding `A->B->C` at the `D` and the max event level is 2.
|
||||
- `'strict_increase'` — Apply conditions only to events with strictly increasing timestamps.
|
||||
|
||||
|
@ -143,7 +143,9 @@ It works faster than intHash32. Average quality.
|
||||
|
||||
## SHA256 {#sha256}
|
||||
|
||||
Calculates SHA-1, SHA-224, or SHA-256 from a string and returns the resulting set of bytes as FixedString(20), FixedString(28), or FixedString(32).
|
||||
## SHA512 {#sha512}
|
||||
|
||||
Calculates SHA-1, SHA-224, SHA-256 or SHA-512 from a string and returns the resulting set of bytes as FixedString(20), FixedString(28), FixedString(32), or FixedString(64).
|
||||
The function works fairly slowly (SHA-1 processes about 5 million short strings per second per processor core, while SHA-224 and SHA-256 process about 2.2 million).
|
||||
We recommend using this function only in cases when you need a specific hash function and you can’t select it.
|
||||
Even in these cases, we recommend applying the function offline and pre-calculating values when inserting them into the table, instead of applying it in SELECTS.
|
||||
|
@ -38,13 +38,13 @@ ENGINE = MySQL('host:port', ['database' | database], 'user', 'password')
|
||||
| BIGINT | [Int64](../../sql-reference/data-types/int-uint.md) |
|
||||
| FLOAT | [Float32](../../sql-reference/data-types/float.md) |
|
||||
| DOUBLE | [Float64](../../sql-reference/data-types/float.md) |
|
||||
| DATE | [日期](../../sql-reference/data-types/date.md) |
|
||||
| DATETIME, TIMESTAMP | [日期时间](../../sql-reference/data-types/datetime.md) |
|
||||
| BINARY | [固定字符串](../../sql-reference/data-types/fixedstring.md) |
|
||||
| DATE | [Date](../../sql-reference/data-types/date.md) |
|
||||
| DATETIME, TIMESTAMP | [DateTime](../../sql-reference/data-types/datetime.md) |
|
||||
| BINARY | [FixedString](../../sql-reference/data-types/fixedstring.md) |
|
||||
|
||||
其他的MySQL数据类型将全部都转换为[字符串](../../sql-reference/data-types/string.md)。
|
||||
其他的MySQL数据类型将全部都转换为[String](../../sql-reference/data-types/string.md)。
|
||||
|
||||
同时以上的所有类型都支持[可为空](../../sql-reference/data-types/nullable.md)。
|
||||
同时以上的所有类型都支持[Nullable](../../sql-reference/data-types/nullable.md)。
|
||||
|
||||
## 使用示例 {#shi-yong-shi-li}
|
||||
|
||||
|
@ -357,6 +357,7 @@ void Server::createServer(const std::string & listen_host, const char * port_nam
|
||||
try
|
||||
{
|
||||
func(port);
|
||||
global_context->registerServerPort(port_name, port);
|
||||
}
|
||||
catch (const Poco::Exception &)
|
||||
{
|
||||
|
@ -137,8 +137,8 @@ class AggregateFunctionWindowFunnel final
|
||||
private:
|
||||
UInt64 window;
|
||||
UInt8 events_size;
|
||||
/// When the 'strict' is set, it applies conditions only for the not repeating values.
|
||||
bool strict;
|
||||
/// When the 'strict_deduplication' is set, it applies conditions only for the not repeating values.
|
||||
bool strict_deduplication;
|
||||
|
||||
/// When the 'strict_order' is set, it doesn't allow interventions of other events.
|
||||
/// In the case of 'A->B->D->C', it stops finding 'A->B->C' at the 'D' and the max event level is 2.
|
||||
@ -150,7 +150,7 @@ private:
|
||||
/// Loop through the entire events_list, update the event timestamp value
|
||||
/// The level path must be 1---2---3---...---check_events_size, find the max event level that satisfied the path in the sliding window.
|
||||
/// If found, returns the max event level, else return 0.
|
||||
/// The Algorithm complexity is O(n).
|
||||
/// The algorithm works in O(n) time, but the overall function works in O(n * log(n)) due to sorting.
|
||||
UInt8 getEventLevel(Data & data) const
|
||||
{
|
||||
if (data.size() == 0)
|
||||
@ -163,10 +163,10 @@ private:
|
||||
/// events_timestamp stores the timestamp of the first and previous i-th level event happen within time window
|
||||
std::vector<std::optional<std::pair<UInt64, UInt64>>> events_timestamp(events_size);
|
||||
bool first_event = false;
|
||||
for (const auto & pair : data.events_list)
|
||||
for (size_t i = 0; i < data.events_list.size(); ++i)
|
||||
{
|
||||
const T & timestamp = pair.first;
|
||||
const auto & event_idx = pair.second - 1;
|
||||
const T & timestamp = data.events_list[i].first;
|
||||
const auto & event_idx = data.events_list[i].second - 1;
|
||||
if (strict_order && event_idx == -1)
|
||||
{
|
||||
if (first_event)
|
||||
@ -179,9 +179,9 @@ private:
|
||||
events_timestamp[0] = std::make_pair(timestamp, timestamp);
|
||||
first_event = true;
|
||||
}
|
||||
else if (strict && events_timestamp[event_idx].has_value())
|
||||
else if (strict_deduplication && events_timestamp[event_idx].has_value())
|
||||
{
|
||||
return event_idx + 1;
|
||||
return data.events_list[i - 1].second;
|
||||
}
|
||||
else if (strict_order && first_event && !events_timestamp[event_idx - 1].has_value())
|
||||
{
|
||||
@ -226,18 +226,20 @@ public:
|
||||
events_size = arguments.size() - 1;
|
||||
window = params.at(0).safeGet<UInt64>();
|
||||
|
||||
strict = false;
|
||||
strict_deduplication = false;
|
||||
strict_order = false;
|
||||
strict_increase = false;
|
||||
for (size_t i = 1; i < params.size(); ++i)
|
||||
{
|
||||
String option = params.at(i).safeGet<String>();
|
||||
if (option == "strict")
|
||||
strict = true;
|
||||
if (option == "strict_deduplication")
|
||||
strict_deduplication = true;
|
||||
else if (option == "strict_order")
|
||||
strict_order = true;
|
||||
else if (option == "strict_increase")
|
||||
strict_increase = true;
|
||||
else if (option == "strict")
|
||||
throw Exception{"strict is replaced with strict_deduplication in Aggregate function " + getName(), ErrorCodes::BAD_ARGUMENTS};
|
||||
else
|
||||
throw Exception{"Aggregate function " + getName() + " doesn't support a parameter: " + option, ErrorCodes::BAD_ARGUMENTS};
|
||||
}
|
||||
|
@ -55,7 +55,7 @@ class IColumn;
|
||||
M(Seconds, receive_timeout, DBMS_DEFAULT_RECEIVE_TIMEOUT_SEC, "", 0) \
|
||||
M(Seconds, send_timeout, DBMS_DEFAULT_SEND_TIMEOUT_SEC, "", 0) \
|
||||
M(Seconds, drain_timeout, DBMS_DEFAULT_DRAIN_TIMEOUT_SEC, "", 0) \
|
||||
M(Seconds, tcp_keep_alive_timeout, 0, "The time in seconds the connection needs to remain idle before TCP starts sending keepalive probes", 0) \
|
||||
M(Seconds, tcp_keep_alive_timeout, 290 /* less than DBMS_DEFAULT_RECEIVE_TIMEOUT_SEC */, "The time in seconds the connection needs to remain idle before TCP starts sending keepalive probes", 0) \
|
||||
M(Milliseconds, hedged_connection_timeout_ms, DBMS_DEFAULT_HEDGED_CONNECTION_TIMEOUT_MS, "Connection timeout for establishing connection with replica for Hedged requests", 0) \
|
||||
M(Milliseconds, receive_data_timeout_ms, DBMS_DEFAULT_RECEIVE_DATA_TIMEOUT_MS, "Connection timeout for receiving first packet of data or packet with positive progress from replica", 0) \
|
||||
M(Bool, use_hedged_requests, true, "Use hedged requests for distributed queries", 0) \
|
||||
|
@ -14,6 +14,7 @@ void registerFunctionsHashing(FunctionFactory & factory)
|
||||
factory.registerFunction<FunctionSHA1>();
|
||||
factory.registerFunction<FunctionSHA224>();
|
||||
factory.registerFunction<FunctionSHA256>();
|
||||
factory.registerFunction<FunctionSHA512>();
|
||||
#endif
|
||||
factory.registerFunction<FunctionSipHash64>();
|
||||
factory.registerFunction<FunctionSipHash128>();
|
||||
|
@ -193,6 +193,20 @@ struct SHA256Impl
|
||||
SHA256_Final(out_char_data, &ctx);
|
||||
}
|
||||
};
|
||||
|
||||
struct SHA512Impl
|
||||
{
|
||||
static constexpr auto name = "SHA512";
|
||||
enum { length = 64 };
|
||||
|
||||
static void apply(const char * begin, const size_t size, unsigned char * out_char_data)
|
||||
{
|
||||
SHA512_CTX ctx;
|
||||
SHA512_Init(&ctx);
|
||||
SHA512_Update(&ctx, reinterpret_cast<const unsigned char *>(begin), size);
|
||||
SHA512_Final(out_char_data, &ctx);
|
||||
}
|
||||
};
|
||||
#endif
|
||||
|
||||
struct SipHash64Impl
|
||||
@ -1318,6 +1332,7 @@ using FunctionMD5 = FunctionStringHashFixedString<MD5Impl>;
|
||||
using FunctionSHA1 = FunctionStringHashFixedString<SHA1Impl>;
|
||||
using FunctionSHA224 = FunctionStringHashFixedString<SHA224Impl>;
|
||||
using FunctionSHA256 = FunctionStringHashFixedString<SHA256Impl>;
|
||||
using FunctionSHA512 = FunctionStringHashFixedString<SHA512Impl>;
|
||||
#endif
|
||||
using FunctionSipHash128 = FunctionStringHashFixedString<SipHash128Impl>;
|
||||
using FunctionCityHash64 = FunctionAnyHash<ImplCityHash64>;
|
||||
|
@ -696,6 +696,8 @@ struct JSONExtractTree
|
||||
{
|
||||
if (element.isString())
|
||||
return JSONExtractStringImpl<JSONParser>::insertResultToColumn(dest, element, {});
|
||||
else if (element.isNull())
|
||||
return false;
|
||||
else
|
||||
return JSONExtractRawImpl<JSONParser>::insertResultToColumn(dest, element, {});
|
||||
}
|
||||
|
136
src/Functions/getServerPort.cpp
Normal file
136
src/Functions/getServerPort.cpp
Normal file
@ -0,0 +1,136 @@
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <Interpreters/Context.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
||||
extern const int ILLEGAL_COLUMN;
|
||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||
}
|
||||
|
||||
namespace
|
||||
{
|
||||
|
||||
class ExecutableFunctionGetServerPort : public IExecutableFunction
|
||||
{
|
||||
public:
|
||||
explicit ExecutableFunctionGetServerPort(UInt16 port_) : port(port_) {}
|
||||
|
||||
String getName() const override { return "getServerPort"; }
|
||||
|
||||
bool useDefaultImplementationForNulls() const override { return false; }
|
||||
|
||||
ColumnPtr executeImpl(const ColumnsWithTypeAndName &, const DataTypePtr &, size_t input_rows_count) const override
|
||||
{
|
||||
return DataTypeNumber<UInt16>().createColumnConst(input_rows_count, port);
|
||||
}
|
||||
|
||||
private:
|
||||
UInt16 port;
|
||||
};
|
||||
|
||||
class FunctionBaseGetServerPort : public IFunctionBase
|
||||
{
|
||||
public:
|
||||
explicit FunctionBaseGetServerPort(bool is_distributed_, UInt16 port_, DataTypes argument_types_, DataTypePtr return_type_)
|
||||
: is_distributed(is_distributed_), port(port_), argument_types(std::move(argument_types_)), return_type(std::move(return_type_))
|
||||
{
|
||||
}
|
||||
|
||||
String getName() const override { return "getServerPort"; }
|
||||
|
||||
const DataTypes & getArgumentTypes() const override
|
||||
{
|
||||
return argument_types;
|
||||
}
|
||||
|
||||
const DataTypePtr & getResultType() const override
|
||||
{
|
||||
return return_type;
|
||||
}
|
||||
|
||||
bool isDeterministic() const override { return false; }
|
||||
bool isDeterministicInScopeOfQuery() const override { return true; }
|
||||
bool isSuitableForConstantFolding() const override { return !is_distributed; }
|
||||
bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; }
|
||||
|
||||
ExecutableFunctionPtr prepare(const ColumnsWithTypeAndName &) const override
|
||||
{
|
||||
return std::make_unique<ExecutableFunctionGetServerPort>(port);
|
||||
}
|
||||
|
||||
private:
|
||||
bool is_distributed;
|
||||
UInt16 port;
|
||||
DataTypes argument_types;
|
||||
DataTypePtr return_type;
|
||||
};
|
||||
|
||||
class GetServerPortOverloadResolver : public IFunctionOverloadResolver, WithContext
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "getServerPort";
|
||||
|
||||
String getName() const override { return name; }
|
||||
|
||||
static FunctionOverloadResolverPtr create(ContextPtr context_)
|
||||
{
|
||||
return std::make_unique<GetServerPortOverloadResolver>(context_);
|
||||
}
|
||||
|
||||
explicit GetServerPortOverloadResolver(ContextPtr context_) : WithContext(context_) {}
|
||||
|
||||
size_t getNumberOfArguments() const override { return 1; }
|
||||
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {0}; }
|
||||
bool isDeterministic() const override { return false; }
|
||||
bool isDeterministicInScopeOfQuery() const override { return true; }
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & data_types) const override
|
||||
{
|
||||
size_t number_of_arguments = data_types.size();
|
||||
if (number_of_arguments != 1)
|
||||
throw Exception(
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
|
||||
"Number of arguments for function {} doesn't match: passed {}, should be 1",
|
||||
getName(),
|
||||
number_of_arguments);
|
||||
return std::make_shared<DataTypeNumber<UInt16>>();
|
||||
}
|
||||
|
||||
FunctionBasePtr buildImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & return_type) const override
|
||||
{
|
||||
if (!isString(arguments[0].type))
|
||||
throw Exception(
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
||||
"The argument of function {} should be a constant string with the name of a setting",
|
||||
getName());
|
||||
const auto * column = arguments[0].column.get();
|
||||
if (!column || !checkAndGetColumnConstStringOrFixedString(column))
|
||||
throw Exception(
|
||||
ErrorCodes::ILLEGAL_COLUMN,
|
||||
"The argument of function {} should be a constant string with the name of a setting",
|
||||
getName());
|
||||
|
||||
String port_name{column->getDataAt(0)};
|
||||
auto port = getContext()->getServerPort(port_name);
|
||||
|
||||
DataTypes argument_types;
|
||||
argument_types.emplace_back(arguments.back().type);
|
||||
return std::make_unique<FunctionBaseGetServerPort>(getContext()->isDistributed(), port, argument_types, return_type);
|
||||
}
|
||||
};
|
||||
|
||||
}
|
||||
|
||||
void registerFunctionGetServerPort(FunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<GetServerPortOverloadResolver>();
|
||||
}
|
||||
|
||||
}
|
@ -71,6 +71,7 @@ void registerFunctionHasThreadFuzzer(FunctionFactory &);
|
||||
void registerFunctionInitializeAggregation(FunctionFactory &);
|
||||
void registerFunctionErrorCodeToName(FunctionFactory &);
|
||||
void registerFunctionTcpPort(FunctionFactory &);
|
||||
void registerFunctionGetServerPort(FunctionFactory &);
|
||||
void registerFunctionByteSize(FunctionFactory &);
|
||||
void registerFunctionFile(FunctionFactory & factory);
|
||||
void registerFunctionConnectionId(FunctionFactory & factory);
|
||||
@ -150,6 +151,7 @@ void registerFunctionsMiscellaneous(FunctionFactory & factory)
|
||||
registerFunctionInitializeAggregation(factory);
|
||||
registerFunctionErrorCodeToName(factory);
|
||||
registerFunctionTcpPort(factory);
|
||||
registerFunctionGetServerPort(factory);
|
||||
registerFunctionByteSize(factory);
|
||||
registerFunctionFile(factory);
|
||||
registerFunctionConnectionId(factory);
|
||||
|
@ -228,6 +228,8 @@ struct ContextSharedPart
|
||||
ConfigurationPtr clusters_config; /// Stores updated configs
|
||||
mutable std::mutex clusters_mutex; /// Guards clusters and clusters_config
|
||||
|
||||
std::map<String, UInt16> server_ports;
|
||||
|
||||
bool shutdown_called = false;
|
||||
|
||||
Stopwatch uptime_watch;
|
||||
@ -1798,6 +1800,20 @@ std::optional<UInt16> Context::getTCPPortSecure() const
|
||||
return {};
|
||||
}
|
||||
|
||||
void Context::registerServerPort(String port_name, UInt16 port)
|
||||
{
|
||||
shared->server_ports.emplace(std::move(port_name), port);
|
||||
}
|
||||
|
||||
UInt16 Context::getServerPort(const String & port_name) const
|
||||
{
|
||||
auto it = shared->server_ports.find(port_name);
|
||||
if (it == shared->server_ports.end())
|
||||
throw Exception(ErrorCodes::BAD_GET, "There is no port named {}", port_name);
|
||||
else
|
||||
return it->second;
|
||||
}
|
||||
|
||||
std::shared_ptr<Cluster> Context::getCluster(const std::string & cluster_name) const
|
||||
{
|
||||
auto res = getClusters()->getCluster(cluster_name);
|
||||
|
@ -580,6 +580,11 @@ public:
|
||||
|
||||
std::optional<UInt16> getTCPPortSecure() const;
|
||||
|
||||
/// Register server ports during server starting up. No lock is held.
|
||||
void registerServerPort(String port_name, UInt16 port);
|
||||
|
||||
UInt16 getServerPort(const String & port_name) const;
|
||||
|
||||
/// For methods below you may need to acquire the context lock by yourself.
|
||||
|
||||
ContextMutablePtr getQueryContext() const;
|
||||
|
@ -37,6 +37,7 @@
|
||||
[5, 2]
|
||||
[6, 1]
|
||||
[7, 1]
|
||||
[1]
|
||||
[1, 2]
|
||||
[2, 2]
|
||||
[3, 0]
|
||||
|
@ -43,7 +43,7 @@ drop table if exists funnel_test_strict;
|
||||
create table funnel_test_strict (timestamp UInt32, event UInt32) engine=Memory;
|
||||
insert into funnel_test_strict values (00,1000),(10,1001),(20,1002),(30,1003),(40,1004),(50,1005),(51,1005),(60,1006),(70,1007),(80,1008);
|
||||
|
||||
select 6 = windowFunnel(10000, 'strict')(timestamp, event = 1000, event = 1001, event = 1002, event = 1003, event = 1004, event = 1005, event = 1006) from funnel_test_strict;
|
||||
select 6 = windowFunnel(10000, 'strict_deduplication')(timestamp, event = 1000, event = 1001, event = 1002, event = 1003, event = 1004, event = 1005, event = 1006) from funnel_test_strict;
|
||||
select 7 = windowFunnel(10000)(timestamp, event = 1000, event = 1001, event = 1002, event = 1003, event = 1004, event = 1005, event = 1006) from funnel_test_strict;
|
||||
|
||||
|
||||
@ -62,11 +62,18 @@ insert into funnel_test_strict_order values (1, 5, 'a') (2, 5, 'a') (3, 5, 'b')
|
||||
insert into funnel_test_strict_order values (1, 6, 'c') (2, 6, 'c') (3, 6, 'b') (4, 6, 'b') (5, 6, 'a') (6, 6, 'a');
|
||||
select user, windowFunnel(86400)(dt, event='a', event='b', event='c') as s from funnel_test_strict_order group by user order by user format JSONCompactEachRow;
|
||||
select user, windowFunnel(86400, 'strict_order')(dt, event='a', event='b', event='c') as s from funnel_test_strict_order group by user order by user format JSONCompactEachRow;
|
||||
select user, windowFunnel(86400, 'strict', 'strict_order')(dt, event='a', event='b', event='c') as s from funnel_test_strict_order group by user order by user format JSONCompactEachRow;
|
||||
select user, windowFunnel(86400, 'strict_deduplication', 'strict_order')(dt, event='a', event='b', event='c') as s from funnel_test_strict_order group by user order by user format JSONCompactEachRow;
|
||||
insert into funnel_test_strict_order values (1, 7, 'a') (2, 7, 'c') (3, 7, 'b');
|
||||
select user, windowFunnel(10, 'strict_order')(dt, event = 'a', event = 'b', event = 'c') as s from funnel_test_strict_order where user = 7 group by user format JSONCompactEachRow;
|
||||
drop table funnel_test_strict_order;
|
||||
|
||||
--https://github.com/ClickHouse/ClickHouse/issues/27469
|
||||
drop table if exists strict_BiteTheDDDD;
|
||||
create table strict_BiteTheDDDD (ts UInt64, event String) engine = Log();
|
||||
insert into strict_BiteTheDDDD values (1,'a') (2,'b') (3,'c') (4,'b') (5,'d');
|
||||
select 3 = windowFunnel(86400, 'strict_deduplication')(ts, event='a', event='b', event='c', event='d') from strict_BiteTheDDDD format JSONCompactEachRow;
|
||||
drop table strict_BiteTheDDDD;
|
||||
|
||||
drop table if exists funnel_test_non_null;
|
||||
create table funnel_test_non_null (`dt` DateTime, `u` int, `a` Nullable(String), `b` Nullable(String)) engine = MergeTree() partition by dt order by u;
|
||||
insert into funnel_test_non_null values (1, 1, 'a1', 'b1') (2, 1, 'a2', 'b2');
|
||||
|
@ -1 +1,2 @@
|
||||
('123','456','[7,8,9]')
|
||||
\N
|
||||
|
@ -1 +1,3 @@
|
||||
select JSONExtract('{"a": "123", "b": 456, "c": [7, 8, 9]}', 'Tuple(a String, b String, c String)');
|
||||
|
||||
with '{"string_value":null}' as json select JSONExtract(json, 'string_value', 'Nullable(String)');
|
||||
|
@ -0,0 +1 @@
|
||||
9000
|
3
tests/queries/0_stateless/02012_get_server_port.sql
Normal file
3
tests/queries/0_stateless/02012_get_server_port.sql
Normal file
@ -0,0 +1,3 @@
|
||||
select getServerPort('tcp_port');
|
||||
|
||||
select getServerPort('unknown'); -- { serverError 170 }
|
22
tests/queries/0_stateless/02012_sha512_fixedstring.reference
Normal file
22
tests/queries/0_stateless/02012_sha512_fixedstring.reference
Normal file
@ -0,0 +1,22 @@
|
||||
CF83E1357EEFB8BDF1542850D66D8007D620E4050B5715DC83F4A921D36CE9CE47D0D13C5D85F2B0FF8318D2877EEC2F63B931BD47417A81A538327AF927DA3E
|
||||
DDAF35A193617ABACC417349AE20413112E6FA4E89A97EA20A9EEEE64B55D39A2192992A274FC1A836BA3C23A3FEEBBD454D4423643CE80E2A9AC94FA54CA49F
|
||||
5809F3ECB4AA006F71AF562D4381F2BF64EA0931FD530E939740D0C38F6EEB2A71FA0113A21C170569D8319B8C4DE8A1C1A5ABA1A1C5B23A886B06712D373B9E
|
||||
6FDB5E5BCCBD093ECC48DD262A99E6B867D6F48E1DAE014D26428365E7529B0022F000CBF852BEA38F43A2034E8FE7555AC41B9EA9E27FE72F4E968926998EA8
|
||||
8018978D8AAE19322205E4CACFA045CDF7A0C4A5773A93FD24331064AFC5726F324B76802AA6FC30DFC412A6E5C3EEF4693AE4E2D0A1EA24A2D3EC46439B7923
|
||||
22F1080BFD1CEA5B86CFDEFE129D2FF67E756AA3DA0EADB4C6EBA86A0710F3C2E0F91815A28062BDA4FBF23E5FA1BD66A3CFDA37AC7354516943AB6E08E88106
|
||||
08788919600C5C9B93704570858D8B64D5B335FAD2B23E28A5E68A08BCC4AD9BCA1991178FEA22BDAAA3C9C2FEB061FF4919C1C840DFA8188858D80EDCE98499
|
||||
0EFD835B217A94C32F45B9C3720644D7A4A49DD3A74EDE4257CFBB74164C80382322B8EA7B6131747609BA1B2776E84AEC2820D9414FC5F66B6A42D3F90F1D9F
|
||||
C161DA64EE30E86768040C4BB7B88C3FAE5A1E79CA2441B2DB66552B126C00F952D2D854DF2F5D56FB85ED57C5A39E95BFE485A6AF70963A81BE67DFDB96CFBF
|
||||
F65B5379D687EA1FB38E32F8251F0FB270061DC6DF9AECA1E258A6F1BAFF70D4E979FC9315ED1FAE24D000CC0EF7937D02703B2061694B3DCA2308C5C0779184
|
||||
DF25395F5FFF00E5B64BF5EEAFD94A1A21B4F4FC4FAE01DF48C27C77E0C9DC6FC253C7E7F32185146048190B73E3ED53BC76F626D24E9DE3FBA3EBC48B671CC0
|
||||
F1DD6EFB26D0169259FBD53150D6E0C4F2E1FFF2F9DA6B77EC9AD065AD67D33F95411CC5E5B31CEAB970974E5C8E3C6376445D8422F052CABB121BCE1449C494
|
||||
561A1DEDBB03FD632DBFDCC73E83A8DBD17D6E30B14642BAC8EBF3DF6622150A5E51ACC059E2C44EF71A00FBEEB062F2EE7A610E5462E6B4805449574AAE857F
|
||||
D59943FE43CB50F2C3AE74F9BD09F72140519EA30C1A24E6F7A4BCBBD12860D954F9752A24FBAA1D606F362564D905E8B29806903B1542D948C2B8293FC09A59
|
||||
E18A99CED7BD69979406C5A666336D7316E11240FF5F8C1632F17BEB1BD58838E0222CCD48D4227F0751592F1C731B4BB11C325F09FB15923F0334C64303D73A
|
||||
0508B92D15B0750343F9B3E2218C7D54BB6D9A3ABA5FA941977832AA89300516505E91034E4ECD4812A7A58438251651C00974DDCD3C471B9ED02451871F4A48
|
||||
ADFDC00D41881138C96581D43298C5724C20500C5A966144B51531FFB2FE6CE265BE3101CD19B985A2253A7B48EE4EC5EBD4B2876CF0E66F1095EB0A2521C525
|
||||
758ABA28159AABFFF659E6698397AD7A9EBBA77AA3FFC25551B7CF2057930AD051D357866259E5288B0B837E3F5C4ED1D1229320595F914381903CD48DF76CB2
|
||||
E7A061D9B066E2CA44CF959A76FC04D8B02998CB9D46A60C19E015EA9389F3F9595CBBC4CC46E1319B02B3269FBD21F62D15A6F13428F8389CB4507AF6DB6D89
|
||||
3438D75650E1EDB8A11EF4F63A5DFF239A70B28B6A14F13FCFDD14D02BE8BD00E84DF956C159CFDC85D6E44DB62D00E9206F40453FFD9CC97C38449527D33FF6
|
||||
DB229C3A53B0340E94EFDA4D03B54F161313699757CAC312F377B731AE6C62010E0C0010E78F73E6D6B0BB438F644D176244B2614897799F9FA3F85DA980C218
|
||||
FDD9FD54050D95855B8E3A34F3A54E309E1CA87CD44A8506EB10051D1CA650DB64ABD0BE4F4F44E45F630C22CA270FA7694AC2261DF2EFD766B8CED53F285A27
|
14
tests/queries/0_stateless/02012_sha512_fixedstring.sql
Normal file
14
tests/queries/0_stateless/02012_sha512_fixedstring.sql
Normal file
@ -0,0 +1,14 @@
|
||||
SELECT hex(SHA512(''));
|
||||
SELECT hex(SHA512('abc'));
|
||||
|
||||
DROP TABLE IF EXISTS defaults;
|
||||
CREATE TABLE defaults
|
||||
(
|
||||
s FixedString(20)
|
||||
)ENGINE = Memory();
|
||||
|
||||
INSERT INTO defaults SELECT s FROM generateRandom('s FixedString(20)', 1, 1, 1) LIMIT 20;
|
||||
|
||||
SELECT hex(SHA512(s)) FROM defaults;
|
||||
|
||||
DROP TABLE defaults;
|
Loading…
Reference in New Issue
Block a user