Merge branch 'master' into format-settings-parsing

This commit is contained in:
Alexey Milovidov 2024-06-23 21:36:50 +02:00
commit 73286bad85
37 changed files with 203 additions and 287 deletions

2
contrib/openssl vendored

@ -1 +1 @@
Subproject commit 277de2ba202af4eb2291b363456d32ff0960e559
Subproject commit 5d81fa7068fc8c07f4d0997d5b703f3c541a637c

2
contrib/re2 vendored

@ -1 +1 @@
Subproject commit a807e8a3aac2cc33c77b7071efea54fcabe38e0c
Subproject commit 85dd7ad833a73095ecf3e3baea608ba051bbe2c7

View File

@ -28,16 +28,20 @@ set(RE2_SOURCES
add_library(_re2 ${RE2_SOURCES})
target_include_directories(_re2 PUBLIC "${SRC_DIR}")
target_link_libraries(_re2 PRIVATE
absl::absl_check
absl::absl_log
absl::base
absl::core_headers
absl::fixed_array
absl::flags
absl::flat_hash_map
absl::flat_hash_set
absl::hash
absl::inlined_vector
absl::strings
absl::str_format
absl::synchronization
absl::optional
absl::span)
absl::span
absl::str_format
absl::strings
absl::synchronization)
add_library(ch_contrib::re2 ALIAS _re2)

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

