Revert unrelated changes

This commit is contained in:
avogar 2024-06-21 18:40:49 +00:00
parent 7e7dd78844
commit 785f3ac5c9
8 changed files with 87 additions and 15 deletions

View File

@ -31,6 +31,7 @@ class IColumn;
* for tracking settings changes in different versions and for special `compatibility` setting to work correctly.
*/
// clang-format off
#define COMMON_SETTINGS(M, ALIAS) \
M(Dialect, dialect, Dialect::clickhouse, "Which dialect will be used to parse query", 0)\
M(UInt64, min_compress_block_size, 65536, "The actual size of the block to compress, if the uncompressed data less than max_compress_block_size is no less than this value and no less than the volume of data for one mark.", 0) \
@ -933,6 +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) \
// 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.
@ -1149,6 +1151,8 @@ class IColumn;
M(UInt64, output_format_pretty_max_value_width_apply_for_single_value, false, "Only cut values (see the `output_format_pretty_max_value_width` setting) when it is not a single value in a block. Otherwise output it entirely, which is useful for the `SHOW CREATE TABLE` query.", 0) \
M(UInt64Auto, output_format_pretty_color, "auto", "Use ANSI escape sequences in Pretty formats. 0 - disabled, 1 - enabled, 'auto' - enabled if a terminal.", 0) \
M(String, output_format_pretty_grid_charset, "UTF-8", "Charset for printing grid borders. Available charsets: ASCII, UTF-8 (default one).", 0) \
M(UInt64, output_format_pretty_display_footer_column_names, true, "Display column names in the footer if there are 999 or more rows.", 0) \
M(UInt64, output_format_pretty_display_footer_column_names_min_rows, 50, "Sets the minimum threshold value of rows for which to enable displaying column names in the footer. 50 (default)", 0) \
M(UInt64, output_format_parquet_row_group_size, 1000000, "Target row group size in rows.", 0) \
M(UInt64, output_format_parquet_row_group_size_bytes, 512 * 1024 * 1024, "Target row group size in bytes, before compression.", 0) \
M(Bool, output_format_parquet_string_as_string, true, "Use Parquet String type instead of Binary for String columns.", 0) \

View File

