diff --git a/contrib/tiflash-proxy-cmake/CMakeLists.txt b/contrib/tiflash-proxy-cmake/CMakeLists.txt index 0dca13f8b81..e440289019e 100644 --- a/contrib/tiflash-proxy-cmake/CMakeLists.txt +++ b/contrib/tiflash-proxy-cmake/CMakeLists.txt @@ -11,7 +11,6 @@ # 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. -option(ENABLE_UNI_PS_FFI "Enable write proxy data to uni ps" OFF) if (CMAKE_BUILD_TYPE_UC STREQUAL "DEBUG" OR SAN_DEBUG) set(_TIFLASH_PROXY_BUILD_PROFILE "debug") @@ -27,11 +26,6 @@ file(GLOB_RECURSE _TIFLASH_PROXY_SRCS "${_TIFLASH_PROXY_SOURCE_DIR}/*.rs") # Build in the build directory instead of the default source directory set(TIFLASH_RUST_ENV "CARGO_TARGET_DIR=${CMAKE_CURRENT_BINARY_DIR}" ${TIFLASH_RUST_ENV}) -if(ENABLE_UNI_PS_FFI) - set(PROXY_ENABLE_FEATURES "ENABLE_FEATURES=raftstore-proxy/enable-pagestorage") -else() - set(PROXY_ENABLE_FEATURES "") -endif() # use `CFLAGS=-w CXXFLAGS=-w` to inhibit warning messages. if (TIFLASH_LLVM_TOOLCHAIN) @@ -62,12 +56,10 @@ if(TIFLASH_LLVM_TOOLCHAIN AND USE_LIBCXX) endif() message(STATUS "Using rust env for tiflash-proxy: ${TIFLASH_RUST_ENV}") -message(STATUS "Using rust features for tiflash-proxy: ${PROXY_ENABLE_FEATURES}") -message(STATUS "Using rust command for tiflash-proxy: ${_TIFLASH_PROXY_MAKE_COMMAND}") add_custom_command(OUTPUT ${_TIFLASH_PROXY_LIBRARY} COMMENT "Building TiFlash Proxy using ${_TIFLASH_PROXY_BUILD_PROFILE} profile" - COMMAND ${CMAKE_COMMAND} -E env ${TIFLASH_RUST_ENV} ${PROXY_ENABLE_FEATURES} ${_TIFLASH_PROXY_MAKE_COMMAND} + COMMAND ${CMAKE_COMMAND} -E env ${TIFLASH_RUST_ENV} ${_TIFLASH_PROXY_MAKE_COMMAND} VERBATIM USES_TERMINAL WORKING_DIRECTORY ${_TIFLASH_PROXY_SOURCE_DIR} diff --git a/dbms/src/Columns/ColumnString.h b/dbms/src/Columns/ColumnString.h index df40832142e..57eef31700b 100644 --- a/dbms/src/Columns/ColumnString.h +++ b/dbms/src/Columns/ColumnString.h @@ -249,20 +249,11 @@ class ColumnString final : public COWPtrHelper { const size_t string_size = *reinterpret_cast(pos); pos += sizeof(string_size); - - if (likely(collator)) - { - // https://github.com/pingcap/tiflash/pull/6135 - // - Generate empty string column - // - Make size of `offsets` as previous way for func `ColumnString::size()` - offsets.push_back(0); - return pos + string_size; - } + if (likely(collator != nullptr)) + insertData(pos, string_size); else - { insertDataWithTerminatingZero(pos, string_size); - return pos + string_size; - } + return pos + string_size; } void updateHashWithValue(size_t n, SipHash & hash, const TiDB::TiDBCollatorPtr & collator, String & sort_key_container) const override diff --git a/dbms/src/Columns/IColumn.h b/dbms/src/Columns/IColumn.h index 2d3762f40a9..f6a0be13f0f 100644 --- a/dbms/src/Columns/IColumn.h +++ b/dbms/src/Columns/IColumn.h @@ -201,6 +201,8 @@ class IColumn : public COWPtr * 2. The input parameter `collator` does not work well for complex columns(column tuple), * but it is only used by TiDB , which does not support complex columns, so just ignore * the complex column will be ok. + * 3. Even if the restored column will be discarded, deserializeAndInsertFromArena still need to + * insert the data because when spill happens, this column will be used during the merge agg stage. */ virtual const char * deserializeAndInsertFromArena(const char * pos, const TiDB::TiDBCollatorPtr & collator) = 0; const char * deserializeAndInsertFromArena(const char * pos) { return deserializeAndInsertFromArena(pos, nullptr); } diff --git a/dbms/src/Common/ErrorCodes.cpp b/dbms/src/Common/ErrorCodes.cpp index dc96adf651f..df388c4efa3 100644 --- a/dbms/src/Common/ErrorCodes.cpp +++ b/dbms/src/Common/ErrorCodes.cpp @@ -406,7 +406,6 @@ extern const int PAGE_SIZE_NOT_MATCH = 9006; extern const int ILLFORMED_PAGE_NAME = 9007; extern const int ILLFORMAT_RAFT_ROW = 9008; extern const int REGION_DATA_SCHEMA_UPDATED = 9009; -extern const int REGION_EPOCH_NOT_MATCH = 9010; extern const int LOCK_EXCEPTION = 10000; extern const int VERSION_ERROR = 10001; @@ -426,6 +425,8 @@ extern const int PTHREAD_ERROR = 10014; extern const int PS_ENTRY_NOT_EXISTS = 10015; extern const int PS_ENTRY_NO_VALID_VERSION = 10016; extern const int PS_DIR_APPLY_INVALID_STATUS = 10017; +extern const int DISAGG_ESTABLISH_RETRYABLE_ERROR = 10018; +extern const int REGION_LOCKED = 10019; extern const int S3_ERROR = 11000; extern const int CANNOT_SCHEDULE_TASK = 11001; diff --git a/dbms/src/Common/FailPoint.cpp b/dbms/src/Common/FailPoint.cpp index e35f618d9df..f83b1357026 100644 --- a/dbms/src/Common/FailPoint.cpp +++ b/dbms/src/Common/FailPoint.cpp @@ -96,6 +96,7 @@ namespace DB M(exception_in_merged_task_init) \ M(invalid_mpp_version) \ M(force_fail_in_flush_region_data) \ + M(force_use_dmfile_format_v3) \ M(force_set_mocked_s3_object_mtime) diff --git a/dbms/src/Common/StringUtils/StringUtils.h b/dbms/src/Common/StringUtils/StringUtils.h index 689b940a9e5..be1c94ab657 100644 --- a/dbms/src/Common/StringUtils/StringUtils.h +++ b/dbms/src/Common/StringUtils/StringUtils.h @@ -93,6 +93,11 @@ inline bool isUpperAlphaASCII(char c) return (c >= 'A' && c <= 'Z'); } +inline bool isLowerAplhaASCII(char c) +{ + return (c >= 'a' && c <= 'z'); +} + inline bool isAlphaASCII(char c) { return (c >= 'a' && c <= 'z') diff --git a/dbms/src/Common/TiFlashMetrics.h b/dbms/src/Common/TiFlashMetrics.h index aad1bd0abc9..60993d13100 100644 --- a/dbms/src/Common/TiFlashMetrics.h +++ b/dbms/src/Common/TiFlashMetrics.h @@ -107,7 +107,8 @@ namespace DB M(tiflash_schema_version, "Current version of tiflash cached schema", Gauge) \ M(tiflash_schema_applying, "Whether the schema is applying or not (holding lock)", Gauge) \ M(tiflash_schema_apply_count, "Total number of each kinds of apply", Counter, F(type_diff, {"type", "diff"}), \ - F(type_full, {"type", "full"}), F(type_failed, {"type", "failed"})) \ + F(type_full, {"type", "full"}), F(type_failed, {"type", "failed"}), \ + F(type_drop_keyspace, {"type", "drop_keyspace"})) \ M(tiflash_schema_trigger_count, "Total number of each kinds of schema sync trigger", Counter, /**/ \ F(type_timer, {"type", "timer"}), F(type_raft_decode, {"type", "raft_decode"}), F(type_cop_read, {"type", "cop_read"})) \ M(tiflash_schema_internal_ddl_count, "Total number of each kinds of internal ddl operations", Counter, \ diff --git a/dbms/src/Common/UniThreadPool.cpp b/dbms/src/Common/UniThreadPool.cpp index 1056445ac69..96aa0848c71 100644 --- a/dbms/src/Common/UniThreadPool.cpp +++ b/dbms/src/Common/UniThreadPool.cpp @@ -337,7 +337,6 @@ void ThreadPoolImpl::worker(typename std::list::iterator thread_ } } - template class ThreadPoolImpl; template class ThreadPoolImpl>; template class ThreadFromGlobalPoolImpl; @@ -355,6 +354,11 @@ void GlobalThreadPool::initialize(size_t max_threads, size_t max_free_threads, s the_instance.reset(new GlobalThreadPool(max_threads, max_free_threads, queue_size, false /*shutdown_on_exception*/)); } +void GlobalThreadPool::registerFinalizer(std::function fn) +{ + finalize_fns.push_back(fn); +} + GlobalThreadPool & GlobalThreadPool::instance() { if (!the_instance) @@ -369,9 +373,8 @@ GlobalThreadPool & GlobalThreadPool::instance() GlobalThreadPool::~GlobalThreadPool() noexcept { - // We must make sure IOThread is released before GlobalThreadPool runs - // `finalize`. Or the threads in GlobalThreadPool never ends. - IOThreadPool::shutdown(); + for (auto & fn : finalize_fns) + fn(); } } // namespace DB diff --git a/dbms/src/Common/UniThreadPool.h b/dbms/src/Common/UniThreadPool.h index 10c817205e3..8f302dfcb05 100644 --- a/dbms/src/Common/UniThreadPool.h +++ b/dbms/src/Common/UniThreadPool.h @@ -161,9 +161,14 @@ class GlobalThreadPool : public FreeThreadPool : FreeThreadPool(max_threads_, max_free_threads_, queue_size_, shutdown_on_exception_) {} + std::vector> finalize_fns; + public: static void initialize(size_t max_threads = 10000, size_t max_free_threads = 1000, size_t queue_size = 10000); static GlobalThreadPool & instance(); + + void registerFinalizer(std::function); + ~GlobalThreadPool() noexcept; }; @@ -187,29 +192,30 @@ class ThreadFromGlobalPoolImpl : boost::noncopyable /// NOTE: /// - If this will throw an exception, the destructor won't be called /// - this pointer cannot be passed in the lambda, since after detach() it will not be valid - GlobalThreadPool::instance().scheduleOrThrow([state = state, - func = std::forward(func), - args = std::make_tuple(std::forward(args)...)]() mutable /// mutable is needed to destroy capture - { - SCOPE_EXIT( - state->thread_id = std::thread::id(); - state->event.set();); - - state->thread_id = std::this_thread::get_id(); - - /// This moves are needed to destroy function and arguments before exit. - /// It will guarantee that after ThreadFromGlobalPool::join all captured params are destroyed. - auto function = std::move(func); - auto arguments = std::move(args); - - /// Thread status holds raw pointer on query context, thus it always must be destroyed - /// before sending signal that permits to join this thread. - // DB::ThreadStatus thread_status; - std::apply(function, arguments); - }, - 0, // default priority - 0, // default wait_microseconds - propagate_opentelemetry_context); + GlobalThreadPool::instance().scheduleOrThrow( + [state = state, + func = std::forward(func), + args = std::make_tuple(std::forward(args)...)]() mutable /// mutable is needed to destroy capture + { + SCOPE_EXIT( + state->thread_id = std::thread::id(); + state->event.set();); + + state->thread_id = std::this_thread::get_id(); + + /// This moves are needed to destroy function and arguments before exit. + /// It will guarantee that after ThreadFromGlobalPool::join all captured params are destroyed. + auto function = std::move(func); + auto arguments = std::move(args); + + /// Thread status holds raw pointer on query context, thus it always must be destroyed + /// before sending signal that permits to join this thread. + // DB::ThreadStatus thread_status; + std::apply(function, arguments); + }, + 0, // default priority + 0, // default wait_microseconds + propagate_opentelemetry_context); } ThreadFromGlobalPoolImpl(ThreadFromGlobalPoolImpl && rhs) noexcept diff --git a/dbms/src/Core/TiFlashDisaggregatedMode.cpp b/dbms/src/Core/TiFlashDisaggregatedMode.cpp index 2dd17703c7a..856d62e3ca2 100644 --- a/dbms/src/Core/TiFlashDisaggregatedMode.cpp +++ b/dbms/src/Core/TiFlashDisaggregatedMode.cpp @@ -24,7 +24,9 @@ DisaggregatedMode getDisaggregatedMode(const Poco::Util::LayeredConfiguration & if (config.has(config_key)) { std::string mode_str = config.getString(config_key); - RUNTIME_ASSERT(mode_str == DISAGGREGATED_MODE_WRITE || mode_str == DISAGGREGATED_MODE_COMPUTE, + RUNTIME_ASSERT(mode_str == DISAGGREGATED_MODE_WRITE + || mode_str == DISAGGREGATED_MODE_STORAGE // backward compatibility + || mode_str == DISAGGREGATED_MODE_COMPUTE, "Expect disaggregated_mode is {} or {}, got: {}", DISAGGREGATED_MODE_WRITE, DISAGGREGATED_MODE_COMPUTE, @@ -45,7 +47,7 @@ DisaggregatedMode getDisaggregatedMode(const Poco::Util::LayeredConfiguration & bool useAutoScaler(const Poco::Util::LayeredConfiguration & config) { static const std::string autoscaler_config_key = "flash.use_autoscaler"; - bool use_autoscaler = true; + bool use_autoscaler = false; if (config.has(autoscaler_config_key)) use_autoscaler = config.getBool(autoscaler_config_key); return use_autoscaler; diff --git a/dbms/src/Core/TiFlashDisaggregatedMode.h b/dbms/src/Core/TiFlashDisaggregatedMode.h index c79743016a4..5e12ff38cd3 100644 --- a/dbms/src/Core/TiFlashDisaggregatedMode.h +++ b/dbms/src/Core/TiFlashDisaggregatedMode.h @@ -23,6 +23,7 @@ // Note that TiFlash Write Node is also named as TiFlash Storage Node in many places. // To make sure it is consistent to our documents, we better stick with "tiflash_write" in the configurations. #define DISAGGREGATED_MODE_WRITE "tiflash_write" +#define DISAGGREGATED_MODE_STORAGE "tiflash_storage" // backward compatibility just for parsing config #define DISAGGREGATED_MODE_COMPUTE "tiflash_compute" // engine_role determine whether TiFlash use S3 to write. #define DISAGGREGATED_MODE_WRITE_ENGINE_ROLE "write" diff --git a/dbms/src/Debug/MockRaftStoreProxy.cpp b/dbms/src/Debug/MockRaftStoreProxy.cpp index 5263c361521..9558fd556ff 100644 --- a/dbms/src/Debug/MockRaftStoreProxy.cpp +++ b/dbms/src/Debug/MockRaftStoreProxy.cpp @@ -207,6 +207,39 @@ MockProxyRegion::MockProxyRegion(uint64_t id_) state.mutable_region()->set_id(id); } +UniversalWriteBatch MockProxyRegion::persistMeta() +{ + auto _ = genLockGuard(); + auto wb = UniversalWriteBatch(); + + auto region_key = UniversalPageIdFormat::toRegionLocalStateKeyInKVEngine(this->id); + auto region_local_state = this->state.SerializeAsString(); + MemoryWriteBuffer buf(0, region_local_state.size()); + buf.write(region_local_state.data(), region_local_state.size()); + wb.putPage(UniversalPageId(region_key.data(), region_key.size()), 0, buf.tryGetReadBuffer(), region_local_state.size()); + + auto apply_key = UniversalPageIdFormat::toRaftApplyStateKeyInKVEngine(this->id); + auto raft_apply_state = this->apply.SerializeAsString(); + MemoryWriteBuffer buf2(0, raft_apply_state.size()); + buf2.write(raft_apply_state.data(), raft_apply_state.size()); + wb.putPage(UniversalPageId(apply_key.data(), apply_key.size()), 0, buf2.tryGetReadBuffer(), raft_apply_state.size()); + + raft_serverpb::RegionLocalState restored_region_state; + raft_serverpb::RaftApplyState restored_apply_state; + restored_region_state.ParseFromArray(region_local_state.data(), region_local_state.size()); + restored_apply_state.ParseFromArray(raft_apply_state.data(), raft_apply_state.size()); + return wb; +} + +void MockProxyRegion::addPeer(uint64_t store_id, uint64_t peer_id, metapb::PeerRole role) +{ + auto _ = genLockGuard(); + auto & peer = *state.mutable_region()->mutable_peers()->Add(); + peer.set_store_id(store_id); + peer.set_id(peer_id); + peer.set_role(role); +} + std::optional RawMockReadIndexTask::poll(std::shared_ptr waker) { auto _ = genLockGuard(); diff --git a/dbms/src/Debug/MockRaftStoreProxy.h b/dbms/src/Debug/MockRaftStoreProxy.h index 6207e7e8b17..97d76c332bd 100644 --- a/dbms/src/Debug/MockRaftStoreProxy.h +++ b/dbms/src/Debug/MockRaftStoreProxy.h @@ -14,6 +14,7 @@ #pragma once +#include #include #include #include @@ -36,6 +37,9 @@ struct MockProxyRegion : MutexLockWrap void updateCommitIndex(uint64_t index); void setSate(raft_serverpb::RegionLocalState); explicit MockProxyRegion(uint64_t id); + UniversalWriteBatch persistMeta(); + void addPeer(uint64_t store_id, uint64_t peer_id, metapb::PeerRole role); + struct RawWrite { std::vector keys; diff --git a/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.cpp b/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.cpp index d594b0e59e1..07e62424847 100644 --- a/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.cpp +++ b/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.cpp @@ -54,10 +54,6 @@ namespace DB { -namespace ErrorCodes -{ -extern const int REGION_EPOCH_NOT_MATCH; -} // namespace ErrorCodes namespace FailPoints { @@ -305,7 +301,13 @@ void DAGStorageInterpreter::executeImpl(DAGPipeline & pipeline) // and ask RN to send requests again with correct region info. When RN updates region info, // RN may be sending requests to other WN. - throw Exception("Rejected disaggregated DAG execute because RN region info does not match", DB::ErrorCodes::REGION_EPOCH_NOT_MATCH); + RegionException::UnavailableRegions region_ids; + for (const auto & info : context.getDAGContext()->retry_regions) + region_ids.insert(info.region_id); + + throw RegionException( + std::move(region_ids), + RegionException::RegionReadStatus::EPOCH_NOT_MATCH); } // A failpoint to test pause before alter lock released @@ -800,9 +802,8 @@ void DAGStorageInterpreter::buildLocalStreams(DAGPipeline & pipeline, size_t max const auto & snap_id = *dag_context.getDisaggTaskId(); auto timeout_s = context.getSettingsRef().disagg_task_snapshot_timeout; auto expired_at = Clock::now() + std::chrono::seconds(timeout_s); - bool register_snapshot_ok = snaps->registerSnapshot(snap_id, std::move(disaggregated_snap), expired_at); - RUNTIME_CHECK_MSG(register_snapshot_ok, "disaggregated task has been registered {}", snap_id); - LOG_INFO(log, "task snapshot registered, snapshot_id={}", snap_id); + bool register_snapshot_ok = snaps->registerSnapshot(snap_id, disaggregated_snap, expired_at); + RUNTIME_CHECK_MSG(register_snapshot_ok, "Disaggregated task has been registered, snap_id={}", snap_id); } if (has_multiple_partitions) diff --git a/dbms/src/Flash/Disaggregated/MockS3LockClient.h b/dbms/src/Flash/Disaggregated/MockS3LockClient.h index 0aae5f3e866..e6e6c9c998b 100644 --- a/dbms/src/Flash/Disaggregated/MockS3LockClient.h +++ b/dbms/src/Flash/Disaggregated/MockS3LockClient.h @@ -30,13 +30,7 @@ class MockS3LockClient : public IS3LockClient { public: explicit MockS3LockClient(std::shared_ptr c) - : MockS3LockClient(c, c->bucket()) - { - } - - MockS3LockClient(std::shared_ptr c, const String & bucket_) : s3_client(std::move(c)) - , bucket(bucket_) { } @@ -45,16 +39,16 @@ class MockS3LockClient : public IS3LockClient { // If the data file exist and no delmark exist, then create a lock file on `data_file_key` auto view = S3FilenameView::fromKey(data_file_key); - if (!objectExists(*s3_client, bucket, data_file_key)) + if (!objectExists(*s3_client, data_file_key)) { return {false, ""}; } auto delmark_key = view.getDelMarkKey(); - if (objectExists(*s3_client, bucket, delmark_key)) + if (objectExists(*s3_client, delmark_key)) { return {false, ""}; } - uploadEmptyFile(*s3_client, bucket, view.getLockKey(lock_store_id, lock_seq)); + uploadEmptyFile(*s3_client, view.getLockKey(lock_store_id, lock_seq)); return {true, ""}; } @@ -64,23 +58,18 @@ class MockS3LockClient : public IS3LockClient // If there is no lock on the given `data_file_key`, then mark as deleted auto view = S3FilenameView::fromKey(data_file_key); auto lock_prefix = view.getLockPrefix(); - bool any_lock_exist = false; - listPrefix(*s3_client, bucket, lock_prefix, [&any_lock_exist](const Aws::S3::Model::ListObjectsV2Result & result) -> S3::PageResult { - if (!result.GetContents().empty()) - any_lock_exist = true; - return S3::PageResult{.num_keys = result.GetContents().size(), .more = false}; - }); + auto lock_key_opt = S3::anyKeyExistWithPrefix(*s3_client, lock_prefix); + bool any_lock_exist = lock_key_opt.has_value(); if (any_lock_exist) { return {false, ""}; } - uploadEmptyFile(*s3_client, bucket, view.getDelMarkKey()); + uploadEmptyFile(*s3_client, view.getDelMarkKey()); return {true, ""}; } private: - std::shared_ptr s3_client; - String bucket; + std::shared_ptr s3_client; }; } // namespace DB::S3 diff --git a/dbms/src/Flash/Disaggregated/RNPagePreparer.cpp b/dbms/src/Flash/Disaggregated/RNPagePreparer.cpp index e9db1ba5ab4..3934376b0c9 100644 --- a/dbms/src/Flash/Disaggregated/RNPagePreparer.cpp +++ b/dbms/src/Flash/Disaggregated/RNPagePreparer.cpp @@ -17,7 +17,7 @@ #include #include #include -#include +#include #include #include #include @@ -65,7 +65,7 @@ RNPagePreparer::RNPagePreparer( }); persist_threads.emplace_back(task->get_future()); - IOThreadPool::get().scheduleOrThrowOnError([task] { (*task)(); }); + RNPagePreparerPool::get().scheduleOrThrowOnError([task] { (*task)(); }); } } catch (...) diff --git a/dbms/src/Flash/Disaggregated/RNPageReceiver.cpp b/dbms/src/Flash/Disaggregated/RNPageReceiver.cpp index 4739bb4c0e7..2f306c3c0a3 100644 --- a/dbms/src/Flash/Disaggregated/RNPageReceiver.cpp +++ b/dbms/src/Flash/Disaggregated/RNPageReceiver.cpp @@ -268,7 +268,7 @@ void RNPageReceiverBase::readLoop() // Keep popping segment fetch pages request to get the task ready while (!meet_error) { - auto req = rpc_context->popRequest(); + auto req = rpc_context->nextFetchPagesRequest(); if (!req.isValid()) break; try diff --git a/dbms/src/Flash/Disaggregated/RNPageReceiverContext.cpp b/dbms/src/Flash/Disaggregated/RNPageReceiverContext.cpp index eebbc5d2ef3..2cfb086631d 100644 --- a/dbms/src/Flash/Disaggregated/RNPageReceiverContext.cpp +++ b/dbms/src/Flash/Disaggregated/RNPageReceiverContext.cpp @@ -147,7 +147,7 @@ const String & FetchPagesRequest::address() const return seg_task->address; } -FetchPagesRequest GRPCPagesReceiverContext::popRequest() const +FetchPagesRequest GRPCPagesReceiverContext::nextFetchPagesRequest() const { auto seg_task = remote_read_tasks->nextFetchTask(); return FetchPagesRequest(std::move(seg_task)); diff --git a/dbms/src/Flash/Disaggregated/RNPageReceiverContext.h b/dbms/src/Flash/Disaggregated/RNPageReceiverContext.h index cb80b5a2566..03649b7126a 100644 --- a/dbms/src/Flash/Disaggregated/RNPageReceiverContext.h +++ b/dbms/src/Flash/Disaggregated/RNPageReceiverContext.h @@ -69,7 +69,7 @@ class GRPCPagesReceiverContext const DM::RNRemoteReadTaskPtr & remote_read_tasks, pingcap::kv::Cluster * cluster_); - FetchPagesRequest popRequest() const; + FetchPagesRequest nextFetchPagesRequest() const; FetchPagesResponseReaderPtr doRequest(const FetchPagesRequest & request) const; diff --git a/dbms/src/Flash/Disaggregated/S3LockService.cpp b/dbms/src/Flash/Disaggregated/S3LockService.cpp index e400d9995d7..f022580f5df 100644 --- a/dbms/src/Flash/Disaggregated/S3LockService.cpp +++ b/dbms/src/Flash/Disaggregated/S3LockService.cpp @@ -17,6 +17,7 @@ #include #include #include +#include #include #include #include @@ -39,18 +40,14 @@ extern const Metric S3LockServiceNumLatches; namespace DB::S3 { - - S3LockService::S3LockService(Context & context_) : S3LockService( - context_.getGlobalContext().getTMTContext().getS3GCOwnerManager(), - S3::ClientFactory::instance().sharedTiFlashClient()) + context_.getGlobalContext().getTMTContext().getS3GCOwnerManager()) { } -S3LockService::S3LockService(OwnerManagerPtr owner_mgr_, std::shared_ptr s3_cli_) +S3LockService::S3LockService(OwnerManagerPtr owner_mgr_) : gc_owner(std::move(owner_mgr_)) - , s3_client(std::move(s3_cli_)) , log(Logger::get()) { } @@ -200,8 +197,10 @@ bool S3LockService::tryAddLockImpl( } }); + auto s3_client = S3::ClientFactory::instance().sharedTiFlashClient(); // make sure data file exists - if (!DB::S3::objectExists(*s3_client, s3_client->bucket(), data_file_key)) + auto object_key = key_view.isDMFile() ? fmt::format("{}/{}", data_file_key, DM::DMFile::metav2FileName()) : data_file_key; + if (!DB::S3::objectExists(*s3_client, object_key)) { auto * e = response->mutable_result()->mutable_conflict(); e->set_reason(fmt::format("data file not exist, key={}", data_file_key)); @@ -211,7 +210,7 @@ bool S3LockService::tryAddLockImpl( // make sure data file is not mark as deleted const auto delmark_key = key_view.getDelMarkKey(); - if (DB::S3::objectExists(*s3_client, s3_client->bucket(), delmark_key)) + if (DB::S3::objectExists(*s3_client, delmark_key)) { auto * e = response->mutable_result()->mutable_conflict(); e->set_reason(fmt::format("data file is mark deleted, key={} delmark={}", data_file_key, delmark_key)); @@ -229,7 +228,7 @@ bool S3LockService::tryAddLockImpl( } const auto lock_key = key_view.getLockKey(lock_store_id, lock_seq); // upload lock file - DB::S3::uploadEmptyFile(*s3_client, s3_client->bucket(), lock_key); + DB::S3::uploadEmptyFile(*s3_client, lock_key); if (!gc_owner->isOwner()) { // although the owner is changed after lock file is uploaded, but @@ -245,27 +244,6 @@ bool S3LockService::tryAddLockImpl( return true; } -std::optional S3LockService::anyLockExist(const String & lock_prefix) const -{ - std::optional lock_key; - DB::S3::listPrefix( - *s3_client, - s3_client->bucket(), - lock_prefix, - [&lock_key](const Aws::S3::Model::ListObjectsV2Result & result) -> S3::PageResult { - const auto & contents = result.GetContents(); - if (!contents.empty()) - { - lock_key = contents.front().GetKey(); - } - return S3::PageResult{ - .num_keys = contents.size(), - .more = false, // do not need more result - }; - }); - return lock_key; -} - bool S3LockService::tryMarkDeleteImpl(const String & data_file_key, disaggregated::TryMarkDeleteResponse * response) { const S3FilenameView key_view = S3FilenameView::fromKey(data_file_key); @@ -294,7 +272,8 @@ bool S3LockService::tryMarkDeleteImpl(const String & data_file_key, disaggregate // make sure data file has not been locked const auto lock_prefix = key_view.getLockPrefix(); - std::optional lock_key = anyLockExist(lock_prefix); + auto s3_client = S3::ClientFactory::instance().sharedTiFlashClient(); + std::optional lock_key = S3::anyKeyExistWithPrefix(*s3_client, lock_prefix); if (lock_key) { auto * e = response->mutable_result()->mutable_conflict(); @@ -318,7 +297,7 @@ bool S3LockService::tryMarkDeleteImpl(const String & data_file_key, disaggregate { tagging = TaggingObjectIsDeleted; } - DB::S3::uploadEmptyFile(*s3_client, s3_client->bucket(), delmark_key, tagging); + DB::S3::uploadEmptyFile(*s3_client, delmark_key, tagging); if (!gc_owner->isOwner()) { // owner changed happens when delmark is uploading, can not diff --git a/dbms/src/Flash/Disaggregated/S3LockService.h b/dbms/src/Flash/Disaggregated/S3LockService.h index 71f8b368cce..ac4196a57bd 100644 --- a/dbms/src/Flash/Disaggregated/S3LockService.h +++ b/dbms/src/Flash/Disaggregated/S3LockService.h @@ -54,7 +54,7 @@ class S3LockService final : private boost::noncopyable public: explicit S3LockService(Context & context_); - S3LockService(OwnerManagerPtr owner_mgr_, std::shared_ptr s3_cli_); + explicit S3LockService(OwnerManagerPtr owner_mgr_); ~S3LockService() = default; @@ -100,14 +100,11 @@ class S3LockService final : private boost::noncopyable DataFileMutexPtr getDataFileLatch(const String & data_file_key); - std::optional anyLockExist(const String & lock_prefix) const; - private: std::unordered_map file_latch_map; std::mutex file_latch_map_mutex; OwnerManagerPtr gc_owner; - const std::shared_ptr s3_client; LoggerPtr log; }; diff --git a/dbms/src/Flash/Disaggregated/WNEstablishDisaggTaskHandler.cpp b/dbms/src/Flash/Disaggregated/WNEstablishDisaggTaskHandler.cpp index eb0ac531740..5d1670bc230 100644 --- a/dbms/src/Flash/Disaggregated/WNEstablishDisaggTaskHandler.cpp +++ b/dbms/src/Flash/Disaggregated/WNEstablishDisaggTaskHandler.cpp @@ -31,11 +31,6 @@ namespace DB { -namespace ErrorCodes -{ -extern const int REGION_EPOCH_NOT_MATCH; -} // namespace ErrorCodes - WNEstablishDisaggTaskHandler::WNEstablishDisaggTaskHandler(ContextPtr context_, const DM::DisaggTaskId & task_id) : context(std::move(context_)) , log(Logger::get(task_id)) @@ -61,7 +56,9 @@ void WNEstablishDisaggTaskHandler::prepare(const disaggregated::EstablishDisaggT context->setSetting("read_tso", start_ts); if (request->timeout_s() < 0) + { throw TiFlashException(Errors::Coprocessor::BadRequest, "invalid timeout={}", request->timeout_s()); + } else if (request->timeout_s() > 0) { context->setSetting("disagg_task_snapshot_timeout", request->timeout_s()); @@ -93,9 +90,9 @@ void WNEstablishDisaggTaskHandler::execute(disaggregated::EstablishDisaggTaskRes response->set_store_id(kvstore->getStoreID()); } - auto snapshots = context->getSharedContextDisagg()->wn_snapshot_manager; + auto snaps = context->getSharedContextDisagg()->wn_snapshot_manager; const auto & task_id = *dag_context->getDisaggTaskId(); - auto snap = snapshots->getSnapshot(task_id); + auto snap = snaps->getSnapshot(task_id); RUNTIME_CHECK_MSG(snap, "Snapshot was missing, task_id={}", task_id); { diff --git a/dbms/src/Flash/Disaggregated/WNFetchPagesStreamWriter.h b/dbms/src/Flash/Disaggregated/WNFetchPagesStreamWriter.h index 260376bdf43..3e37383aab5 100644 --- a/dbms/src/Flash/Disaggregated/WNFetchPagesStreamWriter.h +++ b/dbms/src/Flash/Disaggregated/WNFetchPagesStreamWriter.h @@ -30,7 +30,6 @@ namespace DB { - using SyncPagePacketWriter = grpc::ServerWriter; class WNFetchPagesStreamWriter; diff --git a/dbms/src/Flash/Disaggregated/tests/gtest_s3_lock_service.cpp b/dbms/src/Flash/Disaggregated/tests/gtest_s3_lock_service.cpp index f84a5af44f7..12dbf7e2bee 100644 --- a/dbms/src/Flash/Disaggregated/tests/gtest_s3_lock_service.cpp +++ b/dbms/src/Flash/Disaggregated/tests/gtest_s3_lock_service.cpp @@ -15,6 +15,7 @@ #include #include #include +#include #include #include #include @@ -52,16 +53,9 @@ class S3LockServiceTest owner_manager = std::static_pointer_cast(OwnerManager::createMockOwner("owner_0")); owner_manager->campaignOwner(); - if (is_s3_test_enabled) - { - s3_client = client_factory.sharedTiFlashClient(); - } - else - { - s3_client = std::make_shared(); - } - s3_lock_service = std::make_unique(owner_manager, s3_client); - ::DB::tests::TiFlashTestEnv::createBucketIfNotExist(*s3_client, s3_client->bucket()); + s3_client = client_factory.sharedTiFlashClient(); + s3_lock_service = std::make_unique(owner_manager); + ::DB::tests::TiFlashTestEnv::createBucketIfNotExist(*s3_client); createS3DataFiles(); } CATCH @@ -72,7 +66,7 @@ class S3LockServiceTest for (size_t i = 1; i <= 5; ++i) { auto data_filename = S3Filename::fromDMFileOID(DMFileOID{.store_id = store_id, .table_id = physical_table_id, .file_id = dm_file_id}); - DB::S3::uploadEmptyFile(*s3_client, s3_client->bucket(), data_filename.toFullKey()); + DB::S3::uploadEmptyFile(*s3_client, fmt::format("{}/{}", data_filename.toFullKey(), DM::DMFile::metav2FileName())); ++dm_file_id; } } @@ -84,7 +78,7 @@ class S3LockServiceTest { --dm_file_id; auto data_filename = S3Filename::fromDMFileOID(DMFileOID{.store_id = store_id, .table_id = physical_table_id, .file_id = dm_file_id}); - DB::S3::deleteObject(*s3_client, s3_client->bucket(), data_filename.toFullKey()); + DB::S3::deleteObject(*s3_client, data_filename.toFullKey()); } } @@ -138,9 +132,9 @@ try ASSERT_TRUE(status_code.ok()) << status_code.error_message(); ASSERT_TRUE(response.result().has_success()) << response.ShortDebugString(); - ASSERT_TRUE(DB::S3::objectExists(*s3_client, s3_client->bucket(), lock_key)); + ASSERT_TRUE(DB::S3::objectExists(*s3_client, lock_key)); - DB::S3::deleteObject(*s3_client, s3_client->bucket(), lock_key); + DB::S3::deleteObject(*s3_client, lock_key); } CATCH @@ -159,9 +153,9 @@ try ASSERT_TRUE(status_code.ok()) << status_code.error_message(); ASSERT_TRUE(response.result().has_success()) << response.ShortDebugString(); - ASSERT_TRUE(DB::S3::objectExists(*s3_client, s3_client->bucket(), delmark_key)); + ASSERT_TRUE(DB::S3::objectExists(*s3_client, delmark_key)); - DB::S3::deleteObject(*s3_client, s3_client->bucket(), delmark_key); + DB::S3::deleteObject(*s3_client, delmark_key); } CATCH @@ -181,7 +175,7 @@ try auto status_code = s3_lock_service->tryMarkDelete(&request, &response); ASSERT_TRUE(status_code.ok()) << status_code.error_message(); - ASSERT_TRUE(DB::S3::objectExists(*s3_client, s3_client->bucket(), delmark_key)); + ASSERT_TRUE(DB::S3::objectExists(*s3_client, delmark_key)); } // Try add lock file, should fail @@ -196,10 +190,10 @@ try ASSERT_TRUE(status_code.ok()); ASSERT_TRUE(!response.result().has_success()) << response.ShortDebugString(); ASSERT_TRUE(response.result().has_conflict()) << response.ShortDebugString(); - ASSERT_TRUE(!DB::S3::objectExists(*s3_client, s3_client->bucket(), lock_key)); + ASSERT_TRUE(!DB::S3::objectExists(*s3_client, lock_key)); } - DB::S3::deleteObject(*s3_client, s3_client->bucket(), delmark_key); + DB::S3::deleteObject(*s3_client, delmark_key); } CATCH @@ -222,7 +216,7 @@ try auto status_code = s3_lock_service->tryAddLock(&request, &response); ASSERT_TRUE(status_code.ok()); - ASSERT_TRUE(DB::S3::objectExists(*s3_client, s3_client->bucket(), lock_key)); + ASSERT_TRUE(DB::S3::objectExists(*s3_client, lock_key)); } // Try add delete mark, should fail @@ -236,10 +230,10 @@ try ASSERT_TRUE(status_code.ok()); ASSERT_TRUE(!response.result().has_success()) << response.ShortDebugString(); ASSERT_TRUE(response.result().has_conflict()) << response.ShortDebugString(); - ASSERT_TRUE(!DB::S3::objectExists(*s3_client, s3_client->bucket(), delmark_key)); + ASSERT_TRUE(!DB::S3::objectExists(*s3_client, delmark_key)); } - DB::S3::deleteObject(*s3_client, s3_client->bucket(), lock_key); + DB::S3::deleteObject(*s3_client, lock_key); } CATCH @@ -263,7 +257,7 @@ try ASSERT_TRUE(status_code.ok()); ASSERT_TRUE(!response.result().has_success()) << response.ShortDebugString(); ASSERT_TRUE(response.result().has_conflict()) << response.ShortDebugString(); - ASSERT_TRUE(!DB::S3::objectExists(*s3_client, s3_client->bucket(), lock_key)); + ASSERT_TRUE(!DB::S3::objectExists(*s3_client, lock_key)); } } CATCH @@ -288,9 +282,9 @@ try ASSERT_TRUE(status_code.ok()); ASSERT_TRUE(response.result().has_success()) << response.ShortDebugString(); - ASSERT_TRUE(DB::S3::objectExists(*s3_client, s3_client->bucket(), lock_key)); + ASSERT_TRUE(DB::S3::objectExists(*s3_client, lock_key)); - DB::S3::deleteObject(*s3_client, s3_client->bucket(), lock_key); + DB::S3::deleteObject(*s3_client, lock_key); } }; @@ -322,7 +316,7 @@ try auto status_code = s3_lock_service->tryMarkDelete(&request, &response); ASSERT_TRUE(status_code.ok()); - ASSERT_TRUE(DB::S3::objectExists(*s3_client, s3_client->bucket(), delmark_key)); + ASSERT_TRUE(DB::S3::objectExists(*s3_client, delmark_key)); }; std::vector threads; @@ -337,7 +331,7 @@ try thread.join(); } - DB::S3::deleteObject(*s3_client, s3_client->bucket(), delmark_key); + DB::S3::deleteObject(*s3_client, delmark_key); } CATCH @@ -396,13 +390,13 @@ try auto delmark_key = data_filename.toView().getDelMarkKey(); // Either lock or delete file should exist - if (DB::S3::objectExists(*s3_client, s3_client->bucket(), delmark_key)) + if (DB::S3::objectExists(*s3_client, delmark_key)) { - DB::S3::deleteObject(*s3_client, s3_client->bucket(), delmark_key); + DB::S3::deleteObject(*s3_client, delmark_key); } - else if (DB::S3::objectExists(*s3_client, s3_client->bucket(), lock_key)) + else if (DB::S3::objectExists(*s3_client, lock_key)) { - DB::S3::deleteObject(*s3_client, s3_client->bucket(), lock_key); + DB::S3::deleteObject(*s3_client, lock_key); } else { diff --git a/dbms/src/Flash/FlashService.cpp b/dbms/src/Flash/FlashService.cpp index 8f4b2b54ca8..7f972dd924b 100644 --- a/dbms/src/Flash/FlashService.cpp +++ b/dbms/src/Flash/FlashService.cpp @@ -39,6 +39,8 @@ #include #include #include +#include +#include #include #include #include @@ -51,6 +53,7 @@ namespace ErrorCodes { extern const int NOT_IMPLEMENTED; extern const int UNKNOWN_EXCEPTION; +extern const int DISAGG_ESTABLISH_RETRYABLE_ERROR; } // namespace ErrorCodes #define CATCH_FLASHSERVICE_EXCEPTION \ @@ -623,12 +626,14 @@ grpc::Status FlashService::EstablishDisaggTask(grpc::ServerContext * grpc_contex current_memory_tracker = nullptr; }); - grpc::Status ret_status = grpc::Status::OK; - auto record_error = [&](grpc::StatusCode err_code, int flash_err_code, const String & err_msg) { + auto record_error = [&](int flash_err_code, const String & err_msg) { + // Note: We intentinally do not remove the snapshot from the SnapshotManager + // when this request is failed. Consider this case: + // EstablishDisagg for A: ---------------- Failed --------------------------------------------- Cleanup Snapshot for A + // EstablishDisagg for B: - Failed - RN retry EstablishDisagg for A+B -- InsertSnapshot for A+B ----- FetchPages (Boom!) auto * err = response->mutable_error(); err->set_code(flash_err_code); err->set_msg(err_msg); - ret_status = grpc::Status(err_code, err_msg); }; try @@ -636,42 +641,63 @@ grpc::Status FlashService::EstablishDisaggTask(grpc::ServerContext * grpc_contex handler->prepare(request); handler->execute(response); } + catch (const RegionException & e) + { + for (const auto & region_id : e.unavailable_region) + { + auto * retry_region = response->add_retry_regions(); + retry_region->set_id(region_id); + // Note: retry_region's version and epoch is not set, because we miss these information + // from the exception. + } + LOG_INFO(logger, "EstablishDisaggTask meet RegionException {} (retryable), regions={}", e.message(), e.unavailable_region); + record_error( + ErrorCodes::DISAGG_ESTABLISH_RETRYABLE_ERROR, + fmt::format("Retryable error: {}", e.message())); + } + catch (const LockException & e) + { + auto * retry_region = response->add_retry_regions(); + retry_region->set_id(e.region_id); + // Note: retry_region's version and epoch is not set, because we miss these information + // from the exception. + + LOG_INFO(logger, "EstablishDisaggTask meet LockException (retryable), region_id={}", e.region_id); + // TODO: We may need to send this error back to TiDB? Otherwise TiDB + // may not resolve lock in time. + record_error( + ErrorCodes::DISAGG_ESTABLISH_RETRYABLE_ERROR, + fmt::format("Retryable error: {}", e.message())); + } catch (Exception & e) { - LOG_ERROR(logger, "EstablishDisaggTask meet Exception: {}\n{}", e.displayText(), e.getStackTrace().toString()); - record_error(grpc::StatusCode::INTERNAL, e.code(), e.message()); + LOG_ERROR(logger, "EstablishDisaggTask meet exception: {}\n{}", e.displayText(), e.getStackTrace().toString()); + record_error(e.code(), e.message()); } catch (const pingcap::Exception & e) { LOG_ERROR(logger, "EstablishDisaggTask meet KV Client Exception: {}", e.message()); - record_error(grpc::StatusCode::INTERNAL, ErrorCodes::UNKNOWN_EXCEPTION, e.message()); + record_error(e.code(), e.message()); } catch (std::exception & e) { LOG_ERROR(logger, "EstablishDisaggTask meet std::exception: {}", e.what()); - record_error(grpc::StatusCode::INTERNAL, ErrorCodes::UNKNOWN_EXCEPTION, e.what()); + record_error(ErrorCodes::UNKNOWN_EXCEPTION, e.what()); } catch (...) { LOG_ERROR(logger, "EstablishDisaggTask meet unknown exception"); - record_error(grpc::StatusCode::INTERNAL, ErrorCodes::UNKNOWN_EXCEPTION, "other exception"); + record_error(ErrorCodes::UNKNOWN_EXCEPTION, "other exception"); } - // When there is a region retry, exceptions will be thrown, so we placed here as a "finally". - if (auto * dag_ctx = db_context->getDAGContext(); dag_ctx) - { - // There may be region errors. Add information about which region to retry. - for (const auto & region : dag_ctx->retry_regions) - { - auto * retry_region = response->add_retry_regions(); - retry_region->set_id(region.region_id); - retry_region->mutable_region_epoch()->set_conf_ver(region.region_conf_version); - retry_region->mutable_region_epoch()->set_version(region.region_version); - } - } + LOG_DEBUG( + logger, + "Handle EstablishDisaggTask request done, resp_err={}", + response->has_error() ? response->error().ShortDebugString() : "(null)"); - LOG_DEBUG(logger, "Handle EstablishDisaggTask request done, resp_err={}", response->error().ShortDebugString()); - return ret_status; + // The response is send back to TiFlash. Always assign gRPC::Status::OK, so that TiFlash response + // handlers can deal with the embedded error correctly. + return grpc::Status::OK; } grpc::Status FlashService::FetchDisaggPages( @@ -732,7 +758,7 @@ grpc::Status FlashService::FetchDisaggPages( } catch (const Exception & e) { - LOG_ERROR(logger, "FetchDisaggPages meet Exception: {}\n{}", e.message(), e.getStackTrace().toString()); + LOG_ERROR(logger, "FetchDisaggPages meet exception: {}\n{}", e.message(), e.getStackTrace().toString()); return record_error(tiflashErrorCodeToGrpcStatusCode(e.code()), e.message()); } catch (const pingcap::Exception & e) diff --git a/dbms/src/Flash/tests/gtest_aggregation_executor.cpp b/dbms/src/Flash/tests/gtest_aggregation_executor.cpp index 1a04717e590..898716b7dfa 100644 --- a/dbms/src/Flash/tests/gtest_aggregation_executor.cpp +++ b/dbms/src/Flash/tests/gtest_aggregation_executor.cpp @@ -12,7 +12,9 @@ // See the License for the specific language governing permissions and // limitations under the License. +#include #include +#include #include #include @@ -20,7 +22,6 @@ namespace DB { namespace tests { - #define DT DecimalField #define COL_GROUP2(a, b) \ { \ @@ -610,6 +611,125 @@ try } CATCH +TEST_F(AggExecutorTestRunner, SplitAggOutputWithSpecialGroupKey) +try +{ + /// prepare data + size_t unique_rows = 3000; + DB::MockColumnInfoVec table_column_infos{{"key_8", TiDB::TP::TypeTiny, false}, {"key_16", TiDB::TP::TypeShort, false}, {"key_32", TiDB::TP::TypeLong, false}, {"key_64", TiDB::TP::TypeLongLong, false}, {"key_string_1", TiDB::TP::TypeString, false}, {"key_string_2", TiDB::TP::TypeString, false}, {"value", TiDB::TP::TypeLong, false}}; + ColumnsWithTypeAndName table_column_data; + for (const auto & column_info : mockColumnInfosToTiDBColumnInfos(table_column_infos)) + { + ColumnGeneratorOpts opts{unique_rows, getDataTypeByColumnInfoForComputingLayer(column_info)->getName(), RANDOM, column_info.name}; + table_column_data.push_back(ColumnGenerator::instance().generate(opts)); + } + for (auto & table_column : table_column_data) + { + table_column.column->assumeMutable()->insertRangeFrom(*table_column.column, 0, unique_rows / 2); + } + ColumnWithTypeAndName shuffle_column = ColumnGenerator::instance().generate({unique_rows + unique_rows / 2, "UInt64", RANDOM}); + IColumn::Permutation perm; + shuffle_column.column->getPermutation(false, 0, -1, perm); + for (auto & column : table_column_data) + { + column.column = column.column->permute(perm, 0); + } + + context.addMockTable("test_db", "agg_table_with_special_key", table_column_infos, table_column_data); + + std::vector max_block_sizes{1, 8, DEFAULT_BLOCK_SIZE}; + std::vector concurrences{1, 8}; + // 0: use one level + // 1: use two level + std::vector two_level_thresholds{0, 1}; + std::vector collators{TiDB::ITiDBCollator::UTF8MB4_BIN, TiDB::ITiDBCollator::UTF8MB4_GENERAL_CI}; + std::vector> group_by_keys{ + /// fast path with one int and one string + {"key_64", "key_string_1"}, + /// fast path with two string + {"key_string_1", "key_string_2"}, + /// fast path with one string + {"key_string_1"}, + /// keys need to be shuffled + {"key_8", "key_16", "key_32", "key_64"}, + }; + for (auto collator_id : collators) + { + for (const auto & keys : group_by_keys) + { + context.setCollation(collator_id); + const auto * current_collator = TiDB::ITiDBCollator::getCollator(collator_id); + ASSERT_TRUE(current_collator != nullptr); + SortDescription sd; + bool has_string_key = false; + MockAstVec key_vec; + for (const auto & key : keys) + key_vec.push_back(col(key)); + auto request = context + .scan("test_db", "agg_table_with_special_key") + .aggregation({Max(col("value"))}, key_vec) + .build(context); + /// use one level, no block split, no spill as the reference + context.context->setSetting("group_by_two_level_threshold_bytes", Field(static_cast(0))); + context.context->setSetting("max_bytes_before_external_group_by", Field(static_cast(0))); + context.context->setSetting("max_block_size", Field(static_cast(unique_rows * 2))); + auto reference = executeStreams(request); + if (current_collator->isCI()) + { + /// for ci collation, need to sort and compare the result manually + for (const auto & result_col : reference) + { + if (!removeNullable(result_col.type)->isString()) + { + sd.push_back(SortColumnDescription(result_col.name, 1, 1, nullptr)); + } + else + { + sd.push_back(SortColumnDescription(result_col.name, 1, 1, current_collator)); + has_string_key = true; + } + } + /// don't run ci test if there is no string key + if (!has_string_key) + continue; + Block tmp_block(reference); + sortBlock(tmp_block, sd); + reference = tmp_block.getColumnsWithTypeAndName(); + } + for (auto two_level_threshold : two_level_thresholds) + { + for (auto block_size : max_block_sizes) + { + for (auto concurrency : concurrences) + { + context.context->setSetting("group_by_two_level_threshold", Field(static_cast(two_level_threshold))); + context.context->setSetting("max_block_size", Field(static_cast(block_size))); + auto blocks = getExecuteStreamsReturnBlocks(request, concurrency); + for (auto & block : blocks) + { + block.checkNumberOfRows(); + ASSERT(block.rows() <= block_size); + } + if (current_collator->isCI()) + { + auto merged_block = vstackBlocks(std::move(blocks)); + sortBlock(merged_block, sd); + auto merged_columns = merged_block.getColumnsWithTypeAndName(); + for (size_t col_index = 0; col_index < reference.size(); col_index++) + ASSERT_TRUE(columnEqual(reference[col_index].column, merged_columns[col_index].column, sd[col_index].collator)); + } + else + { + ASSERT_TRUE(columnsEqual(reference, vstackBlocks(std::move(blocks)).getColumnsWithTypeAndName(), false)); + } + } + } + } + } + } +} +CATCH + TEST_F(AggExecutorTestRunner, Empty) try { diff --git a/dbms/src/Flash/tests/gtest_spill_aggregation.cpp b/dbms/src/Flash/tests/gtest_spill_aggregation.cpp index 42296055aec..417592a9353 100644 --- a/dbms/src/Flash/tests/gtest_spill_aggregation.cpp +++ b/dbms/src/Flash/tests/gtest_spill_aggregation.cpp @@ -21,7 +21,6 @@ namespace DB { namespace tests { - class SpillAggregationTestRunner : public DB::tests::ExecutorTest { public: @@ -96,5 +95,268 @@ try ASSERT_COLUMNS_EQ_UR(ref_columns, vstackBlocks(std::move(blocks)).getColumnsWithTypeAndName()); } CATCH + +TEST_F(SpillAggregationTestRunner, AggWithSpecialGroupKey) +try +{ + /// prepare data + size_t unique_rows = 3000; + DB::MockColumnInfoVec table_column_infos{{"key_8", TiDB::TP::TypeTiny, false}, {"key_16", TiDB::TP::TypeShort, false}, {"key_32", TiDB::TP::TypeLong, false}, {"key_64", TiDB::TP::TypeLongLong, false}, {"key_string_1", TiDB::TP::TypeString, false}, {"key_string_2", TiDB::TP::TypeString, false}, {"value", TiDB::TP::TypeLong, false}}; + ColumnsWithTypeAndName table_column_data; + for (const auto & column_info : mockColumnInfosToTiDBColumnInfos(table_column_infos)) + { + ColumnGeneratorOpts opts{unique_rows, getDataTypeByColumnInfoForComputingLayer(column_info)->getName(), RANDOM, column_info.name}; + table_column_data.push_back(ColumnGenerator::instance().generate(opts)); + } + for (auto & table_column : table_column_data) + { + if (table_column.name != "value") + table_column.column->assumeMutable()->insertRangeFrom(*table_column.column, 0, unique_rows / 2); + else + { + ColumnGeneratorOpts opts{unique_rows / 2, table_column.type->getName(), RANDOM, table_column.name}; + auto column = ColumnGenerator::instance().generate(opts); + table_column.column->assumeMutable()->insertRangeFrom(*column.column, 0, unique_rows / 2); + } + } + ColumnWithTypeAndName shuffle_column = ColumnGenerator::instance().generate({unique_rows + unique_rows / 2, "UInt64", RANDOM}); + IColumn::Permutation perm; + shuffle_column.column->getPermutation(false, 0, -1, perm); + for (auto & column : table_column_data) + { + column.column = column.column->permute(perm, 0); + } + + context.addMockTable("test_db", "agg_table_with_special_key", table_column_infos, table_column_data); + + size_t max_block_size = 800; + size_t max_bytes_before_external_agg = 100; + std::vector concurrences{1, 8}; + std::vector collators{TiDB::ITiDBCollator::UTF8MB4_BIN, TiDB::ITiDBCollator::UTF8MB4_GENERAL_CI}; + std::vector> group_by_keys{ + /// fast path with one int and one string in bin collation + {"key_64", "key_string_1"}, + /// fast path with two string in bin collation + {"key_string_1", "key_string_2"}, + /// fast path with one string in bin collation + {"key_string_1"}, + /// keys need to be shuffled + {"key_8", "key_16", "key_32", "key_64"}, + }; + std::vector> agg_funcs{{Max(col("value"))}, {Max(col("value")), Min(col("value"))}}; + for (auto collator_id : collators) + { + for (const auto & keys : group_by_keys) + { + for (const auto & agg_func : agg_funcs) + { + context.setCollation(collator_id); + const auto * current_collator = TiDB::ITiDBCollator::getCollator(collator_id); + ASSERT_TRUE(current_collator != nullptr); + SortDescription sd; + bool has_string_key = false; + MockAstVec key_vec; + for (const auto & key : keys) + key_vec.push_back(col(key)); + auto request = context + .scan("test_db", "agg_table_with_special_key") + .aggregation(agg_func, key_vec) + .build(context); + /// use one level, no block split, no spill as the reference + context.context->setSetting("group_by_two_level_threshold_bytes", Field(static_cast(0))); + context.context->setSetting("max_bytes_before_external_group_by", Field(static_cast(0))); + context.context->setSetting("max_block_size", Field(static_cast(unique_rows * 2))); + /// here has to enable memory tracker otherwise the processList in the context is the last query's processList + /// and may cause segment fault, maybe a bug but should not happens in TiDB because all the tasks from tidb + /// enable memory tracker + auto reference = executeStreams(request, 1, true); + if (current_collator->isCI()) + { + /// for ci collation, need to sort and compare the result manually + for (const auto & result_col : reference) + { + if (!removeNullable(result_col.type)->isString()) + { + sd.push_back(SortColumnDescription(result_col.name, 1, 1, nullptr)); + } + else + { + sd.push_back(SortColumnDescription(result_col.name, 1, 1, current_collator)); + has_string_key = true; + } + } + /// don't run ci test if there is no string key + if (!has_string_key) + continue; + Block tmp_block(reference); + sortBlock(tmp_block, sd); + reference = tmp_block.getColumnsWithTypeAndName(); + } + for (auto concurrency : concurrences) + { + context.context->setSetting("group_by_two_level_threshold", Field(static_cast(1))); + context.context->setSetting("group_by_two_level_threshold_bytes", Field(static_cast(1))); + context.context->setSetting("max_bytes_before_external_group_by", Field(static_cast(max_bytes_before_external_agg))); + context.context->setSetting("max_block_size", Field(static_cast(max_block_size))); + auto blocks = getExecuteStreamsReturnBlocks(request, concurrency, true); + for (auto & block : blocks) + { + block.checkNumberOfRows(); + ASSERT(block.rows() <= max_block_size); + } + if (current_collator->isCI()) + { + auto merged_block = vstackBlocks(std::move(blocks)); + sortBlock(merged_block, sd); + auto merged_columns = merged_block.getColumnsWithTypeAndName(); + for (size_t col_index = 0; col_index < reference.size(); col_index++) + ASSERT_TRUE(columnEqual(reference[col_index].column, merged_columns[col_index].column, sd[col_index].collator)); + } + else + { + ASSERT_TRUE(columnsEqual(reference, vstackBlocks(std::move(blocks)).getColumnsWithTypeAndName(), false)); + } + } + } + } + } +} +CATCH + +TEST_F(SpillAggregationTestRunner, AggWithDistinctAggFunc) +try +{ + /// prepare data + size_t unique_rows = 3000; + DB::MockColumnInfoVec table_column_infos{ + {"key_8", TiDB::TP::TypeTiny, false}, + {"key_16", TiDB::TP::TypeShort, false}, + {"key_32", TiDB::TP::TypeLong, false}, + {"key_64", TiDB::TP::TypeLongLong, false}, + {"key_string_1", TiDB::TP::TypeString, false}, + {"key_string_2", TiDB::TP::TypeString, false}, + {"value_1", TiDB::TP::TypeString, false}, + {"value_2", TiDB::TP::TypeLong, false}, + }; + size_t key_column = 6; + ColumnsWithTypeAndName table_column_data; + for (const auto & column_info : mockColumnInfosToTiDBColumnInfos(table_column_infos)) + { + ColumnGeneratorOpts opts{unique_rows, getDataTypeByColumnInfoForComputingLayer(column_info)->getName(), RANDOM, column_info.name}; + table_column_data.push_back(ColumnGenerator::instance().generate(opts)); + } + for (size_t i = 0; i < key_column; i++) + table_column_data[i].column->assumeMutable()->insertRangeFrom(*table_column_data[i].column, 0, unique_rows / 2); + for (size_t i = key_column; i < table_column_data.size(); i++) + { + auto & table_column = table_column_data[i]; + ColumnGeneratorOpts opts{unique_rows / 2, table_column.type->getName(), RANDOM, table_column.name}; + auto column = ColumnGenerator::instance().generate(opts); + table_column.column->assumeMutable()->insertRangeFrom(*column.column, 0, unique_rows / 2); + } + + ColumnWithTypeAndName shuffle_column = ColumnGenerator::instance().generate({unique_rows + unique_rows / 2, "UInt64", RANDOM}); + IColumn::Permutation perm; + shuffle_column.column->getPermutation(false, 0, -1, perm); + for (auto & column : table_column_data) + { + column.column = column.column->permute(perm, 0); + } + + context.addMockTable("test_db", "agg_table_with_special_key", table_column_infos, table_column_data); + + size_t max_block_size = 800; + size_t max_bytes_before_external_agg = 100; + std::vector concurrences{1, 8}; + std::vector collators{TiDB::ITiDBCollator::UTF8MB4_BIN, TiDB::ITiDBCollator::UTF8MB4_GENERAL_CI}; + std::vector> group_by_keys{ + /// fast path with one int and one string + {"key_64", "key_string_1"}, + /// fast path with two string + {"key_string_1", "key_string_2"}, + /// fast path with one string + {"key_string_1"}, + /// keys need to be shuffled + {"key_8", "key_16", "key_32", "key_64"}, + }; + std::vector> agg_funcs{{Max(col("value_1")), CountDistinct(col("value_2"))}, {CountDistinct(col("value_1")), CountDistinct(col("value_2"))}, {CountDistinct(col("value_1"))}}; + for (auto collator_id : collators) + { + for (const auto & keys : group_by_keys) + { + for (const auto & agg_func : agg_funcs) + { + context.setCollation(collator_id); + const auto * current_collator = TiDB::ITiDBCollator::getCollator(collator_id); + ASSERT_TRUE(current_collator != nullptr); + SortDescription sd; + bool has_string_key = false; + MockAstVec key_vec; + for (const auto & key : keys) + key_vec.push_back(col(key)); + auto request = context + .scan("test_db", "agg_table_with_special_key") + .aggregation(agg_func, key_vec) + .build(context); + /// use one level, no block split, no spill as the reference + context.context->setSetting("group_by_two_level_threshold_bytes", Field(static_cast(0))); + context.context->setSetting("max_bytes_before_external_group_by", Field(static_cast(0))); + context.context->setSetting("max_block_size", Field(static_cast(unique_rows * 2))); + /// here has to enable memory tracker otherwise the processList in the context is the last query's processList + /// and may cause segment fault, maybe a bug but should not happens in TiDB because all the tasks from tidb + /// enable memory tracker + auto reference = executeStreams(request, 1, true); + if (current_collator->isCI()) + { + /// for ci collation, need to sort and compare the result manually + for (const auto & result_col : reference) + { + if (!removeNullable(result_col.type)->isString()) + { + sd.push_back(SortColumnDescription(result_col.name, 1, 1, nullptr)); + } + else + { + sd.push_back(SortColumnDescription(result_col.name, 1, 1, current_collator)); + has_string_key = true; + } + } + /// don't run ci test if there is no string key + if (!has_string_key) + continue; + Block tmp_block(reference); + sortBlock(tmp_block, sd); + reference = tmp_block.getColumnsWithTypeAndName(); + } + for (auto concurrency : concurrences) + { + context.context->setSetting("group_by_two_level_threshold", Field(static_cast(1))); + context.context->setSetting("group_by_two_level_threshold_bytes", Field(static_cast(1))); + context.context->setSetting("max_bytes_before_external_group_by", Field(static_cast(max_bytes_before_external_agg))); + context.context->setSetting("max_block_size", Field(static_cast(max_block_size))); + auto blocks = getExecuteStreamsReturnBlocks(request, concurrency, true); + for (auto & block : blocks) + { + block.checkNumberOfRows(); + ASSERT(block.rows() <= max_block_size); + } + if (current_collator->isCI()) + { + auto merged_block = vstackBlocks(std::move(blocks)); + sortBlock(merged_block, sd); + auto merged_columns = merged_block.getColumnsWithTypeAndName(); + for (size_t col_index = 0; col_index < reference.size(); col_index++) + ASSERT_TRUE(columnEqual(reference[col_index].column, merged_columns[col_index].column, sd[col_index].collator)); + } + else + { + ASSERT_TRUE(columnsEqual(reference, vstackBlocks(std::move(blocks)).getColumnsWithTypeAndName(), false)); + } + } + } + } + } +} +CATCH } // namespace tests } // namespace DB diff --git a/dbms/src/Flash/tests/gtest_spill_sort.cpp b/dbms/src/Flash/tests/gtest_spill_sort.cpp index 4061ddf000c..95a910f4f83 100644 --- a/dbms/src/Flash/tests/gtest_spill_sort.cpp +++ b/dbms/src/Flash/tests/gtest_spill_sort.cpp @@ -21,7 +21,6 @@ namespace DB { namespace tests { - class SpillSortTestRunner : public DB::tests::ExecutorTest { public: @@ -70,5 +69,48 @@ try ASSERT_COLUMNS_EQ_UR(ref_columns, executeStreams(request, original_max_streams)); } CATCH + +TEST_F(SpillSortTestRunner, CollatorTest) +try +{ + DB::MockColumnInfoVec column_infos{{"a", TiDB::TP::TypeString, false}, {"b", TiDB::TP::TypeString, false}, {"c", TiDB::TP::TypeString, false}, {"d", TiDB::TP::TypeString, false}, {"e", TiDB::TP::TypeString, false}}; + ColumnsWithTypeAndName column_data; + size_t table_rows = 102400; + UInt64 max_block_size = 500; + size_t original_max_streams = 20; + size_t total_data_size = 0; + size_t limit_size = table_rows / 10 * 8; + for (const auto & column_info : mockColumnInfosToTiDBColumnInfos(column_infos)) + { + ColumnGeneratorOpts opts{table_rows, getDataTypeByColumnInfoForComputingLayer(column_info)->getName(), RANDOM, column_info.name, 5}; + column_data.push_back(ColumnGenerator::instance().generate(opts)); + total_data_size += column_data.back().column->byteSize(); + } + context.addMockTable("spill_sort_test", "collation_table", column_infos, column_data, 8); + + MockOrderByItemVec order_by_items{std::make_pair("a", true), std::make_pair("b", true), std::make_pair("c", true), std::make_pair("d", true), std::make_pair("e", true)}; + std::vector collators{TiDB::ITiDBCollator::UTF8MB4_BIN, TiDB::ITiDBCollator::UTF8MB4_GENERAL_CI, TiDB::ITiDBCollator::UTF8MB4_UNICODE_CI}; + for (const auto & collator_id : collators) + { + context.setCollation(collator_id); + auto request = context + .scan("spill_sort_test", "collation_table") + .topN(order_by_items, limit_size) + .build(context); + context.context->setSetting("max_block_size", Field(static_cast(max_block_size))); + /// disable spill + context.context->setSetting("max_bytes_before_external_sort", Field(static_cast(0))); + auto ref_columns = executeStreams(request, original_max_streams); + /// enable spill + context.context->setSetting("max_bytes_before_external_sort", Field(static_cast(total_data_size / 10))); + // don't use `executeAndAssertColumnsEqual` since it takes too long to run + /// todo use ASSERT_COLUMNS_EQ_R once TiFlash support final TopN + ASSERT_COLUMNS_EQ_UR(ref_columns, executeStreams(request, original_max_streams)); + /// enable spill and use small max_cached_data_bytes_in_spiller + context.context->setSetting("max_cached_data_bytes_in_spiller", Field(static_cast(total_data_size / 100))); + ASSERT_COLUMNS_EQ_UR(ref_columns, executeStreams(request, original_max_streams)); + } +} +CATCH } // namespace tests } // namespace DB diff --git a/dbms/src/Functions/FunctionsStringSearch.h b/dbms/src/Functions/FunctionsStringSearch.h index dc9bccd10f6..a023142bbb9 100644 --- a/dbms/src/Functions/FunctionsStringSearch.h +++ b/dbms/src/Functions/FunctionsStringSearch.h @@ -36,61 +36,149 @@ namespace DB using Chars_t = ColumnString::Chars_t; using Offsets = ColumnString::Offsets; -struct IlikeHelper +class IlikeLowerHelper { +public: + static void convertCollatorToBin(TiDB::TiDBCollatorPtr & collator) + { + if (collator == nullptr) + return; + + switch (collator->getCollatorType()) + { + case TiDB::ITiDBCollator::CollatorType::UTF8_GENERAL_CI: + case TiDB::ITiDBCollator::CollatorType::UTF8_UNICODE_CI: + collator = TiDB::ITiDBCollator::getCollator(TiDB::ITiDBCollator::UTF8_BIN); + break; + case TiDB::ITiDBCollator::CollatorType::UTF8MB4_GENERAL_CI: + case TiDB::ITiDBCollator::CollatorType::UTF8MB4_UNICODE_CI: + collator = TiDB::ITiDBCollator::getCollator(TiDB::ITiDBCollator::UTF8MB4_BIN); + break; + default: + break; + } + } + + // Only lower 'A', 'B', 'C'... 'Z', excluding the escape char + static void lowerAlphaASCII(Block & block, const ColumnNumbers & arguments) + { + MutableColumnPtr column_expr = block.getByPosition(arguments[0]).column->assumeMutable(); + MutableColumnPtr column_pat = block.getByPosition(arguments[1]).column->assumeMutable(); + const ColumnPtr & column_escape = block.getByPosition(arguments[2]).column; + + auto * col_haystack_const = typeid_cast(&*column_expr); + auto * col_needle_const = typeid_cast(&*column_pat); + + if (col_haystack_const != nullptr) + lowerColumnConst(col_haystack_const, nullptr); + else + lowerColumnString(column_expr, nullptr); + + if (col_needle_const != nullptr) + lowerColumnConst(col_needle_const, column_escape); + else + lowerColumnString(column_pat, column_escape); + } + +private: static void lowerStrings(Chars_t & chars) { size_t size = chars.size(); - size_t i = 0; - while (i < size) + for (size_t i = 0; i < size; ++i) { if (isUpperAlphaASCII(chars[i])) { chars[i] = toLowerIfAlphaASCII(chars[i]); - ++i; } else { size_t utf8_len = UTF8::seqLength(chars[i]); - i += utf8_len; + i += utf8_len - 1; } } } - static void lowerColumnConst(ColumnConst * lowered_col_const) + // When escape_char is a lower char, we need to convert it to the capital char + // Because: when lowering "ABC" with escape 'a', after lower, "ABC" -> "abc", + // then 'a' will be an escape char and it is not expected. + // Morever, when escape char is uppered we need to tell it to the caller. + static void lowerStringsExcludeEscapeChar(Chars_t & chars, char escape_char) + { + if (!isAlphaASCII(escape_char)) + { + lowerStrings(chars); + return; + } + + size_t size = chars.size(); + bool escaped = false; + char actual_escape_char = isLowerAplhaASCII(escape_char) ? toUpperIfAlphaASCII(escape_char) : escape_char; + + for (size_t i = 0; i < size; ++i) + { + char char_to_lower = chars[i]; + if (isUpperAlphaASCII(char_to_lower)) + { + // Do not lower the escape char, however when a char is equal to + // an escape char and it's after an escape char, we still lower it + // For example: "AA" (escape 'A'), -> "Aa" + if (char_to_lower != escape_char || escaped) + { + chars[i] = toLowerIfAlphaASCII(char_to_lower); + } + else + { + escaped = true; + continue; + } + } + else + { + if ((chars[i] == static_cast(escape_char)) && !escaped) + { + escaped = true; + + // It should be `chars[i] = toUpperIfAlphaASCII(chars[i])`, + // but 'actual_escape_char' is always equal to 'toUpperIfAlphaASCII(str[i])' + chars[i] = actual_escape_char; + continue; + } + size_t utf8_len = UTF8::seqLength(char_to_lower); + i += utf8_len - 1; + } + escaped = false; + } + } + + static void lowerColumnConst(ColumnConst * lowered_col_const, const ColumnPtr & column_escape) { auto * col_data = typeid_cast(&lowered_col_const->getDataColumn()); RUNTIME_ASSERT(col_data != nullptr, "Invalid column type, should be ColumnString"); - lowerStrings(col_data->getChars()); + lowerColumnStringImpl(col_data, column_escape); } - static void lowerColumnString(MutableColumnPtr & col) + static void lowerColumnString(MutableColumnPtr & col, const ColumnPtr & column_escape) { auto * col_vector = typeid_cast(&*col); RUNTIME_ASSERT(col_vector != nullptr, "Invalid column type, should be ColumnString"); - lowerStrings(col_vector->getChars()); + lowerColumnStringImpl(col_vector, column_escape); } - // Only lower the 'A', 'B', 'C'... - static void lowerAlphaASCII(Block & block, const ColumnNumbers & arguments) + static void lowerColumnStringImpl(ColumnString * lowered_col_data, const ColumnPtr & column_escape) { - MutableColumnPtr column_haystack = block.getByPosition(arguments[0]).column->assumeMutable(); - MutableColumnPtr column_needle = block.getByPosition(arguments[1]).column->assumeMutable(); + if (column_escape == nullptr) + { + lowerStrings(lowered_col_data->getChars()); + return; + } - auto * col_haystack_const = typeid_cast(&*column_haystack); - auto * col_needle_const = typeid_cast(&*column_needle); + const auto * col_escape_const = typeid_cast(&*column_escape); + RUNTIME_CHECK_MSG(col_escape_const != nullptr, "escape char column should be constant"); + char escape_char = static_cast(col_escape_const->getValue()); - if (col_haystack_const != nullptr) - lowerColumnConst(col_haystack_const); - else - lowerColumnString(column_haystack); - - if (col_needle_const != nullptr) - lowerColumnConst(col_needle_const); - else - lowerColumnString(column_needle); + lowerStringsExcludeEscapeChar(lowered_col_data->getChars(), escape_char); } }; @@ -197,13 +285,11 @@ class FunctionsStringSearch : public IFunction auto block = result_block; if constexpr (name == std::string_view(NameIlike3Args::name)) { - if (!collator->isCI()) - { - block.getByPosition(arguments[0]).column = (*std::move(result_block.getByPosition(arguments[0]).column)).mutate(); - block.getByPosition(arguments[1]).column = (*std::move(result_block.getByPosition(arguments[1]).column)).mutate(); + block.getByPosition(arguments[0]).column = (*std::move(result_block.getByPosition(arguments[0]).column)).mutate(); + block.getByPosition(arguments[1]).column = (*std::move(result_block.getByPosition(arguments[1]).column)).mutate(); - IlikeHelper::lowerAlphaASCII(block, arguments); - } + IlikeLowerHelper::lowerAlphaASCII(block, arguments); + IlikeLowerHelper::convertCollatorToBin(collator); } using ResultType = typename Impl::ResultType; @@ -235,6 +321,9 @@ class FunctionsStringSearch : public IFunction else { escape_char = static_cast(c); + if constexpr (name == std::string_view(NameIlike3Args::name)) + if (isLowerAplhaASCII(escape_char)) + escape_char = toUpperIfAlphaASCII(escape_char); } } if (!valid_args) @@ -302,7 +391,7 @@ class FunctionsStringSearch : public IFunction } private: - TiDB::TiDBCollatorPtr collator = nullptr; + mutable TiDB::TiDBCollatorPtr collator = nullptr; }; diff --git a/dbms/src/Functions/tests/gtest_strings_search.cpp b/dbms/src/Functions/tests/gtest_strings_search.cpp index 51560cf701a..1ac17f3bb62 100644 --- a/dbms/src/Functions/tests/gtest_strings_search.cpp +++ b/dbms/src/Functions/tests/gtest_strings_search.cpp @@ -17,6 +17,8 @@ #include #include +#include "magic_enum.hpp" + namespace DB { namespace tests @@ -668,6 +670,133 @@ TEST_F(StringMatch, IlikeConstWithConst) } } +TEST_F(StringMatch, CheckEscape) +{ + std::vector collators{ + TiDB::ITiDBCollator::getCollator(TiDB::ITiDBCollator::UTF8_GENERAL_CI), + TiDB::ITiDBCollator::getCollator(TiDB::ITiDBCollator::UTF8MB4_GENERAL_CI), + TiDB::ITiDBCollator::getCollator(TiDB::ITiDBCollator::UTF8_UNICODE_CI), + TiDB::ITiDBCollator::getCollator(TiDB::ITiDBCollator::UTF8MB4_UNICODE_CI), + TiDB::ITiDBCollator::getCollator(TiDB::ITiDBCollator::UTF8MB4_BIN), + TiDB::ITiDBCollator::getCollator(TiDB::ITiDBCollator::LATIN1_BIN), + TiDB::ITiDBCollator::getCollator(TiDB::ITiDBCollator::BINARY), + TiDB::ITiDBCollator::getCollator(TiDB::ITiDBCollator::ASCII_BIN), + TiDB::ITiDBCollator::getCollator(TiDB::ITiDBCollator::UTF8_BIN)}; + + std::vector> expr_vec{"", "aaz", "aaz", "AAz", "aAz", "a啊啊啊aa啊Zz", "ü", "á"}; + std::vector> pat_vec{"", "AAAAz", "Aaaz", "AAAAZ", "aAaAz", "a啊啊啊AaaA啊Zz", "Ü", "a"}; + std::vector> vec_vec_lower_a_expect = {1, 0, 1, 0, 1, 0, 0, 0}; // escape 'a' + std::vector> vec_vec_capital_a_expect = {1, 1, 1, 1, 1, 1, 0, 0}; // escape 'A' + ColumnWithTypeAndName escape_lower_a = createConstColumn(1, static_cast('a')); + ColumnWithTypeAndName escape_capital_a = createConstColumn(1, static_cast('A')); + + for (const auto * collator : collators) + { + // vec vec + ASSERT_COLUMN_EQ( + toNullableVec(vec_vec_lower_a_expect), + executeFunction( + "ilike3Args", + {toNullableVec(expr_vec), toNullableVec(pat_vec), escape_lower_a}, + collator)); + + ASSERT_COLUMN_EQ( + toNullableVec(vec_vec_capital_a_expect), + executeFunction( + "ilike3Args", + {toNullableVec(expr_vec), toNullableVec(pat_vec), escape_capital_a}, + collator)); + + // const const + ASSERT_COLUMN_EQ( + toConst(0), + executeFunction( + "ilike3Args", + {toConst("aa"), toConst("aa"), escape_lower_a}, + collator)); + + ASSERT_COLUMN_EQ( + toConst(1), + executeFunction( + "ilike3Args", + {toConst("aa"), toConst("aa"), escape_capital_a}, + collator)); + + ASSERT_COLUMN_EQ( + toConst(1), + executeFunction( + "ilike3Args", + {toConst("Aa"), toConst("aaA"), escape_lower_a}, + collator)); + + ASSERT_COLUMN_EQ( + toConst(0), + executeFunction( + "ilike3Args", + {toConst("Aa"), toConst("aaA"), escape_capital_a}, + collator)); + + ASSERT_COLUMN_EQ( + toConst(0), + executeFunction( + "ilike3Args", + {toConst("a啊啊a"), toConst("a啊啊A"), escape_lower_a}, + collator)); + + ASSERT_COLUMN_EQ( + toConst(0), + executeFunction( + "ilike3Args", + {toConst("a啊啊a"), toConst("A啊啊a"), escape_capital_a}, + collator)); + + ASSERT_COLUMN_EQ( + toConst(0), + executeFunction( + "ilike3Args", + {toConst("ü"), toConst("Ü"), escape}, + collator)); + + ASSERT_COLUMN_EQ( + toConst(0), + executeFunction( + "ilike3Args", + {toConst("a"), toConst("á"), escape}, + collator)); + + // vec const + ASSERT_COLUMN_EQ( + toNullableVec({0, 1, 1, 1, 1, 0, 0, 0}), + executeFunction( + "ilike3Args", + {toNullableVec(expr_vec), toConst("Aaaz"), escape_lower_a}, + collator)); + + ASSERT_COLUMN_EQ( + toNullableVec({0, 1, 1, 1, 1, 0, 0, 0}), + executeFunction( + "ilike3Args", + {toNullableVec(expr_vec), toConst("aAaZ"), escape_capital_a}, + collator)); + + // const vec + // "", "AAAAz", "Aaaz", "AAAAZ", "aAaAz", "a啊啊啊AaaA啊Zz", "Ü", "a"}; + ASSERT_COLUMN_EQ( + toNullableVec({0, 0, 1, 0, 1, 0, 0, 0}), + executeFunction( + "ilike3Args", + {toConst("aAz"), toNullableVec(pat_vec), escape_lower_a}, + collator)); + + ASSERT_COLUMN_EQ( + toNullableVec({0, 1, 1, 1, 1, 0, 0, 0}), + executeFunction( + "ilike3Args", + {toConst("AaZ"), toNullableVec(pat_vec), escape_capital_a}, + collator)); + } +} + // ilike function will modify the column's content in-place, in order to // ensure the column's content is not modified after function finishes the work, // we need to replace the modified columns with other columns which clone the diff --git a/dbms/src/IO/IOThreadPool.h b/dbms/src/IO/IOThreadPool.h index ad323b23323..b8a8d021e40 100644 --- a/dbms/src/IO/IOThreadPool.h +++ b/dbms/src/IO/IOThreadPool.h @@ -14,24 +14,35 @@ #pragma once +#include #include namespace DB { struct Settings; -/* - * ThreadPool used for the IO. - */ +template class IOThreadPool { friend void adjustThreadPoolSize(const Settings & settings, size_t logical_cores); - static std::unique_ptr instance; + static inline std::unique_ptr instance; public: - static void initialize(size_t max_threads, size_t max_free_threads, size_t queue_size); - static ThreadPool & get(); + static void initialize(size_t max_threads, size_t max_free_threads, size_t queue_size) + { + RUNTIME_CHECK_MSG(!instance, "IO thread pool is initialized twice"); + instance = std::make_unique(max_threads, max_free_threads, queue_size, false /*shutdown_on_exception*/); + GlobalThreadPool::instance().registerFinalizer([] { + instance.reset(); + }); + } + + static ThreadPool & get() + { + RUNTIME_CHECK_MSG(instance, "IO thread pool is not initialized"); + return *instance; + } static void shutdown() noexcept { instance.reset(); } }; diff --git a/dbms/src/IO/IOThreadPool.cpp b/dbms/src/IO/IOThreadPools.h similarity index 50% rename from dbms/src/IO/IOThreadPool.cpp rename to dbms/src/IO/IOThreadPools.h index 761157546cf..5b68a444bfc 100644 --- a/dbms/src/IO/IOThreadPool.cpp +++ b/dbms/src/IO/IOThreadPools.h @@ -12,37 +12,39 @@ // See the License for the specific language governing permissions and // limitations under the License. -#include +#pragma once + #include namespace DB { -namespace ErrorCodes +namespace io_pool_details { -extern const int LOGICAL_ERROR; -} -std::unique_ptr IOThreadPool::instance; +struct S3FileCacheTrait +{ +}; -void IOThreadPool::initialize(size_t max_threads, size_t max_free_threads, size_t queue_size) +struct DataStoreS3Trait { - if (instance) - { - throw Exception(ErrorCodes::LOGICAL_ERROR, "The IO thread pool is initialized twice"); - } +}; - instance = std::make_unique(max_threads, max_free_threads, queue_size, false /*shutdown_on_exception*/); -} +struct RemoteReadTaskTrait +{ +}; -ThreadPool & IOThreadPool::get() +struct RNPreparerTrait { - if (!instance) - { - throw Exception(ErrorCodes::LOGICAL_ERROR, "The IO thread pool is not initialized"); - } +}; + +} // namespace io_pool_details + +// TODO: Move these out. +using DataStoreS3Pool = IOThreadPool; +using S3FileCachePool = IOThreadPool; +using RNRemoteReadTaskPool = IOThreadPool; +using RNPagePreparerPool = IOThreadPool; - return *instance; -} } // namespace DB diff --git a/dbms/src/Interpreters/Aggregator.cpp b/dbms/src/Interpreters/Aggregator.cpp index c3ed6e815bf..50d8a9c7e73 100644 --- a/dbms/src/Interpreters/Aggregator.cpp +++ b/dbms/src/Interpreters/Aggregator.cpp @@ -1211,27 +1211,56 @@ void NO_INLINE Aggregator::convertToBlockImplFinal( }); } +namespace +{ +template +std::optional shuffleKeyColumnsForKeyColumnsVec(Method & method, std::vector> & key_columns_vec, const Sizes & key_sizes) +{ + auto shuffled_key_sizes = method.shuffleKeyColumns(key_columns_vec[0], key_sizes); + for (size_t i = 1; i < key_columns_vec.size(); ++i) + { + auto new_key_sizes = method.shuffleKeyColumns(key_columns_vec[i], key_sizes); + assert(shuffled_key_sizes == new_key_sizes); + } + return shuffled_key_sizes; +} +template +std::vector>> initAggKeysForKeyColumnsVec(Method & method, std::vector> & key_columns_vec, size_t max_block_size, size_t total_row_count) +{ + std::vector>> agg_keys_helpers; + size_t block_row_count = max_block_size; + for (size_t i = 0; i < key_columns_vec.size(); ++i) + { + if (i == key_columns_vec.size() - 1 && total_row_count % block_row_count != 0) + /// update block_row_count for the last block + block_row_count = total_row_count % block_row_count; + agg_keys_helpers.push_back(std::make_unique>(method)); + agg_keys_helpers.back()->initAggKeys(block_row_count, key_columns_vec[i]); + } + return agg_keys_helpers; +} +} // namespace + template void NO_INLINE Aggregator::convertToBlocksImplFinal( Method & method, Table & data, - std::vector> key_columns_vec, + std::vector> && key_columns_vec, std::vector & final_aggregate_columns_vec, Arena * arena) const { assert(!key_columns_vec.empty()); - auto shuffled_key_sizes = method.shuffleKeyColumns(key_columns_vec[0], key_sizes); + auto shuffled_key_sizes = shuffleKeyColumnsForKeyColumnsVec(method, key_columns_vec, key_sizes); const auto & key_sizes_ref = shuffled_key_sizes ? *shuffled_key_sizes : key_sizes; - AggregatorMethodInitKeyColumnHelper agg_keys_helper{method}; - agg_keys_helper.initAggKeys(data.size(), key_columns_vec[0]); + auto agg_keys_helpers = initAggKeysForKeyColumnsVec(method, key_columns_vec, params.max_block_size, data.size()); size_t data_index = 0; data.forEachValue([&](const auto & key, auto & mapped) { size_t key_columns_vec_index = data_index / params.max_block_size; - agg_keys_helper.insertKeyIntoColumns(key, key_columns_vec[key_columns_vec_index], key_sizes_ref, params.collators); + agg_keys_helpers[key_columns_vec_index]->insertKeyIntoColumns(key, key_columns_vec[key_columns_vec_index], key_sizes_ref, params.collators); insertAggregatesIntoColumns(mapped, final_aggregate_columns_vec[key_columns_vec_index], arena); - data_index++; + ++data_index; }); } @@ -1263,25 +1292,24 @@ template void NO_INLINE Aggregator::convertToBlocksImplNotFinal( Method & method, Table & data, - std::vector> key_columns_vec, + std::vector> && key_columns_vec, std::vector & aggregate_columns_vec) const { - auto shuffled_key_sizes = method.shuffleKeyColumns(key_columns_vec[0], key_sizes); + auto shuffled_key_sizes = shuffleKeyColumnsForKeyColumnsVec(method, key_columns_vec, key_sizes); const auto & key_sizes_ref = shuffled_key_sizes ? *shuffled_key_sizes : key_sizes; - AggregatorMethodInitKeyColumnHelper agg_keys_helper{method}; - agg_keys_helper.initAggKeys(data.size(), key_columns_vec[0]); + auto agg_keys_helpers = initAggKeysForKeyColumnsVec(method, key_columns_vec, params.max_block_size, data.size()); size_t data_index = 0; data.forEachValue([&](const auto & key, auto & mapped) { size_t key_columns_vec_index = data_index / params.max_block_size; - agg_keys_helper.insertKeyIntoColumns(key, key_columns_vec[key_columns_vec_index], key_sizes_ref, params.collators); + agg_keys_helpers[key_columns_vec_index]->insertKeyIntoColumns(key, key_columns_vec[key_columns_vec_index], key_sizes_ref, params.collators); /// reserved, so push_back does not throw exceptions for (size_t i = 0; i < params.aggregates_size; ++i) aggregate_columns_vec[key_columns_vec_index][i]->push_back(mapped + offsets_of_aggregate_states[i]); - data_index++; + ++data_index; mapped = nullptr; }); } @@ -1738,7 +1766,8 @@ void NO_INLINE Aggregator::mergeStreamsImplCase( std::vector sort_key_containers; sort_key_containers.resize(params.keys_size, ""); - typename Method::State state(key_columns, key_sizes, params.collators); + /// in merge stage, don't need to care about the collator because the key is already the sort_key of original string + typename Method::State state(key_columns, key_sizes, {}); /// For all rows. size_t rows = block.rows(); diff --git a/dbms/src/Interpreters/Aggregator.h b/dbms/src/Interpreters/Aggregator.h index 1f1f8ddc30f..674dbf98887 100644 --- a/dbms/src/Interpreters/Aggregator.h +++ b/dbms/src/Interpreters/Aggregator.h @@ -220,15 +220,12 @@ struct AggregationMethodOneKeyStringNoCache std::optional shuffleKeyColumns(std::vector &, const Sizes &) { return {}; } - ALWAYS_INLINE static inline void insertKeyIntoColumns(const StringRef &, std::vector &, size_t) + ALWAYS_INLINE static inline void insertKeyIntoColumns(const StringRef & key, std::vector & key_columns, size_t) { - // insert empty because such column will be discarded. - } - // resize offsets for column string - ALWAYS_INLINE static inline void initAggKeys(size_t rows, IColumn * key_column) - { - static_cast(key_column)->getOffsets().resize_fill(rows, 0); + /// still need to insert data to key because spill may will use this + static_cast(key_columns[0])->insertData(key.data, key.size); } + ALWAYS_INLINE static inline void initAggKeys(size_t, IColumn *) {} }; /* @@ -289,20 +286,15 @@ struct AggregationMethodFastPathTwoKeysNoCache column->getData().resize_fill(rows, 0); } - // Only update offsets but DO NOT insert string data. - // Because of https://github.com/pingcap/tiflash/blob/84c2650bc4320919b954babeceb5aeaadb845770/dbms/src/Columns/IColumn.h#L160-L173, such column will be discarded. - ALWAYS_INLINE static inline const char * insertAggKeyIntoColumnString(const char * pos, IColumn *) + ALWAYS_INLINE static inline const char * insertAggKeyIntoColumnString(const char * pos, IColumn * key_column) { + /// still need to insert data to key because spill may will use this const size_t string_size = *reinterpret_cast(pos); pos += sizeof(string_size); + static_cast(key_column)->insertData(pos, string_size); return pos + string_size; } - // resize offsets for column string - ALWAYS_INLINE static inline void initAggKeyString(size_t rows, IColumn * key_column) - { - auto * column = static_cast(key_column); - column->getOffsets().resize_fill(rows, 0); - } + ALWAYS_INLINE static inline void initAggKeyString(size_t, IColumn *) {} template <> ALWAYS_INLINE static inline void initAggKeys(size_t rows, IColumn * key_column) @@ -1220,7 +1212,7 @@ class Aggregator void convertToBlocksImplFinal( Method & method, Table & data, - std::vector> key_columns_vec, + std::vector> && key_columns_vec, std::vector & final_aggregate_columns_vec, Arena * arena) const; @@ -1235,7 +1227,7 @@ class Aggregator void convertToBlocksImplNotFinal( Method & method, Table & data, - std::vector> key_columns_vec, + std::vector> && key_columns_vec, std::vector & aggregate_columns_vec) const; template diff --git a/dbms/src/Interpreters/AsynchronousMetrics.cpp b/dbms/src/Interpreters/AsynchronousMetrics.cpp index f82cb68e8b3..e08725ea91d 100644 --- a/dbms/src/Interpreters/AsynchronousMetrics.cpp +++ b/dbms/src/Interpreters/AsynchronousMetrics.cpp @@ -27,6 +27,7 @@ #include #include #include +#include #include #include #include @@ -141,6 +142,12 @@ FileUsageStatistics AsynchronousMetrics::getPageStorageFileUsage() .merge(meta_usage) .merge(data_usage); } + + if (auto ps_cache = context.getSharedContextDisagg()->rn_page_cache_storage; ps_cache != nullptr) + { + usage.merge(ps_cache->getUniversalPageStorage()->getFileUsageStatistics()); + } + return usage; } diff --git a/dbms/src/Interpreters/Context.cpp b/dbms/src/Interpreters/Context.cpp index 7004eebcd0d..9c1e4e77b4d 100644 --- a/dbms/src/Interpreters/Context.cpp +++ b/dbms/src/Interpreters/Context.cpp @@ -275,6 +275,11 @@ struct ContextShared tmt_context->shutdown(); } + if (schema_sync_service) + { + schema_sync_service = nullptr; + } + /** At this point, some tables may have threads that block our mutex. * To complete them correctly, we will copy the current list of tables, * and ask them all to finish their work. @@ -1708,12 +1713,7 @@ void Context::initializeWriteNodePageStorageIfNeed(const PathPool & path_pool) auto lock = getLock(); if (shared->storage_run_mode == PageStorageRunMode::UNI_PS) { - if (shared->ps_write) - { - // GlobalStoragePool may be initialized many times in some test cases for restore. - LOG_WARNING(shared->log, "GlobalUniversalPageStorage(WriteNode) has already been initialized."); - shared->ps_write->shutdown(); - } + RUNTIME_CHECK(shared->ps_write == nullptr); try { PageStorageConfig config; @@ -1750,6 +1750,33 @@ UniversalPageStoragePtr Context::getWriteNodePageStorage() const } } +// In some unit tests, we may want to reinitialize WriteNodePageStorage multiple times to mock restart. +// And we need to release old one before creating new one. +// And we must do it explicitly. Because if we do it implicitly in `initializeWriteNodePageStorageIfNeed`, there is a potential deadlock here. +// Thread A: +// Get lock on SharedContext -> call UniversalPageStorageService::shutdown -> remove background tasks -> try get rwlock on the task +// Thread B: +// Get rwlock on task -> call a method on Context to get some object -> try to get lock on SharedContext +void Context::tryReleaseWriteNodePageStorageForTest() +{ + UniversalPageStorageServicePtr ps_write; + { + auto lock = getLock(); + if (shared->ps_write) + { + LOG_WARNING(shared->log, "Release GlobalUniversalPageStorage(WriteNode)."); + ps_write = shared->ps_write; + shared->ps_write = nullptr; + } + } + if (ps_write) + { + // call shutdown without lock + ps_write->shutdown(); + ps_write = nullptr; + } +} + SharedContextDisaggPtr Context::getSharedContextDisagg() const { RUNTIME_CHECK(shared->ctx_disagg != nullptr); // We always initialize the shared context in createGlobal() diff --git a/dbms/src/Interpreters/Context.h b/dbms/src/Interpreters/Context.h index 7961ca1e364..3d83faf71d0 100644 --- a/dbms/src/Interpreters/Context.h +++ b/dbms/src/Interpreters/Context.h @@ -429,6 +429,7 @@ class Context void initializeWriteNodePageStorageIfNeed(const PathPool & path_pool); UniversalPageStoragePtr getWriteNodePageStorage() const; + void tryReleaseWriteNodePageStorageForTest(); SharedContextDisaggPtr getSharedContextDisagg() const; diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.cpp b/dbms/src/Interpreters/ExpressionAnalyzer.cpp index 87cb96c9792..87b5d6d2481 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.cpp +++ b/dbms/src/Interpreters/ExpressionAnalyzer.cpp @@ -2144,84 +2144,18 @@ void ExpressionAnalyzer::addJoinAction(ExpressionActionsPtr & actions, bool only actions->add(ExpressionAction::ordinaryJoin(subquery_for_set.second.join, columns_added_by_join)); } -bool ExpressionAnalyzer::appendJoin(ExpressionActionsChain & chain, bool only_types) +bool ExpressionAnalyzer::appendJoin(ExpressionActionsChain &, bool) { assertSelect(); if (!select_query->join()) return false; - initChain(chain, source_columns); - ExpressionActionsChain::Step & step = chain.steps.back(); - - const auto & join_element = static_cast(*select_query->join()); - const auto & join_params = static_cast(*join_element.table_join); - const auto & table_to_join = static_cast(*join_element.table_expression); - - if (join_params.using_expression_list) - getRootActions(join_params.using_expression_list, only_types, false, step.actions); - - /// Two JOINs are not supported with the same subquery, but different USINGs. - auto join_hash = join_element.getTreeHash(); - - SubqueryForSet & subquery_for_set = subqueries_for_sets[toString(join_hash.first) + "_" + toString(join_hash.second)]; - - /// Special case - if table name is specified on the right of JOIN, then the table has the type Join (the previously prepared mapping). - /// TODO This syntax does not support specifying a database name. - if (table_to_join.database_and_table_name) - { - auto database_table = getDatabaseAndTableNameFromIdentifier(static_cast(*table_to_join.database_and_table_name)); - StoragePtr table = context.tryGetTable(database_table.first, database_table.second); - } - - if (!subquery_for_set.join) - { - SpillConfig build_spill_config(context.getTemporaryPath(), fmt::format("hash_join_0_build"), settings.max_cached_data_bytes_in_spiller, settings.max_spilled_rows_per_file, settings.max_spilled_bytes_per_file, context.getFileProvider()); - SpillConfig probe_spill_config(context.getTemporaryPath(), fmt::format("hash_join_0_probe"), settings.max_cached_data_bytes_in_spiller, settings.max_spilled_rows_per_file, settings.max_spilled_bytes_per_file, context.getFileProvider()); - JoinPtr join = std::make_shared( - join_key_names_left, - join_key_names_right, - join_params.kind, - join_params.strictness, - "" /*req_id=*/, - false /*enable_fine_grained_shuffle_*/, - 0 /*fine_grained_shuffle_count_*/, - settings.max_bytes_before_external_join, - build_spill_config, - probe_spill_config, - settings.join_restore_concurrency); - - Names required_joined_columns(join_key_names_right.begin(), join_key_names_right.end()); - for (const auto & name_type : columns_added_by_join) - required_joined_columns.push_back(name_type.name); - - /** For GLOBAL JOINs (in the case, for example, of the push method for executing GLOBAL subqueries), the following occurs - * - in the addExternalStorage function, the JOIN (SELECT ...) subquery is replaced with JOIN _data1, - * in the subquery_for_set object this subquery is exposed as source and the temporary table _data1 as the `table`. - * - this function shows the expression JOIN _data1. - */ - if (!subquery_for_set.source) - { - ASTPtr table; - if (table_to_join.database_and_table_name) - table = table_to_join.database_and_table_name; - else - table = table_to_join.subquery; - - auto interpreter = interpretSubquery(table, context, subquery_depth, required_joined_columns); - subquery_for_set.source = std::make_shared( - interpreter->getSampleBlock(), - [interpreter]() mutable { return interpreter->execute().in; }); - } - - /// TODO You do not need to set this up when JOIN is only needed on remote servers. - subquery_for_set.join = join; - subquery_for_set.join->initBuild(subquery_for_set.source->getHeader()); - } - - addJoinAction(step.actions, false); - - return true; + /// after https://github.com/pingcap/tiflash/pull/6650, join from TiFlash client + /// is no longer supported because the "waiting build finish" step has been moved + /// from Join::joinBlock() to HashJoinProbeInputStream::readImpl, so before support + /// HashJoinProbeInputStream in `ExpressionAnalyzer::appendJoin`, join is disabled. + throw Exception("Join is not supported"); } diff --git a/dbms/src/Interpreters/Settings.h b/dbms/src/Interpreters/Settings.h index 76b1b4cbb98..7bb73a6ba19 100644 --- a/dbms/src/Interpreters/Settings.h +++ b/dbms/src/Interpreters/Settings.h @@ -238,7 +238,7 @@ struct Settings M(SettingCompressionMethod, dt_compression_method, CompressionMethod::LZ4, "The method of data compression when writing.") \ M(SettingInt64, dt_compression_level, 1, "The compression level.") \ \ - M(SettingInt64, remote_checkpoint_interval_seconds, 60, "The interval of uploading checkpoint to the remote store. Unit is second.") \ + M(SettingInt64, remote_checkpoint_interval_seconds, 30, "The interval of uploading checkpoint to the remote store. Unit is second.") \ M(SettingInt64, remote_gc_interval_seconds, 3600, "The interval of running GC task on the remote store. Unit is second.") \ \ M(SettingUInt64, max_rows_in_set, 0, "Maximum size of the set (in number of elements) resulting from the execution of the IN section.") \ diff --git a/dbms/src/Interpreters/SharedContexts/Disagg.cpp b/dbms/src/Interpreters/SharedContexts/Disagg.cpp index edfd6dd5c5b..072fac88aa4 100644 --- a/dbms/src/Interpreters/SharedContexts/Disagg.cpp +++ b/dbms/src/Interpreters/SharedContexts/Disagg.cpp @@ -20,6 +20,7 @@ #include #include #include +#include namespace DB { @@ -91,4 +92,9 @@ void SharedContextDisagg::initRemoteDataStore(const FileProviderPtr & file_provi remote_data_store = std::make_shared(file_provider); } +void SharedContextDisagg::initFastAddPeerContext() +{ + fap_context = std::make_shared(); +} + } // namespace DB diff --git a/dbms/src/Interpreters/SharedContexts/Disagg.h b/dbms/src/Interpreters/SharedContexts/Disagg.h index 91daab20d0b..acc3354914d 100644 --- a/dbms/src/Interpreters/SharedContexts/Disagg.h +++ b/dbms/src/Interpreters/SharedContexts/Disagg.h @@ -29,6 +29,8 @@ namespace DB { +class FastAddPeerContext; +using FastAddPeerContextPtr = std::shared_ptr; /** * A shared context containing disaggregated mode related things. @@ -45,11 +47,14 @@ struct SharedContextDisagg : private boost::noncopyable DisaggregatedMode disaggregated_mode = DisaggregatedMode::None; - bool use_autoscaler = true; // TODO: remove this after AutoScaler is stable. Only meaningful in DisaggregatedComputeMode. + // Only meaningful in DisaggregatedComputeMode. + bool use_autoscaler = false; /// For both read node (downloading) and write node (uploading). DM::Remote::IDataStorePtr remote_data_store; + FastAddPeerContextPtr fap_context; + /// Only for write node. DM::Remote::WNDisaggSnapshotManagerPtr wn_snapshot_manager; @@ -81,6 +86,8 @@ struct SharedContextDisagg : private boost::noncopyable void initRemoteDataStore(const FileProviderPtr & file_provider, bool s3_enabled); + void initFastAddPeerContext(); + bool isDisaggregatedComputeMode() const { return disaggregated_mode == DisaggregatedMode::Compute; diff --git a/dbms/src/Server/BgStorageInit.cpp b/dbms/src/Server/BgStorageInit.cpp new file mode 100644 index 00000000000..c53953cf25a --- /dev/null +++ b/dbms/src/Server/BgStorageInit.cpp @@ -0,0 +1,101 @@ +// Copyright 2023 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 +#include +#include +#include +#include +#include +#include + +#include + +namespace DB +{ +void BgStorageInitHolder::waitUntilFinish() +{ + if (need_join) + { + LOG_INFO(Logger::get(), "Wait for storage init thread to join"); + init_thread->join(); + init_thread.reset(); + need_join = false; // join has been done + } + // else the job is done by not lazily init + // or has been detach +} + +void BgStorageInitHolder::start(Context & global_context, const LoggerPtr & log, bool lazily_init_store, bool is_s3_enabled) +{ + RUNTIME_CHECK_MSG( + lazily_init_store || !is_s3_enabled, + "When S3 enabled, lazily_init_store must be true. lazily_init_store={} s3_enabled={}", + lazily_init_store, + is_s3_enabled); + auto do_init_stores = [&global_context, &log] { + auto storages = global_context.getTMTContext().getStorages().getAllStorage(); + int init_cnt = 0; + int err_cnt = 0; + for (auto & [ks_table_id, storage] : storages) + { + // This will skip the init of storages that do not contain any data. TiFlash now sync the schema and + // create all tables regardless the table have define TiFlash replica or not, so there may be lots + // of empty tables in TiFlash. + // Note that we still need to init stores that contains data (defined by the stable dir of this storage + // is exist), or the data used size reported to PD is not correct. + const auto & [ks_id, table_id] = ks_table_id; + try + { + init_cnt += storage->initStoreIfDataDirExist() ? 1 : 0; + LOG_INFO(log, "Storage inited done, keyspace_id={} table_id={}", ks_id, table_id); + } + catch (...) + { + err_cnt++; + tryLogCurrentException(log, fmt::format("Storage inited fail, keyspace_id={} table_id={}", ks_id, table_id)); + } + } + LOG_INFO( + log, + "Storage inited finish. [total_count={}] [init_count={}] [error_count={}] [datatype_fullname_count={}]", + storages.size(), + init_cnt, + err_cnt, + DataTypeFactory::instance().getFullNameCacheSize()); + }; + + if (!lazily_init_store) + { + LOG_INFO(log, "Not lazily init store."); + need_join = false; + do_init_stores(); + } + + LOG_INFO(log, "Lazily init store."); + // apply the inited in another thread to shorten the start time of TiFlash + if (is_s3_enabled) + { + init_thread = std::make_unique(do_init_stores); + need_join = true; + } + else + { + init_thread = std::make_unique(do_init_stores); + init_thread->detach(); + need_join = false; + } +} + +} // namespace DB diff --git a/dbms/src/Server/BgStorageInit.h b/dbms/src/Server/BgStorageInit.h new file mode 100644 index 00000000000..0449d8e48f0 --- /dev/null +++ b/dbms/src/Server/BgStorageInit.h @@ -0,0 +1,41 @@ +// Copyright 2023 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. + +#pragma once + +#include +#include + +#include + +namespace DB +{ +struct BgStorageInitHolder +{ + bool need_join = false; + std::unique_ptr init_thread; + + void start(Context & global_context, const LoggerPtr & log, bool lazily_init_store, bool is_s3_enabled); + + // wait until finish if need + void waitUntilFinish(); + + // Exception safe for joining the init_thread + ~BgStorageInitHolder() + { + waitUntilFinish(); + } +}; + +} // namespace DB diff --git a/dbms/src/Server/CMakeLists.txt b/dbms/src/Server/CMakeLists.txt index 327d5b75cac..3fb10fbfbb4 100644 --- a/dbms/src/Server/CMakeLists.txt +++ b/dbms/src/Server/CMakeLists.txt @@ -46,6 +46,7 @@ add_library (tiflash-server-lib Server.cpp StatusFile.cpp TCPHandler.cpp + BgStorageInit.cpp StorageConfigParser.cpp UserConfigParser.cpp RaftConfigParser.cpp) diff --git a/dbms/src/Server/Server.cpp b/dbms/src/Server/Server.cpp index 8ce87bd40a5..464c8ccadb4 100644 --- a/dbms/src/Server/Server.cpp +++ b/dbms/src/Server/Server.cpp @@ -47,7 +47,7 @@ #include #include #include -#include +#include #include #include #include @@ -59,6 +59,7 @@ #include #include #include +#include #include #include #include @@ -517,68 +518,6 @@ struct RaftStoreProxyRunner : boost::noncopyable const LoggerPtr & log; }; -std::unique_ptr initStores(Context & global_context, const LoggerPtr & log, bool lazily_init_store, EngineStoreServerWrap * tiflash_instance_wrap) -{ - // If `tiflash_instance_wrap` is not nullptr, S3 is enabled. - // We must wait for tiflash-proxy's initializtion finished before initialzing DeltaMergeStores, - // because we need store_id to scan dmfiles from S3. - RUNTIME_CHECK_MSG(lazily_init_store || tiflash_instance_wrap == nullptr, "When S3 enabled, lazily_init_store must be true."); - auto wait_proxy = [](EngineStoreServerWrap * tiflash_instance_wrap) { - while (tiflash_instance_wrap->proxy_helper->getProxyStatus() == RaftProxyStatus::Idle) - { - std::this_thread::sleep_for(std::chrono::milliseconds(200)); - } - auto status = tiflash_instance_wrap->proxy_helper->getProxyStatus(); - RUNTIME_CHECK_MSG(status == RaftProxyStatus::Running, "RaftProxyStatus::{}", magic_enum::enum_name(status)); - }; - - auto do_init_stores = [&global_context, &log, tiflash_instance_wrap, wait_proxy]() { - if (tiflash_instance_wrap != nullptr) - { - wait_proxy(tiflash_instance_wrap); - } - auto storages = global_context.getTMTContext().getStorages().getAllStorage(); - int init_cnt = 0; - int err_cnt = 0; - for (auto & [table_id, storage] : storages) - { - // This will skip the init of storages that do not contain any data. TiFlash now sync the schema and - // create all tables regardless the table have define TiFlash replica or not, so there may be lots - // of empty tables in TiFlash. - // Note that we still need to init stores that contains data (defined by the stable dir of this storage - // is exist), or the data used size reported to PD is not correct. - try - { - init_cnt += storage->initStoreIfDataDirExist() ? 1 : 0; - LOG_INFO(log, "Storage inited done [table_id={}]", table_id); - } - catch (...) - { - err_cnt++; - tryLogCurrentException(log, fmt::format("Storage inited fail, [table_id={}]", table_id)); - } - } - LOG_INFO( - log, - "Storage inited finish. [total_count={}] [init_count={}] [error_count={}] [datatype_fullname_count={}]", - storages.size(), - init_cnt, - err_cnt, - DataTypeFactory::instance().getFullNameCacheSize()); - }; - if (lazily_init_store) - { - LOG_INFO(log, "Lazily init store."); - // apply the inited in another thread to shorten the start time of TiFlash - return std::make_unique(do_init_stores); - } - else - { - LOG_INFO(log, "Not lazily init store."); - do_init_stores(); - return nullptr; - } -} class Server::TcpHttpServersHolder { @@ -858,32 +797,76 @@ class Server::TcpHttpServersHolder // By default init global thread pool by hardware_concurrency // Later we will adjust it by `adjustThreadPoolSize` -void initThreadPool() +void initThreadPool(Poco::Util::LayeredConfiguration & config) { size_t default_num_threads = std::max(4UL, 2 * std::thread::hardware_concurrency()); + + // Note: Global Thread Pool must be larger than sub thread pools. GlobalThreadPool::initialize( - /*max_threads*/ default_num_threads, - /*max_free_threads*/ default_num_threads / 2, - /*queue_size*/ default_num_threads * 2); - IOThreadPool::initialize( - /*max_threads*/ default_num_threads, - /*max_free_threads*/ default_num_threads / 2, - /*queue_size*/ default_num_threads * 2); + /*max_threads*/ default_num_threads * 20, + /*max_free_threads*/ default_num_threads, + /*queue_size*/ default_num_threads * 8); + + auto disaggregated_mode = getDisaggregatedMode(config); + if (disaggregated_mode == DisaggregatedMode::Compute) + { + RNPagePreparerPool::initialize( + /*max_threads*/ default_num_threads, + /*max_free_threads*/ default_num_threads / 2, + /*queue_size*/ default_num_threads * 2); + RNRemoteReadTaskPool::initialize( + /*max_threads*/ default_num_threads, + /*max_free_threads*/ default_num_threads / 2, + /*queue_size*/ default_num_threads * 2); + } + + if (disaggregated_mode == DisaggregatedMode::Compute || disaggregated_mode == DisaggregatedMode::Storage) + { + DataStoreS3Pool::initialize( + /*max_threads*/ default_num_threads, + /*max_free_threads*/ default_num_threads / 2, + /*queue_size*/ default_num_threads * 2); + S3FileCachePool::initialize( + /*max_threads*/ default_num_threads, + /*max_free_threads*/ default_num_threads / 2, + /*queue_size*/ default_num_threads * 2); + } } void adjustThreadPoolSize(const Settings & settings, size_t logical_cores) { // TODO: make BackgroundPool/BlockableBackgroundPool/DynamicThreadPool spawned from `GlobalThreadPool` size_t max_io_thread_count = std::ceil(settings.io_thread_count_scale * logical_cores); - // Currently, `GlobalThreadPool` is only used by `IOThreadPool`, so they have the same number of threads. - GlobalThreadPool::instance().setMaxThreads(max_io_thread_count); - GlobalThreadPool::instance().setMaxFreeThreads(max_io_thread_count / 2); - GlobalThreadPool::instance().setQueueSize(max_io_thread_count * 2); + // Note: Global Thread Pool must be larger than sub thread pools. + GlobalThreadPool::instance().setMaxThreads(max_io_thread_count * 20); + GlobalThreadPool::instance().setMaxFreeThreads(max_io_thread_count); + GlobalThreadPool::instance().setQueueSize(max_io_thread_count * 8); - IOThreadPool::instance->setMaxFreeThreads(max_io_thread_count); - IOThreadPool::instance->setMaxFreeThreads(max_io_thread_count / 2); - IOThreadPool::instance->setQueueSize(max_io_thread_count * 2); + if (RNPagePreparerPool::instance) + { + RNPagePreparerPool::instance->setMaxThreads(max_io_thread_count); + RNPagePreparerPool::instance->setMaxFreeThreads(max_io_thread_count / 2); + RNPagePreparerPool::instance->setQueueSize(max_io_thread_count * 2); + } + if (RNRemoteReadTaskPool::instance) + { + RNRemoteReadTaskPool::instance->setMaxThreads(max_io_thread_count); + RNRemoteReadTaskPool::instance->setMaxFreeThreads(max_io_thread_count / 2); + RNRemoteReadTaskPool::instance->setQueueSize(max_io_thread_count * 2); + } + if (DataStoreS3Pool::instance) + { + DataStoreS3Pool::instance->setMaxThreads(max_io_thread_count); + DataStoreS3Pool::instance->setMaxFreeThreads(max_io_thread_count / 2); + DataStoreS3Pool::instance->setQueueSize(max_io_thread_count * 2); + } + if (S3FileCachePool::instance) + { + S3FileCachePool::instance->setMaxThreads(max_io_thread_count); + S3FileCachePool::instance->setMaxFreeThreads(max_io_thread_count / 2); + S3FileCachePool::instance->setQueueSize(max_io_thread_count * 2); + } } int Server::main(const std::vector & /*args*/) @@ -921,7 +904,7 @@ int Server::main(const std::vector & /*args*/) // Later we may create thread pool from GlobalThreadPool // init it before other components - initThreadPool(); + initThreadPool(config()); TiFlashErrorRegistry::instance(); // This invocation is for initializing @@ -932,14 +915,30 @@ int Server::main(const std::vector & /*args*/) // "0" by default, means no quota, the actual disk capacity is used. size_t global_capacity_quota = 0; std::tie(global_capacity_quota, storage_config) = TiFlashStorageConfig::parseSettings(config(), log); - if (storage_config.format_version) + if (storage_config.format_version != 0) { + if (storage_config.s3_config.isS3Enabled() && storage_config.format_version != STORAGE_FORMAT_V5.identifier) + { + LOG_WARNING(log, "'storage.format_version' must be set to 5 when S3 is enabled!"); + throw Exception("'storage.format_version' must be set to 5 when S3 is enabled!"); + } setStorageFormat(storage_config.format_version); LOG_INFO(log, "Using format_version={} (explicit storage format detected).", storage_config.format_version); } else { - LOG_INFO(log, "Using format_version={} (default settings).", STORAGE_FORMAT_CURRENT.identifier); + if (storage_config.s3_config.isS3Enabled()) + { + // If the user does not explicitly set format_version in the config file but + // enables S3, then we set up a proper format version to support S3. + setStorageFormat(5); + LOG_INFO(log, "Using format_version={} (infer by S3 is enabled).", STORAGE_FORMAT_V5.identifier); + } + else + { + // Use the default settings + LOG_INFO(log, "Using format_version={} (default settings).", STORAGE_FORMAT_CURRENT.identifier); + } } LOG_INFO(log, "Using api_version={}", storage_config.api_version); @@ -1067,11 +1066,7 @@ int Server::main(const std::vector & /*args*/) } S3::ClientFactory::instance().init(storage_config.s3_config); } - if (const auto & config = storage_config.remote_cache_config; config.isCacheEnabled()) - { - config.initCacheDir(); - FileCache::initialize(config.getDTFileCacheDir(), config.getDTFileCapacity(), config.dtfile_level, config.dtfile_cache_min_age_seconds); - } + global_context->getSharedContextDisagg()->initRemoteDataStore(global_context->getFileProvider(), storage_config.s3_config.isS3Enabled()); global_context->initializePathCapacityMetric( // @@ -1089,6 +1084,11 @@ int Server::main(const std::vector & /*args*/) storage_config.kvstore_data_path, // global_context->getPathCapacity(), global_context->getFileProvider()); + if (const auto & config = storage_config.remote_cache_config; config.isCacheEnabled()) + { + config.initCacheDir(); + FileCache::initialize(global_context->getPathCapacity(), config); + } /// Determining PageStorage run mode based on current files on disk and storage config. /// Do it as early as possible after loading storage config. @@ -1230,8 +1230,10 @@ int Server::main(const std::vector & /*args*/) LOG_INFO(log, "Global PageStorage run mode is {}", magic_enum::enum_name(global_context->getPageStorageRunMode())); } - if (global_context->getSharedContextDisagg()->isDisaggregatedStorageMode() - || global_context->getSharedContextDisagg()->notDisaggregatedMode()) + // Only when this node is disagg compute node and autoscaler is enabled, we don't need the WriteNodePageStorage instance + // Disagg compute node without autoscaler still need this instance for proxy's data + if (!(global_context->getSharedContextDisagg()->isDisaggregatedComputeMode() + && global_context->getSharedContextDisagg()->use_autoscaler)) { global_context->initializeWriteNodePageStorageIfNeed(global_context->getPathPool()); } @@ -1239,6 +1241,7 @@ int Server::main(const std::vector & /*args*/) if (global_context->getSharedContextDisagg()->isDisaggregatedStorageMode()) { global_context->getSharedContextDisagg()->initWriteNodeSnapManager(); + global_context->getSharedContextDisagg()->initFastAddPeerContext(); } if (global_context->getSharedContextDisagg()->isDisaggregatedComputeMode()) @@ -1259,7 +1262,11 @@ int Server::main(const std::vector & /*args*/) } else { - LOG_INFO(log, fmt::format("Detected memory capacity {} bytes, you have config `max_memory_usage_for_all_queries` to {}, finally limit to {} bytes.", server_info.memory_info.capacity, settings.max_memory_usage_for_all_queries.toString(), settings.max_memory_usage_for_all_queries.getActualBytes(server_info.memory_info.capacity))); + LOG_INFO(log, + "Detected memory capacity {} bytes, you have config `max_memory_usage_for_all_queries` to {}, finally limit to {} bytes.", + server_info.memory_info.capacity, + settings.max_memory_usage_for_all_queries.toString(), + settings.max_memory_usage_for_all_queries.getActualBytes(server_info.memory_info.capacity)); } /// Initialize main config reloader. @@ -1374,7 +1381,7 @@ int Server::main(const std::vector & /*args*/) // Load remaining databases loadMetadata(*global_context); LOG_DEBUG(log, "Load metadata done."); - std::unique_ptr init_stores_thread; + BgStorageInitHolder bg_init_stores; if (!global_context->getSharedContextDisagg()->isDisaggregatedComputeMode()) { /// Then, sync schemas with TiDB, and initialize schema sync service. @@ -1403,13 +1410,18 @@ int Server::main(const std::vector & /*args*/) LOG_DEBUG(log, "Sync schemas done."); } - init_stores_thread = initStores(*global_context, log, storage_config.lazily_init_store, storage_config.s3_config.isS3Enabled() ? &tiflash_instance_wrap : nullptr); - - // After schema synced, set current database. - global_context->setCurrentDatabase(default_database); + bg_init_stores.start( + *global_context, + log, + storage_config.lazily_init_store, + storage_config.s3_config.isS3Enabled()); + // init schema sync service with tidb global_context->initializeSchemaSyncService(); } + // set default database for ch-client + global_context->setCurrentDatabase(default_database); + CPUAffinityManager::initCPUAffinityManager(config()); LOG_INFO(log, "CPUAffinity: {}", CPUAffinityManager::getInstance().toString()); SCOPE_EXIT({ @@ -1481,19 +1493,11 @@ int Server::main(const std::vector & /*args*/) GRPCCompletionQueuePool::global_instance = std::make_unique(size); } - if (init_stores_thread != nullptr) - { - if (storage_config.s3_config.isS3Enabled()) - { - // If S3 enabled, wait for all DeltaMergeStores' initialization - // before this instance can accept requests. - init_stores_thread->join(); - } - else - { - init_stores_thread->detach(); - } - } + // If S3 enabled, wait for all DeltaMergeStores' initialization + // before this instance can accept requests. + // Else it just do nothing. + bg_init_stores.waitUntilFinish(); + /// Then, startup grpc server to serve raft and/or flash services. FlashGrpcServerHolder flash_grpc_server_holder(this->context(), this->config(), raft_config, log); @@ -1560,23 +1564,36 @@ int Server::main(const std::vector & /*args*/) tiflash_instance_wrap.tmt = &tmt_context; LOG_INFO(log, "Let tiflash proxy start all services"); + // Set tiflash instance status to running, then wait for proxy enter running status tiflash_instance_wrap.status = EngineStoreServerStatus::Running; while (tiflash_instance_wrap.proxy_helper->getProxyStatus() == RaftProxyStatus::Idle) std::this_thread::sleep_for(std::chrono::milliseconds(200)); // proxy update store-id before status set `RaftProxyStatus::Running` assert(tiflash_instance_wrap.proxy_helper->getProxyStatus() == RaftProxyStatus::Running); - LOG_INFO(log, "store_id={}, tiflash proxy is ready to serve, try to wake up all regions' leader", tmt_context.getKVStore()->getStoreID(std::memory_order_seq_cst)); - size_t runner_cnt = config().getUInt("flash.read_index_runner_count", 1); // if set 0, DO NOT enable read-index worker - auto & kvstore_ptr = tmt_context.getKVStore(); - kvstore_ptr->initReadIndexWorkers( - [&]() { - // get from tmt context - return std::chrono::milliseconds(tmt_context.readIndexWorkerTick()); - }, - /*running thread count*/ runner_cnt); - tmt_context.getKVStore()->asyncRunReadIndexWorkers(); - WaitCheckRegionReady(tmt_context, *kvstore_ptr, terminate_signals_counter); + if (global_context->getSharedContextDisagg()->isDisaggregatedComputeMode()) + { + // compute node do not need to handle read index + LOG_INFO(log, "store_id={}, tiflash proxy is ready to serve", tmt_context.getKVStore()->getStoreID(std::memory_order_seq_cst)); + } + else + { + LOG_INFO(log, "store_id={}, tiflash proxy is ready to serve, try to wake up all regions' leader", tmt_context.getKVStore()->getStoreID(std::memory_order_seq_cst)); + + size_t runner_cnt = config().getUInt("flash.read_index_runner_count", 1); // if set 0, DO NOT enable read-index worker + if (runner_cnt > 0) + { + auto & kvstore_ptr = tmt_context.getKVStore(); + kvstore_ptr->initReadIndexWorkers( + [&]() { + // get from tmt context + return std::chrono::milliseconds(tmt_context.readIndexWorkerTick()); + }, + /*running thread count*/ runner_cnt); + tmt_context.getKVStore()->asyncRunReadIndexWorkers(); + WaitCheckRegionReady(tmt_context, *kvstore_ptr, terminate_signals_counter); + } + } } SCOPE_EXIT({ if (!proxy_conf.is_proxy_runnable) diff --git a/dbms/src/Server/StorageConfigParser.cpp b/dbms/src/Server/StorageConfigParser.cpp index 563bded1de4..ce56f9dca02 100644 --- a/dbms/src/Server/StorageConfigParser.cpp +++ b/dbms/src/Server/StorageConfigParser.cpp @@ -27,6 +27,7 @@ #endif #include +#include #include #include #include @@ -50,11 +51,11 @@ namespace ErrorCodes extern const int INVALID_CONFIG_PARAMETER; } // namespace ErrorCodes -static std::string getCanonicalPath(std::string path) +static std::string getCanonicalPath(std::string path, std::string_view hint = "path") { Poco::trimInPlace(path); if (path.empty()) - throw Exception("path configuration parameter is empty"); + throw Exception(ErrorCodes::INVALID_CONFIG_PARAMETER, "'{}' configuration parameter is empty", hint); if (path.back() != '/') path += '/'; return path; @@ -211,10 +212,7 @@ void TiFlashStorageConfig::parseMisc(const String & storage_section, const Logge readConfig(table, "format_version", format_version); - if (auto version = table->get_qualified_as("api_version"); version) - { - api_version = *version; - } + readConfig(table, "api_version", api_version); auto get_bool_config_or_default = [&](const String & name, bool default_value) { #ifndef NDEBUG @@ -553,7 +551,7 @@ void StorageS3Config::parse(const String & content, const LoggerPtr & log) readConfig(table, "request_timeout_ms", request_timeout_ms); RUNTIME_CHECK(request_timeout_ms > 0); readConfig(table, "root", root); - RUNTIME_CHECK(!root.empty()); + getCanonicalPath(root, "root"); // ensure not empty and ends with '/' auto read_s3_auth_info_from_env = [&]() { access_key_id = Poco::Environment::get(S3_ACCESS_KEY_ID, /*default*/ ""); @@ -574,10 +572,12 @@ void StorageS3Config::parse(const String & content, const LoggerPtr & log) LOG_INFO( log, - "endpoint={} bucket={} max_connections={} connection_timeout_ms={} " + "endpoint={} bucket={} root={} " + "max_connections={} connection_timeout_ms={} " "request_timeout_ms={} access_key_id_size={} secret_access_key_size={}", endpoint, bucket, + root, max_connections, connection_timeout_ms, request_timeout_ms, diff --git a/dbms/src/Storages/DeltaMerge/ColumnFile/ColumnFileBig.cpp b/dbms/src/Storages/DeltaMerge/ColumnFile/ColumnFileBig.cpp index 2cb7f092f22..0380528f239 100644 --- a/dbms/src/Storages/DeltaMerge/ColumnFile/ColumnFileBig.cpp +++ b/dbms/src/Storages/DeltaMerge/ColumnFile/ColumnFileBig.cpp @@ -13,14 +13,18 @@ // limitations under the License. #include +#include #include #include #include +#include #include #include #include +#include #include + namespace DB { namespace DM @@ -86,14 +90,76 @@ ColumnFilePersistedPtr ColumnFileBig::deserializeMetadata(const DMContext & cont readIntBinary(valid_bytes, buf); auto file_id = context.storage_pool->dataReader()->getNormalPageId(file_page_id); - auto file_parent_path = context.path_pool->getStableDiskDelegator().getDTFilePath(file_id); - - auto dmfile = DMFile::restore(context.db_context.getFileProvider(), file_id, file_page_id, file_parent_path, DMFile::ReadMetaMode::all()); + String file_parent_path; + DMFilePtr dmfile; + auto remote_data_store = context.db_context.getSharedContextDisagg()->remote_data_store; + if (remote_data_store) + { + auto wn_ps = context.db_context.getWriteNodePageStorage(); + auto full_page_id = UniversalPageIdFormat::toFullPageId(UniversalPageIdFormat::toFullPrefix(StorageType::Data, context.storage_pool->getNamespaceId()), file_page_id); + auto remote_data_location = wn_ps->getCheckpointLocation(full_page_id); + const auto & lock_key_view = S3::S3FilenameView::fromKey(*(remote_data_location->data_file_id)); + auto dtfile_key = lock_key_view.asDataFile(); + auto file_oid = dtfile_key.getDMFileOID(); + auto prepared = remote_data_store->prepareDMFile(file_oid, file_page_id); + dmfile = prepared->restore(DMFile::ReadMetaMode::all()); + } + else + { + file_parent_path = context.path_pool->getStableDiskDelegator().getDTFilePath(file_id); + dmfile = DMFile::restore(context.db_context.getFileProvider(), file_id, file_page_id, file_parent_path, DMFile::ReadMetaMode::all()); + } auto * dp_file = new ColumnFileBig(dmfile, valid_rows, valid_bytes, segment_range); return std::shared_ptr(dp_file); } +ColumnFilePersistedPtr ColumnFileBig::createFromCheckpoint(DMContext & context, // + const RowKeyRange & target_range, + ReadBuffer & buf, + UniversalPageStoragePtr temp_ps, + TableID ns_id, + WriteBatches & wbs) +{ + UInt64 file_page_id; + size_t valid_rows, valid_bytes; + + readIntBinary(file_page_id, buf); + readIntBinary(valid_rows, buf); + readIntBinary(valid_bytes, buf); + + // get target dtfile s3 key + auto remote_page_id = UniversalPageIdFormat::toFullPageId(UniversalPageIdFormat::toFullPrefix(StorageType::Data, ns_id), file_page_id); + auto remote_data_location = temp_ps->getCheckpointLocation(remote_page_id); + auto remote_file_id = temp_ps->getNormalPageId(remote_page_id); + auto file_id = UniversalPageIdFormat::getU64ID(remote_file_id); + + const auto & lock_key_view = S3::S3FilenameView::fromKey(*(remote_data_location->data_file_id)); + S3::DMFileOID file_oid{ + .store_id = lock_key_view.store_id, + .table_id = ns_id, + .file_id = file_id}; + RUNTIME_CHECK(lock_key_view.asDataFile().toFullKey() == S3::S3Filename::fromDMFileOID(file_oid).toFullKey()); + + auto data_key = S3::S3Filename::fromDMFileOID(file_oid).toFullKey(); + auto delegator = context.path_pool->getStableDiskDelegator(); + auto & storage_pool = context.storage_pool; + auto new_local_page_id = storage_pool->newDataPageIdForDTFile(delegator, __PRETTY_FUNCTION__); + PS::V3::CheckpointLocation loc{ + .data_file_id = std::make_shared(data_key), + .offset_in_file = 0, + .size_in_file = 0, + }; + wbs.data.putRemoteExternal(new_local_page_id, loc); + + auto parent_path = S3::S3Filename::fromTableID(lock_key_view.store_id, ns_id).toFullKeyWithPrefix(); + auto new_dmfile = DMFile::restore(context.db_context.getFileProvider(), file_id, new_local_page_id, parent_path, DMFile::ReadMetaMode::all()); + wbs.writeLogAndData(); + new_dmfile->enableGC(); + auto * dp_file = new ColumnFileBig(new_dmfile, valid_rows, valid_bytes, target_range); + return std::shared_ptr(dp_file); +} + void ColumnFileBigReader::initStream() { if (file_stream) diff --git a/dbms/src/Storages/DeltaMerge/ColumnFile/ColumnFileBig.h b/dbms/src/Storages/DeltaMerge/ColumnFile/ColumnFileBig.h index 9b4293296c9..cf0c8080626 100644 --- a/dbms/src/Storages/DeltaMerge/ColumnFile/ColumnFileBig.h +++ b/dbms/src/Storages/DeltaMerge/ColumnFile/ColumnFileBig.h @@ -88,6 +88,13 @@ class ColumnFileBig : public ColumnFilePersisted const RowKeyRange & segment_range, ReadBuffer & buf); + static ColumnFilePersistedPtr createFromCheckpoint(DMContext & context, // + const RowKeyRange & target_range, + ReadBuffer & buf, + UniversalPageStoragePtr temp_ps, + TableID ns_id, + WriteBatches & wbs); + String toString() const override { String s = "{big_file,rows:" + DB::toString(getRows()) // diff --git a/dbms/src/Storages/DeltaMerge/ColumnFile/ColumnFilePersisted.cpp b/dbms/src/Storages/DeltaMerge/ColumnFile/ColumnFilePersisted.cpp index f575d4c4418..e9a22a28eaf 100644 --- a/dbms/src/Storages/DeltaMerge/ColumnFile/ColumnFilePersisted.cpp +++ b/dbms/src/Storages/DeltaMerge/ColumnFile/ColumnFilePersisted.cpp @@ -131,5 +131,30 @@ ColumnFilePersisteds deserializeSavedColumnFiles(const DMContext & context, cons } return column_files; } + +ColumnFilePersisteds createColumnFilesFromCheckpoint( // + DMContext & context, + const RowKeyRange & segment_range, + ReadBuffer & buf, + UniversalPageStoragePtr temp_ps, + TableID ns_id, + WriteBatches & wbs) +{ + // Check binary version + DeltaFormat::Version version; + readIntBinary(version, buf); + + ColumnFilePersisteds column_files; + switch (version) + { + case DeltaFormat::V3: + column_files = createColumnFilesInV3FormatFromCheckpoint(context, segment_range, buf, temp_ps, ns_id, wbs); + break; + default: + throw Exception("Unexpected delta value version: " + DB::toString(version) + ", latest version: " + DB::toString(DeltaFormat::V3), + ErrorCodes::LOGICAL_ERROR); + } + return column_files; +} } // namespace DM } // namespace DB diff --git a/dbms/src/Storages/DeltaMerge/ColumnFile/ColumnFilePersisted.h b/dbms/src/Storages/DeltaMerge/ColumnFile/ColumnFilePersisted.h index 241ab351c4f..43707e178ea 100644 --- a/dbms/src/Storages/DeltaMerge/ColumnFile/ColumnFilePersisted.h +++ b/dbms/src/Storages/DeltaMerge/ColumnFile/ColumnFilePersisted.h @@ -19,6 +19,8 @@ namespace DB { +class UniversalPageStorage; +using UniversalPageStoragePtr = std::shared_ptr; namespace DM { struct WriteBatches; @@ -48,11 +50,26 @@ void serializeSavedColumnFiles(WriteBuffer & buf, const ColumnFilePersisteds & c /// Recreate column file instances from buf. ColumnFilePersisteds deserializeSavedColumnFiles(const DMContext & context, const RowKeyRange & segment_range, ReadBuffer & buf); +ColumnFilePersisteds createColumnFilesFromCheckpoint( // + DMContext & context, + const RowKeyRange & segment_range, + ReadBuffer & buf, + UniversalPageStoragePtr temp_ps, + TableID ns_id, + WriteBatches & wbs); + void serializeSavedColumnFilesInV2Format(WriteBuffer & buf, const ColumnFilePersisteds & column_files); ColumnFilePersisteds deserializeSavedColumnFilesInV2Format(const DMContext & context, ReadBuffer & buf, UInt64 version); void serializeSavedColumnFilesInV3Format(WriteBuffer & buf, const ColumnFilePersisteds & column_files); ColumnFilePersisteds deserializeSavedColumnFilesInV3Format(const DMContext & context, const RowKeyRange & segment_range, ReadBuffer & buf); +ColumnFilePersisteds createColumnFilesInV3FormatFromCheckpoint( // + DMContext & context, + const RowKeyRange & segment_range, + ReadBuffer & buf, + UniversalPageStoragePtr temp_ps, + TableID ns_id, + WriteBatches & wbs); } // namespace DM } // namespace DB diff --git a/dbms/src/Storages/DeltaMerge/ColumnFile/ColumnFileSetSnapshot.h b/dbms/src/Storages/DeltaMerge/ColumnFile/ColumnFileSetSnapshot.h index 781af5a6dc5..71f19763b44 100644 --- a/dbms/src/Storages/DeltaMerge/ColumnFile/ColumnFileSetSnapshot.h +++ b/dbms/src/Storages/DeltaMerge/ColumnFile/ColumnFileSetSnapshot.h @@ -103,7 +103,7 @@ class ColumnFileSetSnapshot : public std::enable_shared_from_this #include #include +#include #include @@ -174,6 +175,30 @@ ColumnFilePersistedPtr ColumnFileTiny::deserializeMetadata(const DMContext & con return std::make_shared(schema, rows, bytes, data_page_id); } +std::tuple ColumnFileTiny::createFromCheckpoint(const DMContext & context, ReadBuffer & buf, UniversalPageStoragePtr temp_ps, const BlockPtr & last_schema, TableID ns_id, WriteBatches & wbs) +{ + auto schema = deserializeSchema(buf); + if (!schema) + schema = last_schema; + RUNTIME_CHECK(schema != nullptr); + + PageIdU64 data_page_id; + size_t rows, bytes; + + readIntBinary(data_page_id, buf); + readIntBinary(rows, buf); + readIntBinary(bytes, buf); + auto new_cf_id = context.storage_pool->newLogPageId(); + auto remote_page_id = UniversalPageIdFormat::toFullPageId(UniversalPageIdFormat::toFullPrefix(StorageType::Log, ns_id), data_page_id); + auto remote_data_location = temp_ps->getCheckpointLocation(remote_page_id); + RUNTIME_CHECK(remote_data_location.has_value()); + auto entry = temp_ps->getEntry(remote_page_id); + wbs.log.putRemotePage(new_cf_id, 0, *remote_data_location, std::move(entry.field_offsets)); + + auto column_file_schema = std::make_shared(*schema); + return {std::make_shared(column_file_schema, rows, bytes, new_cf_id), std::move(schema)}; +} + Block ColumnFileTiny::readBlockForMinorCompaction(const PageReader & page_reader) const { if (cache) diff --git a/dbms/src/Storages/DeltaMerge/ColumnFile/ColumnFileTiny.h b/dbms/src/Storages/DeltaMerge/ColumnFile/ColumnFileTiny.h index 5013518e760..164c81f8ef2 100644 --- a/dbms/src/Storages/DeltaMerge/ColumnFile/ColumnFileTiny.h +++ b/dbms/src/Storages/DeltaMerge/ColumnFile/ColumnFileTiny.h @@ -119,6 +119,8 @@ class ColumnFileTiny : public ColumnFilePersisted static ColumnFilePersistedPtr deserializeMetadata(const DMContext & context, ReadBuffer & buf, ColumnFileSchemaPtr & last_schema); + static std::tuple createFromCheckpoint(const DMContext & context, ReadBuffer & buf, UniversalPageStoragePtr temp_ps, const BlockPtr & last_schema, TableID ns_id, WriteBatches & wbs); + bool mayBeFlushedFrom(ColumnFile * from_file) const override { // The current ColumnFileTiny may come from a ColumnFileInMemory (which contains data in memory) diff --git a/dbms/src/Storages/DeltaMerge/ColumnFile/ColumnFile_V3.cpp b/dbms/src/Storages/DeltaMerge/ColumnFile/ColumnFile_V3.cpp index cb7378c5d24..29dfb0f9229 100644 --- a/dbms/src/Storages/DeltaMerge/ColumnFile/ColumnFile_V3.cpp +++ b/dbms/src/Storages/DeltaMerge/ColumnFile/ColumnFile_V3.cpp @@ -97,5 +97,46 @@ ColumnFilePersisteds deserializeSavedColumnFilesInV3Format(const DMContext & con return column_files; } +ColumnFilePersisteds createColumnFilesInV3FormatFromCheckpoint( // + DMContext & context, + const RowKeyRange & segment_range, + ReadBuffer & buf, + UniversalPageStoragePtr temp_ps, + TableID ns_id, + WriteBatches & wbs) +{ + size_t column_file_count; + readIntBinary(column_file_count, buf); + ColumnFilePersisteds column_files; + column_files.reserve(column_file_count); + BlockPtr last_schema; + for (size_t i = 0; i < column_file_count; ++i) + { + std::underlying_type::type column_file_type; + readIntBinary(column_file_type, buf); + ColumnFilePersistedPtr column_file; + switch (column_file_type) + { + case ColumnFile::Type::DELETE_RANGE: + column_file = ColumnFileDeleteRange::deserializeMetadata(buf); + break; + case ColumnFile::Type::TINY_FILE: + { + std::tie(column_file, last_schema) = ColumnFileTiny::createFromCheckpoint(context, buf, temp_ps, last_schema, ns_id, wbs); + break; + } + case ColumnFile::Type::BIG_FILE: + { + column_file = ColumnFileBig::createFromCheckpoint(context, segment_range, buf, temp_ps, ns_id, wbs); + break; + } + default: + throw Exception("Unexpected column file type: " + DB::toString(column_file_type), ErrorCodes::LOGICAL_ERROR); + } + column_files.emplace_back(std::move(column_file)); + } + return column_files; +} + } // namespace DM } // namespace DB diff --git a/dbms/src/Storages/DeltaMerge/DMContext.cpp b/dbms/src/Storages/DeltaMerge/DMContext.cpp index e33746c046e..2a5a4fa876d 100644 --- a/dbms/src/Storages/DeltaMerge/DMContext.cpp +++ b/dbms/src/Storages/DeltaMerge/DMContext.cpp @@ -17,7 +17,6 @@ namespace DB::DM { - WriteLimiterPtr DMContext::getWriteLimiter() const { return db_context.getWriteLimiter(); diff --git a/dbms/src/Storages/DeltaMerge/DMContext.h b/dbms/src/Storages/DeltaMerge/DMContext.h index 0362e84df4a..c63f2987b31 100644 --- a/dbms/src/Storages/DeltaMerge/DMContext.h +++ b/dbms/src/Storages/DeltaMerge/DMContext.h @@ -100,7 +100,7 @@ struct DMContext : private boost::noncopyable bool is_common_handle_, size_t rowkey_column_size_, const DB::Settings & settings, - const ScanContextPtr & scan_context_ = std::make_shared(), + const ScanContextPtr scan_context_ = nullptr, const String & tracing_id_ = "") : db_context(db_context_) , path_pool(path_pool_) @@ -125,7 +125,7 @@ struct DMContext : private boost::noncopyable , enable_relevant_place(settings.dt_enable_relevant_place) , enable_skippable_place(settings.dt_enable_skippable_place) , tracing_id(tracing_id_) - , scan_context(scan_context_) + , scan_context(scan_context_ ? scan_context_ : std::make_shared()) { } diff --git a/dbms/src/Storages/DeltaMerge/Delta/ColumnFilePersistedSet.cpp b/dbms/src/Storages/DeltaMerge/Delta/ColumnFilePersistedSet.cpp index 5fd66750d75..9fae9f45200 100644 --- a/dbms/src/Storages/DeltaMerge/Delta/ColumnFilePersistedSet.cpp +++ b/dbms/src/Storages/DeltaMerge/Delta/ColumnFilePersistedSet.cpp @@ -20,6 +20,7 @@ #include #include #include +#include #include #include @@ -96,6 +97,28 @@ ColumnFilePersistedSetPtr ColumnFilePersistedSet::restore( // return std::make_shared(id, column_files); } +ColumnFilePersistedSetPtr ColumnFilePersistedSet::createFromCheckpoint( // + DMContext & context, + UniversalPageStoragePtr temp_ps, + const RowKeyRange & segment_range, + NamespaceId ns_id, + PageIdU64 delta_id, + WriteBatches & wbs) +{ + auto delta_page_id = UniversalPageIdFormat::toFullPageId(UniversalPageIdFormat::toFullPrefix(StorageType::Meta, ns_id), delta_id); + auto meta_page = temp_ps->read(delta_page_id); + ReadBufferFromMemory meta_buf(meta_page.data.begin(), meta_page.data.size()); + auto column_files = createColumnFilesFromCheckpoint( + context, + segment_range, + meta_buf, + temp_ps, + ns_id, + wbs); + auto new_persisted_set = std::make_shared(delta_id, column_files); + return new_persisted_set; +} + void ColumnFilePersistedSet::saveMeta(WriteBatches & wbs) const { serializeColumnFilePersisteds(wbs, metadata_id, persisted_files); diff --git a/dbms/src/Storages/DeltaMerge/Delta/ColumnFilePersistedSet.h b/dbms/src/Storages/DeltaMerge/Delta/ColumnFilePersistedSet.h index aa5f8a86b7d..5e60ecec05d 100644 --- a/dbms/src/Storages/DeltaMerge/Delta/ColumnFilePersistedSet.h +++ b/dbms/src/Storages/DeltaMerge/Delta/ColumnFilePersistedSet.h @@ -75,6 +75,14 @@ class ColumnFilePersistedSet : public std::enable_shared_from_this #include #include +#include #include #include #include #include +#include #include #include @@ -62,6 +64,18 @@ DeltaValueSpacePtr DeltaValueSpace::restore(DMContext & context, const RowKeyRan return std::make_shared(std::move(persisted_file_set)); } +DeltaValueSpacePtr DeltaValueSpace::createFromCheckpoint( // + DMContext & context, + UniversalPageStoragePtr temp_ps, + const RowKeyRange & segment_range, + NamespaceId ns_id, + PageIdU64 delta_id, + WriteBatches & wbs) +{ + auto persisted_file_set = ColumnFilePersistedSet::createFromCheckpoint(context, temp_ps, segment_range, ns_id, delta_id, wbs); + return std::make_shared(std::move(persisted_file_set)); +} + void DeltaValueSpace::saveMeta(WriteBatches & wbs) const { persisted_file_set->saveMeta(wbs); @@ -129,7 +143,12 @@ std::vector CloneColumnFilesHelper::clone( // to the file id. wbs.data.putRefPage(new_page_id, f->getDataPageId()); auto file_id = f->getFile()->fileId(); - auto file_parent_path = delegator.getDTFilePath(file_id); + auto old_dmfile = f->getFile(); + auto file_parent_path = old_dmfile->parentPath(); + if (!context.db_context.getSharedContextDisagg()->remote_data_store) + { + RUNTIME_CHECK(file_parent_path == delegator.getDTFilePath(file_id)); + } auto new_file = DMFile::restore(context.db_context.getFileProvider(), file_id, /* page_id= */ new_page_id, file_parent_path, DMFile::ReadMetaMode::all()); auto new_column_file = f->cloneWith(context, new_file, target_range); diff --git a/dbms/src/Storages/DeltaMerge/Delta/DeltaValueSpace.h b/dbms/src/Storages/DeltaMerge/Delta/DeltaValueSpace.h index 0b80d552779..71e54836129 100644 --- a/dbms/src/Storages/DeltaMerge/Delta/DeltaValueSpace.h +++ b/dbms/src/Storages/DeltaMerge/Delta/DeltaValueSpace.h @@ -111,6 +111,14 @@ class DeltaValueSpace /// Only called after reboot. static DeltaValueSpacePtr restore(DMContext & context, const RowKeyRange & segment_range, PageIdU64 id); + static DeltaValueSpacePtr createFromCheckpoint( // + DMContext & context, + UniversalPageStoragePtr temp_ps, + const RowKeyRange & segment_range, + NamespaceId ns_id, + PageIdU64 delta_id, + WriteBatches & wbs); + /** * Resets the logger by using the one from the segment. * Segment_log is not available when constructing, because usually diff --git a/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp b/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp index 66c86ad5bf9..a4c6b7f48f8 100644 --- a/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp +++ b/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp @@ -43,8 +43,6 @@ #include #include #include -#include -#include #include #include @@ -423,7 +421,7 @@ void DeltaMergeStore::shutdown() LOG_TRACE(log, "Shutdown DeltaMerge end"); } -DMContextPtr DeltaMergeStore::newDMContext(const Context & db_context, const DB::Settings & db_settings, const String & tracing_id, const ScanContextPtr & scan_context_) +DMContextPtr DeltaMergeStore::newDMContext(const Context & db_context, const DB::Settings & db_settings, const String & tracing_id, ScanContextPtr scan_context_) { std::shared_lock lock(read_write_mutex); @@ -1000,7 +998,7 @@ BlockInputStreams DeltaMergeStore::read(const Context & db_context, size_t expected_block_size, const SegmentIdSet & read_segments, size_t extra_table_id_index, - const ScanContextPtr & scan_context) + ScanContextPtr scan_context) { // Use the id from MPP/Coprocessor level as tracing_id auto dm_context = newDMContext(db_context, db_settings, tracing_id, scan_context); @@ -1092,7 +1090,7 @@ SourceOps DeltaMergeStore::readSourceOps( size_t expected_block_size, const SegmentIdSet & read_segments, size_t extra_table_id_index, - const ScanContextPtr & scan_context) + ScanContextPtr scan_context) { // Use the id from MPP/Coprocessor level as tracing_id auto dm_context = newDMContext(db_context, db_settings, tracing_id, scan_context); @@ -1158,7 +1156,7 @@ DeltaMergeStore::writeNodeBuildRemoteReadSnapshot( size_t num_streams, const String & tracing_id, const SegmentIdSet & read_segments, - const ScanContextPtr & scan_context) + ScanContextPtr scan_context) { auto dm_context = newDMContext(db_context, db_settings, tracing_id, scan_context); auto log_tracing_id = getLogTracingId(*dm_context); @@ -1614,19 +1612,6 @@ SortDescription DeltaMergeStore::getPrimarySortDescription() const return desc; } -void DeltaMergeStore::restoreStableFilesFromS3() -{ - auto file_provider = global_context.getFileProvider(); - auto store_id = global_context.getTMTContext().getKVStore()->getStoreID(); - auto stable_path = S3::S3Filename::fromTableID(store_id, physical_table_id).toFullKeyWithPrefix(); - - auto file_ids = DMFile::listAllInPath(file_provider, stable_path, DMFile::ListOptions{.only_list_can_gc = false}); - LOG_DEBUG(log, "s3_stable_path {} => file_ids {}", stable_path, file_ids); - auto path_delegate = path_pool->getStableDiskDelegator(); - path_delegate.addS3DTFiles(stable_path, std::move(file_ids)); - // TODO: remove local dmfile? -} - void DeltaMergeStore::restoreStableFilesFromLocal() { DMFile::ListOptions options; @@ -1650,11 +1635,7 @@ void DeltaMergeStore::restoreStableFiles() { LOG_DEBUG(log, "Loading dt files"); - if (global_context.getSharedContextDisagg()->remote_data_store) - { - restoreStableFilesFromS3(); - } - else + if (!global_context.getSharedContextDisagg()->remote_data_store) { restoreStableFilesFromLocal(); } diff --git a/dbms/src/Storages/DeltaMerge/DeltaMergeStore.h b/dbms/src/Storages/DeltaMerge/DeltaMergeStore.h index f20f975e3e9..28b71c2024c 100644 --- a/dbms/src/Storages/DeltaMerge/DeltaMergeStore.h +++ b/dbms/src/Storages/DeltaMerge/DeltaMergeStore.h @@ -21,6 +21,7 @@ #include #include #include +#include #include #include #include @@ -38,6 +39,8 @@ namespace DB class Logger; using LoggerPtr = std::shared_ptr; +struct CheckpointInfo; +using CheckpointInfoPtr = std::shared_ptr; class StoragePathPool; @@ -299,6 +302,29 @@ class DeltaMergeStore : private boost::noncopyable return ingestFiles(dm_context, range, external_files, clear_data_in_range); } + std::vector ingestSegmentsUsingSplit( + const DMContextPtr & dm_context, + const RowKeyRange & ingest_range, + const std::vector & target_segments); + + bool ingestSegmentDataIntoSegmentUsingSplit( + DMContext & dm_context, + const SegmentPtr & segment, + const SegmentPtr & segment_to_ingest); + + void ingestSegmentsFromCheckpointInfo(const DMContextPtr & dm_context, + const DM::RowKeyRange & range, + CheckpointInfoPtr checkpoint_info); + + void ingestSegmentsFromCheckpointInfo(const Context & db_context, + const DB::Settings & db_settings, + const DM::RowKeyRange & range, + CheckpointInfoPtr checkpoint_info) + { + auto dm_context = newDMContext(db_context, db_settings); + return ingestSegmentsFromCheckpointInfo(dm_context, range, checkpoint_info); + } + /// Read all rows without MVCC filtering BlockInputStreams readRaw(const Context & db_context, const DB::Settings & db_settings, @@ -326,7 +352,7 @@ class DeltaMergeStore : private boost::noncopyable size_t expected_block_size = DEFAULT_BLOCK_SIZE, const SegmentIdSet & read_segments = {}, size_t extra_table_id_index = InvalidColumnID, - const ScanContextPtr & scan_context = std::make_shared()); + ScanContextPtr scan_context = nullptr); /// Read rows in two modes: @@ -347,7 +373,7 @@ class DeltaMergeStore : private boost::noncopyable size_t expected_block_size = DEFAULT_BLOCK_SIZE, const SegmentIdSet & read_segments = {}, size_t extra_table_id_index = InvalidColumnID, - const ScanContextPtr & scan_context = std::make_shared()); + ScanContextPtr scan_context = nullptr); Remote::DisaggPhysicalTableReadSnapshotPtr writeNodeBuildRemoteReadSnapshot( @@ -357,7 +383,7 @@ class DeltaMergeStore : private boost::noncopyable size_t num_streams, const String & tracing_id, const SegmentIdSet & read_segments = {}, - const ScanContextPtr & scan_context = std::make_shared()); + ScanContextPtr scan_context = nullptr); /// Try flush all data in `range` to disk and return whether the task succeed. bool flushCache(const Context & context, const RowKeyRange & range, bool try_until_succeed = true); @@ -458,7 +484,7 @@ class DeltaMergeStore : private boost::noncopyable private: #endif - DMContextPtr newDMContext(const Context & db_context, const DB::Settings & db_settings, const String & tracing_id = "", const ScanContextPtr & scan_context = std::make_shared()); + DMContextPtr newDMContext(const Context & db_context, const DB::Settings & db_settings, const String & tracing_id = "", ScanContextPtr scan_context = nullptr); static bool pkIsHandle(const ColumnDefine & handle_define) { @@ -575,6 +601,26 @@ class DeltaMergeStore : private boost::noncopyable const DMFilePtr & data_file, bool clear_all_data_in_segment); + /** + * Discard all data in the segment, and use the specified DMFile as the stable instead. + * The specified DMFile is safe to be shared for multiple segments. + * + * Note 1: This function will not enable GC for the new_stable_file for you, in case of you may want to share the same + * stable file for multiple segments. It is your own duty to enable GC later. + * + * Note 2: You must ensure the specified new_stable_file has been managed by the storage pool, and has been written + * to the PageStorage's data. Otherwise there will be exceptions. + * + * Note 3: This API is subjected to be changed in future, as it relies on the knowledge that all current data + * in this segment is useless, which is a pretty tough requirement. + * TODO: use `segmentIngestData` to replace this api + */ + SegmentPtr segmentDangerouslyReplaceDataFromCheckpoint( + DMContext & dm_context, + const SegmentPtr & segment, + const DMFilePtr & data_file, + const ColumnFilePersisteds & column_file_persisteds); + // isSegmentValid should be protected by lock on `read_write_mutex` bool isSegmentValid(const std::shared_lock &, const SegmentPtr & segment) { @@ -615,7 +661,6 @@ class DeltaMergeStore : private boost::noncopyable bool handleBackgroundTask(bool heavy); void restoreStableFiles(); - void restoreStableFilesFromS3(); void restoreStableFilesFromLocal(); SegmentReadTasks getReadTasksByRanges(DMContext & dm_context, diff --git a/dbms/src/Storages/DeltaMerge/DeltaMergeStore_Ingest.cpp b/dbms/src/Storages/DeltaMerge/DeltaMergeStore_Ingest.cpp index 8023c8f5299..7f7971b580f 100644 --- a/dbms/src/Storages/DeltaMerge/DeltaMergeStore_Ingest.cpp +++ b/dbms/src/Storages/DeltaMerge/DeltaMergeStore_Ingest.cpp @@ -24,6 +24,8 @@ #include #include #include +#include +#include #include #include @@ -710,5 +712,339 @@ void DeltaMergeStore::ingestFiles( checkSegmentUpdate(dm_context, segment, ThreadType::Write); } +std::vector DeltaMergeStore::ingestSegmentsUsingSplit( + const DMContextPtr & dm_context, + const RowKeyRange & ingest_range, + const std::vector & target_segments) +{ + std::set updated_segments; + + // First phase (DeleteRange Phase): + // Write DeleteRange to the covered segments to ensure that all data in the `ingest_range` is cleared. + { + RowKeyRange remaining_delete_range = ingest_range; + LOG_INFO( + log, + "Table ingest checkpoint using split - delete range phase - begin, remaining_delete_range={}", + remaining_delete_range.toDebugString()); + + while (!remaining_delete_range.none()) + { + SegmentPtr segment; + { + std::shared_lock lock(read_write_mutex); + + auto segment_it = segments.upper_bound(remaining_delete_range.getStart()); + RUNTIME_CHECK(segment_it != segments.end(), remaining_delete_range.toDebugString()); + segment = segment_it->second; + } + + const auto delete_range = remaining_delete_range.shrink(segment->getRowKeyRange()); + RUNTIME_CHECK( + !delete_range.none(), // as remaining_delete_range is not none, we expect the shrinked range to be not none. + delete_range.toDebugString(), + segment->simpleInfo(), + remaining_delete_range.toDebugString()); + LOG_DEBUG( + log, + "Table ingest checkpoint using split - delete range phase - Try to delete range in segment, delete_range={} segment={} remaining_delete_range={} updated_segments_n={}", + delete_range.toDebugString(), + segment->simpleInfo(), + remaining_delete_range.toDebugString(), + updated_segments.size()); + + const bool succeeded = segment->write(*dm_context, delete_range); + if (succeeded) + { + updated_segments.insert(segment); + RUNTIME_CHECK(compare(delete_range.getEnd(), remaining_delete_range.getStart()) >= 0); + remaining_delete_range.setStart(delete_range.end); // We always move forward + } + else + { + // segment may be abandoned, retry current range by finding the segment again. + } + } + } + + LOG_DEBUG( + log, + "Table ingest checkpoint using split - delete range phase - finished, updated_segments_n={}", + updated_segments.size()); + + /* + * In second phase (SplitIngest Phase), + * we will try to ingest DMFile one by one into the segments in order. + * + * Consider the following case: + * -Inf +Inf + * │ │--------------- Ingest Range ---------------│ │ + * │ │-- DMFile --│- DMFile --│---- DMFile ----│- DMFile --│ │ + * │- Segment --│-- Seg --│------- Segment -----│- Seg -│------- Segment --------│ + * + * This is what we will ingest: + * Iterate 0: + * -Inf +Inf + * │ │--------------- Ingest Range ---------------│ │ + * │ │-- DMFile --│ │ + * │ │-- Seg --│------- Segment -----│ │ + * ↑ ↑ The segment we ingest DMFile into + * + * Iterate 1: + * -Inf +Inf + * │ │--------------- Ingest Range ---------------│ │ + * │ │************│- DMFile --│ │ + * │ │------- Segment -----│ │ + * ↑ The segment we ingest DMFile into + */ + + LOG_DEBUG( + log, + "Table ingest checkpoint using split - split ingest phase - begin, ingest_range={}, files_n={}", + ingest_range.toDebugString(), + target_segments.size()); + + for (size_t file_idx = 0; file_idx < target_segments.size(); file_idx++) + { + // This should not happen. Just check to be confident. + // Even if it happened, we could handle it gracefully here. (but it really means somewhere else is broken) + if (target_segments[file_idx]->getEstimatedRows() == 0) + { + LOG_WARNING( + log, + "Table ingest checkpoint using split - split ingest phase - Unexpected empty DMFile, skipped. ingest_range={} file_idx={}", + ingest_range.toDebugString(), + file_idx); + continue; + } + + /** + * Each DMFile (bounded by the ingest range) may overlap with multiple segments, like: + * -Inf +Inf + * │ │--------------- Ingest Range ---------------│ │ + * │ │-- DMFile --│ │ + * │ │-- Seg --│------- Segment -----│ │ + * We will try to ingest it into all overlapped segments. + */ + auto file_ingest_range = target_segments[file_idx]->getRowKeyRange(); + while (!file_ingest_range.none()) // This DMFile has remaining data to ingest + { + SegmentPtr segment; + { + std::shared_lock lock(read_write_mutex); + auto segment_it = segments.upper_bound(file_ingest_range.getStart()); + RUNTIME_CHECK(segment_it != segments.end()); + segment = segment_it->second; + } + + if (segment->hasAbandoned()) + continue; // retry with current range and file + + /** + * -Inf +Inf + * │ │--------------- Ingest Range ---------------│ │ + * │ │-- DMFile --│ │ + * │ │-- Seg --│------- Segment -----│ │ + * ^^^^^^^^ segment_ingest_range + */ + const auto segment_ingest_range = file_ingest_range.shrink(segment->getRowKeyRange()); + RUNTIME_CHECK(!segment_ingest_range.none()); + + LOG_INFO( + log, + "Table ingest checkpoint using split - split ingest phase - Try to ingest file into segment, segment_idx={} segment_id={} segment_ingest_range={} segment={} segment_ingest_range={}", + file_idx, + target_segments[file_idx]->segmentId(), + file_ingest_range.toDebugString(), + segment->simpleInfo(), + segment_ingest_range.toDebugString()); + + const bool succeeded = ingestSegmentDataIntoSegmentUsingSplit(*dm_context, segment, target_segments[file_idx]); + if (succeeded) + { + updated_segments.insert(segment); + // We have ingested (DTFileRange ∪ ThisSegmentRange), let's try with next overlapped segment. + RUNTIME_CHECK(compare(segment_ingest_range.getEnd(), file_ingest_range.getStart()) > 0); + file_ingest_range.setStart(segment_ingest_range.end); + } + else + { + // this segment is abandoned, or may be split into multiples. + // retry with current range and file and find segment again. + } + } + } + + LOG_DEBUG( + log, + "Table ingest checkpoint using split - split ingest phase - finished, updated_segments_n={}", + updated_segments.size()); + + return std::vector( + updated_segments.begin(), + updated_segments.end()); +} + +bool DeltaMergeStore::ingestSegmentDataIntoSegmentUsingSplit( + DMContext & dm_context, + const SegmentPtr & segment, + const SegmentPtr & segment_to_ingest) +{ + const auto & segment_range = segment->getRowKeyRange(); + const auto & ingest_range = segment_to_ingest->getRowKeyRange(); + + // The ingest_range must fall in segment's range. + RUNTIME_CHECK( + !ingest_range.none(), + ingest_range.toDebugString()); + RUNTIME_CHECK( + compare(segment_range.getStart(), ingest_range.getStart()) <= 0, + segment_range.toDebugString(), + ingest_range.toDebugString()); + RUNTIME_CHECK( + compare(segment_range.getEnd(), ingest_range.getEnd()) >= 0, + segment_range.toDebugString(), + ingest_range.toDebugString()); + + const bool is_start_matching = (compare(segment_range.getStart(), ingest_range.getStart()) == 0); + const bool is_end_matching = (compare(segment_range.getEnd(), ingest_range.getEnd()) == 0); + + if (is_start_matching && is_end_matching) + { + /* + * The segment and the ingest range is perfectly matched. We can + * simply replace all of the data from this segment. + * + * Example: + * │----------- Segment ----------│ + * │-------- Ingest Range --------│ + */ + WriteBatches wbs{*dm_context.storage_pool}; + auto dm_files = segment_to_ingest->getStable()->getDMFiles(); + auto [in_memory_files, column_file_persisteds] = segment_to_ingest->getDelta()->cloneAllColumnFiles( + segment_to_ingest->mustGetUpdateLock(), + dm_context, + ingest_range, + wbs); + wbs.writeLogAndData(); + RUNTIME_CHECK(in_memory_files.empty()); + RUNTIME_CHECK(dm_files.size() == 1); + const auto new_segment_or_null = segmentDangerouslyReplaceDataFromCheckpoint(dm_context, segment, dm_files[0], column_file_persisteds); + const bool succeeded = new_segment_or_null != nullptr; + if (!succeeded) + { + wbs.rollbackWrittenLogAndData(); + } + return succeeded; + } + else if (is_start_matching) + { + /* + * Example: + * │--------------- Segment ---------------│ + * │-------- Ingest Range --------│ + * + * We will logical split the segment to form a perfect matching segment: + * │--------------- Segment ------│--------│ + * │-------- Ingest Range --------│ + */ + const auto [left, right] = segmentSplit(dm_context, segment, SegmentSplitReason::ForIngest, ingest_range.end, SegmentSplitMode::Logical); + if (left == nullptr || right == nullptr) + { + // Split failed, likely caused by snapshot failed. + // Sleep awhile and retry. + std::this_thread::sleep_for(std::chrono::milliseconds(15)); + } + // Always returning false, because we need to retry to get a new segment (as the old segment is abandoned) + // even when split succeeded. + return false; + } + else if (is_end_matching) + { + /* + * Example: + * │--------------- Segment ---------------│ + * │-------- Ingest Range --------│ + * + * We will logical split the segment to form a perfect matching segment: + * │--------│------ Segment ---------------│ + * │-------- Ingest Range --------│ + */ + const auto [left, right] = segmentSplit(dm_context, segment, SegmentSplitReason::ForIngest, ingest_range.start, SegmentSplitMode::Logical); + if (left == nullptr || right == nullptr) + { + std::this_thread::sleep_for(std::chrono::milliseconds(15)); + } + return false; + } + else + { + /* + * Example: + * │--------------- Segment ---------------│ + * │-------- Ingest Range --------│ + * + * We invoke a logical split first: + * │---│----------- Segment ---------------│ + * │-------- Ingest Range --------│ + */ + const auto [left, right] = segmentSplit(dm_context, segment, SegmentSplitReason::ForIngest, ingest_range.start, SegmentSplitMode::Logical); + if (left == nullptr || right == nullptr) + { + std::this_thread::sleep_for(std::chrono::milliseconds(15)); + } + return false; + } +} + +void DeltaMergeStore::ingestSegmentsFromCheckpointInfo( + const DMContextPtr & dm_context, + const DM::RowKeyRange & range, + CheckpointInfoPtr checkpoint_info) +{ + if (unlikely(shutdown_called.load(std::memory_order_relaxed))) + { + const auto msg = fmt::format("Try to ingest files into a shutdown table, store={}", log->identifier()); + LOG_WARNING(log, "{}", msg); + throw Exception(msg); + } + LOG_INFO(log, "Ingest checkpoint from store {}", checkpoint_info->remote_store_id); + + auto segment_meta_infos = Segment::readAllSegmentsMetaInfoInRange(*dm_context, checkpoint_info->remote_store_id, physical_table_id, range, checkpoint_info->temp_ps); + LOG_INFO(log, "Ingest checkpoint segments num {}", segment_meta_infos.size()); + WriteBatches wbs{*dm_context->storage_pool}; + auto restored_segments = Segment::createTargetSegmentsFromCheckpoint( // + log, + *dm_context, + checkpoint_info->remote_store_id, + physical_table_id, + segment_meta_infos, + range, + checkpoint_info->temp_ps, + wbs); + + if (restored_segments.empty()) + { + LOG_DEBUG(log, "No segments to ingest."); + return; + } + wbs.writeLogAndData(); + + auto updated_segments = ingestSegmentsUsingSplit(dm_context, range, restored_segments); + LOG_INFO(log, "Ingest checkpoint from store {} done", checkpoint_info->remote_store_id); + + for (auto & segment : restored_segments) + { + auto delta = segment->getDelta(); + auto stable = segment->getStable(); + delta->recordRemoveColumnFilesPages(wbs); + stable->recordRemovePacksPages(wbs); + wbs.writeRemoves(); + } + + for (auto & segment : updated_segments) + checkSegmentUpdate(dm_context, segment, ThreadType::Write); +} + } // namespace DM } // namespace DB diff --git a/dbms/src/Storages/DeltaMerge/DeltaMergeStore_InternalSegment.cpp b/dbms/src/Storages/DeltaMerge/DeltaMergeStore_InternalSegment.cpp index b7f81100ea7..f1f39ec6c72 100644 --- a/dbms/src/Storages/DeltaMerge/DeltaMergeStore_InternalSegment.cpp +++ b/dbms/src/Storages/DeltaMerge/DeltaMergeStore_InternalSegment.cpp @@ -587,6 +587,49 @@ SegmentPtr DeltaMergeStore::segmentIngestData( return new_segment; } +SegmentPtr DeltaMergeStore::segmentDangerouslyReplaceDataFromCheckpoint( + DMContext & dm_context, + const SegmentPtr & segment, + const DMFilePtr & data_file, + const ColumnFilePersisteds & column_file_persisteds) +{ + LOG_INFO(log, "ReplaceData - Begin, segment={} data_file={} column_files_num={}", segment->info(), data_file->path(), column_file_persisteds.size()); + + WriteBatches wbs(*storage_pool, dm_context.getWriteLimiter()); + + SegmentPtr new_segment; + { + std::unique_lock lock(read_write_mutex); + if (!isSegmentValid(lock, segment)) + { + LOG_DEBUG(log, "ReplaceData - Give up segment replace data because segment not valid, segment={} data_file={}", segment->simpleInfo(), data_file->path()); + return {}; + } + + auto segment_lock = segment->mustGetUpdateLock(); + new_segment = segment->dangerouslyReplaceDataFromCheckpoint(segment_lock, dm_context, data_file, wbs, column_file_persisteds); + + RUNTIME_CHECK(compare(segment->getRowKeyRange().getEnd(), new_segment->getRowKeyRange().getEnd()) == 0, segment->info(), new_segment->info()); + RUNTIME_CHECK(segment->segmentId() == new_segment->segmentId(), segment->info(), new_segment->info()); + + wbs.writeLogAndData(); + wbs.writeMeta(); + + segment->abandon(dm_context); + segments[segment->getRowKeyRange().getEnd()] = new_segment; + id_to_segment[segment->segmentId()] = new_segment; + + LOG_INFO(log, "ReplaceData - Finish, old_segment={} new_segment={}", segment->info(), new_segment->info()); + } + + wbs.writeRemoves(); + + if constexpr (DM_RUN_CHECK) + check(dm_context.db_context); + + return new_segment; +} + bool DeltaMergeStore::doIsSegmentValid(const SegmentPtr & segment) { if (segment->hasAbandoned()) diff --git a/dbms/src/Storages/DeltaMerge/File/DMFile.cpp b/dbms/src/Storages/DeltaMerge/File/DMFile.cpp index e165889e859..d3d76ea4cfc 100644 --- a/dbms/src/Storages/DeltaMerge/File/DMFile.cpp +++ b/dbms/src/Storages/DeltaMerge/File/DMFile.cpp @@ -28,6 +28,7 @@ #include #include #include +#include #include #include #include @@ -49,6 +50,7 @@ namespace FailPoints { extern const char exception_before_dmfile_remove_encryption[]; extern const char exception_before_dmfile_remove_from_disk[]; +extern const char force_use_dmfile_format_v3[]; } // namespace FailPoints namespace DM @@ -110,6 +112,11 @@ String DMFile::ngcPath() const DMFilePtr DMFile::create(UInt64 file_id, const String & parent_path, DMConfigurationOpt configuration, DMFileFormat::Version version) { + fiu_do_on(FailPoints::force_use_dmfile_format_v3, { + // some unit test we need mock upload DMFile to S3, which only support DMFileFormat::V3 + version = DMFileFormat::V3; + LOG_WARNING(Logger::get(), "!!!force use DMFileFormat::V3!!!"); + }); // On create, ref_id is the same as file_id. DMFilePtr new_dmfile(new DMFile(file_id, file_id, @@ -641,24 +648,17 @@ std::vector DMFile::listLocal(const String & parent_path) std::vector DMFile::listS3(const String & parent_path) { std::vector filenames; - auto client = S3::ClientFactory::instance().sharedClient(); - const auto & bucket = S3::ClientFactory::instance().bucket(); + auto client = S3::ClientFactory::instance().sharedTiFlashClient(); auto list_prefix = parent_path + "/"; - S3::listPrefix( + S3::listPrefixWithDelimiter( *client, - bucket, list_prefix, /*delimiter*/ "/", - [&filenames, &list_prefix](const Aws::S3::Model::ListObjectsV2Result & result) { - const Aws::Vector & prefixes = result.GetCommonPrefixes(); - filenames.reserve(filenames.size() + prefixes.size()); - for (const auto & prefix : prefixes) - { - RUNTIME_CHECK(prefix.GetPrefix().size() > list_prefix.size(), prefix.GetPrefix(), list_prefix); - auto short_name_size = prefix.GetPrefix().size() - list_prefix.size() - 1; // `1` for the delimiter in last. - filenames.push_back(prefix.GetPrefix().substr(list_prefix.size(), short_name_size)); // Cut prefix and last delimiter. - } - return S3::PageResult{.num_keys = prefixes.size(), .more = true}; + [&filenames, &list_prefix](const Aws::S3::Model::CommonPrefix & prefix) { + RUNTIME_CHECK(prefix.GetPrefix().size() > list_prefix.size(), prefix.GetPrefix(), list_prefix); + auto short_name_size = prefix.GetPrefix().size() - list_prefix.size() - 1; // `1` for the delimiter in last. + filenames.push_back(prefix.GetPrefix().substr(list_prefix.size(), short_name_size)); // Cut prefix and last delimiter. + return S3::PageResult{.num_keys = 1, .more = true}; }); return filenames; } @@ -805,20 +805,16 @@ void DMFile::initializeIndices() } } -DMFile::MetaBlockHandle DMFile::writeSLPackStatToBuffer(WriteBuffer & buffer, DB::UnifiedDigestBaseBox & digest) +DMFile::MetaBlockHandle DMFile::writeSLPackStatToBuffer(WriteBuffer & buffer) { auto offset = buffer.count(); const char * data = reinterpret_cast(&pack_stats[0]); size_t size = pack_stats.size() * sizeof(PackStat); - if (digest) - { - digest->update(data, size); - } writeString(data, size, buffer); return MetaBlockHandle{MetaBlockType::PackStat, offset, buffer.count() - offset}; } -DMFile::MetaBlockHandle DMFile::writeSLPackPropertyToBuffer(WriteBuffer & buffer, DB::UnifiedDigestBaseBox & digest) +DMFile::MetaBlockHandle DMFile::writeSLPackPropertyToBuffer(WriteBuffer & buffer) { auto offset = buffer.count(); for (const auto & pb : pack_properties.property()) @@ -826,55 +822,44 @@ DMFile::MetaBlockHandle DMFile::writeSLPackPropertyToBuffer(WriteBuffer & buffer PackProperty tmp{pb}; const char * data = reinterpret_cast(&tmp); size_t size = sizeof(PackProperty); - if (digest) - { - digest->update(data, size); - } writeString(data, size, buffer); } return MetaBlockHandle{MetaBlockType::PackProperty, offset, buffer.count() - offset}; } -DMFile::MetaBlockHandle DMFile::writeColumnStatToBuffer(WriteBuffer & buffer, DB::UnifiedDigestBaseBox & digest) +DMFile::MetaBlockHandle DMFile::writeColumnStatToBuffer(WriteBuffer & buffer) { auto offset = buffer.count(); writeIntBinary(column_stats.size(), buffer); for (const auto & [id, stat] : column_stats) { - auto tmp_buffer = WriteBufferFromOwnString{}; - stat.serializeToBuffer(tmp_buffer); - auto serialized = tmp_buffer.releaseStr(); - if (digest) - { - digest->update(serialized.data(), serialized.length()); - } - writeString(serialized.data(), serialized.size(), buffer); + stat.serializeToBuffer(buffer); } return MetaBlockHandle{MetaBlockType::ColumnStat, offset, buffer.count() - offset}; } void DMFile::finalizeMetaV2(WriteBuffer & buffer) { + auto tmp_buffer = WriteBufferFromOwnString{}; auto digest = configuration ? configuration->createUnifiedDigest() : nullptr; - auto pack_stats_handle = writeSLPackStatToBuffer(buffer, digest); - auto pack_properties_handle = writeSLPackPropertyToBuffer(buffer, digest); - auto column_stats_handle = writeColumnStatToBuffer(buffer, digest); + auto pack_stats_handle = writeSLPackStatToBuffer(tmp_buffer); + auto pack_properties_handle = writeSLPackPropertyToBuffer(tmp_buffer); + auto column_stats_handle = writeColumnStatToBuffer(tmp_buffer); - writePODBinary(pack_stats_handle, buffer); - writePODBinary(pack_properties_handle, buffer); - writePODBinary(column_stats_handle, buffer); + writePODBinary(pack_stats_handle, tmp_buffer); + writePODBinary(pack_properties_handle, tmp_buffer); + writePODBinary(column_stats_handle, tmp_buffer); UInt64 meta_block_handle_count = 3; - writeIntBinary(meta_block_handle_count, buffer); - writeIntBinary(version, buffer); + writeIntBinary(meta_block_handle_count, tmp_buffer); + writeIntBinary(version, tmp_buffer); - if (digest) + // Write to file and do checksums. + auto s = tmp_buffer.releaseStr(); + writeString(s.data(), s.size(), buffer); + if (configuration) { - digest->update(reinterpret_cast(&pack_stats_handle), sizeof(MetaBlockHandle)); - digest->update(reinterpret_cast(&pack_properties_handle), sizeof(MetaBlockHandle)); - digest->update(reinterpret_cast(&column_stats_handle), sizeof(MetaBlockHandle)); - digest->update(reinterpret_cast(&meta_block_handle_count), sizeof(UInt64)); - digest->update(reinterpret_cast(&version), sizeof(DMFileFormat::Version)); - + auto digest = configuration->createUnifiedDigest(); + digest->update(s.data(), s.size()); auto checksum_result = digest->raw(); writeString(checksum_result.data(), checksum_result.size(), buffer); } @@ -929,7 +914,7 @@ void DMFile::parseMetaV2(std::string_view buffer) auto hash_size = digest->hashSize(); ptr = ptr - hash_size; digest->update(buffer.data(), buffer.size() - sizeof(MetaFooter) - hash_size); - if (unlikely(digest->compareRaw(ptr))) + if (unlikely(!digest->compareRaw(ptr))) { LOG_ERROR(log, "{} checksum invalid", metav2Path()); throw Exception(ErrorCodes::CORRUPTED_DATA, "{} checksum invalid", metav2Path()); @@ -1056,4 +1041,4 @@ void DMFile::switchToRemote(const S3::DMFileOID & oid) } // namespace DM -} // namespace DB \ No newline at end of file +} // namespace DB diff --git a/dbms/src/Storages/DeltaMerge/File/DMFile.h b/dbms/src/Storages/DeltaMerge/File/DMFile.h index 5f5e28b49cf..196e5250df5 100644 --- a/dbms/src/Storages/DeltaMerge/File/DMFile.h +++ b/dbms/src/Storages/DeltaMerge/File/DMFile.h @@ -444,9 +444,9 @@ class DMFile : private boost::noncopyable // Meta data is small and 64KB is enough. static constexpr size_t meta_buffer_size = 64 * 1024; void finalizeMetaV2(WriteBuffer & buffer); - MetaBlockHandle writeSLPackStatToBuffer(WriteBuffer & buffer, DB::UnifiedDigestBaseBox & digest); - MetaBlockHandle writeSLPackPropertyToBuffer(WriteBuffer & buffer, DB::UnifiedDigestBaseBox & digest); - MetaBlockHandle writeColumnStatToBuffer(WriteBuffer & buffer, DB::UnifiedDigestBaseBox & digest); + MetaBlockHandle writeSLPackStatToBuffer(WriteBuffer & buffer); + MetaBlockHandle writeSLPackPropertyToBuffer(WriteBuffer & buffer); + MetaBlockHandle writeColumnStatToBuffer(WriteBuffer & buffer); std::vector readMetaV2(const FileProviderPtr & file_provider); void parseMetaV2(std::string_view buffer); void parseColumnStat(std::string_view buffer); @@ -475,6 +475,7 @@ class DMFile : private boost::noncopyable DMFileFormat::Version version; friend class DMFileWriter; + friend class DMFileWriterRemote; friend class DMFileReader; friend class DMFilePackFilter; friend class DMFileBlockInputStreamBuilder; diff --git a/dbms/src/Storages/DeltaMerge/File/DMFileBlockOutputStream.cpp b/dbms/src/Storages/DeltaMerge/File/DMFileBlockOutputStream.cpp index 0dffeb280c6..4c15c44418e 100644 --- a/dbms/src/Storages/DeltaMerge/File/DMFileBlockOutputStream.cpp +++ b/dbms/src/Storages/DeltaMerge/File/DMFileBlockOutputStream.cpp @@ -17,7 +17,6 @@ namespace DB::DM { - DMFileBlockOutputStream::DMFileBlockOutputStream(const Context & context, const DMFilePtr & dmfile, const ColumnDefines & write_columns) diff --git a/dbms/src/Storages/DeltaMerge/Remote/DataStore/DataStore.h b/dbms/src/Storages/DeltaMerge/Remote/DataStore/DataStore.h index 6e2a30dbe86..eab78c70bdb 100644 --- a/dbms/src/Storages/DeltaMerge/Remote/DataStore/DataStore.h +++ b/dbms/src/Storages/DeltaMerge/Remote/DataStore/DataStore.h @@ -38,10 +38,12 @@ class IPreparedDMFileToken : boost::noncopyable // These should be the required information for any kind of DataStore. const FileProviderPtr file_provider; const S3::DMFileOID oid; + UInt64 page_id; - IPreparedDMFileToken(const FileProviderPtr & file_provider_, const S3::DMFileOID & oid_) + IPreparedDMFileToken(const FileProviderPtr & file_provider_, const S3::DMFileOID & oid_, UInt64 page_id_) : file_provider(file_provider_) , oid(oid_) + , page_id(page_id_ == 0 ? oid.file_id : page_id_) {} }; @@ -56,8 +58,6 @@ class IDataStore : boost::noncopyable */ virtual void putDMFile(DMFilePtr local_dm_file, const S3::DMFileOID & oid, bool remove_local) = 0; - virtual void copyDMFileMetaToLocalPath(const S3::DMFileOID & remote_oid, const String & local_path) = 0; - /** * Blocks until a DMFile in the remote data store is successfully prepared in a local cache. * If the DMFile exists in the local cache, it will not be prepared again. @@ -65,9 +65,11 @@ class IDataStore : boost::noncopyable * Returns a "token", which can be used to rebuild the `DMFile` object. * The DMFile in the local cache may be invalidated if you deconstructs the token. * - * Should be used by a read node. + * When page_id is 0, will use its file_id as page_id.(Used by WN, RN can just use default value) */ - virtual IPreparedDMFileTokenPtr prepareDMFile(const S3::DMFileOID & oid) = 0; + virtual IPreparedDMFileTokenPtr prepareDMFile(const S3::DMFileOID & oid, UInt64 page_id = 0) = 0; + + virtual IPreparedDMFileTokenPtr prepareDMFileByKey(const String & remote_key) = 0; /** * Blocks until all checkpoint files are successfully put in the remote data store. diff --git a/dbms/src/Storages/DeltaMerge/Remote/DataStore/DataStoreS3.cpp b/dbms/src/Storages/DeltaMerge/Remote/DataStore/DataStoreS3.cpp index 05115f56223..d3a93849f24 100644 --- a/dbms/src/Storages/DeltaMerge/Remote/DataStore/DataStoreS3.cpp +++ b/dbms/src/Storages/DeltaMerge/Remote/DataStore/DataStoreS3.cpp @@ -12,8 +12,9 @@ // See the License for the specific language governing permissions and // limitations under the License. +#include #include -#include +#include #include #include #include @@ -36,8 +37,7 @@ void DataStoreS3::putDMFile(DMFilePtr local_dmfile, const S3::DMFileOID & oid, b const auto remote_dir = S3::S3Filename::fromDMFileOID(oid).toFullKey(); LOG_DEBUG(log, "Start upload DMFile, local_dir={} remote_dir={} local_files={}", local_dir, remote_dir, local_files); - auto s3_client = S3::ClientFactory::instance().sharedClient(); - const auto & bucket = S3::ClientFactory::instance().bucket(); + auto s3_client = S3::ClientFactory::instance().sharedTiFlashClient(); std::vector> upload_results; for (const auto & fname : local_files) @@ -51,10 +51,10 @@ void DataStoreS3::putDMFile(DMFilePtr local_dmfile, const S3::DMFileOID & oid, b auto remote_fname = fmt::format("{}/{}", remote_dir, fname); auto task = std::make_shared>( [&, local_fname = std::move(local_fname), remote_fname = std::move(remote_fname)]() { - S3::uploadFile(*s3_client, bucket, local_fname, remote_fname); + S3::uploadFile(*s3_client, local_fname, remote_fname); }); upload_results.push_back(task->get_future()); - IOThreadPool::get().scheduleOrThrowOnError([task]() { (*task)(); }); + DataStoreS3Pool::get().scheduleOrThrowOnError([task]() { (*task)(); }); } for (auto & f : upload_results) { @@ -64,7 +64,7 @@ void DataStoreS3::putDMFile(DMFilePtr local_dmfile, const S3::DMFileOID & oid, b // Only when the meta upload is successful, the dmfile upload can be considered successful. auto local_meta_fname = fmt::format("{}/{}", local_dir, DMFile::metav2FileName()); auto remote_meta_fname = fmt::format("{}/{}", remote_dir, DMFile::metav2FileName()); - S3::uploadFile(*s3_client, bucket, local_meta_fname, remote_meta_fname); + S3::uploadFile(*s3_client, local_meta_fname, remote_meta_fname); if (remove_local) { @@ -73,18 +73,9 @@ void DataStoreS3::putDMFile(DMFilePtr local_dmfile, const S3::DMFileOID & oid, b LOG_INFO(log, "Upload DMFile finished, key={}, cost={}ms", remote_dir, sw.elapsedMilliseconds()); } -void DataStoreS3::copyDMFileMetaToLocalPath(const S3::DMFileOID & remote_oid, const String & local_dir) -{ - Stopwatch sw; - std::vector target_fnames = {DMFile::metav2FileName(), IDataType::getFileNameForStream(DB::toString(-1), {}) + ".idx"}; - copyToLocal(remote_oid, target_fnames, local_dir); - LOG_DEBUG(log, "copyDMFileMetaToLocalPath finished. Local_dir={} cost={}ms", local_dir, sw.elapsedMilliseconds()); -} - bool DataStoreS3::putCheckpointFiles(const PS::V3::LocalCheckpointFiles & local_files, StoreID store_id, UInt64 upload_seq) { - auto s3_client = S3::ClientFactory::instance().sharedClient(); - const auto & bucket = S3::ClientFactory::instance().bucket(); + auto s3_client = S3::ClientFactory::instance().sharedTiFlashClient(); /// First upload all CheckpointData files and their locks, /// then upload the CheckpointManifest to make the files within @@ -98,11 +89,11 @@ bool DataStoreS3::putCheckpointFiles(const PS::V3::LocalCheckpointFiles & local_ const auto & local_datafile = local_files.data_files[idx]; auto s3key = S3::S3Filename::newCheckpointData(store_id, upload_seq, idx); auto lock_key = s3key.toView().getLockKey(store_id, upload_seq); - S3::uploadFile(*s3_client, bucket, local_datafile, s3key.toFullKey()); - S3::uploadEmptyFile(*s3_client, bucket, lock_key); + S3::uploadFile(*s3_client, local_datafile, s3key.toFullKey()); + S3::uploadEmptyFile(*s3_client, lock_key); }); upload_results.push_back(task->get_future()); - IOThreadPool::get().scheduleOrThrowOnError([task] { (*task)(); }); + DataStoreS3Pool::get().scheduleOrThrowOnError([task] { (*task)(); }); } for (auto & f : upload_results) { @@ -111,15 +102,14 @@ bool DataStoreS3::putCheckpointFiles(const PS::V3::LocalCheckpointFiles & local_ // upload manifest after all CheckpointData uploaded auto s3key = S3::S3Filename::newCheckpointManifest(store_id, upload_seq); - S3::uploadFile(*s3_client, bucket, local_files.manifest_file, s3key.toFullKey()); + S3::uploadFile(*s3_client, local_files.manifest_file, s3key.toFullKey()); return true; // upload success } void DataStoreS3::copyToLocal(const S3::DMFileOID & remote_oid, const std::vector & target_short_fnames, const String & local_dir) { - auto s3_client = S3::ClientFactory::instance().sharedClient(); - const auto & bucket = S3::ClientFactory::instance().bucket(); + auto s3_client = S3::ClientFactory::instance().sharedTiFlashClient(); const auto remote_dir = S3::S3Filename::fromDMFileOID(remote_oid).toFullKey(); std::vector> results; for (const auto & fname : target_short_fnames) @@ -129,11 +119,11 @@ void DataStoreS3::copyToLocal(const S3::DMFileOID & remote_oid, const std::vecto auto task = std::make_shared>( [&, local_fname = std::move(local_fname), remote_fname = std::move(remote_fname)]() { auto tmp_fname = fmt::format("{}.tmp", local_fname); - S3::downloadFile(*s3_client, bucket, tmp_fname, remote_fname); + S3::downloadFile(*s3_client, tmp_fname, remote_fname); Poco::File(tmp_fname).renameTo(local_fname); }); results.push_back(task->get_future()); - IOThreadPool::get().scheduleOrThrowOnError([task]() { (*task)(); }); + DataStoreS3Pool::get().scheduleOrThrowOnError([task]() { (*task)(); }); } for (auto & f : results) { @@ -141,9 +131,18 @@ void DataStoreS3::copyToLocal(const S3::DMFileOID & remote_oid, const std::vecto } } -IPreparedDMFileTokenPtr DataStoreS3::prepareDMFile(const S3::DMFileOID & oid) + +IPreparedDMFileTokenPtr DataStoreS3::prepareDMFile(const S3::DMFileOID & oid, UInt64 page_id) { - return std::make_shared(file_provider, oid); + return std::make_shared(file_provider, oid, page_id); +} + +IPreparedDMFileTokenPtr DataStoreS3::prepareDMFileByKey(const String & remote_key) +{ + const auto view = S3::S3FilenameView::fromKeyWithPrefix(remote_key); + RUNTIME_CHECK(view.isDMFile(), magic_enum::enum_name(view.type), remote_key); + auto oid = view.getDMFileOID(); + return prepareDMFile(oid, 0); } DMFilePtr S3PreparedDMFileToken::restore(DMFile::ReadMetaMode read_mode) @@ -151,7 +150,7 @@ DMFilePtr S3PreparedDMFileToken::restore(DMFile::ReadMetaMode read_mode) return DMFile::restore( file_provider, oid.file_id, - oid.file_id, + page_id, S3::S3Filename::fromTableID(oid.store_id, oid.table_id).toFullKeyWithPrefix(), read_mode); } diff --git a/dbms/src/Storages/DeltaMerge/Remote/DataStore/DataStoreS3.h b/dbms/src/Storages/DeltaMerge/Remote/DataStore/DataStoreS3.h index e61a03eccd7..722128b434c 100644 --- a/dbms/src/Storages/DeltaMerge/Remote/DataStore/DataStoreS3.h +++ b/dbms/src/Storages/DeltaMerge/Remote/DataStore/DataStoreS3.h @@ -36,8 +36,6 @@ class DataStoreS3 final : public IDataStore */ void putDMFile(DMFilePtr local_dmfile, const S3::DMFileOID & oid, bool remove_local) override; - void copyDMFileMetaToLocalPath(const S3::DMFileOID & remote_oid, const String & local_dir) override; - /** * Blocks until a DMFile in the remote data store is successfully prepared in a local cache. * If the DMFile exists in the local cache, it will not be prepared again. @@ -47,7 +45,9 @@ class DataStoreS3 final : public IDataStore * * Should be used by a read node. */ - IPreparedDMFileTokenPtr prepareDMFile(const S3::DMFileOID & oid) override; + IPreparedDMFileTokenPtr prepareDMFile(const S3::DMFileOID & oid, UInt64 page_id) override; + + IPreparedDMFileTokenPtr prepareDMFileByKey(const String & remote_key) override; bool putCheckpointFiles(const PS::V3::LocalCheckpointFiles & local_files, StoreID store_id, UInt64 upload_seq) override; @@ -66,8 +66,8 @@ class DataStoreS3 final : public IDataStore class S3PreparedDMFileToken : public IPreparedDMFileToken { public: - S3PreparedDMFileToken(const FileProviderPtr & file_provider_, const S3::DMFileOID & oid_) - : IPreparedDMFileToken::IPreparedDMFileToken(file_provider_, oid_) + S3PreparedDMFileToken(const FileProviderPtr & file_provider_, const S3::DMFileOID & oid_, UInt64 page_id) + : IPreparedDMFileToken::IPreparedDMFileToken(file_provider_, oid_, page_id) {} ~S3PreparedDMFileToken() override = default; diff --git a/dbms/src/Storages/DeltaMerge/Remote/Proto/remote.proto b/dbms/src/Storages/DeltaMerge/Remote/Proto/remote.proto index 00006886d27..f622b2f08c9 100644 --- a/dbms/src/Storages/DeltaMerge/Remote/Proto/remote.proto +++ b/dbms/src/Storages/DeltaMerge/Remote/Proto/remote.proto @@ -72,6 +72,7 @@ message ColumnFileInMemory { message ColumnFileTiny { uint64 page_id = 1; uint64 page_size = 2; + CheckpointInfo checkpoint_info = 3; // serialized schema bytes schema = 5; @@ -82,7 +83,7 @@ message ColumnFileTiny { message ColumnFileBig { uint64 page_id = 1; - uint64 file_id = 2; + CheckpointInfo checkpoint_info = 2; // TODO: We should better recalculate these fields from local DTFile. uint64 valid_rows = 10; @@ -93,6 +94,14 @@ message ColumnFileDeleteRange { bytes key_range = 1; } +message CheckpointInfo { + bytes data_file_id = 1; + uint64 offset = 2; + uint64 size = 3; + // whether the data reclaimed on the write node or not + bool is_local_data_reclaimed = 4; +} + message TiFlashColumnInfo { int64 column_id = 1; // serialized name by IDataType::getName() diff --git a/dbms/src/Storages/DeltaMerge/Remote/RNLocalPageCache.h b/dbms/src/Storages/DeltaMerge/Remote/RNLocalPageCache.h index 6a48d9c260b..6ecd6460e4e 100644 --- a/dbms/src/Storages/DeltaMerge/Remote/RNLocalPageCache.h +++ b/dbms/src/Storages/DeltaMerge/Remote/RNLocalPageCache.h @@ -15,6 +15,7 @@ #pragma once #include +#include #include #include #include @@ -25,7 +26,6 @@ namespace DB { -class Context; class ReadBuffer; using ReadBufferPtr = std::shared_ptr; diff --git a/dbms/src/Storages/DeltaMerge/Remote/RNRemoteReadTask.cpp b/dbms/src/Storages/DeltaMerge/Remote/RNRemoteReadTask.cpp index c00b01165c6..1a934a56687 100644 --- a/dbms/src/Storages/DeltaMerge/Remote/RNRemoteReadTask.cpp +++ b/dbms/src/Storages/DeltaMerge/Remote/RNRemoteReadTask.cpp @@ -18,7 +18,7 @@ #include #include #include -#include +#include #include #include #include @@ -340,7 +340,7 @@ RNRemoteTableReadTaskPtr RNRemoteTableReadTask::buildFrom( }); futures.emplace_back(task->get_future()); - IOThreadPool::get().scheduleOrThrowOnError([task] { (*task)(); }); + RNRemoteReadTaskPool::get().scheduleOrThrowOnError([task] { (*task)(); }); } for (auto & f : futures) diff --git a/dbms/src/Storages/DeltaMerge/Remote/RNRemoteSegmentThreadInputStream.cpp b/dbms/src/Storages/DeltaMerge/Remote/RNRemoteSegmentThreadInputStream.cpp index 7a3607a161a..1461ee9b4db 100644 --- a/dbms/src/Storages/DeltaMerge/Remote/RNRemoteSegmentThreadInputStream.cpp +++ b/dbms/src/Storages/DeltaMerge/Remote/RNRemoteSegmentThreadInputStream.cpp @@ -29,7 +29,7 @@ namespace DB::DM BlockInputStreams RNRemoteSegmentThreadInputStream::buildInputStreams( const Context & db_context, const RNRemoteReadTaskPtr & remote_read_tasks, - const RNPagePreparerPtr & page_downloader, + const RNPagePreparerPtr & page_preparer, const DM::ColumnDefinesPtr & columns_to_read, UInt64 read_tso, size_t num_streams, @@ -46,7 +46,7 @@ BlockInputStreams RNRemoteSegmentThreadInputStream::buildInputStreams( BlockInputStreamPtr stream = std::make_shared( db_context, remote_read_tasks, - page_downloader, + page_preparer, *columns_to_read, rs_filter, read_tso, @@ -63,7 +63,7 @@ BlockInputStreams RNRemoteSegmentThreadInputStream::buildInputStreams( RNRemoteSegmentThreadInputStream::RNRemoteSegmentThreadInputStream( const Context & db_context_, RNRemoteReadTaskPtr read_tasks_, - RNPagePreparerPtr page_downloader_, + RNPagePreparerPtr page_preparer_, const ColumnDefines & columns_to_read_, const RSOperatorPtr & filter_, UInt64 max_version_, @@ -73,7 +73,7 @@ RNRemoteSegmentThreadInputStream::RNRemoteSegmentThreadInputStream( std::string_view req_id) : db_context(db_context_) , read_tasks(std::move(read_tasks_)) - , page_downloader(std::move(page_downloader_)) + , page_preparer(std::move(page_preparer_)) , columns_to_read(columns_to_read_) , filter(filter_) , header(toEmptyBlock(columns_to_read)) diff --git a/dbms/src/Storages/DeltaMerge/Remote/RNRemoteSegmentThreadInputStream.h b/dbms/src/Storages/DeltaMerge/Remote/RNRemoteSegmentThreadInputStream.h index 43f26590413..81c5811ee2c 100644 --- a/dbms/src/Storages/DeltaMerge/Remote/RNRemoteSegmentThreadInputStream.h +++ b/dbms/src/Storages/DeltaMerge/Remote/RNRemoteSegmentThreadInputStream.h @@ -18,7 +18,6 @@ #include #include #include -#include #include #include #include @@ -45,7 +44,7 @@ class RNRemoteSegmentThreadInputStream : public IProfilingBlockInputStream static BlockInputStreams buildInputStreams( const Context & db_context, const RNRemoteReadTaskPtr & remote_read_tasks, - const RNPagePreparerPtr & page_downloader, + const RNPagePreparerPtr & page_preparer, const DM::ColumnDefinesPtr & columns_to_read, UInt64 read_tso, size_t num_streams, @@ -58,7 +57,7 @@ class RNRemoteSegmentThreadInputStream : public IProfilingBlockInputStream RNRemoteSegmentThreadInputStream( const Context & db_context_, RNRemoteReadTaskPtr read_tasks_, - RNPagePreparerPtr page_downloader_, + RNPagePreparerPtr page_preparer_, const ColumnDefines & columns_to_read_, const RSOperatorPtr & filter_, UInt64 max_version_, @@ -85,7 +84,7 @@ class RNRemoteSegmentThreadInputStream : public IProfilingBlockInputStream private: const Context & db_context; RNRemoteReadTaskPtr read_tasks; - RNPagePreparerPtr page_downloader; + RNPagePreparerPtr page_preparer; ColumnDefines columns_to_read; RSOperatorPtr filter; Block header; diff --git a/dbms/src/Storages/DeltaMerge/Remote/Serializer.cpp b/dbms/src/Storages/DeltaMerge/Remote/Serializer.cpp index 926caf21d70..492cfcb3424 100644 --- a/dbms/src/Storages/DeltaMerge/Remote/Serializer.cpp +++ b/dbms/src/Storages/DeltaMerge/Remote/Serializer.cpp @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +#include #include #include #include @@ -86,7 +87,9 @@ Serializer::serializeTo( { auto * remote_file = remote.add_stable_pages(); remote_file->set_page_id(dt_file->pageId()); - remote_file->set_file_id(dt_file->fileId()); + auto * checkpoint_info = remote_file->mutable_checkpoint_info(); + RUNTIME_CHECK(startsWith(dt_file->path(), "s3://"), dt_file->path()); + checkpoint_info->set_data_file_id(dt_file->path()); // It should be a key to remote path } remote.mutable_column_files_memtable()->CopyFrom(serializeTo(snap->delta->getMemTableSetSnapshot())); remote.mutable_column_files_persisted()->CopyFrom(serializeTo(snap->delta->getPersistedFileSetSnapshot())); @@ -121,14 +124,10 @@ SegmentSnapshotPtr Serializer::deserializeSegmentSnapshotFrom( delta_snap->mem_table_snap = deserializeColumnFileSet( proto.column_files_memtable(), data_store, - remote_store_id, - table_id, segment_range); delta_snap->persisted_files_snap = deserializeColumnFileSet( proto.column_files_persisted(), data_store, - remote_store_id, - table_id, segment_range); // Note: At this moment, we still cannot read from `delta_snap->mem_table_snap` and `delta_snap->persisted_files_snap`, @@ -157,12 +156,8 @@ SegmentSnapshotPtr Serializer::deserializeSegmentSnapshotFrom( dmfiles.reserve(proto.stable_pages().size()); for (const auto & stable_file : proto.stable_pages()) { - auto oid = Remote::DMFileOID{ - .store_id = remote_store_id, - .table_id = table_id, - .file_id = stable_file.file_id(), - }; - auto prepared = data_store->prepareDMFile(oid); + auto remote_key = stable_file.checkpoint_info().data_file_id(); + auto prepared = data_store->prepareDMFileByKey(remote_key); auto dmfile = prepared->restore(DMFile::ReadMetaMode::all()); dmfiles.emplace_back(std::move(dmfile)); } @@ -208,8 +203,6 @@ Serializer::serializeTo(const ColumnFileSetSnapshotPtr & snap) ColumnFileSetSnapshotPtr Serializer::deserializeColumnFileSet( const RepeatedPtrField & proto, const Remote::IDataStorePtr & data_store, - StoreID remote_store_id, - TableID table_id, const RowKeyRange & segment_range) { auto empty_data_provider = std::make_shared(); @@ -230,14 +223,8 @@ ColumnFileSetSnapshotPtr Serializer::deserializeColumnFileSet( else if (remote_column_file.has_big()) { const auto & big_file = remote_column_file.big(); - auto file_oid = Remote::DMFileOID{ - .store_id = remote_store_id, - .table_id = table_id, - .file_id = big_file.file_id(), - }; ret->column_files.push_back(deserializeCFBig( big_file, - file_oid, data_store, segment_range)); } @@ -334,6 +321,8 @@ RemotePb::ColumnFileRemote Serializer::serializeTo(const ColumnFileTiny & cf_tin remote_tiny->set_rows(cf_tiny.rows); remote_tiny->set_bytes(cf_tiny.bytes); + // TODO: read the checkpoint info from data_provider and send it to the compute node + return ret; } @@ -378,7 +367,8 @@ RemotePb::ColumnFileRemote Serializer::serializeTo(const ColumnFileBig & cf_big) { RemotePb::ColumnFileRemote ret; auto * remote_big = ret.mutable_big(); - remote_big->set_file_id(cf_big.file->fileId()); + auto * checkpoint_info = remote_big->mutable_checkpoint_info(); + checkpoint_info->set_data_file_id(cf_big.file->path()); remote_big->set_page_id(cf_big.file->pageId()); remote_big->set_valid_rows(cf_big.valid_rows); remote_big->set_valid_bytes(cf_big.valid_bytes); @@ -387,14 +377,13 @@ RemotePb::ColumnFileRemote Serializer::serializeTo(const ColumnFileBig & cf_big) ColumnFileBigPtr Serializer::deserializeCFBig( const RemotePb::ColumnFileBig & proto, - const Remote::DMFileOID & oid, const Remote::IDataStorePtr & data_store, const RowKeyRange & segment_range) { - RUNTIME_CHECK(proto.file_id() == oid.file_id); - LOG_DEBUG(Logger::get(), "Rebuild local ColumnFileBig from remote, dmf_oid={}", oid); + RUNTIME_CHECK(proto.has_checkpoint_info()); + LOG_DEBUG(Logger::get(), "Rebuild local ColumnFileBig from remote, key={}", proto.checkpoint_info().data_file_id()); - auto prepared = data_store->prepareDMFile(oid); + auto prepared = data_store->prepareDMFileByKey(proto.checkpoint_info().data_file_id()); auto dmfile = prepared->restore(DMFile::ReadMetaMode::all()); auto * cf_big = new ColumnFileBig(dmfile, proto.valid_rows(), proto.valid_bytes(), segment_range); return std::shared_ptr(cf_big); // The constructor is private, so we cannot use make_shared. diff --git a/dbms/src/Storages/DeltaMerge/Remote/Serializer.h b/dbms/src/Storages/DeltaMerge/Remote/Serializer.h index 540b6ecdbc2..0ede1479a65 100644 --- a/dbms/src/Storages/DeltaMerge/Remote/Serializer.h +++ b/dbms/src/Storages/DeltaMerge/Remote/Serializer.h @@ -84,8 +84,6 @@ struct Serializer static ColumnFileSetSnapshotPtr deserializeColumnFileSet( const google::protobuf::RepeatedPtrField & proto, const Remote::IDataStorePtr & data_store, - StoreID remote_store_id, - TableID table_id, const RowKeyRange & segment_range); /// column file /// @@ -102,7 +100,6 @@ struct Serializer static RemotePb::ColumnFileRemote serializeTo(const ColumnFileBig & cf_big); static ColumnFileBigPtr deserializeCFBig( const RemotePb::ColumnFileBig & proto, - const Remote::DMFileOID & oid, const Remote::IDataStorePtr & data_store, const RowKeyRange & segment_range); }; diff --git a/dbms/src/Storages/DeltaMerge/Remote/WNDisaggSnapshotManager.cpp b/dbms/src/Storages/DeltaMerge/Remote/WNDisaggSnapshotManager.cpp index 7f0f22a2032..ee9bce370c6 100644 --- a/dbms/src/Storages/DeltaMerge/Remote/WNDisaggSnapshotManager.cpp +++ b/dbms/src/Storages/DeltaMerge/Remote/WNDisaggSnapshotManager.cpp @@ -17,7 +17,6 @@ namespace DB::DM::Remote { - WNDisaggSnapshotManager::WNDisaggSnapshotManager(BackgroundProcessingPool & bg_pool) : pool(bg_pool) , log(Logger::get()) diff --git a/dbms/src/Storages/DeltaMerge/Remote/WNDisaggSnapshotManager.h b/dbms/src/Storages/DeltaMerge/Remote/WNDisaggSnapshotManager.h index b8a3f3bb73d..1918ae57c42 100644 --- a/dbms/src/Storages/DeltaMerge/Remote/WNDisaggSnapshotManager.h +++ b/dbms/src/Storages/DeltaMerge/Remote/WNDisaggSnapshotManager.h @@ -31,7 +31,6 @@ namespace DB::DM::Remote { - /** * WNDisaggSnapshotManager holds all snapshots for disaggregated read tasks * in the write node. It's a single instance for each TiFlash node. @@ -41,8 +40,8 @@ class WNDisaggSnapshotManager public: struct SnapshotWithExpireTime { - const DisaggReadSnapshotPtr snap; - const Timepoint expired_at; + DisaggReadSnapshotPtr snap; + Timepoint expired_at; }; public: @@ -50,15 +49,16 @@ class WNDisaggSnapshotManager ~WNDisaggSnapshotManager(); - bool registerSnapshot(const DisaggTaskId & task_id, DisaggReadSnapshotPtr && snap, const Timepoint & expired_at) + bool registerSnapshot(const DisaggTaskId & task_id, const DisaggReadSnapshotPtr & snap, const Timepoint & expired_at) { - LOG_DEBUG(log, "Register Disaggregated Snapshot, task_id={}", task_id); - std::unique_lock lock(mtx); - if (auto iter = snapshots.find(task_id); iter != snapshots.end()) - return false; + LOG_DEBUG(log, "Register Disaggregated Snapshot, task_id={}", task_id); - snapshots.emplace(task_id, SnapshotWithExpireTime{.snap = std::move(snap), .expired_at = expired_at}); + // Since EstablishDisagg may be retried, there may be existing snapshot. + // We replace these existing snapshot using a new one. + snapshots.insert_or_assign( + task_id, + SnapshotWithExpireTime{.snap = snap, .expired_at = expired_at}); return true; } @@ -77,11 +77,10 @@ class WNDisaggSnapshotManager private: bool unregisterSnapshot(const DisaggTaskId & task_id) { - LOG_DEBUG(log, "Unregister Disaggregated Snapshot, task_id={}", task_id); - std::unique_lock lock(mtx); if (auto iter = snapshots.find(task_id); iter != snapshots.end()) { + LOG_DEBUG(log, "Unregister Disaggregated Snapshot, task_id={}", task_id); snapshots.erase(iter); return true; } diff --git a/dbms/src/Storages/DeltaMerge/Remote/WNDisaggSnapshotManager_fwd.h b/dbms/src/Storages/DeltaMerge/Remote/WNDisaggSnapshotManager_fwd.h index bf78c0f59a4..54597a32a6c 100644 --- a/dbms/src/Storages/DeltaMerge/Remote/WNDisaggSnapshotManager_fwd.h +++ b/dbms/src/Storages/DeltaMerge/Remote/WNDisaggSnapshotManager_fwd.h @@ -18,7 +18,6 @@ namespace DB::DM::Remote { - class WNDisaggSnapshotManager; using WNDisaggSnapshotManagerPtr = std::shared_ptr; diff --git a/dbms/src/Storages/DeltaMerge/Segment.cpp b/dbms/src/Storages/DeltaMerge/Segment.cpp index a2a18f74384..cf0aee022d5 100644 --- a/dbms/src/Storages/DeltaMerge/Segment.cpp +++ b/dbms/src/Storages/DeltaMerge/Segment.cpp @@ -45,8 +45,11 @@ #include #include #include +#include +#include #include #include +#include #include #include #include @@ -192,6 +195,7 @@ StableValueSpacePtr createNewStable( // if (auto data_store = context.db_context.getSharedContextDisagg()->remote_data_store; !data_store) { wbs.data.putExternal(dtfile_id, 0); + delegator.addDTFile(dtfile_id, dtfile->getBytesOnDisk(), store_path); } else { @@ -205,7 +209,6 @@ StableValueSpacePtr createNewStable( // }; wbs.data.putRemoteExternal(dtfile_id, loc); } - delegator.addDTFile(dtfile_id, dtfile->getBytesOnDisk(), store_path); return stable; } @@ -286,53 +289,141 @@ SegmentPtr Segment::newSegment( // context.storage_pool->newMetaPageId()); } -SegmentPtr Segment::restoreSegment( // - const LoggerPtr & parent_log, - DMContext & context, - PageIdU64 segment_id) +inline void readSegmentMetaInfo(ReadBuffer & buf, Segment::SegmentMetaInfo & segment_info) { - Page page = context.storage_pool->metaReader()->read(segment_id); // not limit restore - - ReadBufferFromMemory buf(page.data.begin(), page.data.size()); - SegmentFormat::Version version; - - readIntBinary(version, buf); - UInt64 epoch; - RowKeyRange rowkey_range; - PageIdU64 next_segment_id, delta_id, stable_id; - - readIntBinary(epoch, buf); + readIntBinary(segment_info.version, buf); + readIntBinary(segment_info.epoch, buf); - switch (version) + switch (segment_info.version) { case SegmentFormat::V1: { HandleRange range; readIntBinary(range.start, buf); readIntBinary(range.end, buf); - rowkey_range = RowKeyRange::fromHandleRange(range); + segment_info.range = RowKeyRange::fromHandleRange(range); break; } case SegmentFormat::V2: { - rowkey_range = RowKeyRange::deserialize(buf); + segment_info.range = RowKeyRange::deserialize(buf); break; } default: - throw Exception(fmt::format("Illegal version: {}", version), ErrorCodes::LOGICAL_ERROR); + throw Exception(fmt::format("Illegal version: {}", segment_info.version), ErrorCodes::LOGICAL_ERROR); } - readIntBinary(next_segment_id, buf); - readIntBinary(delta_id, buf); - readIntBinary(stable_id, buf); + readIntBinary(segment_info.next_segment_id, buf); + readIntBinary(segment_info.delta_id, buf); + readIntBinary(segment_info.stable_id, buf); +} + +SegmentPtr Segment::restoreSegment( // + const LoggerPtr & parent_log, + DMContext & context, + PageIdU64 segment_id) +{ + Page page = context.storage_pool->metaReader()->read(segment_id); // not limit restore + + ReadBufferFromMemory buf(page.data.begin(), page.data.size()); + Segment::SegmentMetaInfo segment_info; + readSegmentMetaInfo(buf, segment_info); - auto delta = DeltaValueSpace::restore(context, rowkey_range, delta_id); - auto stable = StableValueSpace::restore(context, stable_id); - auto segment = std::make_shared(parent_log, epoch, rowkey_range, segment_id, next_segment_id, delta, stable); + auto delta = DeltaValueSpace::restore(context, segment_info.range, segment_info.delta_id); + auto stable = StableValueSpace::restore(context, segment_info.stable_id); + auto segment = std::make_shared(parent_log, segment_info.epoch, segment_info.range, segment_id, segment_info.next_segment_id, delta, stable); return segment; } +Segment::SegmentMetaInfos Segment::readAllSegmentsMetaInfoInRange( // + DMContext & context, + UInt64 remote_store_id, + NamespaceId ns_id, + const RowKeyRange & target_range, + UniversalPageStoragePtr temp_ps) +{ + PageIdU64 current_segment_id = 1; // DELTA_MERGE_FIRST_SEGMENT_ID + SegmentMetaInfos segment_infos; + auto fap_context = context.db_context.getSharedContextDisagg()->fap_context; + TableIdentifier identifier{ + .key_space_id = 0, + .store_id = remote_store_id, + .table_id = ns_id, + }; + auto first_segment_id = fap_context->getSegmentIdContainingKey(identifier, target_range.getStart().toRowKeyValue()); + if (first_segment_id != 0) + { + bool hit = false; + auto target_id = UniversalPageIdFormat::toFullPageId(UniversalPageIdFormat::toFullPrefix(StorageType::Meta, ns_id), first_segment_id); + auto page = temp_ps->read(target_id, /*read_limiter*/ nullptr, {}, /*throw_on_not_exist*/ false); + if (page.isValid()) + { + Segment::SegmentMetaInfo segment_info; + segment_info.segment_id = first_segment_id; + ReadBufferFromMemory buf(page.data.begin(), page.data.size()); + readSegmentMetaInfo(buf, segment_info); + if (segment_info.range.check(target_range.getStart())) + { + segment_infos.push_back(segment_info); + current_segment_id = segment_info.next_segment_id; + hit = true; + } + } + if (!hit) + { + fap_context->invalidateCache(identifier); + } + } + std::vector> end_key_and_segment_ids; + LOG_DEBUG(Logger::get(), "Read segment meta info from segment {}", current_segment_id); + while (current_segment_id != 0) + { + Segment::SegmentMetaInfo segment_info; + auto target_id = UniversalPageIdFormat::toFullPageId(UniversalPageIdFormat::toFullPrefix(StorageType::Meta, ns_id), current_segment_id); + auto page = temp_ps->read(target_id); + segment_info.segment_id = current_segment_id; + ReadBufferFromMemory buf(page.data.begin(), page.data.size()); + readSegmentMetaInfo(buf, segment_info); + end_key_and_segment_ids.emplace_back(segment_info.range.getEnd().toRowKeyValue(), segment_info.segment_id); + current_segment_id = segment_info.next_segment_id; + if (!(segment_info.range.shrink(target_range).none())) + { + segment_infos.emplace_back(segment_info); + } + if (segment_info.range.end.value->compare(*target_range.end.value) >= 0) + { + break; + } + } + fap_context->insertSegmentEndKeyInfoToCache(identifier, end_key_and_segment_ids); + return segment_infos; +} + +Segments Segment::createTargetSegmentsFromCheckpoint( // + const LoggerPtr & parent_log, + DMContext & context, + UInt64 remote_store_id, + NamespaceId ns_id, + const SegmentMetaInfos & meta_infos, + const RowKeyRange & range, + UniversalPageStoragePtr temp_ps, + WriteBatches & wbs) +{ + UNUSED(remote_store_id); + Segments segments; + for (const auto & segment_info : meta_infos) + { + LOG_DEBUG(parent_log, "Create segment begin. Delta id {} stable id {} range {} epoch {} next_segment_id {}", segment_info.delta_id, segment_info.stable_id, segment_info.range.toDebugString(), segment_info.epoch, segment_info.next_segment_id); + auto stable = StableValueSpace::createFromCheckpoint(context, temp_ps, ns_id, segment_info.stable_id, wbs); + auto delta = DeltaValueSpace::createFromCheckpoint(context, temp_ps, segment_info.range, ns_id, segment_info.delta_id, wbs); + auto segment = std::make_shared(Logger::get("Checkpoint"), segment_info.epoch, segment_info.range.shrink(range), segment_info.segment_id, segment_info.next_segment_id, delta, stable); + segments.push_back(segment); + LOG_DEBUG(parent_log, "Create segment end. Delta id {} stable id {} range {} epoch {} next_segment_id {}", segment_info.delta_id, segment_info.stable_id, segment_info.range.toDebugString(), segment_info.epoch, segment_info.next_segment_id); + } + return segments; +} + void Segment::serialize(WriteBatchWrapper & wb) { MemoryWriteBuffer buf(0, SEGMENT_BUFFER_SIZE); @@ -1062,6 +1153,97 @@ SegmentPtr Segment::replaceData(const Segment::Lock & lock, // return new_me; } +SegmentPtr Segment::dangerouslyReplaceDataFromCheckpoint(const Segment::Lock &, // + DMContext & dm_context, + const DMFilePtr & data_file, + WriteBatches & wbs, + const ColumnFilePersisteds & column_file_persisteds) const +{ + LOG_DEBUG(log, "ReplaceData - Begin, data_file={}, column_files_num={}", data_file->path(), column_file_persisteds.size()); + + auto & storage_pool = dm_context.storage_pool; + auto delegate = dm_context.path_pool->getStableDiskDelegator(); + + // Always create a ref to the file to allow `data_file` being shared. + auto new_page_id = storage_pool->newDataPageIdForDTFile(delegate, __PRETTY_FUNCTION__); + auto ref_file = DMFile::restore( + dm_context.db_context.getFileProvider(), + data_file->fileId(), + new_page_id, + data_file->parentPath(), + DMFile::ReadMetaMode::all()); + wbs.data.putRefPage(new_page_id, data_file->pageId()); + + auto new_stable = std::make_shared(stable->getId()); + new_stable->setFiles({ref_file}, rowkey_range, &dm_context); + new_stable->saveMeta(wbs.meta); + + ColumnFilePersisteds new_column_file_persisteds; + for (const auto & column_file : column_file_persisteds) + { + if (auto * t = column_file->tryToTinyFile(); t) + { + // This column file may be ingested into multiple segments, so we cannot reuse its page id here. + auto new_cf_id = storage_pool->newLogPageId(); + wbs.log.putRefPage(new_cf_id, t->getDataPageId()); + new_column_file_persisteds.push_back(t->cloneWith(new_cf_id)); + } + else if (auto * d = column_file->tryToDeleteRange(); d) + { + new_column_file_persisteds.push_back(column_file); + } + else if (auto * b = column_file->tryToBigFile(); b) + { + auto new_file_id = storage_pool->newDataPageIdForDTFile(delegate, __PRETTY_FUNCTION__); + auto old_page_id = b->getDataPageId(); + wbs.data.putRefPage(new_file_id, old_page_id); + auto old_file_id = b->getFile()->fileId(); + String file_parent_path; + if (dm_context.db_context.getSharedContextDisagg()->remote_data_store) + { + auto wn_ps = dm_context.db_context.getWriteNodePageStorage(); + auto full_page_id = UniversalPageIdFormat::toFullPageId(UniversalPageIdFormat::toFullPrefix(StorageType::Data, dm_context.storage_pool->getNamespaceId()), old_page_id); + auto remote_data_location = wn_ps->getCheckpointLocation(full_page_id); + const auto & lock_key_view = S3::S3FilenameView::fromKey(*(remote_data_location->data_file_id)); + file_parent_path = S3::S3Filename::fromTableID(lock_key_view.store_id, dm_context.storage_pool->getNamespaceId()).toFullKeyWithPrefix(); + } + else + { + file_parent_path = delegate.getDTFilePath(old_file_id); + } + auto new_dmfile = DMFile::restore(dm_context.db_context.getFileProvider(), old_file_id, new_file_id, file_parent_path, DMFile::ReadMetaMode::all()); + auto new_column_file = b->cloneWith(dm_context, new_dmfile, rowkey_range); + new_column_file_persisteds.push_back(new_column_file); + } + else + { + RUNTIME_CHECK(false); + } + } + + auto new_delta = std::make_shared( + delta->getId(), + new_column_file_persisteds); + new_delta->saveMeta(wbs); + + auto new_me = std::make_shared( // + parent_log, + epoch + 1, + rowkey_range, + segment_id, + next_segment_id, + new_delta, + new_stable); + new_me->serialize(wbs.meta); + + delta->recordRemoveColumnFilesPages(wbs); + stable->recordRemovePacksPages(wbs); + + LOG_DEBUG(log, "ReplaceData - Finish, old_me={} new_me={}", info(), new_me->info()); + + return new_me; +} + SegmentPair Segment::split(DMContext & dm_context, const ColumnDefinesPtr & schema_snap, std::optional opt_split_at, SplitMode opt_split_mode) const { WriteBatches wbs(*dm_context.storage_pool, dm_context.getWriteLimiter()); @@ -1391,7 +1573,11 @@ Segment::prepareSplitLogical( // { auto ori_page_id = dmfile->pageId(); auto file_id = dmfile->fileId(); - auto file_parent_path = delegate.getDTFilePath(file_id); + auto file_parent_path = dmfile->parentPath(); + if (!dm_context.db_context.getSharedContextDisagg()->remote_data_store) + { + RUNTIME_CHECK(file_parent_path == delegate.getDTFilePath(file_id)); + } auto my_dmfile_page_id = storage_pool->newDataPageIdForDTFile(delegate, __PRETTY_FUNCTION__); auto other_dmfile_page_id = storage_pool->newDataPageIdForDTFile(delegate, __PRETTY_FUNCTION__); diff --git a/dbms/src/Storages/DeltaMerge/Segment.h b/dbms/src/Storages/DeltaMerge/Segment.h index cd59d57943a..b44facd69c8 100644 --- a/dbms/src/Storages/DeltaMerge/Segment.h +++ b/dbms/src/Storages/DeltaMerge/Segment.h @@ -142,6 +142,37 @@ class Segment static SegmentPtr restoreSegment(const LoggerPtr & parent_log, DMContext & context, PageIdU64 segment_id); + struct SegmentMetaInfo + { + SegmentFormat::Version version; + UInt64 epoch; + RowKeyRange range; + PageIdU64 segment_id; + PageIdU64 next_segment_id; + PageIdU64 delta_id; + PageIdU64 stable_id; + }; + + using SegmentMetaInfos = std::vector; + static SegmentMetaInfos readAllSegmentsMetaInfoInRange( // + DMContext & context, + UInt64 remote_store_id, + NamespaceId ns_id, + const RowKeyRange & target_range, + UniversalPageStoragePtr temp_ps); + + // Create a list of temp segments from checkpoint. + // The data of these temp segments will be included in `wbs`. + static Segments createTargetSegmentsFromCheckpoint( // + const LoggerPtr & parent_log, + DMContext & context, + UInt64 remote_store_id, + NamespaceId ns_id, + const SegmentMetaInfos & meta_infos, + const RowKeyRange & range, + UniversalPageStoragePtr temp_ps, + WriteBatches & wbs); + void serialize(WriteBatchWrapper & wb); /// Attach a new ColumnFile into the Segment. The ColumnFile will be added to MemFileSet and flushed to disk later. @@ -437,6 +468,8 @@ class Segment */ [[nodiscard]] SegmentPtr replaceData(const Lock &, DMContext & dm_context, const DMFilePtr & data_file, SegmentSnapshotPtr segment_snap_opt = nullptr) const; + [[nodiscard]] SegmentPtr dangerouslyReplaceDataFromCheckpoint(const Lock &, DMContext & dm_context, const DMFilePtr & data_file, WriteBatches & wbs, const ColumnFilePersisteds & column_file_persisteds) const; + [[nodiscard]] SegmentPtr dropNextSegment(WriteBatches & wbs, const RowKeyRange & next_segment_range); /** diff --git a/dbms/src/Storages/DeltaMerge/StableValueSpace.cpp b/dbms/src/Storages/DeltaMerge/StableValueSpace.cpp index 42257449b4f..2159251802d 100644 --- a/dbms/src/Storages/DeltaMerge/StableValueSpace.cpp +++ b/dbms/src/Storages/DeltaMerge/StableValueSpace.cpp @@ -19,12 +19,15 @@ #include #include #include +#include #include #include #include #include #include +#include #include + namespace DB { namespace ErrorCodes @@ -103,22 +106,30 @@ StableValueSpacePtr StableValueSpace::restore(DMContext & context, PageIdU64 id) readIntBinary(valid_bytes, buf); readIntBinary(size, buf); UInt64 page_id; - bool restore_from_s3 = context.db_context.getSharedContextDisagg()->remote_data_store != nullptr; + auto remote_data_store = context.db_context.getSharedContextDisagg()->remote_data_store; for (size_t i = 0; i < size; ++i) { readIntBinary(page_id, buf); - auto file_id = context.storage_pool->dataReader()->getNormalPageId(page_id); - auto path_delegate = context.path_pool->getStableDiskDelegator(); - auto file_parent_path = restore_from_s3 ? path_delegate.getS3DTFile(file_id) : path_delegate.getDTFilePath(file_id); - auto dmfile = DMFile::restore(context.db_context.getFileProvider(), file_id, page_id, file_parent_path, DMFile::ReadMetaMode::all()); - if (restore_from_s3) + DMFilePtr dmfile; + if (remote_data_store) { - path_delegate.addS3DTFileSize(file_id, dmfile->getBytesOnDisk()); + auto wn_ps = context.db_context.getWriteNodePageStorage(); + auto full_page_id = UniversalPageIdFormat::toFullPageId(UniversalPageIdFormat::toFullPrefix(StorageType::Data, context.storage_pool->getNamespaceId()), page_id); + auto remote_data_location = wn_ps->getCheckpointLocation(full_page_id); + const auto & lock_key_view = S3::S3FilenameView::fromKey(*(remote_data_location->data_file_id)); + auto dtfile_key = lock_key_view.asDataFile(); + auto file_oid = dtfile_key.getDMFileOID(); + RUNTIME_CHECK(file_oid.table_id == context.physical_table_id); + auto prepared = remote_data_store->prepareDMFile(file_oid, page_id); + dmfile = prepared->restore(DMFile::ReadMetaMode::all()); } else { + auto path_delegate = context.path_pool->getStableDiskDelegator(); + auto file_parent_path = path_delegate.getDTFilePath(file_id); + dmfile = DMFile::restore(context.db_context.getFileProvider(), file_id, page_id, file_parent_path, DMFile::ReadMetaMode::all()); auto res = path_delegate.updateDTFileSize(file_id, dmfile->getBytesOnDisk()); RUNTIME_CHECK_MSG(res, "update dt file size failed, path={}", dmfile->path()); } @@ -131,6 +142,67 @@ StableValueSpacePtr StableValueSpace::restore(DMContext & context, PageIdU64 id) return stable; } +StableValueSpacePtr StableValueSpace::createFromCheckpoint( // + DMContext & context, + UniversalPageStoragePtr temp_ps, + TableID ns_id, + PageIdU64 stable_id, + WriteBatches & wbs) +{ + auto stable = std::make_shared(stable_id); + + auto stable_page_id = UniversalPageIdFormat::toFullPageId(UniversalPageIdFormat::toFullPrefix(StorageType::Meta, ns_id), stable_id); + auto page = temp_ps->read(stable_page_id); + ReadBufferFromMemory buf(page.data.begin(), page.data.size()); + + // read stable meta info + UInt64 version, valid_rows, valid_bytes, size; + { + readIntBinary(version, buf); + if (version != StableFormat::V1) + throw Exception("Unexpected version: " + DB::toString(version)); + + readIntBinary(valid_rows, buf); + readIntBinary(valid_bytes, buf); + readIntBinary(size, buf); + } + + for (size_t i = 0; i < size; ++i) + { + UInt64 page_id; + readIntBinary(page_id, buf); + auto remote_page_id = UniversalPageIdFormat::toFullPageId(UniversalPageIdFormat::toFullPrefix(StorageType::Data, ns_id), page_id); + auto remote_file_id = temp_ps->getNormalPageId(remote_page_id); + auto file_id = UniversalPageIdFormat::getU64ID(remote_file_id); + auto remote_data_location = temp_ps->getCheckpointLocation(remote_page_id); + const auto & lock_key_view = S3::S3FilenameView::fromKey(*(remote_data_location->data_file_id)); + S3::DMFileOID file_oid{ + .store_id = lock_key_view.store_id, + .table_id = ns_id, + .file_id = file_id}; + RUNTIME_CHECK(lock_key_view.asDataFile().toFullKey() == S3::S3Filename::fromDMFileOID(file_oid).toFullKey()); + auto data_key = S3::S3Filename::fromDMFileOID(file_oid).toFullKey(); + auto delegator = context.path_pool->getStableDiskDelegator(); + auto new_local_page_id = context.storage_pool->newDataPageIdForDTFile(delegator, __PRETTY_FUNCTION__); + PS::V3::CheckpointLocation loc{ + .data_file_id = std::make_shared(data_key), + .offset_in_file = 0, + .size_in_file = 0, + }; + wbs.data.putRemoteExternal(new_local_page_id, loc); + + auto parent_path = S3::S3Filename::fromTableID(lock_key_view.store_id, ns_id).toFullKeyWithPrefix(); + auto new_dmfile = DMFile::restore(context.db_context.getFileProvider(), file_id, new_local_page_id, parent_path, DMFile::ReadMetaMode::all()); + wbs.writeLogAndData(); + stable->files.push_back(new_dmfile); + } + + stable->valid_rows = valid_rows; + stable->valid_bytes = valid_bytes; + + return stable; +} + size_t StableValueSpace::getRows() const { return valid_rows; diff --git a/dbms/src/Storages/DeltaMerge/StableValueSpace.h b/dbms/src/Storages/DeltaMerge/StableValueSpace.h index ba0bd2ae411..8eb0ef37cff 100644 --- a/dbms/src/Storages/DeltaMerge/StableValueSpace.h +++ b/dbms/src/Storages/DeltaMerge/StableValueSpace.h @@ -46,6 +46,13 @@ class StableValueSpace : public std::enable_shared_from_this static StableValueSpacePtr restore(DMContext & context, PageIdU64 id); + static StableValueSpacePtr createFromCheckpoint( // + DMContext & context, + UniversalPageStoragePtr temp_ps, + TableID ns_id, + PageIdU64 stable_id, + WriteBatches & wbs); + /** * Resets the logger by using the one from the segment. * Segment_log is not available when constructing, because usually diff --git a/dbms/src/Storages/DeltaMerge/tests/gtest_dm_delta_merge_store_fast_add_peer.cpp b/dbms/src/Storages/DeltaMerge/tests/gtest_dm_delta_merge_store_fast_add_peer.cpp new file mode 100644 index 00000000000..dd6affa3ea0 --- /dev/null +++ b/dbms/src/Storages/DeltaMerge/tests/gtest_dm_delta_merge_store_fast_add_peer.cpp @@ -0,0 +1,370 @@ +// 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 +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ +namespace FailPoints +{ +extern const char force_use_dmfile_format_v3[]; +} // namespace FailPoints +namespace DM +{ +extern DMFilePtr writeIntoNewDMFile(DMContext & dm_context, + const ColumnDefinesPtr & schema_snap, + const BlockInputStreamPtr & input_stream, + UInt64 file_id, + const String & parent_path); +namespace tests +{ +// Simple test suit for DeltaMergeStoreTestFastAddPeer. +class DeltaMergeStoreTestFastAddPeer : public DB::base::TiFlashStorageTestBasic +{ +public: + void SetUp() override + { + FailPointHelper::enableFailPoint(FailPoints::force_use_dmfile_format_v3); + auto s3_client = S3::ClientFactory::instance().sharedTiFlashClient(); + ASSERT_TRUE(::DB::tests::TiFlashTestEnv::createBucketIfNotExist(*s3_client)); + TiFlashStorageTestBasic::SetUp(); + auto & global_context = TiFlashTestEnv::getGlobalContext(); + if (global_context.getSharedContextDisagg()->remote_data_store == nullptr) + { + already_initialize_data_store = false; + global_context.getSharedContextDisagg()->initRemoteDataStore(global_context.getFileProvider(), /*s3_enabled*/ true); + ASSERT_TRUE(global_context.getSharedContextDisagg()->remote_data_store != nullptr); + } + else + { + already_initialize_data_store = true; + } + if (global_context.getWriteNodePageStorage() == nullptr) + { + already_initialize_write_ps = false; + orig_mode = global_context.getPageStorageRunMode(); + global_context.setPageStorageRunMode(PageStorageRunMode::UNI_PS); + global_context.tryReleaseWriteNodePageStorageForTest(); + global_context.initializeWriteNodePageStorageIfNeed(global_context.getPathPool()); + } + else + { + already_initialize_write_ps = true; + } + auto kvstore = db_context->getTMTContext().getKVStore(); + { + auto meta_store = metapb::Store{}; + meta_store.set_id(store_id); + kvstore->setStore(meta_store); + } + global_context.getSharedContextDisagg()->initFastAddPeerContext(); + } + + void TearDown() override + { + FailPointHelper::disableFailPoint(FailPoints::force_use_dmfile_format_v3); + auto & global_context = TiFlashTestEnv::getGlobalContext(); + if (!already_initialize_data_store) + { + global_context.getSharedContextDisagg()->remote_data_store = nullptr; + } + if (!already_initialize_write_ps) + { + global_context.setPageStorageRunMode(orig_mode); + } + } + + DeltaMergeStorePtr + reload(const ColumnDefinesPtr & pre_define_columns = {}, bool is_common_handle = false, size_t rowkey_column_size = 1) + { + TiFlashStorageTestBasic::reload(); + if (auto ps = DB::tests::TiFlashTestEnv::getGlobalContext().getWriteNodePageStorage(); ps) + { + auto mock_s3lock_client = std::make_shared(DB::S3::ClientFactory::instance().sharedTiFlashClient()); + ps->initLocksLocalManager(store_id, mock_s3lock_client); + } + ColumnDefinesPtr cols; + if (!pre_define_columns) + cols = DMTestEnv::getDefaultColumns(is_common_handle ? DMTestEnv::PkType::CommonHandle : DMTestEnv::PkType::HiddenTiDBRowID); + else + cols = pre_define_columns; + + ColumnDefine handle_column_define = (*cols)[0]; + + DeltaMergeStorePtr s = std::make_shared(*db_context, + false, + "test", + "t_100", + 100, + true, + *cols, + handle_column_define, + is_common_handle, + rowkey_column_size, + DeltaMergeStore::Settings()); + return s; + } + +protected: + std::pair> genDMFile(DMContext & context, const Block & block) + { + auto input_stream = std::make_shared(block); + auto [store_path, file_id] = store->preAllocateIngestFile(); + + auto dmfile = writeIntoNewDMFile( + context, + std::make_shared(store->getTableColumns()), + input_stream, + file_id, + store_path); + + store->preIngestFile(store_path, file_id, dmfile->getBytesOnDisk()); + + const auto & pk_column = block.getByPosition(0).column; + auto min_pk = pk_column->getInt(0); + auto max_pk = pk_column->getInt(block.rows() - 1); + HandleRange range(min_pk, max_pk + 1); + auto handle_range = RowKeyRange::fromHandleRange(range); + auto external_file = ExternalDTFileInfo{.id = file_id, .range = handle_range}; + return {handle_range, {external_file}}; // There are some duplicated info. This is to minimize the change to our test code. + } + + void dumpCheckpoint() + { + auto temp_dir = getTemporaryPath() + "/"; + auto page_storage = db_context->getWriteNodePageStorage(); + auto wi = PS::V3::CheckpointProto::WriterInfo(); + { + wi.set_store_id(store_id); + } + + + auto remote_store = db_context->getSharedContextDisagg()->remote_data_store; + assert(remote_store != nullptr); + UniversalPageStorage::DumpCheckpointOptions opts{ + .data_file_id_pattern = S3::S3Filename::newCheckpointDataNameTemplate(store_id, upload_sequence), + .data_file_path_pattern = temp_dir + "dat_{seq}_{index}", + .manifest_file_id_pattern = S3::S3Filename::newCheckpointManifestNameTemplate(store_id), + .manifest_file_path_pattern = temp_dir + "mf_{seq}", + .writer_info = wi, + .must_locked_files = {}, + .persist_checkpoint = CheckpointUploadFunctor{ + .store_id = store_id, + // Note that we use `upload_sequence` but not `snapshot.sequence` for + // the S3 key. + .sequence = upload_sequence, + .remote_store = remote_store, + }, + .override_sequence = upload_sequence, // override by upload_sequence + }; + page_storage->dumpIncrementalCheckpoint(opts); + } + + void clearData() + { + // clear data + store->clearData(); + auto table_column_defines = DMTestEnv::getDefaultColumns(); + store = reload(table_column_defines); + store->deleteRange(*db_context, db_context->getSettingsRef(), RowKeyRange::newAll(false, 1)); + store->flushCache(*db_context, RowKeyRange::newAll(false, 1), true); + store->mergeDeltaAll(*db_context); + } + + void verifyRows(const RowKeyRange & range, size_t rows) + { + const auto & columns = store->getTableColumns(); + BlockInputStreamPtr in = store->read(*db_context, + db_context->getSettingsRef(), + columns, + {range}, + /* num_streams= */ 1, + /* max_version= */ std::numeric_limits::max(), + EMPTY_FILTER, + TRACING_NAME, + /* keep_order= */ false, + /* is_fast_scan= */ false, + /* expected_block_size= */ 1024)[0]; + ASSERT_INPUTSTREAM_NROWS(in, rows); + } + +protected: + DeltaMergeStorePtr store; + UInt64 store_id = 100; + UInt64 upload_sequence = 1000; + bool already_initialize_data_store = false; + bool already_initialize_write_ps = false; + DB::PageStorageRunMode orig_mode; + + constexpr static const char * TRACING_NAME = "DeltaMergeStoreTestFastAddPeer"; +}; + +TEST_F(DeltaMergeStoreTestFastAddPeer, SimpleWriteReadAfterRestoreFromCheckPoint) +try +{ + { + auto table_column_defines = DMTestEnv::getDefaultColumns(); + + store = reload(table_column_defines); + } + + const size_t num_rows_write = 128; + // write DMFile + { + Block block = DMTestEnv::prepareSimpleWriteBlock(0, num_rows_write, false); + store->write(*db_context, db_context->getSettingsRef(), block); + store->flushCache(*db_context, RowKeyRange::newAll(false, 1), true); + store->mergeDeltaAll(*db_context); + } + + // Write ColumnFileTiny + { + Block block = DMTestEnv::prepareSimpleWriteBlock(num_rows_write, num_rows_write + num_rows_write, false); + store->write(*db_context, db_context->getSettingsRef(), block); + store->flushCache(*db_context, RowKeyRange::newAll(false, 1), true); + } + + // write ColumnFileDeleteRange + { + HandleRange handle_range(0, num_rows_write / 2); + store->deleteRange(*db_context, db_context->getSettingsRef(), RowKeyRange::fromHandleRange(handle_range)); + store->flushCache(*db_context, RowKeyRange::newAll(false, 1), true); + } + + // write ColumnFileBig + { + Block block = DMTestEnv::prepareSimpleWriteBlock(num_rows_write + num_rows_write, num_rows_write + 2 * num_rows_write, false); + auto dm_context = store->newDMContext(*db_context, db_context->getSettingsRef()); + auto [range, file_ids] = genDMFile(*dm_context, block); + store->ingestFiles(dm_context, range, file_ids, false); + store->flushCache(*db_context, RowKeyRange::newAll(false, 1), true); + } + + dumpCheckpoint(); + + clearData(); + + verifyRows(RowKeyRange::newAll(store->isCommonHandle(), store->getRowKeyColumnSize()), 0); + + const auto manifest_key = S3::S3Filename::newCheckpointManifest(store_id, upload_sequence).toFullKey(); + auto checkpoint_info = std::make_shared(); + checkpoint_info->remote_store_id = store_id; + checkpoint_info->temp_ps_wrapper = createTempPageStorage(*db_context, manifest_key, /*dir_seq*/ 100); + checkpoint_info->temp_ps = checkpoint_info->temp_ps_wrapper->temp_ps; + store->ingestSegmentsFromCheckpointInfo(*db_context, db_context->getSettingsRef(), RowKeyRange::newAll(false, 1), checkpoint_info); + + verifyRows(RowKeyRange::newAll(store->isCommonHandle(), store->getRowKeyColumnSize()), num_rows_write / 2 + 2 * num_rows_write); + + reload(); + + verifyRows(RowKeyRange::newAll(store->isCommonHandle(), store->getRowKeyColumnSize()), num_rows_write / 2 + 2 * num_rows_write); +} +CATCH + +TEST_F(DeltaMergeStoreTestFastAddPeer, SimpleWriteReadAfterRestoreFromCheckPointWithSplit) +try +{ + auto & global_settings = TiFlashTestEnv::getGlobalContext().getSettingsRef(); + // store the old value to restore global_context settings after the test finish to avoid influence other tests + auto old_global_settings = global_settings; + + // change the settings to make it easier to trigger splitting segments + Settings settings; + settings.dt_segment_limit_rows = 11; + settings.dt_segment_limit_size = 20; + settings.dt_segment_delta_limit_rows = 7; + settings.dt_segment_delta_limit_size = 20; + settings.dt_segment_force_split_size = 100; + settings.dt_segment_delta_cache_limit_size = 20; + + // we need change the settings in both the ctx we get just below and the global_context above. + // because when processing write request, `DeltaMergeStore` will call `checkSegmentUpdate` with the context we just get below. + // and when initialize `DeltaMergeStore`, it will call `checkSegmentUpdate` with the global_context above. + // so we need to make the settings in these two contexts consistent. + global_settings = settings; + auto old_db_context = std::move(db_context); + db_context = DMTestEnv::getContext(settings); + SCOPE_EXIT({ + global_settings = old_global_settings; + db_context = std::move(old_db_context); + }); + { + auto table_column_defines = DMTestEnv::getDefaultColumns(); + + store = reload(table_column_defines); + } + + size_t num_rows_write = 0; + size_t num_rows_write_per_batch = 128; + // write until split and use a big enough finite for loop to make sure the test won't hang forever + for (size_t i = 0; i < 100000; i++) + { + // write to store + Block block = DMTestEnv::prepareSimpleWriteBlock(num_rows_write, num_rows_write + num_rows_write_per_batch, false); + store->write(*db_context, settings, block); + store->flushCache(*db_context, RowKeyRange::newAll(false, 1), true); + num_rows_write += num_rows_write_per_batch; + if (store->getSegmentsStats().size() > 1) + break; + } + { + ASSERT_GT(store->getSegmentsStats().size(), 1); + } + store->mergeDeltaAll(*db_context); + + dumpCheckpoint(); + + clearData(); + + verifyRows(RowKeyRange::newAll(store->isCommonHandle(), store->getRowKeyColumnSize()), 0); + + const auto manifest_key = S3::S3Filename::newCheckpointManifest(store_id, upload_sequence).toFullKey(); + auto checkpoint_info = std::make_shared(); + checkpoint_info->remote_store_id = store_id; + checkpoint_info->temp_ps_wrapper = createTempPageStorage(*db_context, manifest_key, /*dir_seq*/ 100); + checkpoint_info->temp_ps = checkpoint_info->temp_ps_wrapper->temp_ps; + store->ingestSegmentsFromCheckpointInfo(*db_context, db_context->getSettingsRef(), RowKeyRange::fromHandleRange(HandleRange(0, num_rows_write / 2)), checkpoint_info); + verifyRows(RowKeyRange::newAll(store->isCommonHandle(), store->getRowKeyColumnSize()), num_rows_write / 2); + + store->ingestSegmentsFromCheckpointInfo(*db_context, db_context->getSettingsRef(), RowKeyRange::fromHandleRange(HandleRange(num_rows_write / 2, num_rows_write)), checkpoint_info); + verifyRows(RowKeyRange::newAll(store->isCommonHandle(), store->getRowKeyColumnSize()), num_rows_write); +} +CATCH +} // namespace tests +} // namespace DM +} // namespace DB diff --git a/dbms/src/Storages/DeltaMerge/tests/gtest_dm_file.cpp b/dbms/src/Storages/DeltaMerge/tests/gtest_dm_file.cpp index 83aeeabef32..9e10a84b2c9 100644 --- a/dbms/src/Storages/DeltaMerge/tests/gtest_dm_file.cpp +++ b/dbms/src/Storages/DeltaMerge/tests/gtest_dm_file.cpp @@ -14,6 +14,7 @@ #include #include +#include #include #include #include @@ -34,9 +35,12 @@ #include #include #include - namespace DB { +namespace ErrorCodes +{ +extern const int CORRUPTED_DATA; +} namespace FailPoints { extern const char exception_before_dmfile_remove_encryption[]; @@ -382,6 +386,57 @@ try } CATCH +TEST_P(DMFileTest, MetaV2Broken) +try +{ + auto cols = DMTestEnv::getDefaultColumns(DMTestEnv::PkType::HiddenTiDBRowID, /*add_nullable*/ true); + + const size_t num_rows_write = 128; + + DMFileBlockOutputStream::BlockProperty block_property1; + block_property1.effective_num_rows = 1; + block_property1.gc_hint_version = 1; + block_property1.deleted_rows = 1; + DMFileBlockOutputStream::BlockProperty block_property2; + block_property2.effective_num_rows = 2; + block_property2.gc_hint_version = 2; + block_property2.deleted_rows = 2; + std::vector block_propertys; + block_propertys.push_back(block_property1); + block_propertys.push_back(block_property2); + DMFilePtr dmfile; + + Block block1 = DMTestEnv::prepareSimpleWriteBlockWithNullable(0, num_rows_write / 2); + Block block2 = DMTestEnv::prepareSimpleWriteBlockWithNullable(num_rows_write / 2, num_rows_write); + auto mode = DMFileMode::DirectoryChecksum; + auto configuration = createConfiguration(mode); + dmfile = DMFile::create(1, parent_path, std::move(configuration), DMFileFormat::V3); + auto stream = std::make_shared(dbContext(), dmfile, *cols); + stream->writePrefix(); + stream->write(block1, block_property1); + stream->write(block2, block_property2); + stream->writeSuffix(); + + { + PosixWritableFile file(dmfile->metav2Path(), false, -1, 0666); + String s = "hello"; + auto n = file.pwrite(s.data(), s.size(), 0); + ASSERT_EQ(n, s.size()); + } + + try + { + auto file_provider = dbContext().getFileProvider(); + auto t = DMFile::restore(file_provider, dmfile->fileId(), dmfile->pageId(), dmfile->parentPath(), DMFile::ReadMetaMode::all()); + FAIL(); // Should not come here. + } + catch (const DB::Exception & e) + { + ASSERT_EQ(e.code(), ErrorCodes::CORRUPTED_DATA) << e.message(); + } +} +CATCH + TEST_P(DMFileTest, GcFlag) try { diff --git a/dbms/src/Storages/DeltaMerge/workload/MainEntry.cpp b/dbms/src/Storages/DeltaMerge/workload/MainEntry.cpp index 2d6620c2ebf..16a827eed63 100644 --- a/dbms/src/Storages/DeltaMerge/workload/MainEntry.cpp +++ b/dbms/src/Storages/DeltaMerge/workload/MainEntry.cpp @@ -16,7 +16,7 @@ #include #include #include -#include +#include #include #include #include @@ -66,10 +66,22 @@ void initThreadPool() { size_t default_num_threads = std::max(4UL, 2 * std::thread::hardware_concurrency()); GlobalThreadPool::initialize( + /*max_threads*/ default_num_threads * 20, + /*max_free_threads*/ default_num_threads, + /*queue_size*/ default_num_threads * 10); + S3FileCachePool::initialize( /*max_threads*/ default_num_threads, /*max_free_threads*/ default_num_threads / 2, /*queue_size*/ default_num_threads * 2); - IOThreadPool::initialize( + DataStoreS3Pool::initialize( + /*max_threads*/ default_num_threads, + /*max_free_threads*/ default_num_threads / 2, + /*queue_size*/ default_num_threads * 2); + RNRemoteReadTaskPool::initialize( + /*max_threads*/ default_num_threads, + /*max_free_threads*/ default_num_threads / 2, + /*queue_size*/ default_num_threads * 2); + RNPagePreparerPool::initialize( /*max_threads*/ default_num_threads, /*max_free_threads*/ default_num_threads / 2, /*queue_size*/ default_num_threads * 2); diff --git a/dbms/src/Storages/Page/V3/CheckpointFile/Proto/common.proto b/dbms/src/Storages/Page/V3/CheckpointFile/Proto/common.proto index 9954bd95e91..7b8935906a1 100644 --- a/dbms/src/Storages/Page/V3/CheckpointFile/Proto/common.proto +++ b/dbms/src/Storages/Page/V3/CheckpointFile/Proto/common.proto @@ -27,4 +27,5 @@ message WriterInfo { message RemoteInfo { string type_name = 1; // e.g. "S3" / "LocalFS" string name = 2; // Remote-type specific name for description purpose. + string root = 3; // Identifier for the cluster. It is the `storage.s3.root` when using S3 } diff --git a/dbms/src/Storages/Page/V3/CheckpointFile/Proto/data_file.proto b/dbms/src/Storages/Page/V3/CheckpointFile/Proto/data_file.proto index d9740982316..37d29d0e0cd 100644 --- a/dbms/src/Storages/Page/V3/CheckpointFile/Proto/data_file.proto +++ b/dbms/src/Storages/Page/V3/CheckpointFile/Proto/data_file.proto @@ -54,6 +54,7 @@ message DataFileSuffix { } message EntryEditRecord { + // Note page_id are not valid utf-8 string. Don't use string type. bytes page_id = 1; uint64 version_sequence = 2; uint64 version_epoch = 3; diff --git a/dbms/src/Storages/Page/V3/CheckpointFile/Proto/manifest_file.proto b/dbms/src/Storages/Page/V3/CheckpointFile/Proto/manifest_file.proto index dd9cdd20b88..23e28e2e7bc 100644 --- a/dbms/src/Storages/Page/V3/CheckpointFile/Proto/manifest_file.proto +++ b/dbms/src/Storages/Page/V3/CheckpointFile/Proto/manifest_file.proto @@ -68,6 +68,7 @@ message EntryDataLocation { message EditRecord { EditType type = 1; + // Note page_id are not valid utf-8 string. Don't use string type. bytes page_id = 2; bytes ori_page_id = 3; uint64 version_sequence = 4; diff --git a/dbms/src/Storages/Page/V3/PageDirectory.cpp b/dbms/src/Storages/Page/V3/PageDirectory.cpp index de030559a59..d327f765ecc 100644 --- a/dbms/src/Storages/Page/V3/PageDirectory.cpp +++ b/dbms/src/Storages/Page/V3/PageDirectory.cpp @@ -1551,12 +1551,31 @@ typename PageDirectory::PageEntries PageDirectory::updateLocalCach for (const auto & r : edit.getRecords()) { - auto iter = mvcc_table_directory.lower_bound(r.page_id); - assert(iter != mvcc_table_directory.end()); - auto & version_list = iter->second; - if (!version_list->updateLocalCacheForRemotePage(PageVersion(seq, 0), r.entry)) + auto id_to_resolve = r.page_id; + auto sequence_to_resolve = seq; + while (true) { - ignored_entries.push_back(r.entry); + auto iter = mvcc_table_directory.lower_bound(id_to_resolve); + assert(iter != mvcc_table_directory.end()); + auto & version_list = iter->second; + auto [resolve_state, next_id_to_resolve, next_ver_to_resolve] = version_list->resolveToPageId(sequence_to_resolve, /*ignore_delete=*/id_to_resolve != r.page_id, nullptr); + if (resolve_state == ResolveResult::TO_NORMAL) + { + if (!version_list->updateLocalCacheForRemotePage(PageVersion(sequence_to_resolve, 0), r.entry)) + { + ignored_entries.push_back(r.entry); + } + break; + } + else if (resolve_state == ResolveResult::TO_REF) + { + id_to_resolve = next_id_to_resolve; + sequence_to_resolve = next_ver_to_resolve.sequence; + } + else + { + RUNTIME_CHECK(false); + } } } } diff --git a/dbms/src/Storages/Page/V3/PageDirectoryFactory.cpp b/dbms/src/Storages/Page/V3/PageDirectoryFactory.cpp index d8b953c656a..fb75d4a123c 100644 --- a/dbms/src/Storages/Page/V3/PageDirectoryFactory.cpp +++ b/dbms/src/Storages/Page/V3/PageDirectoryFactory.cpp @@ -215,8 +215,33 @@ void PageDirectoryFactory::applyRecord( version_list->createNewEntry(restored_version, r.entry); break; case EditRecordType::UPDATE_DATA_FROM_REMOTE: - version_list->updateLocalCacheForRemotePage(restored_version, r.entry); + { + auto id_to_resolve = r.page_id; + auto sequence_to_resolve = restored_version.sequence; + auto version_list_iter = iter; + while (true) + { + const auto & current_version_list = version_list_iter->second; + auto [resolve_state, next_id_to_resolve, next_ver_to_resolve] = current_version_list->resolveToPageId(sequence_to_resolve, /*ignore_delete=*/id_to_resolve != r.page_id, nullptr); + if (resolve_state == ResolveResult::TO_NORMAL) + { + current_version_list->updateLocalCacheForRemotePage(PageVersion(sequence_to_resolve, 0), r.entry); + break; + } + else if (resolve_state == ResolveResult::TO_REF) + { + id_to_resolve = next_id_to_resolve; + sequence_to_resolve = next_ver_to_resolve.sequence; + } + else + { + RUNTIME_CHECK(false); + } + version_list_iter = dir->mvcc_table_directory.lower_bound(id_to_resolve); + assert(version_list_iter != dir->mvcc_table_directory.end()); + } break; + } case EditRecordType::DEL: case EditRecordType::VAR_DELETE: // nothing different from `DEL` version_list->createDelete(restored_version); diff --git a/dbms/src/Storages/Page/V3/Universal/RaftDataReader.cpp b/dbms/src/Storages/Page/V3/Universal/RaftDataReader.cpp index 052c678700a..3596f79f91c 100644 --- a/dbms/src/Storages/Page/V3/Universal/RaftDataReader.cpp +++ b/dbms/src/Storages/Page/V3/Universal/RaftDataReader.cpp @@ -13,6 +13,7 @@ // limitations under the License. #include +#include namespace DB { @@ -40,6 +41,16 @@ void RaftDataReader::traverse(const UniversalPageId & start, const UniversalPage } } +void RaftDataReader::traverseRaftLogForRegion(UInt64 region_id, const std::function & acceptor) +{ + auto start = UniversalPageIdFormat::toFullRaftLogPrefix(region_id); + auto end = UniversalPageIdFormat::toFullRaftLogPrefix(region_id + 1); + traverse(start, end, [&](const UniversalPageId & page_id, const DB::Page & page) { + if (page_id.hasPrefix(start)) + acceptor(page_id, page); + }); +} + std::optional RaftDataReader::getLowerBound(const UniversalPageId & page_id) { auto snapshot = uni_ps.getSnapshot(fmt::format("lower_bound_r_{}", page_id)); diff --git a/dbms/src/Storages/Page/V3/Universal/RaftDataReader.h b/dbms/src/Storages/Page/V3/Universal/RaftDataReader.h index 08bb1a25186..610c9670202 100644 --- a/dbms/src/Storages/Page/V3/Universal/RaftDataReader.h +++ b/dbms/src/Storages/Page/V3/Universal/RaftDataReader.h @@ -31,6 +31,8 @@ class RaftDataReader final // if end is empty, it will be transformed to a key larger than all raft data key void traverse(const UniversalPageId & start, const UniversalPageId & end, const std::function & acceptor); + void traverseRaftLogForRegion(UInt64 region_id, const std::function & acceptor); + // return the first id not less than `page_id` std::optional getLowerBound(const UniversalPageId & page_id); diff --git a/dbms/src/Storages/Page/V3/Universal/S3LockLocalManager.cpp b/dbms/src/Storages/Page/V3/Universal/S3LockLocalManager.cpp index 71513c2c4e0..4b8c9156dbe 100644 --- a/dbms/src/Storages/Page/V3/Universal/S3LockLocalManager.cpp +++ b/dbms/src/Storages/Page/V3/Universal/S3LockLocalManager.cpp @@ -34,14 +34,9 @@ S3LockLocalManager::S3LockLocalManager() : store_id(InvalidStoreID) , log(Logger::get()) { - // store_id, s3lock_client are inited later because they may not - // accessable when S3LockLocalManager is created. - auto & ins = S3::ClientFactory::instance(); - s3_client = ins.sharedClient(); - bucket = ins.bucket(); } -// `store_id`, `s3lock_client` are inited later because they may not +// `store_id` is inited later because they may not // accessable when S3LockLocalManager is created. void S3LockLocalManager::initStoreInfo(StoreID actual_store_id, DB::S3::S3LockClientPtr s3lock_client_) { @@ -57,7 +52,8 @@ void S3LockLocalManager::initStoreInfo(StoreID actual_store_id, DB::S3::S3LockCl break; // we need to restore the last_upload_sequence from S3 - const auto manifests = S3::CheckpointManifestS3Set::getFromS3(*s3_client, bucket, actual_store_id); + auto s3_client = S3::ClientFactory::instance().sharedTiFlashClient(); + const auto manifests = S3::CheckpointManifestS3Set::getFromS3(*s3_client, actual_store_id); if (!manifests.empty()) { last_upload_sequence = manifests.latestUploadSequence(); @@ -174,7 +170,8 @@ String S3LockLocalManager::createS3Lock(const String & datafile_key, const S3::S // e.g. the CheckpointDataFile or DTFile generated by this store. // directly create lock through S3 client // TODO: handle s3 network error and retry? - S3::uploadEmptyFile(*s3_client, bucket, lockkey); + auto s3_client = S3::ClientFactory::instance().sharedTiFlashClient(); + S3::uploadEmptyFile(*s3_client, lockkey); LOG_DEBUG(log, "S3 lock created for local datafile, lockkey={}", lockkey); } else diff --git a/dbms/src/Storages/Page/V3/Universal/S3LockLocalManager.h b/dbms/src/Storages/Page/V3/Universal/S3LockLocalManager.h index badeadd1388..34d9cbd068d 100644 --- a/dbms/src/Storages/Page/V3/Universal/S3LockLocalManager.h +++ b/dbms/src/Storages/Page/V3/Universal/S3LockLocalManager.h @@ -62,9 +62,6 @@ class S3LockLocalManager String createS3Lock(const String & datafile_key, const S3::S3FilenameView & s3_file, UInt64 lock_store_id); private: - std::shared_ptr s3_client; - String bucket; - std::mutex mtx_store_init; std::condition_variable cv_init; StoreID store_id; diff --git a/dbms/src/Storages/Page/V3/Universal/S3PageReader.cpp b/dbms/src/Storages/Page/V3/Universal/S3PageReader.cpp index 5d56bcd97aa..20aa1493045 100644 --- a/dbms/src/Storages/Page/V3/Universal/S3PageReader.cpp +++ b/dbms/src/Storages/Page/V3/Universal/S3PageReader.cpp @@ -12,9 +12,11 @@ // See the License for the specific language governing permissions and // limitations under the License. -#include +#include #include #include +#include +#include #include namespace DB::PS::V3 @@ -24,20 +26,33 @@ Page S3PageReader::read(const UniversalPageIdAndEntry & page_id_and_entry) const auto & page_entry = page_id_and_entry.second; RUNTIME_CHECK(page_entry.checkpoint_info.has_value()); auto location = page_entry.checkpoint_info.data_location; - S3::S3RandomAccessFile file(s3_client, bucket, *location.data_file_id); - file.seek(location.offset_in_file, SEEK_SET); + const auto & remote_name = *location.data_file_id; + auto remote_name_view = S3::S3FilenameView::fromKey(remote_name); + RandomAccessFilePtr remote_file; + auto s3_client = S3::ClientFactory::instance().sharedTiFlashClient(); +#ifdef DBMS_PUBLIC_GTEST + if (remote_name_view.isLockFile()) + { +#endif + remote_file = std::make_shared(s3_client, remote_name_view.asDataFile().toFullKey()); +#ifdef DBMS_PUBLIC_GTEST + } + else + { + // Just used in unit test which want to just focus on read write logic + remote_file = std::make_shared(s3_client, *location.data_file_id); + } +#endif + ReadBufferFromRandomAccessFile buf(remote_file); + + buf.seek(location.offset_in_file, SEEK_SET); auto buf_size = location.size_in_file; char * data_buf = static_cast(alloc(buf_size)); MemHolder mem_holder = createMemHolder(data_buf, [&, buf_size](char * p) { free(p, buf_size); }); // TODO: support checksum verification - size_t pos = 0; - while (pos < buf_size) - { - auto n = file.read(data_buf + pos, buf_size - pos); - pos += n; - } + buf.readStrict(data_buf, buf_size); Page page{UniversalPageIdFormat::getU64ID(page_id_and_entry.first)}; page.data = std::string_view(data_buf, buf_size); page.mem_holder = mem_holder; diff --git a/dbms/src/Storages/Page/V3/Universal/S3PageReader.h b/dbms/src/Storages/Page/V3/Universal/S3PageReader.h index bbf92437c61..0507b569419 100644 --- a/dbms/src/Storages/Page/V3/Universal/S3PageReader.h +++ b/dbms/src/Storages/Page/V3/Universal/S3PageReader.h @@ -38,10 +38,7 @@ using UniversalPageIdAndEntries = std::vector; class S3PageReader : private Allocator { public: - explicit S3PageReader(std::shared_ptr s3_client_, const String & bucket_) - : s3_client(s3_client_) - , bucket(bucket_) - {} + S3PageReader() = default; Page read(const UniversalPageIdAndEntry & page_id_and_entry); @@ -51,10 +48,6 @@ class S3PageReader : private Allocator // return two page_maps, the first contains the whole page for given page id which is used to update local cache, // the second just contains read fields data. std::pair read(const FieldReadInfos & to_read); - -private: - std::shared_ptr s3_client; - String bucket; }; using S3PageReaderPtr = std::unique_ptr; diff --git a/dbms/src/Storages/Page/V3/Universal/UniversalPageIdFormatImpl.h b/dbms/src/Storages/Page/V3/Universal/UniversalPageIdFormatImpl.h index eb70a41dffc..64b8c262fda 100644 --- a/dbms/src/Storages/Page/V3/Universal/UniversalPageIdFormatImpl.h +++ b/dbms/src/Storages/Page/V3/Universal/UniversalPageIdFormatImpl.h @@ -90,7 +90,12 @@ struct UniversalPageIdFormat return buff.releaseStr(); } - // data is in kv engine, so it is prepend by KV_PREFIX + static UniversalPageId toKVStoreKey(UInt64 region_id) + { + return toFullPageId(toSubPrefix(StorageType::KVStore), region_id); + } + + // data is in kv engine, so it is prepended by KV_PREFIX // KV_PREFIX LOCAL_PREFIX REGION_RAFT_PREFIX region_id APPLY_STATE_SUFFIX static UniversalPageId toRaftApplyStateKeyInKVEngine(UInt64 region_id) { @@ -103,7 +108,7 @@ struct UniversalPageIdFormat return buff.releaseStr(); } - // data is in kv engine, so it is prepend by KV_PREFIX + // data is in kv engine, so it is prepended by KV_PREFIX // KV_PREFIX LOCAL_PREFIX REGION_META_PREFIX region_id REGION_STATE_SUFFIX static UniversalPageId toRegionLocalStateKeyInKVEngine(UInt64 region_id) { diff --git a/dbms/src/Storages/Page/V3/Universal/UniversalPageStorage.cpp b/dbms/src/Storages/Page/V3/Universal/UniversalPageStorage.cpp index 0de563d3b1c..d02d615e579 100644 --- a/dbms/src/Storages/Page/V3/Universal/UniversalPageStorage.cpp +++ b/dbms/src/Storages/Page/V3/Universal/UniversalPageStorage.cpp @@ -35,9 +35,7 @@ UniversalPageStoragePtr UniversalPageStorage::create( const String & name, PSDiskDelegatorPtr delegator, const PageStorageConfig & config, - const FileProviderPtr & file_provider, - std::shared_ptr s3_client, - const String & bucket) + const FileProviderPtr & file_provider) { UniversalPageStoragePtr storage = std::make_shared(name, delegator, config, file_provider); storage->blob_store = std::make_unique( @@ -45,9 +43,9 @@ UniversalPageStoragePtr UniversalPageStorage::create( file_provider, delegator, PS::V3::BlobConfig::from(config)); - if (s3_client != nullptr) + if (S3::ClientFactory::instance().isEnabled()) { - storage->remote_reader = std::make_unique(s3_client, bucket); + storage->remote_reader = std::make_unique(); storage->remote_locks_local_mgr = std::make_unique(); } return storage; diff --git a/dbms/src/Storages/Page/V3/Universal/UniversalPageStorage.h b/dbms/src/Storages/Page/V3/Universal/UniversalPageStorage.h index 565fb646d8e..328f7ec0de9 100644 --- a/dbms/src/Storages/Page/V3/Universal/UniversalPageStorage.h +++ b/dbms/src/Storages/Page/V3/Universal/UniversalPageStorage.h @@ -73,9 +73,7 @@ class UniversalPageStorage final const String & name, PSDiskDelegatorPtr delegator, const PageStorageConfig & config, - const FileProviderPtr & file_provider, - std::shared_ptr s3_client = nullptr, - const String & bucket = ""); + const FileProviderPtr & file_provider); UniversalPageStorage( String name, diff --git a/dbms/src/Storages/Page/V3/Universal/UniversalPageStorageService.cpp b/dbms/src/Storages/Page/V3/Universal/UniversalPageStorageService.cpp index 4c781983300..81e0b1f8fa3 100644 --- a/dbms/src/Storages/Page/V3/Universal/UniversalPageStorageService.cpp +++ b/dbms/src/Storages/Page/V3/Universal/UniversalPageStorageService.cpp @@ -47,20 +47,12 @@ UniversalPageStorageServicePtr UniversalPageStorageService::create( const PageStorageConfig & config) { auto service = UniversalPageStorageServicePtr(new UniversalPageStorageService(context)); - auto & s3factory = S3::ClientFactory::instance(); - std::shared_ptr s3_client; - String bucket; - if (s3factory.isEnabled()) - { - s3_client = s3factory.sharedClient(); - bucket = s3factory.bucket(); - } - service->uni_page_storage = UniversalPageStorage::create(name, delegator, config, context.getFileProvider(), s3_client, bucket); + service->uni_page_storage = UniversalPageStorage::create(name, delegator, config, context.getFileProvider()); service->uni_page_storage->restore(); // Starts the checkpoint upload timer // for disagg tiflash write node - if (s3factory.isEnabled() && !context.getSharedContextDisagg()->isDisaggregatedComputeMode()) + if (S3::ClientFactory::instance().isEnabled() && !context.getSharedContextDisagg()->isDisaggregatedComputeMode()) { // TODO: make this interval reloadable auto interval_s = context.getSettingsRef().remote_checkpoint_interval_seconds; @@ -89,29 +81,20 @@ UniversalPageStorageService::createForTest( Context & context, const String & name, PSDiskDelegatorPtr delegator, - const PageStorageConfig & config, - std::shared_ptr s3_client, - String bucket) + const PageStorageConfig & config) { auto service = UniversalPageStorageServicePtr(new UniversalPageStorageService(context)); - service->uni_page_storage = UniversalPageStorage::create(name, delegator, config, context.getFileProvider(), s3_client, bucket); + service->uni_page_storage = UniversalPageStorage::create(name, delegator, config, context.getFileProvider()); service->uni_page_storage->restore(); // not register background task under test return service; } -struct CheckpointUploadFunctor +bool CheckpointUploadFunctor::operator()(const PS::V3::LocalCheckpointFiles & checkpoint) const { - const StoreID store_id; - const UInt64 sequence; - const DM::Remote::IDataStorePtr remote_store; - - bool operator()(const PS::V3::LocalCheckpointFiles & checkpoint) const - { - // Persist checkpoint to remote_source - return remote_store->putCheckpointFiles(checkpoint, store_id, sequence); - } -}; + // Persist checkpoint to remote_source + return remote_store->putCheckpointFiles(checkpoint, store_id, sequence); +} bool UniversalPageStorageService::uploadCheckpoint() { @@ -143,7 +126,10 @@ bool UniversalPageStorageService::uploadCheckpoint() return uploadCheckpointImpl(store_info, s3lock_client, remote_store); } -bool UniversalPageStorageService::uploadCheckpointImpl(const metapb::Store & store_info, const S3::S3LockClientPtr & s3lock_client, const DM::Remote::IDataStorePtr & remote_store) +bool UniversalPageStorageService::uploadCheckpointImpl( + const metapb::Store & store_info, + const S3::S3LockClientPtr & s3lock_client, + const DM::Remote::IDataStorePtr & remote_store) { uni_page_storage->initLocksLocalManager(store_info.id(), s3lock_client); const auto upload_info = uni_page_storage->allocateNewUploadLocksInfo(); @@ -157,8 +143,11 @@ bool UniversalPageStorageService::uploadCheckpointImpl(const metapb::Store & sto auto * ri = wi.mutable_remote_info(); ri->set_type_name("S3"); // ri->set_name(); this field is not used currently + auto client = S3::ClientFactory::instance().sharedTiFlashClient(); + ri->set_root(client->root()); } + auto local_dir = Poco::Path(global_context.getTemporaryPath() + fmt::format("/checkpoint_upload_{}", upload_info.upload_sequence)).absolute(); Poco::File(local_dir).createDirectories(); auto local_dir_str = local_dir.toString() + "/"; @@ -200,6 +189,8 @@ bool UniversalPageStorageService::uploadCheckpointImpl(const metapb::Store & sto }; uni_page_storage->dumpIncrementalCheckpoint(opts); + LOG_DEBUG(log, "Upload checkpoint with upload sequence {} success", upload_info.upload_sequence); + // the checkpoint is uploaded to remote data store, remove local temp files Poco::File(local_dir).remove(true); diff --git a/dbms/src/Storages/Page/V3/Universal/UniversalPageStorageService.h b/dbms/src/Storages/Page/V3/Universal/UniversalPageStorageService.h index 5580b9b065c..f1e4d301a47 100644 --- a/dbms/src/Storages/Page/V3/Universal/UniversalPageStorageService.h +++ b/dbms/src/Storages/Page/V3/Universal/UniversalPageStorageService.h @@ -26,6 +26,14 @@ using IDataStorePtr = std::shared_ptr; namespace DB { +struct CheckpointUploadFunctor +{ + const StoreID store_id; + const UInt64 sequence; + const DM::Remote::IDataStorePtr remote_store; + + bool operator()(const PS::V3::LocalCheckpointFiles & checkpoint) const; +}; // This is wrapper class for UniversalPageStorage. // It mainly manages background tasks like gc for UniversalPageStorage. @@ -54,9 +62,7 @@ class UniversalPageStorageService final Context & context, const String & name, PSDiskDelegatorPtr delegator, - const PageStorageConfig & config, - std::shared_ptr s3_client, - String bucket); + const PageStorageConfig & config); private: explicit UniversalPageStorageService(Context & global_context_); diff --git a/dbms/src/Storages/Page/V3/Universal/UniversalWriteBatchImpl.h b/dbms/src/Storages/Page/V3/Universal/UniversalWriteBatchImpl.h index 1fc14f639bb..31949fe9348 100644 --- a/dbms/src/Storages/Page/V3/Universal/UniversalWriteBatchImpl.h +++ b/dbms/src/Storages/Page/V3/Universal/UniversalWriteBatchImpl.h @@ -58,6 +58,11 @@ class UniversalWriteBatch : private boost::noncopyable putPage(UniversalPageIdFormat::toFullPageId(prefix, page_id), tag, read_buffer, size, data_sizes); } + void putRemotePage(PageIdU64 page_id, UInt64 tag, const PS::V3::CheckpointLocation & data_location, PageFieldOffsetChecksums && offset_and_checksums) + { + putRemotePage(UniversalPageIdFormat::toFullPageId(prefix, page_id), tag, data_location, std::move(offset_and_checksums)); + } + void putExternal(PageIdU64 page_id, UInt64 tag) { putExternal(UniversalPageIdFormat::toFullPageId(prefix, page_id), tag); diff --git a/dbms/src/Storages/Page/V3/Universal/tests/gtest_checkpoint.cpp b/dbms/src/Storages/Page/V3/Universal/tests/gtest_checkpoint.cpp index ae3f931ecae..5d33614b377 100644 --- a/dbms/src/Storages/Page/V3/Universal/tests/gtest_checkpoint.cpp +++ b/dbms/src/Storages/Page/V3/Universal/tests/gtest_checkpoint.cpp @@ -752,17 +752,14 @@ class UniversalPageStorageServiceCheckpointTest : public DB::base::TiFlashStorag auto path = getTemporaryPath(); auto delegator = std::make_shared(path); auto & global_context = DB::tests::TiFlashTestEnv::getGlobalContext(); - s3_client = S3::ClientFactory::instance().sharedClient(); - bucket = S3::ClientFactory::instance().bucket(); uni_ps_service = UniversalPageStorageService::createForTest( global_context, "test.t", delegator, - PageStorageConfig{.blob_heavy_gc_valid_rate = 1.0}, - s3_client, - bucket); + PageStorageConfig{.blob_heavy_gc_valid_rate = 1.0}); log = Logger::get("UniversalPageStorageServiceCheckpointTest"); - ASSERT_TRUE(::DB::tests::TiFlashTestEnv::createBucketIfNotExist(*s3_client, bucket)); + s3_client = S3::ClientFactory::instance().sharedTiFlashClient(); + ASSERT_TRUE(::DB::tests::TiFlashTestEnv::createBucketIfNotExist(*s3_client)); } protected: @@ -793,8 +790,7 @@ class UniversalPageStorageServiceCheckpointTest : public DB::base::TiFlashStorag protected: UniversalPageStorageServicePtr uni_ps_service; - std::shared_ptr s3_client; - String bucket; + std::shared_ptr s3_client; UInt64 tag = 0; UInt64 store_id = 2; @@ -807,7 +803,7 @@ try auto page_storage = uni_ps_service->getUniversalPageStorage(); auto store_info = metapb::Store{}; store_info.set_id(store_id); - auto s3lock_client = std::make_shared(s3_client, bucket); + auto s3lock_client = std::make_shared(s3_client); auto remote_store = std::make_shared(::DB::tests::TiFlashTestEnv::getMockFileProvider()); // Mock normal writes { @@ -866,7 +862,6 @@ try ASSERT_EQ("The flower carriage rocked", readData(iter->entry.checkpoint_info.data_location)); } // check the first manifest -#if 0 // Mock normal writes && FAP ingest remote page { UniversalWriteBatch batch; @@ -878,8 +873,8 @@ try auto ingest_from_dtfile = S3::S3Filename::fromDMFileOID(S3::DMFileOID{.store_id = another_store_id, .table_id = 50, .file_id = 999}); { // create object on s3 for locking - S3::uploadEmptyFile(*s3_client, bucket, ingest_from_data_file.toFullKey()); - S3::uploadEmptyFile(*s3_client, bucket, ingest_from_dtfile.toFullKey()); + S3::uploadEmptyFile(*s3_client, ingest_from_data_file.toFullKey()); + S3::uploadEmptyFile(*s3_client, ingest_from_dtfile.toFullKey()); UniversalWriteBatch batch; PS::V3::CheckpointLocation loc21{ @@ -920,6 +915,20 @@ try ASSERT_EQ(EditRecordType::VAR_REF, iter->type); ASSERT_EQ("2", iter->page_id); + iter++; + ASSERT_EQ(EditRecordType::VAR_ENTRY, iter->type); + ASSERT_EQ("20", iter->page_id); + + iter++; + ASSERT_EQ(EditRecordType::VAR_ENTRY, iter->type); + ASSERT_EQ("21", iter->page_id); + ASSERT_EQ("lock/s99/dat_100_1.lock_s2_2", *iter->entry.checkpoint_info.data_location.data_file_id); // this is the lock key to CPDataFile + + iter++; + ASSERT_EQ(EditRecordType::VAR_EXTERNAL, iter->type); + ASSERT_EQ("22", iter->page_id); + ASSERT_EQ("lock/s99/t_50/dmf_999.lock_s2_2", *iter->entry.checkpoint_info.data_location.data_file_id); // this is the lock key to DMFile + iter++; ASSERT_EQ(EditRecordType::VAR_ENTRY, iter->type); ASSERT_EQ("3", iter->page_id); @@ -937,7 +946,6 @@ try ASSERT_EQ("lock/s2/dat_1_0.lock_s2_1", *iter->entry.checkpoint_info.data_location.data_file_id); // this is the lock key to CPDataFile ASSERT_EQ("The flower carriage rocked", readData(iter->entry.checkpoint_info.data_location)); } // check the first manifest -#endif } CATCH diff --git a/dbms/src/Storages/Page/V3/Universal/tests/gtest_lock_local_mgr.cpp b/dbms/src/Storages/Page/V3/Universal/tests/gtest_lock_local_mgr.cpp index 5584ac9b4a4..9baa7ce0b52 100644 --- a/dbms/src/Storages/Page/V3/Universal/tests/gtest_lock_local_mgr.cpp +++ b/dbms/src/Storages/Page/V3/Universal/tests/gtest_lock_local_mgr.cpp @@ -39,24 +39,22 @@ class S3LockLocalManagerTest : public testing::Test { public: S3LockLocalManagerTest() - : s3_client(S3::ClientFactory::instance().sharedClient()) - , bucket(S3::ClientFactory::instance().bucket()) + : s3_client(S3::ClientFactory::instance().sharedTiFlashClient()) , log(Logger::get()) {} void SetUp() override { - ::DB::tests::TiFlashTestEnv::createBucketIfNotExist(*s3_client, bucket); + ::DB::tests::TiFlashTestEnv::createBucketIfNotExist(*s3_client); } void TearDown() override { - ::DB::tests::TiFlashTestEnv::deleteBucket(*s3_client, bucket); + ::DB::tests::TiFlashTestEnv::deleteBucket(*s3_client); } protected: - std::shared_ptr s3_client; - String bucket; + std::shared_ptr s3_client; LoggerPtr log; }; @@ -65,7 +63,7 @@ try { StoreID this_store_id = 100; PS::V3::S3LockLocalManager mgr; - auto mock_s3lock_client = std::make_shared(s3_client, bucket); + auto mock_s3lock_client = std::make_shared(s3_client); mgr.initStoreInfo(this_store_id, mock_s3lock_client); auto info = mgr.allocateNewUploadLocksInfo(); @@ -81,7 +79,7 @@ try auto s3name_dtfile = S3::S3Filename::fromDMFileOID(S3::DMFileOID{.store_id = old_store_id, .table_id = 10, .file_id = 5}); auto s3name_datafile = S3::S3Filename::newCheckpointData(old_store_id, old_store_seq, 1); { - S3::uploadEmptyFile(*s3_client, bucket, s3name_dtfile.toFullKey()); + S3::uploadEmptyFile(*s3_client, s3name_dtfile.toFullKey()); PS::V3::CheckpointLocation loc{ .data_file_id = std::make_shared(s3name_dtfile.toFullKey()), .offset_in_file = 0, @@ -91,7 +89,7 @@ try } { auto key = std::make_shared(s3name_datafile.toFullKey()); - S3::uploadEmptyFile(*s3_client, bucket, *key); + S3::uploadEmptyFile(*s3_client, *key); PS::V3::CheckpointLocation loc2{ .data_file_id = key, .offset_in_file = 0, @@ -117,8 +115,8 @@ try ASSERT_GT(info.pre_lock_keys.count(expected_lockkey1), 0) << fmt::format("{}", lock_by_seq); const String expected_lockkey2 = s3name_dtfile.toView().getLockKey(this_store_id, info.upload_sequence); ASSERT_GT(info.pre_lock_keys.count(expected_lockkey2), 0) << fmt::format("{}", info.pre_lock_keys); - EXPECT_TRUE(S3::objectExists(*s3_client, bucket, expected_lockkey1)); - EXPECT_TRUE(S3::objectExists(*s3_client, bucket, expected_lockkey2)); + EXPECT_TRUE(S3::objectExists(*s3_client, expected_lockkey1)); + EXPECT_TRUE(S3::objectExists(*s3_client, expected_lockkey2)); // pre_lock_keys won't be cleaned after `allocateNewUploadLocksInfo` info = mgr.allocateNewUploadLocksInfo(); @@ -140,7 +138,7 @@ try { StoreID this_store_id = 100; PS::V3::S3LockLocalManager mgr; - auto mock_s3lock_client = std::make_shared(s3_client, bucket); + auto mock_s3lock_client = std::make_shared(s3_client); mgr.initStoreInfo(this_store_id, mock_s3lock_client); // Mock FAP ingest following pages from another store @@ -152,7 +150,7 @@ try auto s3name_dtfile = S3::S3Filename::fromDMFileOID(S3::DMFileOID{.store_id = old_store_id, .table_id = 10, .file_id = 5}); auto s3name_datafile = S3::S3Filename::newCheckpointData(old_store_id, old_store_seq, 1); { - S3::uploadEmptyFile(*s3_client, bucket, s3name_dtfile.toFullKey()); + S3::uploadEmptyFile(*s3_client, s3name_dtfile.toFullKey()); PS::V3::CheckpointLocation loc{ .data_file_id = std::make_shared(s3name_dtfile.toFullKey()), .offset_in_file = 0, @@ -162,7 +160,7 @@ try } { auto key = std::make_shared(s3name_datafile.toFullKey()); - S3::uploadEmptyFile(*s3_client, bucket, *key); + S3::uploadEmptyFile(*s3_client, *key); PS::V3::CheckpointLocation loc2{ .data_file_id = key, .offset_in_file = 0, diff --git a/dbms/src/Storages/Page/V3/Universal/tests/gtest_remote_read.cpp b/dbms/src/Storages/Page/V3/Universal/tests/gtest_remote_read.cpp index dca79d63027..6c35ea3ff8a 100644 --- a/dbms/src/Storages/Page/V3/Universal/tests/gtest_remote_read.cpp +++ b/dbms/src/Storages/Page/V3/Universal/tests/gtest_remote_read.cpp @@ -56,12 +56,11 @@ class UniPageStorageRemoteReadTest : public DB::base::TiFlashStorageTestBasic createIfNotExist(path); file_provider = DB::tests::TiFlashTestEnv::getDefaultFileProvider(); delegator = std::make_shared(path); - s3_client = S3::ClientFactory::instance().sharedClient(); - bucket = S3::ClientFactory::instance().bucket(); + s3_client = S3::ClientFactory::instance().sharedTiFlashClient(); - ASSERT_TRUE(::DB::tests::TiFlashTestEnv::createBucketIfNotExist(*s3_client, bucket)); + ASSERT_TRUE(::DB::tests::TiFlashTestEnv::createBucketIfNotExist(*s3_client)); - page_storage = UniversalPageStorage::create("write", delegator, config, file_provider, s3_client, bucket); + page_storage = UniversalPageStorage::create("write", delegator, config, file_provider); page_storage->restore(); } @@ -74,8 +73,10 @@ class UniPageStorageRemoteReadTest : public DB::base::TiFlashStorageTestBasic { auto path = getTemporaryPath(); delegator = std::make_shared(path); - auto storage = UniversalPageStorage::create("test.t", delegator, config_, file_provider, s3_client, bucket); + auto storage = UniversalPageStorage::create("test.t", delegator, config_, file_provider); storage->restore(); + auto mock_s3lock_client = std::make_shared(s3_client); + storage->initLocksLocalManager(100, mock_s3lock_client); return storage; } @@ -84,7 +85,7 @@ class UniPageStorageRemoteReadTest : public DB::base::TiFlashStorageTestBasic const String & full_path = dir + "/" + f_name; ReadBufferPtr src_buf = std::make_shared(full_path); S3::WriteSettings write_setting; - WritableFilePtr dst_file = std::make_shared(s3_client, bucket, f_name, write_setting); + WritableFilePtr dst_file = std::make_shared(s3_client, f_name, write_setting); WriteBufferPtr dst_buf = std::make_shared(dst_file); copyData(*src_buf, *dst_buf); dst_buf->next(); @@ -95,7 +96,7 @@ class UniPageStorageRemoteReadTest : public DB::base::TiFlashStorageTestBasic protected: void deleteBucket() { - ::DB::tests::TiFlashTestEnv::deleteBucket(*s3_client, bucket); + ::DB::tests::TiFlashTestEnv::deleteBucket(*s3_client); } protected: @@ -103,8 +104,7 @@ class UniPageStorageRemoteReadTest : public DB::base::TiFlashStorageTestBasic String remote_dir; FileProviderPtr file_provider; PSDiskDelegatorPtr delegator; - std::shared_ptr s3_client; - String bucket; + std::shared_ptr s3_client; PageStorageConfig config; std::shared_ptr page_storage; @@ -130,6 +130,7 @@ try { auto edits = PS::V3::universal::PageEntriesEdit{}; edits.appendRecord({.type = PS::V3::EditRecordType::VAR_ENTRY, .page_id = "aaabbb", .entry = {.size = 22, .offset = 10}}); + edits.appendRecord({.type = PS::V3::EditRecordType::VAR_REF, .page_id = "aaabbb2", .ori_page_id = "aaabbb"}); writer->writeEditsAndApplyCheckpointInfo(edits); } writer->writeSuffix(); @@ -146,12 +147,13 @@ try { auto edits_r = manifest_reader->readEdits(im); auto r = edits_r->getRecords(); - ASSERT_EQ(1, r.size()); + ASSERT_EQ(2, r.size()); UniversalWriteBatch wb; wb.disableRemoteLock(); wb.putPage(r[0].page_id, 0, "local data"); wb.putRemotePage(r[0].page_id, 0, r[0].entry.checkpoint_info.data_location, std::move(r[0].entry.field_offsets)); + wb.putRefPage(r[1].page_id, r[0].page_id); page_storage->write(std::move(wb)); } @@ -161,6 +163,12 @@ try ASSERT_EQ("nahida opened her eyes", String(page.data.begin(), page.data.size())); } + { + auto page = page_storage->read("aaabbb2"); + ASSERT_TRUE(page.isValid()); + ASSERT_EQ("nahida opened her eyes", String(page.data.begin(), page.data.size())); + } + // clear remote data and read again to make sure local cache exists deleteBucket(); @@ -169,6 +177,12 @@ try ASSERT_TRUE(page.isValid()); ASSERT_EQ("nahida opened her eyes", String(page.data.begin(), page.data.size())); } + + { + auto page = page_storage->read("aaabbb2"); + ASSERT_TRUE(page.isValid()); + ASSERT_EQ("nahida opened her eyes", String(page.data.begin(), page.data.size())); + } } CATCH @@ -224,13 +238,92 @@ try ASSERT_EQ("nahida opened her eyes", String(page.data.begin(), page.data.size())); } + reload(); + // clear remote data and read again to make sure local cache exists deleteBucket(); + { + auto page = page_storage->read("aaabbb"); + ASSERT_TRUE(page.isValid()); + ASSERT_EQ("nahida opened her eyes", String(page.data.begin(), page.data.size())); + } +} +CATCH + +TEST_F(UniPageStorageRemoteReadTest, WriteReadWithRef) +try +{ + auto writer = PS::V3::CPFilesWriter::create({ + .data_file_path = remote_dir + "/data_1", + .data_file_id = "data_1", + .manifest_file_path = remote_dir + "/manifest_foo", + .manifest_file_id = "manifest_foo", + .data_source = PS::V3::CPWriteDataSourceFixture::create({{10, "nahida opened her eyes"}}), + }); + + writer->writePrefix({ + .writer = {}, + .sequence = 5, + .last_sequence = 3, + }); + { + auto edits = PS::V3::universal::PageEntriesEdit{}; + edits.appendRecord({.type = PS::V3::EditRecordType::VAR_ENTRY, .page_id = "aaabbb", .entry = {.size = 22, .offset = 10}}); + writer->writeEditsAndApplyCheckpointInfo(edits); + } + writer->writeSuffix(); + writer.reset(); + uploadFile(remote_dir, "data_1"); + uploadFile(remote_dir, "manifest_foo"); + + auto manifest_file = PosixRandomAccessFile::create(remote_dir + "/manifest_foo"); + auto manifest_reader = PS::V3::CPManifestFileReader::create({ + .plain_file = manifest_file, + }); + manifest_reader->readPrefix(); + PS::V3::CheckpointProto::StringsInternMap im; + { + auto edits_r = manifest_reader->readEdits(im); + auto r = edits_r->getRecords(); + ASSERT_EQ(1, r.size()); + + UniversalWriteBatch wb; + wb.disableRemoteLock(); + wb.putPage(r[0].page_id, 0, "local data"); + wb.putRemotePage(r[0].page_id, 0, r[0].entry.checkpoint_info.data_location, std::move(r[0].entry.field_offsets)); + page_storage->write(std::move(wb)); + } + + { + UniversalWriteBatch wb; + wb.disableRemoteLock(); + wb.putRefPage("aaabbb2", "aaabbb"); + wb.delPage("aaabbb"); + page_storage->write(std::move(wb)); + } + + { + auto page = page_storage->read("aaabbb2"); + ASSERT_TRUE(page.isValid()); + ASSERT_EQ("nahida opened her eyes", String(page.data.begin(), page.data.size())); + } + + // clear remote data and read again to make sure local cache exists + deleteBucket(); + + { + auto page = page_storage->read("aaabbb2"); + ASSERT_TRUE(page.isValid()); + ASSERT_EQ("nahida opened her eyes", String(page.data.begin(), page.data.size())); + } + + // create an empty bucket because reload will try to read from S3 + ASSERT_TRUE(::DB::tests::TiFlashTestEnv::createBucketIfNotExist(*s3_client)); reload(); { - auto page = page_storage->read("aaabbb"); + auto page = page_storage->read("aaabbb2"); ASSERT_TRUE(page.isValid()); ASSERT_EQ("nahida opened her eyes", String(page.data.begin(), page.data.size())); } @@ -322,6 +415,7 @@ try auto blob_store_edits = blob_store.write(std::move(wb), nullptr); edits.appendRecord({.type = PS::V3::EditRecordType::VAR_ENTRY, .page_id = "page_foo", .entry = blob_store_edits.getRecords()[0].entry}); + edits.appendRecord({.type = PS::V3::EditRecordType::VAR_REF, .page_id = "page_foo2", .ori_page_id = "page_foo"}); } auto writer = PS::V3::CPFilesWriter::create({ @@ -351,19 +445,20 @@ try { auto edits_r = manifest_reader->readEdits(im); auto r = edits_r->getRecords(); - ASSERT_EQ(1, r.size()); + ASSERT_EQ(2, r.size()); UniversalWriteBatch wb; wb.disableRemoteLock(); wb.putRemotePage(r[0].page_id, 0, r[0].entry.checkpoint_info.data_location, std::move(r[0].entry.field_offsets)); + wb.putRefPage(r[1].page_id, r[0].page_id); page_storage->write(std::move(wb)); } - std::vector page_fields; - std::vector read_indices = {0, 2}; - UniversalPageStorage::PageReadFields read_fields = std::make_pair("page_foo", read_indices); - page_fields.emplace_back(read_fields); { + std::vector page_fields; + std::vector read_indices = {0, 2}; + UniversalPageStorage::PageReadFields read_fields = std::make_pair("page_foo", read_indices); + page_fields.emplace_back(read_fields); auto page_map = page_storage->read(page_fields); ASSERT_EQ(page_map.size(), 1); auto & page = page_map.at("page_foo"); @@ -375,13 +470,31 @@ try auto fields3_buf = page.getFieldData(2); ASSERT_EQ("riage rocked", String(fields3_buf.begin(), fields3_buf.size())); } + + { + std::vector page_fields; + std::vector read_indices = {0, 2}; + UniversalPageStorage::PageReadFields read_fields = std::make_pair("page_foo2", read_indices); + page_fields.emplace_back(read_fields); + auto page_map = page_storage->read(page_fields); + ASSERT_EQ(page_map.size(), 1); + auto & page = page_map.at("page_foo2"); + ASSERT_TRUE(page.isValid()); + ASSERT_EQ(page.field_offsets.size(), 2); + ASSERT_EQ(page.data.size(), 4 + 12); + auto fields0_buf = page.getFieldData(0); + ASSERT_EQ("The ", String(fields0_buf.begin(), fields0_buf.size())); + auto fields3_buf = page.getFieldData(2); + ASSERT_EQ("riage rocked", String(fields3_buf.begin(), fields3_buf.size())); + } } CATCH TEST_F(UniPageStorageRemoteReadTest, WriteReadExternal) try { - UniversalPageId page_id{"aaabbb"}; + UniversalPageId page_id1{"aaabbb"}; + UniversalPageId page_id2{"aaabbb2"}; { UniversalWriteBatch wb; wb.disableRemoteLock(); @@ -390,12 +503,19 @@ try .offset_in_file = 0, .size_in_file = 0, }; - wb.putRemoteExternal(page_id, data_location); + wb.putRemoteExternal(page_id1, data_location); + wb.putRefPage(page_id2, page_id1); page_storage->write(std::move(wb)); } { - auto location = page_storage->getCheckpointLocation(page_id); + auto location = page_storage->getCheckpointLocation(page_id1); + ASSERT_TRUE(location.has_value()); + ASSERT_EQ(*(location->data_file_id), "nahida opened her eyes"); + } + + { + auto location = page_storage->getCheckpointLocation(page_id2); ASSERT_TRUE(location.has_value()); ASSERT_EQ(*(location->data_file_id), "nahida opened her eyes"); } @@ -404,7 +524,13 @@ try reload(); { - auto location = page_storage->getCheckpointLocation(page_id); + auto location = page_storage->getCheckpointLocation(page_id1); + ASSERT_TRUE(location.has_value()); + ASSERT_EQ(*(location->data_file_id), "nahida opened her eyes"); + } + + { + auto location = page_storage->getCheckpointLocation(page_id2); ASSERT_TRUE(location.has_value()); ASSERT_EQ(*(location->data_file_id), "nahida opened her eyes"); } diff --git a/dbms/src/Storages/Page/WriteBatchWrapperImpl.h b/dbms/src/Storages/Page/WriteBatchWrapperImpl.h index 7f222d3a6c8..7fadc91bece 100644 --- a/dbms/src/Storages/Page/WriteBatchWrapperImpl.h +++ b/dbms/src/Storages/Page/WriteBatchWrapperImpl.h @@ -91,6 +91,14 @@ class WriteBatchWrapper : private boost::noncopyable putPage(page_id, tag, buffer_ptr, data.size()); } + void putRemotePage(PageIdU64 page_id, UInt64 tag, const PS::V3::CheckpointLocation & data_location, PageFieldOffsetChecksums && offset_and_checksums) + { + if (uwb) + uwb->putRemotePage(page_id, tag, data_location, std::move(offset_and_checksums)); + else + throw Exception(ErrorCodes::LOGICAL_ERROR, "try to put remote page with remote location with u64 id, page_id={}", page_id); + } + void putExternal(PageIdU64 page_id, UInt64 tag) { if (wb) diff --git a/dbms/src/Storages/PathCapacityMetrics.cpp b/dbms/src/Storages/PathCapacityMetrics.cpp index a263d90959b..f32f573b30f 100644 --- a/dbms/src/Storages/PathCapacityMetrics.cpp +++ b/dbms/src/Storages/PathCapacityMetrics.cpp @@ -210,7 +210,8 @@ FsStats PathCapacityMetrics::getFsStats(bool finalize_capacity) if (finalize_capacity && S3::ClientFactory::instance().isEnabled()) { // When S3 is enabled, use a large fake stat to avoid disk limitation by PD. - total_stat.capacity_size = 1024UL * 1024UL * 1024UL * 1024UL * 1024UL * 1024UL; // 1024PB + // EiB is not supported by TiUP now. https://github.com/pingcap/tiup/issues/2139 + total_stat.capacity_size = 1024UL * 1024UL * 1024UL * 1024UL * 1024UL; // 1PB total_stat.avail_size = total_stat.capacity_size - total_stat.used_size; } diff --git a/dbms/src/Storages/S3/CheckpointManifestS3Set.cpp b/dbms/src/Storages/S3/CheckpointManifestS3Set.cpp index 894afc2b354..7cd8ac85c43 100644 --- a/dbms/src/Storages/S3/CheckpointManifestS3Set.cpp +++ b/dbms/src/Storages/S3/CheckpointManifestS3Set.cpp @@ -22,28 +22,17 @@ namespace DB::S3 { CheckpointManifestS3Set CheckpointManifestS3Set::getFromS3(const S3::TiFlashS3Client & client, StoreID store_id) -{ - return CheckpointManifestS3Set::getFromS3(client, client.bucket(), store_id); -} - -CheckpointManifestS3Set -CheckpointManifestS3Set::getFromS3(const Aws::S3::S3Client & client, const String & bucket, StoreID store_id) { const auto manifest_prefix = S3::S3Filename::fromStoreId(store_id).toManifestPrefix(); std::vector manifests; - listPrefix(client, bucket, manifest_prefix, [&](const Aws::S3::Model::ListObjectsV2Result & result) { - const auto & objects = result.GetContents(); - manifests.reserve(manifests.size() + objects.size()); - for (const auto & object : objects) - { - const auto & mf_key = object.GetKey(); - // also store the object.GetLastModified() for removing - // outdated manifest objects - manifests.emplace_back(CheckpointManifestS3Object{mf_key, object.GetLastModified()}); - } - return DB::S3::PageResult{.num_keys = objects.size(), .more = true}; + S3::listPrefix(client, manifest_prefix, [&](const Aws::S3::Model::Object & object) { + const auto & mf_key = object.GetKey(); + // also store the object.GetLastModified() for removing + // outdated manifest objects + manifests.emplace_back(CheckpointManifestS3Object{mf_key, object.GetLastModified()}); + return DB::S3::PageResult{.num_keys = 1, .more = true}; }); return CheckpointManifestS3Set::create(manifests); } diff --git a/dbms/src/Storages/S3/CheckpointManifestS3Set.h b/dbms/src/Storages/S3/CheckpointManifestS3Set.h index d88e090e40e..21b1328dce9 100644 --- a/dbms/src/Storages/S3/CheckpointManifestS3Set.h +++ b/dbms/src/Storages/S3/CheckpointManifestS3Set.h @@ -38,8 +38,6 @@ class CheckpointManifestS3Set public: static CheckpointManifestS3Set getFromS3(const S3::TiFlashS3Client & client, StoreID store_id); - static CheckpointManifestS3Set getFromS3(const Aws::S3::S3Client & client, const String & bucket, StoreID store_id); - static CheckpointManifestS3Set create(const std::vector & manifest_keys); ALWAYS_INLINE bool empty() const { return manifests.empty(); } diff --git a/dbms/src/Storages/S3/FileCache.cpp b/dbms/src/Storages/S3/FileCache.cpp index 59eac2db4a3..b9e335aad8d 100644 --- a/dbms/src/Storages/S3/FileCache.cpp +++ b/dbms/src/Storages/S3/FileCache.cpp @@ -18,7 +18,7 @@ #include #include #include -#include +#include #include #include #include @@ -47,15 +47,15 @@ extern const int S3_ERROR; namespace DB { - using FileType = FileSegment::FileType; -FileCache::FileCache(const String & cache_dir_, UInt64 cache_capacity_, UInt64 cache_level_, UInt64 cache_min_age_seconds_) - : cache_dir(cache_dir_) - , cache_capacity(cache_capacity_) - , cache_level(cache_level_) +FileCache::FileCache(PathCapacityMetricsPtr capacity_metrics_, const StorageRemoteCacheConfig & config_) + : capacity_metrics(capacity_metrics_) + , cache_dir(config_.getDTFileCacheDir()) + , cache_capacity(config_.getDTFileCapacity()) + , cache_level(config_.dtfile_level) , cache_used(0) - , cache_min_age_seconds(cache_min_age_seconds_) + , cache_min_age_seconds(config_.dtfile_cache_min_age_seconds) , log(Logger::get("FileCache")) { prepareDir(cache_dir); @@ -128,12 +128,19 @@ void FileCache::removeDiskFile(const String & local_fname) { try { + auto fsize = std::filesystem::file_size(local_fname); for (std::filesystem::path p(local_fname); p.is_absolute() && std::filesystem::exists(p); p = p.parent_path()) { auto s = p.string(); if (s != cache_dir && (s == local_fname || std::filesystem::is_empty(p))) { std::filesystem::remove(p); // If p is a directory, remove success only when it is empty. + // Temporary files are not reported size to metrics until they are renamed. + // So we don't need to free its size here. + if (s == local_fname && !isTemporaryFilename(local_fname)) + { + capacity_metrics->freeUsedSize(local_fname, fsize); + } } else { @@ -297,7 +304,7 @@ bool FileCache::canCache(FileType file_type) const { return file_type != FileType::Unknow && static_cast(file_type) <= cache_level - && bg_downloading_count.load(std::memory_order_relaxed) < IOThreadPool::get().getMaxThreads(); + && bg_downloading_count.load(std::memory_order_relaxed) < S3FileCachePool::get().getMaxThreads(); } FileType FileCache::getFileTypeOfColData(const std::filesystem::path & p) @@ -371,11 +378,9 @@ bool FileCache::finalizeReservedSize(FileType reserve_for, UInt64 reserved_size, void FileCache::downloadImpl(const String & s3_key, FileSegmentPtr & file_seg) { Stopwatch sw; - auto client = S3::ClientFactory::instance().sharedClient(); - const auto & bucket = S3::ClientFactory::instance().bucket(); + auto client = S3::ClientFactory::instance().sharedTiFlashClient(); Aws::S3::Model::GetObjectRequest req; - req.SetBucket(bucket); - req.SetKey(s3_key); + client->setBucketAndKeyWithRoot(req, s3_key); ProfileEvents::increment(ProfileEvents::S3GetObject); auto outcome = client->GetObject(req); if (!outcome.IsSuccess()) @@ -384,7 +389,7 @@ void FileCache::downloadImpl(const String & s3_key, FileSegmentPtr & file_seg) } auto & result = outcome.GetResult(); auto content_length = result.GetContentLength(); - RUNTIME_CHECK_MSG(content_length > 0, "s3_key={}, content_length={}", s3_key, content_length); + RUNTIME_CHECK(content_length >= 0, s3_key, content_length); ProfileEvents::increment(ProfileEvents::S3ReadBytes, content_length); GET_METRIC(tiflash_storage_s3_request_seconds, type_get_object).Observe(sw.elapsedSeconds()); if (!finalizeReservedSize(file_seg->getFileType(), file_seg->getSize(), content_length)) @@ -400,12 +405,17 @@ void FileCache::downloadImpl(const String & s3_key, FileSegmentPtr & file_seg) { Aws::OFStream ostr(temp_fname, std::ios_base::out | std::ios_base::binary); RUNTIME_CHECK_MSG(ostr.is_open(), "Open {} failed: {}", temp_fname, strerror(errno)); - ostr << result.GetBody().rdbuf(); - RUNTIME_CHECK_MSG(ostr.good(), "Write {} failed: {}", temp_fname, strerror(errno)); - ostr.flush(); + if (content_length > 0) + { + ostr << result.GetBody().rdbuf(); + // If content_length == 0, ostr.good() is false. Does not know the reason. + RUNTIME_CHECK_MSG(ostr.good(), "Write {} content_length {} failed: {}", temp_fname, content_length, strerror(errno)); + ostr.flush(); + } } std::filesystem::rename(temp_fname, local_fname); auto fsize = std::filesystem::file_size(local_fname); + capacity_metrics->addUsedSize(local_fname, fsize); RUNTIME_CHECK_MSG(fsize == static_cast(content_length), "local_fname={}, file_size={}, content_length={}", local_fname, fsize, content_length); file_seg->setStatus(FileSegment::Status::Complete); LOG_DEBUG(log, "Download s3_key={} to local={} size={} cost={}ms", s3_key, local_fname, content_length, sw.elapsedMilliseconds()); @@ -440,7 +450,7 @@ void FileCache::bgDownload(const String & s3_key, FileSegmentPtr & file_seg) { bg_downloading_count.fetch_add(1, std::memory_order_relaxed); LOG_DEBUG(log, "downloading count {} => s3_key {} start", bg_downloading_count.load(std::memory_order_relaxed), s3_key); - IOThreadPool::get().scheduleOrThrowOnError( + S3FileCachePool::get().scheduleOrThrowOnError( [this, s3_key = s3_key, file_seg = file_seg]() mutable { download(s3_key, file_seg); }); @@ -587,4 +597,4 @@ void FileCache::updateConfig(Poco::Util::AbstractConfiguration & config_) } } -} // namespace DB \ No newline at end of file +} // namespace DB diff --git a/dbms/src/Storages/S3/FileCache.h b/dbms/src/Storages/S3/FileCache.h index 44780b62aa5..c5d48132fd9 100644 --- a/dbms/src/Storages/S3/FileCache.h +++ b/dbms/src/Storages/S3/FileCache.h @@ -18,6 +18,8 @@ #include #include #include +#include +#include #include #include @@ -189,9 +191,9 @@ class LRUFileTable class FileCache { public: - static void initialize(const String & cache_dir_, UInt64 cache_capacity_, UInt64 cache_level_, UInt64 cache_min_age_seconds_) + static void initialize(PathCapacityMetricsPtr capacity_metrics_, const StorageRemoteCacheConfig & config_) { - global_file_cache_instance = std::make_unique(cache_dir_, cache_capacity_, cache_level_, cache_min_age_seconds_); + global_file_cache_instance = std::make_unique(capacity_metrics_, config_); global_file_cache_initialized.store(true, std::memory_order_release); } @@ -207,7 +209,7 @@ class FileCache global_file_cache_instance = nullptr; } - FileCache(const String & cache_dir_, UInt64 cache_capacity_, UInt64 cache_level_, UInt64 cache_min_age_seconds_); + FileCache(PathCapacityMetricsPtr capacity_metrics_, const StorageRemoteCacheConfig & config_); RandomAccessFilePtr getRandomAccessFile(const S3::S3FilenameView & s3_fname); @@ -282,6 +284,7 @@ class FileCache std::vector getAll(); std::mutex mtx; + PathCapacityMetricsPtr capacity_metrics; String cache_dir; UInt64 cache_capacity; UInt64 cache_level; diff --git a/dbms/src/Storages/S3/MockS3Client.cpp b/dbms/src/Storages/S3/MockS3Client.cpp index 1326dffc2c6..d074a80ef39 100644 --- a/dbms/src/Storages/S3/MockS3Client.cpp +++ b/dbms/src/Storages/S3/MockS3Client.cpp @@ -24,6 +24,7 @@ #include #include #include +#include #include #include #include @@ -54,6 +55,13 @@ namespace DB::S3::tests { using namespace Aws::S3; +String MockS3Client::normalizedKey(String ori_key) +{ + if (ori_key.starts_with('/')) + return ori_key.substr(1, ori_key.size()); + return ori_key; +} + Model::GetObjectOutcome MockS3Client::GetObject(const Model::GetObjectRequest & request) const { std::lock_guard lock(mtx); @@ -63,7 +71,7 @@ Model::GetObjectOutcome MockS3Client::GetObject(const Model::GetObjectRequest & return Aws::S3::S3ErrorMapper::GetErrorForName("NoSuchBucket"); } const auto & bucket_storage = itr->second; - auto itr_obj = bucket_storage.find(request.GetKey()); + auto itr_obj = bucket_storage.find(normalizedKey(request.GetKey())); if (itr_obj == bucket_storage.end()) { return Aws::S3::S3ErrorMapper::GetErrorForName("NoSuchKey"); @@ -84,21 +92,35 @@ Model::PutObjectOutcome MockS3Client::PutObject(const Model::PutObjectRequest & return Aws::S3::S3ErrorMapper::GetErrorForName("NoSuchBucket"); } auto & bucket_storage = itr->second; - bucket_storage[request.GetKey()] = String{std::istreambuf_iterator(*request.GetBody()), {}}; + bucket_storage[normalizedKey(request.GetKey())] = String{std::istreambuf_iterator(*request.GetBody()), {}}; return Model::PutObjectResult{}; } Model::CopyObjectOutcome MockS3Client::CopyObject(const Model::CopyObjectRequest & request) const { std::lock_guard lock(mtx); - auto itr = storage.find(request.GetBucket()); - if (itr == storage.end()) + auto first_pos = request.GetCopySource().find_first_of('/'); + RUNTIME_CHECK(first_pos != String::npos, request.GetCopySource()); + auto src_bucket = request.GetCopySource().substr(0, first_pos); + auto src_key = request.GetCopySource().substr(first_pos + 1, request.GetCopySource().size()); + + auto src_itr = storage.find(src_bucket); + if (src_itr == storage.end()) + { + return Aws::S3::S3ErrorMapper::GetErrorForName("NoSuckBucket"); + } + + auto dst_itr = storage.find(request.GetBucket()); + if (dst_itr == storage.end()) { return Aws::S3::S3ErrorMapper::GetErrorForName("NoSuckBucket"); } - auto bucket_storage = itr->second; - bucket_storage[request.GetKey()] = bucket_storage[request.GetCopySource()]; - storage_tagging[request.GetBucket()][request.GetKey()] = request.GetTagging(); + + auto src_bucket_storage = src_itr->second; + auto dst_bucket_storage = dst_itr->second; + RUNTIME_CHECK(src_bucket_storage.contains(src_key), src_bucket, src_key); + dst_bucket_storage[normalizedKey(request.GetKey())] = src_bucket_storage[src_key]; + storage_tagging[request.GetBucket()][normalizedKey(request.GetKey())] = request.GetTagging(); return Model::CopyObjectResult{}; } @@ -110,9 +132,9 @@ Model::GetObjectTaggingOutcome MockS3Client::GetObjectTagging(const Model::GetOb { return Aws::S3::S3ErrorMapper::GetErrorForName("NoSuckBucket"); } - auto taggings = storage_tagging[request.GetBucket()][request.GetKey()]; + auto taggings = storage_tagging[request.GetBucket()][normalizedKey(request.GetKey())]; auto pos = taggings.find('='); - RUNTIME_CHECK(pos != String::npos, pos, taggings.size()); + RUNTIME_CHECK(pos != String::npos, taggings, pos, taggings.size()); Aws::S3::Model::Tag tag; tag.WithKey(taggings.substr(0, pos)) .WithValue(taggings.substr(pos + 1, taggings.size())); @@ -130,7 +152,7 @@ Model::DeleteObjectOutcome MockS3Client::DeleteObject(const Model::DeleteObjectR return Aws::S3::S3ErrorMapper::GetErrorForName("NoSuchBucket"); } auto & bucket_storage = itr->second; - bucket_storage.erase(request.GetKey()); + bucket_storage.erase(normalizedKey(request.GetKey())); return Model::DeleteObjectResult{}; } @@ -144,18 +166,43 @@ Model::ListObjectsV2Outcome MockS3Client::ListObjectsV2(const Model::ListObjects } const auto & bucket_storage = itr->second; Model::ListObjectsV2Result result; - for (auto itr_obj = bucket_storage.lower_bound(request.GetPrefix()); itr_obj != bucket_storage.end(); ++itr_obj) + RUNTIME_CHECK(!request.DelimiterHasBeenSet() || request.GetDelimiter() == "/", request.GetDelimiter()); + + auto normalized_prefix = normalizedKey(request.GetPrefix()); + if (!request.DelimiterHasBeenSet()) + { + for (auto itr_obj = bucket_storage.lower_bound(normalized_prefix); itr_obj != bucket_storage.end(); ++itr_obj) + { + if (startsWith(itr_obj->first, normalized_prefix)) + { + Model::Object obj; + obj.SetKey(itr_obj->first); + obj.SetSize(itr_obj->second.size()); + result.AddContents(std::move(obj)); + } + else + { + break; + } + } + } + else { - if (startsWith(itr_obj->first, request.GetPrefix())) + std::set common_prefix; + const auto & delimiter = request.GetDelimiter(); + for (auto itr_obj = bucket_storage.lower_bound(normalized_prefix); itr_obj != bucket_storage.end(); ++itr_obj) { - Model::Object obj; - obj.SetKey(itr_obj->first); - obj.SetSize(itr_obj->second.size()); - result.AddContents(std::move(obj)); + if (!startsWith(itr_obj->first, normalized_prefix)) + break; + const auto & key = itr_obj->first; + auto pos = key.find(delimiter, normalized_prefix.size()); + if (pos == String::npos) + continue; + common_prefix.insert(key.substr(0, pos + delimiter.size())); } - else + for (const auto & p : common_prefix) { - break; + result.AddCommonPrefixes(Model::CommonPrefix().WithPrefix(p)); } } return result; @@ -170,7 +217,7 @@ Model::HeadObjectOutcome MockS3Client::HeadObject(const Model::HeadObjectRequest return Aws::S3::S3ErrorMapper::GetErrorForName("NoSuchBucket"); } const auto & bucket_storage = itr->second; - auto itr_obj = bucket_storage.find(request.GetKey()); + auto itr_obj = bucket_storage.find(normalizedKey(request.GetKey())); if (itr_obj != bucket_storage.end()) { auto r = Model::HeadObjectResult{}; @@ -178,7 +225,7 @@ Model::HeadObjectOutcome MockS3Client::HeadObject(const Model::HeadObjectRequest if (auto v = FailPointHelper::getFailPointVal(FailPoints::force_set_mocked_s3_object_mtime); v) { auto m = std::any_cast>(v.value()); - if (auto iter_m = m.find(request.GetKey()); iter_m != m.end()) + if (auto iter_m = m.find(normalizedKey(request.GetKey())); iter_m != m.end()) { r.SetLastModified(iter_m->second); } @@ -223,7 +270,7 @@ Model::CompleteMultipartUploadOutcome MockS3Client::CompleteMultipartUpload(cons return Aws::S3::S3ErrorMapper::GetErrorForName("NoSuchBucket"); } auto & bucket_storage = itr->second; - bucket_storage[request.GetKey()] = s; + bucket_storage[normalizedKey(request.GetKey())] = s; return Model::CompleteMultipartUploadResult{}; } diff --git a/dbms/src/Storages/S3/MockS3Client.h b/dbms/src/Storages/S3/MockS3Client.h index 069c11c1712..d967d1c6491 100644 --- a/dbms/src/Storages/S3/MockS3Client.h +++ b/dbms/src/Storages/S3/MockS3Client.h @@ -25,8 +25,8 @@ using namespace Aws::S3; class MockS3Client final : public S3::TiFlashS3Client { public: - explicit MockS3Client(const String & bucket = "") - : TiFlashS3Client(bucket) + explicit MockS3Client(const String & bucket, const String & root) + : TiFlashS3Client(bucket, root) {} ~MockS3Client() override = default; @@ -45,6 +45,8 @@ class MockS3Client final : public S3::TiFlashS3Client Model::GetObjectTaggingOutcome GetObjectTagging(const Model::GetObjectTaggingRequest & request) const override; private: + static String normalizedKey(String ori_key); + // Object key -> Object data using BucketStorage = std::map; // Object key -> Object tagging diff --git a/dbms/src/Storages/S3/S3Common.cpp b/dbms/src/Storages/S3/S3Common.cpp index e63f0cc6150..69897e7c438 100644 --- a/dbms/src/Storages/S3/S3Common.cpp +++ b/dbms/src/Storages/S3/S3Common.cpp @@ -12,8 +12,10 @@ // See the License for the specific language governing permissions and // limitations under the License. +#include #include #include +#include #include #include #include @@ -32,11 +34,13 @@ #include #include #include +#include #include #include #include #include #include +#include #include #include @@ -70,7 +74,8 @@ Poco::Message::Priority convertLogLevel(Aws::Utils::Logging::LogLevel log_level) case Aws::Utils::Logging::LogLevel::Warn: return Poco::Message::PRIO_WARNING; case Aws::Utils::Logging::LogLevel::Info: - return Poco::Message::PRIO_INFORMATION; + // treat aws info logging as debug level + return Poco::Message::PRIO_DEBUG; case Aws::Utils::Logging::LogLevel::Debug: return Poco::Message::PRIO_DEBUG; case Aws::Utils::Logging::LogLevel::Trace: @@ -122,24 +127,49 @@ class AWSLogger final : public Aws::Utils::Logging::LogSystemInterface namespace DB::S3 { -TiFlashS3Client::TiFlashS3Client(const String & bucket_name_) +// ensure the `key_root` format like "user0/". No '/' at the beginning and '/' at the end +String normalizedRoot(String ori_root) // a copy for changing +{ + if (startsWith(ori_root, "/") && ori_root.size() != 1) + { + ori_root = ori_root.substr(1, ori_root.size()); + } + if (!endsWith(ori_root, "/")) + { + ori_root += "/"; + } + return ori_root; +} + +TiFlashS3Client::TiFlashS3Client(const String & bucket_name_, const String & root_) : bucket_name(bucket_name_) -{} + , key_root(normalizedRoot(root_)) + , log(Logger::get(fmt::format("bucket={} root={}", bucket_name, key_root))) +{ +} TiFlashS3Client::TiFlashS3Client( const String & bucket_name_, + const String & root_, const Aws::Auth::AWSCredentials & credentials, const Aws::Client::ClientConfiguration & clientConfiguration, Aws::Client::AWSAuthV4Signer::PayloadSigningPolicy signPayloads, bool useVirtualAddressing) : Aws::S3::S3Client(credentials, clientConfiguration, signPayloads, useVirtualAddressing) , bucket_name(bucket_name_) + , key_root(normalizedRoot(root_)) + , log(Logger::get(fmt::format("bucket={} root={}", bucket_name, key_root))) { } -TiFlashS3Client::TiFlashS3Client(const String & bucket_name_, std::unique_ptr && raw_client) +TiFlashS3Client::TiFlashS3Client( + const String & bucket_name_, + const String & root_, + std::unique_ptr && raw_client) : Aws::S3::S3Client(std::move(*raw_client)) , bucket_name(bucket_name_) + , key_root(normalizedRoot(root_)) + , log(Logger::get(fmt::format("bucket={} root={}", bucket_name, key_root))) { } @@ -151,24 +181,23 @@ bool ClientFactory::isEnabled() const void ClientFactory::init(const StorageS3Config & config_, bool mock_s3_) { config = config_; + config.root = normalizedRoot(config.root); Aws::InitAPI(aws_options); Aws::Utils::Logging::InitializeAWSLogging(std::make_shared()); if (!mock_s3_) { - shared_client = create(); - shared_tiflash_client = std::make_shared(config.bucket, create()); + shared_tiflash_client = std::make_shared(config.bucket, config.root, create()); } else { - shared_tiflash_client = std::make_unique(config.bucket); - shared_client = shared_tiflash_client; // share the same object + shared_tiflash_client = std::make_unique(config.bucket, config.root); } } void ClientFactory::shutdown() { + // Reset S3Client before Aws::ShutdownAPI. shared_tiflash_client.reset(); - shared_client.reset(); // Reset S3Client before Aws::ShutdownAPI. Aws::Utils::Logging::ShutdownAWSLogging(); Aws::ShutdownAPI(aws_options); } @@ -186,22 +215,11 @@ std::unique_ptr ClientFactory::create() const return create(config); } -const String & ClientFactory::bucket() const -{ - // `bucket` is read-only. - return config.bucket; -} - -std::shared_ptr ClientFactory::sharedClient() const -{ - // `shared_client` is created during initialization and destroyed when process exits - // which means it is read-only when processing requests. So, it is safe to read `shared_client` - // without acquiring lock. - return shared_client; -} - std::shared_ptr ClientFactory::sharedTiFlashClient() const { + // `shared_tiflash_client` is created during initialization and destroyed + // when process exits which means it is read-only when processing requests. + // So, it is safe to read `shared_tiflash_client` without acquiring lock. return shared_tiflash_client; } @@ -247,44 +265,19 @@ bool isNotFoundError(Aws::S3::S3Errors error) return error == Aws::S3::S3Errors::RESOURCE_NOT_FOUND || error == Aws::S3::S3Errors::NO_SUCH_KEY; } -Aws::S3::Model::HeadObjectOutcome headObject(const Aws::S3::S3Client & client, const String & bucket, const String & key, const String & version_id) +Aws::S3::Model::HeadObjectOutcome headObject(const TiFlashS3Client & client, const String & key) { ProfileEvents::increment(ProfileEvents::S3HeadObject); Stopwatch sw; SCOPE_EXIT({ GET_METRIC(tiflash_storage_s3_request_seconds, type_head_object).Observe(sw.elapsedSeconds()); }); Aws::S3::Model::HeadObjectRequest req; - req.WithBucket(bucket).WithKey(key); - if (!version_id.empty()) - { - req.SetVersionId(version_id); - } + client.setBucketAndKeyWithRoot(req, key); return client.HeadObject(req); } -S3::ObjectInfo getObjectInfo(const Aws::S3::S3Client & client, const String & bucket, const String & key, const String & version_id, bool throw_on_error) +bool objectExists(const TiFlashS3Client & client, const String & key) { - auto outcome = headObject(client, bucket, key, version_id); - - if (outcome.IsSuccess()) - { - auto read_result = outcome.GetResultWithOwnership(); - return {.size = static_cast(read_result.GetContentLength()), .last_modification_time = read_result.GetLastModified().Millis() / 1000}; - } - else if (throw_on_error) - { - throw fromS3Error(outcome.GetError(), "Failed to HEAD object, key={}", key); - } - return {}; -} - -size_t getObjectSize(const Aws::S3::S3Client & client, const String & bucket, const String & key, const String & version_id, bool throw_on_error) -{ - return getObjectInfo(client, bucket, key, version_id, throw_on_error).size; -} - -bool objectExists(const Aws::S3::S3Client & client, const String & bucket, const String & key, const String & version_id) -{ - auto outcome = headObject(client, bucket, key, version_id); + auto outcome = headObject(client, key); if (outcome.IsSuccess()) { return true; @@ -294,14 +287,14 @@ bool objectExists(const Aws::S3::S3Client & client, const String & bucket, const { return false; } - throw fromS3Error(outcome.GetError(), "Failed to check existence of object, bucket={} key={}", bucket, key); + throw fromS3Error(outcome.GetError(), "S3 HeadObject failed, bucket={} root={} key={}", client.bucket(), client.root(), key); } -void uploadEmptyFile(const Aws::S3::S3Client & client, const String & bucket, const String & key, const String & tagging) +void uploadEmptyFile(const TiFlashS3Client & client, const String & key, const String & tagging) { Stopwatch sw; Aws::S3::Model::PutObjectRequest req; - req.WithBucket(bucket).WithKey(key); + client.setBucketAndKeyWithRoot(req, key); if (!tagging.empty()) req.SetTagging(tagging); req.SetContentType("binary/octet-stream"); @@ -311,19 +304,18 @@ void uploadEmptyFile(const Aws::S3::S3Client & client, const String & bucket, co auto result = client.PutObject(req); if (!result.IsSuccess()) { - throw fromS3Error(result.GetError(), "S3 PutEmptyObject failed, bucket={} key={}", bucket, key); + throw fromS3Error(result.GetError(), "S3 PutEmptyObject failed, bucket={} root={} key={}", client.bucket(), client.root(), key); } auto elapsed_seconds = sw.elapsedSeconds(); GET_METRIC(tiflash_storage_s3_request_seconds, type_put_object).Observe(elapsed_seconds); - static auto log = Logger::get(); - LOG_DEBUG(log, "uploadEmptyFile remote_fname={}, cost={:.2f}s", key, elapsed_seconds); + LOG_DEBUG(client.log, "uploadEmptyFile key={}, cost={:.2f}s", key, elapsed_seconds); } -void uploadFile(const Aws::S3::S3Client & client, const String & bucket, const String & local_fname, const String & remote_fname) +void uploadFile(const TiFlashS3Client & client, const String & local_fname, const String & remote_fname) { Stopwatch sw; Aws::S3::Model::PutObjectRequest req; - req.WithBucket(bucket).WithKey(remote_fname); + client.setBucketAndKeyWithRoot(req, remote_fname); req.SetContentType("binary/octet-stream"); auto istr = Aws::MakeShared("PutObjectInputStream", local_fname, std::ios_base::in | std::ios_base::binary); RUNTIME_CHECK_MSG(istr->is_open(), "Open {} fail: {}", local_fname, strerror(errno)); @@ -333,26 +325,24 @@ void uploadFile(const Aws::S3::S3Client & client, const String & bucket, const S auto result = client.PutObject(req); if (!result.IsSuccess()) { - throw fromS3Error(result.GetError(), "S3 PutObject failed, local_fname={} bucket={} key={}", local_fname, bucket, remote_fname); + throw fromS3Error(result.GetError(), "S3 PutObject failed, local_fname={} bucket={} root={} key={}", local_fname, client.bucket(), client.root(), remote_fname); } ProfileEvents::increment(ProfileEvents::S3WriteBytes, write_bytes); auto elapsed_seconds = sw.elapsedSeconds(); GET_METRIC(tiflash_storage_s3_request_seconds, type_put_object).Observe(elapsed_seconds); - static auto log = Logger::get(); - LOG_DEBUG(log, "uploadFile local_fname={}, remote_fname={}, write_bytes={} cost={:.2f}s", local_fname, remote_fname, write_bytes, elapsed_seconds); + LOG_DEBUG(client.log, "uploadFile local_fname={}, key={}, write_bytes={} cost={:.2f}s", local_fname, remote_fname, write_bytes, elapsed_seconds); } -void downloadFile(const Aws::S3::S3Client & client, const String & bucket, const String & local_fname, const String & remote_fname) +void downloadFile(const TiFlashS3Client & client, const String & local_fname, const String & remote_fname) { Stopwatch sw; Aws::S3::Model::GetObjectRequest req; - req.SetBucket(bucket); - req.SetKey(remote_fname); + client.setBucketAndKeyWithRoot(req, remote_fname); ProfileEvents::increment(ProfileEvents::S3GetObject); auto outcome = client.GetObject(req); if (!outcome.IsSuccess()) { - throw fromS3Error(outcome.GetError(), "remote_fname={}", remote_fname); + throw fromS3Error(outcome.GetError(), "S3 GetObject failed, bucket={} root={} key={}", client.bucket(), client.root(), remote_fname); } ProfileEvents::increment(ProfileEvents::S3ReadBytes, outcome.GetResult().GetContentLength()); GET_METRIC(tiflash_storage_s3_request_seconds, type_get_object).Observe(sw.elapsedSeconds()); @@ -362,72 +352,103 @@ void downloadFile(const Aws::S3::S3Client & client, const String & bucket, const RUNTIME_CHECK_MSG(ostr.good(), "Write {} fail: {}", local_fname, strerror(errno)); } -void rewriteObjectWithTagging(const Aws::S3::S3Client & client, const String & bucket, const String & key, const String & tagging) +void rewriteObjectWithTagging(const TiFlashS3Client & client, const String & key, const String & tagging) { Stopwatch sw; Aws::S3::Model::CopyObjectRequest req; // rewrite the object with `key`, adding tagging to the new object - req.WithBucket(bucket).WithCopySource(key).WithKey(key).WithTagging(tagging); + // The copy_source format is "${source_bucket}/${source_key}" + auto copy_source = client.bucket() + "/" + (client.root() == "/" ? "" : client.root()) + key; + client.setBucketAndKeyWithRoot(req, key); + req.WithCopySource(copy_source) // + .WithTagging(tagging) + .WithTaggingDirective(Aws::S3::Model::TaggingDirective::REPLACE); ProfileEvents::increment(ProfileEvents::S3CopyObject); auto outcome = client.CopyObject(req); if (!outcome.IsSuccess()) { - throw fromS3Error(outcome.GetError(), "key={}", key); + throw fromS3Error(outcome.GetError(), "S3 CopyObject failed, bucket={} root={} key={}", client.bucket(), client.root(), key); } auto elapsed_seconds = sw.elapsedSeconds(); GET_METRIC(tiflash_storage_s3_request_seconds, type_copy_object).Observe(elapsed_seconds); - static auto log = Logger::get(); - LOG_DEBUG(log, "rewrite object key={} cost={:.2f}s", key, elapsed_seconds); + LOG_DEBUG(client.log, "rewrite object key={} cost={:.2f}s", key, elapsed_seconds); } -void ensureLifecycleRuleExist(const Aws::S3::S3Client & client, const String & bucket, Int32 expire_days) +void ensureLifecycleRuleExist(const TiFlashS3Client & client, Int32 expire_days) { bool lifecycle_rule_has_been_set = false; Aws::Vector old_rules; { Aws::S3::Model::GetBucketLifecycleConfigurationRequest req; - req.SetBucket(bucket); + req.SetBucket(client.bucket()); auto outcome = client.GetBucketLifecycleConfiguration(req); if (!outcome.IsSuccess()) { throw fromS3Error(outcome.GetError(), "GetBucketLifecycle fail"); } - auto res = outcome.GetResult(); + auto res = outcome.GetResultWithOwnership(); old_rules = res.GetRules(); static_assert(TaggingObjectIsDeleted == "tiflash_deleted=true"); for (const auto & rule : old_rules) { const auto & filt = rule.GetFilter(); - const auto & tag = filt.GetTag(); + if (!filt.AndHasBeenSet()) + { + continue; + } + const auto & and_op = filt.GetAnd(); + const auto & tags = and_op.GetTags(); + if (tags.size() != 1 || !and_op.PrefixHasBeenSet() || !and_op.GetPrefix().empty()) + { + continue; + } + + const auto & tag = tags[0]; if (rule.GetStatus() == Aws::S3::Model::ExpirationStatus::Enabled && tag.GetKey() == "tiflash_deleted" && tag.GetValue() == "true") { lifecycle_rule_has_been_set = true; + break; } } } - static LoggerPtr log = Logger::get(); if (lifecycle_rule_has_been_set) { - LOG_INFO(log, "The lifecycle rule has been set, n_rules={} tag={}", old_rules.size(), TaggingObjectIsDeleted); + LOG_INFO(client.log, "The lifecycle rule has been set, n_rules={} filter={}", old_rules.size(), TaggingObjectIsDeleted); + return; + } + else + { + UNUSED(expire_days); + LOG_WARNING(client.log, "The lifecycle rule with filter \"{}\" has not been set, please check the bucket lifecycle configuration", TaggingObjectIsDeleted); return; } +#if 0 + // Adding rule by AWS SDK is failed, don't know why + // Reference: https://docs.aws.amazon.com/AmazonS3/latest/userguide/S3OutpostsLifecycleCLIJava.html + LOG_INFO(client.log, "The lifecycle rule with filter \"{}\" has not been added, n_rules={}", TaggingObjectIsDeleted, old_rules.size()); static_assert(TaggingObjectIsDeleted == "tiflash_deleted=true"); + std::vector filter_tags{Aws::S3::Model::Tag().WithKey("tiflash_deleted").WithValue("true")}; Aws::S3::Model::LifecycleRuleFilter filter; - filter.SetTag(Aws::S3::Model::Tag().WithKey("tiflash_deleted").WithValue("true")); + filter.WithAnd(Aws::S3::Model::LifecycleRuleAndOperator() + .WithPrefix("") + .WithTags(filter_tags)); Aws::S3::Model::LifecycleRule rule; rule.WithStatus(Aws::S3::Model::ExpirationStatus::Enabled) .WithFilter(filter) - .WithExpiration(Aws::S3::Model::LifecycleExpiration().WithDays(expire_days)); + .WithExpiration(Aws::S3::Model::LifecycleExpiration() + .WithExpiredObjectDeleteMarker(false) + .WithDays(expire_days)) + .WithID("tiflashgc"); + old_rules.emplace_back(rule); // existing rules + new rule Aws::S3::Model::BucketLifecycleConfiguration lifecycle_config; lifecycle_config - .WithRules(old_rules) // existing rules - .AddRules(rule); + .WithRules(old_rules); Aws::S3::Model::PutBucketLifecycleConfigurationRequest request; request.WithBucket(bucket) @@ -438,37 +459,90 @@ void ensureLifecycleRuleExist(const Aws::S3::S3Client & client, const String & b { throw fromS3Error(outcome.GetError(), "PutBucketLifecycle fail"); } - LOG_INFO(log, "The lifecycle rule has been added, new_n_rules={} tag={}", old_rules.size() + 1, TaggingObjectIsDeleted); + LOG_INFO(client.log, "The lifecycle rule has been added, new_n_rules={} tag={}", old_rules.size(), TaggingObjectIsDeleted); +#endif } void listPrefix( - const Aws::S3::S3Client & client, - const String & bucket, + const TiFlashS3Client & client, const String & prefix, - std::function pager) + std::function pager) { - // Usually we don't need to set delimiter. - // Check the docs here for Delimiter && CommonPrefixes when you really need it. - // https://docs.aws.amazon.com/AmazonS3/latest/userguide/using-prefixes.html - listPrefix(client, bucket, prefix, /*delimiter*/ "", pager); + Stopwatch sw; + Aws::S3::Model::ListObjectsV2Request req; + req.WithBucket(client.bucket()).WithPrefix(client.root() + prefix); + + // If the `root == '/'`, then the return result will cut it off + // else we need to cut the root in the following codes + bool need_cut = client.root() != "/"; + size_t cut_size = client.root().size(); + + bool done = false; + size_t num_keys = 0; + while (!done) + { + Stopwatch sw_list; + ProfileEvents::increment(ProfileEvents::S3ListObjects); + auto outcome = client.ListObjectsV2(req); + if (!outcome.IsSuccess()) + { + throw fromS3Error(outcome.GetError(), "S3 ListObjectV2s failed, bucket={} root={} prefix={}", client.bucket(), client.root(), prefix); + } + GET_METRIC(tiflash_storage_s3_request_seconds, type_list_objects).Observe(sw_list.elapsedSeconds()); + + PageResult page_res{}; + const auto & result = outcome.GetResult(); + auto page_keys = result.GetCommonPrefixes().size(); + num_keys += page_keys; + for (const auto & object : result.GetContents()) + { + if (!need_cut) + { + page_res = pager(object); + } + else + { + // Copy the `Object` to cut off the `root` from key, the cost should be acceptable :( + auto object_without_root = object; + object_without_root.SetKey(object.GetKey().substr(cut_size, object.GetKey().size())); + page_res = pager(object_without_root); + } + if (!page_res.more) + break; + } + + // handle the result size over max size + done = !result.GetIsTruncated(); + if (!done && page_res.more) + { + const auto & next_token = result.GetNextContinuationToken(); + req.SetContinuationToken(next_token); + LOG_DEBUG(client.log, "listPrefix prefix={}, keys={}, total_keys={}, next_token={}", prefix, page_keys, num_keys, next_token); + } + } + LOG_DEBUG(client.log, "listPrefix prefix={}, total_keys={}, cost={:.2f}s", prefix, num_keys, sw.elapsedSeconds()); } -void listPrefix( - const Aws::S3::S3Client & client, - const String & bucket, +// Check the docs here for Delimiter && CommonPrefixes when you really need it. +// https://docs.aws.amazon.com/AmazonS3/latest/userguide/using-prefixes.html +void listPrefixWithDelimiter( + const TiFlashS3Client & client, const String & prefix, std::string_view delimiter, - std::function pager) + std::function pager) { Stopwatch sw; Aws::S3::Model::ListObjectsV2Request req; - req.WithBucket(bucket).WithPrefix(prefix); + req.WithBucket(client.bucket()).WithPrefix(client.root() + prefix); if (!delimiter.empty()) { req.SetDelimiter(String(delimiter)); } - static auto log = Logger::get("S3ListPrefix"); + // If the `root == '/'`, then the return result will cut it off + // else we need to cut the root in the following codes + bool need_cut = client.root() != "/"; + size_t cut_size = client.root().size(); bool done = false; size_t num_keys = 0; @@ -479,13 +553,30 @@ void listPrefix( auto outcome = client.ListObjectsV2(req); if (!outcome.IsSuccess()) { - throw fromS3Error(outcome.GetError(), "S3 ListObjects failed, bucket={} prefix={} delimiter={}", bucket, prefix, delimiter); + throw fromS3Error(outcome.GetError(), "S3 ListObjectV2s failed, bucket={} root={} prefix={} delimiter={}", client.bucket(), client.root(), prefix, delimiter); } GET_METRIC(tiflash_storage_s3_request_seconds, type_list_objects).Observe(sw_list.elapsedSeconds()); + PageResult page_res{}; const auto & result = outcome.GetResult(); - PageResult page_res = pager(result); - num_keys += page_res.num_keys; + auto page_keys = result.GetCommonPrefixes().size(); + num_keys += page_keys; + for (const auto & prefix : result.GetCommonPrefixes()) + { + if (!need_cut) + { + page_res = pager(prefix); + } + else + { + // Copy the `CommonPrefix` to cut off the `root`, the cost should be acceptable :( + auto prefix_without_root = prefix; + prefix_without_root.SetPrefix(prefix.GetPrefix().substr(cut_size, prefix.GetPrefix().size())); + page_res = pager(prefix_without_root); + } + if (!page_res.more) + break; + } // handle the result size over max size done = !result.GetIsTruncated(); @@ -493,55 +584,61 @@ void listPrefix( { const auto & next_token = result.GetNextContinuationToken(); req.SetContinuationToken(next_token); - LOG_DEBUG(log, "listPrefix prefix={}, keys={}, total_keys={}, next_token={}", prefix, page_res.num_keys, num_keys, next_token); + LOG_DEBUG(client.log, "listPrefixWithDelimiter prefix={}, delimiter={}, keys={}, total_keys={}, next_token={}", prefix, delimiter, page_keys, num_keys, next_token); } } - LOG_DEBUG(log, "listPrefix prefix={}, total_keys={}, cost={:.2f}s", prefix, num_keys, sw.elapsedSeconds()); + LOG_DEBUG(client.log, "listPrefixWithDelimiter prefix={}, delimiter={}, total_keys={}, cost={:.2f}s", prefix, delimiter, num_keys, sw.elapsedSeconds()); } -std::unordered_map listPrefixWithSize(const Aws::S3::S3Client & client, const String & bucket, const String & prefix) +std::optional anyKeyExistWithPrefix(const TiFlashS3Client & client, const String & prefix) +{ + std::optional key_opt; + listPrefix(client, prefix, [&key_opt](const Aws::S3::Model::Object & object) { + key_opt = object.GetKey(); + return PageResult{ + .num_keys = 1, + .more = false, // do not need more result + }; + }); + return key_opt; +} + +std::unordered_map listPrefixWithSize(const TiFlashS3Client & client, const String & prefix) { std::unordered_map keys_with_size; - listPrefix(client, bucket, prefix, "", [&](const Aws::S3::Model::ListObjectsV2Result & result) { - const auto & objects = result.GetContents(); - keys_with_size.reserve(keys_with_size.size() + objects.size()); - for (const auto & object : objects) - { - keys_with_size.emplace(object.GetKey().substr(prefix.size()), object.GetSize()); // Cut prefix - } - return PageResult{.num_keys = objects.size(), .more = true}; + listPrefix(client, prefix, [&](const Aws::S3::Model::Object & object) { + keys_with_size.emplace(object.GetKey().substr(prefix.size()), object.GetSize()); // Cut prefix + return PageResult{.num_keys = 1, .more = true}; }); return keys_with_size; } std::pair tryGetObjectModifiedTime( - const Aws::S3::S3Client & client, - const String & bucket, + const TiFlashS3Client & client, const String & key) { - auto o = headObject(client, bucket, key); + auto o = headObject(client, key); if (!o.IsSuccess()) { if (const auto & err = o.GetError(); isNotFoundError(err.GetErrorType())) { return {false, {}}; } - throw fromS3Error(o.GetError(), "Failed to check existence of object, bucket={} key={}", bucket, key); + throw fromS3Error(o.GetError(), "Failed to check existence of object, bucket={} key={}", client.bucket(), key); } // Else the object still exist const auto & res = o.GetResult(); // "DeleteMark" of S3 service, don't know what will lead to this - RUNTIME_CHECK(!res.GetDeleteMarker(), bucket, key); + RUNTIME_CHECK(!res.GetDeleteMarker(), client.bucket(), key); return {true, res.GetLastModified()}; } -void deleteObject(const Aws::S3::S3Client & client, const String & bucket, const String & key) +void deleteObject(const TiFlashS3Client & client, const String & key) { Stopwatch sw; Aws::S3::Model::DeleteObjectRequest req; - req.SetBucket(bucket); - req.SetKey(key); + client.setBucketAndKeyWithRoot(req, key); ProfileEvents::increment(ProfileEvents::S3DeleteObject); auto o = client.DeleteObject(req); RUNTIME_CHECK(o.IsSuccess(), o.GetError().GetMessage()); @@ -550,4 +647,50 @@ void deleteObject(const Aws::S3::S3Client & client, const String & bucket, const GET_METRIC(tiflash_storage_s3_request_seconds, type_delete_object).Observe(sw.elapsedSeconds()); } +void rawListPrefix( + const Aws::S3::S3Client & client, + const String & bucket, + const String & prefix, + std::string_view delimiter, + std::function pager) +{ + Stopwatch sw; + Aws::S3::Model::ListObjectsV2Request req; + req.WithBucket(bucket).WithPrefix(prefix); + if (!delimiter.empty()) + { + req.SetDelimiter(String(delimiter)); + } + + static auto log = Logger::get("S3RawListPrefix"); + + bool done = false; + size_t num_keys = 0; + while (!done) + { + Stopwatch sw_list; + ProfileEvents::increment(ProfileEvents::S3ListObjects); + auto outcome = client.ListObjectsV2(req); + if (!outcome.IsSuccess()) + { + throw fromS3Error(outcome.GetError(), "S3 ListObjectV2s failed, bucket={} prefix={} delimiter={}", bucket, prefix, delimiter); + } + GET_METRIC(tiflash_storage_s3_request_seconds, type_list_objects).Observe(sw_list.elapsedSeconds()); + + const auto & result = outcome.GetResult(); + PageResult page_res = pager(result); + num_keys += page_res.num_keys; + + // handle the result size over max size + done = !result.GetIsTruncated(); + if (!done && page_res.more) + { + const auto & next_token = result.GetNextContinuationToken(); + req.SetContinuationToken(next_token); + LOG_DEBUG(log, "rawListPrefix bucket={} prefix={} delimiter={} keys={} total_keys={} next_token={}", bucket, prefix, delimiter, page_res.num_keys, num_keys, next_token); + } + } + LOG_DEBUG(log, "rawListPrefix bucket={} prefix={} delimiter={} total_keys={} cost={:.2f}s", bucket, prefix, delimiter, num_keys, sw.elapsedSeconds()); +} + } // namespace DB::S3 diff --git a/dbms/src/Storages/S3/S3Common.h b/dbms/src/Storages/S3/S3Common.h index b52b1e75fed..88f4be2e3b3 100644 --- a/dbms/src/Storages/S3/S3Common.h +++ b/dbms/src/Storages/S3/S3Common.h @@ -15,6 +15,7 @@ #pragma once #include +#include #include #include #include @@ -47,21 +48,37 @@ class TiFlashS3Client : public Aws::S3::S3Client // Usually one tiflash instance only need access one bucket. // Store the bucket name to simpilfy some param passing. - explicit TiFlashS3Client(const String & bucket_name_); + TiFlashS3Client(const String & bucket_name_, const String & root_); TiFlashS3Client( const String & bucket_name_, + const String & root_, const Aws::Auth::AWSCredentials & credentials, const Aws::Client::ClientConfiguration & clientConfiguration, Aws::Client::AWSAuthV4Signer::PayloadSigningPolicy signPayloads, bool useVirtualAddressing); - TiFlashS3Client(const String & bucket_name_, std::unique_ptr && raw_client); + TiFlashS3Client( + const String & bucket_name_, + const String & root_, + std::unique_ptr && raw_client); const String & bucket() const { return bucket_name; } + const String & root() const { return key_root; } + + template + void setBucketAndKeyWithRoot(Request & req, const String & key) const + { + req.WithBucket(bucket_name).WithKey(key_root + key); + } + private: const String bucket_name; + String key_root; + +public: + LoggerPtr log; }; enum class S3GCMethod @@ -83,8 +100,7 @@ class ClientFactory void shutdown(); - const String & bucket() const; - std::shared_ptr sharedClient() const; + const String & bucket() const { return config.bucket; } std::shared_ptr sharedTiFlashClient() const; @@ -100,40 +116,29 @@ class ClientFactory Aws::SDKOptions aws_options; StorageS3Config config; - std::shared_ptr shared_client; std::shared_ptr shared_tiflash_client; }; -struct ObjectInfo -{ - size_t size = 0; - time_t last_modification_time = 0; -}; - bool isNotFoundError(Aws::S3::S3Errors error); -Aws::S3::Model::HeadObjectOutcome headObject(const Aws::S3::S3Client & client, const String & bucket, const String & key, const String & version_id = ""); - -S3::ObjectInfo getObjectInfo(const Aws::S3::S3Client & client, const String & bucket, const String & key, const String & version_id, bool throw_on_error); - -size_t getObjectSize(const Aws::S3::S3Client & client, const String & bucket, const String & key, const String & version_id, bool throw_on_error); +Aws::S3::Model::HeadObjectOutcome headObject(const TiFlashS3Client & client, const String & key); -bool objectExists(const Aws::S3::S3Client & client, const String & bucket, const String & key, const String & version_id = ""); +bool objectExists(const TiFlashS3Client & client, const String & key); -void uploadFile(const Aws::S3::S3Client & client, const String & bucket, const String & local_fname, const String & remote_fname); +void uploadFile(const TiFlashS3Client & client, const String & local_fname, const String & remote_fname); constexpr std::string_view TaggingObjectIsDeleted = "tiflash_deleted=true"; -void ensureLifecycleRuleExist(const Aws::S3::S3Client & client, const String & bucket, Int32 expire_days); +void ensureLifecycleRuleExist(const TiFlashS3Client & client, Int32 expire_days); /** * tagging is the tag-set for the object. The tag-set must be encoded as URL Query * parameters. (For example, "Key1=Value1") */ -void uploadEmptyFile(const Aws::S3::S3Client & client, const String & bucket, const String & key, const String & tagging = ""); +void uploadEmptyFile(const TiFlashS3Client & client, const String & key, const String & tagging = ""); -void downloadFile(const Aws::S3::S3Client & client, const String & bucket, const String & local_fname, const String & remote_fname); +void downloadFile(const TiFlashS3Client & client, const String & local_fname, const String & remote_fname); -void rewriteObjectWithTagging(const Aws::S3::S3Client & client, const String & bucket, const String & key, const String & tagging); +void rewriteObjectWithTagging(const TiFlashS3Client & client, const String & key, const String & tagging); struct PageResult { @@ -143,25 +148,33 @@ struct PageResult bool more; }; void listPrefix( - const Aws::S3::S3Client & client, - const String & bucket, + const TiFlashS3Client & client, const String & prefix, - std::function pager); -void listPrefix( - const Aws::S3::S3Client & client, - const String & bucket, + std::function pager); +void listPrefixWithDelimiter( + const TiFlashS3Client & client, const String & prefix, std::string_view delimiter, - std::function pager); + std::function pager); -std::unordered_map listPrefixWithSize(const Aws::S3::S3Client & client, const String & bucket, const String & prefix); +std::optional anyKeyExistWithPrefix(const TiFlashS3Client & client, const String & prefix); + +std::unordered_map listPrefixWithSize(const TiFlashS3Client & client, const String & prefix); std::pair tryGetObjectModifiedTime( - const Aws::S3::S3Client & client, - const String & bucket, + const TiFlashS3Client & client, const String & key); -void deleteObject(const Aws::S3::S3Client & client, const String & bucket, const String & key); +void deleteObject(const TiFlashS3Client & client, const String & key); + +// Unlike `listPrefix` or other methods above, this does not handle +// the TiFlashS3Client `root`. +void rawListPrefix( + const Aws::S3::S3Client & client, + const String & bucket, + const String & prefix, + std::string_view delimiter, + std::function pager); } // namespace DB::S3 diff --git a/dbms/src/Storages/S3/S3Filename.cpp b/dbms/src/Storages/S3/S3Filename.cpp index 333cde27d79..df3f499c86c 100644 --- a/dbms/src/Storages/S3/S3Filename.cpp +++ b/dbms/src/Storages/S3/S3Filename.cpp @@ -41,6 +41,10 @@ const static re2::RE2 rgx_store_prefix("^s(?P[0-9]+)/$"); const static re2::RE2 rgx_data_or_manifest("^s(?P[0-9]+)/(data|manifest)/(?P.+)$"); const static re2::RE2 rgx_subpath_manifest("mf_(?P[0-9]+)"); +/// parsing DTFile +const static re2::RE2 rgx_subpath_dtfile("t_(?P[0-9]+)/dmf_(?P[0-9]+)"); +const static re2::RE2 rgx_subpath_keyspace_dtfile("ks_(?P[0-9]+)_t_(?P[0-9]+)/dmf_(?P[0-9]+)"); + constexpr static std::string_view DELMARK_SUFFIX = ".del"; // clang-format off @@ -101,6 +105,33 @@ bool S3FilenameView::isDMFile() const return (startsWith(data_subpath, "t_") || startsWith(data_subpath, "ks_")); } +DMFileOID S3FilenameView::getDMFileOID() const +{ + RUNTIME_CHECK(isDMFile()); + TableID table_id; + UInt64 file_id; + re2::StringPiece prefix_sp{data_subpath.data(), data_subpath.size()}; + if (startsWith(data_subpath, "t_")) + { + RUNTIME_CHECK(re2::RE2::FullMatch(prefix_sp, details::rgx_subpath_dtfile, &table_id, &file_id)); + return DMFileOID{ + .store_id = store_id, + .table_id = table_id, + .file_id = file_id, + }; + } + else + { + UInt64 keyspace_id; + RUNTIME_CHECK(re2::RE2::FullMatch(prefix_sp, details::rgx_subpath_keyspace_dtfile, &keyspace_id, &table_id, &file_id)); + return DMFileOID{ + .store_id = store_id, + .table_id = table_id, + .file_id = file_id, + }; + } +} + String S3FilenameView::toFullKey() const { return details::toFullKey(type, store_id, data_subpath); diff --git a/dbms/src/Storages/S3/S3Filename.h b/dbms/src/Storages/S3/S3Filename.h index bdec58c8409..4236aa38af3 100644 --- a/dbms/src/Storages/S3/S3Filename.h +++ b/dbms/src/Storages/S3/S3Filename.h @@ -80,6 +80,7 @@ struct S3FilenameView ALWAYS_INLINE bool isDataFile() const { return type == S3FilenameType::DataFile; } bool isDMFile() const; + DMFileOID getDMFileOID() const; // Return the lock key prefix for finding any locks on this data file through `S3::LIST` String getLockPrefix() const; // Return the lock key for writing lock file on S3 diff --git a/dbms/src/Storages/S3/S3GCManager.cpp b/dbms/src/Storages/S3/S3GCManager.cpp index 9230a990cff..ae163ec27b3 100644 --- a/dbms/src/Storages/S3/S3GCManager.cpp +++ b/dbms/src/Storages/S3/S3GCManager.cpp @@ -51,12 +51,10 @@ namespace DB::S3 S3GCManager::S3GCManager( pingcap::pd::ClientPtr pd_client_, - std::shared_ptr client_, OwnerManagerPtr gc_owner_manager_, S3LockClientPtr lock_client_, S3GCConfig config_) : pd_client(std::move(pd_client_)) - , client(std::move(client_)) , gc_owner_manager(std::move(gc_owner_manager_)) , lock_client(std::move(lock_client_)) , shutdown_called(false) @@ -88,7 +86,8 @@ bool S3GCManager::runOnAllStores() if (config.method == S3GCMethod::Lifecycle && !lifecycle_has_been_set) { - ensureLifecycleRuleExist(*client, client->bucket(), /*expire_days*/ 1); + auto client = S3::ClientFactory::instance().sharedTiFlashClient(); + ensureLifecycleRuleExist(*client, /*expire_days*/ 1); lifecycle_has_been_set = true; } @@ -135,6 +134,7 @@ void S3GCManager::runForStore(UInt64 gc_store_id) { // get a timepoint at the begin, only remove objects that expired compare // to this timepoint + auto client = S3::ClientFactory::instance().sharedTiFlashClient(); const Aws::Utils::DateTime gc_timepoint = Aws::Utils::DateTime::Now(); LOG_DEBUG(log, "run gc, gc_store_id={} timepoint={}", gc_store_id, gc_timepoint.ToGmtString(Aws::Utils::DateFormat::ISO_8601)); @@ -196,6 +196,7 @@ void S3GCManager::runForTombstoneStore(UInt64 gc_store_id) cleanUnusedLocks(gc_store_id, lock_prefix, std::numeric_limits::max(), valid_lock_files, gc_timepoint); // clean all manifest objects + auto client = S3::ClientFactory::instance().sharedTiFlashClient(); const auto manifests = CheckpointManifestS3Set::getFromS3(*client, gc_store_id); removeOutdatedManifest(manifests, nullptr); @@ -227,39 +228,42 @@ void S3GCManager::cleanUnusedLocks( const std::unordered_set & valid_lock_files, const Aws::Utils::DateTime & timepoint) { + auto client = S3::ClientFactory::instance().sharedTiFlashClient(); // All locks (even for different stores) share the same prefix, list the lock files under this prefix - listPrefix(*client, client->bucket(), scan_prefix, [&](const Aws::S3::Model::ListObjectsV2Result & result) { - const auto & objects = result.GetContents(); - if (shutdown_called) - { - LOG_INFO(log, "shutting down, break"); - // .more=false to break the list - return PageResult{.num_keys = objects.size(), .more = false}; - } + S3::listPrefix( + *client, + scan_prefix, + [&](const Aws::S3::Model::Object & object) { + if (shutdown_called) + { + LOG_INFO(log, "shutting down, break"); + // .more=false to break the list + return PageResult{.num_keys = 1, .more = false}; + } - for (const auto & object : objects) - { - const auto & lock_key = object.GetKey(); - LOG_TRACE(log, "lock_key={}", lock_key); - const auto lock_filename_view = S3FilenameView::fromKey(lock_key); - RUNTIME_CHECK(lock_filename_view.isLockFile(), lock_key); - const auto lock_info = lock_filename_view.getLockInfo(); - // The lock file is not managed by `gc_store_id`, skip - if (lock_info.store_id != gc_store_id) - continue; - // The lock is not managed by the latest manifest yet, wait for - // next GC round - if (lock_info.sequence > safe_sequence) - continue; - // The lock is still valid - if (valid_lock_files.count(lock_key) > 0) - continue; - - // The data file is not used by `gc_store_id` anymore, remove the lock file - cleanOneLock(lock_key, lock_filename_view, timepoint); - } - return PageResult{.num_keys = objects.size(), .more = true}; - }); + do + { + const auto & lock_key = object.GetKey(); + LOG_TRACE(log, "lock_key={}", lock_key); + const auto lock_filename_view = S3FilenameView::fromKey(lock_key); + RUNTIME_CHECK(lock_filename_view.isLockFile(), lock_key); + const auto lock_info = lock_filename_view.getLockInfo(); + // The lock file is not managed by `gc_store_id`, skip + if (lock_info.store_id != gc_store_id) + break; + // The lock is not managed by the latest manifest yet, wait for + // next GC round + if (lock_info.sequence > safe_sequence) + break; + // The lock is still valid + if (valid_lock_files.count(lock_key) > 0) + break; + + // The data file is not used by `gc_store_id` anymore, remove the lock file + cleanOneLock(lock_key, lock_filename_view, timepoint); + } while (false); + return PageResult{.num_keys = 1, .more = true}; + }); } void S3GCManager::cleanOneLock(const String & lock_key, const S3FilenameView & lock_filename_view, const Aws::Utils::DateTime & timepoint) @@ -270,7 +274,8 @@ void S3GCManager::cleanOneLock(const String & lock_key, const S3FilenameView & l const auto unlocked_datafile_delmark_key = unlocked_datafilename_view.getDelMarkKey(); // delete S3 lock file - deleteObject(*client, client->bucket(), lock_key); + auto client = S3::ClientFactory::instance().sharedTiFlashClient(); + deleteObject(*client, lock_key); // TODO: If `lock_key` is the only lock to datafile and GCManager crashes // after the lock deleted but before delmark uploaded, then the @@ -279,7 +284,7 @@ void S3GCManager::cleanOneLock(const String & lock_key, const S3FilenameView & l bool delmark_exists = false; Aws::Utils::DateTime mtime; - std::tie(delmark_exists, mtime) = tryGetObjectModifiedTime(*client, client->bucket(), unlocked_datafile_delmark_key); + std::tie(delmark_exists, mtime) = tryGetObjectModifiedTime(*client, unlocked_datafile_delmark_key); if (!delmark_exists) { bool ok; @@ -380,7 +385,8 @@ void S3GCManager::removeDataFileIfDelmarkExpired( // error when the key is not exist physicalRemoveDataFile(datafile_key); - deleteObject(*client, client->bucket(), delmark_key); + auto client = S3::ClientFactory::instance().sharedTiFlashClient(); + deleteObject(*client, delmark_key); LOG_INFO(log, "datafile delmark deleted, key={}", delmark_key); } @@ -390,27 +396,27 @@ void S3GCManager::tryCleanExpiredDataFiles(UInt64 gc_store_id, const Aws::Utils: // the keys by prefix, and if there is an expired delmark, then try to remove // its correspond StableFile or CheckpointDataFile. const auto prefix = S3Filename::fromStoreId(gc_store_id).toDataPrefix(); - listPrefix(*client, client->bucket(), prefix, [&](const Aws::S3::Model::ListObjectsV2Result & result) { - const auto & objects = result.GetContents(); + auto client = S3::ClientFactory::instance().sharedTiFlashClient(); + S3::listPrefix(*client, prefix, [&](const Aws::S3::Model::Object & object) { if (shutdown_called) { LOG_INFO(log, "shutting down, break"); // .more=false to break the list - return PageResult{.num_keys = objects.size(), .more = false}; + return PageResult{.num_keys = 1, .more = false}; } - for (const auto & object : objects) + do { const auto & delmark_key = object.GetKey(); LOG_TRACE(log, "key={}", object.GetKey()); const auto filename_view = S3FilenameView::fromKey(delmark_key); // Only remove the data file with expired delmark if (!filename_view.isDelMark()) - continue; + break; auto datafile_key = filename_view.asDataFile().toFullKey(); removeDataFileIfDelmarkExpired(datafile_key, delmark_key, timepoint, object.GetLastModified()); - } - return PageResult{.num_keys = objects.size(), .more = true}; + } while (false); + return PageResult{.num_keys = 1, .more = true}; }); } @@ -419,10 +425,11 @@ void S3GCManager::lifecycleMarkDataFileDeleted(const String & datafile_key) assert(config.method == S3GCMethod::Lifecycle); auto view = S3FilenameView::fromKey(datafile_key); + auto client = S3::ClientFactory::instance().sharedTiFlashClient(); if (!view.isDMFile()) { // CheckpointDataFile is a single object, add tagging for it and update its mtime - rewriteObjectWithTagging(*client, client->bucket(), datafile_key, String(TaggingObjectIsDeleted)); + rewriteObjectWithTagging(*client, datafile_key, String(TaggingObjectIsDeleted)); LOG_INFO(log, "datafile deleted by lifecycle tagging, key={}", datafile_key); } else @@ -431,15 +438,11 @@ void S3GCManager::lifecycleMarkDataFileDeleted(const String & datafile_key) // Rewrite all objects with tagging belong to this DMFile // TODO: If GCManager unexpectedly exit in the middle, it will leave some broken // sub file for DMFile, try clean them later. - S3::listPrefix(*client, client->bucket(), datafile_key, [this, &datafile_key](const Aws::S3::Model::ListObjectsV2Result & result) { - const auto & objs = result.GetContents(); - for (const auto & obj : objs) - { - const auto & sub_key = obj.GetKey(); - rewriteObjectWithTagging(*client, client->bucket(), sub_key, String(TaggingObjectIsDeleted)); - LOG_INFO(log, "datafile deleted by lifecycle tagging, key={} sub_key={}", datafile_key, sub_key); - } - return PageResult{.num_keys = objs.size(), .more = true}; + S3::listPrefix(*client, datafile_key, [this, &client, &datafile_key](const Aws::S3::Model::Object & object) { + const auto & sub_key = object.GetKey(); + rewriteObjectWithTagging(*client, sub_key, String(TaggingObjectIsDeleted)); + LOG_INFO(log, "datafile deleted by lifecycle tagging, key={} sub_key={}", datafile_key, sub_key); + return PageResult{.num_keys = 1, .more = true}; }); LOG_INFO(log, "datafile deleted by lifecycle tagging, all sub keys are deleted, key={}", datafile_key); } @@ -450,10 +453,11 @@ void S3GCManager::physicalRemoveDataFile(const String & datafile_key) assert(config.method == S3GCMethod::ScanThenDelete); auto view = S3FilenameView::fromKey(datafile_key); + auto client = S3::ClientFactory::instance().sharedTiFlashClient(); if (!view.isDMFile()) { // CheckpointDataFile is a single object, remove it. - deleteObject(*client, client->bucket(), datafile_key); + deleteObject(*client, datafile_key); LOG_INFO(log, "datafile deleted, key={}", datafile_key); } else @@ -462,40 +466,32 @@ void S3GCManager::physicalRemoveDataFile(const String & datafile_key) // Remove all objects belong to this DMFile // TODO: If GCManager unexpectedly exit in the middle, it will leave some broken // sub file for DMFile, try clean them later. - S3::listPrefix(*client, client->bucket(), datafile_key, [this, &datafile_key](const Aws::S3::Model::ListObjectsV2Result & result) { - const auto & objs = result.GetContents(); - for (const auto & obj : objs) - { - const auto & sub_key = obj.GetKey(); - deleteObject(*client, client->bucket(), sub_key); - LOG_INFO(log, "datafile deleted, key={} sub_key={}", datafile_key, sub_key); - } - return PageResult{.num_keys = objs.size(), .more = true}; + S3::listPrefix(*client, datafile_key, [this, &client, &datafile_key](const Aws::S3::Model::Object & object) { + const auto & sub_key = object.GetKey(); + deleteObject(*client, sub_key); + LOG_INFO(log, "datafile deleted, key={} sub_key={}", datafile_key, sub_key); + return PageResult{.num_keys = 1, .more = true}; }); LOG_INFO(log, "datafile deleted, all sub keys are deleted, key={}", datafile_key); } } -std::vector S3GCManager::getAllStoreIds() const +std::vector S3GCManager::getAllStoreIds() { std::vector all_store_ids; // The store key are "s${store_id}/", we need setting delimiter "/" to get the // common prefixes result. // Reference: https://docs.aws.amazon.com/AmazonS3/latest/userguide/using-prefixes.html - listPrefix( + auto client = S3::ClientFactory::instance().sharedTiFlashClient(); + S3::listPrefixWithDelimiter( *client, - client->bucket(), /*prefix*/ S3Filename::allStorePrefix(), /*delimiter*/ "/", - [&all_store_ids](const Aws::S3::Model::ListObjectsV2Result & result) { - const Aws::Vector & prefixes = result.GetCommonPrefixes(); - for (const auto & prefix : prefixes) - { - const auto filename_view = S3FilenameView::fromStoreKeyPrefix(prefix.GetPrefix()); - RUNTIME_CHECK(filename_view.type == S3FilenameType::StorePrefix, prefix.GetPrefix()); - all_store_ids.emplace_back(filename_view.store_id); - } - return PageResult{.num_keys = prefixes.size(), .more = true}; + [&all_store_ids](const Aws::S3::Model::CommonPrefix & prefix) { + const auto filename_view = S3FilenameView::fromStoreKeyPrefix(prefix.GetPrefix()); + RUNTIME_CHECK(filename_view.type == S3FilenameType::StorePrefix, prefix.GetPrefix()); + all_store_ids.emplace_back(filename_view.store_id); + return PageResult{.num_keys = 1, .more = true}; }); return all_store_ids; @@ -536,12 +532,13 @@ std::unordered_set S3GCManager::getValidLocksFromManifest(const Strings void S3GCManager::removeOutdatedManifest(const CheckpointManifestS3Set & manifests, const Aws::Utils::DateTime * const timepoint) { + auto client = S3::ClientFactory::instance().sharedTiFlashClient(); if (timepoint == nullptr) { for (const auto & mf : manifests.objects()) { // store is tombstone, remove all manifests - deleteObject(*client, client->bucket(), mf.second.key); + deleteObject(*client, mf.second.key); LOG_INFO( log, "remove outdated manifest because store is tombstone, key={} mtime={}", @@ -557,7 +554,7 @@ void S3GCManager::removeOutdatedManifest(const CheckpointManifestS3Set & manifes for (const auto & mf : outdated_mfs) { // expired manifest, remove - deleteObject(*client, client->bucket(), mf.key); + deleteObject(*client, mf.key); LOG_INFO( log, "remove outdated manifest, key={} mtime={}", @@ -576,8 +573,7 @@ S3GCManagerService::S3GCManagerService( const S3GCConfig & config) : global_ctx(context.getGlobalContext()) { - auto s3_client = S3::ClientFactory::instance().sharedTiFlashClient(); - manager = std::make_unique(std::move(pd_client), std::move(s3_client), std::move(gc_owner_manager_), std::move(lock_client), config); + manager = std::make_unique(std::move(pd_client), std::move(gc_owner_manager_), std::move(lock_client), config); timer = global_ctx.getBackgroundPool().addTask( [this]() { @@ -594,11 +590,18 @@ S3GCManagerService::~S3GCManagerService() void S3GCManagerService::shutdown() { - manager->shutdown(); + if (manager) + { + // set the shutdown flag + manager->shutdown(); + } + if (timer) { + // Remove the task handler. It will block until the task break global_ctx.getBackgroundPool().removeTask(timer); timer = nullptr; + // then we can reset the manager manager = nullptr; } } diff --git a/dbms/src/Storages/S3/S3GCManager.h b/dbms/src/Storages/S3/S3GCManager.h index 6863976aace..a11106084df 100644 --- a/dbms/src/Storages/S3/S3GCManager.h +++ b/dbms/src/Storages/S3/S3GCManager.h @@ -85,7 +85,6 @@ class S3GCManager public: explicit S3GCManager( pingcap::pd::ClientPtr pd_client_, - std::shared_ptr client_, OwnerManagerPtr gc_owner_manager_, S3LockClientPtr lock_client_, S3GCConfig config_); @@ -121,7 +120,7 @@ class S3GCManager void lifecycleMarkDataFileDeleted(const String & datafile_key); void physicalRemoveDataFile(const String & datafile_key); - std::vector getAllStoreIds() const; + static std::vector getAllStoreIds(); std::unordered_set getValidLocksFromManifest(const Strings & manifest_keys); @@ -130,8 +129,6 @@ class S3GCManager private: const pingcap::pd::ClientPtr pd_client; - const std::shared_ptr client; - const OwnerManagerPtr gc_owner_manager; const S3LockClientPtr lock_client; diff --git a/dbms/src/Storages/S3/S3RandomAccessFile.cpp b/dbms/src/Storages/S3/S3RandomAccessFile.cpp index 843d2517fa1..92bc1301e2a 100644 --- a/dbms/src/Storages/S3/S3RandomAccessFile.cpp +++ b/dbms/src/Storages/S3/S3RandomAccessFile.cpp @@ -30,11 +30,9 @@ extern const Event S3ReadBytes; namespace DB::S3 { S3RandomAccessFile::S3RandomAccessFile( - std::shared_ptr client_ptr_, - const String & bucket_, + std::shared_ptr client_ptr_, const String & remote_fname_) : client_ptr(std::move(client_ptr_)) - , bucket(bucket_) , remote_fname(remote_fname_) , log(Logger::get("S3RandomAccessFile")) { @@ -48,7 +46,7 @@ ssize_t S3RandomAccessFile::read(char * buf, size_t size) size_t gcount = istr.gcount(); if (gcount == 0 && !istr.eof()) { - LOG_ERROR(log, "Cannot read from istream. bucket={} key={}", bucket, remote_fname); + LOG_ERROR(log, "Cannot read from istream. bucket={} root={} key={}", client_ptr->bucket(), client_ptr->root(), remote_fname); return -1; } ProfileEvents::increment(ProfileEvents::S3ReadBytes, gcount); @@ -76,13 +74,12 @@ void S3RandomAccessFile::initialize() { Stopwatch sw; Aws::S3::Model::GetObjectRequest req; - req.SetBucket(bucket); - req.SetKey(remote_fname); + client_ptr->setBucketAndKeyWithRoot(req, remote_fname); ProfileEvents::increment(ProfileEvents::S3GetObject); auto outcome = client_ptr->GetObject(req); if (!outcome.IsSuccess()) { - throw S3::fromS3Error(outcome.GetError(), "bucket={} key={}", bucket, remote_fname); + throw S3::fromS3Error(outcome.GetError(), "S3 GetObject failed, bucket={} root={} key={}", client_ptr->bucket(), client_ptr->root(), remote_fname); } ProfileEvents::increment(ProfileEvents::S3ReadBytes, outcome.GetResult().GetContentLength()); GET_METRIC(tiflash_storage_s3_request_seconds, type_get_object).Observe(sw.elapsedSeconds()); @@ -111,6 +108,6 @@ RandomAccessFilePtr S3RandomAccessFile::create(const String & remote_fname) return file; } auto & ins = S3::ClientFactory::instance(); - return std::make_shared(ins.sharedClient(), ins.bucket(), remote_fname); + return std::make_shared(ins.sharedTiFlashClient(), remote_fname); } } // namespace DB::S3 diff --git a/dbms/src/Storages/S3/S3RandomAccessFile.h b/dbms/src/Storages/S3/S3RandomAccessFile.h index 767dbe49da2..11c1ec29e98 100644 --- a/dbms/src/Storages/S3/S3RandomAccessFile.h +++ b/dbms/src/Storages/S3/S3RandomAccessFile.h @@ -17,6 +17,7 @@ #include #include #include +#include #include #include @@ -38,8 +39,7 @@ class S3RandomAccessFile final : public RandomAccessFile static RandomAccessFilePtr create(const String & remote_fname); S3RandomAccessFile( - std::shared_ptr client_ptr_, - const String & bucket_, + std::shared_ptr client_ptr_, const String & remote_fname_); off_t seek(off_t offset, int whence) override; @@ -48,7 +48,7 @@ class S3RandomAccessFile final : public RandomAccessFile std::string getFileName() const override { - return fmt::format("{}/{}", bucket, remote_fname); + return fmt::format("{}/{}", client_ptr->bucket(), remote_fname); } ssize_t pread(char * /*buf*/, size_t /*size*/, off_t /*offset*/) const override @@ -74,8 +74,7 @@ class S3RandomAccessFile final : public RandomAccessFile private: void initialize(); - std::shared_ptr client_ptr; - String bucket; + std::shared_ptr client_ptr; String remote_fname; Aws::S3::Model::GetObjectResult read_result; diff --git a/dbms/src/Storages/S3/S3WritableFile.cpp b/dbms/src/Storages/S3/S3WritableFile.cpp index 30cbfa291b0..03884ed18b4 100644 --- a/dbms/src/Storages/S3/S3WritableFile.cpp +++ b/dbms/src/Storages/S3/S3WritableFile.cpp @@ -15,6 +15,7 @@ #include #include #include +#include #include #include #include @@ -56,12 +57,10 @@ struct S3WritableFile::PutObjectTask }; S3WritableFile::S3WritableFile( - std::shared_ptr client_ptr_, - const String & bucket_, + std::shared_ptr client_ptr_, const String & remote_fname_, const WriteSettings & write_settings_) - : bucket(bucket_) - , remote_fname(remote_fname_) + : remote_fname(remote_fname_) , client_ptr(std::move(client_ptr_)) , write_settings(write_settings_) , log(Logger::get("S3WritableFile")) @@ -76,7 +75,7 @@ ssize_t S3WritableFile::write(char * buf, size_t size) temporary_buffer->write(buf, size); if (!temporary_buffer->good()) { - LOG_ERROR(log, "write size={} failed: bucket={} key={}", size, bucket, remote_fname); + LOG_ERROR(log, "write size={} failed: bucket={} root={} key={}", size, client_ptr->bucket(), client_ptr->root(), remote_fname); return -1; } ProfileEvents::increment(ProfileEvents::S3WriteBytes, size); @@ -127,7 +126,7 @@ void S3WritableFile::finalize() if (write_settings.check_objects_after_upload) { // TODO(jinhe): check checksums. - auto resp = S3::headObject(*client_ptr, bucket, remote_fname); + auto resp = S3::headObject(*client_ptr, remote_fname); checkS3Outcome(resp); } } @@ -139,8 +138,7 @@ void S3WritableFile::createMultipartUpload() GET_METRIC(tiflash_storage_s3_request_seconds, type_create_multi_part_upload).Observe(sw.elapsedSeconds()); }); Aws::S3::Model::CreateMultipartUploadRequest req; - req.SetBucket(bucket); - req.SetKey(remote_fname); + client_ptr->setBucketAndKeyWithRoot(req, remote_fname); req.SetContentType("binary/octet-stream"); ProfileEvents::increment(ProfileEvents::S3CreateMultipartUpload); auto outcome = client_ptr->CreateMultipartUpload(req); @@ -153,7 +151,7 @@ void S3WritableFile::writePart() auto size = temporary_buffer->tellp(); if (size < 0) { - throw Exception(ErrorCodes::CORRUPTED_DATA, "Buffer is in bad state. bucket={}, key={}", bucket, remote_fname); + throw Exception(ErrorCodes::CORRUPTED_DATA, "Buffer is in bad state. bucket={} root={} key={}", client_ptr->bucket(), client_ptr->root(), remote_fname); } if (size == 0) { @@ -172,8 +170,7 @@ void S3WritableFile::fillUploadRequest(Aws::S3::Model::UploadPartRequest & req) // Increase part number. ++part_number; // Setup request. - req.SetBucket(bucket); - req.SetKey(remote_fname); + client_ptr->setBucketAndKeyWithRoot(req, remote_fname); req.SetPartNumber(static_cast(part_number)); req.SetUploadId(multipart_upload_id); req.SetContentLength(temporary_buffer->tellp()); @@ -195,11 +192,10 @@ void S3WritableFile::processUploadRequest(UploadPartTask & task) void S3WritableFile::completeMultipartUpload() { - RUNTIME_CHECK_MSG(!part_tags.empty(), "Failed to complete multipart upload. No parts have uploaded. bucket={}, key={}", bucket, remote_fname); + RUNTIME_CHECK_MSG(!part_tags.empty(), "Failed to complete multipart upload. No parts have uploaded. bucket={} root={} key={}", client_ptr->bucket(), client_ptr->root(), remote_fname); Aws::S3::Model::CompleteMultipartUploadRequest req; - req.SetBucket(bucket); - req.SetKey(remote_fname); + client_ptr->setBucketAndKeyWithRoot(req, remote_fname); req.SetUploadId(multipart_upload_id); Aws::S3::Model::CompletedMultipartUpload multipart_upload; for (size_t i = 0; i < part_tags.size(); ++i) @@ -220,17 +216,33 @@ void S3WritableFile::completeMultipartUpload() auto outcome = client_ptr->CompleteMultipartUpload(req); if (outcome.IsSuccess()) { - LOG_DEBUG(log, "Multipart upload has completed. bucket={} key={} upload_id={} parts={}", bucket, remote_fname, multipart_upload_id, part_tags.size()); + LOG_DEBUG(log, "Multipart upload has completed. bucket={} root={} key={} upload_id={} parts={}", client_ptr->bucket(), client_ptr->root(), remote_fname, multipart_upload_id, part_tags.size()); break; } if (i + 1 < max_retry) { const auto & e = outcome.GetError(); - LOG_INFO(log, "Multipart upload failed and need retry: bucket={} key={} upload_id={} parts={} error={} message={}", bucket, remote_fname, multipart_upload_id, part_tags.size(), magic_enum::enum_name(e.GetErrorType()), e.GetMessage()); + LOG_INFO( + log, + "Multipart upload failed and need retry: bucket={} root={} key={} upload_id={} parts={} error={} message={}", + client_ptr->bucket(), + client_ptr->root(), + remote_fname, + multipart_upload_id, + part_tags.size(), + magic_enum::enum_name(e.GetErrorType()), + e.GetMessage()); } else { - throw fromS3Error(outcome.GetError(), "bucket={} key={} upload_id={} parts={}", bucket, remote_fname, multipart_upload_id, part_tags.size()); + throw fromS3Error( + outcome.GetError(), + "S3 CompleteMultipartUpload failed, bucket={} root={} key={} upload_id={} parts={}", + client_ptr->bucket(), + client_ptr->root(), + remote_fname, + multipart_upload_id, + part_tags.size()); } } } @@ -240,7 +252,7 @@ void S3WritableFile::makeSinglepartUpload() auto size = temporary_buffer->tellp(); if (size < 0) { - throw Exception(ErrorCodes::CORRUPTED_DATA, "Buffer is in bad state. bucket={}, key={}", bucket, remote_fname); + throw Exception(ErrorCodes::CORRUPTED_DATA, "Buffer is in bad state. bucket={} root={} key={}", client_ptr->bucket(), client_ptr->root(), remote_fname); } PutObjectTask task; fillPutRequest(task.req); @@ -249,8 +261,7 @@ void S3WritableFile::makeSinglepartUpload() void S3WritableFile::fillPutRequest(Aws::S3::Model::PutObjectRequest & req) { - req.SetBucket(bucket); - req.SetKey(remote_fname); + client_ptr->setBucketAndKeyWithRoot(req, remote_fname); req.SetContentLength(temporary_buffer->tellp()); req.SetBody(temporary_buffer); req.SetContentType("binary/octet-stream"); @@ -269,17 +280,17 @@ void S3WritableFile::processPutRequest(const PutObjectTask & task) auto outcome = client_ptr->PutObject(task.req); if (outcome.IsSuccess()) { - LOG_DEBUG(log, "Single part upload has completed. bucket={}, key={}, size={}", bucket, remote_fname, task.req.GetContentLength()); + LOG_DEBUG(log, "Single part upload has completed. bucket={} root={} key={}, size={}", client_ptr->bucket(), client_ptr->root(), remote_fname, task.req.GetContentLength()); break; } if (i + 1 < max_retry) { const auto & e = outcome.GetError(); - LOG_INFO(log, "Single part upload failed: bucket={} key={} error={} message={}", bucket, remote_fname, magic_enum::enum_name(e.GetErrorType()), e.GetMessage()); + LOG_INFO(log, "Single part upload failed: bucket={} root={} key={} error={} message={}", client_ptr->bucket(), client_ptr->root(), remote_fname, magic_enum::enum_name(e.GetErrorType()), e.GetMessage()); } else { - throw fromS3Error(outcome.GetError(), "bucket={} key={}", bucket, remote_fname); + throw fromS3Error(outcome.GetError(), "S3 PutObject failed, bucket={} root={} key={}", client_ptr->bucket(), client_ptr->root(), remote_fname); } } } @@ -287,8 +298,7 @@ void S3WritableFile::processPutRequest(const PutObjectTask & task) std::shared_ptr S3WritableFile::create(const String & remote_fname_) { return std::make_shared( - S3::ClientFactory::instance().sharedClient(), - S3::ClientFactory::instance().bucket(), + S3::ClientFactory::instance().sharedTiFlashClient(), remote_fname_, WriteSettings{}); } diff --git a/dbms/src/Storages/S3/S3WritableFile.h b/dbms/src/Storages/S3/S3WritableFile.h index 809cb2eaf65..2b4003d309f 100644 --- a/dbms/src/Storages/S3/S3WritableFile.h +++ b/dbms/src/Storages/S3/S3WritableFile.h @@ -51,8 +51,7 @@ class S3WritableFile final : public WritableFile static std::shared_ptr create(const String & remote_fname_); S3WritableFile( - std::shared_ptr client_ptr_, - const String & bucket_, + std::shared_ptr client_ptr_, const String & remote_fname_, const WriteSettings & write_settings_); @@ -65,7 +64,7 @@ class S3WritableFile final : public WritableFile std::string getFileName() const override { - return fmt::format("{}/{}", bucket, remote_fname); + return fmt::format("{}/{}", client_ptr->bucket(), remote_fname); } void close() override @@ -145,13 +144,12 @@ class S3WritableFile final : public WritableFile { if (!outcome.IsSuccess()) { - throw S3::fromS3Error(outcome.GetError(), "bucket={} key={}", bucket, remote_fname); + throw S3::fromS3Error(outcome.GetError(), "bucket={} root={} key={}", client_ptr->bucket(), client_ptr->root(), remote_fname); } } - const String bucket; const String remote_fname; - const std::shared_ptr client_ptr; + const std::shared_ptr client_ptr; const WriteSettings write_settings; std::shared_ptr temporary_buffer; // Buffer to accumulate data. @@ -168,4 +166,4 @@ class S3WritableFile final : public WritableFile bool is_close = false; }; -} // namespace DB::S3 \ No newline at end of file +} // namespace DB::S3 diff --git a/dbms/src/Storages/S3/tests/gtest_filecache.cpp b/dbms/src/Storages/S3/tests/gtest_filecache.cpp index c9da8670b1b..dd8df69f793 100644 --- a/dbms/src/Storages/S3/tests/gtest_filecache.cpp +++ b/dbms/src/Storages/S3/tests/gtest_filecache.cpp @@ -14,8 +14,11 @@ #include #include -#include +#include +#include +#include #include +#include #include #include #include @@ -52,19 +55,18 @@ class FileCacheTest : public ::testing::Test tmp_dir = DB::tests::TiFlashTestEnv::getTemporaryPath("FileCacheTest"); log = Logger::get("FileCacheTest"); std::filesystem::remove_all(std::filesystem::path(tmp_dir)); - s3_client = ::DB::S3::ClientFactory::instance().sharedClient(); - bucket = ::DB::S3::ClientFactory::instance().bucket(); - ASSERT_TRUE(createBucketIfNotExist()); + s3_client = ::DB::S3::ClientFactory::instance().sharedTiFlashClient(); + ASSERT_TRUE(DB::tests::TiFlashTestEnv::createBucketIfNotExist(*s3_client)); std::random_device dev; rng = std::mt19937{dev()}; next_id = std::chrono::system_clock::to_time_t(std::chrono::system_clock::now()); + capacity_metrics = TiFlashTestEnv::getContext()->getPathCapacity(); } protected: - std::shared_ptr s3_client; - String bucket; + std::shared_ptr s3_client; std::mt19937 rng; - UInt64 next_id; + UInt64 next_id = 0; inline static const std::vector basenames = { "%2D1.dat", @@ -84,31 +86,11 @@ class FileCacheTest : public ::testing::Test "meta", }; - bool createBucketIfNotExist() - { - Aws::S3::Model::CreateBucketRequest request; - request.SetBucket(bucket); - Aws::S3::Model::CreateBucketOutcome outcome = s3_client->CreateBucket(request); - if (outcome.IsSuccess()) - { - LOG_DEBUG(log, "Created bucket {}", bucket); - } - else if (outcome.GetError().GetExceptionName() == "BucketAlreadyOwnedByYou") - { - LOG_DEBUG(log, "Bucket {} already exist", bucket); - } - else - { - const auto & err = outcome.GetError(); - LOG_ERROR(log, "CreateBucket: {}:{}", err.GetExceptionName(), err.GetMessage()); - } - return outcome.IsSuccess() || outcome.GetError().GetExceptionName() == "BucketAlreadyOwnedByYou"; - } void writeFile(const String & key, char value, size_t size, const WriteSettings & write_setting) { Stopwatch sw; - S3WritableFile file(s3_client, bucket, key, write_setting); + S3WritableFile file(s3_client, key, write_setting); size_t write_size = 0; constexpr size_t buf_size = 1024 * 1024 * 10; String buf_unit(buf_size, value); @@ -150,7 +132,7 @@ class FileCacheTest : public ::testing::Test writeFile(key, value, size, WriteSettings{}); }); upload_results.push_back(task->get_future()); - IOThreadPool::get().scheduleOrThrowOnError([task]() { (*task)(); }); + S3FileCachePool::get().scheduleOrThrowOnError([task]() { (*task)(); }); objects.emplace_back(ObjectInfo{.key = key, .value = value, .size = size}); } for (auto & f : upload_results) @@ -202,11 +184,33 @@ class FileCacheTest : public ::testing::Test LOG_DEBUG(log, "Download summary: succ={} fail={} cost={}s", file_cache.bg_download_succ_count.load(std::memory_order_relaxed), file_cache.bg_download_fail_count.load(std::memory_order_relaxed), sw.elapsedSeconds()); } + static void calculateCacheCapacity(StorageRemoteCacheConfig & config, UInt64 dt_size) + { + config.capacity = dt_size / (1.0 - config.delta_rate); + bool forward = false; + bool backward = false; + while (config.getDTFileCapacity() != dt_size) + { + if (config.getDTFileCapacity() > dt_size) + { + backward = true; + config.capacity--; + } + else + { + forward = true; + config.capacity++; + } + ASSERT_FALSE(forward && backward) << fmt::format("delta_rate {} dt_size {}", config.delta_rate, dt_size); + } + } + String tmp_dir; UInt64 cache_capacity = 100 * 1024 * 1024; UInt64 cache_level = 5; UInt64 cache_min_age_seconds = 30 * 60; LoggerPtr log; + PathCapacityMetricsPtr capacity_metrics; }; TEST_F(FileCacheTest, Main) @@ -218,10 +222,13 @@ try LOG_DEBUG(log, "genObjects: count={} total_size={} cost={}s", objects.size(), total_size, sw.elapsedSeconds()); auto cache_dir = fmt::format("{}/file_cache_all", tmp_dir); + StorageRemoteCacheConfig cache_config{.dir = cache_dir, .dtfile_level = 100}; + calculateCacheCapacity(cache_config, total_size); + LOG_DEBUG(log, "total_size={} dt_cache_capacity={}", total_size, cache_config.getDTFileCapacity()); { LOG_DEBUG(log, "Cache all data"); - FileCache file_cache(cache_dir, /*cache_capacity*/ total_size, /*cache_level*/ 100, cache_min_age_seconds); + FileCache file_cache(capacity_metrics, cache_config); for (const auto & obj : objects) { auto s3_fname = ::DB::S3::S3FilenameView::fromKey(obj.key); @@ -245,7 +252,7 @@ try { LOG_DEBUG(log, "Cache restore"); - FileCache file_cache(cache_dir, /*cache_capacity*/ total_size, /*cache_level*/ 100, cache_min_age_seconds); + FileCache file_cache(capacity_metrics, cache_config); ASSERT_EQ(file_cache.cache_used, file_cache.cache_capacity); for (const auto & obj : objects) { @@ -262,7 +269,7 @@ try ASSERT_EQ(meta_objects.size(), 2 * 2 * 2); { LOG_DEBUG(log, "Evict success"); - FileCache file_cache(cache_dir, /*cache_capacity*/ total_size, /*cache_level*/ 100, cache_min_age_seconds); + FileCache file_cache(capacity_metrics, cache_config); ASSERT_LE(file_cache.cache_used, file_cache.cache_capacity); for (const auto & obj : meta_objects) { @@ -284,7 +291,7 @@ try ASSERT_EQ(meta_objects2.size(), 2 * 2 * 2); { LOG_DEBUG(log, "Evict failed"); - FileCache file_cache(cache_dir, /*cache_capacity*/ total_size, /*cache_level*/ 100, cache_min_age_seconds); + FileCache file_cache(capacity_metrics, cache_config); ASSERT_LE(file_cache.cache_used, file_cache.cache_capacity); UInt64 free_size = file_cache.cache_capacity - file_cache.cache_used; auto file_seg = file_cache.getAll(); // Prevent file_segment from evicted. @@ -319,7 +326,8 @@ CATCH TEST_F(FileCacheTest, FileSystem) { auto cache_dir = fmt::format("{}/filesystem", tmp_dir); - FileCache file_cache(cache_dir, cache_capacity, cache_level, cache_min_age_seconds); + StorageRemoteCacheConfig cache_config{.dir = cache_dir, .capacity = cache_capacity, .dtfile_level = cache_level}; + FileCache file_cache(capacity_metrics, cache_config); DMFileOID dmfile_oid = {.store_id = 1, .table_id = 2, .file_id = 3}; // s1/data/t_2/dmf_3 @@ -327,7 +335,7 @@ TEST_F(FileCacheTest, FileSystem) ASSERT_EQ(s3_fname, "s1/data/t_2/dmf_3"); auto remote_fname1 = fmt::format("{}/1.dat", s3_fname); auto local_fname1 = file_cache.toLocalFilename(remote_fname1); - ASSERT_EQ(local_fname1, fmt::format("{}/{}", cache_dir, remote_fname1)); + ASSERT_EQ(local_fname1, fmt::format("{}/{}", cache_config.getDTFileCacheDir(), remote_fname1)); auto tmp_remote_fname1 = file_cache.toS3Key(local_fname1); ASSERT_EQ(tmp_remote_fname1, remote_fname1); @@ -369,7 +377,7 @@ TEST_F(FileCacheTest, FileSystem) ASSERT_TRUE(std::filesystem::exists(store)) << store.generic_string(); auto cache_root = store.parent_path(); ASSERT_TRUE(std::filesystem::exists(cache_root)) << cache_root.generic_string(); - ASSERT_EQ(cache_root.generic_string(), cache_dir); + ASSERT_EQ(cache_root.generic_string(), cache_config.getDTFileCacheDir()); file_cache.removeDiskFile(local_fname2); ASSERT_FALSE(std::filesystem::exists(local_file2)) << local_file2.generic_string(); @@ -412,7 +420,8 @@ try { UInt64 cache_level_ = 0; auto cache_dir = fmt::format("{}/filetype{}", tmp_dir, cache_level_); - FileCache file_cache(cache_dir, cache_capacity, cache_level_, cache_min_age_seconds); + StorageRemoteCacheConfig cache_config{.dir = cache_dir, .capacity = cache_capacity, .dtfile_level = cache_level_}; + FileCache file_cache(capacity_metrics, cache_config); ASSERT_FALSE(file_cache.canCache(FileType::Unknow)); ASSERT_FALSE(file_cache.canCache(FileType::Meta)); ASSERT_FALSE(file_cache.canCache(FileType::Index)); @@ -426,7 +435,8 @@ try { UInt64 cache_level_ = 1; auto cache_dir = fmt::format("{}/filetype{}", tmp_dir, cache_level_); - FileCache file_cache(cache_dir, cache_capacity, cache_level_, cache_min_age_seconds); + StorageRemoteCacheConfig cache_config{.dir = cache_dir, .capacity = cache_capacity, .dtfile_level = cache_level_}; + FileCache file_cache(capacity_metrics, cache_config); ASSERT_FALSE(file_cache.canCache(FileType::Unknow)); ASSERT_TRUE(file_cache.canCache(FileType::Meta)); ASSERT_FALSE(file_cache.canCache(FileType::Index)); @@ -440,7 +450,8 @@ try { UInt64 cache_level_ = 2; auto cache_dir = fmt::format("{}/filetype{}", tmp_dir, cache_level_); - FileCache file_cache(cache_dir, cache_capacity, cache_level_, cache_min_age_seconds); + StorageRemoteCacheConfig cache_config{.dir = cache_dir, .capacity = cache_capacity, .dtfile_level = cache_level_}; + FileCache file_cache(capacity_metrics, cache_config); ASSERT_FALSE(file_cache.canCache(FileType::Unknow)); ASSERT_TRUE(file_cache.canCache(FileType::Meta)); ASSERT_TRUE(file_cache.canCache(FileType::Index)); @@ -454,7 +465,8 @@ try { UInt64 cache_level_ = 3; auto cache_dir = fmt::format("{}/filetype{}", tmp_dir, cache_level_); - FileCache file_cache(cache_dir, cache_capacity, cache_level_, cache_min_age_seconds); + StorageRemoteCacheConfig cache_config{.dir = cache_dir, .capacity = cache_capacity, .dtfile_level = cache_level_}; + FileCache file_cache(capacity_metrics, cache_config); ASSERT_FALSE(file_cache.canCache(FileType::Unknow)); ASSERT_TRUE(file_cache.canCache(FileType::Meta)); ASSERT_TRUE(file_cache.canCache(FileType::Index)); @@ -468,7 +480,8 @@ try { UInt64 cache_level_ = 4; auto cache_dir = fmt::format("{}/filetype{}", tmp_dir, cache_level_); - FileCache file_cache(cache_dir, cache_capacity, cache_level_, cache_min_age_seconds); + StorageRemoteCacheConfig cache_config{.dir = cache_dir, .capacity = cache_capacity, .dtfile_level = cache_level_}; + FileCache file_cache(capacity_metrics, cache_config); ASSERT_FALSE(file_cache.canCache(FileType::Unknow)); ASSERT_TRUE(file_cache.canCache(FileType::Meta)); ASSERT_TRUE(file_cache.canCache(FileType::Index)); @@ -482,7 +495,8 @@ try { UInt64 cache_level_ = 5; auto cache_dir = fmt::format("{}/filetype{}", tmp_dir, cache_level_); - FileCache file_cache(cache_dir, cache_capacity, cache_level_, cache_min_age_seconds); + StorageRemoteCacheConfig cache_config{.dir = cache_dir, .capacity = cache_capacity, .dtfile_level = cache_level_}; + FileCache file_cache(capacity_metrics, cache_config); ASSERT_FALSE(file_cache.canCache(FileType::Unknow)); ASSERT_TRUE(file_cache.canCache(FileType::Meta)); ASSERT_TRUE(file_cache.canCache(FileType::Index)); @@ -496,7 +510,8 @@ try { UInt64 cache_level_ = 6; auto cache_dir = fmt::format("{}/filetype{}", tmp_dir, cache_level_); - FileCache file_cache(cache_dir, cache_capacity, cache_level_, cache_min_age_seconds); + StorageRemoteCacheConfig cache_config{.dir = cache_dir, .capacity = cache_capacity, .dtfile_level = cache_level_}; + FileCache file_cache(capacity_metrics, cache_config); ASSERT_FALSE(file_cache.canCache(FileType::Unknow)); ASSERT_TRUE(file_cache.canCache(FileType::Meta)); ASSERT_TRUE(file_cache.canCache(FileType::Index)); @@ -510,7 +525,8 @@ try { UInt64 cache_level_ = 7; auto cache_dir = fmt::format("{}/filetype{}", tmp_dir, cache_level_); - FileCache file_cache(cache_dir, cache_capacity, cache_level_, cache_min_age_seconds); + StorageRemoteCacheConfig cache_config{.dir = cache_dir, .capacity = cache_capacity, .dtfile_level = cache_level_}; + FileCache file_cache(capacity_metrics, cache_config); ASSERT_FALSE(file_cache.canCache(FileType::Unknow)); ASSERT_TRUE(file_cache.canCache(FileType::Meta)); ASSERT_TRUE(file_cache.canCache(FileType::Index)); @@ -524,7 +540,8 @@ try { UInt64 cache_level_ = 8; auto cache_dir = fmt::format("{}/filetype{}", tmp_dir, cache_level_); - FileCache file_cache(cache_dir, cache_capacity, cache_level_, cache_min_age_seconds); + StorageRemoteCacheConfig cache_config{.dir = cache_dir, .capacity = cache_capacity, .dtfile_level = cache_level_}; + FileCache file_cache(capacity_metrics, cache_config); ASSERT_FALSE(file_cache.canCache(FileType::Unknow)); ASSERT_TRUE(file_cache.canCache(FileType::Meta)); ASSERT_TRUE(file_cache.canCache(FileType::Index)); @@ -541,8 +558,10 @@ CATCH TEST_F(FileCacheTest, Space) { auto cache_dir = fmt::format("{}/space", tmp_dir); - FileCache file_cache(cache_dir, cache_capacity, cache_level, cache_min_age_seconds); - ASSERT_TRUE(file_cache.reserveSpace(FileType::Meta, cache_capacity - 1024, /*try_evict*/ false)); + StorageRemoteCacheConfig cache_config{.dir = cache_dir, .capacity = cache_capacity, .dtfile_level = cache_level}; + FileCache file_cache(capacity_metrics, cache_config); + auto dt_cache_capacity = cache_config.getDTFileCapacity(); + ASSERT_TRUE(file_cache.reserveSpace(FileType::Meta, dt_cache_capacity - 1024, /*try_evict*/ false)); ASSERT_TRUE(file_cache.reserveSpace(FileType::Meta, 512, /*try_evict*/ false)); ASSERT_TRUE(file_cache.reserveSpace(FileType::Meta, 256, /*try_evict*/ false)); ASSERT_TRUE(file_cache.reserveSpace(FileType::Meta, 256, /*try_evict*/ false)); @@ -551,8 +570,8 @@ TEST_F(FileCacheTest, Space) ASSERT_TRUE(file_cache.finalizeReservedSize(FileType::Meta, /*reserved_size*/ 512, /*content_length*/ 511)); ASSERT_TRUE(file_cache.reserveSpace(FileType::Meta, 1, /*try_evict*/ false)); ASSERT_FALSE(file_cache.reserveSpace(FileType::Meta, 1, /*try_evict*/ false)); - file_cache.releaseSpace(cache_capacity); - ASSERT_TRUE(file_cache.reserveSpace(FileType::Meta, cache_capacity, /*try_evict*/ false)); + file_cache.releaseSpace(dt_cache_capacity); + ASSERT_TRUE(file_cache.reserveSpace(FileType::Meta, dt_cache_capacity, /*try_evict*/ false)); ASSERT_FALSE(file_cache.reserveSpace(FileType::Meta, 1, /*try_evict*/ false)); } @@ -633,4 +652,4 @@ TEST_F(FileCacheTest, LRUFileTable) ASSERT_EQ(table.begin(), table.end()); } } -} // namespace DB::tests::S3 \ No newline at end of file +} // namespace DB::tests::S3 diff --git a/dbms/src/Storages/S3/tests/gtest_s3client.cpp b/dbms/src/Storages/S3/tests/gtest_s3client.cpp new file mode 100644 index 00000000000..948777d8ba1 --- /dev/null +++ b/dbms/src/Storages/S3/tests/gtest_s3client.cpp @@ -0,0 +1,82 @@ +// Copyright 2023 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 +#include +#include +#include +#include + +namespace DB::S3::tests +{ + +class S3ClientTest : public ::testing::Test +{ +public: + void SetUp() override + { + client = ClientFactory::instance().sharedTiFlashClient(); + ::DB::tests::TiFlashTestEnv::deleteBucket(*client); + ::DB::tests::TiFlashTestEnv::createBucketIfNotExist(*client); + } + + std::shared_ptr client; +}; + +TEST_F(S3ClientTest, UploadRead) +try +{ + deleteObject(*client, "s999/manifest/mf_1"); + ASSERT_FALSE(objectExists(*client, "s999/manifest/mf_1")); + uploadEmptyFile(*client, "s999/manifest/mf_1"); + ASSERT_TRUE(objectExists(*client, "s999/manifest/mf_1")); + + uploadEmptyFile(*client, "s999/manifest/mf_2"); + uploadEmptyFile(*client, "s999/manifest/mf_789"); + + uploadEmptyFile(*client, "s999/data/dat_789_0"); + uploadEmptyFile(*client, "s999/data/dat_790_0"); + + uploadEmptyFile(*client, "s999/abcd"); + + { + Strings prefixes; + listPrefixWithDelimiter(*client, "s999/", "/", [&](const Aws::S3::Model::CommonPrefix & p) { + prefixes.emplace_back(p.GetPrefix()); + return PageResult{.num_keys = 1, .more = true}; + }); + ASSERT_EQ(prefixes.size(), 2) << fmt::format("{}", prefixes); + EXPECT_EQ(prefixes[0], "s999/data/"); + EXPECT_EQ(prefixes[1], "s999/manifest/"); + } + + // check the keys with raw `LIST` request + { + Strings prefixes; + rawListPrefix(*client, client->bucket(), client->root() + "s999/", "/", [&](const Aws::S3::Model::ListObjectsV2Result & result) { + const auto & ps = result.GetCommonPrefixes(); + for (const auto & p : ps) + { + prefixes.emplace_back(p.GetPrefix()); + } + return PageResult{.num_keys = ps.size(), .more = true}; + }); + ASSERT_EQ(prefixes.size(), 2) << fmt::format("{}", prefixes); + EXPECT_EQ(prefixes[0], client->root() + "s999/data/"); + EXPECT_EQ(prefixes[1], client->root() + "s999/manifest/"); + } +} +CATCH + +} // namespace DB::S3::tests diff --git a/dbms/src/Storages/S3/tests/gtest_s3file.cpp b/dbms/src/Storages/S3/tests/gtest_s3file.cpp index 7bc44648cf2..fb436baa8ec 100644 --- a/dbms/src/Storages/S3/tests/gtest_s3file.cpp +++ b/dbms/src/Storages/S3/tests/gtest_s3file.cpp @@ -67,10 +67,9 @@ class S3FileTest : public DB::base::TiFlashStorageTestBasic buf_unit.resize(256); std::iota(buf_unit.begin(), buf_unit.end(), 0); - s3_client = S3::ClientFactory::instance().sharedClient(); - bucket = S3::ClientFactory::instance().bucket(); + s3_client = S3::ClientFactory::instance().sharedTiFlashClient(); data_store = std::make_shared(dbContext().getFileProvider()); - ASSERT_TRUE(::DB::tests::TiFlashTestEnv::createBucketIfNotExist(*s3_client, bucket)); + ASSERT_TRUE(::DB::tests::TiFlashTestEnv::createBucketIfNotExist(*s3_client)); } void reload() @@ -83,7 +82,7 @@ class S3FileTest : public DB::base::TiFlashStorageTestBasic protected: void writeFile(const String & key, size_t size, const WriteSettings & write_setting) { - S3WritableFile file(s3_client, bucket, key, write_setting); + S3WritableFile file(s3_client, key, write_setting); size_t write_size = 0; while (write_size < size) { @@ -99,7 +98,7 @@ class S3FileTest : public DB::base::TiFlashStorageTestBasic void verifyFile(const String & key, size_t size) { - S3RandomAccessFile file(s3_client, bucket, key); + S3RandomAccessFile file(s3_client, key); std::vector tmp_buf; size_t read_size = 0; while (read_size < size) @@ -152,7 +151,7 @@ class S3FileTest : public DB::base::TiFlashStorageTestBasic S3::S3Filename::fromTableID(oid.store_id, oid.table_id).toFullKey(), oid.file_id, DMFile::Status::READABLE); - return S3::listPrefixWithSize(*s3_client, bucket, dmfile_dir + "/"); + return S3::listPrefixWithSize(*s3_client, dmfile_dir + "/"); } DMFilePtr restoreDMFile(const DMFileOID & oid) @@ -162,8 +161,7 @@ class S3FileTest : public DB::base::TiFlashStorageTestBasic LoggerPtr log; std::vector buf_unit; - std::shared_ptr s3_client; - String bucket; + std::shared_ptr s3_client; S3WritableFile::UploadInfo last_upload_info; Remote::IDataStorePtr data_store; }; @@ -209,7 +207,7 @@ try WriteSettings write_setting; const String key = "/a/b/c/seek"; writeFile(key, size, write_setting); - S3RandomAccessFile file(s3_client, bucket, key); + S3RandomAccessFile file(s3_client, key); { std::vector tmp_buf(256); auto n = file.read(tmp_buf.data(), tmp_buf.size()); @@ -430,15 +428,14 @@ try data_store->putCheckpointFiles(checkpoint, store_id, sequence); // ensure CheckpointDataFile, theirs lock file and CheckpointManifest are uploaded - auto s3client = S3::ClientFactory::instance().sharedClient(); - auto s3bucket = S3::ClientFactory::instance().bucket(); + auto s3client = S3::ClientFactory::instance().sharedTiFlashClient(); auto cp_data0 = S3::S3Filename::newCheckpointData(store_id, sequence, 0); - ASSERT_TRUE(S3::objectExists(*s3client, s3bucket, cp_data0.toFullKey())); - ASSERT_TRUE(S3::objectExists(*s3client, s3bucket, cp_data0.toView().getLockKey(store_id, sequence))); + ASSERT_TRUE(S3::objectExists(*s3client, cp_data0.toFullKey())); + ASSERT_TRUE(S3::objectExists(*s3client, cp_data0.toView().getLockKey(store_id, sequence))); auto cp_data1 = S3::S3Filename::newCheckpointData(store_id, sequence, 1); - ASSERT_TRUE(S3::objectExists(*s3client, s3bucket, cp_data1.toFullKey())); - ASSERT_TRUE(S3::objectExists(*s3client, s3bucket, cp_data1.toView().getLockKey(store_id, sequence))); - ASSERT_TRUE(S3::objectExists(*s3client, s3bucket, S3::S3Filename::newCheckpointManifest(store_id, sequence).toFullKey())); + ASSERT_TRUE(S3::objectExists(*s3client, cp_data1.toFullKey())); + ASSERT_TRUE(S3::objectExists(*s3client, cp_data1.toView().getLockKey(store_id, sequence))); + ASSERT_TRUE(S3::objectExists(*s3client, S3::S3Filename::newCheckpointManifest(store_id, sequence).toFullKey())); } CATCH diff --git a/dbms/src/Storages/S3/tests/gtest_s3gcmanager.cpp b/dbms/src/Storages/S3/tests/gtest_s3gcmanager.cpp index f3ba7024c38..a8244b7e7e8 100644 --- a/dbms/src/Storages/S3/tests/gtest_s3gcmanager.cpp +++ b/dbms/src/Storages/S3/tests/gtest_s3gcmanager.cpp @@ -44,7 +44,6 @@ extern const char force_set_mocked_s3_object_mtime[]; } // namespace DB::FailPoints namespace DB::S3::tests { - class S3GCManagerTest : public DB::base::TiFlashStorageTestBasic { public: @@ -70,9 +69,9 @@ class S3GCManagerTest : public DB::base::TiFlashStorageTestBasic auto mock_gc_owner = OwnerManager::createMockOwner("owner_0"); auto mock_lock_client = std::make_shared(mock_s3_client); auto mock_pd_client = std::make_shared(); - gc_mgr = std::make_unique(mock_pd_client, mock_s3_client, mock_gc_owner, mock_lock_client, config); + gc_mgr = std::make_unique(mock_pd_client, mock_gc_owner, mock_lock_client, config); - ::DB::tests::TiFlashTestEnv::createBucketIfNotExist(*mock_s3_client, mock_s3_client->bucket()); + ::DB::tests::TiFlashTestEnv::createBucketIfNotExist(*mock_s3_client); dir = getTemporaryPath(); dropDataOnDisk(dir); @@ -81,7 +80,7 @@ class S3GCManagerTest : public DB::base::TiFlashStorageTestBasic void TearDown() override { - ::DB::tests::TiFlashTestEnv::deleteBucket(*mock_s3_client, mock_s3_client->bucket()); + ::DB::tests::TiFlashTestEnv::deleteBucket(*mock_s3_client); } protected: @@ -124,7 +123,7 @@ try for (const auto & [seq, diff_sec] : mfs) { auto key = S3Filename::newCheckpointManifest(store_id, seq).toFullKey(); - uploadEmptyFile(*mock_s3_client, mock_s3_client->bucket(), key); + uploadEmptyFile(*mock_s3_client, key); objs.emplace_back(CheckpointManifestS3Object{ .key = key, .last_modification = timepoint + std::chrono::milliseconds(diff_sec * 1000), @@ -175,11 +174,11 @@ try if (seq == 4 || seq == 5) { // deleted - ASSERT_FALSE(S3::objectExists(*mock_s3_client, mock_s3_client->bucket(), obj.key)); + ASSERT_FALSE(S3::objectExists(*mock_s3_client, obj.key)); } else { - ASSERT_TRUE(S3::objectExists(*mock_s3_client, mock_s3_client->bucket(), obj.key)); + ASSERT_TRUE(S3::objectExists(*mock_s3_client, obj.key)); } } } @@ -191,28 +190,28 @@ try { auto timepoint = Aws::Utils::DateTime("2023-02-01T08:00:00Z", Aws::Utils::DateFormat::ISO_8601); { - uploadEmptyFile(*mock_s3_client, mock_s3_client->bucket(), "datafile_key"); - uploadEmptyFile(*mock_s3_client, mock_s3_client->bucket(), "datafile_key.del"); + uploadEmptyFile(*mock_s3_client, "datafile_key"); + uploadEmptyFile(*mock_s3_client, "datafile_key.del"); // delmark expired auto delmark_mtime = timepoint - std::chrono::milliseconds(3601 * 1000); gc_mgr->removeDataFileIfDelmarkExpired("datafile_key", "datafile_key.del", timepoint, delmark_mtime); // removed - ASSERT_FALSE(S3::objectExists(*mock_s3_client, mock_s3_client->bucket(), "datafile_key")); - ASSERT_FALSE(S3::objectExists(*mock_s3_client, mock_s3_client->bucket(), "datafile_key.del")); + ASSERT_FALSE(S3::objectExists(*mock_s3_client, "datafile_key")); + ASSERT_FALSE(S3::objectExists(*mock_s3_client, "datafile_key.del")); } { - uploadEmptyFile(*mock_s3_client, mock_s3_client->bucket(), "datafile_key"); - uploadEmptyFile(*mock_s3_client, mock_s3_client->bucket(), "datafile_key.del"); + uploadEmptyFile(*mock_s3_client, "datafile_key"); + uploadEmptyFile(*mock_s3_client, "datafile_key.del"); // delmark not expired auto delmark_mtime = timepoint - std::chrono::milliseconds(3599 * 1000); gc_mgr->removeDataFileIfDelmarkExpired("datafile_key", "datafile_key.del", timepoint, delmark_mtime); // removed - ASSERT_TRUE(S3::objectExists(*mock_s3_client, mock_s3_client->bucket(), "datafile_key")); - ASSERT_TRUE(S3::objectExists(*mock_s3_client, mock_s3_client->bucket(), "datafile_key.del")); + ASSERT_TRUE(S3::objectExists(*mock_s3_client, "datafile_key")); + ASSERT_TRUE(S3::objectExists(*mock_s3_client, "datafile_key.del")); } } CATCH @@ -230,69 +229,69 @@ try auto timepoint = Aws::Utils::DateTime("2023-02-01T08:00:00Z", Aws::Utils::DateFormat::ISO_8601); auto clear_bucket = [&] { - DB::tests::TiFlashTestEnv::deleteBucket(*mock_s3_client, mock_s3_client->bucket()); - DB::tests::TiFlashTestEnv::createBucketIfNotExist(*mock_s3_client, mock_s3_client->bucket()); + DB::tests::TiFlashTestEnv::deleteBucket(*mock_s3_client); + DB::tests::TiFlashTestEnv::createBucketIfNotExist(*mock_s3_client); }; { clear_bucket(); // delmark not exist, and no more lockfile - S3::uploadEmptyFile(*mock_s3_client, mock_s3_client->bucket(), df.toFullKey()); - S3::uploadEmptyFile(*mock_s3_client, mock_s3_client->bucket(), lock_key); + S3::uploadEmptyFile(*mock_s3_client, df.toFullKey()); + S3::uploadEmptyFile(*mock_s3_client, lock_key); - ASSERT_FALSE(S3::objectExists(*mock_s3_client, mock_s3_client->bucket(), delmark_key)); + ASSERT_FALSE(S3::objectExists(*mock_s3_client, delmark_key)); gc_mgr->cleanOneLock(lock_key, lock_view, timepoint); // lock is deleted and delmark is created - ASSERT_FALSE(S3::objectExists(*mock_s3_client, mock_s3_client->bucket(), lock_key)); - ASSERT_TRUE(S3::objectExists(*mock_s3_client, mock_s3_client->bucket(), delmark_key)); - ASSERT_TRUE(S3::objectExists(*mock_s3_client, mock_s3_client->bucket(), df.toFullKey())); + ASSERT_FALSE(S3::objectExists(*mock_s3_client, lock_key)); + ASSERT_TRUE(S3::objectExists(*mock_s3_client, delmark_key)); + ASSERT_TRUE(S3::objectExists(*mock_s3_client, df.toFullKey())); } { clear_bucket(); // delmark not exist, but still locked by another lockfile - S3::uploadEmptyFile(*mock_s3_client, mock_s3_client->bucket(), df.toFullKey()); - S3::uploadEmptyFile(*mock_s3_client, mock_s3_client->bucket(), lock_key); + S3::uploadEmptyFile(*mock_s3_client, df.toFullKey()); + S3::uploadEmptyFile(*mock_s3_client, lock_key); // another lock auto another_lock_key = df.toView().getLockKey(store_id + 1, 450); - S3::uploadEmptyFile(*mock_s3_client, mock_s3_client->bucket(), another_lock_key); + S3::uploadEmptyFile(*mock_s3_client, another_lock_key); gc_mgr->cleanOneLock(lock_key, lock_view, timepoint); // lock is deleted but delmark is not created - ASSERT_FALSE(S3::objectExists(*mock_s3_client, mock_s3_client->bucket(), lock_key)); - ASSERT_FALSE(S3::objectExists(*mock_s3_client, mock_s3_client->bucket(), delmark_key)); - ASSERT_TRUE(S3::objectExists(*mock_s3_client, mock_s3_client->bucket(), another_lock_key)); - ASSERT_TRUE(S3::objectExists(*mock_s3_client, mock_s3_client->bucket(), df.toFullKey())); + ASSERT_FALSE(S3::objectExists(*mock_s3_client, lock_key)); + ASSERT_FALSE(S3::objectExists(*mock_s3_client, delmark_key)); + ASSERT_TRUE(S3::objectExists(*mock_s3_client, another_lock_key)); + ASSERT_TRUE(S3::objectExists(*mock_s3_client, df.toFullKey())); } { clear_bucket(); // delmark exist, not expired - S3::uploadEmptyFile(*mock_s3_client, mock_s3_client->bucket(), df.toFullKey()); - S3::uploadEmptyFile(*mock_s3_client, mock_s3_client->bucket(), delmark_key); + S3::uploadEmptyFile(*mock_s3_client, df.toFullKey()); + S3::uploadEmptyFile(*mock_s3_client, delmark_key); auto delmark_mtime = timepoint - std::chrono::milliseconds(3599 * 1000); - FailPointHelper::enableFailPoint(FailPoints::force_set_mocked_s3_object_mtime, std::map{{delmark_key, delmark_mtime}}); + FailPointHelper::enableFailPoint(FailPoints::force_set_mocked_s3_object_mtime, std::map{{mock_s3_client->root() + delmark_key, delmark_mtime}}); // mock_s3_client->head_result_mtime = delmark_mtime; gc_mgr->cleanOneLock(lock_key, lock_view, timepoint); // lock is deleted, datafile and delmark remain - ASSERT_FALSE(S3::objectExists(*mock_s3_client, mock_s3_client->bucket(), lock_key)); - ASSERT_TRUE(S3::objectExists(*mock_s3_client, mock_s3_client->bucket(), delmark_key)); - ASSERT_TRUE(S3::objectExists(*mock_s3_client, mock_s3_client->bucket(), df.toFullKey())); + ASSERT_FALSE(S3::objectExists(*mock_s3_client, lock_key)); + ASSERT_TRUE(S3::objectExists(*mock_s3_client, delmark_key)); + ASSERT_TRUE(S3::objectExists(*mock_s3_client, df.toFullKey())); } { clear_bucket(); // delmark exist, expired - S3::uploadEmptyFile(*mock_s3_client, mock_s3_client->bucket(), df.toFullKey()); - S3::uploadEmptyFile(*mock_s3_client, mock_s3_client->bucket(), delmark_key); + S3::uploadEmptyFile(*mock_s3_client, df.toFullKey()); + S3::uploadEmptyFile(*mock_s3_client, delmark_key); auto delmark_mtime = timepoint - std::chrono::milliseconds(3601 * 1000); - FailPointHelper::enableFailPoint(FailPoints::force_set_mocked_s3_object_mtime, std::map{{delmark_key, delmark_mtime}}); + FailPointHelper::enableFailPoint(FailPoints::force_set_mocked_s3_object_mtime, std::map{{mock_s3_client->root() + delmark_key, delmark_mtime}}); gc_mgr->cleanOneLock(lock_key, lock_view, timepoint); // lock datafile and delmark are deleted - ASSERT_FALSE(S3::objectExists(*mock_s3_client, mock_s3_client->bucket(), lock_key)); - ASSERT_FALSE(S3::objectExists(*mock_s3_client, mock_s3_client->bucket(), delmark_key)); - ASSERT_FALSE(S3::objectExists(*mock_s3_client, mock_s3_client->bucket(), df.toFullKey())); + ASSERT_FALSE(S3::objectExists(*mock_s3_client, lock_key)); + ASSERT_FALSE(S3::objectExists(*mock_s3_client, delmark_key)); + ASSERT_FALSE(S3::objectExists(*mock_s3_client, df.toFullKey())); } } CATCH @@ -311,28 +310,29 @@ try auto timepoint = Aws::Utils::DateTime("2023-02-01T08:00:00Z", Aws::Utils::DateFormat::ISO_8601); auto clear_bucket = [&] { - DB::tests::TiFlashTestEnv::deleteBucket(*mock_s3_client, mock_s3_client->bucket()); - DB::tests::TiFlashTestEnv::createBucketIfNotExist(*mock_s3_client, mock_s3_client->bucket()); + DB::tests::TiFlashTestEnv::deleteBucket(*mock_s3_client); + DB::tests::TiFlashTestEnv::createBucketIfNotExist(*mock_s3_client); }; { clear_bucket(); // delmark not exist, and no more lockfile - S3::uploadEmptyFile(*mock_s3_client, mock_s3_client->bucket(), df.toFullKey()); - S3::uploadEmptyFile(*mock_s3_client, mock_s3_client->bucket(), lock_key); + S3::uploadEmptyFile(*mock_s3_client, df.toFullKey()); + S3::uploadEmptyFile(*mock_s3_client, lock_key); - ASSERT_FALSE(S3::objectExists(*mock_s3_client, mock_s3_client->bucket(), delmark_key)); + ASSERT_FALSE(S3::objectExists(*mock_s3_client, delmark_key)); gc_mgr->cleanOneLock(lock_key, lock_view, timepoint); // lock is deleted and delmark is created, object is rewrite with tagging - ASSERT_FALSE(S3::objectExists(*mock_s3_client, mock_s3_client->bucket(), lock_key)); - ASSERT_TRUE(S3::objectExists(*mock_s3_client, mock_s3_client->bucket(), delmark_key)); - ASSERT_TRUE(S3::objectExists(*mock_s3_client, mock_s3_client->bucket(), df.toFullKey())); - const auto res = static_cast(mock_s3_client.get())->GetObjectTagging( // - Aws::S3::Model::GetObjectTaggingRequest() // - .WithBucket(mock_s3_client->bucket()) - .WithKey(df.toFullKey())); + ASSERT_FALSE(S3::objectExists(*mock_s3_client, lock_key)); + ASSERT_TRUE(S3::objectExists(*mock_s3_client, delmark_key)); + ASSERT_TRUE(S3::objectExists(*mock_s3_client, df.toFullKey())); + + auto req = Aws::S3::Model::GetObjectTaggingRequest(); // + mock_s3_client->setBucketAndKeyWithRoot(req, df.toFullKey()); + const auto res = mock_s3_client->GetObjectTagging(req); + auto tags = res.GetResult().GetTagSet(); ASSERT_EQ(tags.size(), 1); EXPECT_EQ(tags[0].GetKey(), "tiflash_deleted"); @@ -341,18 +341,18 @@ try { clear_bucket(); // delmark not exist, but still locked by another lockfile - S3::uploadEmptyFile(*mock_s3_client, mock_s3_client->bucket(), df.toFullKey()); - S3::uploadEmptyFile(*mock_s3_client, mock_s3_client->bucket(), lock_key); + S3::uploadEmptyFile(*mock_s3_client, df.toFullKey()); + S3::uploadEmptyFile(*mock_s3_client, lock_key); // another lock auto another_lock_key = df.toView().getLockKey(store_id + 1, 450); - S3::uploadEmptyFile(*mock_s3_client, mock_s3_client->bucket(), another_lock_key); + S3::uploadEmptyFile(*mock_s3_client, another_lock_key); gc_mgr->cleanOneLock(lock_key, lock_view, timepoint); // lock is deleted but delmark is not created - ASSERT_FALSE(S3::objectExists(*mock_s3_client, mock_s3_client->bucket(), lock_key)); - ASSERT_FALSE(S3::objectExists(*mock_s3_client, mock_s3_client->bucket(), delmark_key)); - ASSERT_TRUE(S3::objectExists(*mock_s3_client, mock_s3_client->bucket(), another_lock_key)); - ASSERT_TRUE(S3::objectExists(*mock_s3_client, mock_s3_client->bucket(), df.toFullKey())); + ASSERT_FALSE(S3::objectExists(*mock_s3_client, lock_key)); + ASSERT_FALSE(S3::objectExists(*mock_s3_client, delmark_key)); + ASSERT_TRUE(S3::objectExists(*mock_s3_client, another_lock_key)); + ASSERT_TRUE(S3::objectExists(*mock_s3_client, df.toFullKey())); } } CATCH @@ -373,22 +373,26 @@ try { // not managed by lock_store_id auto df = S3Filename::newCheckpointData(store_id, 300, 1); + keys.emplace_back(df.toFullKey()); auto lock_key = df.toView().getLockKey(store_id, safe_sequence + 1); keys.emplace_back(lock_key); // not managed by the latest manifest yet df = S3Filename::newCheckpointData(store_id, 300, 1); + keys.emplace_back(df.toFullKey()); lock_key = df.toView().getLockKey(lock_store_id, safe_sequence + 1); keys.emplace_back(lock_key); // still valid in latest manifest df = S3Filename::newCheckpointData(store_id, 300, 1); + keys.emplace_back(df.toFullKey()); lock_key = df.toView().getLockKey(lock_store_id, safe_sequence - 1); valid_lock_files.emplace(lock_key); keys.emplace_back(lock_key); // not valid in latest manfiest, should be delete df = S3Filename::newCheckpointData(store_id, 300, 2); + keys.emplace_back(df.toFullKey()); lock_key = df.toView().getLockKey(lock_store_id, safe_sequence - 1); expected_deleted_lock_key = lock_key; expected_created_delmark = df.toView().getDelMarkKey(); @@ -398,7 +402,7 @@ try // prepare for `LIST` for (const auto & k : keys) { - uploadEmptyFile(*mock_s3_client, mock_s3_client->bucket(), k); + uploadEmptyFile(*mock_s3_client, k); } } @@ -407,8 +411,8 @@ try gc_mgr->cleanUnusedLocks(lock_store_id, S3Filename::getLockPrefix(), safe_sequence, valid_lock_files, timepoint); // lock is deleted and delmark is created - ASSERT_FALSE(S3::objectExists(*mock_s3_client, mock_s3_client->bucket(), expected_deleted_lock_key)); - ASSERT_TRUE(S3::objectExists(*mock_s3_client, mock_s3_client->bucket(), expected_created_delmark)); + ASSERT_FALSE(S3::objectExists(*mock_s3_client, expected_deleted_lock_key)); + ASSERT_TRUE(S3::objectExists(*mock_s3_client, expected_created_delmark)); } } CATCH @@ -472,7 +476,7 @@ try writer->writeSuffix(); writer.reset(); - S3::uploadFile(*mock_s3_client, mock_s3_client->bucket(), dir + "/" + mf_key, mf_key); + S3::uploadFile(*mock_s3_client, dir + "/" + mf_key, mf_key); } { // prepare the second manifest on S3 const String entry_data = "cherry_value"; @@ -504,7 +508,7 @@ try writer->writeSuffix(); writer.reset(); - S3::uploadFile(*mock_s3_client, mock_s3_client->bucket(), dir + "/" + mf_key2, mf_key2); + S3::uploadFile(*mock_s3_client, dir + "/" + mf_key2, mf_key2); } // read from S3 key diff --git a/dbms/src/Storages/StorageDeltaMerge.cpp b/dbms/src/Storages/StorageDeltaMerge.cpp index 9cb158930aa..295e57e6528 100644 --- a/dbms/src/Storages/StorageDeltaMerge.cpp +++ b/dbms/src/Storages/StorageDeltaMerge.cpp @@ -971,7 +971,6 @@ StorageDeltaMerge::writeNodeBuildRemoteReadSnapshot( RUNTIME_CHECK(query_info.mvcc_query_info != nullptr); const auto & mvcc_query_info = *query_info.mvcc_query_info; auto ranges = parseMvccQueryInfo(mvcc_query_info, num_streams, context, query_info.req_id, tracing_logger); - const auto & scan_context = mvcc_query_info.scan_context; auto read_segments = parseSegmentSet(select_query.segment_expression_list); auto snap = store->writeNodeBuildRemoteReadSnapshot( @@ -981,7 +980,7 @@ StorageDeltaMerge::writeNodeBuildRemoteReadSnapshot( num_streams, query_info.req_id, read_segments, - scan_context); + mvcc_query_info.scan_context); snap->column_defines = std::make_shared(columns_to_read); @@ -1036,6 +1035,18 @@ void StorageDeltaMerge::ingestFiles( clear_data_in_range); } +void StorageDeltaMerge::ingestSegmentsFromCheckpointInfo( + const DM::RowKeyRange & range, + CheckpointInfoPtr checkpoint_info, + const Settings & settings) +{ + return getAndMaybeInitStore()->ingestSegmentsFromCheckpointInfo( + global_context, + settings, + range, + checkpoint_info); +} + UInt64 StorageDeltaMerge::onSyncGc(Int64 limit, const GCOptions & gc_options) { if (storeInited()) diff --git a/dbms/src/Storages/StorageDeltaMerge.h b/dbms/src/Storages/StorageDeltaMerge.h index 0d9a40c79b7..75d32add76d 100644 --- a/dbms/src/Storages/StorageDeltaMerge.h +++ b/dbms/src/Storages/StorageDeltaMerge.h @@ -31,6 +31,8 @@ namespace DB { +struct CheckpointInfo; +using CheckpointInfoPtr = std::shared_ptr; namespace DM { struct RowKeyRange; @@ -112,6 +114,11 @@ class StorageDeltaMerge bool clear_data_in_range, const Settings & settings); + void ingestSegmentsFromCheckpointInfo( + const DM::RowKeyRange & range, + CheckpointInfoPtr checkpoint_info, + const Settings & settings); + UInt64 onSyncGc(Int64, const DM::GCOptions &) override; void rename( diff --git a/dbms/src/Storages/StorageDisaggregatedRemote.cpp b/dbms/src/Storages/StorageDisaggregatedRemote.cpp index cb232347bf8..6deb3496849 100644 --- a/dbms/src/Storages/StorageDisaggregatedRemote.cpp +++ b/dbms/src/Storages/StorageDisaggregatedRemote.cpp @@ -62,7 +62,7 @@ struct RpcTypeTraits { using RequestType = disaggregated::EstablishDisaggTaskRequest; using ResultType = disaggregated::EstablishDisaggTaskResponse; - static const char * err_msg() { return "EstablishDisaggregatedTask Failed"; } // NOLINT(readability-identifier-naming) + static const char * err_msg() { return "EstablishDisaggTask Failed"; } // NOLINT(readability-identifier-naming) static ::grpc::Status doRPCCall( grpc::ClientContext * context, std::shared_ptr client, @@ -79,7 +79,7 @@ namespace DB namespace ErrorCodes { -extern const int REGION_EPOCH_NOT_MATCH; +extern const int DISAGG_ESTABLISH_RETRYABLE_ERROR; } // namespace ErrorCodes BlockInputStreams StorageDisaggregated::readFromWriteNode( @@ -88,7 +88,7 @@ BlockInputStreams StorageDisaggregated::readFromWriteNode( { DM::RNRemoteReadTaskPtr remote_read_tasks; - pingcap::kv::Backoffer bo(pingcap::kv::copBuildTaskMaxBackoff); + pingcap::kv::Backoffer bo(pingcap::kv::scanMaxBackoff); while (true) { // TODO: We could only retry failed stores. @@ -108,11 +108,11 @@ BlockInputStreams StorageDisaggregated::readFromWriteNode( } catch (DB::Exception & e) { - if (e.code() != ErrorCodes::REGION_EPOCH_NOT_MATCH) + if (e.code() != ErrorCodes::DISAGG_ESTABLISH_RETRYABLE_ERROR) throw; - bo.backoff(pingcap::kv::boRegionMiss, pingcap::Exception(e.message())); - LOG_INFO(log, "meets region epoch not match, retry to build remote read tasks"); + bo.backoff(pingcap::kv::boRegionMiss, pingcap::Exception(e.message(), e.code())); + LOG_INFO(log, "Meets retryable error: {}, retry to build remote read tasks", e.message()); } } @@ -163,33 +163,51 @@ DM::RNRemoteReadTaskPtr StorageDisaggregated::buildDisaggregatedTask( if (resp->has_error()) { - LOG_DEBUG( - log, - "Received EstablishDisaggregated response with error, addr={} err={}", - batch_cop_task.store_addr, - resp->error().msg()); - - if (resp->error().code() == ErrorCodes::REGION_EPOCH_NOT_MATCH) + if (resp->error().code() == ErrorCodes::DISAGG_ESTABLISH_RETRYABLE_ERROR) { - // Refresh region cache and throw an exception for retrying + // Refresh region cache and throw an exception for retrying. + // Note: retry_region's region epoch is not set. We need to recover from the request. + + std::unordered_set retry_regions; for (const auto & retry_region : resp->retry_regions()) + retry_regions.insert(retry_region.id()); + + LOG_INFO( + log, + "Received EstablishDisaggregated response with retryable error: {}, addr={} retry_regions={}", + resp->error().msg(), + batch_cop_task.store_addr, + retry_regions); + + for (const auto & region : req->regions()) { - auto region_id = pingcap::kv::RegionVerID( - retry_region.id(), - retry_region.region_epoch().conf_ver(), - retry_region.region_epoch().version()); - cluster->region_cache->dropRegion(region_id); + if (retry_regions.contains(region.region_id())) + { + auto region_ver_id = pingcap::kv::RegionVerID( + region.region_id(), + region.region_epoch().conf_ver(), + region.region_epoch().version()); + cluster->region_cache->dropRegion(region_ver_id); + } } - throw Exception(resp->error().msg(), ErrorCodes::REGION_EPOCH_NOT_MATCH); + throw Exception( + resp->error().msg(), + ErrorCodes::DISAGG_ESTABLISH_RETRYABLE_ERROR); } else { - // Meet other errors... May be not retirable? - throw Exception(ErrorCodes::LOGICAL_ERROR, - "EstablishDisaggregatedTask failed, addr={} error={} code={}", - batch_cop_task.store_addr, - resp->error().msg(), - resp->error().code()); + LOG_WARNING( + log, + "Received EstablishDisaggregated response with error, addr={} err={}", + batch_cop_task.store_addr, + resp->error().msg()); + + // Meet other errors... May be not retryable? + throw Exception( + resp->error().code(), + "EstablishDisaggregatedTask failed: {}, addr={}", + resp->error().msg(), + batch_cop_task.store_addr); } } @@ -227,7 +245,7 @@ DM::RNRemoteReadTaskPtr StorageDisaggregated::buildDisaggregatedTask( LOG_DEBUG( log, - "Build RNRemoteTableReadTask finished, elapsed={}s store={} addr={} segments={} task_id={}", + "Build RNRemoteTableReadTask finished, elapsed={:.3f}s store={} addr={} segments={} task_id={}", watch_table.elapsedSeconds(), resp->store_id(), batch_cop_task.store_addr, @@ -362,7 +380,7 @@ void StorageDisaggregated::buildRemoteSegmentInputStreams( log->identifier(), executor_id); - bool do_prepare = true; + bool do_prepare = false; // Build the input streams to read blocks from remote segments auto [column_defines, extra_table_id_index] = genColumnDefinesForDisaggregatedRead(table_scan); diff --git a/dbms/src/Storages/Transaction/ApplySnapshot.cpp b/dbms/src/Storages/Transaction/ApplySnapshot.cpp index 6d7d88a1426..3a8a8df672d 100644 --- a/dbms/src/Storages/Transaction/ApplySnapshot.cpp +++ b/dbms/src/Storages/Transaction/ApplySnapshot.cpp @@ -180,6 +180,10 @@ void KVStore::onSnapshot(const RegionPtrWrap & new_region_wrap, RegionPtr old_re // Call `ingestFiles` to delete data for range and ingest external DTFiles. dm_storage->ingestFiles(new_key_range, new_region_wrap.external_files, /*clear_data_in_range=*/true, context.getSettingsRef()); } + else if constexpr (std::is_same_v) + { + dm_storage->ingestSegmentsFromCheckpointInfo(new_key_range, new_region_wrap.checkpoint_info, context.getSettingsRef()); + } else { // Call `deleteRange` to delete data for range @@ -478,6 +482,11 @@ void KVStore::handleApplySnapshot( applyPreHandledSnapshot(RegionPtrWithSnapshotFiles{new_region, std::move(external_files)}, tmt); } +void KVStore::handleIngestCheckpoint(CheckpointInfoPtr checkpoint_info, TMTContext & tmt) +{ + applyPreHandledSnapshot(RegionPtrWithCheckpointInfo{checkpoint_info}, tmt); +} + EngineStoreApplyRes KVStore::handleIngestSST(UInt64 region_id, const SSTViewVec snaps, UInt64 index, UInt64 term, TMTContext & tmt) { auto region_task_lock = region_manager.genRegionTaskLock(region_id); diff --git a/dbms/src/Storages/Transaction/BackgroundService.cpp b/dbms/src/Storages/Transaction/BackgroundService.cpp index ef1003a6c4b..4da10f002ec 100644 --- a/dbms/src/Storages/Transaction/BackgroundService.cpp +++ b/dbms/src/Storages/Transaction/BackgroundService.cpp @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +#include #include #include #include @@ -27,19 +28,19 @@ BackgroundService::BackgroundService(TMTContext & tmt_) , background_pool(tmt.getContext().getBackgroundPool()) , log(Logger::get()) { - if (!tmt.isInitialized()) - throw Exception("TMTContext is not initialized", ErrorCodes::LOGICAL_ERROR); - - single_thread_task_handle = background_pool.addTask( - [this] { - tmt.getKVStore()->gcRegionPersistedCache(); - return false; - }, - false); + RUNTIME_CHECK_MSG(tmt.isInitialized(), "TMTContext is not initialized"); auto & global_context = tmt.getContext(); if (!global_context.getSharedContextDisagg()->isDisaggregatedComputeMode()) { + // compute node does not contains region + single_thread_task_handle = background_pool.addTask( + [this] { + tmt.getKVStore()->gcRegionPersistedCache(); + return false; + }, + false); + // compute node does not contain long-live tables and segments auto & global_settings = global_context.getSettingsRef(); storage_gc_handle = background_pool.addTask( diff --git a/dbms/src/Storages/Transaction/CheckpointInfo.h b/dbms/src/Storages/Transaction/CheckpointInfo.h new file mode 100644 index 00000000000..b8e460089fe --- /dev/null +++ b/dbms/src/Storages/Transaction/CheckpointInfo.h @@ -0,0 +1,38 @@ +// 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 +#include +#include + +using raft_serverpb::PeerState; +using raft_serverpb::RaftApplyState; +using raft_serverpb::RegionLocalState; + +namespace DB +{ +struct TempUniversalPageStorage; +using TempUniversalPageStoragePtr = std::shared_ptr; + +struct CheckpointInfo +{ + UInt64 remote_store_id; + RegionLocalState region_state; + RaftApplyState apply_state; + RegionPtr region; + TempUniversalPageStoragePtr temp_ps_wrapper; // a wrapper to protect the path of `temp_ps` to be deleted + UniversalPageStoragePtr temp_ps; +}; +using CheckpointInfoPtr = std::shared_ptr; +} // namespace DB \ No newline at end of file diff --git a/dbms/src/Storages/Transaction/FastAddPeer.cpp b/dbms/src/Storages/Transaction/FastAddPeer.cpp new file mode 100644 index 00000000000..dc6bfd21ac7 --- /dev/null +++ b/dbms/src/Storages/Transaction/FastAddPeer.cpp @@ -0,0 +1,478 @@ +// 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 +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +FastAddPeerContext::FastAddPeerContext(uint64_t thread_count) +{ + if (thread_count == 0) + { + static constexpr int ffi_handle_sec = 5; + static constexpr int region_per_sec = 2; + thread_count = ffi_handle_sec * region_per_sec; + } + tasks_trace = std::make_shared(thread_count); +} + +TempUniversalPageStoragePtr FastAddPeerContext::getTempUniversalPageStorage(UInt64 store_id, UInt64 upload_seq) +{ + std::unique_lock lock(ps_cache_mu); + auto iter = temp_ps_cache.find(store_id); + if (iter != temp_ps_cache.end() && iter->second.first >= upload_seq) + { + return iter->second.second; + } + return nullptr; +} + +void FastAddPeerContext::updateTempUniversalPageStorage(UInt64 store_id, UInt64 upload_seq, TempUniversalPageStoragePtr temp_ps) +{ + std::unique_lock lock(ps_cache_mu); + auto iter = temp_ps_cache.find(store_id); + if (iter != temp_ps_cache.end() && iter->second.first >= upload_seq) + return; + + temp_ps_cache[store_id] = std::make_pair(upload_seq, temp_ps); +} + +void FastAddPeerContext::insertSegmentEndKeyInfoToCache(TableIdentifier table_identifier, const std::vector> & end_key_and_segment_ids) +{ + std::unique_lock lock(range_cache_mu); + auto & end_key_to_id_map = segment_range_cache[table_identifier]; + for (const auto & [end_key, segment_id] : end_key_and_segment_ids) + { + end_key_to_id_map[end_key] = segment_id; + } +} + +UInt64 FastAddPeerContext::getSegmentIdContainingKey(TableIdentifier table_identifier, const DM::RowKeyValue & key) +{ + std::unique_lock lock(range_cache_mu); + auto iter = segment_range_cache.find(table_identifier); + if (iter != segment_range_cache.end()) + { + auto & end_key_to_id_map = iter->second; + auto key_iter = end_key_to_id_map.lower_bound(key); + if (key_iter != end_key_to_id_map.end()) + { + return key_iter->second; + } + } + return 0; +} + +void FastAddPeerContext::invalidateCache(TableIdentifier table_identifier) +{ + std::unique_lock lock(range_cache_mu); + segment_range_cache.erase(table_identifier); +} + +FastAddPeerRes genFastAddPeerRes(FastAddPeerStatus status, std::string && apply_str, std::string && region_str) +{ + auto * apply = RawCppString::New(apply_str); + auto * region = RawCppString::New(region_str); + return FastAddPeerRes{ + .status = status, + .apply_state = CppStrWithView{.inner = GenRawCppPtr(apply, RawCppPtrTypeImpl::String), .view = BaseBuffView{apply->data(), apply->size()}}, + .region = CppStrWithView{.inner = GenRawCppPtr(region, RawCppPtrTypeImpl::String), .view = BaseBuffView{region->data(), region->size()}}, + }; +} + +TempUniversalPageStoragePtr createTempPageStorage(Context & context, const String & manifest_key, UInt64 dir_seq) +{ + auto file_provider = context.getFileProvider(); + PageStorageConfig config; + const auto dir_prefix = fmt::format("local_{}", dir_seq); + auto temp_ps_wrapper = std::make_shared(); + auto delegator = context.getPathPool().getPSDiskDelegatorGlobalMulti(dir_prefix); + for (const auto & path : delegator->listPaths()) + { + temp_ps_wrapper->paths.push_back(path); + auto file = Poco::File(path); + if (file.exists()) + { + LOG_WARNING(Logger::get("createTempPageStorage"), "Path {} already exists, removing it", path); + file.remove(true); + } + } + auto local_ps = UniversalPageStorage::create( // + dir_prefix, + delegator, + config, + file_provider); + local_ps->restore(); + temp_ps_wrapper->temp_ps = local_ps; + auto * log = &Poco::Logger::get("FastAddPeer"); + LOG_DEBUG(log, "Begin to create temp ps from {}", manifest_key); + + RandomAccessFilePtr manifest_file = S3::S3RandomAccessFile::create(manifest_key); + auto reader = PS::V3::CPManifestFileReader::create({ + .plain_file = manifest_file, + }); + auto im = PS::V3::CheckpointProto::StringsInternMap{}; + auto prefix = reader->readPrefix(); + UniversalWriteBatch wb; + wb.disableRemoteLock(); + // insert delete records at last + PS::V3::PageEntriesEdit::EditRecords ref_records; + PS::V3::PageEntriesEdit::EditRecords delete_records; + while (true) + { + auto edits = reader->readEdits(im); + if (!edits.has_value()) + break; + auto records = edits->getRecords(); + for (auto & record : records) + { + if (record.type == PS::V3::EditRecordType::VAR_ENTRY) + { + wb.putRemotePage(record.page_id, record.entry.tag, record.entry.checkpoint_info.data_location, std::move(record.entry.field_offsets)); + } + else if (record.type == PS::V3::EditRecordType::VAR_REF) + { + ref_records.emplace_back(record); + } + else if (record.type == PS::V3::EditRecordType::VAR_DELETE) + { + delete_records.emplace_back(record); + } + else if (record.type == PS::V3::EditRecordType::VAR_EXTERNAL) + { + RUNTIME_CHECK(record.entry.checkpoint_info.has_value()); + wb.putRemoteExternal(record.page_id, record.entry.checkpoint_info.data_location); + } + else + { + RUNTIME_CHECK(false); + } + } + } + + for (const auto & record : ref_records) + { + RUNTIME_CHECK(record.type == PS::V3::EditRecordType::VAR_REF); + wb.putRefPage(record.page_id, record.ori_page_id); + } + for (const auto & record : delete_records) + { + RUNTIME_CHECK(record.type == PS::V3::EditRecordType::VAR_DELETE); + wb.delPage(record.page_id); + } + local_ps->write(std::move(wb)); + return temp_ps_wrapper; +} + +TempUniversalPageStoragePtr reuseOrCreateTempPageStorage(Context & context, const String & manifest_key) +{ + auto fap_ctx = context.getSharedContextDisagg()->fap_context; + auto manifest_key_view = S3::S3FilenameView::fromKey(manifest_key); + auto upload_seq = manifest_key_view.getUploadSequence(); + auto temp_ps = fap_ctx->getTempUniversalPageStorage(manifest_key_view.store_id, upload_seq); + if (!temp_ps) + { + temp_ps = createTempPageStorage(context, manifest_key, fap_ctx->temp_ps_dir_sequence++); + fap_ctx->updateTempUniversalPageStorage(manifest_key_view.store_id, upload_seq, temp_ps); + } + return temp_ps; +} + +std::optional tryGetCheckpointInfo(Context & context, const String & manifest_key, uint64_t region_id, TiFlashRaftProxyHelper * proxy_helper) +{ + auto * log = &Poco::Logger::get("FastAddPeer"); + + auto checkpoint_info = std::make_shared(); + auto manifest_key_view = S3::S3FilenameView::fromKey(manifest_key); + checkpoint_info->remote_store_id = manifest_key_view.store_id; + checkpoint_info->temp_ps_wrapper = reuseOrCreateTempPageStorage(context, manifest_key); + checkpoint_info->temp_ps = checkpoint_info->temp_ps_wrapper->temp_ps; + + try + { + auto apply_state_key = UniversalPageIdFormat::toRaftApplyStateKeyInKVEngine(region_id); + auto page = checkpoint_info->temp_ps->read(apply_state_key); + checkpoint_info->apply_state.ParseFromArray(page.data.begin(), page.data.size()); + } + catch (...) + { + LOG_DEBUG(log, "Failed to find apply state key [region_id={}]", region_id); + return std::nullopt; + } + + try + { + auto local_state_key = UniversalPageIdFormat::toRegionLocalStateKeyInKVEngine(region_id); + auto page = checkpoint_info->temp_ps->read(local_state_key); + checkpoint_info->region_state.ParseFromArray(page.data.begin(), page.data.size()); + } + catch (...) + { + LOG_DEBUG(log, "Failed to find region local state key [region_id={}]", region_id); + return std::nullopt; + } + + try + { + auto region_key = UniversalPageIdFormat::toKVStoreKey(region_id); + auto page = checkpoint_info->temp_ps->read(region_key); + ReadBufferFromMemory buf(page.data.begin(), page.data.size()); + checkpoint_info->region = Region::deserialize(buf, proxy_helper); + } + catch (...) + { + LOG_DEBUG(log, "Failed to find region key [region_id={}]", region_id); + return std::nullopt; + } + + return checkpoint_info; +} + +std::vector getAllStoreIDsFromPD(TMTContext & tmt_context) +{ + auto pd_client = tmt_context.getPDClient(); + auto stores_from_pd = pd_client->getAllStores(/*exclude_tombstone*/ true); + std::vector store_ids; + store_ids.reserve(stores_from_pd.size()); + for (const auto & s : stores_from_pd) + { + store_ids.push_back(s.id()); + } + return store_ids; +} + +CheckpointInfoPtr selectCheckpointInfo(Context & context, uint64_t region_id, uint64_t new_peer_id, TiFlashRaftProxyHelper * proxy_helper) +{ + auto * log = &Poco::Logger::get("FastAddPeer"); + + std::vector candidates; + std::map reason; + std::map candidate_stat; + + auto & tmt_context = context.getTMTContext(); + std::vector all_store_ids = getAllStoreIDsFromPD(tmt_context); + auto current_store_id = tmt_context.getKVStore()->getStoreMeta().id(); + auto s3_client = S3::ClientFactory::instance().sharedTiFlashClient(); + for (const auto & store_id : all_store_ids) + { + if (store_id == current_store_id) + continue; + const auto manifests = S3::CheckpointManifestS3Set::getFromS3(*s3_client, store_id); + if (manifests.empty()) + { + LOG_DEBUG(log, "no manifest on this store, skip store_id={}", store_id); + continue; + } + const auto & latest_manifest_key = manifests.latestManifestKey(); + auto region_info = tryGetCheckpointInfo(context, latest_manifest_key, region_id, proxy_helper); + if (region_info.has_value()) + { + candidates.push_back(std::move(*region_info)); + } + } + + if (candidates.empty()) + { + LOG_INFO(log, "No candidate. [region_id={}]", region_id); + return nullptr; + } + + CheckpointInfoPtr winner = nullptr; + uint64_t largest_applied_index = 0; + for (const auto & candidate : candidates) + { + auto store_id = candidate->remote_store_id; + const auto & region_state = candidate->region_state; + const auto & apply_state = candidate->apply_state; + const auto & peers = region_state.region().peers(); + bool ok = false; + for (auto && pr : peers) + { + if (pr.id() == new_peer_id) + { + ok = true; + break; + } + } + if (!ok) + { + // Can't use this peer if it has no new_peer_id. + reason[store_id] = fmt::format("has no peer_id {}", region_state.ShortDebugString()); + continue; + } + auto peer_state = region_state.state(); + if (peer_state == PeerState::Tombstone || peer_state == PeerState::Applying) + { + // Can't use this peer in these states. + reason[store_id] = fmt::format("bad peer_state {}", region_state.ShortDebugString()); + continue; + } + auto applied_index = apply_state.applied_index(); + if (winner == nullptr || applied_index > largest_applied_index) + { + candidate_stat[store_id] = fmt::format("applied index {}", applied_index); + winner = candidate; + } + } + + if (winner != nullptr) + { + return winner; + } + else + { + FmtBuffer fmt_buf; + for (const auto & r : reason) + { + fmt_buf.fmtAppend("store {} reason {}, ", r.first, r.second); + } + std::string failed_reason = fmt_buf.toString(); + fmt_buf.clear(); + for (const auto & c : candidate_stat) + { + fmt_buf.fmtAppend("store {} stat {}, ", c.first, c.second); + } + std::string choice_stat = fmt_buf.toString(); + LOG_INFO(log, "Failed to find remote checkpoint [region_id={}] [new_peer_id={}] [total_candidates={}]; reason: {}; candidates_stat: {};", region_id, new_peer_id, candidates.size(), failed_reason, choice_stat); + return nullptr; + } +} + +void resetPeerIdInRegion(RegionPtr region, const RegionLocalState & region_state, uint64_t new_peer_id) +{ + for (auto && pr : region_state.region().peers()) + { + if (pr.id() == new_peer_id) + { + auto cpr = pr; + region->mutMeta().setPeer(std::move(cpr)); + return; + } + } + RUNTIME_CHECK(false); +} + +FastAddPeerRes FastAddPeerImpl(EngineStoreServerWrap * server, uint64_t region_id, uint64_t new_peer_id) +{ + try + { + auto * log = &Poco::Logger::get("FastAddPeer"); + auto kvstore = server->tmt->getKVStore(); + Stopwatch watch; + CheckpointInfoPtr checkpoint_info; + while (true) + { + checkpoint_info = selectCheckpointInfo(server->tmt->getContext(), region_id, new_peer_id, server->proxy_helper); + if (checkpoint_info == nullptr) + { + // TODO: make it a config + constexpr int wait_source_apply_timeout_seconds = 60; + if (watch.elapsedSeconds() >= wait_source_apply_timeout_seconds) + return genFastAddPeerRes(FastAddPeerStatus::NoSuitable, "", ""); + std::this_thread::sleep_for(std::chrono::milliseconds(500)); + } + else + break; + } + LOG_INFO(log, "Select checkpoint from store {} takes {} seconds; [region_id={}]", checkpoint_info->remote_store_id, watch.elapsedSeconds(), region_id); + + resetPeerIdInRegion(checkpoint_info->region, checkpoint_info->region_state, new_peer_id); + + kvstore->handleIngestCheckpoint(checkpoint_info, *server->tmt); + + // Write raft log to uni ps + UniversalWriteBatch wb; + RaftDataReader raft_data_reader(*(checkpoint_info->temp_ps)); + raft_data_reader.traverseRaftLogForRegion(region_id, [&](const UniversalPageId & page_id, DB::Page page) { + MemoryWriteBuffer buf; + buf.write(page.data.begin(), page.data.size()); + wb.putPage(page_id, 0, buf.tryGetReadBuffer(), page.data.size()); + }); + auto wn_ps = server->tmt->getContext().getWriteNodePageStorage(); + wn_ps->write(std::move(wb)); + + return genFastAddPeerRes(FastAddPeerStatus::Ok, checkpoint_info->apply_state.SerializeAsString(), checkpoint_info->region_state.region().SerializeAsString()); + } + catch (...) + { + DB::tryLogCurrentException("FastAddPeer", "Failed when try to restore from checkpoint"); + return genFastAddPeerRes(FastAddPeerStatus::BadData, "", ""); + } +} + +FastAddPeerRes FastAddPeer(EngineStoreServerWrap * server, uint64_t region_id, uint64_t new_peer_id) +{ + try + { + auto * log = &Poco::Logger::get("FastAddPeer"); + auto fap_ctx = server->tmt->getContext().getSharedContextDisagg()->fap_context; + RUNTIME_CHECK(fap_ctx != nullptr); + if (!fap_ctx->tasks_trace->isScheduled(region_id)) + { + // We need to schedule the task. + auto res = fap_ctx->tasks_trace->addTask(region_id, [server, region_id, new_peer_id]() { + return FastAddPeerImpl(server, region_id, new_peer_id); + }); + if (res) + { + LOG_INFO(log, "add new task [new_peer_id={}] [region_id={}]", new_peer_id, region_id); + } + else + { + LOG_INFO(log, "add new task fail(queue full) [new_peer_id={}] [region_id={}]", new_peer_id, region_id); + return genFastAddPeerRes(FastAddPeerStatus::WaitForData, "", ""); + } + } + + if (fap_ctx->tasks_trace->isReady(region_id)) + { + LOG_INFO(log, "fetch task result [new_peer_id={}] [region_id={}]", new_peer_id, region_id); + return fap_ctx->tasks_trace->fetchResult(region_id); + } + else + { + LOG_DEBUG(log, "the task is still pending [new_peer_id={}] [region_id={}]", new_peer_id, region_id); + return genFastAddPeerRes(FastAddPeerStatus::WaitForData, "", ""); + } + } + catch (...) + { + DB::tryLogCurrentException("FastAddPeer", fmt::format("Failed when try to restore from checkpoint {}", StackTrace().toString())); + return genFastAddPeerRes(FastAddPeerStatus::OtherError, "", ""); + } +} +} // namespace DB diff --git a/dbms/src/Storages/Transaction/FastAddPeer.h b/dbms/src/Storages/Transaction/FastAddPeer.h new file mode 100644 index 00000000000..e59c5344df3 --- /dev/null +++ b/dbms/src/Storages/Transaction/FastAddPeer.h @@ -0,0 +1,120 @@ +// 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. + +#pragma once + +#include +#include + +#include + +namespace DB +{ +struct TableIdentifier +{ + UInt64 key_space_id; + UInt64 store_id; + DB::NamespaceId table_id; + + bool operator==(const TableIdentifier & other) const + { + return key_space_id == other.key_space_id && store_id == other.store_id && table_id == other.table_id; + } +}; +} // namespace DB + +namespace std +{ +template <> +struct hash +{ + size_t operator()(const DB::TableIdentifier & k) const + { + size_t seed = 0; + boost::hash_combine(seed, boost::hash_value(k.key_space_id)); + boost::hash_combine(seed, boost::hash_value(k.store_id)); + boost::hash_combine(seed, boost::hash_value(k.table_id)); + return seed; + } +}; +} // namespace std + +namespace DB +{ +class UniversalPageStorage; +using UniversalPageStoragePtr = std::shared_ptr; + +struct TempUniversalPageStorage +{ + UniversalPageStoragePtr temp_ps; + std::vector paths = {}; + + ~TempUniversalPageStorage() + { + for (const auto & path : paths) + { + Poco::File(path).remove(true); + } + } +}; +using TempUniversalPageStoragePtr = std::shared_ptr; + +struct AsyncTasks; + +class FastAddPeerContext +{ +public: + explicit FastAddPeerContext(uint64_t thread_count = 0); + + // return a TempUniversalPageStoragePtr which have sequence >= upload_seq + TempUniversalPageStoragePtr getTempUniversalPageStorage(UInt64 store_id, UInt64 upload_seq); + + void updateTempUniversalPageStorage(UInt64 store_id, UInt64 upload_seq, TempUniversalPageStoragePtr temp_ps); + + void insertSegmentEndKeyInfoToCache(TableIdentifier table_identifier, const std::vector> & end_key_and_segment_ids); + + // return the cached id of the segment which contains the target key + // return 0 means no cache info for the key + UInt64 getSegmentIdContainingKey(TableIdentifier table_identifier, const DM::RowKeyValue & key); + + // TODO: invalidate cache at segment level + void invalidateCache(TableIdentifier table_identifier); + +public: + std::shared_ptr tasks_trace; + std::atomic temp_ps_dir_sequence; + +private: + std::mutex ps_cache_mu; + // Store the latest manifest data for every store + // StoreId -> pair + std::unordered_map> temp_ps_cache; + + std::mutex range_cache_mu; + + struct KeyComparator + { + bool operator()(const DM::RowKeyValue & key1, const DM::RowKeyValue & key2) const + { + return compare(key1.toRowKeyValueRef(), key2.toRowKeyValueRef()); + } + }; + + // Store the mapping from end key to segment id for each table + // TableIdentifier -> (Segment Range End -> Segment ID) + std::unordered_map> segment_range_cache; +}; + +TempUniversalPageStoragePtr createTempPageStorage(Context & context, const String & manifest_key, UInt64 dir_seq); +} // namespace DB diff --git a/dbms/src/Storages/Transaction/FastAddPeerAsyncTasksImpl.h b/dbms/src/Storages/Transaction/FastAddPeerAsyncTasksImpl.h new file mode 100644 index 00000000000..f13023ccc77 --- /dev/null +++ b/dbms/src/Storages/Transaction/FastAddPeerAsyncTasksImpl.h @@ -0,0 +1,81 @@ +// 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. + +#pragma once + +#include +#include + +#include + +namespace DB +{ +struct AsyncTasks +{ + using Key = uint64_t; + using Func = std::function; + + // We use a big queue to cache, to reduce ass task failures. + explicit AsyncTasks(uint64_t pool_size) + : thread_pool(std::make_unique(pool_size, pool_size, 300)) + {} + explicit AsyncTasks(uint64_t pool_size, uint64_t free_pool_size, uint64_t queue_size) + : thread_pool(std::make_unique(pool_size, free_pool_size, queue_size)) + {} + + bool addTask(Key k, Func f) + { + using P = std::packaged_task; + std::shared_ptr

p = std::make_shared

(P(f)); + + auto res = thread_pool->trySchedule([p]() { (*p)(); }, 0, 0); + if (res) + { + std::scoped_lock l(mtx); + futures[k] = p->get_future(); + } + return res; + } + + bool isScheduled(Key key) const + { + std::scoped_lock l(mtx); + return futures.count(key); + } + + bool isReady(Key key) const + { + using namespace std::chrono_literals; + std::scoped_lock l(mtx); + if (!futures.count(key)) + return false; + return futures.at(key).wait_for(0ms) == std::future_status::ready; + } + + FastAddPeerRes fetchResult(Key key) + { + std::unique_lock l(mtx); + auto it = futures.find(key); + auto fut = std::move(it->second); + futures.erase(it); + l.unlock(); + return fut.get(); + } + +protected: + std::map> futures; + std::unique_ptr thread_pool; + mutable std::mutex mtx; +}; +} // namespace DB \ No newline at end of file diff --git a/dbms/src/Storages/Transaction/KVStore.cpp b/dbms/src/Storages/Transaction/KVStore.cpp index 81da81d34e0..51ec448d7d7 100644 --- a/dbms/src/Storages/Transaction/KVStore.cpp +++ b/dbms/src/Storages/Transaction/KVStore.cpp @@ -17,6 +17,7 @@ #include #include #include +#include #include #include #include @@ -44,7 +45,7 @@ extern const char force_fail_in_flush_region_data[]; } // namespace FailPoints KVStore::KVStore(Context & context) - : region_persister(std::make_unique(context, region_manager)) + : region_persister(context.getSharedContextDisagg()->isDisaggregatedComputeMode() ? nullptr : std::make_unique(context, region_manager)) , raft_cmd_res(std::make_unique()) , log(Logger::get()) , region_compact_log_period(120) @@ -56,6 +57,9 @@ KVStore::KVStore(Context & context) void KVStore::restore(PathPool & path_pool, const TiFlashRaftProxyHelper * proxy_helper) { + if (!region_persister) + return; + auto task_lock = genTaskLock(); auto manage_lock = genRegionWriteLock(task_lock); @@ -174,6 +178,7 @@ void KVStore::tryPersist(RegionID region_id) if (region) { LOG_INFO(log, "Try to persist {}", region->toString(false)); + RUNTIME_CHECK_MSG(region_persister, "try access to region_persister without initialization, stack={}", StackTrace().toString()); region_persister->persist(*region); LOG_INFO(log, "After persisted {}, cache {} bytes", region->toString(false), region->dataSize()); } @@ -190,6 +195,7 @@ void KVStore::gcRegionPersistedCache(Seconds gc_persist_period) if (now < (last_gc_time.load() + gc_persist_period)) return; last_gc_time = now; + RUNTIME_CHECK_MSG(region_persister, "try access to region_persister without initialization, stack={}", StackTrace().toString()); region_persister->gc(); } @@ -211,6 +217,7 @@ void KVStore::removeRegion(RegionID region_id, bool remove_data, RegionTable & r } } + RUNTIME_CHECK_MSG(region_persister, "try access to region_persister without initialization, stack={}", StackTrace().toString()); region_persister->drop(region_id, region_lock); LOG_INFO(log, "Persisted [region {}] deleted", region_id); @@ -239,7 +246,7 @@ EngineStoreApplyRes KVStore::handleWriteRaftCmd( UInt64 region_id, UInt64 index, UInt64 term, - TMTContext & tmt) + TMTContext & tmt) const { std::vector keys; std::vector vals; @@ -280,7 +287,7 @@ EngineStoreApplyRes KVStore::handleWriteRaftCmd( tmt); } -EngineStoreApplyRes KVStore::handleWriteRaftCmd(const WriteCmdsView & cmds, UInt64 region_id, UInt64 index, UInt64 term, TMTContext & tmt) +EngineStoreApplyRes KVStore::handleWriteRaftCmd(const WriteCmdsView & cmds, UInt64 region_id, UInt64 index, UInt64 term, TMTContext & tmt) const { auto region_persist_lock = region_manager.genRegionTaskLock(region_id); @@ -329,6 +336,7 @@ void KVStore::setRegionCompactLogConfig(UInt64 sec, UInt64 rows, UInt64 bytes) void KVStore::persistRegion(const Region & region, const RegionTaskLock & region_task_lock, const char * caller) { LOG_INFO(log, "Start to persist {}, cache size: {} bytes for `{}`", region.toString(true), region.dataSize(), caller); + RUNTIME_CHECK_MSG(region_persister, "try access to region_persister without initialization, stack={}", StackTrace().toString()); region_persister->persist(region, region_task_lock); LOG_DEBUG(log, "Persist {} done", region.toString(false)); } @@ -872,6 +880,11 @@ KVStore::~KVStore() FileUsageStatistics KVStore::getFileUsageStatistics() const { + if (!region_persister) + { + return {}; + } + return region_persister->getFileUsageStatistics(); } diff --git a/dbms/src/Storages/Transaction/KVStore.h b/dbms/src/Storages/Transaction/KVStore.h index e5c171fe8ca..e40954bfbd8 100644 --- a/dbms/src/Storages/Transaction/KVStore.h +++ b/dbms/src/Storages/Transaction/KVStore.h @@ -73,12 +73,14 @@ class ReadIndexStressTest; struct FileUsageStatistics; class PathPool; class RegionPersister; +struct CheckpointInfo; +using CheckpointInfoPtr = std::shared_ptr; /// TODO: brief design document. class KVStore final : private boost::noncopyable { public: - KVStore(Context & context); + explicit KVStore(Context & context); void restore(PathPool & path_pool, const TiFlashRaftProxyHelper *); RegionPtr getRegion(RegionID region_id) const; @@ -96,19 +98,20 @@ class KVStore final : private boost::noncopyable static bool tryFlushRegionCacheInStorage(TMTContext & tmt, const Region & region, const LoggerPtr & log, bool try_until_succeed = true); size_t regionSize() const; - EngineStoreApplyRes handleAdminRaftCmd(raft_cmdpb::AdminRequest && request, - raft_cmdpb::AdminResponse && response, - UInt64 region_id, - UInt64 index, - UInt64 term, - TMTContext & tmt); + EngineStoreApplyRes handleAdminRaftCmd( + raft_cmdpb::AdminRequest && request, + raft_cmdpb::AdminResponse && response, + UInt64 region_id, + UInt64 index, + UInt64 term, + TMTContext & tmt); EngineStoreApplyRes handleWriteRaftCmd( raft_cmdpb::RaftCmdRequest && request, UInt64 region_id, UInt64 index, UInt64 term, - TMTContext & tmt); - EngineStoreApplyRes handleWriteRaftCmd(const WriteCmdsView & cmds, UInt64 region_id, UInt64 index, UInt64 term, TMTContext & tmt); + TMTContext & tmt) const; + EngineStoreApplyRes handleWriteRaftCmd(const WriteCmdsView & cmds, UInt64 region_id, UInt64 index, UInt64 term, TMTContext & tmt) const; bool needFlushRegionData(UInt64 region_id, TMTContext & tmt); bool tryFlushRegionData(UInt64 region_id, bool force_persist, bool try_until_succeed, TMTContext & tmt, UInt64 index, UInt64 term); @@ -118,6 +121,8 @@ class KVStore final : private boost::noncopyable */ void handleApplySnapshot(metapb::Region && region, uint64_t peer_id, SSTViewVec, uint64_t index, uint64_t term, TMTContext & tmt); + void handleIngestCheckpoint(CheckpointInfoPtr checkpoint_info, TMTContext & tmt); + std::vector preHandleSnapshotToFiles( RegionPtr new_region, SSTViewVec, diff --git a/dbms/src/Storages/Transaction/LearnerRead.cpp b/dbms/src/Storages/Transaction/LearnerRead.cpp index 2118a67f608..053dea273c1 100644 --- a/dbms/src/Storages/Transaction/LearnerRead.cpp +++ b/dbms/src/Storages/Transaction/LearnerRead.cpp @@ -384,7 +384,7 @@ LearnerReadSnapshot doLearnerRead( region_to_query.region_id, region_to_query.version, RecordKVFormat::DecodedTiKVKeyRangeToDebugString(region_to_query.range_in_table), - RegionException::RegionReadStatusString(status)); + magic_enum::enum_name(status)); unavailable_regions.add(region->id(), status); } }, @@ -469,7 +469,7 @@ void validateQueryInfo( region_query_info.region_id, region_query_info.version, RecordKVFormat::DecodedTiKVKeyRangeToDebugString(region_query_info.range_in_table), - RegionException::RegionReadStatusString(status)); + magic_enum::enum_name(status)); } } diff --git a/dbms/src/Storages/Transaction/LockException.h b/dbms/src/Storages/Transaction/LockException.h index c5c4e517bb6..4dc07883896 100644 --- a/dbms/src/Storages/Transaction/LockException.h +++ b/dbms/src/Storages/Transaction/LockException.h @@ -16,24 +16,22 @@ #include #include -#pragma GCC diagnostic push -#pragma GCC diagnostic ignored "-Wunused-parameter" -#pragma GCC diagnostic ignored "-Wnon-virtual-dtor" -#ifdef __clang__ -#pragma GCC diagnostic ignored "-Wdeprecated-declarations" -#endif #include -#pragma GCC diagnostic pop namespace DB { -using RegionVerID = pingcap::kv::RegionVerID; + +namespace ErrorCodes +{ +extern const int REGION_LOCKED; +} // namespace ErrorCodes class LockException : public Exception { public: explicit LockException(RegionID region_id_, LockInfoPtr lock_info) - : region_id(region_id_) + : Exception("Region is locked", ErrorCodes::REGION_LOCKED) + , region_id(region_id_) , lock_info(std::move(lock_info)) {} diff --git a/dbms/src/Storages/Transaction/ProxyFFI.h b/dbms/src/Storages/Transaction/ProxyFFI.h index bc71e47c529..f8d2658d60d 100644 --- a/dbms/src/Storages/Transaction/ProxyFFI.h +++ b/dbms/src/Storages/Transaction/ProxyFFI.h @@ -169,6 +169,7 @@ CppStrWithView GetConfig(EngineStoreServerWrap *, uint8_t full); void SetStore(EngineStoreServerWrap *, BaseBuffView); void SetPBMsByBytes(MsgPBType type, RawVoidPtr ptr, BaseBuffView view); void HandleSafeTSUpdate(EngineStoreServerWrap * server, uint64_t region_id, uint64_t self_safe_ts, uint64_t leader_safe_ts); +FastAddPeerRes FastAddPeer(EngineStoreServerWrap * server, uint64_t region_id, uint64_t new_peer_id); } inline EngineStoreServerHelper GetEngineStoreServerHelper( @@ -215,6 +216,7 @@ inline EngineStoreServerHelper GetEngineStoreServerHelper( .fn_set_store = SetStore, .fn_set_pb_msg_by_bytes = SetPBMsByBytes, .fn_handle_safe_ts_update = HandleSafeTSUpdate, + .fn_fast_add_peer = FastAddPeer, }; } diff --git a/dbms/src/Storages/Transaction/ProxyFFIStatusService.cpp b/dbms/src/Storages/Transaction/ProxyFFIStatusService.cpp index 45c7c4a4a9b..46d77e6f23c 100644 --- a/dbms/src/Storages/Transaction/ProxyFFIStatusService.cpp +++ b/dbms/src/Storages/Transaction/ProxyFFIStatusService.cpp @@ -35,7 +35,7 @@ HttpRequestRes HandleHttpRequestSyncStatus( pingcap::pd::KeyspaceID keyspace_id = NullspaceID; { auto * log = &Poco::Logger::get("HandleHttpRequestSyncStatus"); - LOG_DEBUG(log, "handling sync status request, path: {}, api_name: {}", path, api_name); + LOG_TRACE(log, "handling sync status request, path: {}, api_name: {}", path, api_name); // Try to handle sync status request with old schema. // Old schema: /{table_id} diff --git a/dbms/src/Storages/Transaction/Region.h b/dbms/src/Storages/Transaction/Region.h index 869a2ca44fe..403c1b15f2a 100644 --- a/dbms/src/Storages/Transaction/Region.h +++ b/dbms/src/Storages/Transaction/Region.h @@ -203,6 +203,8 @@ class Region : public std::enable_shared_from_this std::pair getApproxMemCacheInfo() const; void cleanApproxMemCacheInfo() const; + RegionMeta & mutMeta() { return meta; } + private: Region() = delete; friend class RegionRaftCommandDelegate; diff --git a/dbms/src/Storages/Transaction/RegionException.h b/dbms/src/Storages/Transaction/RegionException.h index 54254f03be5..d359a1c3ac4 100644 --- a/dbms/src/Storages/Transaction/RegionException.h +++ b/dbms/src/Storages/Transaction/RegionException.h @@ -17,6 +17,8 @@ #include #include +#include + namespace DB { @@ -32,25 +34,15 @@ class RegionException : public Exception EPOCH_NOT_MATCH, }; - static const char * RegionReadStatusString(RegionReadStatus s) - { - switch (s) - { - case RegionReadStatus::OK: - return "OK"; - case RegionReadStatus::NOT_FOUND: - return "NOT_FOUND"; - case RegionReadStatus::EPOCH_NOT_MATCH: - return "EPOCH_NOT_MATCH"; - } - return "Unknown"; - }; - using UnavailableRegions = std::unordered_set; public: RegionException(UnavailableRegions && unavailable_region_, RegionReadStatus status_) - : Exception(RegionReadStatusString(status_)), unavailable_region(std::move(unavailable_region_)), status(status_) + : Exception(fmt::format( + "Region error {}", + magic_enum::enum_name(status_))) + , unavailable_region(std::move(unavailable_region_)) + , status(status_) {} /// Region could be found with correct epoch, but unavailable (e.g. its lease in proxy has not been built with leader). diff --git a/dbms/src/Storages/Transaction/RegionMeta.cpp b/dbms/src/Storages/Transaction/RegionMeta.cpp index 13a20a39a9b..de9c0d042de 100644 --- a/dbms/src/Storages/Transaction/RegionMeta.cpp +++ b/dbms/src/Storages/Transaction/RegionMeta.cpp @@ -72,6 +72,12 @@ metapb::Peer RegionMeta::getPeer() const return peer; } +void RegionMeta::setPeer(metapb::Peer && p) +{ + std::lock_guard lock(mutex); + peer = p; +} + raft_serverpb::RaftApplyState RegionMeta::getApplyState() const { std::lock_guard lock(mutex); diff --git a/dbms/src/Storages/Transaction/RegionMeta.h b/dbms/src/Storages/Transaction/RegionMeta.h index 39616c1fa8d..08fed625c2a 100644 --- a/dbms/src/Storages/Transaction/RegionMeta.h +++ b/dbms/src/Storages/Transaction/RegionMeta.h @@ -75,6 +75,7 @@ class RegionMeta ImutRegionRangePtr getRange() const; metapb::Peer getPeer() const; + void setPeer(metapb::Peer &&); UInt64 version() const; diff --git a/dbms/src/Storages/Transaction/RegionTable.cpp b/dbms/src/Storages/Transaction/RegionTable.cpp index 18d34980247..c2078a8dc46 100644 --- a/dbms/src/Storages/Transaction/RegionTable.cpp +++ b/dbms/src/Storages/Transaction/RegionTable.cpp @@ -19,6 +19,7 @@ #include #include #include +#include #include #include #include @@ -501,6 +502,11 @@ RegionPtrWithSnapshotFiles::RegionPtrWithSnapshotFiles( , external_files(std::move(external_files_)) {} +RegionPtrWithCheckpointInfo::RegionPtrWithCheckpointInfo(CheckpointInfoPtr checkpoint_info_) + : base(checkpoint_info_->region) + , checkpoint_info(std::move(checkpoint_info_)) +{} + bool RegionTable::isSafeTSLag(UInt64 region_id, UInt64 * leader_safe_ts, UInt64 * self_safe_ts) { { diff --git a/dbms/src/Storages/Transaction/RegionTable.h b/dbms/src/Storages/Transaction/RegionTable.h index 03f60f1644f..0150a77edeb 100644 --- a/dbms/src/Storages/Transaction/RegionTable.h +++ b/dbms/src/Storages/Transaction/RegionTable.h @@ -55,6 +55,8 @@ struct RegionPtrWithBlock; struct RegionPtrWithSnapshotFiles; class RegionScanFilter; using RegionScanFilterPtr = std::shared_ptr; +struct CheckpointInfo; +using CheckpointInfoPtr = std::shared_ptr; using SafeTS = UInt64; enum : SafeTS @@ -298,4 +300,22 @@ struct RegionPtrWithSnapshotFiles const std::vector external_files; }; +// A wrap of RegionPtr, with checkpoint info to be ingested +struct RegionPtrWithCheckpointInfo +{ + using Base = RegionPtr; + + RegionPtrWithCheckpointInfo(CheckpointInfoPtr checkpoint_info_); + + /// to be compatible with usage as RegionPtr. + Base::element_type * operator->() const { return base.operator->(); } + const Base::element_type & operator*() const { return base.operator*(); } + + /// make it could be cast into RegionPtr implicitly. + operator const Base &() const { return base; } + + const Base & base; + CheckpointInfoPtr checkpoint_info; +}; + } // namespace DB diff --git a/dbms/src/Storages/Transaction/tests/gtest_kvstore_fast_add_peer.cpp b/dbms/src/Storages/Transaction/tests/gtest_kvstore_fast_add_peer.cpp new file mode 100644 index 00000000000..f3715bcad8f --- /dev/null +++ b/dbms/src/Storages/Transaction/tests/gtest_kvstore_fast_add_peer.cpp @@ -0,0 +1,235 @@ +// 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 +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include +#include +#include +#include + +using raft_serverpb::RaftApplyState; +using raft_serverpb::RegionLocalState; + +namespace DB +{ +FastAddPeerRes genFastAddPeerRes(FastAddPeerStatus status, std::string && apply_str, std::string && region_str); +TempUniversalPageStoragePtr reuseOrCreateTempPageStorage(Context & context, const String & manifest_key); + +namespace tests +{ +class RegionKVStoreTestFAP : public RegionKVStoreTest +{ +public: + void SetUp() override + { + auto & global_context = TiFlashTestEnv::getGlobalContext(); + if (global_context.getSharedContextDisagg()->remote_data_store == nullptr) + { + already_initialize_data_store = false; + global_context.getSharedContextDisagg()->initRemoteDataStore(global_context.getFileProvider(), /*s3_enabled*/ true); + ASSERT_TRUE(global_context.getSharedContextDisagg()->remote_data_store != nullptr); + } + else + { + already_initialize_data_store = true; + } + orig_mode = global_context.getPageStorageRunMode(); + global_context.setPageStorageRunMode(PageStorageRunMode::UNI_PS); + global_context.getSharedContextDisagg()->initFastAddPeerContext(); + RegionKVStoreTest::SetUp(); + } + + void TearDown() override + { + auto & global_context = TiFlashTestEnv::getGlobalContext(); + if (!already_initialize_data_store) + { + global_context.getSharedContextDisagg()->remote_data_store = nullptr; + } + global_context.setPageStorageRunMode(orig_mode); + } + +protected: + void dumpCheckpoint() + { + auto & global_context = TiFlashTestEnv::getGlobalContext(); + auto temp_dir = TiFlashTestEnv::getTemporaryPath() + "/"; + auto page_storage = global_context.getWriteNodePageStorage(); + KVStore & kvs = getKVS(); + auto store_id = kvs.getStore().store_id.load(); + auto wi = PS::V3::CheckpointProto::WriterInfo(); + { + wi.set_store_id(store_id); + } + + auto remote_store = global_context.getSharedContextDisagg()->remote_data_store; + assert(remote_store != nullptr); + UniversalPageStorage::DumpCheckpointOptions opts{ + .data_file_id_pattern = S3::S3Filename::newCheckpointDataNameTemplate(store_id, upload_sequence), + .data_file_path_pattern = temp_dir + "dat_{seq}_{index}", + .manifest_file_id_pattern = S3::S3Filename::newCheckpointManifestNameTemplate(store_id), + .manifest_file_path_pattern = temp_dir + "mf_{seq}", + .writer_info = wi, + .must_locked_files = {}, + .persist_checkpoint = CheckpointUploadFunctor{ + .store_id = store_id, + // Note that we use `upload_sequence` but not `snapshot.sequence` for + // the S3 key. + .sequence = upload_sequence, + .remote_store = remote_store, + }, + .override_sequence = upload_sequence, // override by upload_sequence + }; + page_storage->dumpIncrementalCheckpoint(opts); + } + +protected: + UInt64 upload_sequence = 1000; + +private: + ContextPtr context; + bool already_initialize_data_store = false; + DB::PageStorageRunMode orig_mode; +}; + +TEST_F(RegionKVStoreTestFAP, FAPThreadPool) +try +{ + auto * log = &Poco::Logger::get("RegionKVStoreTest"); + using namespace std::chrono_literals; + auto fap_context = std::make_shared(1); + auto async_tasks = fap_context->tasks_trace; + + int total = 5; + std::vector f(total, false); + while (true) + { + auto count = std::accumulate(f.begin(), f.end(), 0, [&](int a, bool b) -> int { + return a + int(b); + }); + if (count >= total) + { + break; + } + else + { + LOG_DEBUG(log, "finished {}/{}", count, total); + } + for (int i = 0; i < total; i++) + { + if (!async_tasks->isScheduled(i)) + { + auto res = async_tasks->addTask(i, []() { + std::this_thread::sleep_for(1000ms); + return genFastAddPeerRes(FastAddPeerStatus::WaitForData, "", ""); + }); + UNUSED(res); + } + } + + for (int i = 0; i < total; i++) + { + if (!f[i]) + { + if (async_tasks->isReady(i)) + { + auto r = async_tasks->fetchResult(i); + UNUSED(r); + f[i] = true; + } + } + } + std::this_thread::sleep_for(1000ms); + } +} +CATCH + +void persistAfterWrite(Context & ctx, KVStore & kvs, std::unique_ptr & proxy_instance, UniversalPageStoragePtr page_storage, uint64_t region_id, uint64_t index) +{ + MockRaftStoreProxy::FailCond cond; + proxy_instance->doApply(kvs, ctx.getTMTContext(), cond, region_id, index); + auto region = proxy_instance->getRegion(region_id); + auto wb = region->persistMeta(); + page_storage->write(std::move(wb), nullptr); + // There shall be data to flush. + ASSERT_EQ(kvs.needFlushRegionData(region_id, ctx.getTMTContext()), true); + ASSERT_EQ(kvs.tryFlushRegionData(region_id, false, false, ctx.getTMTContext(), 0, 0), true); +} + +TEST_F(RegionKVStoreTestFAP, RestoreRaftState) +try +{ + auto & global_context = TiFlashTestEnv::getGlobalContext(); + uint64_t region_id = 1; + auto peer_id = 1; + KVStore & kvs = getKVS(); + auto page_storage = global_context.getWriteNodePageStorage(); + + proxy_instance->bootstrap(kvs, global_context.getTMTContext(), region_id); + auto region = proxy_instance->getRegion(region_id); + auto store_id = kvs.getStore().store_id.load(); + region->addPeer(store_id, peer_id, metapb::PeerRole::Learner); + + // Write some data, and persist meta. + auto [index, term] = proxy_instance->normalWrite(region_id, {34}, {"v2"}, {WriteCmdType::Put}, {ColumnFamilyType::Default}); + persistAfterWrite(global_context, kvs, proxy_instance, page_storage, region_id, index); + + auto s3_client = S3::ClientFactory::instance().sharedTiFlashClient(); + ASSERT_TRUE(::DB::tests::TiFlashTestEnv::createBucketIfNotExist(*s3_client)); + dumpCheckpoint(); + + std::optional checkpoint_info; + const auto manifests = S3::CheckpointManifestS3Set::getFromS3(*s3_client, store_id); + ASSERT_TRUE(!manifests.empty()); + const auto & latest_manifest_key = manifests.latestManifestKey(); + auto temp_ps_wrapper = reuseOrCreateTempPageStorage(global_context, latest_manifest_key); + + RaftApplyState apply_state; + { + auto apply_state_key = UniversalPageIdFormat::toRaftApplyStateKeyInKVEngine(region_id); + auto page = temp_ps_wrapper->temp_ps->read(apply_state_key); + apply_state.ParseFromArray(page.data.begin(), page.data.size()); + } + + RegionLocalState region_state; + { + auto local_state_key = UniversalPageIdFormat::toRegionLocalStateKeyInKVEngine(region_id); + auto page = temp_ps_wrapper->temp_ps->read(local_state_key); + region_state.ParseFromArray(page.data.begin(), page.data.size()); + } + + ASSERT_TRUE(apply_state == region->getApply()); + ASSERT_TRUE(region_state == region->getState()); + + auto fap_context = global_context.getSharedContextDisagg()->fap_context; + ASSERT_TRUE(fap_context->getTempUniversalPageStorage(store_id, upload_sequence) != nullptr); + ASSERT_TRUE(fap_context->getTempUniversalPageStorage(store_id, upload_sequence - 1) != nullptr); + ASSERT_TRUE(fap_context->getTempUniversalPageStorage(store_id, upload_sequence + 1) == nullptr); +} +CATCH +} // namespace tests +} // namespace DB diff --git a/dbms/src/Storages/Transaction/tests/gtest_region_persister.cpp b/dbms/src/Storages/Transaction/tests/gtest_region_persister.cpp index ca4dbbe0565..3c850e5bf76 100644 --- a/dbms/src/Storages/Transaction/tests/gtest_region_persister.cpp +++ b/dbms/src/Storages/Transaction/tests/gtest_region_persister.cpp @@ -227,12 +227,14 @@ class RegionPersisterTest /*kvstore_paths=*/Strings{}, path_capacity, provider); + global_ctx.tryReleaseWriteNodePageStorageForTest(); global_ctx.initializeWriteNodePageStorageIfNeed(*mocked_path_pool); } void reload() { auto & global_ctx = DB::tests::TiFlashTestEnv::getGlobalContext(); + global_ctx.tryReleaseWriteNodePageStorageForTest(); global_ctx.initializeWriteNodePageStorageIfNeed(*mocked_path_pool); } diff --git a/dbms/src/Storages/Transaction/tests/kvstore_helper.h b/dbms/src/Storages/Transaction/tests/kvstore_helper.h index 79593bd1020..8b191d462c5 100644 --- a/dbms/src/Storages/Transaction/tests/kvstore_helper.h +++ b/dbms/src/Storages/Transaction/tests/kvstore_helper.h @@ -101,6 +101,7 @@ class RegionKVStoreTest : public ::testing::Test { kvstore.reset(); auto & global_ctx = TiFlashTestEnv::getGlobalContext(); + global_ctx.tryReleaseWriteNodePageStorageForTest(); global_ctx.initializeWriteNodePageStorageIfNeed(*path_pool); kvstore = std::make_unique(global_ctx); // only recreate kvstore and restore data from disk, don't recreate proxy instance diff --git a/dbms/src/TestUtils/ColumnGenerator.cpp b/dbms/src/TestUtils/ColumnGenerator.cpp index 36b4bc2d4d9..a027d97d273 100644 --- a/dbms/src/TestUtils/ColumnGenerator.cpp +++ b/dbms/src/TestUtils/ColumnGenerator.cpp @@ -55,18 +55,36 @@ ColumnWithTypeAndName ColumnGenerator::generate(const ColumnGeneratorOpts & opts switch (type_id) { case TypeIndex::UInt8: + for (size_t i = 0; i < opts.size; ++i) + genUInt(col); + break; case TypeIndex::UInt16: + for (size_t i = 0; i < opts.size; ++i) + genUInt(col); + break; case TypeIndex::UInt32: + for (size_t i = 0; i < opts.size; ++i) + genUInt(col); + break; case TypeIndex::UInt64: for (size_t i = 0; i < opts.size; ++i) - genUInt(col); + genUInt(col); break; case TypeIndex::Int8: + for (size_t i = 0; i < opts.size; ++i) + genInt(col); + break; case TypeIndex::Int16: + for (size_t i = 0; i < opts.size; ++i) + genInt(col); + break; case TypeIndex::Int32: + for (size_t i = 0; i < opts.size; ++i) + genInt(col); + break; case TypeIndex::Int64: for (size_t i = 0; i < opts.size; ++i) - genInt(col); + genInt(col); break; case TypeIndex::Float32: case TypeIndex::Float64: @@ -178,9 +196,18 @@ String ColumnGenerator::randomDecimal(uint64_t prec, uint64_t scale) return s.substr(0, prec - scale) + "." + s.substr(prec - scale); } +template void ColumnGenerator::genInt(MutableColumnPtr & col) { - Field f = static_cast(rand_gen()); + static_assert(std::is_signed_v); + constexpr Int64 min_value = std::numeric_limits::min(); + constexpr Int64 max_value = std::numeric_limits::max(); + auto init_value = static_cast(rand_gen()); + if (init_value > max_value || init_value < min_value) + { + init_value = init_value % max_value; + } + Field f = init_value; col->insert(f); } @@ -198,9 +225,17 @@ void ColumnGenerator::genEnumValue(MutableColumnPtr & col, DataTypePtr & enum_ty col->insert(enum_value); } +template void ColumnGenerator::genUInt(MutableColumnPtr & col) { - Field f = static_cast(rand_gen()); + static_assert(std::is_unsigned_v); + constexpr UInt64 max_value = std::numeric_limits::max(); + auto init_value = static_cast(rand_gen()); + if (init_value > max_value) + { + init_value = init_value % max_value; + } + Field f = init_value; col->insert(f); } diff --git a/dbms/src/TestUtils/ColumnGenerator.h b/dbms/src/TestUtils/ColumnGenerator.h index 3edce5a40ec..fff794bc633 100644 --- a/dbms/src/TestUtils/ColumnGenerator.h +++ b/dbms/src/TestUtils/ColumnGenerator.h @@ -47,7 +47,8 @@ class ColumnGenerator : public ext::Singleton std::mt19937_64 rand_gen; std::uniform_int_distribution int_rand_gen = std::uniform_int_distribution(0, 128); std::uniform_real_distribution real_rand_gen; - const std::string charset{"0123456789ABCDEFGHIJKLMNOPQRSTUVWXYZabcdefghijklmnopqrstuvwxyz!@#$%^&*()、|【】[]{}「」;::;'‘,<《.>》。?·~`~"}; + /// todo support multibyte characters + const std::string charset{"0123456789ABCDEFGHIJKLMNOPQRSTUVWXYZabcdefghijklmnopqrstuvwxyz!@#$%^&*()|[]{}:;',<.>`~"}; String randomString(); int randomTimeOffset(); @@ -61,7 +62,9 @@ class ColumnGenerator : public ext::Singleton DataTypePtr createDecimalType(); void genBool(MutableColumnPtr & col); + template void genInt(MutableColumnPtr & col); + template void genUInt(MutableColumnPtr & col); void genFloat(MutableColumnPtr & col); void genString(MutableColumnPtr & col); diff --git a/dbms/src/TestUtils/FunctionTestUtils.cpp b/dbms/src/TestUtils/FunctionTestUtils.cpp index 6cf9387543d..43310601df7 100644 --- a/dbms/src/TestUtils/FunctionTestUtils.cpp +++ b/dbms/src/TestUtils/FunctionTestUtils.cpp @@ -98,6 +98,7 @@ ::testing::AssertionResult dataTypeEqual( ::testing::AssertionResult columnEqual( const ColumnPtr & expected, const ColumnPtr & actual, + const ICollator * collator, bool is_floating_point) { ASSERT_EQUAL(expected->getName(), actual->getName(), "Column name mismatch"); @@ -120,6 +121,14 @@ ::testing::AssertionResult columnEqual( if (!is_floating_point) { + if (collator != nullptr && !expected_field.isNull() && !actual_field.isNull()) + { + auto e_string = expected_field.get(); + auto a_string = actual_field.get(); + if (collator->compare(e_string.data(), e_string.size(), a_string.data(), a_string.size()) == 0) + continue; + /// if not equal, fallback to the original compare so we can reuse the code to get error message + } ASSERT_EQUAL_WITH_TEXT(expected_field, actual_field, fmt::format("Value at index {} mismatch", i), expected_field.toString(), actual_field.toString()); } else @@ -140,12 +149,13 @@ ::testing::AssertionResult columnEqual( ::testing::AssertionResult columnEqual( const ColumnWithTypeAndName & expected, - const ColumnWithTypeAndName & actual) + const ColumnWithTypeAndName & actual, + const ICollator * collator) { if (auto ret = dataTypeEqual(expected.type, actual.type); !ret) return ret; - return columnEqual(expected.column, actual.column, expected.type->isFloatingPoint()); + return columnEqual(expected.column, actual.column, collator, expected.type->isFloatingPoint()); } ::testing::AssertionResult blockEqual( diff --git a/dbms/src/TestUtils/FunctionTestUtils.h b/dbms/src/TestUtils/FunctionTestUtils.h index fad289043af..15b7748092a 100644 --- a/dbms/src/TestUtils/FunctionTestUtils.h +++ b/dbms/src/TestUtils/FunctionTestUtils.h @@ -586,12 +586,14 @@ ::testing::AssertionResult dataTypeEqual( ::testing::AssertionResult columnEqual( const ColumnPtr & expected, const ColumnPtr & actual, + const ICollator * collator = nullptr, bool is_floating_point = false); // ignore name ::testing::AssertionResult columnEqual( const ColumnWithTypeAndName & expected, - const ColumnWithTypeAndName & actual); + const ColumnWithTypeAndName & actual, + const ICollator * collator = nullptr); ::testing::AssertionResult blockEqual( const Block & expected, diff --git a/dbms/src/TestUtils/TiFlashStorageTestBasic.cpp b/dbms/src/TestUtils/TiFlashStorageTestBasic.cpp index f3a00259617..2a5798b0123 100644 --- a/dbms/src/TestUtils/TiFlashStorageTestBasic.cpp +++ b/dbms/src/TestUtils/TiFlashStorageTestBasic.cpp @@ -17,7 +17,6 @@ namespace DB::base { - void TiFlashStorageTestBasic::reload() { reload({}); diff --git a/dbms/src/TestUtils/TiFlashTestEnv.cpp b/dbms/src/TestUtils/TiFlashTestEnv.cpp index 670e9b65a5a..fa788a8f580 100644 --- a/dbms/src/TestUtils/TiFlashTestEnv.cpp +++ b/dbms/src/TestUtils/TiFlashTestEnv.cpp @@ -24,6 +24,7 @@ #include #include #include +#include #include #include #include @@ -183,6 +184,7 @@ ContextPtr TiFlashTestEnv::getContext(const DB::Settings & settings, Strings tes auto paths = getPathPool(testdata_path); context.setPathPool(paths.first, paths.second, Strings{}, context.getPathCapacity(), context.getFileProvider()); global_contexts[0]->initializeGlobalStoragePoolIfNeed(context.getPathPool()); + global_contexts[0]->tryReleaseWriteNodePageStorageForTest(); global_contexts[0]->initializeWriteNodePageStorageIfNeed(context.getPathPool()); context.getSettingsRef() = settings; return std::make_shared(context); @@ -237,32 +239,31 @@ FileProviderPtr TiFlashTestEnv::getMockFileProvider() return std::make_shared(std::make_shared(encryption_enabled), encryption_enabled); } -bool TiFlashTestEnv::createBucketIfNotExist(Aws::S3::S3Client & s3_client, const String & bucket) +bool TiFlashTestEnv::createBucketIfNotExist(::DB::S3::TiFlashS3Client & s3_client) { - auto log = Logger::get(); Aws::S3::Model::CreateBucketRequest request; - request.SetBucket(bucket); + request.SetBucket(s3_client.bucket()); auto outcome = s3_client.CreateBucket(request); if (outcome.IsSuccess()) { - LOG_DEBUG(log, "Created bucket {}", bucket); + LOG_DEBUG(s3_client.log, "Created bucket {}", s3_client.bucket()); } else if (outcome.GetError().GetExceptionName() == "BucketAlreadyOwnedByYou") { - LOG_DEBUG(log, "Bucket {} already exist", bucket); + LOG_DEBUG(s3_client.log, "Bucket {} already exist", s3_client.bucket()); } else { const auto & err = outcome.GetError(); - LOG_ERROR(log, "CreateBucket: {}:{}", err.GetExceptionName(), err.GetMessage()); + LOG_ERROR(s3_client.log, "CreateBucket: {}:{}", err.GetExceptionName(), err.GetMessage()); } return outcome.IsSuccess() || outcome.GetError().GetExceptionName() == "BucketAlreadyOwnedByYou"; } -void TiFlashTestEnv::deleteBucket(Aws::S3::S3Client & s3_client, const String & bucket) +void TiFlashTestEnv::deleteBucket(::DB::S3::TiFlashS3Client & s3_client) { Aws::S3::Model::DeleteBucketRequest request; - request.SetBucket(bucket); + request.SetBucket(s3_client.bucket()); s3_client.DeleteBucket(request); } diff --git a/dbms/src/TestUtils/TiFlashTestEnv.h b/dbms/src/TestUtils/TiFlashTestEnv.h index 1036f69369d..2cc5a94701a 100644 --- a/dbms/src/TestUtils/TiFlashTestEnv.h +++ b/dbms/src/TestUtils/TiFlashTestEnv.h @@ -24,16 +24,16 @@ #include #include -namespace Aws::S3 -{ -class S3Client; -} namespace DB { struct Settings; class DAGContext; class MockStorage; +namespace S3 +{ +class TiFlashS3Client; +} } // namespace DB namespace DB::tests @@ -105,9 +105,9 @@ class TiFlashTestEnv static FileProviderPtr getMockFileProvider(); - static bool createBucketIfNotExist(Aws::S3::S3Client & s3_client, const String & bucket); + static bool createBucketIfNotExist(::DB::S3::TiFlashS3Client & s3_client); - static void deleteBucket(Aws::S3::S3Client & s3_client, const String & bucket); + static void deleteBucket(::DB::S3::TiFlashS3Client & s3_client); TiFlashTestEnv() = delete; diff --git a/dbms/src/TestUtils/gtests_dbms_main.cpp b/dbms/src/TestUtils/gtests_dbms_main.cpp index 51a4b32462a..47a77671be0 100644 --- a/dbms/src/TestUtils/gtests_dbms_main.cpp +++ b/dbms/src/TestUtils/gtests_dbms_main.cpp @@ -14,7 +14,7 @@ #include #include -#include +#include #include #include #include @@ -77,10 +77,14 @@ int main(int argc, char ** argv) DB::tests::TiFlashTestEnv::getGlobalContext().getSettingsRef().dt_read_thread_count_scale); DB::DM::SegmentReadTaskScheduler::instance(); - DB::GlobalThreadPool::initialize(/*max_threads*/ 20, /*max_free_threds*/ 10, /*queue_size*/ 1000); - DB::IOThreadPool::initialize(/*max_threads*/ 20, /*max_free_threds*/ 10, /*queue_size*/ 1000); + DB::GlobalThreadPool::initialize(/*max_threads*/ 100, /*max_free_threds*/ 10, /*queue_size*/ 1000); + DB::S3FileCachePool::initialize(/*max_threads*/ 20, /*max_free_threds*/ 10, /*queue_size*/ 1000); + DB::DataStoreS3Pool::initialize(/*max_threads*/ 20, /*max_free_threds*/ 10, /*queue_size*/ 1000); + DB::RNRemoteReadTaskPool::initialize(/*max_threads*/ 20, /*max_free_threds*/ 10, /*queue_size*/ 1000); + DB::RNPagePreparerPool::initialize(/*max_threads*/ 20, /*max_free_threds*/ 10, /*queue_size*/ 1000); const auto s3_endpoint = Poco::Environment::get("S3_ENDPOINT", ""); - const auto s3_bucket = Poco::Environment::get("S3_BUCKET", "mock_bucket"); + const auto s3_bucket = Poco::Environment::get("S3_BUCKET", "mockbucket"); + const auto s3_root = Poco::Environment::get("S3_ROOT", "tiflash_ut/"); const auto access_key_id = Poco::Environment::get("AWS_ACCESS_KEY_ID", ""); const auto secret_access_key = Poco::Environment::get("AWS_SECRET_ACCESS_KEY", ""); const auto mock_s3 = Poco::Environment::get("MOCK_S3", "true"); // In unit-tests, use MockS3Client by default. @@ -89,6 +93,7 @@ int main(int argc, char ** argv) .bucket = s3_bucket, .access_key_id = access_key_id, .secret_access_key = secret_access_key, + .root = s3_root, }; Poco::Environment::set("AWS_EC2_METADATA_DISABLED", "true"); // disable to speedup testing DB::S3::ClientFactory::instance().init(s3config, mock_s3 == "true"); diff --git a/dbms/src/TestUtils/mockExecutor.h b/dbms/src/TestUtils/mockExecutor.h index b3fb5d522c8..0b18bd3b54f 100644 --- a/dbms/src/TestUtils/mockExecutor.h +++ b/dbms/src/TestUtils/mockExecutor.h @@ -261,6 +261,7 @@ MockWindowFrame buildDefaultRowsFrame(); #define Min(expr) makeASTFunction("min", (expr)) #define Count(expr) makeASTFunction("count", (expr)) #define Sum(expr) makeASTFunction("sum", (expr)) +#define CountDistinct(expr) makeASTFunction("countDistinct", (expr)) /// Window functions #define RowNumber() makeASTFunction("RowNumber") diff --git a/dbms/src/TiDB/Etcd/Client.cpp b/dbms/src/TiDB/Etcd/Client.cpp index 15ec15dc72c..91a51053e66 100644 --- a/dbms/src/TiDB/Etcd/Client.cpp +++ b/dbms/src/TiDB/Etcd/Client.cpp @@ -189,7 +189,8 @@ Client::campaign(const String & name, const String & value, LeaseID lease_id) req.set_lease(lease_id); grpc::ClientContext context; - context.set_deadline(std::chrono::system_clock::now() + timeout); + // usually use `campaign` blocks until become leader or error happens, + // don't set timeout. v3electionpb::CampaignResponse resp; auto status = leaderClient()->election_stub->Campaign(&context, req, &resp); diff --git a/dbms/src/TiDB/OwnerManager.cpp b/dbms/src/TiDB/OwnerManager.cpp index 162e9fe80c0..92ed8f89d89 100644 --- a/dbms/src/TiDB/OwnerManager.cpp +++ b/dbms/src/TiDB/OwnerManager.cpp @@ -247,6 +247,7 @@ void EtcdOwnerManager::camaignLoop(Etcd::SessionPtr session) const auto lease_id = session->leaseID(); LOG_DEBUG(log, "new campaign loop with lease_id={:x}", lease_id); + // Let this thread blocks until becone owner or error occurs auto && [new_leader, status] = client->campaign(campaign_name, id, lease_id); if (!status.ok()) { diff --git a/dbms/src/TiDB/Schema/SchemaBuilder.cpp b/dbms/src/TiDB/Schema/SchemaBuilder.cpp index f3e7105cbb8..71bbd32a337 100644 --- a/dbms/src/TiDB/Schema/SchemaBuilder.cpp +++ b/dbms/src/TiDB/Schema/SchemaBuilder.cpp @@ -1379,6 +1379,42 @@ void SchemaBuilder::syncAllSchema() LOG_INFO(log, "Loaded all schemas."); } +template +void SchemaBuilder::dropAllSchema() +{ + LOG_INFO(log, "Dropping all schemas."); + + auto & tmt_context = context.getTMTContext(); + + /// Drop all tables. + auto storage_map = tmt_context.getStorages().getAllStorage(); + for (const auto & storage : storage_map) + { + auto table_info = storage.second->getTableInfo(); + if (table_info.keyspace_id != keyspace_id) + { + continue; + } + applyDropPhysicalTable(storage.second->getDatabaseName(), table_info.id); + LOG_DEBUG(log, "Table {}.{} dropped during drop all schemas", storage.second->getDatabaseName(), name_mapper.debugTableName(table_info)); + } + + /// Drop all dbs. + const auto & dbs = context.getDatabases(); + for (const auto & db : dbs) + { + auto db_keyspace_id = SchemaNameMapper::getMappedNameKeyspaceID(db.first); + if (db_keyspace_id != keyspace_id) + { + continue; + } + applyDropSchema(db.first); + LOG_DEBUG(log, "DB {} dropped during drop all schemas", db.first); + } + + LOG_INFO(log, "Dropped all schemas."); +} + // product env template struct SchemaBuilder; // mock test diff --git a/dbms/src/TiDB/Schema/SchemaBuilder.h b/dbms/src/TiDB/Schema/SchemaBuilder.h index 4e8797e9504..de217df0292 100644 --- a/dbms/src/TiDB/Schema/SchemaBuilder.h +++ b/dbms/src/TiDB/Schema/SchemaBuilder.h @@ -51,6 +51,8 @@ struct SchemaBuilder void syncAllSchema(); + void dropAllSchema(); + private: void applyDropSchema(DatabaseID schema_id); diff --git a/dbms/src/TiDB/Schema/SchemaGetter.cpp b/dbms/src/TiDB/Schema/SchemaGetter.cpp index 636da7a4180..07807a418c9 100644 --- a/dbms/src/TiDB/Schema/SchemaGetter.cpp +++ b/dbms/src/TiDB/Schema/SchemaGetter.cpp @@ -199,7 +199,7 @@ Int64 SchemaGetter::getVersion() { String ver = TxnStructure::get(snap, schemaVersionKey); if (ver.empty()) - return 0; + return SchemaVersionNotExist; return std::stoll(ver); } diff --git a/dbms/src/TiDB/Schema/SchemaGetter.h b/dbms/src/TiDB/Schema/SchemaGetter.h index 8eb4644a19a..5666bcd75cc 100644 --- a/dbms/src/TiDB/Schema/SchemaGetter.h +++ b/dbms/src/TiDB/Schema/SchemaGetter.h @@ -129,6 +129,8 @@ struct SchemaDiff struct SchemaGetter { + static constexpr Int64 SchemaVersionNotExist = -1; + KeyspaceSnapshot snap; KeyspaceID keyspace_id; diff --git a/dbms/src/TiDB/Schema/SchemaSyncService.cpp b/dbms/src/TiDB/Schema/SchemaSyncService.cpp index 862165f3207..822fa7648cc 100644 --- a/dbms/src/TiDB/Schema/SchemaSyncService.cpp +++ b/dbms/src/TiDB/Schema/SchemaSyncService.cpp @@ -32,6 +32,9 @@ namespace ErrorCodes extern const int DEADLOCK_AVOIDED; } // namespace ErrorCodes +// TODO: make this interval configurable +constexpr size_t interval_seconds = 60; + SchemaSyncService::SchemaSyncService(DB::Context & context_) : context(context_) , background_pool(context_.getBackgroundPool()) @@ -45,7 +48,8 @@ SchemaSyncService::SchemaSyncService(DB::Context & context_) return false; }, - false); + false, + interval_seconds * 1000); } void SchemaSyncService::addKeyspaceGCTasks() @@ -62,12 +66,11 @@ void SchemaSyncService::addKeyspaceGCTasks() auto ks_log = log->getChild(fmt::format("keyspace={}", ks)); LOG_INFO(ks_log, "add sync schema task"); auto task_handle = background_pool.addTask( - [&, this, ks, ks_log] { + [&, this, ks, ks_log]() noexcept { String stage; bool done_anything = false; try { - LOG_DEBUG(ks_log, "auto sync schema", ks); /// Do sync schema first, then gc. /// They must be performed synchronously, /// otherwise table may get mis-GC-ed if RECOVER was not properly synced caused by schema sync pause but GC runs too aggressively. @@ -97,7 +100,8 @@ void SchemaSyncService::addKeyspaceGCTasks() } return false; }, - false); + false, + interval_seconds * 1000); ks_handle_map.emplace(ks, task_handle); } @@ -122,6 +126,8 @@ void SchemaSyncService::removeKeyspaceGCTasks() LOG_INFO(ks_log, "remove sync schema task"); background_pool.removeTask(ks_handle_iter->second); ks_handle_iter = ks_handle_map.erase(ks_handle_iter); + // remove schema version for this keyspace + removeCurrentVersion(ks); } } @@ -133,6 +139,7 @@ SchemaSyncService::~SchemaSyncService() auto task_handle = iter.second; background_pool.removeTask(task_handle); } + LOG_INFO(log, "SchemaSyncService stopped"); } bool SchemaSyncService::syncSchemas(KeyspaceID keyspace_id) @@ -140,6 +147,12 @@ bool SchemaSyncService::syncSchemas(KeyspaceID keyspace_id) return context.getTMTContext().getSchemaSyncer()->syncSchemas(context, keyspace_id); } + +void SchemaSyncService::removeCurrentVersion(KeyspaceID keyspace_id) +{ + context.getTMTContext().getSchemaSyncer()->removeCurrentVersion(keyspace_id); +} + template inline bool isSafeForGC(const DatabaseOrTablePtr & ptr, Timestamp gc_safe_point) { diff --git a/dbms/src/TiDB/Schema/SchemaSyncService.h b/dbms/src/TiDB/Schema/SchemaSyncService.h index 6ff5d3c7904..b0ce85140a5 100644 --- a/dbms/src/TiDB/Schema/SchemaSyncService.h +++ b/dbms/src/TiDB/Schema/SchemaSyncService.h @@ -45,6 +45,7 @@ class SchemaSyncService private: bool syncSchemas(KeyspaceID keyspace_id); + void removeCurrentVersion(KeyspaceID keyspace_id); struct GCContext { diff --git a/dbms/src/TiDB/Schema/SchemaSyncer.h b/dbms/src/TiDB/Schema/SchemaSyncer.h index 4c4d41d978a..6c0db6e4ffa 100644 --- a/dbms/src/TiDB/Schema/SchemaSyncer.h +++ b/dbms/src/TiDB/Schema/SchemaSyncer.h @@ -48,6 +48,11 @@ class SchemaSyncer */ virtual bool syncSchemas(Context & context, KeyspaceID keyspace_id) = 0; + /** + * Remove current version of CH schema. + */ + virtual void removeCurrentVersion(KeyspaceID keyspace_id) = 0; + virtual void reset() = 0; virtual TiDB::DBInfoPtr getDBInfoByName(const String & database_name) = 0; diff --git a/dbms/src/TiDB/Schema/TiDBSchemaSyncer.h b/dbms/src/TiDB/Schema/TiDBSchemaSyncer.h index f33beaa9361..5243e022668 100644 --- a/dbms/src/TiDB/Schema/TiDBSchemaSyncer.h +++ b/dbms/src/TiDB/Schema/TiDBSchemaSyncer.h @@ -102,6 +102,13 @@ struct TiDBSchemaSyncer : public SchemaSyncer return it->second; } + // After all tables have been physical dropped, remove the `cur_version` of given keyspace + void removeCurrentVersion(KeyspaceID keyspace_id) override + { + std::lock_guard lock(schema_mutex); + cur_versions.erase(keyspace_id); + } + bool syncSchemas(Context & context, KeyspaceID keyspace_id) override { std::lock_guard lock(schema_mutex); @@ -110,37 +117,57 @@ struct TiDBSchemaSyncer : public SchemaSyncer auto getter = createSchemaGetter(keyspace_id); Int64 version = getter.getVersion(); - if (version <= cur_version) - { - return false; - } + Stopwatch watch; SCOPE_EXIT({ GET_METRIC(tiflash_schema_apply_duration_seconds).Observe(watch.elapsedSeconds()); }); - LOG_INFO(ks_log, "Start to sync schemas. current version is: {} and try to sync schema version to: {}", cur_version, version); - // Show whether the schema mutex is held for a long time or not. GET_METRIC(tiflash_schema_applying).Set(1.0); SCOPE_EXIT({ GET_METRIC(tiflash_schema_applying).Set(0.0); }); - GET_METRIC(tiflash_schema_apply_count, type_diff).Increment(); - // After the feature concurrent DDL, TiDB does `update schema version` before `set schema diff`, and they are done in separate transactions. - // So TiFlash may see a schema version X but no schema diff X, meaning that the transaction of schema diff X has not been committed or has - // been aborted. - // However, TiDB makes sure that if we get a schema version X, then the schema diff X-1 must exist. Otherwise the transaction of schema diff - // X-1 is aborted and we can safely ignore it. - // Since TiDB can not make sure the schema diff of the latest schema version X is not empty, under this situation we should set the `cur_version` - // to X-1 and try to fetch the schema diff X next time. - Int64 version_after_load_diff = 0; - if (version_after_load_diff = tryLoadSchemaDiffs(getter, cur_version, version, context, ks_log); version_after_load_diff == -1) + // If the schema version not exists, drop all schemas. + if (version == SchemaGetter::SchemaVersionNotExist) { - GET_METRIC(tiflash_schema_apply_count, type_full).Increment(); - version_after_load_diff = loadAllSchema(getter, version, context); + // Tables and databases are already tombstoned and waiting for GC. + if (SchemaGetter::SchemaVersionNotExist == cur_version) + return false; + + LOG_INFO(ks_log, "Start to drop schemas. schema version key not exists, keyspace should be deleted"); + GET_METRIC(tiflash_schema_apply_count, type_drop_keyspace).Increment(); + + // The key range of the given keyspace is deleted by `UnsafeDestroyRange`, so the return result + // of `SchemaGetter::listDBs` is not reliable. Directly mark all databases and tables of this keyspace + // as a tombstone and let the SchemaSyncService drop them physically. + dropAllSchema(getter, context); + cur_versions[keyspace_id] = SchemaGetter::SchemaVersionNotExist; } - cur_versions[keyspace_id] = version_after_load_diff; + else + { + if (version <= cur_version) + return false; + + LOG_INFO(ks_log, "Start to sync schemas. current version is: {} and try to sync schema version to: {}", cur_version, version); + GET_METRIC(tiflash_schema_apply_count, type_diff).Increment(); + + // After the feature concurrent DDL, TiDB does `update schema version` before `set schema diff`, and they are done in separate transactions. + // So TiFlash may see a schema version X but no schema diff X, meaning that the transaction of schema diff X has not been committed or has + // been aborted. + // However, TiDB makes sure that if we get a schema version X, then the schema diff X-1 must exist. Otherwise the transaction of schema diff + // X-1 is aborted and we can safely ignore it. + // Since TiDB can not make sure the schema diff of the latest schema version X is not empty, under this situation we should set the `cur_version` + // to X-1 and try to fetch the schema diff X next time. + Int64 version_after_load_diff = 0; + if (version_after_load_diff = tryLoadSchemaDiffs(getter, cur_version, version, context, ks_log); version_after_load_diff == -1) + { + GET_METRIC(tiflash_schema_apply_count, type_full).Increment(); + version_after_load_diff = loadAllSchema(getter, version, context); + } + cur_versions[keyspace_id] = version_after_load_diff; + } + // TODO: (keyspace) attach keyspace id to the metrics. GET_METRIC(tiflash_schema_version).Set(cur_version); - LOG_INFO(ks_log, "End sync schema, version has been updated to {}{}", keyspace_id, cur_version, cur_version == version ? "" : "(latest diff is empty)"); + LOG_INFO(ks_log, "End sync schema, version has been updated to {}{}", cur_version, cur_version == version ? "" : "(latest diff is empty)"); return true; } @@ -278,6 +305,12 @@ struct TiDBSchemaSyncer : public SchemaSyncer builder.syncAllSchema(); return version; } + + void dropAllSchema(Getter & getter, Context & context) + { + SchemaBuilder builder(getter, context, databases, -1); + builder.dropAllSchema(); + } }; } // namespace DB diff --git a/docs/design/2023-03-13-tiflash-supports-spill-to-disk.md b/docs/design/2023-03-13-tiflash-supports-spill-to-disk.md new file mode 100644 index 00000000000..df5099b3d0e --- /dev/null +++ b/docs/design/2023-03-13-tiflash-supports-spill-to-disk.md @@ -0,0 +1,53 @@ +# TiFlash support spill to disk design doc + +* Author(s): [mengxin9014](https://github.com/mengxin9014), [windtalker](https://github.com/windtalker) +* Tracking Issue: + +## Table of Contents + +* [Motivation or Background](#motivation-or-background) +* [Detailed Design](#detailed-design) + * [Hash Join](#hash-join) + * [Hash Aggregation](#hash-agg) + * [Sort/TopN](#sort) +* [Impacts & Risks](#impacts-risks) + +## Motivation or Background +At present, TiFlash does not support the operation of spilling to disk. All computing operations are based on memory. However, in actual application scenarios, many users do not have enough available memory resources. As a result, many queries containing memory intensive operators (such as hash join, hash agg, etc.) cannot be executed in TiFlash at all. So it is imperative to support the operation of spilling to disk in TiFlash. + +## Detailed Design +In this design, we support 3 operators spilling to disk. + +### Hash Join +The hash join spilling algorithm is complex. We dispatch the build and probe data to n partitions by the join key, and the build and probe data in the same partition is the same unit. This way we can divide a large join into n smaller joins that have nothing to do with each other. +- #### Build Stage + When building a hash table for input data, if the memory usage exceeds the limit, the partition that needs to be spilled is selected by the related algorithm and all data in the partition is spilled to disk. For partitions that do not require spill, hash tables can be built normally. +- #### Probe Stage + For a partition that has been spilled to disk on the build side, data on the probe is also spilled. For a partition that has not been spilled to disk, the probe is executed normally. +- #### Restore Stage + Read the partition build and probe data spilled to disk and join them again. Note that some partitions in restore join operation may be spilled to disk again due to the memory usage exceeds limit. +- #### Overview + ![join_spill_overview](./images/2023-03-13-tiflash-supports-spill-to-disk-hash_join_with_spill.png) + +### Hash Aggregation +Each thread does operation of local aggregation with its own input data. If the memory exceeds the limit, each partition in the Hash table will be converted into an independent block and spilled to disk. +- If there is no data spilled to disk, merging the local aggregation based on the original algorithm. +- If some data is spilled to disk, spill all the in memory data to spill at the end, and doing memory efficient merge aggregation based on the spilled data +- #### Overview + ![agg_spill_overview](./images/2023-03-13-tiflash-supports-spill-to-disk-agg_with_spill.png) + +### Sort/TopN +- #### PartialSortingBlockInputStream + Not accumulating the memory, so we don't consider temporarily support spill operations +- #### MergeSortingBlockInputStream + 1. Constantly reading blocks and putting them into memory, if the memory usage exceeds limit, all existing blocks in memory are sorted globally and the sorted blocks are spilled to disk. + 2. Iterate through the first step until all the blocks have been read. + 3. Sort all blocks that are still in memory. + 4. Perform merge sort between the blocks in memory and the restore blocks that spilled to disk. +- #### Overview + ![agg_spill_overview](./images/2023-03-13-tiflash-supports-spill-to-disk-sort_with_spill.png) + +## Impacts & Risks +- For all operators, when the memory usage exceeds the limit, data will be spilled and restored, so the performance will be lower than that without spilling to disk. +- For hash join, even if spilling to disk does not occur after the memory limit is configured, the performance will be slightly lower than that without memory limit configured because the build side data always need to be partitioned. +- For hash join, the operation of spilling to disk is limited to three rounds at most. If the third round hash join still exceeds the memory limit, the memory limit will be ignored, and there is a risk of memory quota exceeded. diff --git a/docs/design/images/2023-03-13-tiflash-supports-spill-to-disk-agg_with_spill.png b/docs/design/images/2023-03-13-tiflash-supports-spill-to-disk-agg_with_spill.png new file mode 100644 index 00000000000..cd43a760483 Binary files /dev/null and b/docs/design/images/2023-03-13-tiflash-supports-spill-to-disk-agg_with_spill.png differ diff --git a/docs/design/images/2023-03-13-tiflash-supports-spill-to-disk-hash_join_with_spill.png b/docs/design/images/2023-03-13-tiflash-supports-spill-to-disk-hash_join_with_spill.png new file mode 100644 index 00000000000..85a03894753 Binary files /dev/null and b/docs/design/images/2023-03-13-tiflash-supports-spill-to-disk-hash_join_with_spill.png differ diff --git a/docs/design/images/2023-03-13-tiflash-supports-spill-to-disk-sort_with_spill.png b/docs/design/images/2023-03-13-tiflash-supports-spill-to-disk-sort_with_spill.png new file mode 100644 index 00000000000..8225d8462a3 Binary files /dev/null and b/docs/design/images/2023-03-13-tiflash-supports-spill-to-disk-sort_with_spill.png differ diff --git a/etc/config-template.toml b/etc/config-template.toml index a717491ec9a..4f8ae3763b3 100644 --- a/etc/config-template.toml +++ b/etc/config-template.toml @@ -107,9 +107,9 @@ # read_index_runner_count = 1 ## The minimum duration to handle read-index tasks in each worker. # read_index_worker_tick_ms = 10 -# disaggregated_mode = "tiflash_storage" or "tiflash_compute" -## Means whether we use AutoScaler or PD for tiflash_compute nodes. Default use AutoScaler. Will remove this after AutoScaler is stable. -# use_autoscaler = true +# disaggregated_mode = "tiflash_write" or "tiflash_compute" +## Means whether we use AutoScaler or PD for tiflash_compute nodes. Default is false. +# use_autoscaler = false [flash.proxy] # addr = "0.0.0.0:20170" diff --git a/tests/docker/config/tiflash_dt.toml b/tests/docker/config/tiflash_dt.toml index 14c7bb1c400..775105dbbc9 100644 --- a/tests/docker/config/tiflash_dt.toml +++ b/tests/docker/config/tiflash_dt.toml @@ -13,71 +13,15 @@ # limitations under the License. tmp_path = "/tmp/tiflash/data/tmp" -display_name = "TiFlash" -## Deprecated storage path setting style. Check [storage] section for new style. path = "/tmp/tiflash/data/db" capacity = "10737418240" -## Deprecated storage path setting style of multi-disks. Check [storage] section for new style. -# path = "/tmp/tiflash/data/db,/tmp/tiflash1,/tmp/tiflash2" -# capacity = "0" mark_cache_size = 1073741824 minmax_index_cache_size = 1073741824 tcp_port = 9000 http_port = 8123 -## Storage paths settings. -# [storage] -## The storage format version in storage engine. Valid values: 1, 2 (experimental). -## format_version = 1 - -## If there are multiple SSD disks on the machine, -## specify the path list on `storage.main.dir` can improve TiFlash performance. - -## If there are multiple disks with different IO metrics (e.g. one SSD and some HDDs) -## on the machine, -## set `storage.latest.dir` to store the latest data on SSD (disks with higher IOPS metrics) -## set `storage.main.dir` to store the main data on HDD (disks with lower IOPS metrics) -## can improve TiFlash performance. - -# [storage.main] -## The path to store main data. -# e.g. -# dir = [ "/data0/tiflash" ] -# or -# dir = [ "/data0/tiflash", "/data1/tiflash" ] - -## Store capacity of each path, i.e. max data size allowed. -## If it is not set, or is set to 0s, the actual disk capacity is used. -## Note that we don't support human-readable big numbers(like "10GB") yet. -## Please set in the specified number of bytes. -# e.g. -# capacity = [ 10737418240, 10737418240 ] - -# [storage.latest] -## The path(s) to store latest data. -## If not set, it will be the same with `storage.main.dir`. -# dir = [ ] - -## Store capacity of each path, i.e. max data size allowed. -## If it is not set, or is set to 0s, the actual disk capacity is used. -# e.g. -# capacity = [ 10737418240, 10737418240 ] - -# [storage.raft] -## The path(s) to store Raft data. -## If not set, it will be the paths in `storage.latest.dir` appended with "/kvstore". -# dir = [ ] - -# [storage.io_rate_limit] -## The max I/O bandwith. Default value is 0 and I/O rate limit is disabled. -# max_bytes_per_sec = 268435456 -## max_read_bytes_per_sec and max_write_bytes_per_sec are the same meaning as max_bytes_per_sec, -## but for disk that read bandwidth and write bandwith are calculated separatly, such as GCP's persistent disks. -# max_read_bytes_per_sec = 0 -# max_write_bytes_per_sec = 0 - [flash] service_addr = "0.0.0.0:3930" [flash.flash_cluster] @@ -105,16 +49,6 @@ runAsDaemon = true [raft] pd_addr = "pd0:2379" ignore_databases = "system,default" -# specify which storage engine we use. tmt or dt -storage_engine = "dt" -# Deprecated Raft data storage path setting style. Check [storage.raft] section for new style. -# If it is not set, it will be the first path of "path" appended with "/kvstore". -# kvstore_path = "" - -[raft.snapshot] -# The way to apply snapshot data -# The value is one of "block" / "file1" / "file2". -# method = "file1" [profiles] [profiles.default] diff --git a/tests/tidb-ci/new_collation_fullstack/function_collator.test b/tests/tidb-ci/new_collation_fullstack/function_collator.test index 8821de30bfb..ed2cfda4203 100644 --- a/tests/tidb-ci/new_collation_fullstack/function_collator.test +++ b/tests/tidb-ci/new_collation_fullstack/function_collator.test @@ -12,6 +12,9 @@ # See the License for the specific language governing permissions and # limitations under the License. +#TODO: Fix this test https://github.com/pingcap/tiflash/issues/7079 +#RETURN + mysql> drop table if exists test.t1 mysql> drop table if exists test.t2 mysql> create table test.t1(col_varchar_20_key_signed varchar(20) COLLATE utf8mb4_general_ci, col_varbinary_20_key_signed varbinary(20), col_varbinary_20_undef_signed varbinary(20));