Merge branch 'master' of github.com:ClickHouse/ClickHouse into no-jemalloc-message

This commit is contained in:
Alexey Milovidov 2024-06-23 21:34:55 +02:00
commit 8d75fd53b2
20 changed files with 106 additions and 147 deletions

View File

@ -5418,11 +5418,14 @@ When set to `false` than all attempts are made with identical timeouts.
Default value: `true`.
## uniform_snowflake_conversion_functions {#uniform_snowflake_conversion_functions}
## allow_deprecated_snowflake_conversion_functions {#allow_deprecated_snowflake_conversion_functions}
If set to `true`, then functions `snowflakeIDToDateTime`, `snowflakeIDToDateTime64`, `dateTimeToSnowflakeID`, and `dateTime64ToSnowflakeID` are enabled, and functions `snowflakeToDateTime`, `snowflakeToDateTime64`, `dateTimeToSnowflake`, and `dateTime64ToSnowflake` are disabled (and vice versa if set to `false`).
Functions `snowflakeToDateTime`, `snowflakeToDateTime64`, `dateTimeToSnowflake`, and `dateTime64ToSnowflake` are deprecated and disabled by default.
Please use functions `snowflakeIDToDateTime`, `snowflakeIDToDateTime64`, `dateTimeToSnowflakeID`, and `dateTime64ToSnowflakeID` instead.
Default value: `true`
To re-enable the deprecated functions (e.g., during a transition period), please set this setting to `true`.
Default value: `false`
## allow_experimental_variant_type {#allow_experimental_variant_type}

View File

