Skip to content

Commit

Permalink
PageStorage: Refactor config (pingcap#6138)
Browse files Browse the repository at this point in the history
  • Loading branch information
JaySon-Huang committed Oct 21, 2022
1 parent af0308e commit 2c210e5
Show file tree
Hide file tree
Showing 53 changed files with 506 additions and 436 deletions.
2 changes: 1 addition & 1 deletion dbms/src/Server/tests/gtest_server_config.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -289,7 +289,7 @@ dt_page_gc_low_write_prob = 0.2
auto & global_path_pool = global_ctx.getPathPool();
RegionManager region_manager;
RegionPersister persister(global_ctx, region_manager);
persister.restore(global_path_pool, nullptr, PageStorage::Config{});
persister.restore(global_path_pool, nullptr, PageStorageConfig{});

auto verify_persister_reload_config = [&global_ctx](RegionPersister & persister) {
DB::Settings & settings = global_ctx.getSettingsRef();
Expand Down
4 changes: 2 additions & 2 deletions dbms/src/Storages/DeltaMerge/StoragePool.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -54,7 +54,7 @@ enum class StorageType
Meta = 3,
};

PageStorage::Config extractConfig(const Settings & settings, StorageType subtype)
PageStorageConfig extractConfig(const Settings & settings, StorageType subtype)
{
#define SET_CONFIG(NAME) \
config.num_write_slots = settings.dt_storage_pool_##NAME##_write_slots; \
Expand All @@ -64,7 +64,7 @@ PageStorage::Config extractConfig(const Settings & settings, StorageType subtype
config.gc_max_valid_rate = settings.dt_storage_pool_##NAME##_gc_max_valid_rate; \
config.blob_heavy_gc_valid_rate = settings.dt_page_gc_threshold;

PageStorage::Config config = getConfigFromSettings(settings);
PageStorageConfig config = getConfigFromSettings(settings);

switch (subtype)
{
Expand Down
8 changes: 4 additions & 4 deletions dbms/src/Storages/DeltaMerge/WriteBatches.h
Original file line number Diff line number Diff line change
Expand Up @@ -97,7 +97,7 @@ struct WriteBatches : private boost::noncopyable
return;
for (const auto & w : wb.getWrites())
{
if (unlikely(w.type == WriteBatch::WriteType::DEL))
if (unlikely(w.type == WriteBatchWriteType::DEL))
throw Exception("Unexpected deletes in " + what);
}
LOG_FMT_TRACE(logger, "Write into {} : {}", what, wb.toString());
Expand Down Expand Up @@ -142,7 +142,7 @@ struct WriteBatches : private boost::noncopyable
return;
for (const auto & w : wb.getWrites())
{
if (unlikely(w.type != WriteBatch::WriteType::DEL))
if (unlikely(w.type != WriteBatchWriteType::DEL))
throw Exception("Expected deletes in " + what);
}
LOG_FMT_TRACE(logger, "Rollback remove from {} : {}", what, wb.toString());
Expand Down Expand Up @@ -170,7 +170,7 @@ struct WriteBatches : private boost::noncopyable
return;
for (const auto & w : wb.getWrites())
{
if (unlikely(w.type != WriteBatch::WriteType::PUT))
if (unlikely(w.type != WriteBatchWriteType::PUT))
throw Exception("Expected puts in " + what);
}
LOG_FMT_TRACE(logger, "Write into {} : {}", what, wb.toString());
Expand All @@ -194,7 +194,7 @@ struct WriteBatches : private boost::noncopyable
return;
for (const auto & w : wb.getWrites())
{
if (unlikely(w.type != WriteBatch::WriteType::DEL))
if (unlikely(w.type != WriteBatchWriteType::DEL))
throw Exception("Expected deletes in " + what);
}
LOG_FMT_TRACE(logger, "Write into {} : {}", what, wb.toString());
Expand Down
28 changes: 28 additions & 0 deletions dbms/src/Storages/Page/Config.cpp
Original file line number Diff line number Diff line change
@@ -0,0 +1,28 @@
// Copyright 2022 PingCAP, Ltd.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.

#include <Storages/Page/Config.h>
#include <Storages/Page/PageUtil.h>

namespace DB
{
namespace MVCC
{
bool VersionSetConfig::doCleanup() const
{
return PageUtil::randInt(0, 1000) < prob_cleanup_invalid_snapshot;
}
} // namespace MVCC

} // namespace DB
126 changes: 120 additions & 6 deletions dbms/src/Storages/Page/Config.h
Original file line number Diff line number Diff line change
Expand Up @@ -14,13 +14,15 @@

#pragma once

// todo move this util into other dir
#include <Core/Types.h>
#include <Storages/Page/PageUtil.h>
#include <Interpreters/SettingsCommon.h>
#include <Storages/Page/PageDefines.h>

namespace DB::MVCC
namespace DB
{
// MVCC config
namespace MVCC
{
// V2 MVCC config
struct VersionSetConfig
{
size_t compact_hint_delta_deletions = 5000;
Expand All @@ -35,10 +37,122 @@ struct VersionSetConfig
prob_cleanup_invalid_snapshot = prob;
}

bool doCleanup() const { return PageUtil::randInt(0, 1000) < prob_cleanup_invalid_snapshot; }
bool doCleanup() const;

private:
// Probability to cleanup invalid snapshots. 10 out of 1000 by default.
size_t prob_cleanup_invalid_snapshot = 10;
};
} // namespace DB::MVCC
} // namespace MVCC

struct PageStorageConfig
{
//==========================================================================================
// V2 config
//==========================================================================================
SettingBool sync_on_write = true;

SettingUInt64 file_roll_size = PAGE_FILE_ROLL_SIZE;
SettingUInt64 file_max_size = PAGE_FILE_MAX_SIZE;
SettingUInt64 file_small_size = PAGE_FILE_SMALL_SIZE;

SettingUInt64 file_meta_roll_size = PAGE_META_ROLL_SIZE;

// When the value of gc_force_hardlink_rate is less than or equal to 1,
// It means that candidates whose valid rate is greater than this value will be forced to hardlink(This will reduce the gc duration).
// Otherwise, if gc_force_hardlink_rate is greater than 1, hardlink won't happen
SettingDouble gc_force_hardlink_rate = 2;

SettingDouble gc_max_valid_rate = 0.35;
SettingUInt64 gc_min_bytes = PAGE_FILE_ROLL_SIZE;
SettingUInt64 gc_min_files = 10;
// Minimum number of legacy files to be selected for compaction
SettingUInt64 gc_min_legacy_num = 3;

SettingUInt64 gc_max_expect_legacy_files = 100;
SettingDouble gc_max_valid_rate_bound = 1.0;

// Maximum write concurrency. Must not be changed once the PageStorage object is created.
SettingUInt64 num_write_slots = 1;

// Maximum seconds of reader / writer idle time.
// 0 for never reclaim idle file descriptor.
SettingUInt64 open_file_max_idle_time = 15;

// Probability to do gc when write is low.
// The probability is `prob_do_gc_when_write_is_low` out of 1000.
SettingUInt64 prob_do_gc_when_write_is_low = 10;

MVCC::VersionSetConfig version_set_config;

//==========================================================================================
// V3 config
//==========================================================================================
SettingUInt64 blob_file_limit_size = BLOBFILE_LIMIT_SIZE;
SettingUInt64 blob_spacemap_type = 2;
SettingUInt64 blob_cached_fd_size = BLOBSTORE_CACHED_FD_SIZE;
SettingDouble blob_heavy_gc_valid_rate = 0.5;
SettingUInt64 blob_block_alignment_bytes = 0;

SettingUInt64 wal_roll_size = PAGE_META_ROLL_SIZE;
SettingUInt64 wal_max_persisted_log_files = MAX_PERSISTED_LOG_FILES;

void reload(const PageStorageConfig & rhs)
{
// Reload is not atomic, but should be good enough

// Reload gc threshold
gc_force_hardlink_rate = rhs.gc_force_hardlink_rate;
gc_max_valid_rate = rhs.gc_max_valid_rate;
gc_min_bytes = rhs.gc_min_bytes;
gc_min_files = rhs.gc_min_files;
gc_min_legacy_num = rhs.gc_min_legacy_num;
prob_do_gc_when_write_is_low = rhs.prob_do_gc_when_write_is_low;
// Reload fd idle time
open_file_max_idle_time = rhs.open_file_max_idle_time;

// Reload V3 setting
blob_file_limit_size = rhs.blob_file_limit_size;
blob_spacemap_type = rhs.blob_spacemap_type;
blob_cached_fd_size = rhs.blob_cached_fd_size;
blob_heavy_gc_valid_rate = rhs.blob_heavy_gc_valid_rate;
blob_block_alignment_bytes = rhs.blob_block_alignment_bytes;

wal_roll_size = rhs.wal_roll_size;
wal_max_persisted_log_files = rhs.wal_max_persisted_log_files;
}

String toDebugStringV2() const
{
return fmt::format(
"PageStorageConfig {{gc_min_files: {}, gc_min_bytes:{}, gc_force_hardlink_rate: {:.3f}, gc_max_valid_rate: {:.3f}, "
"gc_min_legacy_num: {}, gc_max_expect_legacy: {}, gc_max_valid_rate_bound: {:.3f}, prob_do_gc_when_write_is_low: {}, "
"open_file_max_idle_time: {}}}",
gc_min_files,
gc_min_bytes,
gc_force_hardlink_rate.get(),
gc_max_valid_rate.get(),
gc_min_legacy_num,
gc_max_expect_legacy_files.get(),
gc_max_valid_rate_bound.get(),
prob_do_gc_when_write_is_low,
open_file_max_idle_time);
}

String toDebugStringV3() const
{
return fmt::format(
"PageStorageConfig V3 {{"
"blob_file_limit_size: {}, blob_spacemap_type: {}, "
"blob_cached_fd_size: {}, blob_heavy_gc_valid_rate: {:.3f}, blob_block_alignment_bytes: {}, "
"wal_roll_size: {}, wal_max_persisted_log_files: {}}}",
blob_file_limit_size.get(),
blob_spacemap_type.get(),
blob_cached_fd_size.get(),
blob_heavy_gc_valid_rate.get(),
blob_block_alignment_bytes.get(),
wal_roll_size.get(),
wal_max_persisted_log_files.get());
}
};
} // namespace DB
6 changes: 3 additions & 3 deletions dbms/src/Storages/Page/ConfigSettings.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,7 @@

namespace DB
{
void mergeConfigFromSettings(const DB::Settings & settings, PageStorage::Config & config)
void mergeConfigFromSettings(const DB::Settings & settings, PageStorageConfig & config)
{
config.open_file_max_idle_time = settings.dt_open_file_max_idle_seconds;

Expand All @@ -38,9 +38,9 @@ void mergeConfigFromSettings(const DB::Settings & settings, PageStorage::Config
config.blob_heavy_gc_valid_rate = settings.dt_page_gc_threshold;
}

PageStorage::Config getConfigFromSettings(const DB::Settings & settings)
PageStorageConfig getConfigFromSettings(const DB::Settings & settings)
{
PageStorage::Config config;
PageStorageConfig config;
mergeConfigFromSettings(settings, config);
return config;
}
Expand Down
4 changes: 2 additions & 2 deletions dbms/src/Storages/Page/ConfigSettings.h
Original file line number Diff line number Diff line change
Expand Up @@ -20,8 +20,8 @@ namespace DB
{
struct Settings;

void mergeConfigFromSettings(const DB::Settings & settings, PageStorage::Config & config);
void mergeConfigFromSettings(const DB::Settings & settings, PageStorageConfig & config);

PageStorage::Config getConfigFromSettings(const DB::Settings & settings);
PageStorageConfig getConfigFromSettings(const DB::Settings & settings);

} // namespace DB
42 changes: 26 additions & 16 deletions dbms/src/Storages/Page/Page.h
Original file line number Diff line number Diff line change
Expand Up @@ -32,38 +32,48 @@ inline MemHolder createMemHolder(char * memory, const std::function<void(char *)
return std::shared_ptr<char>(memory, free);
}

struct FieldOffsetInsidePage
{
size_t index;
size_t offset;

FieldOffsetInsidePage(size_t index_) // NOLINT(google-explicit-constructor)
: index(index_)
, offset(0)
{}
FieldOffsetInsidePage(size_t index_, size_t offset_)
: index(index_)
, offset(offset_)
{}

bool operator<(const FieldOffsetInsidePage & rhs) const { return index < rhs.index; }
};

struct Page
{
public:
struct FieldOffset
// only take the low u64, ignoring the high u64(NamespaceId)
explicit Page(const PageIdV3Internal & page_id_v3_)
: page_id(page_id_v3_.low)
{
size_t index;
size_t offset;

FieldOffset(size_t index_) // NOLINT(google-explicit-constructor)
: index(index_)
, offset(0)
{}
FieldOffset(size_t index_, size_t offset_)
: index(index_)
, offset(offset_)
{}
}

bool operator<(const FieldOffset & rhs) const { return index < rhs.index; }
};
Page()
: page_id(INVALID_PAGE_ID)
{}

PageId page_id;
ByteBuffer data;
MemHolder mem_holder;
// Field offsets inside this page.
std::set<FieldOffset> field_offsets;
std::set<FieldOffsetInsidePage> field_offsets;

public:
inline bool isValid() const { return page_id != INVALID_PAGE_ID; }

ByteBuffer getFieldData(size_t index) const
{
auto iter = field_offsets.find(FieldOffset(index));
auto iter = field_offsets.find(FieldOffsetInsidePage(index));
if (unlikely(iter == field_offsets.end()))
throw Exception(fmt::format("Try to getFieldData with invalid field index [page_id={}] [field_index={}]", page_id, index),
ErrorCodes::LOGICAL_ERROR);
Expand Down
6 changes: 6 additions & 0 deletions dbms/src/Storages/Page/PageDefines.h
Original file line number Diff line number Diff line change
Expand Up @@ -61,6 +61,12 @@ static constexpr PageId INVALID_PAGE_ID = 0;
using PageIdV3Internal = UInt128;
using PageIdV3Internals = std::vector<PageIdV3Internal>;

inline PageIdV3Internal buildV3Id(NamespaceId n_id, PageId p_id)
{
// low bits first
return PageIdV3Internal(p_id, n_id);
}

using PageFieldOffset = UInt64;
using PageFieldOffsets = std::vector<PageFieldOffset>;
using PageFieldSizes = std::vector<UInt64>;
Expand Down
14 changes: 7 additions & 7 deletions dbms/src/Storages/Page/PageStorage.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,7 @@ namespace DB
PageStoragePtr PageStorage::create(
String name,
PSDiskDelegatorPtr delegator,
const PageStorage::Config & config,
const PageStorageConfig & config,
const FileProviderPtr & file_provider,
bool use_v3,
bool no_more_insert_to_v2)
Expand Down Expand Up @@ -501,19 +501,19 @@ void PageWriter::writeIntoMixMode(WriteBatch && write_batch, WriteLimiterPtr wri
switch (write.type)
{
// PUT/PUT_EXTERNAL only for V3
case WriteBatch::WriteType::PUT:
case WriteBatch::WriteType::PUT_EXTERNAL:
case WriteBatchWriteType::PUT:
case WriteBatchWriteType::PUT_EXTERNAL:
{
page_ids_before_ref.insert(write.page_id);
break;
}
// Both need del in v2 and v3
case WriteBatch::WriteType::DEL:
case WriteBatchWriteType::DEL:
{
wb_for_v2.copyWrite(write);
break;
}
case WriteBatch::WriteType::REF:
case WriteBatchWriteType::REF:
{
// 1. Try to resolve normal page id
PageId resolved_page_id = storage_v3->getNormalPageId(ns_id,
Expand Down Expand Up @@ -620,7 +620,7 @@ void PageWriter::writeIntoMixMode(WriteBatch && write_batch, WriteLimiterPtr wri
}


PageStorage::Config PageWriter::getSettings() const
PageStorageConfig PageWriter::getSettings() const
{
switch (run_mode)
{
Expand All @@ -641,7 +641,7 @@ PageStorage::Config PageWriter::getSettings() const
}
}

void PageWriter::reloadSettings(const PageStorage::Config & new_config) const
void PageWriter::reloadSettings(const PageStorageConfig & new_config) const
{
switch (run_mode)
{
Expand Down
Loading

0 comments on commit 2c210e5

Please sign in to comment.