mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Make MemorySettings a pImpl and simplify style checker even more
This commit is contained in:
parent
984d3e888e
commit
6727af1236
@ -55,23 +55,8 @@ SETTINGS_FILE=$(mktemp)
|
||||
cat $ROOT_PATH/src/Core/Settings.cpp $ROOT_PATH/src/Core/FormatFactorySettingsDeclaration.h | grep "M(" | awk '{print substr($2, 0, length($2) - 1) " " substr($1, 3, length($1) - 3) " SettingsDeclaration" }' > ${SETTINGS_FILE}
|
||||
find $ROOT_PATH/{src,base,programs,utils} -name '*.h' -or -name '*.cpp' | xargs grep "extern const Settings" -T | awk '{print substr($5, 0, length($5) -1) " " substr($4, 9) " " substr($1, 0, length($1) - 1)}' >> ${SETTINGS_FILE}
|
||||
|
||||
# Duplicate extern declarations for settings
|
||||
awk '{if (seen[$0]++) print $3 " -> " $1 ;}' ${SETTINGS_FILE} | while read line;
|
||||
do
|
||||
echo "Found duplicated setting declaration in: $line"
|
||||
done
|
||||
|
||||
# Incorrect declarations for settings
|
||||
for setting in $(awk '{print $1 " " $2}' ${SETTINGS_FILE} | sort | uniq | awk '{ print $1 }' | sort | uniq -d);
|
||||
do
|
||||
expected=$(grep "^$setting " ${SETTINGS_FILE} | grep SettingsDeclaration | awk '{ print $2 }')
|
||||
grep "^$setting " ${SETTINGS_FILE} | grep -v " $expected" | awk '{ print $3 " found setting " $1 " with type " $2 }' | while read line;
|
||||
do
|
||||
echo "In $line but it should be $expected"
|
||||
done
|
||||
done
|
||||
|
||||
rm ${SETTINGS_FILE}
|
||||
# Duplicated or incorrect setting declarations
|
||||
bash $ROOT_PATH/utils/check-style/check-settings-style
|
||||
|
||||
# Unused/Undefined/Duplicates ErrorCodes/ProfileEvents/CurrentMetrics
|
||||
declare -A EXTERN_TYPES
|
||||
|
@ -1,5 +1,5 @@
|
||||
#include <Core/BaseSettings.h>
|
||||
#include <Core/FormatFactorySettingsDeclaration.h>
|
||||
#include <Core/FormatFactorySettings.h>
|
||||
#include <Core/SettingsEnums.h>
|
||||
|
||||
namespace DB
|
||||
|
File diff suppressed because it is too large
Load Diff
File diff suppressed because it is too large
Load Diff
@ -4,7 +4,7 @@
|
||||
#include <Core/BaseSettingsFwdMacros.h>
|
||||
#include <Core/BaseSettingsFwdMacrosImpl.h>
|
||||
#include <Core/BaseSettingsProgramOptions.h>
|
||||
#include <Core/FormatFactorySettingsDeclaration.h>
|
||||
#include <Core/FormatFactorySettings.h>
|
||||
#include <Core/Settings.h>
|
||||
#include <Core/SettingsChangesHistory.h>
|
||||
#include <Core/SettingsEnums.h>
|
||||
@ -5817,7 +5817,7 @@ Allow writing simple SELECT queries without the leading SELECT keyword, which ma
|
||||
|
||||
|
||||
// End of COMMON_SETTINGS
|
||||
// Please add settings related to formats in FormatFactorySettingsDeclaration.h, move obsolete settings to OBSOLETE_SETTINGS and obsolete format settings to OBSOLETE_FORMAT_SETTINGS.
|
||||
// Please add settings related to formats in Core/FormatFactorySettings.h, move obsolete settings to OBSOLETE_SETTINGS and obsolete format settings to OBSOLETE_FORMAT_SETTINGS.
|
||||
|
||||
#define OBSOLETE_SETTINGS(M, ALIAS) \
|
||||
/** Obsolete settings that do nothing but left for compatibility reasons. Remove each one after half a year of obsolescence. */ \
|
||||
|
@ -18,7 +18,6 @@
|
||||
#include <Common/KnownObjectNames.h>
|
||||
#include <Common/RemoteHostFilter.h>
|
||||
#include <Common/tryGetFileNameByFileDescriptor.h>
|
||||
#include <Core/FormatFactorySettingsDeclaration.h>
|
||||
#include <Core/FormatFactorySettings.h>
|
||||
#include <Core/Settings.h>
|
||||
|
||||
|
@ -13,6 +13,7 @@
|
||||
#include <Databases/DatabaseMemory.h>
|
||||
#include <Databases/DatabaseOnDisk.h>
|
||||
#include <Disks/IDisk.h>
|
||||
#include <Storages/MemorySettings.h>
|
||||
#include <Storages/StorageMemory.h>
|
||||
#include <Core/BackgroundSchedulePool.h>
|
||||
#include <Parsers/formatAST.h>
|
||||
@ -141,7 +142,7 @@ TemporaryTableHolder::TemporaryTableHolder(
|
||||
context_,
|
||||
[&](const StorageID & table_id)
|
||||
{
|
||||
auto storage = std::make_shared<StorageMemory>(table_id, ColumnsDescription{columns}, ConstraintsDescription{constraints}, String{});
|
||||
auto storage = std::make_shared<StorageMemory>(table_id, ColumnsDescription{columns}, ConstraintsDescription{constraints}, String{}, MemorySettings{});
|
||||
|
||||
if (create_for_global_subquery)
|
||||
storage->delayReadForGlobalSubqueries();
|
||||
|
@ -1,6 +1,6 @@
|
||||
#include <Core/BaseSettings.h>
|
||||
#include <Core/BaseSettingsFwdMacrosImpl.h>
|
||||
#include <Core/FormatFactorySettingsDeclaration.h>
|
||||
#include <Core/FormatFactorySettings.h>
|
||||
#include <Parsers/ASTCreateQuery.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/ASTSetQuery.h>
|
||||
|
@ -4,7 +4,7 @@
|
||||
|
||||
#include <Core/BaseSettings.h>
|
||||
#include <Core/BaseSettingsFwdMacrosImpl.h>
|
||||
#include <Core/FormatFactorySettingsDeclaration.h>
|
||||
#include <Core/FormatFactorySettings.h>
|
||||
#include <Parsers/ASTCreateQuery.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/ASTSetQuery.h>
|
||||
|
@ -1,6 +1,6 @@
|
||||
#include <Core/BaseSettings.h>
|
||||
#include <Core/BaseSettingsFwdMacrosImpl.h>
|
||||
#include <Core/FormatFactorySettingsDeclaration.h>
|
||||
#include <Core/FormatFactorySettings.h>
|
||||
#include <Parsers/ASTCreateQuery.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/ASTSetQuery.h>
|
||||
|
@ -1,6 +1,8 @@
|
||||
#include <Storages/MemorySettings.h>
|
||||
#include <Core/BaseSettings.h>
|
||||
#include <Core/BaseSettingsFwdMacrosImpl.h>
|
||||
#include <Parsers/ASTCreateQuery.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Storages/MemorySettings.h>
|
||||
#include <Common/Exception.h>
|
||||
|
||||
|
||||
@ -13,7 +15,51 @@ namespace ErrorCodes
|
||||
extern const int SETTING_CONSTRAINT_VIOLATION;
|
||||
}
|
||||
|
||||
IMPLEMENT_SETTINGS_TRAITS(memorySettingsTraits, MEMORY_SETTINGS)
|
||||
#define MEMORY_SETTINGS(M, ALIAS) \
|
||||
M(Bool, compress, false, "Compress data in memory", 0) \
|
||||
M(UInt64, min_rows_to_keep, 0, "Minimum block size (in rows) to retain in Memory table buffer.", 0) \
|
||||
M(UInt64, max_rows_to_keep, 0, "Maximum block size (in rows) to retain in Memory table buffer.", 0) \
|
||||
M(UInt64, min_bytes_to_keep, 0, "Minimum block size (in bytes) to retain in Memory table buffer.", 0) \
|
||||
M(UInt64, max_bytes_to_keep, 0, "Maximum block size (in bytes) to retain in Memory table buffer.", 0) \
|
||||
|
||||
DECLARE_SETTINGS_TRAITS(MemorySettingsTraits, MEMORY_SETTINGS)
|
||||
IMPLEMENT_SETTINGS_TRAITS(MemorySettingsTraits, MEMORY_SETTINGS)
|
||||
|
||||
|
||||
struct MemorySettingsImpl : public BaseSettings<MemorySettingsTraits>
|
||||
{
|
||||
};
|
||||
|
||||
#define INITIALIZE_SETTING_EXTERN(TYPE, NAME, DEFAULT, DESCRIPTION, FLAGS) MemorySettings##TYPE NAME = &MemorySettingsImpl ::NAME;
|
||||
|
||||
namespace MemorySetting
|
||||
{
|
||||
MEMORY_SETTINGS(INITIALIZE_SETTING_EXTERN, SKIP_ALIAS)
|
||||
}
|
||||
|
||||
#undef INITIALIZE_SETTING_EXTERN
|
||||
|
||||
MemorySettings::MemorySettings() : impl(std::make_unique<MemorySettingsImpl>())
|
||||
{
|
||||
}
|
||||
|
||||
MemorySettings::MemorySettings(const MemorySettings & settings) : impl(std::make_unique<MemorySettingsImpl>(*settings.impl))
|
||||
{
|
||||
}
|
||||
|
||||
MemorySettings::MemorySettings(MemorySettings && settings) noexcept : impl(std::make_unique<MemorySettingsImpl>(std::move(*settings.impl)))
|
||||
{
|
||||
}
|
||||
|
||||
MemorySettings::~MemorySettings() = default;
|
||||
|
||||
MemorySettings & MemorySettings::operator=(DB::MemorySettings && settings)
|
||||
{
|
||||
*impl = std::move(*settings.impl);
|
||||
return *this;
|
||||
}
|
||||
|
||||
MEMORY_SETTINGS_SUPPORTED_TYPES(MemorySettings, IMPLEMENT_SETTING_SUBSCRIPT_OPERATOR)
|
||||
|
||||
void MemorySettings::loadFromQuery(ASTStorage & storage_def)
|
||||
{
|
||||
@ -21,7 +67,7 @@ void MemorySettings::loadFromQuery(ASTStorage & storage_def)
|
||||
{
|
||||
try
|
||||
{
|
||||
applyChanges(storage_def.settings->changes);
|
||||
impl->applyChanges(storage_def.settings->changes);
|
||||
}
|
||||
catch (Exception & e)
|
||||
{
|
||||
@ -36,7 +82,7 @@ ASTPtr MemorySettings::getSettingsChangesQuery()
|
||||
{
|
||||
auto settings_ast = std::make_shared<ASTSetQuery>();
|
||||
settings_ast->is_standalone = false;
|
||||
for (const auto & change : changes())
|
||||
for (const auto & change : impl->changes())
|
||||
settings_ast->changes.push_back(change);
|
||||
|
||||
return settings_ast;
|
||||
@ -44,19 +90,25 @@ ASTPtr MemorySettings::getSettingsChangesQuery()
|
||||
|
||||
void MemorySettings::sanityCheck() const
|
||||
{
|
||||
if (min_bytes_to_keep > max_bytes_to_keep)
|
||||
throw Exception(ErrorCodes::SETTING_CONSTRAINT_VIOLATION,
|
||||
"Setting `min_bytes_to_keep` cannot be higher than the `max_bytes_to_keep`. `min_bytes_to_keep`: {}, `max_bytes_to_keep`: {}",
|
||||
min_bytes_to_keep,
|
||||
max_bytes_to_keep);
|
||||
if (impl->min_bytes_to_keep > impl->max_bytes_to_keep)
|
||||
throw Exception(
|
||||
ErrorCodes::SETTING_CONSTRAINT_VIOLATION,
|
||||
"Setting `min_bytes_to_keep` cannot be higher than the `max_bytes_to_keep`. `min_bytes_to_keep`: {}, `max_bytes_to_keep`: {}",
|
||||
impl->min_bytes_to_keep,
|
||||
impl->max_bytes_to_keep);
|
||||
|
||||
|
||||
if (min_rows_to_keep > max_rows_to_keep)
|
||||
throw Exception(ErrorCodes::SETTING_CONSTRAINT_VIOLATION,
|
||||
"Setting `min_rows_to_keep` cannot be higher than the `max_rows_to_keep`. `min_rows_to_keep`: {}, `max_rows_to_keep`: {}",
|
||||
min_rows_to_keep,
|
||||
max_rows_to_keep);
|
||||
if (impl->min_rows_to_keep > impl->max_rows_to_keep)
|
||||
throw Exception(
|
||||
ErrorCodes::SETTING_CONSTRAINT_VIOLATION,
|
||||
"Setting `min_rows_to_keep` cannot be higher than the `max_rows_to_keep`. `min_rows_to_keep`: {}, `max_rows_to_keep`: {}",
|
||||
impl->min_rows_to_keep,
|
||||
impl->max_rows_to_keep);
|
||||
}
|
||||
|
||||
void MemorySettings::applyChanges(const DB::SettingsChanges & changes)
|
||||
{
|
||||
return impl->applyChanges(changes);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -1,32 +1,46 @@
|
||||
#pragma once
|
||||
|
||||
#include <Core/BaseSettings.h>
|
||||
#include <Parsers/ASTSetQuery.h>
|
||||
|
||||
#include <Core/BaseSettingsFwdMacros.h>
|
||||
#include <Core/SettingsFields.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
class ASTStorage;
|
||||
struct MemorySettingsImpl;
|
||||
|
||||
class IAST;
|
||||
using ASTPtr = std::shared_ptr<IAST>;
|
||||
|
||||
#define MEMORY_SETTINGS(M, ALIAS) \
|
||||
M(Bool, compress, false, "Compress data in memory", 0) \
|
||||
M(UInt64, min_rows_to_keep, 0, "Minimum block size (in rows) to retain in Memory table buffer.", 0) \
|
||||
M(UInt64, max_rows_to_keep, 0, "Maximum block size (in rows) to retain in Memory table buffer.", 0) \
|
||||
M(UInt64, min_bytes_to_keep, 0, "Minimum block size (in bytes) to retain in Memory table buffer.", 0) \
|
||||
M(UInt64, max_bytes_to_keep, 0, "Maximum block size (in bytes) to retain in Memory table buffer.", 0) \
|
||||
class SettingsChanges;
|
||||
|
||||
DECLARE_SETTINGS_TRAITS(memorySettingsTraits, MEMORY_SETTINGS)
|
||||
/// List of available types supported in MemorySettings object
|
||||
#define MEMORY_SETTINGS_SUPPORTED_TYPES(CLASS_NAME, M) \
|
||||
M(CLASS_NAME, Bool) \
|
||||
M(CLASS_NAME, UInt64)
|
||||
|
||||
MEMORY_SETTINGS_SUPPORTED_TYPES(MemorySettings, DECLARE_SETTING_TRAIT)
|
||||
|
||||
/** Settings for the Memory engine.
|
||||
* Could be loaded from a CREATE TABLE query (SETTINGS clause).
|
||||
*/
|
||||
struct MemorySettings : public BaseSettings<memorySettingsTraits>
|
||||
struct MemorySettings
|
||||
{
|
||||
MemorySettings();
|
||||
MemorySettings(const MemorySettings & settings);
|
||||
MemorySettings(MemorySettings && settings) noexcept;
|
||||
~MemorySettings();
|
||||
|
||||
MemorySettings & operator=(MemorySettings && settings);
|
||||
|
||||
MEMORY_SETTINGS_SUPPORTED_TYPES(MemorySettings, DECLARE_SETTING_SUBSCRIPT_OPERATOR)
|
||||
|
||||
void loadFromQuery(ASTStorage & storage_def);
|
||||
ASTPtr getSettingsChangesQuery();
|
||||
void sanityCheck() const;
|
||||
void applyChanges(const SettingsChanges & changes);
|
||||
|
||||
private:
|
||||
std::unique_ptr<MemorySettingsImpl> impl;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -1,6 +1,6 @@
|
||||
#include <Core/BaseSettings.h>
|
||||
#include <Core/BaseSettingsFwdMacrosImpl.h>
|
||||
#include <Core/FormatFactorySettingsDeclaration.h>
|
||||
#include <Core/FormatFactorySettings.h>
|
||||
#include <Parsers/ASTCreateQuery.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/ASTSetQuery.h>
|
||||
|
@ -1,7 +1,7 @@
|
||||
#pragma once
|
||||
|
||||
#include <Core/BaseSettingsFwdMacros.h>
|
||||
#include <Core/FormatFactorySettingsDeclaration.h>
|
||||
#include <Core/FormatFactorySettings.h>
|
||||
#include <Core/SettingsEnums.h>
|
||||
#include <Core/SettingsFields.h>
|
||||
|
||||
|
@ -1,6 +1,6 @@
|
||||
#include <Core/BaseSettings.h>
|
||||
#include <Core/BaseSettingsFwdMacrosImpl.h>
|
||||
#include <Core/FormatFactorySettingsDeclaration.h>
|
||||
#include <Core/FormatFactorySettings.h>
|
||||
#include <Parsers/ASTCreateQuery.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/ASTSetQuery.h>
|
||||
|
@ -1,6 +1,6 @@
|
||||
#include <Core/BaseSettings.h>
|
||||
#include <Core/BaseSettingsFwdMacrosImpl.h>
|
||||
#include <Core/FormatFactorySettingsDeclaration.h>
|
||||
#include <Core/FormatFactorySettings.h>
|
||||
#include <Parsers/ASTCreateQuery.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/ASTSetQuery.h>
|
||||
|
@ -44,6 +44,15 @@ namespace Setting
|
||||
extern const SettingsUInt64 max_compress_block_size;
|
||||
}
|
||||
|
||||
namespace MemorySetting
|
||||
{
|
||||
extern const MemorySettingsBool compress;
|
||||
extern const MemorySettingsUInt64 max_bytes_to_keep;
|
||||
extern const MemorySettingsUInt64 max_rows_to_keep;
|
||||
extern const MemorySettingsUInt64 min_bytes_to_keep;
|
||||
extern const MemorySettingsUInt64 min_rows_to_keep;
|
||||
}
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
||||
@ -78,7 +87,7 @@ public:
|
||||
convertDynamicColumnsToTuples(block, storage_snapshot);
|
||||
}
|
||||
|
||||
if (storage.getMemorySettingsRef().compress)
|
||||
if (storage.getMemorySettingsRef()[MemorySetting::compress])
|
||||
{
|
||||
Block compressed_block;
|
||||
for (const auto & elem : block)
|
||||
@ -110,14 +119,14 @@ public:
|
||||
UInt64 new_total_bytes = storage.total_size_bytes.load(std::memory_order_relaxed) + inserted_bytes;
|
||||
const auto & memory_settings = storage.getMemorySettingsRef();
|
||||
while (!new_data->empty()
|
||||
&& ((memory_settings.max_bytes_to_keep && new_total_bytes > memory_settings.max_bytes_to_keep)
|
||||
|| (memory_settings.max_rows_to_keep && new_total_rows > memory_settings.max_rows_to_keep)))
|
||||
&& ((memory_settings[MemorySetting::max_bytes_to_keep] && new_total_bytes > memory_settings[MemorySetting::max_bytes_to_keep])
|
||||
|| (memory_settings[MemorySetting::max_rows_to_keep] && new_total_rows > memory_settings[MemorySetting::max_rows_to_keep])))
|
||||
{
|
||||
Block oldest_block = new_data->front();
|
||||
UInt64 rows_to_remove = oldest_block.rows();
|
||||
UInt64 bytes_to_remove = oldest_block.allocatedBytes();
|
||||
if (new_total_bytes - bytes_to_remove < memory_settings.min_bytes_to_keep
|
||||
|| new_total_rows - rows_to_remove < memory_settings.min_rows_to_keep)
|
||||
if (new_total_bytes - bytes_to_remove < memory_settings[MemorySetting::min_bytes_to_keep]
|
||||
|| new_total_rows - rows_to_remove < memory_settings[MemorySetting::min_rows_to_keep])
|
||||
{
|
||||
break; // stop - removing next block will put us under min_bytes / min_rows threshold
|
||||
}
|
||||
@ -151,16 +160,18 @@ StorageMemory::StorageMemory(
|
||||
const MemorySettings & memory_settings_)
|
||||
: IStorage(table_id_)
|
||||
, data(std::make_unique<const Blocks>())
|
||||
, memory_settings(memory_settings_)
|
||||
, memory_settings(std::make_unique<MemorySettings>(memory_settings_))
|
||||
{
|
||||
StorageInMemoryMetadata storage_metadata;
|
||||
storage_metadata.setColumns(std::move(columns_description_));
|
||||
storage_metadata.setConstraints(std::move(constraints_));
|
||||
storage_metadata.setComment(comment);
|
||||
storage_metadata.setSettingsChanges(memory_settings.getSettingsChangesQuery());
|
||||
storage_metadata.setSettingsChanges(memory_settings->getSettingsChangesQuery());
|
||||
setInMemoryMetadata(storage_metadata);
|
||||
}
|
||||
|
||||
StorageMemory::~StorageMemory() = default;
|
||||
|
||||
StorageSnapshotPtr StorageMemory::getStorageSnapshot(const StorageMetadataPtr & metadata_snapshot, ContextPtr /*query_context*/) const
|
||||
{
|
||||
auto snapshot_data = std::make_unique<SnapshotData>();
|
||||
@ -246,7 +257,7 @@ void StorageMemory::mutate(const MutationCommands & commands, ContextPtr context
|
||||
Block block;
|
||||
while (executor.pull(block))
|
||||
{
|
||||
if (memory_settings.compress)
|
||||
if ((*memory_settings)[MemorySetting::compress])
|
||||
for (auto & elem : block)
|
||||
elem.column = elem.column->compress();
|
||||
|
||||
@ -310,14 +321,14 @@ void StorageMemory::alter(const DB::AlterCommands & params, DB::ContextPtr conte
|
||||
if (params.isSettingsAlter())
|
||||
{
|
||||
auto & settings_changes = new_metadata.settings_changes->as<ASTSetQuery &>();
|
||||
auto changed_settings = memory_settings;
|
||||
auto changed_settings = *memory_settings;
|
||||
changed_settings.applyChanges(settings_changes.changes);
|
||||
changed_settings.sanityCheck();
|
||||
|
||||
/// When modifying the values of max_bytes_to_keep and max_rows_to_keep to be smaller than the old values,
|
||||
/// the old data needs to be removed.
|
||||
if (!memory_settings.max_bytes_to_keep || memory_settings.max_bytes_to_keep > changed_settings.max_bytes_to_keep
|
||||
|| !memory_settings.max_rows_to_keep || memory_settings.max_rows_to_keep > changed_settings.max_rows_to_keep)
|
||||
if (!(*memory_settings)[MemorySetting::max_bytes_to_keep] || (*memory_settings)[MemorySetting::max_bytes_to_keep] > changed_settings[MemorySetting::max_bytes_to_keep]
|
||||
|| !(*memory_settings)[MemorySetting::max_rows_to_keep] || (*memory_settings)[MemorySetting::max_rows_to_keep] > changed_settings[MemorySetting::max_rows_to_keep])
|
||||
{
|
||||
std::lock_guard lock(mutex);
|
||||
|
||||
@ -325,14 +336,14 @@ void StorageMemory::alter(const DB::AlterCommands & params, DB::ContextPtr conte
|
||||
UInt64 new_total_rows = total_size_rows.load(std::memory_order_relaxed);
|
||||
UInt64 new_total_bytes = total_size_bytes.load(std::memory_order_relaxed);
|
||||
while (!new_data->empty()
|
||||
&& ((changed_settings.max_bytes_to_keep && new_total_bytes > changed_settings.max_bytes_to_keep)
|
||||
|| (changed_settings.max_rows_to_keep && new_total_rows > changed_settings.max_rows_to_keep)))
|
||||
&& ((changed_settings[MemorySetting::max_bytes_to_keep] && new_total_bytes > changed_settings[MemorySetting::max_bytes_to_keep])
|
||||
|| (changed_settings[MemorySetting::max_rows_to_keep] && new_total_rows > changed_settings[MemorySetting::max_rows_to_keep])))
|
||||
{
|
||||
Block oldest_block = new_data->front();
|
||||
UInt64 rows_to_remove = oldest_block.rows();
|
||||
UInt64 bytes_to_remove = oldest_block.allocatedBytes();
|
||||
if (new_total_bytes - bytes_to_remove < changed_settings.min_bytes_to_keep
|
||||
|| new_total_rows - rows_to_remove < changed_settings.min_rows_to_keep)
|
||||
if (new_total_bytes - bytes_to_remove < changed_settings[MemorySetting::min_bytes_to_keep]
|
||||
|| new_total_rows - rows_to_remove < changed_settings[MemorySetting::min_rows_to_keep])
|
||||
{
|
||||
break; // stop - removing next block will put us under min_bytes / min_rows threshold
|
||||
}
|
||||
@ -347,7 +358,7 @@ void StorageMemory::alter(const DB::AlterCommands & params, DB::ContextPtr conte
|
||||
total_size_rows.store(new_total_rows, std::memory_order_relaxed);
|
||||
total_size_bytes.store(new_total_bytes, std::memory_order_relaxed);
|
||||
}
|
||||
memory_settings = std::move(changed_settings);
|
||||
*memory_settings = std::move(changed_settings);
|
||||
}
|
||||
|
||||
DatabaseCatalog::instance().getDatabase(table_id.database_name)->alterTable(context, table_id, new_metadata);
|
||||
@ -559,7 +570,7 @@ void StorageMemory::restoreDataImpl(const BackupPtr & backup, const String & dat
|
||||
|
||||
while (auto block = block_in.read())
|
||||
{
|
||||
if (memory_settings.compress)
|
||||
if ((*memory_settings)[MemorySetting::compress])
|
||||
{
|
||||
Block compressed_block;
|
||||
for (const auto & elem : block)
|
||||
|
@ -7,7 +7,6 @@
|
||||
#include <Core/NamesAndTypes.h>
|
||||
#include <Interpreters/DatabaseCatalog.h>
|
||||
#include <Storages/IStorage.h>
|
||||
#include <Storages/MemorySettings.h>
|
||||
|
||||
#include <Common/MultiVersion.h>
|
||||
|
||||
@ -15,6 +14,7 @@ namespace DB
|
||||
{
|
||||
class IBackup;
|
||||
using BackupPtr = std::shared_ptr<const IBackup>;
|
||||
struct MemorySettings;
|
||||
|
||||
/** Implements storage in the RAM.
|
||||
* Suitable for temporary data.
|
||||
@ -31,7 +31,9 @@ public:
|
||||
ColumnsDescription columns_description_,
|
||||
ConstraintsDescription constraints_,
|
||||
const String & comment,
|
||||
const MemorySettings & memory_settings_ = MemorySettings());
|
||||
const MemorySettings & memory_settings_);
|
||||
|
||||
~StorageMemory() override;
|
||||
|
||||
String getName() const override { return "Memory"; }
|
||||
|
||||
@ -47,7 +49,7 @@ public:
|
||||
|
||||
StorageSnapshotPtr getStorageSnapshot(const StorageMetadataPtr & metadata_snapshot, ContextPtr query_context) const override;
|
||||
|
||||
const MemorySettings & getMemorySettingsRef() const { return memory_settings; }
|
||||
const MemorySettings & getMemorySettingsRef() const { return *memory_settings; }
|
||||
|
||||
void read(
|
||||
QueryPlan & query_plan,
|
||||
@ -139,7 +141,7 @@ private:
|
||||
std::atomic<size_t> total_size_bytes = 0;
|
||||
std::atomic<size_t> total_size_rows = 0;
|
||||
|
||||
MemorySettings memory_settings;
|
||||
std::unique_ptr<MemorySettings> memory_settings;
|
||||
|
||||
friend class ReadFromMemoryStorageStep;
|
||||
};
|
||||
|
@ -1,5 +1,6 @@
|
||||
#include <gtest/gtest.h>
|
||||
|
||||
#include <Storages/MemorySettings.h>
|
||||
#include <Storages/transformQueryForExternalDatabase.h>
|
||||
#include <Parsers/ParserSelectQuery.h>
|
||||
#include <Parsers/parseQuery.h>
|
||||
@ -105,7 +106,7 @@ private:
|
||||
context,
|
||||
table_name,
|
||||
std::make_shared<StorageMemory>(
|
||||
StorageID(db_name, table_name), ColumnsDescription{tab.columns}, ConstraintsDescription{}, String{}));
|
||||
StorageID(db_name, table_name), ColumnsDescription{tab.columns}, ConstraintsDescription{}, String{}, MemorySettings{}));
|
||||
}
|
||||
DatabaseCatalog::instance().attachDatabase(database->getDatabaseName(), database);
|
||||
|
||||
|
@ -410,7 +410,7 @@ class PRInfo:
|
||||
(ext in DIFF_IN_DOCUMENTATION_EXT and path_in_docs)
|
||||
or "docker/docs" in f
|
||||
or "Settings.cpp" in f
|
||||
or "FormatFactorySettingsDeclaration.h" in f
|
||||
or "FormatFactorySettings.h" in f
|
||||
):
|
||||
return True
|
||||
return False
|
||||
|
@ -12,10 +12,14 @@
|
||||
export LC_COLLATE="C"
|
||||
ROOT_PATH=$(git rev-parse --show-toplevel)
|
||||
|
||||
# Duplicated or incorrect setting declarations
|
||||
|
||||
SETTINGS_FILE=$(mktemp)
|
||||
trap "rm ${SETTINGS_FILE}" EXIT
|
||||
|
||||
|
||||
# Please note that ALL FILES MUST BE NAMED {}Settings and that must match the class name too
|
||||
ALL_DECLARATION_FILES="
|
||||
$ROOT_PATH/src/Core/FormatFactorySettings.h
|
||||
$ROOT_PATH/src/Core/Settings.cpp
|
||||
$ROOT_PATH/src/Core/ServerSettings.cpp
|
||||
$ROOT_PATH/src/Storages/MergeTree/MergeTreeSettings.cpp
|
||||
@ -33,33 +37,26 @@ ALL_DECLARATION_FILES="
|
||||
$ROOT_PATH/src/Storages/FileLog/FileLogSettings.cpp
|
||||
$ROOT_PATH/src/Storages/Distributed/DistributedSettings.cpp
|
||||
$ROOT_PATH/src/Storages/SetSettings.cpp
|
||||
$ROOT_PATH/src/Core/FormatFactorySettingsDeclaration.h"
|
||||
$ROOT_PATH/src/Storages/MemorySettings.cpp
|
||||
"
|
||||
|
||||
# We create an initial file with the shape {setting_name} {ClassName}{Type} SettingsDeclaration
|
||||
# We will use SettingsDeclaration to differentiate between setting declaration and usage
|
||||
function add_setting_declaration_file()
|
||||
{
|
||||
if ! [ -f "$1" ]; then
|
||||
echo "File '$1' does not exist."
|
||||
fi
|
||||
filename=$(basename -- "$1")
|
||||
filename="${filename%.*}"
|
||||
grep " M(" $1 | awk -vfilename="${filename}" '{print substr($2, 0, length($2) - 1) " " filename substr($1, 3, length($1) - 3) " SettingsDeclaration" }' | sort | uniq >> ${SETTINGS_FILE}
|
||||
}
|
||||
|
||||
for settings_file in ${ALL_DECLARATION_FILES};
|
||||
do
|
||||
if ! [ -f "${settings_file}" ]; then
|
||||
echo "File '${settings_file}' does not exist."
|
||||
fi
|
||||
add_setting_declaration_file $settings_file
|
||||
done
|
||||
|
||||
cat $ROOT_PATH/src/Core/Settings.cpp $ROOT_PATH/src/Core/FormatFactorySettingsDeclaration.h | grep " M(" | awk '{print substr($2, 0, length($2) - 1) " *Settings" substr($1, 3, length($1) - 3) " SettingsDeclaration" }' | sort | uniq > ${SETTINGS_FILE}
|
||||
cat $ROOT_PATH/src/Core/ServerSettings.cpp | grep " M(" | awk '{print substr($2, 0, length($2) - 1) " ServerSettings" substr($1, 3, length($1) - 3) " SettingsDeclaration" }' | sort | uniq >> ${SETTINGS_FILE}
|
||||
cat $ROOT_PATH/src/Storages/MergeTree/MergeTreeSettings.cpp | grep " M(" | awk '{print substr($2, 0, length($2) - 1) " MergeTreeSettings" substr($1, 3, length($1) - 3) " SettingsDeclaration" }' | sort | uniq >> ${SETTINGS_FILE}
|
||||
cat $ROOT_PATH/src/Coordination/CoordinationSettings.cpp | grep " M(" | awk '{print substr($2, 0, length($2) - 1) " CoordinationSettings" substr($1, 3, length($1) - 3) " SettingsDeclaration" }' | sort | uniq >> ${SETTINGS_FILE}
|
||||
cat $ROOT_PATH/src/Databases/DatabaseReplicatedSettings.cpp | grep " M(" | awk '{print substr($2, 0, length($2) - 1) " DatabaseReplicatedSettings" substr($1, 3, length($1) - 3) " SettingsDeclaration" }' | sort | uniq >> ${SETTINGS_FILE}
|
||||
cat $ROOT_PATH/src/Storages/TimeSeries/TimeSeriesSettings.cpp | grep " M(" | awk '{print substr($2, 0, length($2) - 1) " TimeSeriesSettings" substr($1, 3, length($1) - 3) " SettingsDeclaration" }' | sort | uniq >> ${SETTINGS_FILE}
|
||||
cat $ROOT_PATH/src/Storages/RocksDB/RocksDBSettings.cpp | grep " M(" | awk '{print substr($2, 0, length($2) - 1) " RocksDBSettings" substr($1, 3, length($1) - 3) " SettingsDeclaration" }' | sort | uniq >> ${SETTINGS_FILE}
|
||||
cat $ROOT_PATH/src/Storages/RabbitMQ/RabbitMQSettings.cpp | grep " M(" | awk '{print substr($2, 0, length($2) - 1) " RabbitMQSettings" substr($1, 3, length($1) - 3) " SettingsDeclaration" }' | sort | uniq >> ${SETTINGS_FILE}
|
||||
cat $ROOT_PATH/src/Storages/PostgreSQL/MaterializedPostgreSQLSettings.cpp | grep " M(" | awk '{print substr($2, 0, length($2) - 1) " MaterializedPostgreSQLSettings" substr($1, 3, length($1) - 3) " SettingsDeclaration" }' | sort | uniq >> ${SETTINGS_FILE}
|
||||
cat $ROOT_PATH/src/Storages/ObjectStorageQueue/ObjectStorageQueueSettings.cpp | grep " M(" | awk '{print substr($2, 0, length($2) - 1) " ObjectStorageQueueSettings" substr($1, 3, length($1) - 3) " SettingsDeclaration" }' | sort | uniq >> ${SETTINGS_FILE}
|
||||
cat $ROOT_PATH/src/Storages/MaterializedView/RefreshSettings.cpp | grep " M(" | awk '{print substr($2, 0, length($2) - 1) " RefreshSettings" substr($1, 3, length($1) - 3) " SettingsDeclaration" }' | sort | uniq >> ${SETTINGS_FILE}
|
||||
cat $ROOT_PATH/src/Storages/NATS/NATSSettings.cpp | grep " M(" | awk '{print substr($2, 0, length($2) - 1) " NATSSettings" substr($1, 3, length($1) - 3) " SettingsDeclaration" }' | sort | uniq >> ${SETTINGS_FILE}
|
||||
cat $ROOT_PATH/src/Storages/Kafka/KafkaSettings.cpp | grep " M(" | awk '{print substr($2, 0, length($2) - 1) " KafkaSettings" substr($1, 3, length($1) - 3) " SettingsDeclaration" }' | sort | uniq >> ${SETTINGS_FILE}
|
||||
cat $ROOT_PATH/src/Storages/Hive/HiveSettings.cpp | grep " M(" | awk '{print substr($2, 0, length($2) - 1) " HiveSettings" substr($1, 3, length($1) - 3) " SettingsDeclaration" }' | sort | uniq >> ${SETTINGS_FILE}
|
||||
cat $ROOT_PATH/src/Storages/FileLog/FileLogSettings.cpp | grep " M(" | awk '{print substr($2, 0, length($2) - 1) " FileLogSettings" substr($1, 3, length($1) - 3) " SettingsDeclaration" }' | sort | uniq >> ${SETTINGS_FILE}
|
||||
cat $ROOT_PATH/src/Storages/Distributed/DistributedSettings.cpp | grep " M(" | awk '{print substr($2, 0, length($2) - 1) " DistributedSettings" substr($1, 3, length($1) - 3) " SettingsDeclaration" }' | sort | uniq >> ${SETTINGS_FILE}
|
||||
cat $ROOT_PATH/src/Storages/SetSettings.cpp | grep " M(" | awk '{print substr($2, 0, length($2) - 1) " SetSettings" substr($1, 3, length($1) - 3) " SettingsDeclaration" }' | sort | uniq >> ${SETTINGS_FILE}
|
||||
|
||||
# Check that if there are duplicated settings (declared in different objects) they all have the same type (it's simpler to validate style with that assert)
|
||||
for setting in $(
|
||||
awk '{ gsub(/^.*Settings/, "", $2); print $1 " " $2}' ${SETTINGS_FILE} | \
|
||||
@ -74,9 +71,9 @@ done
|
||||
# Note that rg outputs 'path:$line', so with replace ':' with a space and then reorder to have "$setting $type $path"
|
||||
find $ROOT_PATH/{src,base,programs,utils} -name '*.h' -or -name '*.cpp' | \
|
||||
xargs rg "^\s*extern const .*Settings" | tr ':' ' ' | \
|
||||
awk '{print substr($5, 0, length($5) -1) " " $4 " " substr($1, 0, length($1) - 1)}' >> ${SETTINGS_FILE}
|
||||
awk '{print substr($5, 0, length($5) -1) " " $4 " " $1}' >> ${SETTINGS_FILE}
|
||||
|
||||
# Duplicate extern declarations for settings
|
||||
# Detect duplicate extern declarations for settings (harmless but better style)
|
||||
awk '{if (seen[$0]++) print $3 " -> " $1 ;}' ${SETTINGS_FILE} | while read line;
|
||||
do
|
||||
echo "# Found duplicated setting declaration in: $line"
|
||||
@ -91,6 +88,8 @@ done
|
||||
#done
|
||||
|
||||
# Look for settings declared with multiple types
|
||||
# This works based on the fact that the if the setting declaration and usage have different types then the pair
|
||||
# <setting, type> won't be unique
|
||||
for setting in $(
|
||||
awk '{ gsub(/^.*Settings/, "", $2); print $1 " " $2}' ${SETTINGS_FILE} | \
|
||||
sort | uniq | awk '{ print $1 }' | uniq -d
|
||||
|
Loading…
Reference in New Issue
Block a user