Corrections after review.

This commit is contained in:
Vitaly Baranov 2024-08-02 16:58:04 +02:00
parent 4ef382360d
commit 9f1753bb4f
4 changed files with 20 additions and 32 deletions

View File

@ -213,8 +213,8 @@ CREATE TABLE my_table
## The `tags` and `all_tags` columns {#tags-and-all-tags}
There are two columns containing maps of tags - `tags` and `all_tags`. In this example they mean the same, however they can be different
if setting `tags_to_columns` is used. This setting allows to specify that a specific tag should be stored in a separate column instead of storing
in a map inside the `tags` column:
if setting `tags_to_columns` is used. This setting allows you to specify that a specific tag should be stored in a separate column instead of storing
it in a map inside the `tags` column:
``` sql
CREATE TABLE my_table ENGINE=TimeSeries SETTINGS = {'instance': 'instance', 'job': 'job'}

View File

@ -903,7 +903,7 @@ class IColumn;
M(Bool, allow_experimental_nlp_functions, false, "Enable experimental functions for natural language processing.", 0) \
M(Bool, allow_experimental_hash_functions, false, "Enable experimental hash functions", 0) \
M(Bool, allow_experimental_object_type, false, "Allow Object and JSON data types", 0) \
M(Bool, allow_experimental_time_series_table, false, "Allows to use the TimeSeries table engine. Disabled by default, because this feature is experimental", 0) \
M(Bool, allow_experimental_time_series_table, false, "Allows experimental TimeSeries table engine", 0) \
M(Bool, allow_experimental_variant_type, false, "Allow Variant data type", 0) \
M(Bool, allow_experimental_dynamic_type, false, "Allow Dynamic data type", 0) \
M(Bool, allow_experimental_annoy_index, false, "Allows to use Annoy index. Disabled by default because this feature is experimental", 0) \

View File

@ -151,25 +151,25 @@ protected:
if (!roles.empty())
context->setCurrentRoles(roles);
auto param_could_be_skipped = [&] (const String & name)
/// Settings can be overridden in the URL query.
auto is_setting_like_parameter = [&] (const String & name)
{
/// Empty parameter appears when URL like ?&a=b or a=b&&c=d. Just skip them for user's convenience.
if (name.empty())
return true;
return false;
/// Some parameters (database, default_format, everything used in the code above) do not
/// belong to the Settings class.
static const NameSet reserved_param_names{"user", "password", "quota_key", "stacktrace", "role", "query_id"};
return reserved_param_names.contains(name);
return !reserved_param_names.contains(name);
};
/// Settings can be overridden in the query.
SettingsChanges settings_changes;
for (const auto & [key, value] : *params)
{
if (!param_could_be_skipped(key))
if (is_setting_like_parameter(key))
{
/// Other than query parameters are treated as settings.
/// This query parameter should be considered as a ClickHouse setting.
settings_changes.push_back({key, value});
}
}

View File

@ -371,9 +371,8 @@ void StorageTimeSeries::alter(const AlterCommands & params, ContextPtr local_con
}
void StorageTimeSeries::renameInMemory(const StorageID & new_table_id)
void StorageTimeSeries::renameInMemory(const StorageID & /* new_table_id */)
{
UNUSED(new_table_id);
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Renaming is not supported by storage {} yet", getName());
}
@ -406,33 +405,22 @@ void StorageTimeSeries::restoreDataFromBackup(RestorerFromBackup & restorer, con
void StorageTimeSeries::read(
QueryPlan & query_plan,
const Names & column_names,
const StorageSnapshotPtr & storage_snapshot,
SelectQueryInfo & query_info,
ContextPtr local_context,
QueryProcessingStage::Enum processed_stage,
size_t max_block_size,
size_t num_streams)
QueryPlan & /* query_plan */,
const Names & /* column_names */,
const StorageSnapshotPtr & /* storage_snapshot */,
SelectQueryInfo & /* query_info */,
ContextPtr /* local_context */,
QueryProcessingStage::Enum /* processed_stage */,
size_t /* max_block_size */,
size_t /* num_streams */)
{
UNUSED(query_plan);
UNUSED(column_names);
UNUSED(storage_snapshot);
UNUSED(query_info);
UNUSED(local_context);
UNUSED(processed_stage);
UNUSED(max_block_size);
UNUSED(num_streams);
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "SELECT is not supported by storage {} yet", getName());
}
SinkToStoragePtr StorageTimeSeries::write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context, bool async_insert)
SinkToStoragePtr StorageTimeSeries::write(
const ASTPtr & /* query */, const StorageMetadataPtr & /* metadata_snapshot */, ContextPtr /* local_context */, bool /* async_insert */)
{
UNUSED(query);
UNUSED(metadata_snapshot);
UNUSED(local_context);
UNUSED(async_insert);
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "INSERT is not supported by storage {} yet", getName());
}