@ -75,6 +75,7 @@ namespace SettingsChangesHistory
using SettingsChanges = std::vector<SettingChange>;
}
// clang-format off
/// History of settings changes that controls some backward incompatible changes
/// across all ClickHouse versions. It maps ClickHouse version to settings changes that were done
/// in this version. This history contains both changes to existing settings and newly added settings.
@ -106,6 +107,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_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`."},
@ -113,6 +115,8 @@ static const std::map<ClickHouseVersion, SettingsChangesHistory::SettingsChanges
{"enable_vertical_final", false, true, "Enable vertical final by default again after fixing bug"},
{"parallel_replicas_custom_key_range_lower", 0, 0, "Add settings to control the range filter when using parallel replicas with dynamic shards"},
{"parallel_replicas_custom_key_range_upper", 0, 0, "Add settings to control the range filter when using parallel replicas with dynamic shards. A value of 0 disables the upper limit"},
{"output_format_pretty_display_footer_column_names", 0, 1, "Add a setting to display column names in the footer if there are many rows. Threshold value is controlled by output_format_pretty_display_footer_column_names_min_rows."},
{"output_format_pretty_display_footer_column_names_min_rows", 0, 50, "Add a setting to control the threshold value for setting output_format_pretty_display_footer_column_names_min_rows. Default 50."},
{"output_format_csv_serialize_tuple_into_separate_columns", true, true, "A new way of how interpret tuples in CSV format was added."},
{"input_format_csv_deserialize_separate_columns_into_tuple", true, true, "A new way of how interpret tuples in CSV format was added."},
{"input_format_csv_try_infer_strings_from_quoted_tuples", true, true, "A new way of how interpret tuples in CSV format was added."},

View File

@ -181,6 +181,8 @@ FormatSettings getFormatSettings(const ContextPtr & context, const Settings & se
format_settings.pretty.highlight_digit_groups = settings.output_format_pretty_highlight_digit_groups;
format_settings.pretty.output_format_pretty_row_numbers = settings.output_format_pretty_row_numbers;
format_settings.pretty.output_format_pretty_single_large_number_tip_threshold = settings.output_format_pretty_single_large_number_tip_threshold;
format_settings.pretty.output_format_pretty_display_footer_column_names = settings.output_format_pretty_display_footer_column_names;
format_settings.pretty.output_format_pretty_display_footer_column_names_min_rows = settings.output_format_pretty_display_footer_column_names_min_rows;
format_settings.protobuf.input_flatten_google_wrappers = settings.input_format_protobuf_flatten_google_wrappers;
format_settings.protobuf.output_nullables_with_google_wrappers = settings.output_format_protobuf_nullables_with_google_wrappers;
format_settings.protobuf.skip_fields_with_unsupported_types_in_schema_inference = settings.input_format_protobuf_skip_fields_with_unsupported_types_in_schema_inference;

View File

@ -295,6 +295,8 @@ struct FormatSettings
bool output_format_pretty_row_numbers = false;
UInt64 output_format_pretty_single_large_number_tip_threshold = 1'000'000;
UInt64 output_format_pretty_display_footer_column_names = 1;
UInt64 output_format_pretty_display_footer_column_names_min_rows = 50;
enum class Charset : uint8_t
{

View File

@ -41,7 +41,7 @@ NativeReader::NativeReader(
ReadBuffer & istr_,
const Block & header_,
UInt64 server_revision_,
std::optional<FormatSettings>format_settings_,
std::optional<FormatSettings> format_settings_,
BlockMissingValues * block_missing_values_)
: istr(istr_)
, header(header_)

View File

@ -18,6 +18,7 @@
#include <Common/callOnce.h>
#include <Common/SharedLockGuard.h>
#include <Common/PageCache.h>
#include <Common/NamedCollections/NamedCollectionsFactory.h>
#include <Coordination/KeeperDispatcher.h>
#include <Core/BackgroundSchedulePool.h>
#include <Formats/FormatFactory.h>
@ -280,6 +281,8 @@ struct ContextSharedPart : boost::noncopyable
String default_profile_name; /// Default profile name used for default values.
String system_profile_name; /// Profile used by system processes
String buffer_profile_name; /// Profile used by Buffer engine for flushing to the underlying
String merge_workload TSA_GUARDED_BY(mutex); /// Workload setting value that is used by all merges
String mutation_workload TSA_GUARDED_BY(mutex); /// Workload setting value that is used by all mutations
std::unique_ptr<AccessControl> access_control TSA_GUARDED_BY(mutex);
mutable OnceFlag resource_manager_initialized;
mutable ResourceManagerPtr resource_manager;
@ -610,6 +613,8 @@ struct ContextSharedPart : boost::noncopyable
LOG_TRACE(log, "Shutting down database catalog");
DatabaseCatalog::shutdown();
NamedCollectionFactory::instance().shutdown();
delete_async_insert_queue.reset();
SHUTDOWN(log, "merges executor", merge_mutate_executor, wait());
@ -830,6 +835,7 @@ ContextMutablePtr Context::createGlobal(ContextSharedPart * shared_part)
auto res = std::shared_ptr<Context>(new Context);
res->shared = shared_part;
res->query_access_info = std::make_shared<QueryAccessInfo>();
res->query_privileges_info = std::make_shared<QueryPrivilegesInfo>();
return res;
}
@ -1422,7 +1428,7 @@ void Context::checkAccess(const AccessFlags & flags, const StorageID & table_id,
void Context::checkAccess(const AccessRightsElement & element) const { checkAccessImpl(element); }
void Context::checkAccess(const AccessRightsElements & elements) const { checkAccessImpl(elements); }
std::shared_ptr<const ContextAccess> Context::getAccess() const
std::shared_ptr<const ContextAccessWrapper> Context::getAccess() const
{
/// A helper function to collect parameters for calculating access rights, called with Context::getLocalSharedLock() acquired.
auto get_params = [this]()
@ -1439,14 +1445,14 @@ std::shared_ptr<const ContextAccess> Context::getAccess() const
{
SharedLockGuard lock(mutex);
if (access && !need_recalculate_access)
return access; /// No need to recalculate access rights.
return std::make_shared<const ContextAccessWrapper>(access, shared_from_this()); /// No need to recalculate access rights.
params.emplace(get_params());
if (access && (access->getParams() == *params))
{
need_recalculate_access = false;
return access; /// No need to recalculate access rights.
return std::make_shared<const ContextAccessWrapper>(access, shared_from_this()); /// No need to recalculate access rights.
}
}
@ -1466,7 +1472,7 @@ std::shared_ptr<const ContextAccess> Context::getAccess() const
}
}
return res;
return std::make_shared<const ContextAccessWrapper>(res, shared_from_this());
}
RowPolicyFilterPtr Context::getRowPolicyFilter(const String & database, const String & table_name, RowPolicyFilterType filter_type) const
@ -1558,11 +1564,36 @@ ResourceManagerPtr Context::getResourceManager() const
ClassifierPtr Context::getWorkloadClassifier() const
{
std::lock_guard lock(mutex);
// NOTE: Workload cannot be changed after query start, and getWorkloadClassifier() should not be called before proper `workload` is set
if (!classifier)
classifier = getResourceManager()->acquire(getSettingsRef().workload);
return classifier;
}
String Context::getMergeWorkload() const
{
SharedLockGuard lock(shared->mutex);
return shared->merge_workload;
}
void Context::setMergeWorkload(const String & value)
{
std::lock_guard lock(shared->mutex);
shared->merge_workload = value;
}
String Context::getMutationWorkload() const
{
SharedLockGuard lock(shared->mutex);
return shared->mutation_workload;
}
void Context::setMutationWorkload(const String & value)
{
std::lock_guard lock(shared->mutex);
shared->mutation_workload = value;
}
Scalars Context::getScalars() const
{
@ -1827,6 +1858,15 @@ void Context::addQueryFactoriesInfo(QueryLogFactories factory_type, const String
}
}
void Context::addQueryPrivilegesInfo(const String & privilege, bool granted) const
{
std::lock_guard lock(query_privileges_info->mutex);
if (granted)
query_privileges_info->used_privileges.emplace(privilege);
else
query_privileges_info->missing_privileges.emplace(privilege);
}
static bool findIdentifier(const ASTFunction * function)
{
if (!function || !function->arguments)
@ -2508,6 +2548,21 @@ void Context::makeQueryContext()
local_read_query_throttler.reset();
local_write_query_throttler.reset();
backups_query_throttler.reset();
query_privileges_info = std::make_shared<QueryPrivilegesInfo>(*query_privileges_info);
}
void Context::makeQueryContextForMerge(const MergeTreeSettings & merge_tree_settings)
{
makeQueryContext();
classifier.reset(); // It is assumed that there are no active queries running using this classifier, otherwise this will lead to crashes
settings.workload = merge_tree_settings.merge_workload.value.empty() ? getMergeWorkload() : merge_tree_settings.merge_workload;
}
void Context::makeQueryContextForMutate(const MergeTreeSettings & merge_tree_settings)
{
makeQueryContext();
classifier.reset(); // It is assumed that there are no active queries running using this classifier, otherwise this will lead to crashes
settings.workload = merge_tree_settings.mutation_workload.value.empty() ? getMutationWorkload() : merge_tree_settings.mutation_workload;
}
void Context::makeSessionContext()
@ -3943,7 +3998,6 @@ std::shared_ptr<QueryThreadLog> Context::getQueryThreadLog() const
std::shared_ptr<QueryViewsLog> Context::getQueryViewsLog() const
{
SharedLockGuard lock(shared->mutex);
if (!shared->system_logs)
return {};

View File

@ -1,9 +1,8 @@
#include "Interpreters/AsynchronousInsertQueue.h"
#include "Interpreters/SquashingTransform.h"
#include "Parsers/ASTInsertQuery.h"
#include <Interpreters/AsynchronousInsertQueue.h>
#include <Interpreters/Squashing.h>
#include <Parsers/ASTInsertQuery.h>
#include <algorithm>
#include <exception>
#include <iterator>
#include <memory>
#include <mutex>
#include <vector>
@ -246,7 +245,6 @@ TCPHandler::~TCPHandler()
void TCPHandler::runImpl()
{
setThreadName("TCPHandler");
ThreadStatus thread_status;
extractConnectionSettingsFromContext(server.context());
@ -886,13 +884,16 @@ AsynchronousInsertQueue::PushResult TCPHandler::processAsyncInsertQuery(Asynchro
using PushResult = AsynchronousInsertQueue::PushResult;
startInsertQuery();
SquashingTransform squashing(0, query_context->getSettingsRef().async_insert_max_data_size);
Squashing squashing(state.input_header, 0, query_context->getSettingsRef().async_insert_max_data_size);
while (readDataNext())
{
auto result = squashing.add(std::move(state.block_for_insert));
if (result)
squashing.header = state.block_for_insert;
auto planned_chunk = squashing.add({state.block_for_insert.getColumns(), state.block_for_insert.rows()});
if (planned_chunk.hasChunkInfo())
{
Chunk result_chunk = DB::Squashing::squash(std::move(planned_chunk));
auto result = state.block_for_insert.cloneWithColumns(result_chunk.getColumns());
return PushResult
{
.status = PushResult::TOO_MUCH_DATA,
@ -901,7 +902,12 @@ AsynchronousInsertQueue::PushResult TCPHandler::processAsyncInsertQuery(Asynchro
}
}
auto result = squashing.add({});
auto planned_chunk = squashing.flush();
Chunk result_chunk;
if (planned_chunk.hasChunkInfo())
result_chunk = DB::Squashing::squash(std::move(planned_chunk));
auto result = squashing.header.cloneWithColumns(result_chunk.getColumns());
return insert_queue.pushQueryWithBlock(state.parsed_query, std::move(result), query_context);
}