Skip to content

Commit

Permalink
Refine some class/function names
Browse files Browse the repository at this point in the history
  • Loading branch information
JaySon-Huang committed Nov 3, 2020
1 parent 9d8148e commit 02448c7
Show file tree
Hide file tree
Showing 15 changed files with 98 additions and 82 deletions.
6 changes: 3 additions & 3 deletions dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -197,7 +197,7 @@ void DeltaMergeStore::setUpBackgroundTask(const DMContextPtr & dm_context)
{
auto dmfile_scanner = [=]() {
PageStorage::PathAndIdsVec path_and_ids_vec;
auto delegate = path_pool.getStableDelegate();
auto delegate = path_pool.getStableDiskDelegator();
for (auto & root_path : delegate.listPaths())
{
auto & path_and_ids = path_and_ids_vec.emplace_back();
Expand All @@ -209,7 +209,7 @@ void DeltaMergeStore::setUpBackgroundTask(const DMContextPtr & dm_context)
return path_and_ids_vec;
};
auto dmfile_remover = [&](const PageStorage::PathAndIdsVec & path_and_ids_vec, const std::set<PageId> & valid_ids) {
auto delegate = path_pool.getStableDelegate();
auto delegate = path_pool.getStableDiskDelegator();
for (auto & [path, ids] : path_and_ids_vec)
{
for (auto id : ids)
Expand Down Expand Up @@ -1496,7 +1496,7 @@ void DeltaMergeStore::restoreStableFiles()
{
LOG_DEBUG(log, "Loading dt files");

auto path_delegate = path_pool.getStableDelegate();
auto path_delegate = path_pool.getStableDiskDelegator();
for (const auto & root_path : path_delegate.listPaths())
{
for (auto & file_id : DMFile::listAllInPath(global_context.getFileProvider(), root_path, false))
Expand Down
4 changes: 2 additions & 2 deletions dbms/src/Storages/DeltaMerge/Segment.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -111,7 +111,7 @@ DMFilePtr writeIntoNewDMFile(DMContext & dm_context, //

StableValueSpacePtr createNewStable(DMContext & context, const BlockInputStreamPtr & input_stream, PageId stable_id, WriteBatches & wbs)
{
auto delegate = context.path_pool.getStableDelegate();
auto delegate = context.path_pool.getStableDiskDelegator();
auto store_path = delegate.choosePath();

PageId dmfile_id = context.storage_pool.newDataPageId();
Expand Down Expand Up @@ -716,7 +716,7 @@ Segment::prepareSplitLogical(DMContext & dm_context, const SegmentSnapshotPtr &
DMFiles my_stable_files;
DMFiles other_stable_files;

auto delegate = dm_context.path_pool.getStableDelegate();
auto delegate = dm_context.path_pool.getStableDiskDelegator();
for (auto & dmfile : segment_snap->stable->getDMFiles())
{
auto ori_ref_id = dmfile->refId();
Expand Down
2 changes: 1 addition & 1 deletion dbms/src/Storages/DeltaMerge/StableValueSpace.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -78,7 +78,7 @@ StableValueSpacePtr StableValueSpace::restore(DMContext & context, PageId id)
readIntBinary(ref_id, buf);

auto file_id = context.storage_pool.data().getNormalPageId(ref_id);
auto file_parent_path = context.path_pool.getStableDelegate().getDTFilePath(file_id);
auto file_parent_path = context.path_pool.getStableDiskDelegator().getDTFilePath(file_id);

auto dmfile = DMFile::restore(context.db_context.getFileProvider(), file_id, ref_id, file_parent_path);
stable->files.push_back(dmfile);
Expand Down
6 changes: 3 additions & 3 deletions dbms/src/Storages/DeltaMerge/StoragePool.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -35,19 +35,19 @@ PageStorage::Config extractConfig(const Settings & settings, UInt64 subtype)
StoragePool::StoragePool(const String & name, StoragePathPool & path_pool, const Context & global_ctx, const Settings & settings)
: // The iops and bandwidth in log_storage are relatively high, use multi-disks if possible
log_storage(name + ".log",
path_pool.getMultiDiskDelegate("log"),
path_pool.getPSDiskDelegatorMulti("log"),
extractConfig(settings, STORAGE_LOG),
global_ctx.getFileProvider(),
global_ctx.getTiFlashMetrics()),
// The iops in data_storage is low, only use the first disk for storing data
data_storage(name + ".data",
path_pool.getNormalDiskDelegate("data"),
path_pool.getPSDiskDelegatorSingle("data"),
extractConfig(settings, STORAGE_DATA),
global_ctx.getFileProvider(),
global_ctx.getTiFlashMetrics()),
// The iops in meta_storage is relatively high, use multi-disks if possible
meta_storage(name + ".meta",
path_pool.getMultiDiskDelegate("meta"),
path_pool.getPSDiskDelegatorMulti("meta"),
extractConfig(settings, STORAGE_META),
global_ctx.getFileProvider(),
global_ctx.getTiFlashMetrics()),
Expand Down
35 changes: 25 additions & 10 deletions dbms/src/Storages/Page/PageStorage.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -167,7 +167,7 @@ static inline bool isPageFileSizeFitsWritable(const PageFile & pf, const PageSto
void PageStorage::restore()
{
LOG_INFO(log,
storage_name << " begin to restore data from disk. [path=" << delegator->normalPath()
storage_name << " begin to restore data from disk. [path=" << delegator->defaultPath()
<< "] [num_writers=" << write_files.size() << "]");

/// page_files are in ascending ordered by (file_id, level).
Expand Down Expand Up @@ -293,7 +293,7 @@ void PageStorage::restore()
std::vector<String> store_paths = delegator->listPaths();
for (size_t i = 0; i < write_files.size(); ++i)
{
auto writer = getWriter(write_files[i], store_paths[i % store_paths.size()]);
auto writer = checkAndRenewWriter(write_files[i], /*parent_path_hint=*/store_paths[i % store_paths.size()]);
idle_writers.emplace_back(std::move(writer));
}
#endif
Expand Down Expand Up @@ -347,9 +347,13 @@ PageEntry PageStorage::getEntry(PageId page_id, SnapshotPtr snapshot)
}
}

PageStorage::WriterPtr PageStorage::getWriter( //
// Check whether `page_file` is writable or not, renew `page_file` if need and return its belonging writer.
// - Writable, reuse `old_writer` if it is not a nullptr, otherwise, create a new writer from `page_file`
// - Not writable, renew the `page_file` and its belonging writer.
// The <id,level> of the new `page_file` is <max_id + 1, 0> of all `write_files`
PageStorage::WriterPtr PageStorage::checkAndRenewWriter( //
PageFile & page_file,
const String & pf_parent_path_hint,
const String & parent_path_hint,
PageStorage::WriterPtr && old_writer,
const String & logging_msg)
{
Expand All @@ -360,22 +364,33 @@ PageStorage::WriterPtr PageStorage::getWriter( //
if (is_writable)
{
if (old_writer)
{
// Reuse the old_writer instead of creating a new writer
write_file_writer.swap(old_writer);
}
else
{
// Create a new writer from `page_file`
write_file_writer = page_file.createWriter(config.sync_on_write, false);
}
}
else
{
String pf_parent_path = delegator->normalPath();
/// Create a new PageFile and generate its belonging writer

String pf_parent_path = delegator->defaultPath();
if (old_writer)
{
// If the old_writer is not NULL, use the same parent path to create a new writer.
// So the number of writers on different paths keep unchanged.
pf_parent_path = old_writer->parentPath();
// Reset writer to ensure all data have been flushed.
old_writer.reset();
}
else if (!pf_parent_path_hint.empty())
else if (!parent_path_hint.empty())
{
pf_parent_path = pf_parent_path_hint;
// Check whether caller has defined a hint path
pf_parent_path = parent_path_hint;
}

PageFileIdAndLevel max_writing_id_lvl{0, 0};
Expand Down Expand Up @@ -452,8 +467,8 @@ void PageStorage::write(WriteBatch && wb)
}
}
auto & page_file = write_files[index];
file_to_write
= getWriter(page_file, "", std::move(file_to_write), " PageFile_" + DB::toString(page_file.getFileId()) + "_0 is full,");
file_to_write = checkAndRenewWriter(
page_file, "", std::move(file_to_write), /*logging_msg=*/" PageFile_" + DB::toString(page_file.getFileId()) + "_0 is full,");

idle_writers.emplace_back(std::move(file_to_write));

Expand Down Expand Up @@ -936,7 +951,7 @@ void PageStorage::archivePageFiles(const PageFileSet & page_files)
if (page_files.empty())
return;

const Poco::Path archive_path(delegator->normalPath(), PageStorage::ARCHIVE_SUBDIR);
const Poco::Path archive_path(delegator->defaultPath(), PageStorage::ARCHIVE_SUBDIR);
Poco::File archive_dir(archive_path);
if (!archive_dir.exists())
archive_dir.createDirectory();
Expand Down
8 changes: 4 additions & 4 deletions dbms/src/Storages/Page/PageStorage.h
Original file line number Diff line number Diff line change
Expand Up @@ -156,10 +156,10 @@ class PageStorage
const ListPageFilesOption & option = ListPageFilesOption());

private:
WriterPtr getWriter(PageFile & page_file,
const String & pf_parent_path_hint = "",
WriterPtr && old_writer = nullptr,
const String & logging_msg = "");
WriterPtr checkAndRenewWriter(PageFile & page_file,
const String & parent_path_hint,
WriterPtr && old_writer = nullptr,
const String & logging_msg = "");
ReaderPtr getReader(const PageFileIdAndLevel & file_id_level);

static constexpr const char * ARCHIVE_SUBDIR = "archive";
Expand Down
2 changes: 1 addition & 1 deletion dbms/src/Storages/Page/gc/LegacyCompactor.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -45,7 +45,7 @@ LegacyCompactor::tryCompact( //
// Use the largest id-level in page_files_to_compact as Checkpoint's file
const PageFileIdAndLevel checkpoint_id = page_files_to_compact.rbegin()->fileIdLevel();

const String storage_path = delegator->normalPath();
const String storage_path = delegator->defaultPath();
if (PageFile::isPageFileExist(checkpoint_id, storage_path, file_provider, PageFile::Type::Checkpoint, page_file_log))
{
LOG_WARNING(log,
Expand Down
2 changes: 1 addition & 1 deletion dbms/src/Storages/Page/tests/gtest_data_compactor.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -38,7 +38,7 @@ try
auto & ctx = TiFlashTestEnv::getContext(DB::Settings(), test_paths);
const FileProviderPtr file_provider = ctx.getFileProvider();
auto pool = ctx.getExtraPaths().withTable("test", "t", false);
auto delegate = pool.getMultiDiskDelegate("log");
auto delegate = pool.getPSDiskDelegatorMulti("log");

PageStorage storage("data_compact_test", delegate, config, file_provider);

Expand Down
4 changes: 2 additions & 2 deletions dbms/src/Storages/Page/tests/gtest_legacy_compactor.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -168,7 +168,7 @@ try
auto & ctx = TiFlashTestEnv::getContext();
const FileProviderPtr file_provider = ctx.getFileProvider();
StoragePathPool spool = ctx.getExtraPaths().withTable("test", "t", false);
auto delegator = spool.getNormalDiskDelegate("meta");
auto delegator = spool.getPSDiskDelegatorSingle("meta");
PageStorage storage("compact_test", delegator, PageStorage::Config{}, file_provider);

PageStorage::ListPageFilesOption opt;
Expand All @@ -185,7 +185,7 @@ try

// TODO:
PageFile page_file
= PageFile::openPageFileForRead(7, 0, delegator->normalPath(), file_provider, PageFile::Type::Checkpoint, storage.page_file_log);
= PageFile::openPageFileForRead(7, 0, delegator->defaultPath(), file_provider, PageFile::Type::Checkpoint, storage.page_file_log);
ASSERT_TRUE(page_file.isExist());

PageStorage::MetaMergingQueue mergine_queue;
Expand Down
8 changes: 4 additions & 4 deletions dbms/src/Storages/Page/tests/gtest_page_storage.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -45,7 +45,7 @@ class PageStorage_test : public ::testing::Test
// drop dir if exists
auto & ctx = TiFlashTestEnv::getContext();
path_pool = std::make_unique<StoragePathPool>(ctx.getExtraPaths().withTable("test", "t1", false));
for (const auto & p : path_pool->getNormalDiskDelegate("log")->listPaths())
for (const auto & p : path_pool->getPSDiskDelegatorSingle("log")->listPaths())
{
if (Poco::File file(p); file.exists())
{
Expand All @@ -62,7 +62,7 @@ class PageStorage_test : public ::testing::Test
std::shared_ptr<PageStorage> reopenWithConfig(const PageStorage::Config & config_)
{
auto & ctx = TiFlashTestEnv::getContext();
auto delegator = path_pool->getNormalDiskDelegate("log");
auto delegator = path_pool->getPSDiskDelegatorSingle("log");
auto storage = std::make_shared<PageStorage>("test.t", delegator, config_, file_provider, ctx.getTiFlashMetrics());
storage->restore();
return storage;
Expand Down Expand Up @@ -382,7 +382,7 @@ try
wb.putPage(1, 0, buf, buf_sz);
storage->write(std::move(wb));

auto f = PageFile::openPageFileForRead(1, 0, storage->delegator->normalPath(), file_provider, PageFile::Type::Formal, storage->log);
auto f = PageFile::openPageFileForRead(1, 0, storage->delegator->defaultPath(), file_provider, PageFile::Type::Formal, storage->log);
f.setLegacy();
}

Expand All @@ -393,7 +393,7 @@ try
auto buf = std::make_shared<ReadBufferFromMemory>(c_buff, sizeof(c_buff));
wb.putPage(1, 0, buf, buf_sz);

auto f = PageFile::newPageFile(2, 0, storage->delegator->normalPath(), file_provider, PageFile::Type::Temp, storage->log);
auto f = PageFile::newPageFile(2, 0, storage->delegator->defaultPath(), file_provider, PageFile::Type::Temp, storage->log);
{
auto w = f.createWriter(false, true);

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -94,7 +94,7 @@ try
auto capacity = std::make_shared<PathCapacityMetrics>(all_paths, 0);
StoragePathPool pool = PathPool(all_paths, all_paths, capacity, file_provider).withTable("test", "table", false);

storage = std::make_shared<PageStorage>("test.table", pool.getMultiDiskDelegate("log"), config, file_provider);
storage = std::make_shared<PageStorage>("test.table", pool.getPSDiskDelegatorMulti("log"), config, file_provider);
storage->restore();

const UInt64 tag = 0;
Expand Down Expand Up @@ -132,7 +132,7 @@ try
}

// restore
storage = std::make_shared<PageStorage>("test.t", pool.getMultiDiskDelegate("log"), config, file_provider);
storage = std::make_shared<PageStorage>("test.t", pool.getPSDiskDelegatorMulti("log"), config, file_provider);
storage->restore();

// Read again
Expand Down Expand Up @@ -188,7 +188,7 @@ try
}

// Restore. This ensure last write is correct.
storage = std::make_shared<PageStorage>("test.t", pool.getMultiDiskDelegate("log"), config, file_provider);
storage = std::make_shared<PageStorage>("test.t", pool.getPSDiskDelegatorMulti("log"), config, file_provider);
storage->restore();

// Read again to check all data.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -66,7 +66,7 @@ class PageStorageMultiWriters_test : public ::testing::Test
std::shared_ptr<PageStorage> reopenWithConfig(const PageStorage::Config & config_)
{
auto spool = TiFlashTestEnv::getContext().getExtraPaths().withTable("test", "t", false);
auto delegator = spool.getNormalDiskDelegate("log");
auto delegator = spool.getPSDiskDelegatorSingle("log");
auto storage = std::make_shared<PageStorage>("test.t", delegator, config_, file_provider);
storage->restore();
return storage;
Expand Down
Loading

0 comments on commit 02448c7

Please sign in to comment.