@ -261,7 +261,24 @@ AccessControl::AccessControl()
}
AccessControl::~AccessControl() = default;
AccessControl::~AccessControl()
{
try
{
AccessControl::shutdown();
}
catch (...)
{
tryLogCurrentException(__PRETTY_FUNCTION__);
}
}
void AccessControl::shutdown()
{
MultipleAccessStorage::shutdown();
removeAllStorages();
}
void AccessControl::setupFromMainConfig(const Poco::Util::AbstractConfiguration & config_, const String & config_path_,

View File

@ -53,6 +53,9 @@ public:
AccessControl();
~AccessControl() override;
/// Shutdown the access control and stops all background activity.
void shutdown() override;
/// Initializes access storage (user directories).
void setupFromMainConfig(const Poco::Util::AbstractConfiguration & config_, const String & config_path_,
const zkutil::GetZooKeeper & get_zookeeper_function_);

View File

@ -194,11 +194,9 @@ DiskAccessStorage::DiskAccessStorage(const String & storage_name_, const String
DiskAccessStorage::~DiskAccessStorage()
{
stopListsWritingThread();
try
{
writeLists();
DiskAccessStorage::shutdown();
}
catch (...)
{
@ -207,6 +205,17 @@ DiskAccessStorage::~DiskAccessStorage()
}
void DiskAccessStorage::shutdown()
{
stopListsWritingThread();
{
std::lock_guard lock{mutex};
writeLists();
}
}
String DiskAccessStorage::getStorageParamsJSON() const
{
std::lock_guard lock{mutex};

View File

@ -18,6 +18,8 @@ public:
DiskAccessStorage(const String & storage_name_, const String & directory_path_, AccessChangesNotifier & changes_notifier_, bool readonly_, bool allow_backup_);
~DiskAccessStorage() override;
void shutdown() override;
const char * getStorageType() const override { return STORAGE_TYPE; }
String getStorageParamsJSON() const override;

View File

@ -44,6 +44,11 @@ public:
explicit IAccessStorage(const String & storage_name_) : storage_name(storage_name_) {}
virtual ~IAccessStorage() = default;
/// If the AccessStorage has to do some complicated work when destroying - do it in advance.
/// For example, if the AccessStorage contains any threads for background work - ask them to complete and wait for completion.
/// By default, does nothing.
virtual void shutdown() {}
/// Returns the name of this storage.
const String & getStorageName() const { return storage_name; }
virtual const char * getStorageType() const = 0;

View File

@ -34,11 +34,23 @@ MultipleAccessStorage::MultipleAccessStorage(const String & storage_name_)
MultipleAccessStorage::~MultipleAccessStorage()
{
/// It's better to remove the storages in the reverse order because they could depend on each other somehow.
try
{
MultipleAccessStorage::shutdown();
}
catch (...)
{
tryLogCurrentException(__PRETTY_FUNCTION__);
}
}
void MultipleAccessStorage::shutdown()
{
/// It's better to shutdown the storages in the reverse order because they could depend on each other somehow.
const auto storages = getStoragesPtr();
for (const auto & storage : *storages | boost::adaptors::reversed)
{
removeStorage(storage);
storage->shutdown();
}
}
@ -72,6 +84,16 @@ void MultipleAccessStorage::removeStorage(const StoragePtr & storage_to_remove)
ids_cache.clear();
}
void MultipleAccessStorage::removeAllStorages()
{
/// It's better to remove the storages in the reverse order because they could depend on each other somehow.
const auto storages = getStoragesPtr();
for (const auto & storage : *storages | boost::adaptors::reversed)
{
removeStorage(storage);
}
}
std::vector<StoragePtr> MultipleAccessStorage::getStorages()
{
return *getStoragesPtr();

View File

@ -21,6 +21,8 @@ public:
explicit MultipleAccessStorage(const String & storage_name_ = STORAGE_TYPE);
~MultipleAccessStorage() override;
void shutdown() override;
const char * getStorageType() const override { return STORAGE_TYPE; }
bool isReadOnly() const override;
bool isReadOnly(const UUID & id) const override;
@ -32,6 +34,7 @@ public:
void setStorages(const std::vector<StoragePtr> & storages);
void addStorage(const StoragePtr & new_storage);
void removeStorage(const StoragePtr & storage_to_remove);
void removeAllStorages();
std::vector<StoragePtr> getStorages();
std::vector<ConstStoragePtr> getStorages() const;
std::shared_ptr<const std::vector<StoragePtr>> getStoragesPtr();

View File

@ -66,6 +66,18 @@ ReplicatedAccessStorage::ReplicatedAccessStorage(
}
ReplicatedAccessStorage::~ReplicatedAccessStorage()
{
try
{
ReplicatedAccessStorage::shutdown();
}
catch (...)
{
tryLogCurrentException(__PRETTY_FUNCTION__);
}
}
void ReplicatedAccessStorage::shutdown()
{
stopWatchingThread();
}

View File

@ -23,6 +23,8 @@ public:
ReplicatedAccessStorage(const String & storage_name, const String & zookeeper_path, zkutil::GetZooKeeper get_zookeeper, AccessChangesNotifier & changes_notifier_, bool allow_backup);
~ReplicatedAccessStorage() override;
void shutdown() override;
const char * getStorageType() const override { return STORAGE_TYPE; }
void startPeriodicReloading() override { startWatchingThread(); }

View File

@ -48,7 +48,7 @@ public:
/// HashFunction usually hashes the entire key and the found key will be equal the provided key. In such cases, use get(). It is also
/// possible to store other, non-hashed data in the key. In that case, the found key is potentially different from the provided key.
/// Then use getWithKey() to also return the found key including it's non-hashed data.
/// Then use getWithKey() to also return the found key including its non-hashed data.
virtual MappedPtr get(const Key & key) = 0;
virtual std::optional<KeyMapped> getWithKey(const Key &) = 0;

View File

@ -86,7 +86,7 @@ namespace DB
M(Double, index_mark_cache_size_ratio, DEFAULT_INDEX_MARK_CACHE_SIZE_RATIO, "The size of the protected queue in the secondary index mark cache relative to the cache's total size.", 0) \
M(UInt64, page_cache_chunk_size, 2 << 20, "Bytes per chunk in userspace page cache. Rounded up to a multiple of page size (typically 4 KiB) or huge page size (typically 2 MiB, only if page_cache_use_thp is enabled).", 0) \
M(UInt64, page_cache_mmap_size, 1 << 30, "Bytes per memory mapping in userspace page cache. Not important.", 0) \
M(UInt64, page_cache_size, 10ul << 30, "Amount of virtual memory to map for userspace page cache. If page_cache_use_madv_free is enabled, it's recommended to set this higher than the machine's RAM size. Use 0 to disable userspace page cache.", 0) \
M(UInt64, page_cache_size, 0, "Amount of virtual memory to map for userspace page cache. If page_cache_use_madv_free is enabled, it's recommended to set this higher than the machine's RAM size. Use 0 to disable userspace page cache.", 0) \
M(Bool, page_cache_use_madv_free, DBMS_DEFAULT_PAGE_CACHE_USE_MADV_FREE, "If true, the userspace page cache will allow the OS to automatically reclaim memory from the cache on memory pressure (using MADV_FREE).", 0) \
M(Bool, page_cache_use_transparent_huge_pages, true, "Userspace will attempt to use transparent huge pages on Linux. This is best-effort.", 0) \
M(UInt64, mmap_cache_size, DEFAULT_MMAP_CACHE_MAX_SIZE, "A cache for mmapped files.", 0) \

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

@ -679,6 +679,9 @@ struct ContextSharedPart : boost::noncopyable
}
}
LOG_TRACE(log, "Shutting down AccessControl");
access_control->shutdown();
{
std::lock_guard lock(mutex);

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

@ -30,5 +30,7 @@
</policies>
</storage_configuration>
<page_cache_size>10000000000</page_cache_size>
<query_log></query_log>
</clickhouse>

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,8 +10,8 @@ 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

View File

@ -1,21 +0,0 @@
cold read 54975576145920
PageCacheBytesUnpinnedRoundedToHugePages 1
PageCacheBytesUnpinnedRoundedToPages 1
PageCacheChunkMisses 1
ReadBufferFromS3Bytes 1
repeat read 1 54975576145920
PageCacheBytesUnpinnedRoundedToHugePages 1
PageCacheBytesUnpinnedRoundedToPages 1
PageCacheChunkDataHits 1
dropped and bypassed cache 54975576145920
PageCacheChunkMisses 1
ReadBufferFromS3Bytes 1
repeat read 2 54975576145920
PageCacheBytesUnpinnedRoundedToHugePages 1
PageCacheBytesUnpinnedRoundedToPages 1
PageCacheChunkMisses 1
ReadBufferFromS3Bytes 1
repeat read 3 54975576145920
PageCacheBytesUnpinnedRoundedToHugePages 1
PageCacheBytesUnpinnedRoundedToPages 1
PageCacheChunkDataHits 1

View File

@ -1,106 +0,0 @@
-- Tags: no-fasttest, no-parallel
-- no-fasttest because we need an S3 storage policy
-- no-parallel because we look at server-wide counters about page cache usage
set use_page_cache_for_disks_without_file_cache = 1;
set page_cache_inject_eviction = 0;
set enable_filesystem_cache = 0;
set use_uncompressed_cache = 0;
create table events_snapshot engine Memory as select * from system.events;
create view events_diff as
-- round all stats to 70 MiB to leave a lot of leeway for overhead
with if(event like '%Bytes%', 70*1024*1024, 35) as granularity,
-- cache hits counter can vary a lot depending on other settings:
-- e.g. if merge_tree_min_bytes_for_concurrent_read is small, multiple threads will read each chunk
-- so we just check that the value is not too low
if(event in (
'PageCacheBytesUnpinnedRoundedToPages', 'PageCacheBytesUnpinnedRoundedToHugePages',
'PageCacheChunkDataHits'), 1, 1000) as clamp
select event, min2(intDiv(new.value - old.value, granularity), clamp) as diff
from system.events new
left outer join events_snapshot old
on old.event = new.event
where diff != 0 and
event in (
'ReadBufferFromS3Bytes', 'PageCacheChunkMisses', 'PageCacheChunkDataMisses',
'PageCacheChunkDataHits', 'PageCacheChunkDataPartialHits',
'PageCacheBytesUnpinnedRoundedToPages', 'PageCacheBytesUnpinnedRoundedToHugePages')
order by event;
drop table if exists page_cache_03055;
create table page_cache_03055 (k Int64 CODEC(NONE)) engine MergeTree order by k settings storage_policy = 's3_cache';
-- Write an 80 MiB file (40 x 2 MiB chunks), and a few small files.
system stop merges page_cache_03055;
insert into page_cache_03055 select * from numbers(10485760) settings max_block_size=100000000, preferred_block_size_bytes=1000000000;
select * from events_diff;
truncate table events_snapshot;
insert into events_snapshot select * from system.events;
system start merges page_cache_03055;
optimize table page_cache_03055 final;
truncate table events_snapshot;
insert into events_snapshot select * from system.events;
-- Cold read, should miss cache. (Populating cache on write is not implemented yet.)
select 'cold read', sum(k) from page_cache_03055;
select * from events_diff where event not in ('PageCacheChunkDataHits');
truncate table events_snapshot;
insert into events_snapshot select * from system.events;
-- Repeat read, should hit cache.
select 'repeat read 1', sum(k) from page_cache_03055;
select * from events_diff;
truncate table events_snapshot;
insert into events_snapshot select * from system.events;
-- Drop cache and read again, should miss. Also don't write to cache.
system drop page cache;
select 'dropped and bypassed cache', sum(k) from page_cache_03055 settings read_from_page_cache_if_exists_otherwise_bypass_cache = 1;
-- Data could be read multiple times because we're not writing to cache.
-- (Not checking PageCacheBytesUnpinned* because it's unreliable in this case because of an intentional race condition, see PageCache::evictChunk.)
select event, if(event in ('PageCacheChunkMisses', 'ReadBufferFromS3Bytes'), diff >= 1, diff) from events_diff where event not in ('PageCacheChunkDataHits', 'PageCacheBytesUnpinnedRoundedToPages', 'PageCacheBytesUnpinnedRoundedToHugePages');
truncate table events_snapshot;
insert into events_snapshot select * from system.events;
-- Repeat read, should still miss, but populate cache.
select 'repeat read 2', sum(k) from page_cache_03055;
select * from events_diff where event not in ('PageCacheChunkDataHits');
truncate table events_snapshot;
insert into events_snapshot select * from system.events;
-- Read again, hit the cache.
select 'repeat read 3', sum(k) from page_cache_03055 settings read_from_page_cache_if_exists_otherwise_bypass_cache = 1;
select * from events_diff;
truncate table events_snapshot;
insert into events_snapshot select * from system.events;
-- Known limitation: cache is not invalidated if a table is dropped and created again at the same path.
-- set allow_deprecated_database_ordinary=1;
-- create database test_03055 engine = Ordinary;
-- create table test_03055.t (k Int64) engine MergeTree order by k settings storage_policy = 's3_cache';
-- insert into test_03055.t values (1);
-- select * from test_03055.t;
-- drop table test_03055.t;
-- create table test_03055.t (k Int64) engine MergeTree order by k settings storage_policy = 's3_cache';
-- insert into test_03055.t values (2);
-- select * from test_03055.t;
drop table events_snapshot;
drop table page_cache_03055;
drop view events_diff;

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'"