mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
rename disable_persistency to persistency
This commit is contained in:
parent
7152fec241
commit
0c535062fc
@ -10,7 +10,7 @@ class ASTStorage;
|
||||
|
||||
|
||||
#define JOIN_RELATED_SETTINGS(M) \
|
||||
M(Bool, disable_persistency, false, "Disable persistency for StorageJoin to reduce IO overhead", 0)
|
||||
M(Bool, persistency, true, "Disable setting to avoid the overhead of writing to disk for StorageJoin", 0)
|
||||
|
||||
#define LIST_OF_JOIN_SETTINGS(M) \
|
||||
JOIN_RELATED_SETTINGS(M) \
|
||||
|
@ -10,7 +10,7 @@ class ASTStorage;
|
||||
|
||||
|
||||
#define SET_RELATED_SETTINGS(M) \
|
||||
M(Bool, disable_persistency, false, "Disable persistency for StorageSet to reduce IO overhead", 0)
|
||||
M(Bool, persistency, true, "Disable setting to avoid the overhead of writing to disk for StorageSet", 0)
|
||||
|
||||
#define LIST_OF_SET_SETTINGS(M) \
|
||||
SET_RELATED_SETTINGS(M) \
|
||||
|
@ -45,8 +45,8 @@ StorageJoin::StorageJoin(
|
||||
const ConstraintsDescription & constraints_,
|
||||
bool overwrite_,
|
||||
const Context & context_,
|
||||
bool disable_persistency_)
|
||||
: StorageSetOrJoinBase{relative_path_, table_id_, columns_, constraints_, context_, disable_persistency_}
|
||||
bool persistency_)
|
||||
: StorageSetOrJoinBase{relative_path_, table_id_, columns_, constraints_, context_, persistency_}
|
||||
, key_names(key_names_)
|
||||
, use_nulls(use_nulls_)
|
||||
, limits(limits_)
|
||||
@ -119,7 +119,7 @@ void registerStorageJoin(StorageFactory & factory)
|
||||
auto join_overflow_mode = settings.join_overflow_mode;
|
||||
auto join_any_take_last_row = settings.join_any_take_last_row;
|
||||
auto old_any_join = settings.any_join_distinct_right_table_keys;
|
||||
bool disable_persistency = false;
|
||||
bool persistency_ = true;
|
||||
|
||||
if (args.storage_def && args.storage_def->settings)
|
||||
{
|
||||
@ -137,11 +137,11 @@ void registerStorageJoin(StorageFactory & factory)
|
||||
join_any_take_last_row = setting.value;
|
||||
else if (setting.name == "any_join_distinct_right_table_keys")
|
||||
old_any_join = setting.value;
|
||||
else if (setting.name == "disable_persistency")
|
||||
else if (setting.name == "persistency")
|
||||
{
|
||||
auto join_settings = std::make_unique<JoinSettings>();
|
||||
join_settings->loadFromQuery(*args.storage_def);
|
||||
disable_persistency = join_settings->disable_persistency;
|
||||
persistency_ = join_settings->persistency;
|
||||
}
|
||||
else
|
||||
throw Exception(
|
||||
@ -226,7 +226,7 @@ void registerStorageJoin(StorageFactory & factory)
|
||||
args.constraints,
|
||||
join_any_take_last_row,
|
||||
args.context,
|
||||
disable_persistency);
|
||||
persistency_);
|
||||
};
|
||||
|
||||
factory.registerStorage("Join", creator_fn, StorageFactory::StorageFeatures{ .supports_settings = true, });
|
||||
|
@ -74,7 +74,7 @@ protected:
|
||||
const ConstraintsDescription & constraints_,
|
||||
bool overwrite,
|
||||
const Context & context_,
|
||||
bool disable_persistency_);
|
||||
bool persistency_);
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -37,7 +37,7 @@ public:
|
||||
SetOrJoinBlockOutputStream(
|
||||
StorageSetOrJoinBase & table_, const StorageMetadataPtr & metadata_snapshot_,
|
||||
const String & backup_path_, const String & backup_tmp_path_,
|
||||
const String & backup_file_name_, bool disable_persistency_);
|
||||
const String & backup_file_name_, bool persistency_);
|
||||
|
||||
Block getHeader() const override { return metadata_snapshot->getSampleBlock(); }
|
||||
void write(const Block & block) override;
|
||||
@ -52,7 +52,7 @@ private:
|
||||
WriteBufferFromFile backup_buf;
|
||||
CompressedWriteBuffer compressed_backup_buf;
|
||||
NativeBlockOutputStream backup_stream;
|
||||
bool disable_persistency;
|
||||
bool persistency;
|
||||
};
|
||||
|
||||
|
||||
@ -62,7 +62,7 @@ SetOrJoinBlockOutputStream::SetOrJoinBlockOutputStream(
|
||||
const String & backup_path_,
|
||||
const String & backup_tmp_path_,
|
||||
const String & backup_file_name_,
|
||||
bool disable_persistency_)
|
||||
bool persistency_)
|
||||
: table(table_)
|
||||
, metadata_snapshot(metadata_snapshot_)
|
||||
, backup_path(backup_path_)
|
||||
@ -71,7 +71,7 @@ SetOrJoinBlockOutputStream::SetOrJoinBlockOutputStream(
|
||||
, backup_buf(backup_tmp_path + backup_file_name)
|
||||
, compressed_backup_buf(backup_buf)
|
||||
, backup_stream(compressed_backup_buf, 0, metadata_snapshot->getSampleBlock())
|
||||
, disable_persistency(disable_persistency_)
|
||||
, persistency(persistency_)
|
||||
{
|
||||
}
|
||||
|
||||
@ -81,14 +81,14 @@ void SetOrJoinBlockOutputStream::write(const Block & block)
|
||||
Block sorted_block = block.sortColumns();
|
||||
|
||||
table.insertBlock(sorted_block);
|
||||
if (!disable_persistency)
|
||||
if (persistency)
|
||||
backup_stream.write(sorted_block);
|
||||
}
|
||||
|
||||
void SetOrJoinBlockOutputStream::writeSuffix()
|
||||
{
|
||||
table.finishInsert();
|
||||
if (!disable_persistency)
|
||||
if (persistency)
|
||||
{
|
||||
backup_stream.flush();
|
||||
compressed_backup_buf.next();
|
||||
@ -102,7 +102,7 @@ void SetOrJoinBlockOutputStream::writeSuffix()
|
||||
BlockOutputStreamPtr StorageSetOrJoinBase::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, const Context & /*context*/)
|
||||
{
|
||||
UInt64 id = ++increment;
|
||||
return std::make_shared<SetOrJoinBlockOutputStream>(*this, metadata_snapshot, path, path + "tmp/", toString(id) + ".bin", disable_persistency);
|
||||
return std::make_shared<SetOrJoinBlockOutputStream>(*this, metadata_snapshot, path, path + "tmp/", toString(id) + ".bin", persistency);
|
||||
}
|
||||
|
||||
|
||||
@ -112,9 +112,9 @@ StorageSetOrJoinBase::StorageSetOrJoinBase(
|
||||
const ColumnsDescription & columns_,
|
||||
const ConstraintsDescription & constraints_,
|
||||
const Context & context_,
|
||||
bool disable_persistency_)
|
||||
bool persistency_)
|
||||
: IStorage(table_id_),
|
||||
disable_persistency(disable_persistency_)
|
||||
persistency(persistency_)
|
||||
{
|
||||
StorageInMemoryMetadata storage_metadata;
|
||||
storage_metadata.setColumns(columns_);
|
||||
@ -136,8 +136,8 @@ StorageSet::StorageSet(
|
||||
const ColumnsDescription & columns_,
|
||||
const ConstraintsDescription & constraints_,
|
||||
const Context & context_,
|
||||
bool disable_persistency_)
|
||||
: StorageSetOrJoinBase{relative_path_, table_id_, columns_, constraints_, context_, disable_persistency_},
|
||||
bool persistency_)
|
||||
: StorageSetOrJoinBase{relative_path_, table_id_, columns_, constraints_, context_, persistency_},
|
||||
set(std::make_shared<Set>(SizeLimits(), false, true))
|
||||
{
|
||||
|
||||
@ -249,7 +249,7 @@ void registerStorageSet(StorageFactory & factory)
|
||||
set_settings->loadFromQuery(*args.storage_def);
|
||||
}
|
||||
|
||||
return StorageSet::create(args.relative_data_path, args.table_id, args.columns, args.constraints, args.context, set_settings->disable_persistency);
|
||||
return StorageSet::create(args.relative_data_path, args.table_id, args.columns, args.constraints, args.context, set_settings->persistency);
|
||||
}, StorageFactory::StorageFeatures{ .supports_settings = true, });
|
||||
}
|
||||
|
||||
|
@ -33,11 +33,11 @@ protected:
|
||||
const ColumnsDescription & columns_,
|
||||
const ConstraintsDescription & constraints_,
|
||||
const Context & context_,
|
||||
bool disable_persistency_);
|
||||
bool persistency_);
|
||||
|
||||
String base_path;
|
||||
String path;
|
||||
bool disable_persistency;
|
||||
bool persistency;
|
||||
|
||||
std::atomic<UInt64> increment = 0; /// For the backup file names.
|
||||
|
||||
@ -86,7 +86,7 @@ protected:
|
||||
const ColumnsDescription & columns_,
|
||||
const ConstraintsDescription & constraints_,
|
||||
const Context & context_,
|
||||
bool disable_persistency_);
|
||||
bool persistency_);
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -0,0 +1,5 @@
|
||||
----- Default Settings -----
|
||||
1
|
||||
----- Settings persistency=1 -----
|
||||
1
|
||||
----- Settings persistency=0 -----
|
@ -1,8 +1,33 @@
|
||||
DROP TABLE IF EXISTS set;
|
||||
DROP TABLE IF EXISTS number;
|
||||
|
||||
CREATE TABLE set (x String) ENGINE = Set() SETTINGS disable_persistency=1;
|
||||
CREATE TABLE number (number UInt64) ENGINE = Memory();
|
||||
INSERT INTO number values (1);
|
||||
|
||||
SELECT '----- Default Settings -----';
|
||||
CREATE TABLE set (val UInt64) ENGINE = Set();
|
||||
INSERT INTO set VALUES (1);
|
||||
DETACH TABLE set;
|
||||
ATTACH TABLE set (x String) ENGINE = Set() SETTINGS disable_persistency=1;
|
||||
ATTACH TABLE set (val UInt64) ENGINE = Set();
|
||||
SELECT number FROM number WHERE number IN set LIMIT 1;
|
||||
|
||||
DROP TABLE set;
|
||||
|
||||
SELECT '----- Settings persistency=1 -----';
|
||||
CREATE TABLE set (val UInt64) ENGINE = Set() SETTINGS persistency=1;
|
||||
INSERT INTO set VALUES (1);
|
||||
DETACH TABLE set;
|
||||
ATTACH TABLE set (val UInt64) ENGINE = Set() SETTINGS persistency=1;
|
||||
SELECT number FROM number WHERE number IN set LIMIT 1;
|
||||
|
||||
DROP TABLE set;
|
||||
|
||||
SELECT '----- Settings persistency=0 -----';
|
||||
CREATE TABLE set (val UInt64) ENGINE = Set() SETTINGS persistency=0;
|
||||
INSERT INTO set VALUES (1);
|
||||
DETACH TABLE set;
|
||||
ATTACH TABLE set (val UInt64) ENGINE = Set() SETTINGS persistency=0;
|
||||
SELECT number FROM number WHERE number IN set LIMIT 1;
|
||||
|
||||
DROP TABLE set;
|
||||
DROP TABLE number;
|
@ -0,0 +1,5 @@
|
||||
----- Default Settings -----
|
||||
1 21
|
||||
----- Settings persistency=1 -----
|
||||
1 21
|
||||
----- Settings persistency=0 -----
|
@ -1,8 +1,28 @@
|
||||
DROP TABLE IF EXISTS join;
|
||||
|
||||
CREATE TABLE join (k UInt64, s String) ENGINE = Join(ANY, LEFT, k) SETTINGS disable_persistency=1;
|
||||
|
||||
SELECT '----- Default Settings -----';
|
||||
CREATE TABLE join (k UInt64, s String) ENGINE = Join(ANY, LEFT, k);
|
||||
INSERT INTO join VALUES (1,21);
|
||||
DETACH TABLE join;
|
||||
ATTACH TABLE join (k UInt64, s String) ENGINE = Join(ANY, LEFT, k) SETTINGS disable_persistency=1;
|
||||
ATTACH TABLE join (k UInt64, s String) ENGINE = Join(ANY, LEFT, k);
|
||||
SELECT * from join;
|
||||
|
||||
DROP TABLE join;
|
||||
|
||||
SELECT '----- Settings persistency=1 -----';
|
||||
CREATE TABLE join (k UInt64, s String) ENGINE = Join(ANY, LEFT, k) SETTINGS persistency=1;
|
||||
INSERT INTO join VALUES (1,21);
|
||||
DETACH TABLE join;
|
||||
ATTACH TABLE join (k UInt64, s String) ENGINE = Join(ANY, LEFT, k) SETTINGS persistency=1;
|
||||
SELECT * from join;
|
||||
|
||||
DROP TABLE join;
|
||||
|
||||
SELECT '----- Settings persistency=0 -----';
|
||||
CREATE TABLE join (k UInt64, s String) ENGINE = Join(ANY, LEFT, k) SETTINGS persistency=0;
|
||||
INSERT INTO join VALUES (1,21);
|
||||
DETACH TABLE join;
|
||||
ATTACH TABLE join (k UInt64, s String) ENGINE = Join(ANY, LEFT, k) SETTINGS persistency=0;
|
||||
SELECT * from join;
|
||||
|
||||
DROP TABLE join;
|
||||
|
Loading…
Reference in New Issue
Block a user