@ -611,7 +611,7 @@ SELECT generateSnowflakeID(1), generateSnowflakeID(2);
## snowflakeToDateTime
:::warning
This function is deprecated and can only be used if setting [uniform_snowflake_conversion_functions](../../operations/settings/settings.md#uniform_snowflake_conversion_functions) is disabled.
This function is deprecated and can only be used if setting [allow_deprecated_snowflake_conversion_functions](../../operations/settings/settings.md#allow_deprecated_snowflake_conversion_functions) is enabled.
The function will be removed at some point in future.
:::
@ -652,7 +652,7 @@ Result:
## snowflakeToDateTime64
:::warning
This function is deprecated and can only be used if setting [uniform_snowflake_conversion_functions](../../operations/settings/settings.md#uniform_snowflake_conversion_functions) is disabled.
This function is deprecated and can only be used if setting [allow_deprecated_snowflake_conversion_functions](../../operations/settings/settings.md#allow_deprecated_snowflake_conversion_functions) is enabled.
The function will be removed at some point in future.
:::
@ -693,7 +693,7 @@ Result:
## dateTimeToSnowflake
:::warning
This function is deprecated and can only be used if setting [uniform_snowflake_conversion_functions](../../operations/settings/settings.md#uniform_snowflake_conversion_functions) is disabled.
This function is deprecated and can only be used if setting [allow_deprecated_snowflake_conversion_functions](../../operations/settings/settings.md#allow_deprecated_snowflake_conversion_functions) is enabled.
The function will be removed at some point in future.
:::
@ -732,7 +732,7 @@ Result:
## dateTime64ToSnowflake
:::warning
This function is deprecated and can only be used if setting [uniform_snowflake_conversion_functions](../../operations/settings/settings.md#uniform_snowflake_conversion_functions) is disabled.
This function is deprecated and can only be used if setting [allow_deprecated_snowflake_conversion_functions](../../operations/settings/settings.md#allow_deprecated_snowflake_conversion_functions) is enabled.
The function will be removed at some point in future.
:::

View File

@ -368,7 +368,7 @@ int KeeperClient::main(const std::vector<String> & /* args */)
return 0;
}
DB::ConfigProcessor config_processor(config().getString("config-file", "config.xml"));
ConfigProcessor config_processor(config().getString("config-file", "config.xml"));
/// This will handle a situation when clickhouse is running on the embedded config, but config.d folder is also present.
ConfigProcessor::registerEmbeddedConfig("config.xml", "<clickhouse/>");

View File

@ -12,8 +12,7 @@ bool parseKeeperArg(IParser::Pos & pos, Expected & expected, String & result)
if (!parseIdentifierOrStringLiteral(pos, expected, result))
return false;
}
while (pos->type != TokenType::Whitespace && pos->type != TokenType::EndOfStream && pos->type != TokenType::Semicolon)
else if (pos->type == TokenType::Number)
{
result.append(pos->begin, pos->end);
++pos;
@ -40,8 +39,8 @@ bool KeeperParser::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
for (const auto & pair : KeeperClient::commands)
expected.add(pos, pair.first.data());
for (const auto & flwc : four_letter_word_commands)
expected.add(pos, flwc.data());
for (const auto & four_letter_word_command : four_letter_word_commands)
expected.add(pos, four_letter_word_command.data());
if (pos->type != TokenType::BareWord)
return false;

View File

@ -11,7 +11,6 @@ namespace DB
{
bool parseKeeperArg(IParser::Pos & pos, Expected & expected, String & result);
bool parseKeeperPath(IParser::Pos & pos, Expected & expected, String & path);

View File

@ -934,7 +934,7 @@ class IColumn;
M(Int64, prefer_warmed_unmerged_parts_seconds, 0, "Only available in ClickHouse Cloud. If a merged part is less than this many seconds old and is not pre-warmed (see cache_populated_by_fetch), but all its source parts are available and pre-warmed, SELECT queries will read from those parts instead. Only for ReplicatedMergeTree. Note that this only checks whether CacheWarmer processed the part; if the part was fetched into cache by something else, it'll still be considered cold until CacheWarmer gets to it; if it was warmed, then evicted from cache, it'll still be considered warm.", 0) \
M(Bool, iceberg_engine_ignore_schema_evolution, false, "Ignore schema evolution in Iceberg table engine and read all data using latest schema saved on table creation. Note that it can lead to incorrect result", 0) \
M(Bool, allow_deprecated_error_prone_window_functions, false, "Allow usage of deprecated error prone window functions (neighbor, runningAccumulate, runningDifferenceStartingWithFirstValue, runningDifference)", 0) \
M(Bool, uniform_snowflake_conversion_functions, true, "Enables functions snowflakeIDToDateTime[64] and dateTime[64]ToSnowflakeID while disabling functions snowflakeToDateTime[64] and dateTime[64]ToSnowflake.", 0) \
M(Bool, allow_deprecated_snowflake_conversion_functions, false, "Enables deprecated functions snowflakeToDateTime[64] and dateTime[64]ToSnowflake.", 0) \
// End of COMMON_SETTINGS
// Please add settings related to formats into the FORMAT_FACTORY_SETTINGS, move obsolete settings to OBSOLETE_SETTINGS and obsolete format settings to OBSOLETE_FORMAT_SETTINGS.

View File

@ -102,7 +102,7 @@ static const std::map<ClickHouseVersion, SettingsChangesHistory::SettingsChanges
{"input_format_parquet_max_block_size", 8192, DEFAULT_BLOCK_SIZE, "Increase block size for parquet reader."},
{"input_format_parquet_prefer_block_bytes", 0, DEFAULT_BLOCK_SIZE * 256, "Average block bytes output by parquet reader."},
{"enable_blob_storage_log", true, true, "Write information about blob storage operations to system.blob_storage_log table"},
{"uniform_snowflake_conversion_functions", false, true, "Enable functions snowflakeIDToDateTime[64] and dateTime[64]ToSnowflakeID."},
{"allow_deprecated_snowflake_conversion_functions", true, false, "Disabled deprecated functions snowflakeToDateTime[64] and dateTime[64]ToSnowflake."},
{"allow_statistic_optimize", false, false, "Old setting which popped up here being renamed."},
{"allow_experimental_statistic", false, false, "Old setting which popped up here being renamed."},
{"allow_statistics_optimize", false, false, "The setting was renamed. The previous name is `allow_statistic_optimize`."},

View File

@ -14,31 +14,20 @@
namespace DB
{
namespace ErrorCodes
{
extern const int UNKNOWN_FUNCTION;
}
namespace
{
/// See generateSnowflakeID.cpp
constexpr int time_shift = 22;
constexpr size_t time_shift = 22;
}
class FunctionDateTimeToSnowflakeID : public IFunction
{
private:
const bool uniform_snowflake_conversion_functions;
public:
static constexpr auto name = "dateTimeToSnowflakeID";
static FunctionPtr create(ContextPtr context) { return std::make_shared<FunctionDateTimeToSnowflakeID>(context); }
explicit FunctionDateTimeToSnowflakeID(ContextPtr context)
: uniform_snowflake_conversion_functions(context->getSettingsRef().uniform_snowflake_conversion_functions)
{}
static FunctionPtr create(ContextPtr /*context*/) { return std::make_shared<FunctionDateTimeToSnowflakeID>(); }
String getName() const override { return name; }
size_t getNumberOfArguments() const override { return 0; }
@ -52,7 +41,7 @@ public:
{"value", static_cast<FunctionArgumentDescriptor::TypeValidator>(&isDateTime), nullptr, "DateTime"}
};
FunctionArgumentDescriptors optional_args{
{"epoch", static_cast<FunctionArgumentDescriptor::TypeValidator>(&isNativeUInt), isColumnConst, "UInt*"}
{"epoch", static_cast<FunctionArgumentDescriptor::TypeValidator>(&isNativeUInt), isColumnConst, "const UInt*"}
};
validateFunctionArgumentTypes(*this, arguments, args, optional_args);
@ -61,12 +50,9 @@ public:
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override
{
if (!uniform_snowflake_conversion_functions)
throw Exception(ErrorCodes::UNKNOWN_FUNCTION, "To use function {}, setting 'uniform_snowflake_conversion_functions' must be enabled", getName());
const auto & col_src = *arguments[0].column;
size_t epoch = 0;
UInt64 epoch = 0;
if (arguments.size() == 2 && input_rows_count != 0)
{
const auto & col_epoch = *arguments[1].column;
@ -86,16 +72,10 @@ public:
class FunctionDateTime64ToSnowflakeID : public IFunction
{
private:
const bool uniform_snowflake_conversion_functions;
public:
static constexpr auto name = "dateTime64ToSnowflakeID";
static FunctionPtr create(ContextPtr context) { return std::make_shared<FunctionDateTime64ToSnowflakeID>(context); }
explicit FunctionDateTime64ToSnowflakeID(ContextPtr context)
: uniform_snowflake_conversion_functions(context->getSettingsRef().uniform_snowflake_conversion_functions)
{}
static FunctionPtr create(ContextPtr /*context*/) { return std::make_shared<FunctionDateTime64ToSnowflakeID>(); }
String getName() const override { return name; }
size_t getNumberOfArguments() const override { return 0; }
@ -109,7 +89,7 @@ public:
{"value", static_cast<FunctionArgumentDescriptor::TypeValidator>(&isDateTime64), nullptr, "DateTime64"}
};
FunctionArgumentDescriptors optional_args{
{"epoch", static_cast<FunctionArgumentDescriptor::TypeValidator>(&isNativeUInt), isColumnConst, "UInt*"}
{"epoch", static_cast<FunctionArgumentDescriptor::TypeValidator>(&isNativeUInt), isColumnConst, "const UInt*"}
};
validateFunctionArgumentTypes(*this, arguments, args, optional_args);
@ -118,13 +98,10 @@ public:
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override
{
if (!uniform_snowflake_conversion_functions)
throw Exception(ErrorCodes::UNKNOWN_FUNCTION, "To use function {}, setting 'uniform_snowflake_conversion_functions' must be enabled", getName());
const auto & col_src = *arguments[0].column;
const auto & src_data = typeid_cast<const ColumnDateTime64 &>(col_src).getData();
size_t epoch = 0;
UInt64 epoch = 0;
if (arguments.size() == 2 && input_rows_count != 0)
{
const auto & col_epoch = *arguments[1].column;
@ -141,7 +118,7 @@ public:
auto factor = multiplier_msec / static_cast<double>(multiplier_src);
for (size_t i = 0; i < input_rows_count; ++i)
res_data[i] = static_cast<UInt64>(src_data[i] * factor - epoch) << time_shift;
res_data[i] = std::llround(src_data[i] * factor - epoch) << time_shift;
return col_res;
}

View File

@ -13,7 +13,7 @@
/// ------------------------------------------------------------------------------------------------------------------------------
/// The functions in this file are deprecated and should be removed in favor of functions 'snowflakeIDToDateTime[64]' and
/// 'dateTime[64]ToSnowflakeID' by summer 2025. Please also mark setting `uniform_snowflake_conversion_functions` as obsolete then.
/// 'dateTime[64]ToSnowflakeID' by summer 2025. Please also mark setting `allow_deprecated_snowflake_conversion_functions` as obsolete then.
/// ------------------------------------------------------------------------------------------------------------------------------
namespace DB
@ -40,7 +40,7 @@ constexpr int time_shift = 22;
class FunctionDateTimeToSnowflake : public IFunction
{
private:
const bool uniform_snowflake_conversion_functions;
const bool allow_deprecated_snowflake_conversion_functions;
public:
static constexpr auto name = "dateTimeToSnowflake";
@ -51,7 +51,7 @@ public:
}
explicit FunctionDateTimeToSnowflake(ContextPtr context)
: uniform_snowflake_conversion_functions(context->getSettingsRef().uniform_snowflake_conversion_functions)
: allow_deprecated_snowflake_conversion_functions(context->getSettingsRef().allow_deprecated_snowflake_conversion_functions)
{}
String getName() const override { return name; }
@ -71,8 +71,8 @@ public:
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override
{
if (uniform_snowflake_conversion_functions)
throw Exception(ErrorCodes::DEPRECATED_FUNCTION, "Function {} is deprecated, to enable it disable setting 'uniform_snowflake_conversion_functions'", getName());
if (!allow_deprecated_snowflake_conversion_functions)
throw Exception(ErrorCodes::DEPRECATED_FUNCTION, "Function {} is deprecated, to enable it set setting 'allow_deprecated_snowflake_conversion_functions' to 'true'", getName());
const auto & src = arguments[0];
const auto & src_column = *src.column;
@ -92,7 +92,7 @@ class FunctionSnowflakeToDateTime : public IFunction
{
private:
const bool allow_nonconst_timezone_arguments;
const bool uniform_snowflake_conversion_functions;
const bool allow_deprecated_snowflake_conversion_functions;
public:
static constexpr auto name = "snowflakeToDateTime";
@ -104,7 +104,7 @@ public:
explicit FunctionSnowflakeToDateTime(ContextPtr context)
: allow_nonconst_timezone_arguments(context->getSettingsRef().allow_nonconst_timezone_arguments)
, uniform_snowflake_conversion_functions(context->getSettingsRef().uniform_snowflake_conversion_functions)
, allow_deprecated_snowflake_conversion_functions(context->getSettingsRef().allow_deprecated_snowflake_conversion_functions)
{}
String getName() const override { return name; }
@ -132,8 +132,8 @@ public:
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override
{
if (uniform_snowflake_conversion_functions)
throw Exception(ErrorCodes::DEPRECATED_FUNCTION, "Function {} is deprecated, to enable it disable setting 'uniform_snowflake_conversion_functions'", getName());
if (!allow_deprecated_snowflake_conversion_functions)
throw Exception(ErrorCodes::DEPRECATED_FUNCTION, "Function {} is deprecated, to enable it set setting 'allow_deprecated_snowflake_conversion_functions' to 'true'", getName());
const auto & src = arguments[0];
const auto & src_column = *src.column;
@ -166,7 +166,7 @@ public:
class FunctionDateTime64ToSnowflake : public IFunction
{
private:
const bool uniform_snowflake_conversion_functions;
const bool allow_deprecated_snowflake_conversion_functions;
public:
static constexpr auto name = "dateTime64ToSnowflake";
@ -177,7 +177,7 @@ public:
}
explicit FunctionDateTime64ToSnowflake(ContextPtr context)
: uniform_snowflake_conversion_functions(context->getSettingsRef().uniform_snowflake_conversion_functions)
: allow_deprecated_snowflake_conversion_functions(context->getSettingsRef().allow_deprecated_snowflake_conversion_functions)
{}
String getName() const override { return name; }
@ -197,8 +197,8 @@ public:
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override
{
if (uniform_snowflake_conversion_functions)
throw Exception(ErrorCodes::DEPRECATED_FUNCTION, "Function {} is deprecated, to enable it disable setting 'uniform_snowflake_conversion_functions'", getName());
if (!allow_deprecated_snowflake_conversion_functions)
throw Exception(ErrorCodes::DEPRECATED_FUNCTION, "Function {} is deprecated, to enable it set setting 'allow_deprecated_snowflake_conversion_functions' to true", getName());
const auto & src = arguments[0];
@ -226,7 +226,7 @@ class FunctionSnowflakeToDateTime64 : public IFunction
{
private:
const bool allow_nonconst_timezone_arguments;
const bool uniform_snowflake_conversion_functions;
const bool allow_deprecated_snowflake_conversion_functions;
public:
static constexpr auto name = "snowflakeToDateTime64";
@ -238,7 +238,7 @@ public:
explicit FunctionSnowflakeToDateTime64(ContextPtr context)
: allow_nonconst_timezone_arguments(context->getSettingsRef().allow_nonconst_timezone_arguments)
, uniform_snowflake_conversion_functions(context->getSettingsRef().uniform_snowflake_conversion_functions)
, allow_deprecated_snowflake_conversion_functions(context->getSettingsRef().allow_deprecated_snowflake_conversion_functions)
{}
String getName() const override { return name; }
@ -266,8 +266,8 @@ public:
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override
{
if (uniform_snowflake_conversion_functions)
throw Exception(ErrorCodes::DEPRECATED_FUNCTION, "Function {} is deprecated, to enable it disable setting 'uniform_snowflake_conversion_functions'", getName());
if (!allow_deprecated_snowflake_conversion_functions)
throw Exception(ErrorCodes::DEPRECATED_FUNCTION, "Function {} is deprecated, to enable it set setting 'allow_deprecated_snowflake_conversion_functions' to true", getName());
const auto & src = arguments[0];
const auto & src_column = *src.column;

View File

@ -18,21 +18,19 @@ namespace DB
namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int UNKNOWN_FUNCTION;
}
namespace
{
/// See generateSnowflakeID.cpp
constexpr int time_shift = 22;
constexpr size_t time_shift = 22;
}
class FunctionSnowflakeIDToDateTime : public IFunction
{
private:
const bool uniform_snowflake_conversion_functions;
const bool allow_nonconst_timezone_arguments;
public:
@ -40,8 +38,7 @@ public:
static FunctionPtr create(ContextPtr context) { return std::make_shared<FunctionSnowflakeIDToDateTime>(context); }
explicit FunctionSnowflakeIDToDateTime(ContextPtr context)
: uniform_snowflake_conversion_functions(context->getSettingsRef().uniform_snowflake_conversion_functions)
, allow_nonconst_timezone_arguments(context->getSettings().allow_nonconst_timezone_arguments)
: allow_nonconst_timezone_arguments(context->getSettings().allow_nonconst_timezone_arguments)
{}
String getName() const override { return name; }
@ -56,7 +53,7 @@ public:
{"value", static_cast<FunctionArgumentDescriptor::TypeValidator>(&isUInt64), nullptr, "UInt64"}
};
FunctionArgumentDescriptors optional_args{
{"epoch", static_cast<FunctionArgumentDescriptor::TypeValidator>(&isNativeUInt), isColumnConst, "UInt*"},
{"epoch", static_cast<FunctionArgumentDescriptor::TypeValidator>(&isNativeUInt), isColumnConst, "const UInt*"},
{"time_zone", static_cast<FunctionArgumentDescriptor::TypeValidator>(&isString), nullptr, "String"}
};
validateFunctionArgumentTypes(*this, arguments, args, optional_args);
@ -70,12 +67,9 @@ public:
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override
{
if (!uniform_snowflake_conversion_functions)
throw Exception(ErrorCodes::UNKNOWN_FUNCTION, "To use function {}, setting 'uniform_snowflake_conversion_functions' must be enabled", getName());
const auto & col_src = *arguments[0].column;
size_t epoch = 0;
UInt64 epoch = 0;
if (arguments.size() >= 2 && input_rows_count != 0)
{
const auto & col_epoch = *arguments[1].column;
@ -108,7 +102,6 @@ public:
class FunctionSnowflakeIDToDateTime64 : public IFunction
{
private:
const bool uniform_snowflake_conversion_functions;
const bool allow_nonconst_timezone_arguments;
public:
@ -116,8 +109,7 @@ public:
static FunctionPtr create(ContextPtr context) { return std::make_shared<FunctionSnowflakeIDToDateTime64>(context); }
explicit FunctionSnowflakeIDToDateTime64(ContextPtr context)
: uniform_snowflake_conversion_functions(context->getSettingsRef().uniform_snowflake_conversion_functions)
, allow_nonconst_timezone_arguments(context->getSettings().allow_nonconst_timezone_arguments)
: allow_nonconst_timezone_arguments(context->getSettings().allow_nonconst_timezone_arguments)
{}
String getName() const override { return name; }
@ -132,7 +124,7 @@ public:
{"value", static_cast<FunctionArgumentDescriptor::TypeValidator>(&isUInt64), nullptr, "UInt64"}
};
FunctionArgumentDescriptors optional_args{
{"epoch", static_cast<FunctionArgumentDescriptor::TypeValidator>(&isNativeUInt), isColumnConst, "UInt*"},
{"epoch", static_cast<FunctionArgumentDescriptor::TypeValidator>(&isNativeUInt), isColumnConst, "const UInt*"},
{"time_zone", static_cast<FunctionArgumentDescriptor::TypeValidator>(&isString), nullptr, "String"}
};
validateFunctionArgumentTypes(*this, arguments, args, optional_args);
@ -146,12 +138,9 @@ public:
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override
{
if (!uniform_snowflake_conversion_functions)
throw Exception(ErrorCodes::UNKNOWN_FUNCTION, "To use function {}, setting 'uniform_snowflake_conversion_functions' must be enabled", getName());
const auto & col_src = *arguments[0].column;
size_t epoch = 0;
UInt64 epoch = 0;
if (arguments.size() >= 2 && input_rows_count != 0)
{
const auto & col_epoch = *arguments[1].column;

View File

@ -124,27 +124,27 @@ class KeeperClient(object):
return data
def cd(self, path: str, timeout: float = 60.0):
self.execute_query(f"cd {path}", timeout)
self.execute_query(f"cd '{path}'", timeout)
def ls(self, path: str, timeout: float = 60.0) -> list[str]:
return self.execute_query(f"ls {path}", timeout).split(" ")
return self.execute_query(f"ls '{path}'", timeout).split(" ")
def create(self, path: str, value: str, timeout: float = 60.0):
self.execute_query(f"create {path} {value}", timeout)
self.execute_query(f"create '{path}' '{value}'", timeout)
def get(self, path: str, timeout: float = 60.0) -> str:
return self.execute_query(f"get {path}", timeout)
return self.execute_query(f"get '{path}'", timeout)
def set(self, path: str, value: str, version: tp.Optional[int] = None) -> None:
self.execute_query(
f"set {path} {value} {version if version is not None else ''}"
f"set '{path}' '{value}' {version if version is not None else ''}"
)
def rm(self, path: str, version: tp.Optional[int] = None) -> None:
self.execute_query(f"rm {path} {version if version is not None else ''}")
self.execute_query(f"rm '{path}' {version if version is not None else ''}")
def exists(self, path: str, timeout: float = 60.0) -> bool:
return bool(int(self.execute_query(f"exists {path}", timeout)))
return bool(int(self.execute_query(f"exists '{path}'", timeout)))
def stop(self):
if not self.stopped:
@ -152,22 +152,22 @@ class KeeperClient(object):
self.proc.communicate(b"exit\n", timeout=10.0)
def sync(self, path: str, timeout: float = 60.0):
self.execute_query(f"sync {path}", timeout)
self.execute_query(f"sync '{path}'", timeout)
def touch(self, path: str, timeout: float = 60.0):
self.execute_query(f"touch {path}", timeout)
self.execute_query(f"touch '{path}'", timeout)
def find_big_family(self, path: str, n: int = 10, timeout: float = 60.0) -> str:
return self.execute_query(f"find_big_family {path} {n}", timeout)
return self.execute_query(f"find_big_family '{path}' {n}", timeout)
def find_super_nodes(self, threshold: int, timeout: float = 60.0) -> str:
return self.execute_query(f"find_super_nodes {threshold}", timeout)
def get_direct_children_number(self, path: str, timeout: float = 60.0) -> str:
return self.execute_query(f"get_direct_children_number {path}", timeout)
return self.execute_query(f"get_direct_children_number '{path}'", timeout)
def get_all_children_number(self, path: str, timeout: float = 60.0) -> str:
return self.execute_query(f"get_all_children_number {path}", timeout)
return self.execute_query(f"get_all_children_number '{path}'", timeout)
def delete_stale_backups(self, timeout: float = 60.0) -> str:
return self.execute_query("delete_stale_backups", timeout)
@ -196,7 +196,7 @@ class KeeperClient(object):
)
return self.execute_query(
f"reconfig {operation} {joining or leaving or new_members}", timeout
f"reconfig {operation} '{joining or leaving or new_members}'", timeout
)
@classmethod

View File

@ -127,7 +127,7 @@ def test_base_commands(client: KeeperClient):
assert client.get("/test_create_zk_node1") == "testvalue1"
client.create("/123", "1=2")
client.create("/123/321", "'foo;bar'")
client.create("/123/321", "foo;bar")
assert client.get("/123") == "1=2"
assert client.get("/123/321") == "foo;bar"

View File

@ -1,4 +1,4 @@
SET uniform_snowflake_conversion_functions = 0;
SET allow_deprecated_snowflake_conversion_functions = 1;
SELECT addMonths(toDateTime('2017-11-05 08:07:47', 'Asia/Istanbul'), 1, 'Asia/Kolkata');
SELECT addMonths(toDateTime('2017-11-05 10:37:47', 'Asia/Kolkata'), 1);

View File

@ -1,4 +1,4 @@
SET uniform_snowflake_conversion_functions = 0; -- Force-disable uniform snowflake conversion functions (in case this is randomized in CI)
SET allow_deprecated_snowflake_conversion_functions = 1; -- Force-enable deprecated snowflake conversion functions (in case this is randomized in CI)
SET session_timezone = 'Africa/Juba';
-- Error cases
@ -11,8 +11,8 @@ SELECT dateTime64ToSnowflake('abc'); -- {serverError ILLEGAL_TYPE_OF_ARGUMENT}
SELECT dateTimeToSnowflake('abc', 123); -- {serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH}
SELECT dateTime64ToSnowflake('abc', 123); -- {serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH}
SELECT dateTimeToSnowflake(now()) SETTINGS uniform_snowflake_conversion_functions = 1; -- { serverError DEPRECATED_FUNCTION }
SELECT dateTime64ToSnowflake(now64()) SETTINGS uniform_snowflake_conversion_functions = 1; -- { serverError DEPRECATED_FUNCTION }
SELECT dateTimeToSnowflake(now()) SETTINGS allow_deprecated_snowflake_conversion_functions = 0; -- { serverError DEPRECATED_FUNCTION }
SELECT dateTime64ToSnowflake(now64()) SETTINGS allow_deprecated_snowflake_conversion_functions = 0; -- { serverError DEPRECATED_FUNCTION }
SELECT '-- const / non-const inputs';

View File

@ -1,6 +1,5 @@
SET session_timezone = 'UTC'; -- disable timezone randomization
SET allow_experimental_analyzer = 1; -- The old path formats the result with different whitespaces
SET uniform_snowflake_conversion_functions = 1; -- Force-enable uniform snowflake conversion functions (in case this is randomized in CI)
SELECT '-- Negative tests';
SELECT dateTimeToSnowflakeID(); -- {serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH}
@ -12,9 +11,6 @@ SELECT dateTime64ToSnowflakeID(now64(), 'invalid_epoch'); -- {serverError ILLEG
SELECT dateTimeToSnowflakeID(now(), 42, 'too_many_args'); -- {serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH}
SELECT dateTime64ToSnowflakeID(now64(), 42, 'too_many_args'); -- {serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH}
SELECT dateTimeToSnowflakeID(now()) SETTINGS uniform_snowflake_conversion_functions = 0; -- { serverError UNKNOWN_FUNCTION }
SELECT dateTime64ToSnowflakeID(now64()) SETTINGS uniform_snowflake_conversion_functions = 0; -- { serverError UNKNOWN_FUNCTION }
SELECT '-- Return type';
SELECT toTypeName(dateTimeToSnowflakeID(now()));
SELECT toTypeName(dateTime64ToSnowflakeID(now64()));

View File

@ -1,6 +1,5 @@
SET session_timezone = 'UTC'; -- disable timezone randomization
SET allow_experimental_analyzer = 1; -- The old path formats the result with different whitespaces
SET uniform_snowflake_conversion_functions = 1; -- Force-enable uniform snowflake conversion functions (in case this is randomized in CI)
SELECT '-- Negative tests';
SELECT snowflakeIDToDateTime(); -- {serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH}
@ -16,9 +15,6 @@ SELECT snowflakeIDToDateTime64(123::UInt64, 42, 42); -- {serverError ILLEGAL_TY
SELECT snowflakeIDToDateTime(123::UInt64, 42, 'UTC', 'too_many_args'); -- {serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH}
SELECT snowflakeIDToDateTime64(123::UInt64, 42, 'UTC', 'too_many_args'); -- {serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH}
SELECT snowflakeIDToDateTime(123::UInt64) SETTINGS uniform_snowflake_conversion_functions = 0; -- { serverError UNKNOWN_FUNCTION }
SELECT snowflakeIDToDateTime64(123::UInt64) SETTINGS uniform_snowflake_conversion_functions = 0; -- { serverError UNKNOWN_FUNCTION }
SELECT '-- Return type';
SELECT toTypeName(snowflakeIDToDateTime(123::UInt64));
SELECT toTypeName(snowflakeIDToDateTime64(123::UInt64));

View File

@ -1,6 +1,6 @@
SET uniform_snowflake_conversion_functions = 0; -- Force-disable uniform snowflake conversion functions (in case this is randomized in CI)
SET allow_deprecated_snowflake_conversion_functions = 1; -- Force-enable deprecated snowflake conversion functions (in case this is randomized in CI)
-- -- Error cases
-- Error cases
SELECT snowflakeToDateTime(); -- {serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH}
SELECT snowflakeToDateTime64(); -- {serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH}
@ -10,35 +10,35 @@ SELECT snowflakeToDateTime64('abc'); -- {serverError ILLEGAL_TYPE_OF_ARGUMENT}
SELECT snowflakeToDateTime('abc', 123); -- {serverError ILLEGAL_TYPE_OF_ARGUMENT}
SELECT snowflakeToDateTime64('abc', 123); -- {serverError ILLEGAL_TYPE_OF_ARGUMENT}
SELECT snowflakeToDateTime(123::Int64) SETTINGS uniform_snowflake_conversion_functions = 1; -- { serverError DEPRECATED_FUNCTION }
SELECT snowflakeToDateTime64(123::Int64) SETTINGS uniform_snowflake_conversion_functions = 1; -- { serverError DEPRECATED_FUNCTION }
SELECT snowflakeToDateTime(123::Int64) SETTINGS allow_deprecated_snowflake_conversion_functions = 0; -- { serverError DEPRECATED_FUNCTION }
SELECT snowflakeToDateTime64(123::Int64) SETTINGS allow_deprecated_snowflake_conversion_functions = 0; -- { serverError DEPRECATED_FUNCTION }
SELECT 'const column';
WITH
CAST(1426860704886947840 AS Int64) AS i64,
'UTC' AS tz
CAST(1426860704886947840 AS Int64) AS i64,
'UTC' AS tz
SELECT
tz,
i64,
snowflakeToDateTime(i64, tz) as dt,
toTypeName(dt),
snowflakeToDateTime64(i64, tz) as dt64,
toTypeName(dt64);
tz,
i64,
snowflakeToDateTime(i64, tz) as dt,
toTypeName(dt),
snowflakeToDateTime64(i64, tz) as dt64,
toTypeName(dt64);
WITH
CAST(1426860704886947840 AS Int64) AS i64,
'Asia/Shanghai' AS tz
CAST(1426860704886947840 AS Int64) AS i64,
'Asia/Shanghai' AS tz
SELECT
tz,
i64,
snowflakeToDateTime(i64, tz) as dt,
toTypeName(dt),
snowflakeToDateTime64(i64, tz) as dt64,
toTypeName(dt64);
tz,
i64,
snowflakeToDateTime(i64, tz) as dt,
toTypeName(dt),
snowflakeToDateTime64(i64, tz) as dt64,
toTypeName(dt64);
DROP TABLE IF EXISTS tab;
CREATE TABLE tab(val Int64, tz String) engine=Log;
CREATE TABLE tab(val Int64, tz String) engine = Log;
INSERT INTO tab VALUES (42, 'Asia/Singapore');
SELECT 1 FROM tab WHERE snowflakeToDateTime(42::Int64, tz) != now() SETTINGS allow_nonconst_timezone_arguments = 1;

View File

@ -6,8 +6,8 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
path="/test-keeper-client-$CLICKHOUSE_DATABASE"
$CLICKHOUSE_KEEPER_CLIENT -q "rm $path" >& /dev/null
$CLICKHOUSE_KEEPER_CLIENT -q "rm '$path'" >& /dev/null
$CLICKHOUSE_KEEPER_CLIENT -q "create $path 'foobar'"
$CLICKHOUSE_KEEPER_CLIENT -q "rmr $path"
$CLICKHOUSE_KEEPER_CLIENT -q "get $path" 2>&1
$CLICKHOUSE_KEEPER_CLIENT -q "create '$path' 'foobar'"
$CLICKHOUSE_KEEPER_CLIENT -q "rmr '$path'"
$CLICKHOUSE_KEEPER_CLIENT -q "get '$path'" 2>&1

View File

@ -7,4 +7,4 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CUR_DIR"/../shell_config.sh
yes /keeper/api_version | head -n1000 | xargs -P30 -i $CLICKHOUSE_KEEPER_CLIENT -q 'get {}' > /dev/null
yes /keeper/api_version | head -n1000 | xargs -P30 -i $CLICKHOUSE_KEEPER_CLIENT -q "get '{}'" > /dev/null

View File

@ -6,24 +6,24 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
path="/test-keeper-client-$CLICKHOUSE_DATABASE"
$CLICKHOUSE_KEEPER_CLIENT -q "rm $path" >& /dev/null
$CLICKHOUSE_KEEPER_CLIENT -q "rm '$path'" >& /dev/null
$CLICKHOUSE_KEEPER_CLIENT -q "create $path 'foobar'"
$CLICKHOUSE_KEEPER_CLIENT -q "create $path/1 'foobar'"
$CLICKHOUSE_KEEPER_CLIENT -q "create $path/1/a 'foobar'"
$CLICKHOUSE_KEEPER_CLIENT -q "create $path/1/a/a 'foobar'"
$CLICKHOUSE_KEEPER_CLIENT -q "create $path/1/b 'foobar'"
$CLICKHOUSE_KEEPER_CLIENT -q "create $path/1/c 'foobar'"
$CLICKHOUSE_KEEPER_CLIENT -q "create $path/1/d 'foobar'"
$CLICKHOUSE_KEEPER_CLIENT -q "create $path/1/d/a 'foobar'"
$CLICKHOUSE_KEEPER_CLIENT -q "create $path/1/d/b 'foobar'"
$CLICKHOUSE_KEEPER_CLIENT -q "create $path/1/d/c 'foobar'"
$CLICKHOUSE_KEEPER_CLIENT -q "create '$path' 'foobar'"
$CLICKHOUSE_KEEPER_CLIENT -q "create '$path/1' 'foobar'"
$CLICKHOUSE_KEEPER_CLIENT -q "create '$path/1/a' 'foobar'"
$CLICKHOUSE_KEEPER_CLIENT -q "create '$path/1/a/a' 'foobar'"
$CLICKHOUSE_KEEPER_CLIENT -q "create '$path/1/b' 'foobar'"
$CLICKHOUSE_KEEPER_CLIENT -q "create '$path/1/c' 'foobar'"
$CLICKHOUSE_KEEPER_CLIENT -q "create '$path/1/d' 'foobar'"
$CLICKHOUSE_KEEPER_CLIENT -q "create '$path/1/d/a' 'foobar'"
$CLICKHOUSE_KEEPER_CLIENT -q "create '$path/1/d/b' 'foobar'"
$CLICKHOUSE_KEEPER_CLIENT -q "create '$path/1/d/c' 'foobar'"
echo 'find_super_nodes'
$CLICKHOUSE_KEEPER_CLIENT -q "find_super_nodes 1000000000"
$CLICKHOUSE_KEEPER_CLIENT -q "find_super_nodes 3 $path" | sort
$CLICKHOUSE_KEEPER_CLIENT -q "find_super_nodes 3 '$path'" | sort
echo 'find_big_family'
$CLICKHOUSE_KEEPER_CLIENT -q "find_big_family $path 3"
$CLICKHOUSE_KEEPER_CLIENT -q "find_big_family '$path' 3"
$CLICKHOUSE_KEEPER_CLIENT -q "rmr $path"
$CLICKHOUSE_KEEPER_CLIENT -q "rmr '$path'"