diff --git a/CMakeLists.txt b/CMakeLists.txt index e17c9fc2aa86..1d07700d5bd2 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -93,7 +93,7 @@ else() endif() if( NOT DEFINED CMAKE_CXX_STANDARD ) - set(CMAKE_CXX_STANDARD 11) + set(CMAKE_CXX_STANDARD 20) endif() include(CMakeDependentOption) diff --git a/Makefile b/Makefile index 89b3b1cdd9e3..8121556a0f42 100644 --- a/Makefile +++ b/Makefile @@ -22,6 +22,8 @@ MACHINE ?= $(shell uname -m) ARFLAGS = ${EXTRA_ARFLAGS} rs STRIPFLAGS = -S -x +CXXFLAGS += -fcoroutines + # Transform parallel LOG output into something more readable. perl_command = perl -n \ -e '@a=split("\t",$$_,-1); $$t=$$a[8];' \ @@ -46,7 +48,7 @@ quoted_perl_command = $(subst ','\'',$(perl_command)) # `make install` # Set the default DEBUG_LEVEL to 1 -DEBUG_LEVEL?=1 +DEBUG_LEVEL?=2 # LIB_MODE says whether or not to use/build "shared" or "static" libraries. # Mode "static" means to link against static libraries (.a) @@ -69,7 +71,7 @@ else ifneq ($(filter shared_lib install-shared, $(MAKECMDGOALS)),) DEBUG_LEVEL=0 LIB_MODE=shared else ifneq ($(filter static_lib install-static, $(MAKECMDGOALS)),) - DEBUG_LEVEL=0 + DEBUG_LEVEL=2 LIB_MODE=static else ifneq ($(filter jtest rocksdbjava%, $(MAKECMDGOALS)),) OBJ_DIR=jl @@ -145,7 +147,7 @@ ifeq ($(DEBUG_LEVEL),0) OPT += -DNDEBUG ifneq ($(USE_RTTI), 1) - CXXFLAGS += -fno-rtti + # CXXFLAGS += -fno-rtti else CXXFLAGS += -DROCKSDB_USE_RTTI endif @@ -153,7 +155,7 @@ else ifneq ($(USE_RTTI), 0) CXXFLAGS += -DROCKSDB_USE_RTTI else - CXXFLAGS += -fno-rtti + # CXXFLAGS += -fno-rtti endif ifdef ASSERT_STATUS_CHECKED diff --git a/TARGETS b/TARGETS index c2aa60fdd9b2..60d1a2d4692c 100644 --- a/TARGETS +++ b/TARGETS @@ -149,6 +149,7 @@ cpp_library( "cache/lru_cache.cc", "cache/sharded_cache.cc", "db/arena_wrapped_db_iter.cc", + "db/async_future.cc", "db/blob/blob_fetcher.cc", "db/blob/blob_file_addition.cc", "db/blob/blob_file_builder.cc", @@ -483,6 +484,7 @@ cpp_library( "cache/lru_cache.cc", "cache/sharded_cache.cc", "db/arena_wrapped_db_iter.cc", + "db/async_future.cc", "db/blob/blob_fetcher.cc", "db/blob/blob_file_addition.cc", "db/blob/blob_file_builder.cc", diff --git a/build_tools/build_detect_platform b/build_tools/build_detect_platform index 102d89f0696e..18704a7fe389 100755 --- a/build_tools/build_detect_platform +++ b/build_tools/build_detect_platform @@ -49,7 +49,7 @@ fi if [ "$ROCKSDB_CXX_STANDARD" ]; then PLATFORM_CXXFLAGS="-std=$ROCKSDB_CXX_STANDARD" else - PLATFORM_CXXFLAGS="-std=c++11" + PLATFORM_CXXFLAGS="-std=c++20" fi # we currently depend on POSIX platform @@ -246,7 +246,7 @@ EOF Cygwin) PLATFORM=CYGWIN PLATFORM_SHARED_CFLAGS="" - PLATFORM_CXXFLAGS="-std=gnu++11" + PLATFORM_CXXFLAGS="-std=gnu++20" COMMON_FLAGS="$COMMON_FLAGS -DCYGWIN" if [ -z "$USE_CLANG" ]; then COMMON_FLAGS="$COMMON_FLAGS -fno-builtin-memcmp" diff --git a/db/arena_wrapped_db_iter.cc b/db/arena_wrapped_db_iter.cc index 20d4655bed2b..33f36dc3f662 100644 --- a/db/arena_wrapped_db_iter.cc +++ b/db/arena_wrapped_db_iter.cc @@ -92,7 +92,7 @@ Status ArenaWrappedDBIter::Refresh() { range_del_iter.reset( sv->mem->NewRangeTombstoneIterator(read_options_, latest_seq)); range_del_agg->AddTombstones(std::move(range_del_iter)); - cfd_->ReturnThreadLocalSuperVersion(sv); + cfd_->ReturnThreadLocalSuperVersion(db_impl_, sv); } // Refresh latest sequence number db_iter_->set_sequence(latest_seq); diff --git a/db/column_family.cc b/db/column_family.cc index 0d71ac388b89..bedecec923a2 100644 --- a/db/column_family.cc +++ b/db/column_family.cc @@ -438,6 +438,10 @@ SuperVersion* SuperVersion::Ref() { return this; } +uint32_t SuperVersion::GetRef() const { + return refs.load(); +} + bool SuperVersion::Unref() { // fetch_sub returns the previous value of ref uint32_t previous_refs = refs.fetch_sub(1); @@ -1200,7 +1204,7 @@ Compaction* ColumnFamilyData::CompactRange( SuperVersion* ColumnFamilyData::GetReferencedSuperVersion(DBImpl* db) { SuperVersion* sv = GetThreadLocalSuperVersion(db); sv->Ref(); - if (!ReturnThreadLocalSuperVersion(sv)) { + if (!ReturnThreadLocalSuperVersion(db, sv)) { // This Unref() corresponds to the Ref() in GetThreadLocalSuperVersion() // when the thread-local pointer was populated. So, the Ref() earlier in // this function still prevents the returned SuperVersion* from being @@ -1219,7 +1223,7 @@ SuperVersion* ColumnFamilyData::GetThreadLocalSuperVersion(DBImpl* db) { // local pointer to guarantee exclusive access. If the thread local pointer // is being used while a new SuperVersion is installed, the cached // SuperVersion can become stale. In that case, the background thread would - // have swapped in kSVObsolete. We re-check the value at when returning + // have swapped in kSVObsolete. We re-check the value when returning // SuperVersion back to thread local, with an atomic compare and swap. // The superversion will need to be released if detected to be stale. void* ptr = local_sv_->Swap(SuperVersion::kSVInUse); @@ -1228,7 +1232,11 @@ SuperVersion* ColumnFamilyData::GetThreadLocalSuperVersion(DBImpl* db) { // (2) the Swap above (always) installs kSVInUse, ThreadLocal storage // should only keep kSVInUse before ReturnThreadLocalSuperVersion call // (if no Scrape happens). - assert(ptr != SuperVersion::kSVInUse); + if (ptr == SuperVersion::kSVInUse) { + // FIXME: Check version number too. + return super_version_->Ref(); + } + SuperVersion* sv = static_cast(ptr); if (sv == SuperVersion::kSVObsolete || sv->version_number != super_version_number_.load()) { @@ -1259,22 +1267,42 @@ SuperVersion* ColumnFamilyData::GetThreadLocalSuperVersion(DBImpl* db) { return sv; } -bool ColumnFamilyData::ReturnThreadLocalSuperVersion(SuperVersion* sv) { +bool ColumnFamilyData::ReturnThreadLocalSuperVersion(DBImpl* db, SuperVersion* sv) { assert(sv != nullptr); + // Put the SuperVersion back void* expected = SuperVersion::kSVInUse; + if (local_sv_->CompareAndSwap(static_cast(sv), expected)) { // When we see kSVInUse in the ThreadLocal, we are sure ThreadLocal // storage has not been altered and no Scrape has happened. The // SuperVersion is still current. return true; + } else if (expected != nullptr) { + // A scrape has happened, we have to adjust the refs in both + // the super version and CFD that it refers to. + // FIXME: This assumes the SVs "release" are not interleaved. + auto ptr = static_cast(local_sv_->Swap(sv)); + if (sv != ptr) { + db->mutex()->Lock(); + assert(ptr != super_version_); + bool last_ref __attribute__((__unused__)); + last_ref = ptr->Unref(); + assert(last_ref); + refs_.fetch_sub(1); + ptr->Cleanup(); + db->mutex()->Unlock(); + // delete ptr; + } else { + sv->Unref(); + } + return true; } else { // ThreadLocal scrape happened in the process of this GetImpl call (after // thread local Swap() at the beginning and before CompareAndSwap()). // This means the SuperVersion it holds is obsolete. - assert(expected == SuperVersion::kSVObsolete); + return false; } - return false; } void ColumnFamilyData::InstallSuperVersion( @@ -1329,6 +1357,7 @@ void ColumnFamilyData::ResetThreadLocalSuperVersions() { continue; } auto sv = static_cast(ptr); + bool was_last_ref __attribute__((__unused__)); was_last_ref = sv->Unref(); // sv couldn't have been the last reference because diff --git a/db/column_family.h b/db/column_family.h index 2ab1f9e04589..4d2e3017b1f6 100644 --- a/db/column_family.h +++ b/db/column_family.h @@ -211,6 +211,9 @@ struct SuperVersion { // should be called outside the mutex SuperVersion() = default; ~SuperVersion(); + + uint32_t GetRef() const; + SuperVersion* Ref(); // If Unref() returns true, Cleanup() should be called with mutex held // before deleting this SuperVersion. @@ -274,7 +277,9 @@ class ColumnFamilyData { // Ref() can only be called from a context where the caller can guarantee // that ColumnFamilyData is alive (while holding a non-zero ref already, // holding a DB mutex, or as the leader in a write batch group). - void Ref() { refs_.fetch_add(1); } + void Ref() { + refs_.fetch_add(1); + } // UnrefAndTryDelete() decreases the reference count and do free if needed, // return true if this is freed else false, UnrefAndTryDelete() can only @@ -440,7 +445,7 @@ class ColumnFamilyData { // Try to return SuperVersion back to thread local storage. Return true on // success and false on failure. It fails when the thread local storage // contains anything other than SuperVersion::kSVInUse flag. - bool ReturnThreadLocalSuperVersion(SuperVersion* sv); + bool ReturnThreadLocalSuperVersion(DBImpl* db, SuperVersion* sv); // thread-safe uint64_t GetSuperVersionNumber() const { return super_version_number_.load(); diff --git a/db/db_basic_test.cc b/db/db_basic_test.cc index 70378f5bd090..f21cacc49d87 100644 --- a/db/db_basic_test.cc +++ b/db/db_basic_test.cc @@ -7,6 +7,8 @@ // Use of this source code is governed by a BSD-style license that can be // found in the LICENSE file. See the AUTHORS file for names of contributors. +#include + #include #include "db/db_test_util.h" @@ -3915,6 +3917,222 @@ TEST_P(DBBasicTestDeadline, IteratorDeadline) { Close(); } +using s_io_uring = struct io_uring; +class DBBasicTestWithAsyncIO : public DBAsyncTestBase { + public: + DBBasicTestWithAsyncIO() + : DBAsyncTestBase("db_basic_asyncio_test"), + io_uring_{new s_io_uring}, + shutDown_{false} { + auto ret = io_uring_queue_init(io_uring_size_, io_uring_.get(), 0); + if (ret < 0) throw "io_uring_queue_init failed"; + } + + ~DBBasicTestWithAsyncIO() { + shutDown_.store(true, std::memory_order_relaxed); + std::this_thread::sleep_for(std::chrono::milliseconds(500)); + io_uring_queue_exit(io_uring_.get()); + } + + s_io_uring* io_uring() { return this->io_uring_.get(); } + + void shutdown() { shutDown_ = true; } + + bool RunAsyncTest(std::function test_func) { + bool success = false; + + auto r = test_func(this); + (void)r; + + struct io_uring_cqe* cqe; + // struct __kernel_timespec ts; + + // msec_to_ts(&ts, 100); + while (true) { + auto ret = io_uring_wait_cqe(io_uring_.get(), &cqe); + if (ret != 0) { + continue; + } + + if (ret == 0 && cqe->res >= 0) { + Async_future::IO_ctx* rdata = (Async_future::IO_ctx*)io_uring_cqe_get_data(cqe); + + OnResume(rdata->m_promise); + io_uring_cqe_seen(io_uring_.get(), cqe); + + if (shutDown_.load(std::memory_order_relaxed)) break; + } + } + return success; + } + + private: + static void OnResume(Async_future::promise_type* promise) { + auto h = std::coroutine_handle::from_promise( + *promise); + h.resume(); + } + + static void msec_to_ts(struct __kernel_timespec* ts, unsigned int msec) { + ts->tv_sec = msec / 1000; + ts->tv_nsec = (msec % 1000) * 1000000; + } + + std::unique_ptr io_uring_; + const int io_uring_size_ = 1024; + std::atomic shutDown_; +}; + +static Async_future SimpleAsyncGetTest(DBAsyncTestBase* testBase) { + auto io_uring = dynamic_cast(testBase)->io_uring(); + auto submit_queue = + testBase->test_delegation() + ? std::make_shared( + [io_uring](Async_future::IO_ctx* data, int fd, uint64_t offset, + Async_future::Submit_queue::Ops op) -> Async_future { + (void)op; + + Async_future a_result(true, data); + auto sqe = io_uring_get_sqe(io_uring); + if (sqe == nullptr) { + // submission queue is full + co_return rocksdb::IOStatus::IOError( + rocksdb::Status::SubCode::kIOUringSqeFull); + } + + io_uring_prep_readv(sqe, fd, data->m_iov.data(), data->m_iov.size(), offset); + io_uring_sqe_set_data(sqe, data); + auto ret = io_uring_submit(io_uring); + if (ret < 0) { + co_return rocksdb::IOStatus::IOError( + rocksdb::Status::SubCode::kIOUringSubmitError, + strerror(-ret)); + } + + co_await a_result; + co_return rocksdb::IOStatus::OK(); + }) + : std::make_shared( + dynamic_cast(testBase)->io_uring()); + ReadOptions options; + options.submit_queue = submit_queue; + options.read_tier = kPersistedTier; + options.verify_checksums = true; + PinnableSlice* v = new PinnableSlice(); + auto asyncResult = testBase->db()->AsyncGet( + options, testBase->db()->DefaultColumnFamily(), "bar", v, nullptr); + co_await asyncResult; + dynamic_cast(testBase)->shutdown(); + + auto r = v->ToString(); + delete v; + if (r == "e1") { + co_return Status::OK(); + } else { + co_return Status::NotFound(); + } +} + +static Async_future SimpleAsyncMultiGetTest(DBAsyncTestBase* testBase) { + auto io_uring = dynamic_cast(testBase)->io_uring(); + auto submit_queue = + testBase->test_delegation() + ? std::make_shared( + [io_uring](Async_future::IO_ctx* data, int fd, uint64_t offset, + Async_future::Submit_queue::Ops op) -> Async_future { + (void)op; + + Async_future a_result(true, data); + auto sqe = io_uring_get_sqe(io_uring); + if (sqe == nullptr) { + // submission queue is full + co_return rocksdb::IOStatus::IOError( + rocksdb::Status::SubCode::kIOUringSqeFull); + } + + io_uring_prep_readv(sqe, fd, data->m_iov.data(), data->m_iov.size(), offset); + io_uring_sqe_set_data(sqe, data); + auto ret = io_uring_submit(io_uring); + if (ret < 0) { + co_return rocksdb::IOStatus::IOError( + rocksdb::Status::SubCode::kIOUringSubmitError, + strerror(-ret)); + } + + co_await a_result; + co_return rocksdb::IOStatus::OK(); + }) + : std::make_shared( + dynamic_cast(testBase)->io_uring()); + ReadOptions options; + options.submit_queue = submit_queue; + options.read_tier = kPersistedTier; + options.verify_checksums = true; + std::vector values; + std::vector keys = {"bar", "foo"}; + auto asyncResult = testBase->db()->AsyncMultiGet(options, keys, &values); + co_await asyncResult; + (void)keys; // hold keys after coroutine + dynamic_cast(testBase)->shutdown(); + + auto statuses = asyncResult.statuses(); + + if (statuses.size() != 2 || values.size() != 2) { + co_return Status::NotFound(); + } + + auto r = values[0]; + if (r != "e1") { + co_return Status::NotFound(); + } + + r = values[1]; + if (r != "f2") { + co_return Status::NotFound(); + } + co_return Status::OK(); +} + +TEST_F(DBBasicTestWithAsyncIO, AsyncGet) { + WriteOptions wo; + wo.disableWAL = true; + auto s = this->db()->Put(wo, "bar", "e1"); + s = this->db()->Flush(FlushOptions()); + + this->RunAsyncTest(SimpleAsyncGetTest); +} + +TEST_F(DBBasicTestWithAsyncIO, AsyncDeletgateGet) { + WriteOptions wo; + wo.disableWAL = true; + auto s = this->db()->Put(wo, "bar", "e1"); + s = this->db()->Flush(FlushOptions()); + this->set_test_delegation(true); + this->RunAsyncTest(SimpleAsyncGetTest); + this->set_test_delegation(false); +} + +TEST_F(DBBasicTestWithAsyncIO, AsyncMultiGet) { + WriteOptions wo; + wo.disableWAL = true; + auto s = this->db()->Put(wo, "bar", "e1"); + s = this->db()->Put(wo, "foo", "f2"); + s = this->db()->Flush(FlushOptions()); + + this->RunAsyncTest(SimpleAsyncMultiGetTest); +} + +TEST_F(DBBasicTestWithAsyncIO, AsyncDeletgateMultiGet) { + WriteOptions wo; + wo.disableWAL = true; + auto s = this->db()->Put(wo, "bar", "e1"); + s = this->db()->Put(wo, "foo", "f2"); + s = this->db()->Flush(FlushOptions()); + this->set_test_delegation(true); + this->RunAsyncTest(SimpleAsyncMultiGetTest); + this->set_test_delegation(false); +} + // Param 0: If true, set read_options.deadline // Param 1: If true, set read_options.io_timeout INSTANTIATE_TEST_CASE_P(DBBasicTestDeadline, DBBasicTestDeadline, diff --git a/db/db_impl/db_impl.cc b/db/db_impl/db_impl.cc index 34aa65e90cf4..6d565ae82d90 100644 --- a/db/db_impl/db_impl.cc +++ b/db/db_impl/db_impl.cc @@ -1333,11 +1333,11 @@ Status DBImpl::FlushWAL(bool sync) { // future writes IOStatusCheck(io_s); // whether sync or not, we should abort the rest of function upon error - return std::move(io_s); + return io_s; } if (!sync) { ROCKS_LOG_DEBUG(immutable_db_options_.info_log, "FlushWAL sync=false"); - return std::move(io_s); + return io_s; } } if (!sync) { @@ -1348,6 +1348,41 @@ Status DBImpl::FlushWAL(bool sync) { return SyncWAL(); } +Async_future DBImpl::AsyncFlushWAL(bool sync) { + if (manual_wal_flush_) { + IOStatus io_s; + { + // We need to lock log_write_mutex_ since logs_ might change concurrently + InstrumentedMutexLock wl(&log_write_mutex_); + log::Writer* cur_log_writer = logs_.back().writer; + auto result = cur_log_writer->AsyncWriteBuffer(); + co_await result; + io_s = result.io_result(); + } + if (!io_s.ok()) { + ROCKS_LOG_ERROR(immutable_db_options_.info_log, "WAL flush error %s", + io_s.ToString().c_str()); + // In case there is a fs error we should set it globally to prevent the + // future writes + IOStatusCheck(io_s); + // whether sync or not, we should abort the rest of function upon error + co_return std::move(io_s); + } + if (!sync) { + ROCKS_LOG_DEBUG(immutable_db_options_.info_log, "FlushWAL sync=false"); + co_return std::move(io_s); + } + } + if (!sync) { + co_return Status::OK(); + } + // sync = true + ROCKS_LOG_DEBUG(immutable_db_options_.info_log, "FlushWAL sync=true"); + auto result = AsSyncWAL(); + co_await result; + co_return result.status(); +} + Status DBImpl::SyncWAL() { TEST_SYNC_POINT("DBImpl::SyncWAL:Begin"); autovector logs_to_sync; @@ -1443,6 +1478,92 @@ Status DBImpl::ApplyWALToManifest(VersionEdit* synced_wals) { return status; } +Async_future DBImpl::AsSyncWAL() { + autovector logs_to_sync; + bool need_log_dir_sync; + uint64_t current_log_number; + + { + InstrumentedMutexLock l(&mutex_); + assert(!logs_.empty()); + + // This SyncWAL() call only cares about logs up to this number. + current_log_number = logfile_number_; + + while (logs_.front().number <= current_log_number && + logs_.front().getting_synced) { + log_sync_cv_.Wait(); + } + // First check that logs are safe to sync in background. + for (auto it = logs_.begin(); + it != logs_.end() && it->number <= current_log_number; ++it) { + if (!it->writer->file()->writable_file()->IsSyncThreadSafe()) { + co_return Status::NotSupported( + "SyncWAL() is not supported for this implementation of WAL file", + immutable_db_options_.allow_mmap_writes + ? "try setting Options::allow_mmap_writes to false" + : Slice()); + } + } + for (auto it = logs_.begin(); + it != logs_.end() && it->number <= current_log_number; ++it) { + auto& log = *it; + assert(!log.getting_synced); + log.getting_synced = true; + logs_to_sync.push_back(log.writer); + } + + need_log_dir_sync = !log_dir_synced_; + } + + TEST_SYNC_POINT("DBWALTest::SyncWALNotWaitWrite:1"); + RecordTick(stats_, WAL_FILE_SYNCED); + Status status; + IOStatus io_s; + for (log::Writer* log : logs_to_sync) { + auto result = + log->file()->AsSyncWithoutFlush(immutable_db_options_.use_fsync); + co_await result; + io_s = result.io_result(); + if (!io_s.ok()) { + status = io_s; + break; + } + } + if (!io_s.ok()) { + ROCKS_LOG_ERROR(immutable_db_options_.info_log, "WAL Sync error %s", + io_s.ToString().c_str()); + // In case there is a fs error we should set it globally to prevent the + // future writes + IOStatusCheck(io_s); + } + if (status.ok() && need_log_dir_sync) { + status = directories_.GetWalDir()->Fsync(IOOptions(), nullptr); + } + TEST_SYNC_POINT("DBWALTest::SyncWALNotWaitWrite:2"); + + TEST_SYNC_POINT("DBImpl::SyncWAL:BeforeMarkLogsSynced:1"); + + VersionEdit synced_wals; + { + InstrumentedMutexLock l(&mutex_); + if (status.ok()) { + MarkLogsSynced(current_log_number, need_log_dir_sync, &synced_wals); + } else { + MarkLogsNotSynced(current_log_number); + } + } + + if (status.ok() && synced_wals.IsWalAddition()) { + InstrumentedMutexLock l(&mutex_); + status = ApplyWALToManifest(&synced_wals); + } + + TEST_SYNC_POINT("DBImpl::SyncWAL:BeforeMarkLogsSynced:2"); + + co_return status; +} + Status DBImpl::LockWAL() { log_write_mutex_.Lock(); auto cur_log_writer = logs_.back().writer; @@ -1454,7 +1575,7 @@ Status DBImpl::LockWAL() { // future writes WriteStatusCheck(status); } - return std::move(status); + return status; } Status DBImpl::UnlockWAL() { @@ -1731,12 +1852,24 @@ Status DBImpl::Get(const ReadOptions& read_options, Status DBImpl::Get(const ReadOptions& read_options, ColumnFamilyHandle* column_family, const Slice& key, PinnableSlice* value, std::string* timestamp) { - GetImplOptions get_impl_options; - get_impl_options.column_family = column_family; - get_impl_options.value = value; - get_impl_options.timestamp = timestamp; - Status s = GetImpl(read_options, key, get_impl_options); - return s; + GetImplOptions get_impl_options = { + .column_family = column_family, + .value = value, + .timestamp = timestamp}; + return GetImpl(read_options, key, get_impl_options); +} + +Async_future DBImpl::AsyncGet(const ReadOptions& read_options, + ColumnFamilyHandle* column_family, + const Slice& key, PinnableSlice* value, + std::string* timestamp) { + GetImplOptions get_impl_options = { + .column_family = column_family, + .value = value, + .timestamp = timestamp}; + auto result = AsyncGetImpl(read_options, key, get_impl_options); + co_await result; + co_return result.status(); } namespace { @@ -1956,6 +2089,214 @@ Status DBImpl::GetImpl(const ReadOptions& read_options, const Slice& key, return s; } +Async_future DBImpl::AsyncGetImpl(const ReadOptions& read_options, const Slice& key, + GetImplOptions& get_impl_options) { + assert(get_impl_options.value != nullptr || + get_impl_options.merge_operands != nullptr); + + assert(get_impl_options.column_family); + + if (read_options.timestamp) { + const Status s = FailIfTsSizesMismatch(get_impl_options.column_family, + *(read_options.timestamp)); + if (!s.ok()) { + co_return s; + } + } else { + const Status s = FailIfCfHasTs(get_impl_options.column_family); + if (!s.ok()) { + co_return s; + } + } + + GetWithTimestampReadCallback read_cb(0); // Will call Refresh + + PERF_CPU_TIMER_GUARD(get_cpu_nanos, immutable_db_options_.clock); + StopWatch sw(immutable_db_options_.clock, stats_, DB_GET); + PERF_TIMER_GUARD(get_snapshot_time); + + auto cfh = static_cast_with_check( + get_impl_options.column_family); + auto cfd = cfh->cfd(); + + if (tracer_) { + // TODO: This mutex should be removed later, to improve performance when + // tracing is enabled. + InstrumentedMutexLock lock(&trace_mutex_); + if (tracer_) { + // TODO: maybe handle the tracing status? + tracer_->Get(get_impl_options.column_family, key).PermitUncheckedError(); + } + } + + // Acquire SuperVersion + SuperVersion* sv = GetAndRefSuperVersion(cfd); + + TEST_SYNC_POINT("DBImpl::GetImpl:1"); + TEST_SYNC_POINT("DBImpl::GetImpl:2"); + + SequenceNumber snapshot; + if (read_options.snapshot != nullptr) { + if (get_impl_options.callback) { + // Already calculated based on read_options.snapshot + snapshot = get_impl_options.callback->max_visible_seq(); + } else { + snapshot = + reinterpret_cast(read_options.snapshot)->number_; + } + } else { + // Note that the snapshot is assigned AFTER referencing the super + // version because otherwise a flush happening in between may compact away + // data for the snapshot, so the reader would see neither data that was be + // visible to the snapshot before compaction nor the newer data inserted + // afterwards. + if (last_seq_same_as_publish_seq_) { + snapshot = versions_->LastSequence(); + } else { + snapshot = versions_->LastPublishedSequence(); + } + if (get_impl_options.callback) { + // The unprep_seqs are not published for write unprepared, so it could be + // that max_visible_seq is larger. Seek to the std::max of the two. + // However, we still want our callback to contain the actual snapshot so + // that it can do the correct visibility filtering. + get_impl_options.callback->Refresh(snapshot); + + // Internally, WriteUnpreparedTxnReadCallback::Refresh would set + // max_visible_seq = max(max_visible_seq, snapshot) + // + // Currently, the commented out assert is broken by + // InvalidSnapshotReadCallback, but if write unprepared recovery followed + // the regular transaction flow, then this special read callback would not + // be needed. + // + // assert(callback->max_visible_seq() >= snapshot); + snapshot = get_impl_options.callback->max_visible_seq(); + } + } + // If timestamp is used, we use read callback to ensure is returned + // only if t <= read_opts.timestamp and s <= snapshot. + // HACK: temporarily overwrite input struct field but restore + SaveAndRestore restore_callback(&get_impl_options.callback); + const Comparator* ucmp = get_impl_options.column_family->GetComparator(); + assert(ucmp); + if (ucmp->timestamp_size() > 0) { + assert(!get_impl_options + .callback); // timestamp with callback is not supported + read_cb.Refresh(snapshot); + get_impl_options.callback = &read_cb; + } + TEST_SYNC_POINT("DBImpl::GetImpl:3"); + TEST_SYNC_POINT("DBImpl::GetImpl:4"); + + // Prepare to store a list of merge operations if merge occurs. + MergeContext merge_context; + SequenceNumber max_covering_tombstone_seq = 0; + + Status s; + // First look in the memtable, then in the immutable memtable (if any). + // s is both in/out. When in, s could either be OK or MergeInProgress. + // merge_operands will contain the sequence of merges in the latter case. + LookupKey lkey(key, snapshot, read_options.timestamp); + PERF_TIMER_STOP(get_snapshot_time); + + bool skip_memtable = (read_options.read_tier == kPersistedTier && + has_unpersisted_data_.load(std::memory_order_relaxed)); + bool done = false; + std::string* timestamp = + ucmp->timestamp_size() > 0 ? get_impl_options.timestamp : nullptr; + if (!skip_memtable) { + // Get value associated with key + if (get_impl_options.get_value) { + if (sv->mem->Get(lkey, get_impl_options.value->GetSelf(), timestamp, &s, + &merge_context, &max_covering_tombstone_seq, + read_options, get_impl_options.callback, + get_impl_options.is_blob_index)) { + done = true; + get_impl_options.value->PinSelf(); + RecordTick(stats_, MEMTABLE_HIT); + } else if ((s.ok() || s.IsMergeInProgress()) && + sv->imm->Get(lkey, get_impl_options.value->GetSelf(), + timestamp, &s, &merge_context, + &max_covering_tombstone_seq, read_options, + get_impl_options.callback, + get_impl_options.is_blob_index)) { + done = true; + get_impl_options.value->PinSelf(); + RecordTick(stats_, MEMTABLE_HIT); + } + } else { + // Get Merge Operands associated with key, Merge Operands should not be + // merged and raw values should be returned to the user. + if (sv->mem->Get(lkey, /*value*/ nullptr, /*timestamp=*/nullptr, &s, + &merge_context, &max_covering_tombstone_seq, + read_options, nullptr, nullptr, false)) { + done = true; + RecordTick(stats_, MEMTABLE_HIT); + } else if ((s.ok() || s.IsMergeInProgress()) && + sv->imm->GetMergeOperands(lkey, &s, &merge_context, + &max_covering_tombstone_seq, + read_options)) { + done = true; + RecordTick(stats_, MEMTABLE_HIT); + } + } + if (!done && !s.ok() && !s.IsMergeInProgress()) { + ReturnAndCleanupSuperVersion(cfd, sv); + co_return s; + } + } + PinnedIteratorsManager pinned_iters_mgr; + if (!done) { + PERF_TIMER_GUARD(get_from_output_files_time); + auto result = sv->current->AsyncGet( + read_options, lkey, get_impl_options.value, timestamp, &s, + &merge_context, &max_covering_tombstone_seq, &pinned_iters_mgr, + get_impl_options.get_value ? get_impl_options.value_found : nullptr, + nullptr, nullptr, + get_impl_options.get_value ? get_impl_options.callback : nullptr, + get_impl_options.get_value ? get_impl_options.is_blob_index : nullptr, + get_impl_options.get_value); + co_await result; + (void) result; // hold async result after wait + RecordTick(stats_, MEMTABLE_MISS); + } + + { + PERF_TIMER_GUARD(get_post_process_time); + + ReturnAndCleanupSuperVersion(cfd, sv); + + RecordTick(stats_, NUMBER_KEYS_READ); + size_t size = 0; + if (s.ok()) { + if (get_impl_options.get_value) { + size = get_impl_options.value->size(); + } else { + // Return all merge operands for get_impl_options.key + *get_impl_options.number_of_operands = + static_cast(merge_context.GetNumOperands()); + if (*get_impl_options.number_of_operands > + get_impl_options.get_merge_operands_options + ->expected_max_number_of_operands) { + s = Status::Incomplete( + Status::SubCode::KMergeOperandsInsufficientCapacity); + } else { + for (const Slice& sl : merge_context.GetOperands()) { + size += sl.size(); + get_impl_options.merge_operands->PinSelf(sl); + get_impl_options.merge_operands++; + } + } + } + RecordTick(stats_, BYTES_READ, size); + PERF_COUNTER_ADD(get_read_bytes, size); + } + RecordInHistogram(stats_, BYTES_PER_READ, size); + } + co_return s; +} + std::vector DBImpl::MultiGet( const ReadOptions& read_options, const std::vector& column_family, @@ -2166,6 +2507,195 @@ std::vector DBImpl::MultiGet( return stat_list; } +Async_future DBImpl::AsyncMultiGet( + const ReadOptions& read_options, + const std::vector& column_family, + const std::vector& keys, std::vector* values, + std::vector* timestamps) { + PERF_CPU_TIMER_GUARD(get_cpu_nanos, immutable_db_options_.clock); + StopWatch sw(immutable_db_options_.clock, stats_, DB_MULTIGET); + PERF_TIMER_GUARD(get_snapshot_time); + +#ifndef NDEBUG + for (const auto* cfh : column_family) { + assert(cfh); + const Comparator* const ucmp = cfh->GetComparator(); + assert(ucmp); + if (ucmp->timestamp_size() > 0) { + assert(read_options.timestamp); + assert(ucmp->timestamp_size() == read_options.timestamp->size()); + } else { + assert(!read_options.timestamp); + } + } +#endif // NDEBUG + + if (tracer_) { + // TODO: This mutex should be removed later, to improve performance when + // tracing is enabled. + InstrumentedMutexLock lock(&trace_mutex_); + if (tracer_) { + // TODO: maybe handle the tracing status? + tracer_->MultiGet(column_family, keys).PermitUncheckedError(); + } + } + + SequenceNumber consistent_seqnum; + + std::unordered_map multiget_cf_data( + column_family.size()); + for (auto cf : column_family) { + auto cfh = static_cast_with_check(cf); + auto cfd = cfh->cfd(); + if (multiget_cf_data.find(cfd->GetID()) == multiget_cf_data.end()) { + multiget_cf_data.emplace(cfd->GetID(), + MultiGetColumnFamilyData(cfh, nullptr)); + } + } + + std::function::iterator&)> + iter_deref_lambda = + [](std::unordered_map::iterator& + cf_iter) { return &cf_iter->second; }; + + bool unref_only = + MultiCFSnapshot>( + read_options, nullptr, iter_deref_lambda, &multiget_cf_data, + &consistent_seqnum); + + TEST_SYNC_POINT("DBImpl::MultiGet:AfterGetSeqNum1"); + TEST_SYNC_POINT("DBImpl::MultiGet:AfterGetSeqNum2"); + + // Contain a list of merge operations if merge occurs. + MergeContext merge_context; + + // Note: this always resizes the values array + size_t num_keys = keys.size(); + std::vector stat_list(num_keys); + values->resize(num_keys); + if (timestamps) { + timestamps->resize(num_keys); + } + + // Keep track of bytes that we read for statistics-recording later + uint64_t bytes_read = 0; + PERF_TIMER_STOP(get_snapshot_time); + + // For each of the given keys, apply the entire "get" process as follows: + // First look in the memtable, then in the immutable memtable (if any). + // s is both in/out. When in, s could either be OK or MergeInProgress. + // merge_operands will contain the sequence of merges in the latter case. + size_t num_found = 0; + size_t keys_read; + uint64_t curr_value_size = 0; + + GetWithTimestampReadCallback timestamp_read_callback(0); + ReadCallback* read_callback = nullptr; + if (read_options.timestamp && read_options.timestamp->size() > 0) { + timestamp_read_callback.Refresh(consistent_seqnum); + read_callback = ×tamp_read_callback; + } + + for (keys_read = 0; keys_read < num_keys; ++keys_read) { + merge_context.Clear(); + Status& s = stat_list[keys_read]; + std::string* value = &(*values)[keys_read]; + std::string* timestamp = timestamps ? &(*timestamps)[keys_read] : nullptr; + + LookupKey lkey(keys[keys_read], consistent_seqnum, read_options.timestamp); + auto cfh = static_cast_with_check( + column_family[keys_read]); + SequenceNumber max_covering_tombstone_seq = 0; + auto mgd_iter = multiget_cf_data.find(cfh->cfd()->GetID()); + assert(mgd_iter != multiget_cf_data.end()); + auto mgd = mgd_iter->second; + auto super_version = mgd.super_version; + bool skip_memtable = + (read_options.read_tier == kPersistedTier && + has_unpersisted_data_.load(std::memory_order_relaxed)); + bool done = false; + if (!skip_memtable) { + if (super_version->mem->Get(lkey, value, timestamp, &s, &merge_context, + &max_covering_tombstone_seq, read_options, + read_callback)) { + done = true; + RecordTick(stats_, MEMTABLE_HIT); + } else if (super_version->imm->Get(lkey, value, timestamp, &s, + &merge_context, + &max_covering_tombstone_seq, + read_options, read_callback)) { + done = true; + RecordTick(stats_, MEMTABLE_HIT); + } + } + if (!done) { + PinnableSlice pinnable_val; + PERF_TIMER_GUARD(get_from_output_files_time); + PinnedIteratorsManager pinned_iters_mgr; + auto r = super_version->current->AsyncGet( + read_options, lkey, &pinnable_val, timestamp, &s, &merge_context, + &max_covering_tombstone_seq, + &pinned_iters_mgr, /*value_found=*/nullptr, + /*key_exists=*/nullptr, + /*seq=*/nullptr, read_callback); + co_await r; + (void)r; // hold Async_future after await + value->assign(pinnable_val.data(), pinnable_val.size()); + RecordTick(stats_, MEMTABLE_MISS); + } + + if (s.ok()) { + bytes_read += value->size(); + num_found++; + curr_value_size += value->size(); + if (curr_value_size > read_options.value_size_soft_limit) { + while (++keys_read < num_keys) { + stat_list[keys_read] = Status::Aborted(); + } + break; + } + } + if (read_options.deadline.count() && + immutable_db_options_.clock->NowMicros() > + static_cast(read_options.deadline.count())) { + break; + } + } + + if (keys_read < num_keys) { + // The only reason to break out of the loop is when the deadline is + // exceeded + assert(immutable_db_options_.clock->NowMicros() > + static_cast(read_options.deadline.count())); + for (++keys_read; keys_read < num_keys; ++keys_read) { + stat_list[keys_read] = Status::TimedOut(); + } + } + + // Post processing (decrement reference counts and record statistics) + PERF_TIMER_GUARD(get_post_process_time); + autovector superversions_to_delete; + + for (auto mgd_iter : multiget_cf_data) { + auto mgd = mgd_iter.second; + if (!unref_only) { + ReturnAndCleanupSuperVersion(mgd.cfd, mgd.super_version); + } else { + mgd.cfd->GetSuperVersion()->Unref(); + } + } + RecordTick(stats_, NUMBER_MULTIGET_CALLS); + RecordTick(stats_, NUMBER_MULTIGET_KEYS_READ, num_keys); + RecordTick(stats_, NUMBER_MULTIGET_KEYS_FOUND, num_found); + RecordTick(stats_, NUMBER_MULTIGET_BYTES_READ, bytes_read); + RecordInHistogram(stats_, BYTES_PER_MULTIGET, bytes_read); + PERF_COUNTER_ADD(multiget_read_bytes, bytes_read); + PERF_TIMER_STOP(get_post_process_time); + + co_return stat_list; +} + template bool DBImpl::MultiCFSnapshot( const ReadOptions& read_options, ReadCallback* callback, @@ -3555,7 +4085,7 @@ void DBImpl::CleanupSuperVersion(SuperVersion* sv) { void DBImpl::ReturnAndCleanupSuperVersion(ColumnFamilyData* cfd, SuperVersion* sv) { - if (!cfd->ReturnThreadLocalSuperVersion(sv)) { + if (!cfd->ReturnThreadLocalSuperVersion(this, sv)) { CleanupSuperVersion(sv); } } diff --git a/db/db_impl/db_impl.h b/db/db_impl/db_impl.h index 09f7d36362ed..d37c51ee43ea 100644 --- a/db/db_impl/db_impl.h +++ b/db/db_impl/db_impl.h @@ -151,9 +151,14 @@ class DBImpl : public DB { using DB::Put; Status Put(const WriteOptions& options, ColumnFamilyHandle* column_family, const Slice& key, const Slice& value) override; + Status Put(const WriteOptions& options, ColumnFamilyHandle* column_family, const Slice& key, const Slice& ts, const Slice& value) override; + virtual Async_future AsyncPut(const WriteOptions& options, + ColumnFamilyHandle* column_family, + const Slice& key, const Slice& value) override; + using DB::Merge; Status Merge(const WriteOptions& options, ColumnFamilyHandle* column_family, const Slice& key, const Slice& value) override; @@ -185,6 +190,9 @@ class DBImpl : public DB { std::vector&& updates, uint64_t* seq) override; + virtual Async_future AsyncWrite(const WriteOptions& options, + WriteBatch* updates) override; + using DB::Get; virtual Status Get(const ReadOptions& options, ColumnFamilyHandle* column_family, const Slice& key, @@ -193,6 +201,11 @@ class DBImpl : public DB { ColumnFamilyHandle* column_family, const Slice& key, PinnableSlice* value, std::string* timestamp) override; + virtual Async_future AsyncGet(const ReadOptions& options, + ColumnFamilyHandle* column_family, + const Slice& key, PinnableSlice* value, + std::string* timestamp) override; + using DB::GetMergeOperands; Status GetMergeOperands(const ReadOptions& options, ColumnFamilyHandle* column_family, const Slice& key, @@ -219,6 +232,12 @@ class DBImpl : public DB { const std::vector& column_family, const std::vector& keys, std::vector* values, std::vector* timestamps) override; + using DB::AsyncMultiGet; + virtual Async_future AsyncMultiGet( + const ReadOptions& options, + const std::vector& column_family, + const std::vector& keys, std::vector* values, + std::vector* timestamps) override; // This MultiGet is a batched version, which may be faster than calling Get // multiple times, especially if the keys have some spatial locality that @@ -363,8 +382,10 @@ class DBImpl : public DB { const FlushOptions& options, const std::vector& column_families) override; virtual Status FlushWAL(bool sync) override; + virtual Async_future AsyncFlushWAL(bool sync) override; bool TEST_WALBufferIsEmpty(bool lock = true); virtual Status SyncWAL() override; + virtual Async_future AsSyncWAL() override; virtual Status LockWAL() override; virtual Status UnlockWAL() override; @@ -559,6 +580,9 @@ class DBImpl : public DB { Status GetImpl(const ReadOptions& options, const Slice& key, GetImplOptions& get_impl_options); + Async_future AsyncGetImpl(const ReadOptions& options, const Slice& key, + GetImplOptions& get_impl_options); + // If `snapshot` == kMaxSequenceNumber, set a recent one inside the file. ArenaWrappedDBIter* NewIteratorImpl(const ReadOptions& options, ColumnFamilyData* cfd, @@ -1331,12 +1355,27 @@ class DBImpl : public DB { uint64_t* seq_used = nullptr); void MultiBatchWriteCommit(CommitRequest* request); + Async_future AsyncWriteImpl( + const WriteOptions& options, WriteBatch* updates, + WriteCallback* callback = nullptr, uint64_t* log_used = nullptr, + uint64_t log_ref = 0, bool disable_memtable = false, + uint64_t* seq_used = nullptr, size_t batch_cnt = 0, + PreReleaseCallback* pre_release_callback = nullptr); + Status PipelinedWriteImpl(const WriteOptions& options, WriteBatch* updates, WriteCallback* callback = nullptr, uint64_t* log_used = nullptr, uint64_t log_ref = 0, bool disable_memtable = false, uint64_t* seq_used = nullptr); + Async_future AsyncPipelinedWriteImpl(const WriteOptions& options, + WriteBatch* updates, + WriteCallback* callback = nullptr, + uint64_t* log_used = nullptr, + uint64_t log_ref = 0, + bool disable_memtable = false, + uint64_t* seq_used = nullptr); + // Write only to memtables without joining any write queue Status UnorderedWriteMemtable(const WriteOptions& write_options, WriteBatch* my_batch, WriteCallback* callback, @@ -1364,6 +1403,13 @@ class DBImpl : public DB { PreReleaseCallback* pre_release_callback, const AssignOrder assign_order, const PublishLastSeq publish_last_seq, const bool disable_memtable); + Async_future AsyncWriteImplWALOnly( + WriteThread* write_thread, const WriteOptions& options, + WriteBatch* updates, WriteCallback* callback, uint64_t* log_used, + const uint64_t log_ref, uint64_t* seq_used, const size_t sub_batch_cnt, + PreReleaseCallback* pre_release_callback, const AssignOrder assign_order, + const PublishLastSeq publish_last_seq, const bool disable_memtable); + // write cached_recoverable_state_ to memtable if it is not empty // The writer must be the leader in write_thread_ and holding mutex_ Status WriteRecoverableState(); @@ -1510,7 +1556,6 @@ class DBImpl : public DB { // of a std::unique_ptr as a member. log::Writer* writer; // own - private: // true for some prefix of logs_ bool getting_synced = false; // The size of the file before the sync happens. This amount is guaranteed @@ -1618,6 +1663,18 @@ class DBImpl : public DB { Env::Priority compaction_pri_; }; + std::tuple + GetSnapshot(const ReadOptions& read_options, const Slice& key, + GetImplOptions& get_impl_options, + const bool useThreadLocalCache = true); + + enum class MemtableLookupStatus { NotFound, Failed, Found }; + std::tuple LookupMemtable( + const ReadOptions& read_options, const LookupKey& lkey, + std::string* timestamp, SuperVersion* sv, ColumnFamilyData* cfd, + MergeContext& merge_context, SequenceNumber& max_covering_tombstone_seq, + GetImplOptions& get_impl_options); + // Initialize the built-in column family for persistent stats. Depending on // whether on-disk persistent stats have been enabled before, it may either // create a new column family and column family handle or just a column family @@ -1825,16 +1882,29 @@ class DBImpl : public DB { uint64_t* log_used, uint64_t* log_size, LogFileNumberSize& log_file_number_size); + Async_future AsyncWriteToWAL(const WriteBatch& merged_batch, + log::Writer* log_writer, uint64_t* log_used, + uint64_t* log_size); + IOStatus WriteToWAL(const WriteThread::WriteGroup& write_group, log::Writer* log_writer, uint64_t* log_used, bool need_log_sync, bool need_log_dir_sync, SequenceNumber sequence, LogFileNumberSize& log_file_number_size); + Async_future AsyncWriteToWAL(const WriteThread::WriteGroup& write_group, + log::Writer* log_writer, uint64_t* log_used, + bool need_log_sync, bool need_log_dir_sync, + SequenceNumber sequence); + IOStatus ConcurrentWriteToWAL(const WriteThread::WriteGroup& write_group, uint64_t* log_used, SequenceNumber* last_sequence, size_t seq_inc); + Async_future AsyncConcurrentWriteToWAL( + const WriteThread::WriteGroup& write_group, uint64_t* log_used, + SequenceNumber* last_sequence, size_t seq_inc); + // Used by WriteImpl to update bg_error_ if paranoid check is enabled. // Caller must hold mutex_. void WriteStatusCheckOnLocked(const Status& status); diff --git a/db/db_impl/db_impl_write.cc b/db/db_impl/db_impl_write.cc index 08d6b584c62d..98252d159718 100644 --- a/db/db_impl/db_impl_write.cc +++ b/db/db_impl/db_impl_write.cc @@ -14,6 +14,7 @@ #include "logging/logging.h" #include "monitoring/perf_context_imp.h" #include "options/options_helper.h" +#include "rocksdb/async_future.h" #include "test_util/sync_point.h" #include "util/cast_util.h" @@ -112,6 +113,13 @@ Status DBImpl::Write(const WriteOptions& write_options, WriteBatch* my_batch, /*disable_memtable=*/false, seq); } +Async_future DBImpl::AsyncWrite(const WriteOptions& write_options, + WriteBatch* my_batch) { + auto result = AsyncWriteImpl(write_options, my_batch, nullptr, nullptr); + co_await result; + co_return result.status(); +} + #ifndef ROCKSDB_LITE Status DBImpl::WriteWithCallback(const WriteOptions& write_options, WriteBatch* my_batch, @@ -776,86 +784,518 @@ Status DBImpl::WriteImpl(const WriteOptions& write_options, return status; } -Status DBImpl::PipelinedWriteImpl(const WriteOptions& write_options, - WriteBatch* my_batch, WriteCallback* callback, - uint64_t* log_used, uint64_t log_ref, - bool disable_memtable, uint64_t* seq_used) { - PERF_TIMER_GUARD(write_pre_and_post_process_time); - StopWatch write_sw(immutable_db_options_.clock, stats_, DB_WRITE); +Async_future DBImpl::AsyncWriteImpl(const WriteOptions& write_options, + WriteBatch* my_batch, + WriteCallback* callback, uint64_t* log_used, + uint64_t log_ref, bool disable_memtable, + uint64_t* seq_used, size_t batch_cnt, + PreReleaseCallback* pre_release_callback) { + assert(!seq_per_batch_ || batch_cnt != 0); + if (my_batch == nullptr) { + co_return Status::Corruption("Batch is nullptr!"); + } + if (tracer_) { + InstrumentedMutexLock lock(&trace_mutex_); + if (tracer_) { + // TODO: maybe handle the tracing status? + tracer_->Write(my_batch).PermitUncheckedError(); + } + } + if (write_options.sync && write_options.disableWAL) { + co_return Status::InvalidArgument("Sync writes has to enable WAL."); + } + if (two_write_queues_ && immutable_db_options_.enable_pipelined_write) { + co_return Status::NotSupported( + "pipelined_writes is not compatible with concurrent prepares"); + } + if (seq_per_batch_ && immutable_db_options_.enable_pipelined_write) { + // TODO(yiwu): update pipeline write with seq_per_batch and batch_cnt + co_return Status::NotSupported( + "pipelined_writes is not compatible with seq_per_batch"); + } + if (immutable_db_options_.unordered_write && + immutable_db_options_.enable_pipelined_write) { + co_return Status::NotSupported( + "pipelined_writes is not compatible with unordered_write"); + } + // Otherwise IsLatestPersistentState optimization does not make sense + assert(!WriteBatchInternal::IsLatestPersistentState(my_batch) || + disable_memtable); - WriteContext write_context; + if (write_options.low_pri) { + Status s = ThrottleLowPriWritesIfNeeded(write_options, my_batch); + if (!s.ok()) { + co_return s; + } + } + + if (two_write_queues_ && disable_memtable) { + AssignOrder assign_order = + seq_per_batch_ ? kDoAssignOrder : kDontAssignOrder; + // Otherwise it is WAL-only Prepare batches in WriteCommitted policy and + // they don't consume sequence. + auto result = AsyncWriteImplWALOnly( + &nonmem_write_thread_, write_options, my_batch, callback, log_used, + log_ref, seq_used, batch_cnt, pre_release_callback, assign_order, + kDontPublishLastSeq, disable_memtable); + co_await result; + co_return result.status(); + } + + if (immutable_db_options_.unordered_write) { + const size_t sub_batch_cnt = batch_cnt != 0 + ? batch_cnt + // every key is a sub-batch consuming a seq + : WriteBatchInternal::Count(my_batch); + uint64_t seq = 0; + // Use a write thread to i) optimize for WAL write, ii) publish last + // sequence in in increasing order, iii) call pre_release_callback serially + auto result = AsyncWriteImplWALOnly( + &write_thread_, write_options, my_batch, callback, log_used, log_ref, + &seq, sub_batch_cnt, pre_release_callback, kDoAssignOrder, + kDoPublishLastSeq, disable_memtable); + co_await result; + Status status = result.status(); + TEST_SYNC_POINT("DBImpl::WriteImpl:UnorderedWriteAfterWriteWAL"); + if (!status.ok()) { + co_return status; + } + if (seq_used) { + *seq_used = seq; + } + if (!disable_memtable) { + TEST_SYNC_POINT("DBImpl::WriteImpl:BeforeUnorderedWriteMemtable"); + status = UnorderedWriteMemtable(write_options, my_batch, callback, + log_ref, seq, sub_batch_cnt); + } + co_return status; + } + + if (immutable_db_options_.enable_pipelined_write) { + auto result = + AsyncPipelinedWriteImpl(write_options, my_batch, callback, log_used, + log_ref, disable_memtable, seq_used); + co_await result; + co_return result.status(); + } + PERF_TIMER_GUARD(write_pre_and_post_process_time); WriteThread::Writer w(write_options, my_batch, callback, log_ref, - disable_memtable, /*_batch_cnt=*/0, - /*_pre_release_callback=*/nullptr); + disable_memtable, batch_cnt, pre_release_callback); + + if (!write_options.disableWAL) { + RecordTick(stats_, WRITE_WITH_WAL); + } + + StopWatch write_sw(immutable_db_options_.clock, immutable_db_options_.stats, + DB_WRITE); + write_thread_.JoinBatchGroup(&w); - TEST_SYNC_POINT("DBImplWrite::PipelinedWriteImpl:AfterJoinBatchGroup"); - if (w.state == WriteThread::STATE_GROUP_LEADER) { - WriteThread::WriteGroup wal_write_group; - if (w.callback && !w.callback->AllowWriteBatching()) { - write_thread_.WaitForMemTableWriters(); + if (w.state == WriteThread::STATE_PARALLEL_MEMTABLE_WRITER) { + // we are a non-leader in a parallel group + + if (w.ShouldWriteToMemtable()) { + PERF_TIMER_STOP(write_pre_and_post_process_time); + PERF_TIMER_GUARD(write_memtable_time); + + ColumnFamilyMemTablesImpl column_family_memtables( + versions_->GetColumnFamilySet()); + w.status = WriteBatchInternal::InsertInto( + &w, w.sequence, &column_family_memtables, &flush_scheduler_, + &trim_history_scheduler_, + write_options.ignore_missing_column_families, 0 /*log_number*/, this, + true /*concurrent_memtable_writes*/, seq_per_batch_, w.batch_cnt, + batch_per_txn_, write_options.memtable_insert_hint_per_batch); + + PERF_TIMER_START(write_pre_and_post_process_time); } - LogContext log_context(!write_options.disableWAL && write_options.sync); + + if (write_thread_.CompleteParallelMemTableWriter(&w)) { + // we're responsible for exit batch group + // TODO(myabandeh): propagate status to write_group + auto last_sequence = w.write_group->last_sequence; + versions_->SetLastSequence(last_sequence); + MemTableInsertStatusCheck(w.status); + write_thread_.ExitAsBatchGroupFollower(&w); + } + assert(w.state == WriteThread::STATE_COMPLETED); + // STATE_COMPLETED conditional below handles exit + } + + if (w.state == WriteThread::STATE_COMPLETED) { + if (log_used != nullptr) { + *log_used = w.log_used; + } + if (seq_used != nullptr) { + *seq_used = w.sequence; + } + // write is complete and leader has updated sequence + co_return w.FinalStatus(); + } + // else we are the leader of the write batch group + assert(w.state == WriteThread::STATE_GROUP_LEADER); + Status status; + // Once reaches this point, the current writer "w" will try to do its write + // job. It may also pick up some of the remaining writers in the "writers_" + // when it finds suitable, and finish them in the same write batch. + // This is how a write job could be done by the other writer. + WriteContext write_context; + WriteThread::WriteGroup write_group; + bool in_parallel_group = false; + uint64_t last_sequence = kMaxSequenceNumber; + + mutex_.Lock(); + + bool need_log_sync = write_options.sync; + bool need_log_dir_sync = need_log_sync && !log_dir_synced_; + assert(!two_write_queues_ || !disable_memtable); + { + // With concurrent writes we do preprocess only in the write thread that + // also does write to memtable to avoid sync issue on shared data structure + // with the other thread + // PreprocessWrite does its own perf timing. PERF_TIMER_STOP(write_pre_and_post_process_time); - w.status = PreprocessWrite(write_options, &log_context, &write_context); + + LogContext log_context(need_log_sync); + status = PreprocessWrite(write_options, &log_context, &write_context); + if (!two_write_queues_) { + // Assign it after ::PreprocessWrite since the sequence might advance + // inside it by WriteRecoverableState + last_sequence = versions_->LastSequence(); + } + PERF_TIMER_START(write_pre_and_post_process_time); + } + log::Writer* log_writer = logs_.back().writer; - // This can set non-OK status if callback fail. - last_batch_group_size_ = - write_thread_.EnterAsBatchGroupLeader(&w, &wal_write_group); - const SequenceNumber current_sequence = - write_thread_.UpdateLastSequence(versions_->LastSequence()) + 1; + mutex_.Unlock(); + + // Add to log and apply to memtable. We can release the lock + // during this phase since &w is currently responsible for logging + // and protects against concurrent loggers and concurrent writes + // into memtables + + TEST_SYNC_POINT("DBImpl::WriteImpl:BeforeLeaderEnters"); + last_batch_group_size_ = + write_thread_.EnterAsBatchGroupLeader(&w, &write_group); + + IOStatus io_s; + Status pre_release_cb_status; + if (status.ok()) { + // Rules for when we can update the memtable concurrently + // 1. supported by memtable + // 2. Puts are not okay if inplace_update_support + // 3. Merges are not okay + // + // Rules 1..2 are enforced by checking the options + // during startup (CheckConcurrentWritesSupported), so if + // options.allow_concurrent_memtable_write is true then they can be + // assumed to be true. Rule 3 is checked for each batch. We could + // relax rules 2 if we could prevent write batches from referring + // more than once to a particular key. + bool parallel = immutable_db_options_.allow_concurrent_memtable_write && + write_group.size > 1; size_t total_count = 0; + size_t valid_batches = 0; size_t total_byte_size = 0; - - if (w.status.ok()) { - // TODO: this use of operator bool on `tracer_` can avoid unnecessary lock - // grabs but does not seem thread-safe. - if (tracer_) { - InstrumentedMutexLock lock(&trace_mutex_); - if (tracer_ != nullptr && tracer_->IsWriteOrderPreserved()) { - for (auto* writer : wal_write_group) { - // TODO: maybe handle the tracing status? - tracer_->Write(writer->multi_batch.batches[0]) - .PermitUncheckedError(); - } + size_t pre_release_callback_cnt = 0; + for (auto* writer : write_group) { + auto &batch{writer->multi_batch.batches[0]}; + if (writer->CheckCallback(this)) { + valid_batches += writer->batch_cnt; + if (writer->ShouldWriteToMemtable()) { + total_count += WriteBatchInternal::Count(batch); + parallel = parallel && !batch->HasMerge(); } - } - SequenceNumber next_sequence = current_sequence; - for (auto* writer : wal_write_group) { - assert(writer); - if (writer->CheckCallback(this)) { - if (writer->ShouldWriteToMemtable()) { - writer->sequence = next_sequence; - size_t count = - WriteBatchInternal::Count(writer->multi_batch.batches[0]); - next_sequence += count; - total_count += count; - } - total_byte_size = WriteBatchInternal::AppendedByteSize( - total_byte_size, - WriteBatchInternal::ByteSize(writer->multi_batch.batches[0])); + total_byte_size = WriteBatchInternal::AppendedByteSize( + total_byte_size, WriteBatchInternal::ByteSize(batch)); + if (writer->pre_release_callback) { + pre_release_callback_cnt++; } } - if (w.disable_wal) { - has_unpersisted_data_.store(true, std::memory_order_relaxed); - } - write_thread_.UpdateLastSequence(current_sequence + total_count - 1); } + // Note about seq_per_batch_: either disableWAL is set for the entire write + // group or not. In either case we inc seq for each write batch with no + // failed callback. This means that there could be a batch with + // disalbe_memtable in between; although we do not write this batch to + // memtable it still consumes a seq. Otherwise, if !seq_per_batch_, we inc + // the seq per valid written key to mem. + size_t seq_inc = seq_per_batch_ ? valid_batches : total_count; + const bool concurrent_update = two_write_queues_; + // Update stats while we are an exclusive group leader, so we know + // that nobody else can be writing to these particular stats. + // We're optimistic, updating the stats before we successfully + // commit. That lets us release our leader status early. auto stats = default_cf_internal_stats_; - stats->AddDBStats(InternalStats::kIntStatsNumKeysWritten, total_count); + stats->AddDBStats(InternalStats::kIntStatsNumKeysWritten, total_count, + concurrent_update); RecordTick(stats_, NUMBER_KEYS_WRITTEN, total_count); - stats->AddDBStats(InternalStats::kIntStatsBytesWritten, total_byte_size); + stats->AddDBStats(InternalStats::kIntStatsBytesWritten, total_byte_size, + concurrent_update); RecordTick(stats_, BYTES_WRITTEN, total_byte_size); - RecordInHistogram(stats_, BYTES_PER_WRITE, total_byte_size); - - PERF_TIMER_STOP(write_pre_and_post_process_time); - - IOStatus io_s; - io_s.PermitUncheckedError(); // Allow io_s to be uninitialized - + stats->AddDBStats(InternalStats::kIntStatsWriteDoneBySelf, 1, + concurrent_update); + RecordTick(stats_, WRITE_DONE_BY_SELF); + auto write_done_by_other = write_group.size - 1; + if (write_done_by_other > 0) { + stats->AddDBStats(InternalStats::kIntStatsWriteDoneByOther, + write_done_by_other, concurrent_update); + RecordTick(stats_, WRITE_DONE_BY_OTHER, write_done_by_other); + } + RecordInHistogram(stats_, BYTES_PER_WRITE, total_byte_size); + + if (write_options.disableWAL) { + has_unpersisted_data_.store(true, std::memory_order_relaxed); + } + + PERF_TIMER_STOP(write_pre_and_post_process_time); + + if (!two_write_queues_) { + if (status.ok() && !write_options.disableWAL) { + PERF_TIMER_GUARD(write_wal_time); + auto result = + AsyncWriteToWAL(write_group, log_writer, log_used, + need_log_sync, need_log_dir_sync, + last_sequence + 1); + co_await result; + io_s = result.io_result(); + } + } else { + if (status.ok() && !write_options.disableWAL) { + PERF_TIMER_GUARD(write_wal_time); + // LastAllocatedSequence is increased inside WriteToWAL under + // wal_write_mutex_ to ensure ordered events in WAL + auto result = AsyncConcurrentWriteToWAL(write_group, log_used, + &last_sequence, seq_inc); + co_await result; + io_s = result.io_result(); + } else { + // Otherwise we inc seq number for memtable writes + last_sequence = versions_->FetchAddLastAllocatedSequence(seq_inc); + } + } + status = io_s; + assert(last_sequence != kMaxSequenceNumber); + const SequenceNumber current_sequence = last_sequence + 1; + last_sequence += seq_inc; + + // PreReleaseCallback is called after WAL write and before memtable write + if (status.ok()) { + SequenceNumber next_sequence = current_sequence; + size_t index = 0; + // Note: the logic for advancing seq here must be consistent with the + // logic in WriteBatchInternal::InsertInto(write_group...) as well as + // with WriteBatchInternal::InsertInto(write_batch...) that is called on + // the merged batch during recovery from the WAL. + for (auto* writer : write_group) { + if (writer->CallbackFailed()) { + continue; + } + writer->sequence = next_sequence; + if (writer->pre_release_callback) { + Status ws = writer->pre_release_callback->Callback( + writer->sequence, disable_memtable, writer->log_used, index++, + pre_release_callback_cnt); + if (!ws.ok()) { + status = pre_release_cb_status = ws; + break; + } + } + if (seq_per_batch_) { + assert(writer->batch_cnt); + next_sequence += writer->batch_cnt; + } else if (writer->ShouldWriteToMemtable()) { + next_sequence += WriteBatchInternal::Count(writer->multi_batch.batches[0]); + } + } + } + + if (status.ok()) { + PERF_TIMER_GUARD(write_memtable_time); + + if (!parallel) { + // w.sequence will be set inside InsertInto + w.status = WriteBatchInternal::InsertInto( + write_group, current_sequence, column_family_memtables_.get(), + &flush_scheduler_, &trim_history_scheduler_, + write_options.ignore_missing_column_families, + 0 /*recovery_log_number*/, this, parallel, seq_per_batch_, + batch_per_txn_); + } else { + write_group.last_sequence = last_sequence; + write_thread_.LaunchParallelMemTableWriters(&write_group); + in_parallel_group = true; + + // Each parallel follower is doing each own writes. The leader should + // also do its own. + if (w.ShouldWriteToMemtable()) { + ColumnFamilyMemTablesImpl column_family_memtables( + versions_->GetColumnFamilySet()); + assert(w.sequence == current_sequence); + w.status = WriteBatchInternal::InsertInto( + &w, w.sequence, &column_family_memtables, &flush_scheduler_, + &trim_history_scheduler_, + write_options.ignore_missing_column_families, 0 /*log_number*/, + this, true /*concurrent_memtable_writes*/, seq_per_batch_, + w.batch_cnt, batch_per_txn_, + write_options.memtable_insert_hint_per_batch); + } + } + if (seq_used != nullptr) { + *seq_used = w.sequence; + } + } + } + PERF_TIMER_START(write_pre_and_post_process_time); + + if (!w.CallbackFailed()) { + if (!io_s.ok()) { + assert(pre_release_cb_status.ok()); + IOStatusCheck(io_s); + } else { + WriteStatusCheck(pre_release_cb_status); + } + } else { + assert(io_s.ok() && pre_release_cb_status.ok()); + } + + if (need_log_sync) { + VersionEdit synced_wals; + + mutex_.Lock(); + + if (status.ok()) { + MarkLogsSynced(logfile_number_, need_log_dir_sync, &synced_wals); + } else { + MarkLogsNotSynced(logfile_number_); + } + + mutex_.Unlock(); + + if (status.ok() && synced_wals.IsWalAddition()) { + InstrumentedMutexLock l(&mutex_); + status = ApplyWALToManifest(&synced_wals); + } + + // Requesting sync with two_write_queues_ is expected to be very rare. We + // hence provide a simple implementation that is not necessarily efficient. + if (two_write_queues_) { + if (manual_wal_flush_) { + auto result = AsyncFlushWAL(true); + co_await result; + status = result.status(); + } else { + auto result = AsSyncWAL(); + co_await result; + status = result.status(); + } + } + } + + bool should_exit_batch_group = true; + if (in_parallel_group) { + // CompleteParallelWorker returns true if this thread should + // handle exit, false means somebody else did + should_exit_batch_group = write_thread_.CompleteParallelMemTableWriter(&w); + } + if (should_exit_batch_group) { + if (status.ok()) { + // Note: if we are to resume after non-OK statuses we need to revisit how + // we reacts to non-OK statuses here. + versions_->SetLastSequence(last_sequence); + } + MemTableInsertStatusCheck(w.status); + write_thread_.ExitAsBatchGroupLeader(write_group, status); + } + + if (status.ok()) { + status = w.FinalStatus(); + } + co_return status; +} + +Status DBImpl::PipelinedWriteImpl(const WriteOptions& write_options, + WriteBatch* my_batch, WriteCallback* callback, + uint64_t* log_used, uint64_t log_ref, + bool disable_memtable, uint64_t* seq_used) { + PERF_TIMER_GUARD(write_pre_and_post_process_time); + StopWatch write_sw(immutable_db_options_.clock, stats_, DB_WRITE); + + WriteContext write_context; + + WriteThread::Writer w(write_options, my_batch, callback, log_ref, + disable_memtable, /*_batch_cnt=*/0, + /*_pre_release_callback=*/nullptr); + write_thread_.JoinBatchGroup(&w); + TEST_SYNC_POINT("DBImplWrite::PipelinedWriteImpl:AfterJoinBatchGroup"); + if (w.state == WriteThread::STATE_GROUP_LEADER) { + WriteThread::WriteGroup wal_write_group; + if (w.callback && !w.callback->AllowWriteBatching()) { + write_thread_.WaitForMemTableWriters(); + } + LogContext log_context(!write_options.disableWAL && write_options.sync); + // PreprocessWrite does its own perf timing. + PERF_TIMER_STOP(write_pre_and_post_process_time); + w.status = PreprocessWrite(write_options, &log_context, &write_context); + PERF_TIMER_START(write_pre_and_post_process_time); + + // This can set non-OK status if callback fail. + last_batch_group_size_ = + write_thread_.EnterAsBatchGroupLeader(&w, &wal_write_group); + const SequenceNumber current_sequence = + write_thread_.UpdateLastSequence(versions_->LastSequence()) + 1; + size_t total_count = 0; + size_t total_byte_size = 0; + + if (w.status.ok()) { + // TODO: this use of operator bool on `tracer_` can avoid unnecessary lock + // grabs but does not seem thread-safe. + if (tracer_) { + InstrumentedMutexLock lock(&trace_mutex_); + if (tracer_ != nullptr && tracer_->IsWriteOrderPreserved()) { + for (auto* writer : wal_write_group) { + // TODO: maybe handle the tracing status? + tracer_->Write(writer->multi_batch.batches[0]) + .PermitUncheckedError(); + } + } + } + SequenceNumber next_sequence = current_sequence; + for (auto* writer : wal_write_group) { + assert(writer); + if (writer->CheckCallback(this)) { + if (writer->ShouldWriteToMemtable()) { + writer->sequence = next_sequence; + size_t count = + WriteBatchInternal::Count(writer->multi_batch.batches[0]); + next_sequence += count; + total_count += count; + } + total_byte_size = WriteBatchInternal::AppendedByteSize( + total_byte_size, + WriteBatchInternal::ByteSize(writer->multi_batch.batches[0])); + } + } + if (w.disable_wal) { + has_unpersisted_data_.store(true, std::memory_order_relaxed); + } + write_thread_.UpdateLastSequence(current_sequence + total_count - 1); + } + + auto stats = default_cf_internal_stats_; + stats->AddDBStats(InternalStats::kIntStatsNumKeysWritten, total_count); + RecordTick(stats_, NUMBER_KEYS_WRITTEN, total_count); + stats->AddDBStats(InternalStats::kIntStatsBytesWritten, total_byte_size); + RecordTick(stats_, BYTES_WRITTEN, total_byte_size); + RecordInHistogram(stats_, BYTES_PER_WRITE, total_byte_size); + + PERF_TIMER_STOP(write_pre_and_post_process_time); + + IOStatus io_s; + io_s.PermitUncheckedError(); // Allow io_s to be uninitialized + if (w.status.ok() && !write_options.disableWAL) { PERF_TIMER_GUARD(write_wal_time); stats->AddDBStats(InternalStats::kIntStatsWriteDoneBySelf, 1); @@ -872,6 +1312,7 @@ Status DBImpl::PipelinedWriteImpl(const WriteOptions& write_options, WriteToWAL(wal_write_group, log_context.writer, log_used, log_context.need_log_sync, log_context.need_log_dir_sync, current_sequence, log_file_number_size); + w.status = io_s; } @@ -922,88 +1363,438 @@ Status DBImpl::PipelinedWriteImpl(const WriteOptions& write_options, versions_->SetLastSequence(memtable_write_group.last_sequence); write_thread_.ExitAsMemTableWriter(&w, memtable_write_group); } - } else { - // NOTE: the memtable_write_group is never really used, - // so we need to set its status to pass ASSERT_STATUS_CHECKED - memtable_write_group.status.PermitUncheckedError(); + } else { + // NOTE: the memtable_write_group is never really used, + // so we need to set its status to pass ASSERT_STATUS_CHECKED + memtable_write_group.status.PermitUncheckedError(); + } + + if (w.state == WriteThread::STATE_PARALLEL_MEMTABLE_WRITER) { + assert(w.ShouldWriteToMemtable()); + ColumnFamilyMemTablesImpl column_family_memtables( + versions_->GetColumnFamilySet()); + w.status = WriteBatchInternal::InsertInto( + &w, w.sequence, &column_family_memtables, &flush_scheduler_, + &trim_history_scheduler_, write_options.ignore_missing_column_families, + 0 /*log_number*/, this, true /*concurrent_memtable_writes*/, + false /*seq_per_batch*/, 0 /*batch_cnt*/, true /*batch_per_txn*/, + write_options.memtable_insert_hint_per_batch); + if (write_thread_.CompleteParallelMemTableWriter(&w)) { + MemTableInsertStatusCheck(w.status); + versions_->SetLastSequence(w.write_group->last_sequence); + write_thread_.ExitAsMemTableWriter(&w, *w.write_group); + } + } + if (seq_used != nullptr) { + *seq_used = w.sequence; + } + + assert(w.state == WriteThread::STATE_COMPLETED); + return w.FinalStatus(); +} + +Async_future DBImpl::AsyncPipelinedWriteImpl( + const WriteOptions& write_options, WriteBatch* my_batch, + WriteCallback* callback, uint64_t* log_used, uint64_t log_ref, + bool disable_memtable, uint64_t* seq_used) { + PERF_TIMER_GUARD(write_pre_and_post_process_time); + StopWatch write_sw(immutable_db_options_.clock, immutable_db_options_.stats, + DB_WRITE); + + WriteContext write_context; + + WriteThread::Writer w(write_options, my_batch, callback, log_ref, + disable_memtable); + write_thread_.JoinBatchGroup(&w); + TEST_SYNC_POINT("DBImplWrite::PipelinedWriteImpl:AfterJoinBatchGroup"); + if (w.state == WriteThread::STATE_GROUP_LEADER) { + WriteThread::WriteGroup wal_write_group; + if (w.callback && !w.callback->AllowWriteBatching()) { + write_thread_.WaitForMemTableWriters(); + } + mutex_.Lock(); + bool need_log_sync = !write_options.disableWAL && write_options.sync; + bool need_log_dir_sync = need_log_sync && !log_dir_synced_; + LogContext log_context(!write_options.disableWAL && write_options.sync); + // PreprocessWrite does its own perf timing. + PERF_TIMER_STOP(write_pre_and_post_process_time); + w.status = PreprocessWrite(write_options, &log_context, &write_context); + PERF_TIMER_START(write_pre_and_post_process_time); + log::Writer* log_writer = logs_.back().writer; + mutex_.Unlock(); + + // This can set non-OK status if callback fail. + last_batch_group_size_ = + write_thread_.EnterAsBatchGroupLeader(&w, &wal_write_group); + const SequenceNumber current_sequence = + write_thread_.UpdateLastSequence(versions_->LastSequence()) + 1; + size_t total_count = 0; + size_t total_byte_size = 0; + + if (w.status.ok()) { + SequenceNumber next_sequence = current_sequence; + for (auto writer : wal_write_group) { + if (writer->CheckCallback(this)) { + if (writer->ShouldWriteToMemtable()) { + writer->sequence = next_sequence; + size_t count = WriteBatchInternal::Count(writer->multi_batch.batches[0]); + next_sequence += count; + total_count += count; + } + total_byte_size = WriteBatchInternal::AppendedByteSize( + total_byte_size, WriteBatchInternal::ByteSize(writer->multi_batch.batches[0])); + } + } + if (w.disable_wal) { + has_unpersisted_data_.store(true, std::memory_order_relaxed); + } + write_thread_.UpdateLastSequence(current_sequence + total_count - 1); + } + + auto stats = default_cf_internal_stats_; + stats->AddDBStats(InternalStats::kIntStatsNumKeysWritten, total_count); + RecordTick(stats_, NUMBER_KEYS_WRITTEN, total_count); + stats->AddDBStats(InternalStats::kIntStatsBytesWritten, total_byte_size); + RecordTick(stats_, BYTES_WRITTEN, total_byte_size); + RecordInHistogram(stats_, BYTES_PER_WRITE, total_byte_size); + + PERF_TIMER_STOP(write_pre_and_post_process_time); + + IOStatus io_s; + io_s.PermitUncheckedError(); // Allow io_s to be uninitialized + + if (w.status.ok() && !write_options.disableWAL) { + PERF_TIMER_GUARD(write_wal_time); + stats->AddDBStats(InternalStats::kIntStatsWriteDoneBySelf, 1); + RecordTick(stats_, WRITE_DONE_BY_SELF, 1); + if (wal_write_group.size > 1) { + stats->AddDBStats(InternalStats::kIntStatsWriteDoneByOther, + wal_write_group.size - 1); + RecordTick(stats_, WRITE_DONE_BY_OTHER, wal_write_group.size - 1); + } + auto result = + AsyncWriteToWAL(wal_write_group, log_writer, log_used, need_log_sync, + need_log_dir_sync, current_sequence); + co_await result; + io_s = result.io_result(); + w.status = io_s; + } + + if (!w.CallbackFailed()) { + if (!io_s.ok()) { + IOStatusCheck(io_s); + } else { + WriteStatusCheck(w.status); + } + } + + VersionEdit synced_wals; + if (log_context.need_log_sync) { + InstrumentedMutexLock l(&log_write_mutex_); + if (w.status.ok()) { + MarkLogsSynced(logfile_number_, log_context.need_log_dir_sync, + &synced_wals); + } else { + MarkLogsNotSynced(logfile_number_); + } + } + + if (w.status.ok() && synced_wals.IsWalAddition()) { + InstrumentedMutexLock l(&mutex_); + w.status = ApplyWALToManifest(&synced_wals); + } + write_thread_.ExitAsBatchGroupLeader(wal_write_group, w.status); + } + + // NOTE: the memtable_write_group is declared before the following + // `if` statement because its lifetime needs to be longer + // that the inner context of the `if` as a reference to it + // may be used further below within the outer _write_thread + WriteThread::WriteGroup memtable_write_group; + + if (w.state == WriteThread::STATE_MEMTABLE_WRITER_LEADER) { + PERF_TIMER_GUARD(write_memtable_time); + assert(w.ShouldWriteToMemtable()); + write_thread_.EnterAsMemTableWriter(&w, &memtable_write_group); + if (memtable_write_group.size > 1 && + immutable_db_options_.allow_concurrent_memtable_write) { + write_thread_.LaunchParallelMemTableWriters(&memtable_write_group); + } else { + memtable_write_group.status = WriteBatchInternal::InsertInto( + memtable_write_group, w.sequence, column_family_memtables_.get(), + &flush_scheduler_, &trim_history_scheduler_, + write_options.ignore_missing_column_families, 0 /*log_number*/, this, + false /*concurrent_memtable_writes*/, seq_per_batch_, batch_per_txn_); + versions_->SetLastSequence(memtable_write_group.last_sequence); + write_thread_.ExitAsMemTableWriter(&w, memtable_write_group); + } + } else { + // NOTE: the memtable_write_group is never really used, + // so we need to set its status to pass ASSERT_STATUS_CHECKED + memtable_write_group.status.PermitUncheckedError(); + } + + if (w.state == WriteThread::STATE_PARALLEL_MEMTABLE_WRITER) { + assert(w.ShouldWriteToMemtable()); + ColumnFamilyMemTablesImpl column_family_memtables( + versions_->GetColumnFamilySet()); + w.status = WriteBatchInternal::InsertInto( + &w, w.sequence, &column_family_memtables, &flush_scheduler_, + &trim_history_scheduler_, write_options.ignore_missing_column_families, + 0 /*log_number*/, this, true /*concurrent_memtable_writes*/, + false /*seq_per_batch*/, 0 /*batch_cnt*/, true /*batch_per_txn*/, + write_options.memtable_insert_hint_per_batch); + if (write_thread_.CompleteParallelMemTableWriter(&w)) { + MemTableInsertStatusCheck(w.status); + versions_->SetLastSequence(w.write_group->last_sequence); + write_thread_.ExitAsMemTableWriter(&w, *w.write_group); + } + } + if (seq_used != nullptr) { + *seq_used = w.sequence; + } + + assert(w.state == WriteThread::STATE_COMPLETED); + co_return w.FinalStatus(); +} + +Status DBImpl::UnorderedWriteMemtable(const WriteOptions& write_options, + WriteBatch* my_batch, + WriteCallback* callback, uint64_t log_ref, + SequenceNumber seq, + const size_t sub_batch_cnt) { + PERF_TIMER_GUARD(write_pre_and_post_process_time); + StopWatch write_sw(immutable_db_options_.clock, stats_, DB_WRITE); + + WriteThread::Writer w(write_options, my_batch, callback, log_ref, + false /*disable_memtable*/); + + if (w.CheckCallback(this) && w.ShouldWriteToMemtable()) { + w.sequence = seq; + size_t total_count = WriteBatchInternal::Count(my_batch); + InternalStats* stats = default_cf_internal_stats_; + stats->AddDBStats(InternalStats::kIntStatsNumKeysWritten, total_count); + RecordTick(stats_, NUMBER_KEYS_WRITTEN, total_count); + + ColumnFamilyMemTablesImpl column_family_memtables( + versions_->GetColumnFamilySet()); + w.status = WriteBatchInternal::InsertInto( + &w, w.sequence, &column_family_memtables, &flush_scheduler_, + &trim_history_scheduler_, write_options.ignore_missing_column_families, + 0 /*log_number*/, this, true /*concurrent_memtable_writes*/, + seq_per_batch_, sub_batch_cnt, true /*batch_per_txn*/, + write_options.memtable_insert_hint_per_batch); + if (write_options.disableWAL) { + has_unpersisted_data_.store(true, std::memory_order_relaxed); + } + } + + size_t pending_cnt = pending_memtable_writes_.fetch_sub(1) - 1; + if (pending_cnt == 0) { + // switch_cv_ waits until pending_memtable_writes_ = 0. Locking its mutex + // before notify ensures that cv is in waiting state when it is notified + // thus not missing the update to pending_memtable_writes_ even though it is + // not modified under the mutex. + std::lock_guard lck(switch_mutex_); + switch_cv_.notify_all(); + } + WriteStatusCheck(w.status); + + if (!w.FinalStatus().ok()) { + return w.FinalStatus(); + } + return Status::OK(); +} + +// The 2nd write queue. If enabled it will be used only for WAL-only writes. +// This is the only queue that updates LastPublishedSequence which is only +// applicable in a two-queue setting. +Status DBImpl::WriteImplWALOnly( + WriteThread* write_thread, const WriteOptions& write_options, + WriteBatch* my_batch, WriteCallback* callback, uint64_t* log_used, + const uint64_t log_ref, uint64_t* seq_used, const size_t sub_batch_cnt, + PreReleaseCallback* pre_release_callback, const AssignOrder assign_order, + const PublishLastSeq publish_last_seq, const bool disable_memtable) { + PERF_TIMER_GUARD(write_pre_and_post_process_time); + WriteThread::Writer w(write_options, my_batch, callback, log_ref, + disable_memtable, sub_batch_cnt, pre_release_callback); + RecordTick(stats_, WRITE_WITH_WAL); + StopWatch write_sw(immutable_db_options_.clock, immutable_db_options_.stats, + DB_WRITE); + + write_thread->JoinBatchGroup(&w); + assert(w.state != WriteThread::STATE_PARALLEL_MEMTABLE_WRITER); + if (w.state == WriteThread::STATE_COMPLETED) { + if (log_used != nullptr) { + *log_used = w.log_used; + } + if (seq_used != nullptr) { + *seq_used = w.sequence; + } + return w.FinalStatus(); + } + // else we are the leader of the write batch group + assert(w.state == WriteThread::STATE_GROUP_LEADER); + + if (publish_last_seq == kDoPublishLastSeq) { + Status status; + + // Currently we only use kDoPublishLastSeq in unordered_write + assert(immutable_db_options_.unordered_write); + WriteContext write_context; + if (error_handler_.IsDBStopped()) { + status = error_handler_.GetBGError(); + } + // TODO(myabandeh): Make preliminary checks thread-safe so we could do them + // without paying the cost of obtaining the mutex. + if (status.ok()) { + InstrumentedMutexLock l(&mutex_); + LogContext log_context(false); + status = PreprocessWrite(write_options, &log_context, &write_context); + WriteStatusCheckOnLocked(status); + } + if (!status.ok()) { + WriteThread::WriteGroup write_group; + write_thread->EnterAsBatchGroupLeader(&w, &write_group); + write_thread->ExitAsBatchGroupLeader(write_group, status); + return status; + } + } + + WriteThread::WriteGroup write_group; + uint64_t last_sequence; + write_thread->EnterAsBatchGroupLeader(&w, &write_group); + // Note: no need to update last_batch_group_size_ here since the batch writes + // to WAL only + + size_t pre_release_callback_cnt = 0; + size_t total_byte_size = 0; + for (auto* writer : write_group) { + if (writer->CheckCallback(this)) { + total_byte_size = WriteBatchInternal::AppendedByteSize( + total_byte_size, WriteBatchInternal::ByteSize(writer->multi_batch.batches[0])); + if (writer->pre_release_callback) { + pre_release_callback_cnt++; + } + } + } + + const bool concurrent_update = true; + // Update stats while we are an exclusive group leader, so we know + // that nobody else can be writing to these particular stats. + // We're optimistic, updating the stats before we successfully + // commit. That lets us release our leader status early. + auto stats = default_cf_internal_stats_; + stats->AddDBStats(InternalStats::kIntStatsBytesWritten, total_byte_size, + concurrent_update); + RecordTick(stats_, BYTES_WRITTEN, total_byte_size); + stats->AddDBStats(InternalStats::kIntStatsWriteDoneBySelf, 1, + concurrent_update); + RecordTick(stats_, WRITE_DONE_BY_SELF); + auto write_done_by_other = write_group.size - 1; + if (write_done_by_other > 0) { + stats->AddDBStats(InternalStats::kIntStatsWriteDoneByOther, + write_done_by_other, concurrent_update); + RecordTick(stats_, WRITE_DONE_BY_OTHER, write_done_by_other); + } + RecordInHistogram(stats_, BYTES_PER_WRITE, total_byte_size); + + PERF_TIMER_STOP(write_pre_and_post_process_time); + + PERF_TIMER_GUARD(write_wal_time); + // LastAllocatedSequence is increased inside WriteToWAL under + // wal_write_mutex_ to ensure ordered events in WAL + size_t seq_inc = 0 /* total_count */; + if (assign_order == kDoAssignOrder) { + size_t total_batch_cnt = 0; + for (auto* writer : write_group) { + assert(writer->batch_cnt || !seq_per_batch_); + if (!writer->CallbackFailed()) { + total_batch_cnt += writer->batch_cnt; + } + } + seq_inc = total_batch_cnt; + } + Status status; + IOStatus io_s; + io_s.PermitUncheckedError(); // Allow io_s to be uninitialized + if (!write_options.disableWAL) { + io_s = ConcurrentWriteToWAL(write_group, log_used, &last_sequence, seq_inc); + status = io_s; + } else { + // Otherwise we inc seq number to do solely the seq allocation + last_sequence = versions_->FetchAddLastAllocatedSequence(seq_inc); + } + + size_t memtable_write_cnt = 0; + auto curr_seq = last_sequence + 1; + for (auto* writer : write_group) { + if (writer->CallbackFailed()) { + continue; + } + writer->sequence = curr_seq; + if (assign_order == kDoAssignOrder) { + assert(writer->batch_cnt || !seq_per_batch_); + curr_seq += writer->batch_cnt; + } + if (!writer->disable_memtable) { + memtable_write_cnt++; + } + // else seq advances only by memtable writes + } + if (status.ok() && write_options.sync) { + assert(!write_options.disableWAL); + // Requesting sync with two_write_queues_ is expected to be very rare. We + // hance provide a simple implementation that is not necessarily efficient. + if (manual_wal_flush_) { + status = FlushWAL(true); + } else { + status = SyncWAL(); + } } + PERF_TIMER_START(write_pre_and_post_process_time); - if (w.state == WriteThread::STATE_PARALLEL_MEMTABLE_WRITER) { - assert(w.ShouldWriteToMemtable()); - ColumnFamilyMemTablesImpl column_family_memtables( - versions_->GetColumnFamilySet()); - w.status = WriteBatchInternal::InsertInto( - &w, w.sequence, &column_family_memtables, &flush_scheduler_, - &trim_history_scheduler_, write_options.ignore_missing_column_families, - 0 /*log_number*/, this, true /*concurrent_memtable_writes*/, - false /*seq_per_batch*/, 0 /*batch_cnt*/, true /*batch_per_txn*/, - write_options.memtable_insert_hint_per_batch); - if (write_thread_.CompleteParallelMemTableWriter(&w)) { - MemTableInsertStatusCheck(w.status); - versions_->SetLastSequence(w.write_group->last_sequence); - write_thread_.ExitAsMemTableWriter(&w, *w.write_group); + if (!w.CallbackFailed()) { + if (!io_s.ok()) { + IOStatusCheck(io_s); + } else { + WriteStatusCheck(status); } } - if (seq_used != nullptr) { - *seq_used = w.sequence; - } - - assert(w.state == WriteThread::STATE_COMPLETED); - return w.FinalStatus(); -} - -Status DBImpl::UnorderedWriteMemtable(const WriteOptions& write_options, - WriteBatch* my_batch, - WriteCallback* callback, uint64_t log_ref, - SequenceNumber seq, - const size_t sub_batch_cnt) { - PERF_TIMER_GUARD(write_pre_and_post_process_time); - StopWatch write_sw(immutable_db_options_.clock, stats_, DB_WRITE); - - WriteThread::Writer w(write_options, my_batch, callback, log_ref, - false /*disable_memtable*/); - - if (w.CheckCallback(this) && w.ShouldWriteToMemtable()) { - w.sequence = seq; - size_t total_count = WriteBatchInternal::Count(my_batch); - InternalStats* stats = default_cf_internal_stats_; - stats->AddDBStats(InternalStats::kIntStatsNumKeysWritten, total_count); - RecordTick(stats_, NUMBER_KEYS_WRITTEN, total_count); - - ColumnFamilyMemTablesImpl column_family_memtables( - versions_->GetColumnFamilySet()); - w.status = WriteBatchInternal::InsertInto( - &w, w.sequence, &column_family_memtables, &flush_scheduler_, - &trim_history_scheduler_, write_options.ignore_missing_column_families, - 0 /*log_number*/, this, true /*concurrent_memtable_writes*/, - seq_per_batch_, sub_batch_cnt, true /*batch_per_txn*/, - write_options.memtable_insert_hint_per_batch); - if (write_options.disableWAL) { - has_unpersisted_data_.store(true, std::memory_order_relaxed); + if (status.ok()) { + size_t index = 0; + for (auto* writer : write_group) { + if (!writer->CallbackFailed() && writer->pre_release_callback) { + assert(writer->sequence != kMaxSequenceNumber); + Status ws = writer->pre_release_callback->Callback( + writer->sequence, disable_memtable, writer->log_used, index++, + pre_release_callback_cnt); + if (!ws.ok()) { + status = ws; + break; + } + } } } - - size_t pending_cnt = pending_memtable_writes_.fetch_sub(1) - 1; - if (pending_cnt == 0) { - // switch_cv_ waits until pending_memtable_writes_ = 0. Locking its mutex - // before notify ensures that cv is in waiting state when it is notified - // thus not missing the update to pending_memtable_writes_ even though it is - // not modified under the mutex. - std::lock_guard lck(switch_mutex_); - switch_cv_.notify_all(); + if (publish_last_seq == kDoPublishLastSeq) { + versions_->SetLastSequence(last_sequence + seq_inc); + // Currently we only use kDoPublishLastSeq in unordered_write + assert(immutable_db_options_.unordered_write); + } + if (immutable_db_options_.unordered_write && status.ok()) { + pending_memtable_writes_ += memtable_write_cnt; + } + write_thread->ExitAsBatchGroupLeader(write_group, status); + if (status.ok()) { + status = w.FinalStatus(); } - WriteStatusCheck(w.status); - - if (!w.FinalStatus().ok()) { - return w.FinalStatus(); + if (seq_used != nullptr) { + *seq_used = w.sequence; } - return Status::OK(); + return status; } -// The 2nd write queue. If enabled it will be used only for WAL-only writes. -// This is the only queue that updates LastPublishedSequence which is only -// applicable in a two-queue setting. -Status DBImpl::WriteImplWALOnly( +Async_future DBImpl::AsyncWriteImplWALOnly( WriteThread* write_thread, const WriteOptions& write_options, WriteBatch* my_batch, WriteCallback* callback, uint64_t* log_used, const uint64_t log_ref, uint64_t* seq_used, const size_t sub_batch_cnt, @@ -1023,7 +1814,7 @@ Status DBImpl::WriteImplWALOnly( if (seq_used != nullptr) { *seq_used = w.sequence; } - return w.FinalStatus(); + co_return w.FinalStatus(); } // else we are the leader of the write batch group assert(w.state == WriteThread::STATE_GROUP_LEADER); @@ -1048,7 +1839,7 @@ Status DBImpl::WriteImplWALOnly( WriteThread::WriteGroup write_group; write_thread->EnterAsBatchGroupLeader(&w, &write_group); write_thread->ExitAsBatchGroupLeader(write_group, status); - return status; + co_return status; } } @@ -1123,7 +1914,10 @@ Status DBImpl::WriteImplWALOnly( IOStatus io_s; io_s.PermitUncheckedError(); // Allow io_s to be uninitialized if (!write_options.disableWAL) { - io_s = ConcurrentWriteToWAL(write_group, log_used, &last_sequence, seq_inc); + auto result = AsyncConcurrentWriteToWAL(write_group, log_used, + &last_sequence, seq_inc); + co_await result; + io_s = result.io_result(); status = io_s; } else { // Otherwise we inc seq number to do solely the seq allocation @@ -1151,9 +1945,13 @@ Status DBImpl::WriteImplWALOnly( // Requesting sync with two_write_queues_ is expected to be very rare. We // hance provide a simple implementation that is not necessarily efficient. if (manual_wal_flush_) { - status = FlushWAL(true); + auto result = AsyncFlushWAL(true); + co_await result; + status = result.status(); } else { - status = SyncWAL(); + auto result = AsSyncWAL(); + co_await result; + status = result.status(); } } PERF_TIMER_START(write_pre_and_post_process_time); @@ -1195,7 +1993,7 @@ Status DBImpl::WriteImplWALOnly( if (seq_used != nullptr) { *seq_used = w.sequence; } - return status; + co_return status; } void DBImpl::WriteStatusCheckOnLocked(const Status& status) { @@ -1442,6 +2240,42 @@ IOStatus DBImpl::WriteToWAL(const WriteBatch& merged_batch, return io_s; } +Async_future DBImpl::AsyncWriteToWAL(const WriteBatch& merged_batch, + log::Writer* log_writer, + uint64_t* log_used, uint64_t* log_size) { + assert(log_size != nullptr); + Slice log_entry = WriteBatchInternal::Contents(&merged_batch); + *log_size = log_entry.size(); + // When two_write_queues_ WriteToWAL has to be protected from concurretn calls + // from the two queues anyway and log_write_mutex_ is already held. Otherwise + // if manual_wal_flush_ is enabled we need to protect log_writer->AddRecord + // from possible concurrent calls via the FlushWAL by the application. + const bool needs_locking = manual_wal_flush_ && !two_write_queues_; + // Due to performance cocerns of missed branch prediction penalize the new + // manual_wal_flush_ feature (by UNLIKELY) instead of the more common case + // when we do not need any locking. + if (UNLIKELY(needs_locking)) { + log_write_mutex_.Lock(); + } + + auto result = log_writer->AsyncAddRecord(log_entry); + co_await result; + IOStatus io_s = result.io_result(); + + if (UNLIKELY(needs_locking)) { + log_write_mutex_.Unlock(); + } + if (log_used != nullptr) { + *log_used = logfile_number_; + } + total_log_size_ += log_entry.size(); + // TODO(myabandeh): it might be unsafe to access alive_log_files_.back() here + // since alive_log_files_ might be modified concurrently + alive_log_files_.back().AddSize(log_entry.size()); + log_empty_ = false; + co_return io_s; +} + IOStatus DBImpl::WriteToWAL(const WriteThread::WriteGroup& write_group, log::Writer* log_writer, uint64_t* log_used, bool need_log_sync, bool need_log_dir_sync, @@ -1533,6 +2367,80 @@ IOStatus DBImpl::WriteToWAL(const WriteThread::WriteGroup& write_group, return io_s; } +Async_future DBImpl::AsyncWriteToWAL(const WriteThread::WriteGroup& write_group, + log::Writer* log_writer, + uint64_t* log_used, bool need_log_sync, + bool need_log_dir_sync, + SequenceNumber sequence) { + IOStatus io_s; + assert(!write_group.leader->disable_wal); + // Same holds for all in the batch group + size_t write_with_wal = 0; + WriteBatch* to_be_cached_state = nullptr; + WriteBatch* merged_batch = MergeBatch(write_group, &tmp_batch_, + &write_with_wal, &to_be_cached_state); + if (merged_batch == write_group.leader->multi_batch.batches[0]) { + write_group.leader->log_used = logfile_number_; + } else if (write_with_wal > 1) { + for (auto writer : write_group) { + writer->log_used = logfile_number_; + } + } + + WriteBatchInternal::SetSequence(merged_batch, sequence); + + uint64_t log_size; + auto result = AsyncWriteToWAL(*merged_batch, log_writer, log_used, &log_size); + co_await result; + io_s = result.io_result(); + if (to_be_cached_state) { + cached_recoverable_state_ = *to_be_cached_state; + cached_recoverable_state_empty_ = false; + } + + if (io_s.ok() && need_log_sync) { + StopWatch sw(immutable_db_options_.clock, stats_, WAL_FILE_SYNC_MICROS); + // It's safe to access logs_ with unlocked mutex_ here because: + // - we've set getting_synced=true for all logs, + // so other threads won't pop from logs_ while we're here, + // - only writer thread can push to logs_, and we're in + // writer thread, so no one will push to logs_, + // - as long as other threads don't modify it, it's safe to read + // from std::deque from multiple threads concurrently. + for (auto& log : logs_) { + auto res = log.writer->file()->AsSync(immutable_db_options_.use_fsync); + co_await res; + io_s = res.io_result(); + if (!io_s.ok()) { + break; + } + } + + if (io_s.ok() && need_log_dir_sync) { + // We only sync WAL directory the first time WAL syncing is + // requested, so that in case users never turn on WAL sync, + // we can avoid the disk I/O in the write code path. + io_s = directories_.GetWalDir()->Fsync(IOOptions(), nullptr); + } + } + + if (merged_batch == &tmp_batch_) { + tmp_batch_.Clear(); + } + if (io_s.ok()) { + auto stats = default_cf_internal_stats_; + if (need_log_sync) { + stats->AddDBStats(InternalStats::kIntStatsWalFileSynced, 1); + RecordTick(stats_, WAL_FILE_SYNCED); + } + stats->AddDBStats(InternalStats::kIntStatsWalFileBytes, log_size); + RecordTick(stats_, WAL_FILE_BYTES, log_size); + stats->AddDBStats(InternalStats::kIntStatsWriteWithWal, write_with_wal); + RecordTick(stats_, WRITE_WITH_WAL, write_with_wal); + } + co_return io_s; +} + IOStatus DBImpl::ConcurrentWriteToWAL( const WriteThread::WriteGroup& write_group, uint64_t* log_used, SequenceNumber* last_sequence, size_t seq_inc) { @@ -1589,6 +2497,57 @@ IOStatus DBImpl::ConcurrentWriteToWAL( return io_s; } +Async_future DBImpl::AsyncConcurrentWriteToWAL( + const WriteThread::WriteGroup& write_group, uint64_t* log_used, + SequenceNumber* last_sequence, size_t seq_inc) { + IOStatus io_s; + + assert(!write_group.leader->disable_wal); + // Same holds for all in the batch group + WriteBatch tmp_batch; + size_t write_with_wal = 0; + WriteBatch* to_be_cached_state = nullptr; + WriteBatch* merged_batch = + MergeBatch(write_group, &tmp_batch, &write_with_wal, &to_be_cached_state); + + // We need to lock log_write_mutex_ since logs_ and alive_log_files might be + // pushed back concurrently + log_write_mutex_.Lock(); + if (merged_batch == write_group.leader->multi_batch.batches[0]) { + write_group.leader->log_used = logfile_number_; + } else if (write_with_wal > 1) { + for (auto writer : write_group) { + writer->log_used = logfile_number_; + } + } + *last_sequence = versions_->FetchAddLastAllocatedSequence(seq_inc); + auto sequence = *last_sequence + 1; + WriteBatchInternal::SetSequence(merged_batch, sequence); + + log::Writer* log_writer = logs_.back().writer; + uint64_t log_size; + auto result = AsyncWriteToWAL(*merged_batch, log_writer, log_used, &log_size); + co_await result; + io_s = result.io_result(); + if (to_be_cached_state) { + cached_recoverable_state_ = *to_be_cached_state; + cached_recoverable_state_empty_ = false; + } + log_write_mutex_.Unlock(); + + if (io_s.ok()) { + const bool concurrent = true; + auto stats = default_cf_internal_stats_; + stats->AddDBStats(InternalStats::kIntStatsWalFileBytes, log_size, + concurrent); + RecordTick(stats_, WAL_FILE_BYTES, log_size); + stats->AddDBStats(InternalStats::kIntStatsWriteWithWal, write_with_wal, + concurrent); + RecordTick(stats_, WRITE_WITH_WAL, write_with_wal); + } + co_return io_s; +} + Status DBImpl::WriteRecoverableState() { mutex_.AssertHeld(); if (!cached_recoverable_state_empty_) { @@ -2384,6 +3343,24 @@ Status DB::Put(const WriteOptions& opt, ColumnFamilyHandle* column_family, return Write(opt, &batch); } +Async_future DBImpl::AsyncPut(const WriteOptions& opt, + ColumnFamilyHandle* column_family, + const Slice& key, const Slice& value) { + + ColumnFamilyHandle* default_cf = DefaultColumnFamily(); + assert(default_cf); + const Comparator* const default_cf_ucmp = default_cf->GetComparator(); + assert(default_cf_ucmp); + WriteBatch batch(0, 0, 0, default_cf_ucmp->timestamp_size()); + Status s = batch.Put(column_family, key, value); + if (!s.ok()) { + co_return s; + } + auto result = AsyncWrite(opt, &batch); + co_await result; + co_return result.status(); +} + Status DB::Delete(const WriteOptions& opt, ColumnFamilyHandle* column_family, const Slice& key) { WriteBatch batch; diff --git a/db/db_test_util.h b/db/db_test_util.h index 5d0e53f2da83..0b779b1216b5 100644 --- a/db/db_test_util.h +++ b/db/db_test_util.h @@ -23,6 +23,7 @@ #include "db/db_impl/db_impl.h" #include "file/filename.h" +#include "rocksdb/async_future.h" #include "rocksdb/cache.h" #include "rocksdb/compaction_filter.h" #include "rocksdb/convenience.h" @@ -831,6 +832,41 @@ class CacheWrapper : public Cache { std::shared_ptr target_; }; +class DBAsyncTestBase : public testing::Test { + public: + DBAsyncTestBase(const std::string path) { + Env* env = Env::Default(); + env->SetBackgroundThreads(1, Env::LOW); + env->SetBackgroundThreads(1, Env::HIGH); + dbname_ = test::PerThreadDBPath(env, path); + Options options; + options.create_if_missing = true; + options.env = env; + auto s = DB::Open(options, dbname_, &db_); + std::cout << "Open:" << s.ToString() << "\n"; + } + + ~DBAsyncTestBase() { + db_->Close(); + delete db_; + db_ = nullptr; + } + + // if set to true, IO_URING handling logic is delegated to lambda passed by + // caller. + bool test_delegation() { return test_delegation_; } + void set_test_delegation(bool test_delegation) { + test_delegation_ = test_delegation; + } + + DB* db() { return db_; } + + private: + std::string dbname_; + DB* db_; + bool test_delegation_ = false; +}; + class DBTestBase : public testing::Test { public: // Sequence of option configurations to try diff --git a/db/log_writer.cc b/db/log_writer.cc index e2e596596aa5..3bdb058529b2 100644 --- a/db/log_writer.cc +++ b/db/log_writer.cc @@ -39,6 +39,12 @@ Writer::~Writer() { IOStatus Writer::WriteBuffer() { return dest_->Flush(); } +Async_future Writer::AsyncWriteBuffer() { + auto result = dest_->AsyncFlush(); + co_await result; + co_return result.io_result(); +} + IOStatus Writer::Close() { IOStatus s; if (dest_) { @@ -112,6 +118,72 @@ IOStatus Writer::AddRecord(const Slice& slice) { return s; } +Async_future Writer::AsyncAddRecord(const Slice& slice) { + const char* ptr = slice.data(); + size_t left = slice.size(); + + // Header size varies depending on whether we are recycling or not. + const int header_size = + recycle_log_files_ ? kRecyclableHeaderSize : kHeaderSize; + + // Fragment the record if necessary and emit it. Note that if slice + // is empty, we still want to iterate once to emit a single + // zero-length record + IOStatus s; + bool begin = true; + do { + const int64_t leftover = kBlockSize - block_offset_; + assert(leftover >= 0); + if (leftover < header_size) { + // Switch to a new block + if (leftover > 0) { + // Fill the trailer (literal below relies on kHeaderSize and + // kRecyclableHeaderSize being <= 11) + assert(header_size <= 11); + s = dest_->Append(Slice("\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00", + static_cast(leftover))); + if (!s.ok()) { + break; + } + } + block_offset_ = 0; + } + + // Invariant: we never leave < header_size bytes in a block. + assert(static_cast(kBlockSize - block_offset_) >= header_size); + + const size_t avail = kBlockSize - block_offset_ - header_size; + const size_t fragment_length = (left < avail) ? left : avail; + + RecordType type; + const bool end = (left == fragment_length); + if (begin && end) { + type = recycle_log_files_ ? kRecyclableFullType : kFullType; + } else if (begin) { + type = recycle_log_files_ ? kRecyclableFirstType : kFirstType; + } else if (end) { + type = recycle_log_files_ ? kRecyclableLastType : kLastType; + } else { + type = recycle_log_files_ ? kRecyclableMiddleType : kMiddleType; + } + + s = EmitPhysicalRecord(type, ptr, fragment_length); + ptr += fragment_length; + left -= fragment_length; + begin = false; + } while (s.ok() && left > 0); + + if (s.ok()) { + if (!manual_flush_) { + auto result = dest_->AsyncFlush(); + co_await result; + s = result.io_result(); + } + } + + co_return s; +} + bool Writer::TEST_BufferIsEmpty() { return dest_->TEST_BufferIsEmpty(); } IOStatus Writer::EmitPhysicalRecord(RecordType t, const char* ptr, size_t n) { diff --git a/db/log_writer.h b/db/log_writer.h index 1a91b21994d3..b0c893c34565 100644 --- a/db/log_writer.h +++ b/db/log_writer.h @@ -12,6 +12,7 @@ #include #include "db/log_format.h" +#include "rocksdb/async_future.h" #include "rocksdb/io_status.h" #include "rocksdb/slice.h" #include "rocksdb/status.h" @@ -81,6 +82,8 @@ class Writer { IOStatus AddRecord(const Slice& slice); + Async_future AsyncAddRecord(const Slice& slice); + WritableFileWriter* file() { return dest_.get(); } const WritableFileWriter* file() const { return dest_.get(); } @@ -88,6 +91,8 @@ class Writer { IOStatus WriteBuffer(); + Async_future AsyncWriteBuffer(); + IOStatus Close(); bool TEST_BufferIsEmpty(); diff --git a/db/table_cache.cc b/db/table_cache.cc index abf5a55d2c39..b6dac811eccf 100644 --- a/db/table_cache.cc +++ b/db/table_cache.cc @@ -485,6 +485,96 @@ Status TableCache::Get( return s; } +Async_future TableCache::AsyncGet(const ReadOptions& options, + const InternalKeyComparator& internal_comparator, + const FileMetaData& file_meta, const Slice& k, + GetContext* get_context, + const std::shared_ptr& prefix_extractor, + HistogramImpl* file_read_hist, bool skip_filters, + int level, size_t max_file_size_for_l0_meta_pin) { + auto& fd = file_meta.fd; + std::string* row_cache_entry = nullptr; + bool done = false; +#ifndef ROCKSDB_LITE + IterKey row_cache_key; + std::string row_cache_entry_buffer; + + // Check row cache if enabled. Since row cache does not currently store + // sequence numbers, we cannot use it if we need to fetch the sequence. + if (ioptions_.row_cache && !get_context->NeedToReadSequence()) { + auto user_key = ExtractUserKey(k); + CreateRowCacheKeyPrefix(options, fd, k, get_context, row_cache_key); + done = GetFromRowCache(user_key, row_cache_key, row_cache_key.Size(), + get_context); + if (!done) { + row_cache_entry = &row_cache_entry_buffer; + } + } +#endif // ROCKSDB_LITE + Status s; + TableReader* t = fd.table_reader; + Cache::Handle* handle = nullptr; + if (!done) { + assert(s.ok()); + if (t == nullptr) { + s = FindTable(options, file_options_, internal_comparator, fd, &handle, + prefix_extractor, + options.read_tier == kBlockCacheTier /* no_io */, + true /* record_read_stats */, file_read_hist, skip_filters, + level, true /* prefetch_index_and_filter_in_cache */, + max_file_size_for_l0_meta_pin); + if (s.ok()) { + t = GetTableReaderFromHandle(handle); + } + } + SequenceNumber* max_covering_tombstone_seq = + get_context->max_covering_tombstone_seq(); + if (s.ok() && max_covering_tombstone_seq != nullptr && + !options.ignore_range_deletions) { + std::unique_ptr range_del_iter( + t->NewRangeTombstoneIterator(options)); + if (range_del_iter != nullptr) { + *max_covering_tombstone_seq = std::max( + *max_covering_tombstone_seq, + range_del_iter->MaxCoveringTombstoneSeqnum(ExtractUserKey(k))); + } + } + if (s.ok()) { + get_context->SetReplayLog(row_cache_entry); // nullptr if no cache. + auto a_result = t->AsyncGet(options, k, get_context, prefix_extractor.get(), + skip_filters); + co_await a_result; + s = a_result.status(); + get_context->SetReplayLog(nullptr); + + } else if (options.read_tier == kBlockCacheTier && s.IsIncomplete()) { + // Couldn't find Table in cache but treat as kFound if no_io set + get_context->MarkKeyMayExist(); + s = Status::OK(); + done = true; + } + } + +#ifndef ROCKSDB_LITE + // Put the replay log in row cache only if something was found. + if (!done && s.ok() && row_cache_entry && !row_cache_entry->empty()) { + size_t charge = + row_cache_key.Size() + row_cache_entry->size() + sizeof(std::string); + void* row_ptr = new std::string(std::move(*row_cache_entry)); + // If row cache is full, it's OK to continue. + ioptions_.row_cache + ->Insert(row_cache_key.GetUserKey(), row_ptr, charge, + &DeleteEntry) + .PermitUncheckedError(); + } +#endif // ROCKSDB_LITE + + if (handle != nullptr) { + ReleaseHandle(handle); + } + co_return s; +} + // Batched version of TableCache::MultiGet. Status TableCache::MultiGet( const ReadOptions& options, diff --git a/db/table_cache.h b/db/table_cache.h index fce50775ba2f..0272a1c5af00 100644 --- a/db/table_cache.h +++ b/db/table_cache.h @@ -18,6 +18,7 @@ #include "db/range_del_aggregator.h" #include "options/cf_options.h" #include "port/port.h" +#include "rocksdb/async_future.h" #include "rocksdb/cache.h" #include "rocksdb/env.h" #include "rocksdb/options.h" @@ -98,6 +99,15 @@ class TableCache { HistogramImpl* file_read_hist = nullptr, bool skip_filters = false, int level = -1, size_t max_file_size_for_l0_meta_pin = 0); + Async_future AsyncGet(const ReadOptions& options, + const InternalKeyComparator& internal_comparator, + const FileMetaData& file_meta, const Slice& k, + GetContext* get_context, + const std::shared_ptr& prefix_extractor = nullptr, + HistogramImpl* file_read_hist = nullptr, + bool skip_filters = false, int level = -1, + size_t max_file_size_for_l0_meta_pin = 0); + // Return the range delete tombstone iterator of the file specified by // `file_meta`. Status GetRangeTombstoneIterator( diff --git a/db/version_set.cc b/db/version_set.cc index 8b4eea234832..8c1003140ea5 100644 --- a/db/version_set.cc +++ b/db/version_set.cc @@ -2134,6 +2134,184 @@ void Version::Get(const ReadOptions& read_options, const LookupKey& k, } } +Async_future Version::AsyncGet(const ReadOptions& read_options, + const LookupKey& k, PinnableSlice* value, + std::string* timestamp, Status* status, + MergeContext* merge_context, + SequenceNumber* max_covering_tombstone_seq, + PinnedIteratorsManager* pinned_iters_mgr, + bool* value_found, bool* key_exists, + SequenceNumber* seq, ReadCallback* callback, + bool* is_blob, bool do_merge) { + Slice ikey = k.internal_key(); + Slice user_key = k.user_key(); + + assert(status->ok() || status->IsMergeInProgress()); + + if (key_exists != nullptr) { + // will falsify below if not found + *key_exists = true; + } + + uint64_t tracing_get_id = BlockCacheTraceHelper::kReservedGetId; + if (vset_ && vset_->block_cache_tracer_ && + vset_->block_cache_tracer_->is_tracing_enabled()) { + tracing_get_id = vset_->block_cache_tracer_->NextGetId(); + } + + // Note: the old StackableDB-based BlobDB passes in + // GetImplOptions::is_blob_index; for the integrated BlobDB implementation, we + // need to provide it here. + bool is_blob_index = false; + bool* const is_blob_to_use = is_blob ? is_blob : &is_blob_index; + BlobFetcher blob_fetcher(this, read_options); + + GetContext get_context( + user_comparator(), merge_operator_, info_log_, db_statistics_, + status->ok() ? GetContext::kNotFound : GetContext::kMerge, user_key, + do_merge ? value : nullptr, do_merge ? timestamp : nullptr, value_found, + merge_context, do_merge, max_covering_tombstone_seq, clock_, seq, + merge_operator_ ? pinned_iters_mgr : nullptr, callback, is_blob_to_use, + tracing_get_id, &blob_fetcher); + + // Pin blocks that we read to hold merge operands + if (merge_operator_) { + pinned_iters_mgr->StartPinning(); + } + + FilePicker fp( + user_key, ikey, &storage_info_.level_files_brief_, + storage_info_.num_non_empty_levels_, &storage_info_.file_indexer_, + user_comparator(), internal_comparator()); + FdWithKeyRange* f = fp.GetNextFile(); + + while (f != nullptr) { + if (*max_covering_tombstone_seq > 0) { + // The remaining files we look at will only contain covered keys, so we + // stop here. + break; + } + if (get_context.sample()) { + sample_file_read_inc(f->file_metadata); + } + + bool timer_enabled = + GetPerfLevel() >= PerfLevel::kEnableTimeExceptForMutex && + get_perf_context()->per_level_perf_context_enabled; + StopWatchNano timer(clock_, timer_enabled /* auto_start */); + auto a_result = table_cache_->AsyncGet( + read_options, *internal_comparator(), *f->file_metadata, ikey, + &get_context, mutable_cf_options_.prefix_extractor, + cfd_->internal_stats()->GetFileReadHist(fp.GetHitFileLevel()), + IsFilterSkipped(static_cast(fp.GetHitFileLevel()), + fp.IsHitFileLastInLevel()), + fp.GetHitFileLevel(), max_file_size_for_l0_meta_pin_); + co_await a_result; + *status = a_result.status(); + + // TODO: examine the behavior for corrupted key + if (timer_enabled) { + PERF_COUNTER_BY_LEVEL_ADD(get_from_table_nanos, timer.ElapsedNanos(), + fp.GetHitFileLevel()); + } + if (!status->ok()) { + if (db_statistics_ != nullptr) { + get_context.ReportCounters(); + } + co_return Status::OK(); + } + + // report the counters before returning + if (get_context.State() != GetContext::kNotFound && + get_context.State() != GetContext::kMerge && + db_statistics_ != nullptr) { + get_context.ReportCounters(); + } + switch (get_context.State()) { + case GetContext::kNotFound: + // Keep searching in other files + break; + case GetContext::kMerge: + // TODO: update per-level perfcontext user_key_return_count for kMerge + break; + case GetContext::kFound: + if (fp.GetHitFileLevel() == 0) { + RecordTick(db_statistics_, GET_HIT_L0); + } else if (fp.GetHitFileLevel() == 1) { + RecordTick(db_statistics_, GET_HIT_L1); + } else if (fp.GetHitFileLevel() >= 2) { + RecordTick(db_statistics_, GET_HIT_L2_AND_UP); + } + + PERF_COUNTER_BY_LEVEL_ADD(user_key_return_count, 1, + fp.GetHitFileLevel()); + + if (is_blob_index) { + if (do_merge && value) { + constexpr uint64_t* bytes_read = nullptr; + constexpr FilePrefetchBuffer* prefetch_buffer = nullptr; + + *status = + GetBlob(read_options, user_key, *value, prefetch_buffer, + value, bytes_read); + if (!status->ok()) { + if (status->IsIncomplete()) { + get_context.MarkKeyMayExist(); + } + co_return Status::OK(); + } + } + } + + co_return Status::OK(); + case GetContext::kDeleted: + // Use empty error message for speed + *status = Status::NotFound(); + co_return Status::OK(); + case GetContext::kCorrupt: + *status = Status::Corruption("corrupted key for ", user_key); + co_return Status::OK(); + case GetContext::kUnexpectedBlobIndex: + ROCKS_LOG_ERROR(info_log_, "Encounter unexpected blob index."); + *status = Status::NotSupported( + "Encounter unexpected blob index. Please open DB with " + "ROCKSDB_NAMESPACE::blob_db::BlobDB instead."); + co_return Status::OK(); + } + f = fp.GetNextFile(); + } + if (db_statistics_ != nullptr) { + get_context.ReportCounters(); + } + if (GetContext::kMerge == get_context.State()) { + if (!do_merge) { + *status = Status::OK(); + co_return Status::OK(); + } + if (!merge_operator_) { + *status = Status::InvalidArgument( + "merge_operator is not properly initialized."); + co_return Status::OK(); + } + // merge_operands are in saver and we hit the beginning of the key history + // do a final merge of nullptr and operands; + std::string* str_value = value != nullptr ? value->GetSelf() : nullptr; + *status = MergeHelper::TimedFullMerge( + merge_operator_, user_key, nullptr, merge_context->GetOperands(), + str_value, info_log_, db_statistics_, clock_, + nullptr /* result_operand */, true); + if (LIKELY(value != nullptr)) { + value->PinSelf(); + } + } else { + if (key_exists != nullptr) { + *key_exists = false; + } + *status = Status::NotFound(); // Use an empty error message for speed + co_return Status::NotFound(); + } +} + void Version::MultiGet(const ReadOptions& read_options, MultiGetRange* range, ReadCallback* callback) { PinnedIteratorsManager pinned_iters_mgr; diff --git a/db/version_set.h b/db/version_set.h index 223c80425200..193cb9006d52 100644 --- a/db/version_set.h +++ b/db/version_set.h @@ -48,6 +48,7 @@ #include "monitoring/instrumented_mutex.h" #include "options/db_options.h" #include "port/port.h" +#include "rocksdb/async_future.h" #include "rocksdb/env.h" #include "rocksdb/file_checksum.h" #include "table/get_context.h" @@ -781,6 +782,15 @@ class Version { SequenceNumber* seq = nullptr, ReadCallback* callback = nullptr, bool* is_blob = nullptr, bool do_merge = true); + Async_future AsyncGet( + const ReadOptions&, const LookupKey& key, PinnableSlice* value, + std::string* timestamp, Status* status, MergeContext* merge_context, + SequenceNumber* max_covering_tombstone_seq, + PinnedIteratorsManager* pinned_iters_mgr, + bool* value_found = nullptr, bool* key_exists = nullptr, + SequenceNumber* seq = nullptr, ReadCallback* callback = nullptr, + bool* is_blob = nullptr, bool do_merge = true); + void MultiGet(const ReadOptions&, MultiGetRange* range, ReadCallback* callback = nullptr); diff --git a/env/env.cc b/env/env.cc index 186429c1573e..5079de1b1929 100644 --- a/env/env.cc +++ b/env/env.cc @@ -152,6 +152,17 @@ class LegacyRandomAccessFileWrapper : public FSRandomAccessFile { return status_to_io_status(target_->Read(offset, n, result, scratch)); } + Async_future AsyncRead(uint64_t offset, size_t n, + const IOOptions& /*options*/, Slice* result, + char* scratch, + IODebugContext* /*dbg*/) const override { + (void)offset; + (void)n; + (void)result; + (void)scratch; + throw "Not supported"; + } + IOStatus MultiRead(FSReadRequest* fs_reqs, size_t num_reqs, const IOOptions& /*options*/, IODebugContext* /*dbg*/) override { @@ -253,6 +264,17 @@ class LegacyWritableFileWrapper : public FSWritableFile { IODebugContext* /*dbg*/) override { return status_to_io_status(target_->Append(data)); } + Async_future AsyncAppend(const Slice& data, const IOOptions& /*options*/, + IODebugContext* /*dbg*/) override { + (void)data; + throw "Not implemented"; + } + Async_future AsyncAppend(const Slice& data, const IOOptions& /*options*/, + const DataVerificationInfo& /*verification_info*/, + IODebugContext* /*dbg*/) override { + (void)data; + throw "Not implemented"; + } IOStatus PositionedAppend(const Slice& data, uint64_t offset, const IOOptions& /*options*/, IODebugContext* /*dbg*/) override { diff --git a/env/io_posix.cc b/env/io_posix.cc index 56494d5651de..f382fe4da475 100644 --- a/env/io_posix.cc +++ b/env/io_posix.cc @@ -9,9 +9,13 @@ #ifdef ROCKSDB_LIB_IO_POSIX #include "env/io_posix.h" + #include #include + #include +#include +#include #if defined(OS_LINUX) #include #ifndef FALLOC_FL_KEEP_SIZE @@ -45,6 +49,8 @@ namespace ROCKSDB_NAMESPACE { +constexpr int ASYNC_PAGE_SIZE{4096}; + std::string IOErrorMsg(const std::string& context, const std::string& file_name) { if (file_name.empty()) { @@ -117,6 +123,37 @@ bool PosixWrite(int fd, const char* buf, size_t nbyte) { return true; } +Async_future AsyncPosixWrite(const IOOptions& opts, int fd, const char* buf, + size_t nbyte) { + int pages = (int)std::ceil((float)nbyte / ASYNC_PAGE_SIZE); + int last_page_size = nbyte % ASYNC_PAGE_SIZE; + int page_size = ASYNC_PAGE_SIZE; + // Will be deleted by the io_uring completion routine. + auto ctx = new Async_future::IO_ctx(pages); + char* no_const_buf = const_cast(buf); + + for (int i = 0; i < pages; ++i) { + ctx->m_iov[i].iov_base = no_const_buf + i * page_size; + if (i == pages - 1 && last_page_size != 0) { + page_size = last_page_size; + } + ctx->m_iov[i].iov_len = page_size; + } + + if (opts.submit_queue->m_iouring != nullptr) { + auto sqe = io_uring_get_sqe(opts.submit_queue->m_iouring); + io_uring_prep_writev(sqe, fd, ctx->m_iov.data(), ctx->m_iov.size(), 0); + io_uring_sqe_set_data(sqe, ctx); + io_uring_submit(opts.submit_queue->m_iouring); + co_await Async_future(true, ctx); + } else { + using Ops = Async_future::Submit_queue::Ops; + co_await opts.submit_queue->m_delegate(ctx, fd, 0, Ops::Write); + } + + co_return true; +} + bool PosixPositionedWrite(int fd, const char* buf, size_t nbyte, off_t offset) { const size_t kLimit1Gb = 1UL << 30; @@ -141,6 +178,38 @@ bool PosixPositionedWrite(int fd, const char* buf, size_t nbyte, off_t offset) { return true; } +Async_future AsyncPosixPositionedWrite(const IOOptions& opts, int fd, + const char* buf, size_t nbyte, + off_t offset) { + int pages = (int)std::ceil((float)nbyte / ASYNC_PAGE_SIZE); + int last_page_size = nbyte % ASYNC_PAGE_SIZE; + int page_size = ASYNC_PAGE_SIZE; + // Will be deleted by the io_uring completion routine. + auto ctx = new Async_future::IO_ctx(pages); + char* no_const_buf = const_cast(buf); + + for (int i = 0; i < pages; i++) { + ctx->m_iov[i].iov_base = no_const_buf + i * page_size; + if (i == pages - 1 && last_page_size != 0) { + page_size = last_page_size; + } + ctx->m_iov[i].iov_len = page_size; + } + + if (opts.submit_queue->m_iouring != nullptr) { + auto sqe = io_uring_get_sqe(opts.submit_queue->m_iouring); + io_uring_prep_writev(sqe, fd, ctx->m_iov.data(), ctx->m_iov.size(), offset); + io_uring_sqe_set_data(sqe, ctx); + io_uring_submit(opts.submit_queue->m_iouring); + co_await Async_future(true, ctx); + } else { + using Ops = Async_future::Submit_queue::Ops; + co_await opts.submit_queue->m_delegate(ctx, fd, offset, Ops::Write); + } + + co_return true; +} + #ifdef ROCKSDB_RANGESYNC_PRESENT #if !defined(ZFS_SUPER_MAGIC) @@ -604,6 +673,81 @@ IOStatus PosixRandomAccessFile::Read(uint64_t offset, size_t n, return s; } +Async_future PosixRandomAccessFile::AsyncRead(uint64_t offset, size_t n, + const IOOptions& opts, + Slice* result, char* scratch, + IODebugContext* /*dbg*/) const { + assert(opts.submit_queue != nullptr); + + if (use_direct_io()) { + assert(IsSectorAligned(offset, GetRequiredBufferAlignment())); + assert(IsSectorAligned(n, GetRequiredBufferAlignment())); + assert(IsSectorAligned(scratch, GetRequiredBufferAlignment())); + } + + int n_pages = n / ASYNC_PAGE_SIZE; + const int last_page_size = n % ASYNC_PAGE_SIZE; + + if (last_page_size > 0) { + ++n_pages; + } + + auto ptr{scratch}; + // Will be deleted by the io_uring completion routine. + auto ctx = new Async_future::IO_ctx(n_pages); + auto iov{&ctx->m_iov[0]}; + + for (int i{}; i < n_pages; ++i, ptr += (i * ASYNC_PAGE_SIZE), ++iov) { + iov->iov_base = ptr; + iov->iov_len = ASYNC_PAGE_SIZE; + } + + if (last_page_size != 0) { + assert(n_pages > 0); + + iov = &ctx->m_iov[n_pages - 1]; + iov->iov_len = last_page_size; + // FIXME: Valgring complains about: + // "Conditional jump or move depends on uninitialised value(s)" + std::memset(iov->iov_base, 0x0, iov->iov_len); + } + + if (opts.submit_queue->m_iouring != nullptr) { + auto sqe{io_uring_get_sqe(opts.submit_queue->m_iouring)}; + + if (sqe == nullptr) { + /* Submission queue is full. */ + const auto error{Status::SubCode::kIOUringSqeFull}; + + co_return IOStatus::IOError(error, Slice()); + } + + io_uring_prep_readv(sqe, fd_, ctx->m_iov.data(), ctx->m_iov.size(), offset); + io_uring_sqe_set_data(sqe, ctx); + + const auto ret = io_uring_submit(opts.submit_queue->m_iouring); + + if (ret < 0) { + const auto error{Status::SubCode::kIOUringSubmitError}; + + co_return IOStatus::IOError(error, strerror(-ret)); + } + + co_await Async_future{true, ctx}; + + *result = Slice(scratch, n); + + co_return IOStatus::OK(); + } else { + using Ops = Async_future::Submit_queue::Ops; + auto delegate{opts.submit_queue->m_delegate}; + auto r = delegate(ctx, fd_, offset, Ops::Read); + + co_await r; + co_return r.io_result(); + } +} + IOStatus PosixRandomAccessFile::MultiRead(FSReadRequest* reqs, size_t num_reqs, const IOOptions& options, @@ -1236,6 +1380,25 @@ IOStatus PosixWritableFile::Append(const Slice& data, const IOOptions& /*opts*/, return IOStatus::OK(); } +Async_future PosixWritableFile::AsyncAppend(const Slice& data, + const IOOptions& opts, + IODebugContext* /*dbg*/) { + if (use_direct_io()) { + assert(IsSectorAligned(data.size(), GetRequiredBufferAlignment())); + assert(IsSectorAligned(data.data(), GetRequiredBufferAlignment())); + } + const char* src = data.data(); + size_t nbytes = data.size(); + auto result = AsyncPosixWrite(opts, fd_, src, nbytes); + co_await result; + if (!result.write_result()) { + co_return IOError("While appending to file", filename_, errno); + } + + filesize_ += nbytes; + co_return IOStatus::OK(); +} + IOStatus PosixWritableFile::PositionedAppend(const Slice& data, uint64_t offset, const IOOptions& /*opts*/, IODebugContext* /*dbg*/) { @@ -1255,6 +1418,30 @@ IOStatus PosixWritableFile::PositionedAppend(const Slice& data, uint64_t offset, return IOStatus::OK(); } +Async_future PosixWritableFile::AsyncPositionedAppend(const Slice& data, + uint64_t offset, + const IOOptions& opts, + IODebugContext* /*dbg*/) { + if (use_direct_io()) { + assert(IsSectorAligned(offset, GetRequiredBufferAlignment())); + assert(IsSectorAligned(data.size(), GetRequiredBufferAlignment())); + assert(IsSectorAligned(data.data(), GetRequiredBufferAlignment())); + } + assert(offset <= static_cast(std::numeric_limits::max())); + const char* src = data.data(); + size_t nbytes = data.size(); + auto result = AsyncPosixPositionedWrite(opts, fd_, src, nbytes, + static_cast(offset)); + co_await result; + if (!result.write_result()) { + co_return IOError("While pwrite to file at offset " + ToString(offset), + filename_, errno); + } + filesize_ = offset + nbytes; + co_return IOStatus::OK(); +} + + IOStatus PosixWritableFile::Truncate(uint64_t size, const IOOptions& /*opts*/, IODebugContext* /*dbg*/) { IOStatus s; @@ -1344,6 +1531,15 @@ IOStatus PosixWritableFile::Sync(const IOOptions& /*opts*/, return IOStatus::OK(); } +Async_future PosixWritableFile::AsSync(const IOOptions& /*opts*/, + IODebugContext* /*dbg*/) { + // TODO: use liburing? + if (fdatasync(fd_) < 0) { + co_return IOError("While fdatasync", filename_, errno); + } + co_return IOStatus::OK(); +} + IOStatus PosixWritableFile::Fsync(const IOOptions& /*opts*/, IODebugContext* /*dbg*/) { #ifdef HAVE_FULLFSYNC @@ -1358,6 +1554,15 @@ IOStatus PosixWritableFile::Fsync(const IOOptions& /*opts*/, return IOStatus::OK(); } +Async_future PosixWritableFile::AsFsync(const IOOptions& /*opts*/, + IODebugContext* /*dbg*/) { + // TODO: use liburing? + if (fsync(fd_) < 0) { + co_return IOError("While fsync", filename_, errno); + } + co_return IOStatus::OK(); +} + bool PosixWritableFile::IsSyncThreadSafe() const { return true; } uint64_t PosixWritableFile::GetFileSize(const IOOptions& /*opts*/, diff --git a/env/io_posix.h b/env/io_posix.h index 94f579c500f7..a276c0fdecd9 100644 --- a/env/io_posix.h +++ b/env/io_posix.h @@ -35,6 +35,7 @@ #endif namespace ROCKSDB_NAMESPACE { + std::string IOErrorMsg(const std::string& context, const std::string& file_name); // file_name can be left empty if it is not unkown. @@ -194,6 +195,11 @@ class PosixRandomAccessFile : public FSRandomAccessFile { Slice* result, char* scratch, IODebugContext* dbg) const override; + virtual Async_future AsyncRead(uint64_t offset, size_t n, + const IOOptions& opts, Slice* result, + char* scratch, + IODebugContext* dbg) const override; + virtual IOStatus MultiRead(FSReadRequest* reqs, size_t num_reqs, const IOOptions& options, IODebugContext* dbg) override; @@ -242,23 +248,48 @@ class PosixWritableFile : public FSWritableFile { virtual IOStatus Close(const IOOptions& opts, IODebugContext* dbg) override; virtual IOStatus Append(const Slice& data, const IOOptions& opts, IODebugContext* dbg) override; + virtual Async_future AsyncAppend(const Slice& data, const IOOptions& opts, + IODebugContext* dbg) override; virtual IOStatus Append(const Slice& data, const IOOptions& opts, const DataVerificationInfo& /* verification_info */, IODebugContext* dbg) override { return Append(data, opts, dbg); } + virtual Async_future AsyncAppend( + const Slice& data, const IOOptions& opts, + const DataVerificationInfo& /* verification_info */, + IODebugContext* dbg) override { + auto result = AsyncAppend(data, opts, dbg); + co_await result; + co_return result.io_result(); + } virtual IOStatus PositionedAppend(const Slice& data, uint64_t offset, const IOOptions& opts, IODebugContext* dbg) override; + virtual Async_future AsyncPositionedAppend(const Slice& data, uint64_t offset, + const IOOptions& opts, + IODebugContext* dbg) override; virtual IOStatus PositionedAppend( const Slice& data, uint64_t offset, const IOOptions& opts, const DataVerificationInfo& /* verification_info */, IODebugContext* dbg) override { return PositionedAppend(data, offset, opts, dbg); } + virtual Async_future AsyncPositionedAppend( + const Slice& data, uint64_t offset, const IOOptions& opts, + const DataVerificationInfo& /* verification_info */, + IODebugContext* dbg) override { + auto result = AsyncPositionedAppend(data, offset, opts, dbg); + co_await result; + co_return result.io_result(); + } virtual IOStatus Flush(const IOOptions& opts, IODebugContext* dbg) override; virtual IOStatus Sync(const IOOptions& opts, IODebugContext* dbg) override; + virtual Async_future AsSync(const IOOptions& opts, + IODebugContext* dbg) override; virtual IOStatus Fsync(const IOOptions& opts, IODebugContext* dbg) override; + virtual Async_future AsFsync(const IOOptions& opts, + IODebugContext* dbg) override; virtual bool IsSyncThreadSafe() const override; virtual bool use_direct_io() const override { return use_direct_io_; } virtual void SetWriteLifeTimeHint(Env::WriteLifeTimeHint hint) override; @@ -296,6 +327,18 @@ class PosixMmapReadableFile : public FSRandomAccessFile { virtual IOStatus Read(uint64_t offset, size_t n, const IOOptions& opts, Slice* result, char* scratch, IODebugContext* dbg) const override; + + Async_future AsyncRead(uint64_t offset, size_t n, const IOOptions& options, + Slice* result, char* scratch, + IODebugContext* dbg) const override { + (void)offset; + (void)n; + (void)options; + (void)result; + (void)scratch; + (void)dbg; + throw "Not implemented"; + } virtual IOStatus InvalidateCache(size_t offset, size_t length) override; }; @@ -342,11 +385,26 @@ class PosixMmapFile : public FSWritableFile { virtual IOStatus Close(const IOOptions& opts, IODebugContext* dbg) override; virtual IOStatus Append(const Slice& data, const IOOptions& opts, IODebugContext* dbg) override; + virtual Async_future AsyncAppend(const Slice& data, const IOOptions& opts, + IODebugContext* dbg) override { + (void)data; + (void)opts; + (void)dbg; + throw "Not implemented"; + } virtual IOStatus Append(const Slice& data, const IOOptions& opts, const DataVerificationInfo& /* verification_info */, IODebugContext* dbg) override { return Append(data, opts, dbg); } + virtual Async_future AsyncAppend( + const Slice& data, const IOOptions& opts, + const DataVerificationInfo& /* verification_info */, + IODebugContext* dbg) override { + auto result = AsyncAppend(data, opts, dbg); + co_await result; + co_return result.io_result(); + } virtual IOStatus Flush(const IOOptions& opts, IODebugContext* dbg) override; virtual IOStatus Sync(const IOOptions& opts, IODebugContext* dbg) override; virtual IOStatus Fsync(const IOOptions& opts, IODebugContext* dbg) override; diff --git a/env/mock_env.cc b/env/mock_env.cc index a85339ea2682..f73862b43374 100644 --- a/env/mock_env.cc +++ b/env/mock_env.cc @@ -326,6 +326,18 @@ class MockRandomAccessFile : public FSRandomAccessFile { } } + Async_future AsyncRead(uint64_t offset, size_t n, const IOOptions& options, + Slice* result, char* scratch, + IODebugContext* dbg) const override { + (void)offset; + (void)n; + (void)options; + (void)result; + (void)scratch; + (void)dbg; + throw "Not implemented"; + } + private: MemFile* file_; bool use_direct_io_; @@ -394,6 +406,21 @@ class MockWritableFile : public FSWritableFile { } return IOStatus::OK(); } + Async_future AsyncAppend(const Slice& data, const IOOptions& options, + IODebugContext* dbg) override { + (void)data; + (void)options; + (void)dbg; + throw "Not implemented"; + } + Async_future AsyncAppend(const Slice& data, const IOOptions& options, + const DataVerificationInfo& /* verification_info */, + IODebugContext* dbg) override { + (void)data; + (void)options; + (void)dbg; + throw "Not implemented"; + } using FSWritableFile::PositionedAppend; IOStatus PositionedAppend(const Slice& data, uint64_t /*offset*/, diff --git a/examples/CMakeLists.txt b/examples/CMakeLists.txt index 0b93a6d8d2d5..4ba3d100fff1 100644 --- a/examples/CMakeLists.txt +++ b/examples/CMakeLists.txt @@ -3,6 +3,11 @@ add_executable(simple_example target_link_libraries(simple_example ${ROCKSDB_LIB}) +add_executable(async_simple_example + async_simple_example.cc) +target_link_libraries(async_simple_example + ${ROCKSDB_LIB}) + add_executable(column_families_example column_families_example.cc) target_link_libraries(column_families_example diff --git a/examples/Makefile b/examples/Makefile index faee6f06bfd4..14db1f012615 100644 --- a/examples/Makefile +++ b/examples/Makefile @@ -9,26 +9,32 @@ ifndef DISABLE_JEMALLOC endif ifneq ($(USE_RTTI), 1) - CXXFLAGS += -fno-rtti + CXXFLAGS += -fno-rtti -fcoroutines endif CFLAGS += -Wstrict-prototypes .PHONY: clean librocksdb -all: simple_example column_families_example compact_files_example c_simple_example optimistic_transaction_example transaction_example compaction_filter_example options_file_example +all: simple_example async_simple_example column_families_example compact_files_example c_simple_example optimistic_transaction_example transaction_example compaction_filter_example options_file_example async_simple_example2 simple_example: librocksdb simple_example.cc - $(CXX) $(CXXFLAGS) $@.cc -o$@ ../librocksdb.a -I../include -O2 -std=c++11 $(PLATFORM_LDFLAGS) $(PLATFORM_CXXFLAGS) $(EXEC_LDFLAGS) + $(CXX) $(CXXFLAGS) $@.cc -o$@ ../librocksdb.a -I../include -O2 -std=c++20 $(PLATFORM_LDFLAGS) $(PLATFORM_CXXFLAGS) $(EXEC_LDFLAGS) + +async_simple_example2: librocksdb async_simple_example2.cc + $(CXX) $(CXXFLAGS) $@.cc -o$@ ../librocksdb_debug.a -I../include -g -O0 -std=c++20 $(PLATFORM_LDFLAGS) $(PLATFORM_CXXFLAGS) $(EXEC_LDFLAGS) + +async_simple_example: librocksdb async_simple_example.cc + $(CXX) $(CXXFLAGS) $@.cc -o$@ ../librocksdb_debug.a -I../include -g -O0 -std=c++20 $(PLATFORM_LDFLAGS) $(PLATFORM_CXXFLAGS) $(EXEC_LDFLAGS) column_families_example: librocksdb column_families_example.cc - $(CXX) $(CXXFLAGS) $@.cc -o$@ ../librocksdb.a -I../include -O2 -std=c++11 $(PLATFORM_LDFLAGS) $(PLATFORM_CXXFLAGS) $(EXEC_LDFLAGS) + $(CXX) $(CXXFLAGS) $@.cc -o$@ ../librocksdb.a -I../include -O2 -std=c++20 $(PLATFORM_LDFLAGS) $(PLATFORM_CXXFLAGS) $(EXEC_LDFLAGS) compaction_filter_example: librocksdb compaction_filter_example.cc - $(CXX) $(CXXFLAGS) $@.cc -o$@ ../librocksdb.a -I../include -O2 -std=c++11 $(PLATFORM_LDFLAGS) $(PLATFORM_CXXFLAGS) $(EXEC_LDFLAGS) + $(CXX) $(CXXFLAGS) $@.cc -o$@ ../librocksdb.a -I../include -O2 -std=c++20 $(PLATFORM_LDFLAGS) $(PLATFORM_CXXFLAGS) $(EXEC_LDFLAGS) compact_files_example: librocksdb compact_files_example.cc - $(CXX) $(CXXFLAGS) $@.cc -o$@ ../librocksdb.a -I../include -O2 -std=c++11 $(PLATFORM_LDFLAGS) $(PLATFORM_CXXFLAGS) $(EXEC_LDFLAGS) + $(CXX) $(CXXFLAGS) $@.cc -o$@ ../librocksdb.a -I../include -O2 -std=c++20 $(PLATFORM_LDFLAGS) $(PLATFORM_CXXFLAGS) $(EXEC_LDFLAGS) .c.o: $(CC) $(CFLAGS) -c $< -o $@ -I../include @@ -37,19 +43,19 @@ c_simple_example: librocksdb c_simple_example.o $(CXX) $@.o -o$@ ../librocksdb.a $(PLATFORM_LDFLAGS) $(EXEC_LDFLAGS) optimistic_transaction_example: librocksdb optimistic_transaction_example.cc - $(CXX) $(CXXFLAGS) $@.cc -o$@ ../librocksdb.a -I../include -O2 -std=c++11 $(PLATFORM_LDFLAGS) $(PLATFORM_CXXFLAGS) $(EXEC_LDFLAGS) + $(CXX) $(CXXFLAGS) $@.cc -o$@ ../librocksdb.a -I../include -O2 -std=c++20 $(PLATFORM_LDFLAGS) $(PLATFORM_CXXFLAGS) $(EXEC_LDFLAGS) transaction_example: librocksdb transaction_example.cc - $(CXX) $(CXXFLAGS) $@.cc -o$@ ../librocksdb.a -I../include -O2 -std=c++11 $(PLATFORM_LDFLAGS) $(PLATFORM_CXXFLAGS) $(EXEC_LDFLAGS) + $(CXX) $(CXXFLAGS) $@.cc -o$@ ../librocksdb.a -I../include -O2 -std=c++20 $(PLATFORM_LDFLAGS) $(PLATFORM_CXXFLAGS) $(EXEC_LDFLAGS) options_file_example: librocksdb options_file_example.cc - $(CXX) $(CXXFLAGS) $@.cc -o$@ ../librocksdb.a -I../include -O2 -std=c++11 $(PLATFORM_LDFLAGS) $(PLATFORM_CXXFLAGS) $(EXEC_LDFLAGS) + $(CXX) $(CXXFLAGS) $@.cc -o$@ ../librocksdb.a -I../include -O2 -std=c++20 $(PLATFORM_LDFLAGS) $(PLATFORM_CXXFLAGS) $(EXEC_LDFLAGS) multi_processes_example: librocksdb multi_processes_example.cc - $(CXX) $(CXXFLAGS) $@.cc -o$@ ../librocksdb.a -I../include -O2 -std=c++11 $(PLATFORM_LDFLAGS) $(PLATFORM_CXXFLAGS) $(EXEC_LDFLAGS) + $(CXX) $(CXXFLAGS) $@.cc -o$@ ../librocksdb.a -I../include -O2 -std=c++20 $(PLATFORM_LDFLAGS) $(PLATFORM_CXXFLAGS) $(EXEC_LDFLAGS) clean: - rm -rf ./simple_example ./column_families_example ./compact_files_example ./compaction_filter_example ./c_simple_example c_simple_example.o ./optimistic_transaction_example ./transaction_example ./options_file_example ./multi_processes_example + rm -rf ./simple_example ./async_simple_example ./column_families_example ./compact_files_example ./compaction_filter_example ./c_simple_example c_simple_example.o ./optimistic_transaction_example ./transaction_example ./options_file_example ./multi_processes_example ./async_simple_example2 -librocksdb: +librocksdb_test_debug.a: cd .. && $(MAKE) static_lib diff --git a/examples/async_simple_example.cc b/examples/async_simple_example.cc new file mode 100644 index 000000000000..e8127cd0aa73 --- /dev/null +++ b/examples/async_simple_example.cc @@ -0,0 +1,204 @@ +// Copyright (c) 2011-present, Facebook, Inc. All rights reserved. +// This source code is licensed under both the GPLv2 (found in the +// COPYING file in the root directory) and Apache 2.0 License +// (found in the LICENSE.Apache file in the root directory). + +#include +#include +#include +#include + +#include "rocksdb/db.h" +#include "rocksdb/slice.h" +#include "rocksdb/options.h" + +using ROCKSDB_NAMESPACE::DB; +using ROCKSDB_NAMESPACE::Async_future; +using ROCKSDB_NAMESPACE::ReadTier; +using ROCKSDB_NAMESPACE::Options; +using ROCKSDB_NAMESPACE::PinnableSlice; +using ROCKSDB_NAMESPACE::ReadOptions; +using ROCKSDB_NAMESPACE::Status; +using ROCKSDB_NAMESPACE::WriteBatch; +using ROCKSDB_NAMESPACE::WriteOptions; +using Submit_queue = Async_future::Submit_queue; + +std::string kDBPath = "/tmp/rocksdb_async_simple_example"; + +class Async { + public: + explicit Async(DB *db) + : m_db(db), + m_io_uring{new io_uring}, + m_shutdown{} { + std::cout << "Async" << std::endl; + + auto ret = io_uring_queue_init(m_io_uring_size, m_io_uring.get(), 0); + + if (ret < 0) { + throw "io_uring_queue_init failed"; + } + + m_submit_queues = std::make_shared( + [this](Async_future::IO_ctx* ctx, int fd, uint64_t off, Submit_queue::Ops op) -> Async_future { + (void)op; + + + auto sqe = io_uring_get_sqe(m_io_uring.get()); + + if (sqe == nullptr) { + /* Submission queue is full */ + co_return rocksdb::IOStatus::IOError(rocksdb::Status::SubCode::kIOUringSqeFull); + } + + io_uring_prep_readv(sqe, fd, ctx->m_iov.data(), ctx->m_iov.size(), off); + + io_uring_sqe_set_data(sqe, ctx); + + const auto ret = io_uring_submit(m_io_uring.get()); + + if (ret < 0) { + co_return rocksdb::IOStatus::IOError(rocksdb::Status::SubCode::kIOUringSubmitError, strerror(-ret)); + } + + std::cout << __LINE__ << " io_uring co_await" << std::endl; + std::cout.flush(); + + co_await Async_future(true, ctx); + + std::cout << __LINE__ << " io_uring after co_await" << std::endl; + std::cout.flush(); + + co_return rocksdb::IOStatus::OK(); + }); + + m_options.verify_checksums = true; + m_options.read_tier = ReadTier::kPersistedTier; + m_options.submit_queue = m_submit_queues; + } + + ~Async() { + std::cout << "~Async" << std::endl; + io_uring_queue_exit(m_io_uring.get()); + } + + void set_shutdown() { + m_shutdown.fetch_sub(1, std::memory_order_seq_cst); + } + + void io_uring_completion() { + do { + std::cout << __LINE__ << " shutdown: " << m_shutdown.load() << std::endl; + + io_uring_cqe* cqe; + const auto ret = io_uring_wait_cqe(m_io_uring.get(), &cqe); + + if (ret == 0 && cqe->res >= 0) { + auto ctx = reinterpret_cast(io_uring_cqe_get_data(cqe)); + + on_resume(ctx->m_promise); + + io_uring_cqe_seen(m_io_uring.get(), cqe); + } + } while (m_shutdown.load(std::memory_order_relaxed) > 0); + } + + Async_future get(const std::string &k, std::string &value) { + m_shutdown.fetch_add(1, std::memory_order_seq_cst); + + auto v = new (std::nothrow) PinnableSlice(); + assert(v != nullptr); + + std::cout << __LINE__ << " - AsyncGet " << k << std::endl; + + auto result = m_db->AsyncGet(m_options, m_db->DefaultColumnFamily(), k, v, nullptr); + + std::cout << __LINE__ << " - co_await " << k << std::endl; + + co_await result; + + std::cout << __LINE__ << " - after co_await " << k << std::endl; + + set_shutdown(); + + value = v->ToString(); + + delete v; + + std::cout << __LINE__ << " co_return - " << k << " -> " << value << std::endl; + + co_return true; + } + + private: + using Promise = Async_future::promise_type; + + static void on_resume(Promise* promise) { + auto h{std::coroutine_handle::from_promise(*promise)}; + + h.resume(); + } + + private: + const int m_io_uring_size = 4; + + DB *m_db{}; + ReadOptions m_options; + std::atomic m_shutdown{}; + std::unique_ptr m_io_uring; + std::shared_ptr m_submit_queues{}; +}; + + +int main() { + DB* db; + + Options options; + + // Optimize RocksDB. This is the easiest way to get RocksDB to perform well + options.IncreaseParallelism(); + options.OptimizeLevelStyleCompaction(); + + options.create_if_missing = true; + + auto s = DB::Open(options, kDBPath, &db); + assert(s.ok()); + + const std::string k1{"k1"}; + const std::string k2{"k2"}; + const std::string k3{"k3"}; + + s = db->Put(WriteOptions(), k1, "v1"); + assert(s.ok()); + + s = db->Put(WriteOptions(), k2, "v2"); + assert(s.ok()); + + delete db; + + std::cout << "Open for real ... " << std::endl; + + s = DB::Open(options, kDBPath, &db); + assert(s.ok()); + + { + Async async{db}; + std::string v1{}; + std::string v2{}; + std::string v3{}; + + auto r1 = async.get(k1, v1); + auto r2 = async.get(k2, v2); + auto r3 = async.get(k3, v3); + + async.io_uring_completion(); + + std::cout << "found: [" + << v1 << "], [" << v2 << "]" << ", [" << v3 + << "]" << std::endl; + } + + delete db; + + return EXIT_SUCCESS;; +} diff --git a/examples/async_simple_example2.cc b/examples/async_simple_example2.cc new file mode 100644 index 000000000000..c1fe4a5f51aa --- /dev/null +++ b/examples/async_simple_example2.cc @@ -0,0 +1,215 @@ +#include +#include +#include +#include + +#include "rocksdb/db.h" +#include "rocksdb/slice.h" +#include "rocksdb/options.h" + +using ROCKSDB_NAMESPACE::DB; +using ROCKSDB_NAMESPACE::Async_future; +using ROCKSDB_NAMESPACE::ReadTier; +using ROCKSDB_NAMESPACE::Options; +using ROCKSDB_NAMESPACE::PinnableSlice; +using ROCKSDB_NAMESPACE::ReadOptions; +using ROCKSDB_NAMESPACE::Status; +using ROCKSDB_NAMESPACE::WriteOptions; +using Submit_queue = Async_future::Submit_queue; + +std::string kDBPath = "/tmp/rocksdb/storage"; + +using Submit_queue = rocksdb::Async_future::Submit_queue; + +struct Async_read { + Async_read(DB *db, size_t io_uring_size); + + ~Async_read() noexcept { + io_uring_queue_exit(m_io_uring.get()); + } + + void io_uring_completion() noexcept; + + Async_future get(ReadOptions &ropts, const std::string &k, std::string &value); + + void set_read_options(ReadOptions &ropts) noexcept { + ropts.verify_checksums = true; + ropts.submit_queue = m_submit_queue; + ropts.read_tier = ReadTier::kPersistedTier; + } + + private: + using Promise = Async_future::promise_type; + + static void schedule_task(Promise* promise) { + auto h{std::coroutine_handle::from_promise(*promise)}; + h.resume(); + } + +private: + using ReadTier = ROCKSDB_NAMESPACE::ReadTier; + using PinnableSlice = ROCKSDB_NAMESPACE::PinnableSlice; + + DB *m_db{}; + std::atomic m_n_pending_sqe{}; + std::unique_ptr m_io_uring{}; + std::shared_ptr m_submit_queue{}; +}; + +Async_read::Async_read(DB* db, size_t io_uring_size) + : m_db(db), + m_io_uring(new io_uring) { + auto ret = io_uring_queue_init(io_uring_size, m_io_uring.get(), 0); + + if (ret < 0) { + throw "io_uring_queue_init failed"; + } + + m_submit_queue = std::make_shared( + [this](Async_future::IO_ctx *ctx, int fd, off_t off, Submit_queue::Ops op) -> Async_future { + using Status = ROCKSDB_NAMESPACE::Status; + using SubCode = Status::SubCode; + using IOStatus = ROCKSDB_NAMESPACE::IOStatus; + + assert(op == Submit_queue::Ops::Read); + + for (auto &iov : ctx->m_iov) { + std::cout << "SUBMIT: " << iov.iov_len << "\n"; + } + + auto io_uring{m_io_uring.get()}; + auto sqe = io_uring_get_sqe(io_uring); + + if (sqe == nullptr) { + co_return IOStatus::IOError(SubCode::kIOUringSqeFull); + } else { + auto &iov{ctx->m_iov}; + + io_uring_prep_readv(sqe, fd, iov.data(), iov.size(), off); + io_uring_sqe_set_data(sqe, ctx); + + const auto ret = io_uring_submit(io_uring); + + if (ret < 0) { + // FIXME: Error handling. + auto msg{strerror(-ret)}; + co_return IOStatus::IOError(SubCode::kIOUringSubmitError, msg); + } else { + m_n_pending_sqe.fetch_add(1, std::memory_order_seq_cst); + + std::cout << "SQE n: " << m_n_pending_sqe << ", " + << " fd: " << fd << ", off: " << off << ", promise: " << ctx->m_promise + << ", ctx: " << ctx << std::endl; + + Async_future r(true, ctx); + co_await r; + co_return IOStatus::OK(); + } + } + }); +} + +void Async_read::io_uring_completion() noexcept { + auto io_uring{m_io_uring.get()}; + + do { + io_uring_cqe* cqe{}; + const auto ret = io_uring_wait_cqe(io_uring, &cqe); + + std::cout << "CQE: " << cqe << "\n"; + + // FIXME: Error handling, short reads etc. + if (ret == 0 && cqe->res >= 0) { + auto ctx = reinterpret_cast(io_uring_cqe_get_data(cqe)); + + io_uring_cqe_seen(io_uring, cqe); + + std::cout << "CQE: " << cqe << ", ret: " << ret << ", res: " << cqe->res + << ", " << m_n_pending_sqe << ", ctx: " << ctx << "\n"; + + auto promise = ctx->m_promise; + + delete ctx; + + if (promise != nullptr) { + schedule_task(promise); + } + } else { + assert(false); + } + + } while (m_n_pending_sqe.fetch_sub(1, std::memory_order_seq_cst) > 1); + + std::cout << "io_uring completion exit\n"; +} + +Async_future Async_read::get(ReadOptions &ropts, const std::string &k, std::string &v) { + auto pinnable = new (std::nothrow) PinnableSlice(); + assert(pinnable != nullptr); + + auto result = m_db->AsyncGet(ropts, m_db->DefaultColumnFamily(), k, pinnable, nullptr); + + std::cout << __FILE__ << ":" << __LINE__ << "\n"; + + co_await result; + + v = pinnable->ToString(); + + delete pinnable; + + std::cout << __FILE__ << ":" << __LINE__ << result.status().code() << "\n"; + + co_return result.status();; +} + +int main() { + DB* db; + + Options options; + + // Optimize RocksDB. This is the easiest way to get RocksDB to perform well + options.IncreaseParallelism(); + options.OptimizeLevelStyleCompaction(); + + options.create_if_missing = true; + + auto s = DB::Open(options, kDBPath, &db); + assert(s.ok()); + + const std::string k1{"k1"}; + const std::string k2{"k2"}; + const std::string k3{"k3"}; + + s = db->Put(WriteOptions(), k1, "v1"); + assert(s.ok()); + + s = db->Put(WriteOptions(), k2, "v2"); + assert(s.ok()); + + delete db; + + s = DB::Open(options, kDBPath, &db); + assert(s.ok()); + + std::string v1{}; + std::string v2{}; + std::string v3{}; + ReadOptions ropts; + Async_read async_read{db, 2}; + + async_read.set_read_options(ropts); + + auto r1 = async_read.get(ropts, k1, v1); + auto r2 = async_read.get(ropts, k2, v2); + auto r3 = async_read.get(ropts, k3, v3); + + async_read.io_uring_completion(); + + std::cout << "found: [" + << v1 << "], [" << v2 << "]" << ", [" << v3 + << "] r3.code: " << (int) r3.status().code() << std::endl; + + delete db; + + return EXIT_SUCCESS;; +} diff --git a/file/file_util.h b/file/file_util.h index 4728e511d9f7..c4e6b70724da 100644 --- a/file/file_util.h +++ b/file/file_util.h @@ -83,6 +83,9 @@ inline IOStatus PrepareIOFromReadOptions(const ReadOptions& ro, (!opts.timeout.count() || ro.io_timeout < opts.timeout)) { opts.timeout = ro.io_timeout; } + + opts.submit_queue = ro.submit_queue; + return IOStatus::OK(); } diff --git a/file/random_access_file_reader.cc b/file/random_access_file_reader.cc index 6857f253ad0b..59c8c39afc70 100644 --- a/file/random_access_file_reader.cc +++ b/file/random_access_file_reader.cc @@ -285,6 +285,159 @@ IOStatus RandomAccessFileReader::Read(const IOOptions& opts, uint64_t offset, return io_s; } +Async_future RandomAccessFileReader::AsyncRead(const IOOptions& opts, + uint64_t offset, size_t n, + Slice* result, char* scratch, + AlignedBuf* aligned_buf, + bool for_compaction) const { + (void)aligned_buf; + + TEST_SYNC_POINT_CALLBACK("RandomAccessFileReader::Read", nullptr); + IOStatus io_s; + uint64_t elapsed = 0; + { + StopWatch sw(clock_, stats_, hist_type_, + (stats_ != nullptr) ? &elapsed : nullptr, true /*overwrite*/, + true /*delay_enabled*/); + auto prev_perf_level = GetPerfLevel(); + IOSTATS_TIMER_GUARD(read_nanos); + if (use_direct_io()) { +#ifndef ROCKSDB_LITE + size_t alignment = file_->GetRequiredBufferAlignment(); + size_t aligned_offset = + TruncateToPageBoundary(alignment, static_cast(offset)); + size_t offset_advance = static_cast(offset) - aligned_offset; + size_t read_size = + Roundup(static_cast(offset + n), alignment) - aligned_offset; + AlignedBuffer buf; + buf.Alignment(alignment); + buf.AllocateNewBuffer(read_size); + while (buf.CurrentSize() < read_size) { + size_t allowed; + if (for_compaction && rate_limiter_ != nullptr) { + allowed = rate_limiter_->RequestToken( + buf.Capacity() - buf.CurrentSize(), buf.Alignment(), + Env::IOPriority::IO_LOW, stats_, RateLimiter::OpType::kRead); + } else { + assert(buf.CurrentSize() == 0); + allowed = read_size; + } + Slice tmp; + + FileOperationInfo::StartTimePoint start_ts; + uint64_t orig_offset = 0; + if (ShouldNotifyListeners()) { + start_ts = FileOperationInfo::StartNow(); + orig_offset = aligned_offset + buf.CurrentSize(); + } + + { + IOSTATS_CPU_TIMER_GUARD(cpu_read_nanos, clock_); + // Only user reads are expected to specify a timeout. And user reads + // are not subjected to rate_limiter and should go through only + // one iteration of this loop, so we don't need to check and adjust + // the opts.timeout before calling file_->Read + assert(!opts.timeout.count() || allowed == read_size); + auto a_result = file_->AsyncRead(aligned_offset + buf.CurrentSize(), allowed, opts, + &tmp, buf.Destination(), nullptr); + co_await a_result; + io_s = a_result.io_result(); + } + if (ShouldNotifyListeners()) { + auto finish_ts = FileOperationInfo::FinishNow(); + NotifyOnFileReadFinish(orig_offset, tmp.size(), start_ts, finish_ts, + io_s); + } + + buf.Size(buf.CurrentSize() + tmp.size()); + if (!io_s.ok() || tmp.size() < allowed) { + break; + } + } + size_t res_len = 0; + if (io_s.ok() && offset_advance < buf.CurrentSize()) { + res_len = std::min(buf.CurrentSize() - offset_advance, n); + if (aligned_buf == nullptr) { + buf.Read(scratch, offset_advance, res_len); + } else { + scratch = buf.BufferStart() + offset_advance; + aligned_buf->reset(buf.Release()); + } + } + *result = Slice(scratch, res_len); +#endif // !ROCKSDB_LITE + } else { + size_t pos = 0; + const char* res_scratch = nullptr; + while (pos < n) { + size_t allowed; + if (for_compaction && rate_limiter_ != nullptr) { + if (rate_limiter_->IsRateLimited(RateLimiter::OpType::kRead)) { + sw.DelayStart(); + } + allowed = rate_limiter_->RequestToken(n - pos, 0 /* alignment */, + Env::IOPriority::IO_LOW, stats_, + RateLimiter::OpType::kRead); + if (rate_limiter_->IsRateLimited(RateLimiter::OpType::kRead)) { + sw.DelayStop(); + } + } else { + allowed = n; + } + Slice tmp_result; + +#ifndef ROCKSDB_LITE + FileOperationInfo::StartTimePoint start_ts; + if (ShouldNotifyListeners()) { + start_ts = FileOperationInfo::StartNow(); + } +#endif + + { + IOSTATS_CPU_TIMER_GUARD(cpu_read_nanos, clock_); + // Only user reads are expected to specify a timeout. And user reads + // are not subjected to rate_limiter and should go through only + // one iteration of this loop, so we don't need to check and adjust + // the opts.timeout before calling file_->Read + assert(!opts.timeout.count() || allowed == n); + auto a_result = file_->AsyncRead(offset + pos, allowed, opts, + &tmp_result, scratch + pos, nullptr); + co_await a_result; + io_s = a_result.io_result(); + } +#ifndef ROCKSDB_LITE + if (ShouldNotifyListeners()) { + auto finish_ts = FileOperationInfo::FinishNow(); + NotifyOnFileReadFinish(offset + pos, tmp_result.size(), start_ts, + finish_ts, io_s); + } +#endif + + if (res_scratch == nullptr) { + // we can't simply use `scratch` because reads of mmap'd files return + // data in a different buffer. + res_scratch = tmp_result.data(); + } else { + // make sure chunks are inserted contiguously into `res_scratch`. + assert(tmp_result.data() == res_scratch + pos); + } + pos += tmp_result.size(); + if (!io_s.ok() || tmp_result.size() < allowed) { + break; + } + } + *result = Slice(res_scratch, io_s.ok() ? pos : 0); + } + IOSTATS_ADD(bytes_read, result->size()); + SetPerfLevel(prev_perf_level); + } + if (stats_ != nullptr && file_read_hist_ != nullptr) { + file_read_hist_->Add(elapsed); + } + + co_return io_s; +} + size_t End(const FSReadRequest& r) { return static_cast(r.offset) + r.len; } diff --git a/file/random_access_file_reader.h b/file/random_access_file_reader.h index 8b4ef4b94306..46117c7b30b8 100644 --- a/file/random_access_file_reader.h +++ b/file/random_access_file_reader.h @@ -14,6 +14,7 @@ #include "env/file_system_tracer.h" #include "port/port.h" +#include "rocksdb/async_future.h" #include "rocksdb/file_system.h" #include "rocksdb/listener.h" #include "rocksdb/options.h" @@ -143,6 +144,10 @@ class RandomAccessFileReader { char* scratch, AlignedBuf* aligned_buf, bool for_compaction = false) const; + Async_future AsyncRead(const IOOptions& opts, uint64_t offset, size_t n, + Slice* result, char* scratch, AlignedBuf* aligned_buf, + bool for_compaction = false) const; + // REQUIRES: // num_reqs > 0, reqs do not overlap, and offsets in reqs are increasing. // In non-direct IO mode, aligned_buf should be null; diff --git a/file/readahead_raf.cc b/file/readahead_raf.cc index 6d346432e226..5a67b12b4a2f 100644 --- a/file/readahead_raf.cc +++ b/file/readahead_raf.cc @@ -74,6 +74,18 @@ class ReadaheadRandomAccessFile : public FSRandomAccessFile { return s; } + Async_future AsyncRead(uint64_t offset, size_t n, const IOOptions& options, + Slice* result, char* scratch, + IODebugContext* dbg) const override { + (void)offset; + (void)n; + (void)options; + (void)result; + (void)scratch; + (void)dbg; + throw "Not implemented"; + } + IOStatus Prefetch(uint64_t offset, size_t n, const IOOptions& options, IODebugContext* dbg) override { if (n < readahead_size_) { diff --git a/file/writable_file_writer.cc b/file/writable_file_writer.cc index 555164b1b0ba..dbf7ebb4088a 100644 --- a/file/writable_file_writer.cc +++ b/file/writable_file_writer.cc @@ -373,6 +373,96 @@ IOStatus WritableFileWriter::Flush() { return s; } +// write out the cached data to the OS cache or storage if direct I/O +// enabled +Async_future WritableFileWriter::AsyncFlush() { + IOStatus s; + TEST_KILL_RANDOM_WITH_WEIGHT("WritableFileWriter::Flush:0", REDUCE_ODDS2); + + if (buf_.CurrentSize() > 0) { + if (use_direct_io()) { +#ifndef ROCKSDB_LITE + if (pending_sync_) { + if (perform_data_verification_ && buffered_data_with_checksum_) { + auto result = AsyncWriteDirectWithChecksum(); + co_await result; + s = result.io_result(); + } else { + auto result = AsyncWriteDirect(); + co_await result; + s = result.io_result(); + } + } +#endif // !ROCKSDB_LITE + } else { + if (perform_data_verification_ && buffered_data_with_checksum_) { + auto result = AsyncWriteBufferedWithChecksum(buf_.BufferStart(), buf_.CurrentSize()); + co_await result; + s = result.io_result(); + } else { + auto result = AsyncWriteBuffered(buf_.BufferStart(), buf_.CurrentSize()); + co_await result; + s = result.io_result(); + } + } + if (!s.ok()) { + co_return s; + } + } + + { +#ifndef ROCKSDB_LITE + FileOperationInfo::StartTimePoint start_ts; + if (ShouldNotifyListeners()) { + start_ts = FileOperationInfo::StartNow(); + } +#endif + s = writable_file_->Flush(IOOptions(), nullptr); +#ifndef ROCKSDB_LITE + if (ShouldNotifyListeners()) { + auto finish_ts = std::chrono::steady_clock::now(); + NotifyOnFileFlushFinish(start_ts, finish_ts, s); + } +#endif + } + + if (!s.ok()) { + co_return s; + } + + // sync OS cache to disk for every bytes_per_sync_ + // TODO: give log file and sst file different options (log + // files could be potentially cached in OS for their whole + // life time, thus we might not want to flush at all). + + // We try to avoid sync to the last 1MB of data. For two reasons: + // (1) avoid rewrite the same page that is modified later. + // (2) for older version of OS, write can block while writing out + // the page. + // Xfs does neighbor page flushing outside of the specified ranges. We + // need to make sure sync range is far from the write offset. + if (!use_direct_io() && bytes_per_sync_) { + const uint64_t kBytesNotSyncRange = + 1024 * 1024; // recent 1MB is not synced. + const uint64_t kBytesAlignWhenSync = 4 * 1024; // Align 4KB. + if (filesize_ > kBytesNotSyncRange) { + uint64_t offset_sync_to = filesize_ - kBytesNotSyncRange; + offset_sync_to -= offset_sync_to % kBytesAlignWhenSync; + assert(offset_sync_to >= last_sync_size_); + if (offset_sync_to > 0 && + offset_sync_to - last_sync_size_ >= bytes_per_sync_) { + auto result = + AsRangeSync(last_sync_size_, offset_sync_to - last_sync_size_); + co_await result; + s = result.io_result(); + last_sync_size_ = offset_sync_to; + } + } + } + + co_return s; +} + std::string WritableFileWriter::GetFileChecksum() { if (checksum_generator_ != nullptr) { assert(checksum_finalized_); @@ -407,6 +497,27 @@ IOStatus WritableFileWriter::Sync(bool use_fsync) { return IOStatus::OK(); } +Async_future WritableFileWriter::AsSync(bool use_fsync) { + auto result = AsyncFlush(); + co_await result; + IOStatus s = result.io_result(); + if (!s.ok()) { + co_return s; + } + TEST_KILL_RANDOM("WritableFileWriter::Sync:0"); + if (!use_direct_io() && pending_sync_) { + auto res = AsSyncInternal(use_fsync); + co_await res; + s = res.io_result(); + if (!s.ok()) { + co_return s; + } + } + TEST_KILL_RANDOM("WritableFileWriter::Sync:1"); + pending_sync_ = false; + co_return IOStatus::OK(); +} + IOStatus WritableFileWriter::SyncWithoutFlush(bool use_fsync) { if (!writable_file_->IsSyncThreadSafe()) { return IOStatus::NotSupported( @@ -419,6 +530,20 @@ IOStatus WritableFileWriter::SyncWithoutFlush(bool use_fsync) { return s; } +Async_future WritableFileWriter::AsSyncWithoutFlush(bool use_fsync) { + if (!writable_file_->IsSyncThreadSafe()) { + co_return IOStatus::NotSupported( + "Can't WritableFileWriter::SyncWithoutFlush() because " + "WritableFile::IsSyncThreadSafe() is false"); + } + TEST_SYNC_POINT("WritableFileWriter::SyncWithoutFlush:1"); + auto result = AsSyncInternal(use_fsync); + co_await result; + IOStatus s = result.io_result(); + TEST_SYNC_POINT("WritableFileWriter::SyncWithoutFlush:2"); + co_return s; +} + IOStatus WritableFileWriter::SyncInternal(bool use_fsync) { IOStatus s; IOSTATS_TIMER_GUARD(fsync_nanos); @@ -453,6 +578,39 @@ IOStatus WritableFileWriter::SyncInternal(bool use_fsync) { return s; } +Async_future WritableFileWriter::AsSyncInternal(bool use_fsync) { + IOStatus s; + IOSTATS_TIMER_GUARD(fsync_nanos); + TEST_SYNC_POINT("WritableFileWriter::SyncInternal:0"); + auto prev_perf_level = GetPerfLevel(); + IOSTATS_CPU_TIMER_GUARD(cpu_write_nanos, clock_); +#ifndef ROCKSDB_LITE + FileOperationInfo::StartTimePoint start_ts; + if (ShouldNotifyListeners()) { + start_ts = FileOperationInfo::StartNow(); + } +#endif + if (use_fsync) { + auto result = writable_file_->AsFsync(IOOptions(), nullptr); + co_await result; + s = result.io_result(); + } else { + auto result = writable_file_->AsSync(IOOptions(), nullptr); + co_await result; + s = result.io_result(); + } +#ifndef ROCKSDB_LITE + if (ShouldNotifyListeners()) { + auto finish_ts = std::chrono::steady_clock::now(); + NotifyOnFileSyncFinish( + start_ts, finish_ts, s, + use_fsync ? FileOperationType::kFsync : FileOperationType::kSync); + } +#endif + SetPerfLevel(prev_perf_level); + co_return s; +} + IOStatus WritableFileWriter::RangeSync(uint64_t offset, uint64_t nbytes) { IOSTATS_TIMER_GUARD(range_sync_nanos); TEST_SYNC_POINT("WritableFileWriter::RangeSync:0"); @@ -476,6 +634,28 @@ IOStatus WritableFileWriter::RangeSync(uint64_t offset, uint64_t nbytes) { return s; } +Async_future WritableFileWriter::AsRangeSync(uint64_t offset, uint64_t nbytes) { + IOSTATS_TIMER_GUARD(range_sync_nanos); + TEST_SYNC_POINT("WritableFileWriter::RangeSync:0"); +#ifndef ROCKSDB_LITE + FileOperationInfo::StartTimePoint start_ts; + if (ShouldNotifyListeners()) { + start_ts = FileOperationInfo::StartNow(); + } +#endif + auto result = + writable_file_->AsRangeSync(offset, nbytes, IOOptions(), nullptr); + co_await result; + IOStatus s = result.io_result(); +#ifndef ROCKSDB_LITE + if (ShouldNotifyListeners()) { + auto finish_ts = std::chrono::steady_clock::now(); + NotifyOnFileRangeSyncFinish(offset, nbytes, start_ts, finish_ts, s); + } +#endif + co_return s; +} + // This method writes to disk the specified data and makes use of the rate // limiter if available IOStatus WritableFileWriter::WriteBuffered(const char* data, size_t size) { @@ -563,6 +743,77 @@ IOStatus WritableFileWriter::WriteBuffered(const char* data, size_t size) { return s; } +Async_future WritableFileWriter::AsyncWriteBuffered(const char* data, + size_t size) { + IOStatus s; + assert(!use_direct_io()); + const char* src = data; + size_t left = size; + DataVerificationInfo v_info; + char checksum_buf[sizeof(uint32_t)]; + + while (left > 0) { + size_t allowed; + if (rate_limiter_ != nullptr) { + allowed = rate_limiter_->RequestToken( + left, 0 /* alignment */, writable_file_->GetIOPriority(), stats_, + RateLimiter::OpType::kWrite); + } else { + allowed = left; + } + + { + IOSTATS_TIMER_GUARD(write_nanos); + TEST_SYNC_POINT("WritableFileWriter::Flush:BeforeAppend"); + +#ifndef ROCKSDB_LITE + FileOperationInfo::StartTimePoint start_ts; + uint64_t old_size = writable_file_->GetFileSize(IOOptions(), nullptr); + if (ShouldNotifyListeners()) { + start_ts = FileOperationInfo::StartNow(); + old_size = next_write_offset_; + } +#endif + { + auto prev_perf_level = GetPerfLevel(); + + IOSTATS_CPU_TIMER_GUARD(cpu_write_nanos, clock_); + if (perform_data_verification_) { + Crc32cHandoffChecksumCalculation(src, allowed, checksum_buf); + v_info.checksum = Slice(checksum_buf, sizeof(uint32_t)); + auto result = writable_file_->AsyncAppend( + Slice(src, allowed), IOOptions(), v_info, nullptr); + co_await result; + s = result.io_result(); + } else { + auto result = writable_file_->AsyncAppend(Slice(src, allowed), IOOptions(), nullptr); + co_await result; + s = result.io_result(); + } + SetPerfLevel(prev_perf_level); + } +#ifndef ROCKSDB_LITE + if (ShouldNotifyListeners()) { + auto finish_ts = std::chrono::steady_clock::now(); + NotifyOnFileWriteFinish(old_size, allowed, start_ts, finish_ts, s); + } +#endif + if (!s.ok()) { + co_return s; + } + } + + IOSTATS_ADD(bytes_written, allowed); + TEST_KILL_RANDOM("WritableFileWriter::WriteBuffered:0"); + + left -= allowed; + src += allowed; + } + buf_.Size(0); + buffered_data_crc32c_checksum_ = 0; + co_return s; +} + IOStatus WritableFileWriter::WriteBufferedWithChecksum(const char* data, size_t size) { IOStatus s; @@ -649,6 +900,77 @@ IOStatus WritableFileWriter::WriteBufferedWithChecksum(const char* data, return s; } +Async_future WritableFileWriter::AsyncWriteBufferedWithChecksum( + const char* data, size_t size) { + IOStatus s; + assert(!use_direct_io()); + assert(perform_data_verification_ && buffered_data_with_checksum_); + const char* src = data; + size_t left = size; + DataVerificationInfo v_info; + char checksum_buf[sizeof(uint32_t)]; + + // Check how much is allowed. Here, we loop until the rate limiter allows to + // write the entire buffer. + // TODO: need to be improved since it sort of defeats the purpose of the rate + // limiter + size_t data_size = left; + if (rate_limiter_ != nullptr) { + while (data_size > 0) { + size_t tmp_size; + tmp_size = rate_limiter_->RequestToken( + data_size, buf_.Alignment(), writable_file_->GetIOPriority(), stats_, + RateLimiter::OpType::kWrite); + data_size -= tmp_size; + } + } + + { + IOSTATS_TIMER_GUARD(write_nanos); + TEST_SYNC_POINT("WritableFileWriter::Flush:BeforeAppend"); + +#ifndef ROCKSDB_LITE + FileOperationInfo::StartTimePoint start_ts; + uint64_t old_size = writable_file_->GetFileSize(IOOptions(), nullptr); + if (ShouldNotifyListeners()) { + start_ts = FileOperationInfo::StartNow(); + old_size = next_write_offset_; + } +#endif + { + auto prev_perf_level = GetPerfLevel(); + + IOSTATS_CPU_TIMER_GUARD(cpu_write_nanos, clock_); + + EncodeFixed32(checksum_buf, buffered_data_crc32c_checksum_); + v_info.checksum = Slice(checksum_buf, sizeof(uint32_t)); + auto result = writable_file_->AsyncAppend(Slice(src, left), IOOptions(), + v_info, nullptr); + co_await result; + s = result.io_result(); + SetPerfLevel(prev_perf_level); + } +#ifndef ROCKSDB_LITE + if (ShouldNotifyListeners()) { + auto finish_ts = std::chrono::steady_clock::now(); + NotifyOnFileWriteFinish(old_size, left, start_ts, finish_ts, s); + } +#endif + if (!s.ok()) { + co_return s; + } + } + + IOSTATS_ADD(bytes_written, left); + TEST_KILL_RANDOM("WritableFileWriter::WriteBuffered:0"); + + // Buffer write is successful, reset the buffer current size to 0 and reset + // the corresponding checksum value + buf_.Size(0); + buffered_data_crc32c_checksum_ = 0; + co_return s; +} + void WritableFileWriter::UpdateFileChecksum(const Slice& data) { if (checksum_generator_ != nullptr) { checksum_generator_->Update(data.data(), data.size()); @@ -767,6 +1089,94 @@ IOStatus WritableFileWriter::WriteDirect() { return s; } +Async_future WritableFileWriter::AsyncWriteDirect() { + assert(use_direct_io()); + IOStatus s; + const size_t alignment = buf_.Alignment(); + assert((next_write_offset_ % alignment) == 0); + + // Calculate whole page final file advance if all writes succeed + size_t file_advance = TruncateToPageBoundary(alignment, buf_.CurrentSize()); + + // Calculate the leftover tail, we write it here padded with zeros BUT we + // will write + // it again in the future either on Close() OR when the current whole page + // fills out + size_t leftover_tail = buf_.CurrentSize() - file_advance; + + // Round up and pad + buf_.PadToAlignmentWith(0); + + const char* src = buf_.BufferStart(); + uint64_t write_offset = next_write_offset_; + size_t left = buf_.CurrentSize(); + DataVerificationInfo v_info; + char checksum_buf[sizeof(uint32_t)]; + + while (left > 0) { + // Check how much is allowed + size_t size; + if (rate_limiter_ != nullptr) { + size = rate_limiter_->RequestToken(left, buf_.Alignment(), + writable_file_->GetIOPriority(), + stats_, RateLimiter::OpType::kWrite); + } else { + size = left; + } + + { + IOSTATS_TIMER_GUARD(write_nanos); + TEST_SYNC_POINT("WritableFileWriter::Flush:BeforeAppend"); + FileOperationInfo::StartTimePoint start_ts; + if (ShouldNotifyListeners()) { + start_ts = FileOperationInfo::StartNow(); + } + // direct writes must be positional + if (perform_data_verification_) { + Crc32cHandoffChecksumCalculation(src, size, checksum_buf); + v_info.checksum = Slice(checksum_buf, sizeof(uint32_t)); + auto result = writable_file_->AsyncPositionedAppend( + Slice(src, size), write_offset, IOOptions(), v_info, nullptr); + co_await result; + s = result.io_result(); + } else { + auto result = writable_file_->AsyncPositionedAppend( + Slice(src, size), write_offset, IOOptions(), nullptr); + co_await result; + s = result.io_result(); + } + + if (ShouldNotifyListeners()) { + auto finish_ts = std::chrono::steady_clock::now(); + NotifyOnFileWriteFinish(write_offset, size, start_ts, finish_ts, s); + } + if (!s.ok()) { + buf_.Size(file_advance + leftover_tail); + co_return s; + } + } + + IOSTATS_ADD(bytes_written, size); + left -= size; + src += size; + write_offset += size; + assert((next_write_offset_ % alignment) == 0); + } + + if (s.ok()) { + // Move the tail to the beginning of the buffer + // This never happens during normal Append but rather during + // explicit call to Flush()/Sync() or Close() + buf_.RefitTail(file_advance, leftover_tail); + // This is where we start writing next time which may or not be + // the actual file size on disk. They match if the buffer size + // is a multiple of whole pages otherwise filesize_ is leftover_tail + // behind + next_write_offset_ += file_advance; + } + co_return s; +} + IOStatus WritableFileWriter::WriteDirectWithChecksum() { assert(use_direct_io()); assert(perform_data_verification_ && buffered_data_with_checksum_); @@ -866,5 +1276,101 @@ IOStatus WritableFileWriter::WriteDirectWithChecksum() { } return s; } + +Async_future WritableFileWriter::AsyncWriteDirectWithChecksum() { + assert(use_direct_io()); + assert(perform_data_verification_ && buffered_data_with_checksum_); + IOStatus s; + const size_t alignment = buf_.Alignment(); + assert((next_write_offset_ % alignment) == 0); + + // Calculate whole page final file advance if all writes succeed + size_t file_advance = TruncateToPageBoundary(alignment, buf_.CurrentSize()); + + // Calculate the leftover tail, we write it here padded with zeros BUT we + // will write + // it again in the future either on Close() OR when the current whole page + // fills out + size_t leftover_tail = buf_.CurrentSize() - file_advance; + + // Round up, pad, and combine the checksum. + size_t last_cur_size = buf_.CurrentSize(); + buf_.PadToAlignmentWith(0); + size_t padded_size = buf_.CurrentSize() - last_cur_size; + const char* padded_start = buf_.BufferStart() + last_cur_size; + uint32_t padded_checksum = crc32c::Value(padded_start, padded_size); + buffered_data_crc32c_checksum_ = crc32c::Crc32cCombine( + buffered_data_crc32c_checksum_, padded_checksum, padded_size); + + const char* src = buf_.BufferStart(); + uint64_t write_offset = next_write_offset_; + size_t left = buf_.CurrentSize(); + DataVerificationInfo v_info; + char checksum_buf[sizeof(uint32_t)]; + + // Check how much is allowed. Here, we loop until the rate limiter allows to + // write the entire buffer. + // TODO: need to be improved since it sort of defeats the purpose of the rate + // limiter + size_t data_size = left; + if (rate_limiter_ != nullptr) { + while (data_size > 0) { + size_t size; + size = rate_limiter_->RequestToken(data_size, buf_.Alignment(), + writable_file_->GetIOPriority(), + stats_, RateLimiter::OpType::kWrite); + data_size -= size; + } + } + + { + IOSTATS_TIMER_GUARD(write_nanos); + TEST_SYNC_POINT("WritableFileWriter::Flush:BeforeAppend"); + FileOperationInfo::StartTimePoint start_ts; + if (ShouldNotifyListeners()) { + start_ts = FileOperationInfo::StartNow(); + } + // direct writes must be positional + EncodeFixed32(checksum_buf, buffered_data_crc32c_checksum_); + v_info.checksum = Slice(checksum_buf, sizeof(uint32_t)); + auto result = writable_file_->AsyncPositionedAppend( + Slice(src, left), write_offset, IOOptions(), v_info, nullptr); + co_await result; + s = result.io_result(); + + if (ShouldNotifyListeners()) { + auto finish_ts = std::chrono::steady_clock::now(); + NotifyOnFileWriteFinish(write_offset, left, start_ts, finish_ts, s); + } + if (!s.ok()) { + // In this case, we do not change buffered_data_crc32c_checksum_ because + // it still aligns with the data in the buffer. + buf_.Size(file_advance + leftover_tail); + buffered_data_crc32c_checksum_ = + crc32c::Value(buf_.BufferStart(), buf_.CurrentSize()); + co_return s; + } + } + + IOSTATS_ADD(bytes_written, left); + assert((next_write_offset_ % alignment) == 0); + + if (s.ok()) { + // Move the tail to the beginning of the buffer + // This never happens during normal Append but rather during + // explicit call to Flush()/Sync() or Close(). Also the buffer checksum will + // recalculated accordingly. + buf_.RefitTail(file_advance, leftover_tail); + // Adjust the checksum value to align with the data in the buffer + buffered_data_crc32c_checksum_ = + crc32c::Value(buf_.BufferStart(), buf_.CurrentSize()); + // This is where we start writing next time which may or not be + // the actual file size on disk. They match if the buffer size + // is a multiple of whole pages otherwise filesize_ is leftover_tail + // behind + next_write_offset_ += file_advance; + } + co_return s; +} #endif // !ROCKSDB_LITE } // namespace ROCKSDB_NAMESPACE diff --git a/file/writable_file_writer.h b/file/writable_file_writer.h index ca25956ce83f..38b9f0a8500b 100644 --- a/file/writable_file_writer.h +++ b/file/writable_file_writer.h @@ -14,6 +14,7 @@ #include "db/version_edit.h" #include "env/file_system_tracer.h" #include "port/port.h" +#include "rocksdb/async_future.h" #include "rocksdb/file_checksum.h" #include "rocksdb/file_system.h" #include "rocksdb/io_status.h" @@ -238,10 +239,14 @@ class WritableFileWriter { IOStatus Flush(); + Async_future AsyncFlush(); + IOStatus Close(); IOStatus Sync(bool use_fsync); + Async_future AsSync(bool use_fsync); + // Sync only the data that was already Flush()ed. Safe to call concurrently // with Append() and Flush(). If !writable_file_->IsSyncThreadSafe(), // returns NotSupported status. @@ -251,6 +256,8 @@ class WritableFileWriter { return filesize_.load(std::memory_order_acquire); } + Async_future AsSyncWithoutFlush(bool use_fsync); + // Returns the size of data flushed to the underlying `FSWritableFile`. // Expected to match `writable_file()->GetFileSize()`. // The return value can serve as a lower-bound for the amount of data synced @@ -283,12 +290,18 @@ class WritableFileWriter { // DMA such as in Direct I/O mode #ifndef ROCKSDB_LITE IOStatus WriteDirect(); + Async_future AsyncWriteDirect(); IOStatus WriteDirectWithChecksum(); + Async_future AsyncWriteDirectWithChecksum(); #endif // !ROCKSDB_LITE // Normal write IOStatus WriteBuffered(const char* data, size_t size); + Async_future AsyncWriteBuffered(const char* data, size_t size); IOStatus WriteBufferedWithChecksum(const char* data, size_t size); + Async_future AsyncWriteBufferedWithChecksum(const char* data, size_t size); IOStatus RangeSync(uint64_t offset, uint64_t nbytes); + Async_future AsRangeSync(uint64_t offset, uint64_t nbytes); IOStatus SyncInternal(bool use_fsync); + Async_future AsSyncInternal(bool use_fsync); }; } // namespace ROCKSDB_NAMESPACE diff --git a/include/rocksdb/async_future.h b/include/rocksdb/async_future.h new file mode 100644 index 000000000000..6f20ea049a7c --- /dev/null +++ b/include/rocksdb/async_future.h @@ -0,0 +1,219 @@ +// Copyright (c) 2011-present, Facebook, Inc. All rights reserved. +// This source code is licensed under both the GPLv2 (found in the +// COPYING file in the root directory) and Apache 2.0 License +// (found in the LICENSE.Apache file in the root directory). + +#pragma once + +#include +#include + +#include +#include +#include +#include +#include +#include + +#include "io_status.h" +#include "rocksdb/status.h" + +struct RustFutureStatus; +struct RustFutureIOStatus; + +namespace ROCKSDB_NAMESPACE { + +struct [[nodiscard]] Async_future { + struct promise_type; + + using Promise_type = promise_type; + using Void_handle_type = std::coroutine_handle<>; + using Typed_handle_type = std::coroutine_handle; + + struct Continuation { + promise_type* m_prev{}; + }; + + struct promise_type { + struct Return_type { + /** Whether the result can be co_returned. */ + // FIXME: Use std::optional + bool m_is_set{}; + + /** Status or statues returned by async functions. */ + std::variant> m_value{}; + }; + + Async_future get_return_object() { + auto h{Typed_handle_type::from_promise(*this)}; + + // FIXME: This is very expensive + assert(m_result == nullptr); + m_result = new(std::nothrow) Return_type{}; + assert(m_result != nullptr); + + return Async_future(h, m_result); + } + + auto initial_suspend() { return std::suspend_never{}; } + + auto final_suspend() noexcept { + if (m_continuation.m_prev != nullptr) { + auto h{Typed_handle_type::from_promise(*m_continuation.m_prev)}; + h.resume(); + } + return std::suspend_never{}; + } + + void unhandled_exception() { + std::abort(); + } + + template + void return_value(T v) { + m_result->m_is_set = true; + m_result->m_value = std::move(v); + } + + Return_type* m_result{}; + Continuation m_continuation{}; + }; + + struct IO_ctx { + explicit IO_ctx(int n_pages) : m_iov(n_pages) {} + + ~IO_ctx() = default; + + int m_fd{}; + off_t m_off{}; + promise_type* m_promise{}; + std::vector m_iov{}; + }; + + /** IOUring callback for Async API. */ + struct Submit_queue { + using FH = int; + using IO_ctx = Async_future::IO_ctx; + + enum class Ops { Read, Write }; + + using FnMut = std::function; + + explicit Submit_queue(io_uring* iouring) : m_iouring{iouring} { + assert(m_iouring != nullptr); + } + + explicit Submit_queue(FnMut&& delegate) + : m_delegate{std::forward(delegate)} {} + + FnMut m_delegate{}; + io_uring* m_iouring{}; + }; + + Async_future() = default; + Async_future& operator=(const Async_future&) = delete; + + Async_future(const Async_future& rhs) + : m_h(rhs.m_h), + m_async(rhs.m_async), + m_ctx(rhs.m_ctx), + m_result() { + assert(rhs.m_result == nullptr); + } + + Async_future(Async_future&& rhs) + : m_h(rhs.m_h), + m_async(rhs.m_async), + m_ctx(rhs.m_ctx), + m_result(rhs.m_result) { + rhs.m_ctx = nullptr; + rhs.m_result = nullptr; + } + + Async_future& operator=(Async_future&& rhs) { + if (this != &rhs) { + m_h = rhs.m_h; + m_ctx = rhs.m_ctx; + m_async = rhs.m_async; + m_result = rhs.m_result; + + rhs.m_ctx = nullptr; + rhs.m_result = nullptr; + } + + return *this; + } + + Async_future(bool async, IO_ctx* ctx) + : m_async(async), + m_ctx(ctx) { } + + Async_future(Typed_handle_type h, Promise_type::Return_type* result) + : m_h(h), + m_result(result) { } + + ~Async_future() { + delete m_result; + m_result = nullptr; + } + + bool await_ready() const noexcept { + if (m_async || m_result == nullptr) { + return false; + } else { + return m_result->m_is_set; + } + } + + void await_suspend(Typed_handle_type h) { + auto promise = &h.promise(); + + if (!m_async) { + m_h.promise().m_continuation.m_prev = promise; + } else if (m_ctx != nullptr) { + m_ctx->m_promise = promise; + } + } + + void await_suspend(Void_handle_type) { + assert(false); + } + + void await_resume() const noexcept {} + + template + auto value() const { + return std::get(m_result->m_value); + } + + Status status() const { + return value(); + } + + IOStatus io_result() const { + return value(); + } + + std::vector statuses() const { + return value>(); + } + + bool write_result() const { + return value(); + } + +private: + Typed_handle_type m_h{}; + + /* true if a custom io_uring handler is installed. */ + bool m_async{}; + + /** IO context for read/write. */ + IO_ctx* m_ctx{}; + + /** Result for the caller. */ + Promise_type::Return_type* m_result{}; +}; + +}// namespace ROCKSDB_NAMESPACE + diff --git a/include/rocksdb/db.h b/include/rocksdb/db.h index 89bdbb89ec44..ddc658759c9f 100644 --- a/include/rocksdb/db.h +++ b/include/rocksdb/db.h @@ -17,6 +17,7 @@ #include #include +#include "rocksdb/async_future.h" #include "rocksdb/iterator.h" #include "rocksdb/listener.h" #include "rocksdb/metadata.h" @@ -370,6 +371,16 @@ class DB { return Put(options, DefaultColumnFamily(), key, ts, value); } + virtual Async_future AsyncPut(const WriteOptions& options, + ColumnFamilyHandle* column_family, + const Slice& key, const Slice& value) { + (void)options; + (void)column_family; + (void)key; + (void)value; + co_return Status::NotSupported("AsyncPut() not implemented."); + } + // Remove the database entry (if any) for "key". Returns OK on // success, and a non-OK status on error. It is not an error if "key" // did not exist in the database. @@ -488,6 +499,13 @@ class DB { return MultiBatchWrite(options, std::move(updates), nullptr); } + virtual Async_future AsyncWrite(const WriteOptions& options, + WriteBatch* updates) { + (void)options; + (void)updates; + co_return Status::NotSupported("AsyncWrite() not implemented."); + } + // If the database contains an entry for "key" store the // corresponding value in *value and return OK. // @@ -518,6 +536,21 @@ class DB { return Get(options, DefaultColumnFamily(), key, value); } + virtual Async_future AsyncGet(const ReadOptions& options, + ColumnFamilyHandle* column_family, + const Slice& key, PinnableSlice* value, + std::string* timestamp) { + assert(options.verify_checksums || column_family != nullptr || + key != nullptr || value != nullptr || timestamp != nullptr); + (void)options; + (void)column_family; + (void)key; + (void)value; + (void)timestamp; + co_return Status::NotSupported( + "AsyncGet() that returns timestamp is not implemented."); + } + // Get() methods that return timestamp. Derived DB classes don't need to worry // about this group of methods if they don't care about timestamp feature. virtual inline Status Get(const ReadOptions& options, @@ -588,6 +621,16 @@ class DB { keys, values); } + virtual Async_future AsyncMultiGet(const ReadOptions& options, + const std::vector& keys, + std::vector* values) { + std::vector cf(keys.size(), DefaultColumnFamily()); + auto r = AsyncMultiGet(options, cf, keys, values, nullptr); + co_await r; + (void)cf; // used after co_await, avoid to destruct + co_return r.statuses(); + } + virtual std::vector MultiGet( const ReadOptions& /*options*/, const std::vector& /*column_family*/, @@ -606,6 +649,16 @@ class DB { std::vector(keys.size(), DefaultColumnFamily()), keys, values, timestamps); } + virtual Async_future AsyncMultiGet( + const ReadOptions& /*options*/, + const std::vector& /*column_family*/, + const std::vector& keys, std::vector* /*values*/, + std::vector* /*timestamps*/) { + co_return std::vector( + keys.size(), + Status::NotSupported( + "AsyncMultiGet() returning timestamps not implemented.")); + } // Overloaded MultiGet API that improves performance by batching operations // in the read path for greater efficiency. Currently, only the block based @@ -1384,12 +1437,18 @@ class DB { virtual Status FlushWAL(bool /*sync*/) { return Status::NotSupported("FlushWAL not implemented"); } + virtual Async_future AsyncFlushWAL(bool /*sync*/) { + co_return Status::NotSupported("FlushWAL not implemented"); + } // Sync the wal. Note that Write() followed by SyncWAL() is not exactly the // same as Write() with sync=true: in the latter case the changes won't be // visible until the sync is done. // Currently only works if allow_mmap_writes = false in Options. virtual Status SyncWAL() = 0; + virtual Async_future AsSyncWAL() { + co_return Status::NotSupported("AsSyncWAL not implemented"); + } // Lock the WAL. Also flushes the WAL after locking. virtual Status LockWAL() { return Status::NotSupported("LockWAL not implemented"); diff --git a/include/rocksdb/env_encryption.h b/include/rocksdb/env_encryption.h index 282db6ed4138..7b5693a72d38 100644 --- a/include/rocksdb/env_encryption.h +++ b/include/rocksdb/env_encryption.h @@ -9,6 +9,7 @@ #include +#include "rocksdb/async_future.h" #include "rocksdb/customizable.h" #include "rocksdb/env.h" #include "rocksdb/file_system.h" @@ -265,6 +266,18 @@ class EncryptedRandomAccessFile : public FSRandomAccessFile { Slice* result, char* scratch, IODebugContext* dbg) const override; + Async_future AsyncRead(uint64_t offset, size_t n, const IOOptions& options, + Slice* result, char* scratch, + IODebugContext* dbg) const override { + (void)offset; + (void)n; + (void)options; + (void)result; + (void)scratch; + (void)dbg; + throw "Not implemented"; + } + // Readahead the file starting from offset by n bytes for caching. IOStatus Prefetch(uint64_t offset, size_t n, const IOOptions& options, IODebugContext* dbg) override; @@ -323,6 +336,22 @@ class EncryptedWritableFile : public FSWritableFile { using FSWritableFile::Append; IOStatus Append(const Slice& data, const IOOptions& options, IODebugContext* dbg) override; + Async_future AsyncAppend(const Slice& data, const IOOptions& options, + IODebugContext* dbg) override { + (void)data; + (void)options; + (void)dbg; + throw "Not implemented"; + }; + + Async_future AsyncAppend(const Slice& data, const IOOptions& options, + const DataVerificationInfo& /* verification_info */, + IODebugContext* dbg) override { + (void)data; + (void)options; + (void)dbg; + throw "Not implemented"; + } using FSWritableFile::PositionedAppend; IOStatus PositionedAppend(const Slice& data, uint64_t offset, diff --git a/include/rocksdb/file_system.h b/include/rocksdb/file_system.h index ec908c4b053a..33dd8376ae2f 100644 --- a/include/rocksdb/file_system.h +++ b/include/rocksdb/file_system.h @@ -28,6 +28,7 @@ #include #include +#include "rocksdb/async_future.h" #include "rocksdb/customizable.h" #include "rocksdb/env.h" #include "rocksdb/io_status.h" @@ -100,17 +101,22 @@ struct IOOptions { // such as NewRandomAccessFile and NewWritableFile. std::unordered_map property_bag; + // Force directory fsync, some file systems like btrfs may skip directory // fsync, set this to force the fsync bool force_dir_fsync; + // + std::shared_ptr submit_queue; + IOOptions() : IOOptions(false) {} explicit IOOptions(bool force_dir_fsync_) : timeout(std::chrono::microseconds::zero()), prio(IOPriority::kIOLow), type(IOType::kUnknown), - force_dir_fsync(force_dir_fsync_) {} + force_dir_fsync(force_dir_fsync_), + submit_queue() {} }; struct DirFsyncOptions { @@ -751,6 +757,10 @@ class FSRandomAccessFile { Slice* result, char* scratch, IODebugContext* dbg) const = 0; + virtual Async_future AsyncRead(uint64_t offset, size_t n, + const IOOptions& options, Slice* result, + char* scratch, IODebugContext* dbg) const = 0; + // Readahead the file starting from offset by n bytes for caching. // If it's not implemented (default: `NotSupported`), RocksDB will create // internal prefetch buffer to improve read performance. @@ -856,6 +866,9 @@ class FSWritableFile { virtual IOStatus Append(const Slice& data, const IOOptions& options, IODebugContext* dbg) = 0; + virtual Async_future AsyncAppend(const Slice& data, const IOOptions& options, + IODebugContext* dbg) = 0; + // Append data with verification information. // Note that this API change is experimental and it might be changed in // the future. Currently, RocksDB only generates crc32c based checksum for @@ -865,11 +878,21 @@ class FSWritableFile { // FSWritableFile, the information in DataVerificationInfo can be ignored // (i.e. does not perform checksum verification). virtual IOStatus Append(const Slice& data, const IOOptions& options, - const DataVerificationInfo& /* verification_info */, + const DataVerificationInfo& verification_info, IODebugContext* dbg) { + (void) verification_info; return Append(data, options, dbg); } + virtual Async_future AsyncAppend( + const Slice& data, const IOOptions& options, + const DataVerificationInfo& verification_info, + IODebugContext* dbg) { + (void) verification_info; + auto result = AsyncAppend(data, options, dbg); + co_await result; + co_return result.io_result(); + } // PositionedAppend data to the specified offset. The new EOF after append // must be larger than the previous EOF. This is to be used when writes are // not backed by OS buffers and hence has to always start from the start of @@ -897,6 +920,13 @@ class FSWritableFile { return IOStatus::NotSupported("PositionedAppend"); } + virtual Async_future AsyncPositionedAppend(const Slice& /* data */, + uint64_t /* offset */, + const IOOptions& /*options*/, + IODebugContext* /*dbg*/) { + co_return IOStatus::NotSupported("PositionedAppend"); + } + // PositionedAppend data with verification information. // Note that this API change is experimental and it might be changed in // the future. Currently, RocksDB only generates crc32c based checksum for @@ -913,6 +943,14 @@ class FSWritableFile { return IOStatus::NotSupported("PositionedAppend"); } + virtual Async_future AsyncPositionedAppend( + const Slice& /* data */, uint64_t /* offset */, + const IOOptions& /*options*/, + const DataVerificationInfo& /* verification_info */, + IODebugContext* /*dbg*/) { + co_return IOStatus::NotSupported("PositionedAppend"); + } + // Truncate is necessary to trim the file to the correct size // before closing. It is not always possible to keep track of the file // size due to whole pages writes. The behavior is undefined if called @@ -925,6 +963,11 @@ class FSWritableFile { virtual IOStatus Flush(const IOOptions& options, IODebugContext* dbg) = 0; virtual IOStatus Sync(const IOOptions& options, IODebugContext* dbg) = 0; // sync data + virtual Async_future AsSync(const IOOptions& options, IODebugContext* dbg) { + (void)options; + (void)dbg; + co_return IOStatus::NotSupported("AsSync"); + } /* * Sync data and/or metadata as well. @@ -936,6 +979,12 @@ class FSWritableFile { return Sync(options, dbg); } + virtual Async_future AsFsync(const IOOptions& options, IODebugContext* dbg) { + auto result = AsSync(options, dbg); + co_await result; + co_return result.io_result(); + } + // true if Sync() and Fsync() are safe to call concurrently with Append() // and Flush(). virtual bool IsSyncThreadSafe() const { return false; } @@ -1008,6 +1057,17 @@ class FSWritableFile { return IOStatus::OK(); } + virtual Async_future AsRangeSync(uint64_t /*offset*/, uint64_t /*nbytes*/, + const IOOptions& options, + IODebugContext* dbg) { + if (strict_bytes_per_sync_) { + auto result = AsSync(options, dbg); + co_await result; + co_return result.io_result(); + } + co_return IOStatus::OK(); + } + // PrepareWrite performs any necessary preparation for a write // before the write actually occurs. This allows for pre-allocation // of space on devices where it can result in less file @@ -1451,6 +1511,13 @@ class FSRandomAccessFileWrapper : public FSRandomAccessFile { IODebugContext* dbg) const override { return target_->Read(offset, n, options, result, scratch, dbg); } + + Async_future AsyncRead(uint64_t offset, size_t n, const IOOptions& options, + Slice* result, char* scratch, + IODebugContext* dbg) const override { + return target_->AsyncRead(offset, n, options, result, scratch, dbg); + } + IOStatus MultiRead(FSReadRequest* reqs, size_t num_reqs, const IOOptions& options, IODebugContext* dbg) override { return target_->MultiRead(reqs, num_reqs, options, dbg); @@ -1505,6 +1572,19 @@ class FSWritableFileWrapper : public FSWritableFile { IODebugContext* dbg) override { return target_->Append(data, options, verification_info, dbg); } + Async_future AsyncAppend(const Slice& data, const IOOptions& options, + IODebugContext* dbg) override { + auto result = target_->AsyncAppend(data, options, dbg); + co_await result; + co_return result.io_result(); + } + Async_future AsyncAppend(const Slice& data, const IOOptions& options, + const DataVerificationInfo& verification_info, + IODebugContext* dbg) override { + auto result = target_->AsyncAppend(data, options, verification_info, dbg); + co_await result; + co_return result.io_result(); + } IOStatus PositionedAppend(const Slice& data, uint64_t offset, const IOOptions& options, IODebugContext* dbg) override { diff --git a/include/rocksdb/io_status.h b/include/rocksdb/io_status.h index ea13d3bc529a..49dc9c865257 100644 --- a/include/rocksdb/io_status.h +++ b/include/rocksdb/io_status.h @@ -104,6 +104,10 @@ class IOStatus : public Status { return IOStatus(kIOError, msg); } + static IOStatus IOError(SubCode sub_code, const Slice& msg) { + return IOStatus(kIOError, sub_code, msg, Slice()); + } + static IOStatus Busy(SubCode msg = kNone) { return IOStatus(kBusy, msg); } static IOStatus Busy(const Slice& msg, const Slice& msg2 = Slice()) { return IOStatus(kBusy, msg, msg2); diff --git a/include/rocksdb/options.h b/include/rocksdb/options.h index 37ea3dd4cc00..fc0dd720f3a5 100644 --- a/include/rocksdb/options.h +++ b/include/rocksdb/options.h @@ -8,6 +8,7 @@ #pragma once +#include #include #include @@ -18,6 +19,7 @@ #include #include "rocksdb/advanced_options.h" +#include "rocksdb/async_future.h" #include "rocksdb/comparator.h" #include "rocksdb/compression_type.h" #include "rocksdb/customizable.h" @@ -1635,6 +1637,9 @@ struct ReadOptions { // Default: false bool adaptive_readahead; + // IOUring callback for async API. + std::shared_ptr submit_queue; + ReadOptions(); ReadOptions(bool cksum, bool cache); }; @@ -1696,13 +1701,17 @@ struct WriteOptions { // Default: false bool memtable_insert_hint_per_batch; + // IOUring callback for async API. + std::shared_ptr submit_queue; + WriteOptions() : sync(false), disableWAL(false), ignore_missing_column_families(false), no_slowdown(false), low_pri(false), - memtable_insert_hint_per_batch(false) {} + memtable_insert_hint_per_batch(false), + submit_queue{} {} }; // Options that control flush operations diff --git a/include/rocksdb/status.h b/include/rocksdb/status.h index 6f6f09c5cc40..719471657d1b 100644 --- a/include/rocksdb/status.h +++ b/include/rocksdb/status.h @@ -114,6 +114,8 @@ class Status { kOverwritten = 12, kTxnNotPrepared = 13, kIOFenced = 14, + kIOUringSqeFull = 15, + kIOUringSubmitError = 16, kMaxSubCode }; diff --git a/table/block_based/block_based_table_reader.cc b/table/block_based/block_based_table_reader.cc index 099064e373b5..7639c812f72f 100644 --- a/table/block_based/block_based_table_reader.cc +++ b/table/block_based/block_based_table_reader.cc @@ -114,6 +114,35 @@ Status ReadBlockFromFile( return s; } +template +Async_future AsyncReadBlockFromFile( + RandomAccessFileReader* file, FilePrefetchBuffer* prefetch_buffer, + const Footer& footer, const ReadOptions& options, const BlockHandle& handle, + std::unique_ptr* result, const ImmutableOptions& ioptions, + bool do_uncompress, bool maybe_compressed, BlockType block_type, + const UncompressionDict& uncompression_dict, + const PersistentCacheOptions& cache_options, size_t read_amp_bytes_per_bit, + MemoryAllocator* memory_allocator, bool for_compaction, bool using_zstd, + const FilterPolicy* filter_policy) { + assert(result); + + BlockContents contents; + BlockFetcher block_fetcher( + file, prefetch_buffer, footer, options, handle, &contents, ioptions, + do_uncompress, maybe_compressed, block_type, uncompression_dict, + cache_options, memory_allocator, nullptr, for_compaction); + auto a_result = block_fetcher.AsyncReadBlockContents(); + co_await a_result; + auto s = a_result.status(); + if (s.ok()) { + result->reset(BlocklikeTraits::Create( + std::move(contents), read_amp_bytes_per_bit, ioptions.stats, using_zstd, + filter_policy)); + } + + co_return s; +} + // Release the cached entry and decrement its ref count. // Do not force erase void ReleaseCachedEntry(void* arg, void* h) { @@ -1603,6 +1632,182 @@ Status BlockBasedTable::MaybeReadBlockAndLoadToCache( return s; } +template +Async_future BlockBasedTable::AsyncMaybeReadBlockAndLoadToCache( + FilePrefetchBuffer* prefetch_buffer, const ReadOptions& ro, + const BlockHandle& handle, const UncompressionDict& uncompression_dict, + const bool wait, const bool for_compaction, + CachableEntry* block_entry, BlockType block_type, + GetContext* get_context, BlockCacheLookupContext* lookup_context, + BlockContents* contents) const { + assert(block_entry != nullptr); + const bool no_io = (ro.read_tier == kBlockCacheTier); + Cache* block_cache = rep_->table_options.block_cache.get(); + Cache* block_cache_compressed = + rep_->table_options.block_cache_compressed.get(); + + // First, try to get the block from the cache + // + // If either block cache is enabled, we'll try to read from it. + Status s; + CacheKey key_data; + Slice key; + bool is_cache_hit = false; + if (block_cache != nullptr || block_cache_compressed != nullptr) { + // create key for block cache + key_data = GetCacheKey(rep_->base_cache_key, handle); + key = key_data.AsSlice(); + + if (!contents) { + s = GetDataBlockFromCache(key, block_cache, block_cache_compressed, ro, + block_entry, uncompression_dict, block_type, + wait, get_context); + // Value could still be null at this point, so check the cache handle + // and update the read pattern for prefetching + if (block_entry->GetValue() || block_entry->GetCacheHandle()) { + // TODO(haoyu): Differentiate cache hit on uncompressed block cache and + // compressed block cache. + is_cache_hit = true; + if (prefetch_buffer) { + // Update the block details so that PrefetchBuffer can use the read + // pattern to determine if reads are sequential or not for + // prefetching. It should also take in account blocks read from cache. + prefetch_buffer->UpdateReadPattern(handle.offset(), + BlockSizeWithTrailer(handle), + ro.adaptive_readahead); + } + } + } + + // Can't find the block from the cache. If I/O is allowed, read from the + // file. + if (block_entry->GetValue() == nullptr && + block_entry->GetCacheHandle() == nullptr && !no_io && ro.fill_cache) { + Statistics* statistics = rep_->ioptions.stats; + const bool maybe_compressed = + block_type != BlockType::kFilter && + block_type != BlockType::kCompressionDictionary && + rep_->blocks_maybe_compressed; + const bool do_uncompress = maybe_compressed && !block_cache_compressed; + CompressionType raw_block_comp_type; + BlockContents raw_block_contents; + if (!contents) { + Histograms histogram = for_compaction ? READ_BLOCK_COMPACTION_MICROS + : READ_BLOCK_GET_MICROS; + StopWatch sw(rep_->ioptions.clock, statistics, histogram); + BlockFetcher block_fetcher( + rep_->file.get(), prefetch_buffer, rep_->footer, ro, handle, + &raw_block_contents, rep_->ioptions, do_uncompress, + maybe_compressed, block_type, uncompression_dict, + rep_->persistent_cache_options, + GetMemoryAllocator(rep_->table_options), + GetMemoryAllocatorForCompressedBlock(rep_->table_options)); + + auto a_result = block_fetcher.AsyncReadBlockContents(); + co_await a_result; + s = a_result.io_result(); + + s = block_fetcher.ReadBlockContents(); + raw_block_comp_type = block_fetcher.get_compression_type(); + contents = &raw_block_contents; + if (get_context) { + switch (block_type) { + case BlockType::kIndex: + ++get_context->get_context_stats_.num_index_read; + break; + case BlockType::kFilter: + ++get_context->get_context_stats_.num_filter_read; + break; + case BlockType::kData: + ++get_context->get_context_stats_.num_data_read; + break; + default: + break; + } + } + } else { + raw_block_comp_type = GetBlockCompressionType(*contents); + } + + if (s.ok()) { + // If filling cache is allowed and a cache is configured, try to put the + // block to the cache. + s = PutDataBlockToCache( + key, block_cache, block_cache_compressed, block_entry, contents, + raw_block_comp_type, uncompression_dict, + GetMemoryAllocator(rep_->table_options), block_type, get_context); + } + } + } + + // Fill lookup_context. + if (block_cache_tracer_ && block_cache_tracer_->is_tracing_enabled() && + lookup_context) { + size_t usage = 0; + uint64_t nkeys = 0; + if (block_entry->GetValue()) { + // Approximate the number of keys in the block using restarts. + nkeys = + rep_->table_options.block_restart_interval * + BlocklikeTraits::GetNumRestarts(*block_entry->GetValue()); + usage = block_entry->GetValue()->ApproximateMemoryUsage(); + } + TraceType trace_block_type = TraceType::kTraceMax; + switch (block_type) { + case BlockType::kData: + trace_block_type = TraceType::kBlockTraceDataBlock; + break; + case BlockType::kFilter: + trace_block_type = TraceType::kBlockTraceFilterBlock; + break; + case BlockType::kCompressionDictionary: + trace_block_type = TraceType::kBlockTraceUncompressionDictBlock; + break; + case BlockType::kRangeDeletion: + trace_block_type = TraceType::kBlockTraceRangeDeletionBlock; + break; + case BlockType::kIndex: + trace_block_type = TraceType::kBlockTraceIndexBlock; + break; + default: + // This cannot happen. + assert(false); + break; + } + bool no_insert = no_io || !ro.fill_cache; + if (BlockCacheTraceHelper::IsGetOrMultiGetOnDataBlock( + trace_block_type, lookup_context->caller)) { + // Defer logging the access to Get() and MultiGet() to trace additional + // information, e.g., referenced_key_exist_in_block. + + // Make a copy of the block key here since it will be logged later. + lookup_context->FillLookupContext( + is_cache_hit, no_insert, trace_block_type, + /*block_size=*/usage, /*block_key=*/key.ToString(), nkeys); + } else { + // Avoid making copy of block_key and cf_name when constructing the access + // record. + BlockCacheTraceRecord access_record( + rep_->ioptions.clock->NowMicros(), + /*block_key=*/"", trace_block_type, + /*block_size=*/usage, rep_->cf_id_for_tracing(), + /*cf_name=*/"", rep_->level_for_tracing(), + rep_->sst_number_for_tracing(), lookup_context->caller, is_cache_hit, + no_insert, lookup_context->get_id, + lookup_context->get_from_user_specified_snapshot, + /*referenced_key=*/""); + // TODO: Should handle this error? + block_cache_tracer_ + ->WriteBlockAccess(access_record, key, rep_->cf_name_for_tracing(), + lookup_context->referenced_key) + .PermitUncheckedError(); + } + } + + assert(s.ok() || block_entry->GetValue() == nullptr); + co_return s; +} + // This function reads multiple data blocks from disk using Env::MultiRead() // and optionally inserts them into the block cache. It uses the scratch // buffer provided by the caller, which is contiguous. If scratch is a nullptr @@ -1978,6 +2183,93 @@ Status BlockBasedTable::RetrieveBlock( return s; } +template +Async_future BlockBasedTable::AsyncRetrieveBlock( + FilePrefetchBuffer* prefetch_buffer, const ReadOptions& ro, + const BlockHandle& handle, const UncompressionDict& uncompression_dict, + CachableEntry* block_entry, BlockType block_type, + GetContext* get_context, BlockCacheLookupContext* lookup_context, + bool for_compaction, bool use_cache, bool wait_for_cache) const { + assert(block_entry); + assert(block_entry->IsEmpty()); + + Status s; + if (use_cache) { + auto a_result = AsyncMaybeReadBlockAndLoadToCache( + prefetch_buffer, ro, handle, uncompression_dict, wait_for_cache, + for_compaction, block_entry, block_type, get_context, lookup_context, + /*contents=*/nullptr); + co_await a_result; + s = a_result.status(); + + if (!s.ok()) { + co_return s; + } + + if (block_entry->GetValue() != nullptr || + block_entry->GetCacheHandle() != nullptr) { + assert(s.ok()); + co_return s; + } + } + + assert(block_entry->IsEmpty()); + + const bool no_io = ro.read_tier == kBlockCacheTier; + if (no_io) { + co_return Status::Incomplete("no blocking io"); + } + + const bool maybe_compressed = + block_type != BlockType::kFilter && + block_type != BlockType::kCompressionDictionary && + rep_->blocks_maybe_compressed; + const bool do_uncompress = maybe_compressed; + std::unique_ptr block; + + { + StopWatch sw(rep_->ioptions.clock, rep_->ioptions.stats, + READ_BLOCK_GET_MICROS); + auto a_result = AsyncReadBlockFromFile( + rep_->file.get(), prefetch_buffer, rep_->footer, ro, handle, &block, + rep_->ioptions, do_uncompress, maybe_compressed, block_type, + uncompression_dict, rep_->persistent_cache_options, + block_type == BlockType::kData + ? rep_->table_options.read_amp_bytes_per_bit + : 0, + GetMemoryAllocator(rep_->table_options), for_compaction, + rep_->blocks_definitely_zstd_compressed, + rep_->table_options.filter_policy.get()); + co_await a_result; + s = a_result.status(); + + if (get_context) { + switch (block_type) { + case BlockType::kIndex: + ++(get_context->get_context_stats_.num_index_read); + break; + case BlockType::kFilter: + ++(get_context->get_context_stats_.num_filter_read); + break; + case BlockType::kData: + ++(get_context->get_context_stats_.num_data_read); + break; + default: + break; + } + } + } + + if (!s.ok()) { + co_return s; + } + + block_entry->SetOwnedValue(block.release()); + + assert(s.ok()); + co_return s; +} + // Explicitly instantiate templates for both "blocklike" types we use. // This makes it possible to keep the template definitions in the .cc file. template Status BlockBasedTable::RetrieveBlock( @@ -2475,6 +2767,196 @@ Status BlockBasedTable::Get(const ReadOptions& read_options, const Slice& key, return s; } +Async_future BlockBasedTable::AsyncGet(const ReadOptions& read_options, + const Slice& key, + GetContext* get_context, + const SliceTransform* prefix_extractor, + bool skip_filters) { + assert(key.size() >= 8); // key must be internal key + assert(get_context != nullptr); + Status s; + const bool no_io = read_options.read_tier == kBlockCacheTier; + + FilterBlockReader* const filter = + !skip_filters ? rep_->filter.get() : nullptr; + + // First check the full filter + // If full filter not useful, Then go into each block + uint64_t tracing_get_id = get_context->get_tracing_get_id(); + BlockCacheLookupContext lookup_context{ + TableReaderCaller::kUserGet, tracing_get_id, + /*get_from_user_specified_snapshot=*/read_options.snapshot != nullptr}; + if (block_cache_tracer_ && block_cache_tracer_->is_tracing_enabled()) { + // Trace the key since it contains both user key and sequence number. + lookup_context.referenced_key = key.ToString(); + lookup_context.get_from_user_specified_snapshot = + read_options.snapshot != nullptr; + } + TEST_SYNC_POINT("BlockBasedTable::Get:BeforeFilterMatch"); + const bool may_match = + FullFilterKeyMayMatch(read_options, filter, key, no_io, prefix_extractor, + get_context, &lookup_context); + TEST_SYNC_POINT("BlockBasedTable::Get:AfterFilterMatch"); + if (!may_match) { + RecordTick(rep_->ioptions.stats, BLOOM_FILTER_USEFUL); + PERF_COUNTER_BY_LEVEL_ADD(bloom_filter_useful, 1, rep_->level); + } else { + IndexBlockIter iiter_on_stack; + // if prefix_extractor found in block differs from options, disable + // BlockPrefixIndex. Only do this check when index_type is kHashSearch. + bool need_upper_bound_check = false; + if (rep_->index_type == BlockBasedTableOptions::kHashSearch) { + need_upper_bound_check = PrefixExtractorChanged(prefix_extractor); + } + auto iiter = + NewIndexIterator(read_options, need_upper_bound_check, &iiter_on_stack, + get_context, &lookup_context); + std::unique_ptr> iiter_unique_ptr; + if (iiter != &iiter_on_stack) { + iiter_unique_ptr.reset(iiter); + } + + size_t ts_sz = + rep_->internal_comparator.user_comparator()->timestamp_size(); + bool matched = false; // if such user key matched a key in SST + bool done = false; + for (iiter->Seek(key); iiter->Valid() && !done; iiter->Next()) { + IndexValue v = iiter->value(); + + bool not_exist_in_filter = + filter != nullptr && filter->IsBlockBased() == true && + !filter->KeyMayMatch(ExtractUserKeyAndStripTimestamp(key, ts_sz), + prefix_extractor, v.handle.offset(), no_io, + /*const_ikey_ptr=*/nullptr, get_context, + &lookup_context); + + if (not_exist_in_filter) { + // Not found + // TODO: think about interaction with Merge. If a user key cannot + // cross one data block, we should be fine. + RecordTick(rep_->ioptions.stats, BLOOM_FILTER_USEFUL); + PERF_COUNTER_BY_LEVEL_ADD(bloom_filter_useful, 1, rep_->level); + break; + } + + if (!v.first_internal_key.empty() && !skip_filters && + UserComparatorWrapper(rep_->internal_comparator.user_comparator()) + .CompareWithoutTimestamp( + ExtractUserKey(key), + ExtractUserKey(v.first_internal_key)) < 0) { + // The requested key falls between highest key in previous block and + // lowest key in current block. + break; + } + + BlockCacheLookupContext lookup_data_block_context{ + TableReaderCaller::kUserGet, tracing_get_id, + /*get_from_user_specified_snapshot=*/read_options.snapshot != + nullptr}; + bool does_referenced_key_exist = false; + DataBlockIter biter; + uint64_t referenced_data_size = 0; + DataBlockIter* result_biter = nullptr; + auto a_result = AsyncNewDataBlockIterator( + read_options, v.handle, &biter, BlockType::kData, get_context, + &lookup_data_block_context, + /*s=*/Status(), &result_biter, /*prefetch_buffer*/ nullptr); + co_await a_result; + + if (no_io && biter.status().IsIncomplete()) { + // couldn't get block from block_cache + // Update Saver.state to Found because we are only looking for + // whether we can guarantee the key is not there when "no_io" is set + get_context->MarkKeyMayExist(); + s = biter.status(); + break; + } + if (!biter.status().ok()) { + s = biter.status(); + break; + } + + bool may_exist = biter.SeekForGet(key); + // If user-specified timestamp is supported, we cannot end the search + // just because hash index lookup indicates the key+ts does not exist. + if (!may_exist && ts_sz == 0) { + // HashSeek cannot find the key this block and the the iter is not + // the end of the block, i.e. cannot be in the following blocks + // either. In this case, the seek_key cannot be found, so we break + // from the top level for-loop. + done = true; + } else { + // Call the *saver function on each entry/block until it returns false + for (; biter.Valid(); biter.Next()) { + ParsedInternalKey parsed_key; + Status pik_status = ParseInternalKey( + biter.key(), &parsed_key, false /* log_err_key */); // TODO + if (!pik_status.ok()) { + s = pik_status; + } + + if (!get_context->SaveValue( + parsed_key, biter.value(), &matched, + biter.IsValuePinned() ? &biter : nullptr)) { + if (get_context->State() == GetContext::GetState::kFound) { + does_referenced_key_exist = true; + referenced_data_size = biter.key().size() + biter.value().size(); + } + done = true; + break; + } + } + s = biter.status(); + } + // Write the block cache access record. + if (block_cache_tracer_ && block_cache_tracer_->is_tracing_enabled()) { + // Avoid making copy of block_key, cf_name, and referenced_key when + // constructing the access record. + Slice referenced_key; + if (does_referenced_key_exist) { + referenced_key = biter.key(); + } else { + referenced_key = key; + } + BlockCacheTraceRecord access_record( + rep_->ioptions.clock->NowMicros(), + /*block_key=*/"", lookup_data_block_context.block_type, + lookup_data_block_context.block_size, rep_->cf_id_for_tracing(), + /*cf_name=*/"", rep_->level_for_tracing(), + rep_->sst_number_for_tracing(), lookup_data_block_context.caller, + lookup_data_block_context.is_cache_hit, + lookup_data_block_context.no_insert, + lookup_data_block_context.get_id, + lookup_data_block_context.get_from_user_specified_snapshot, + /*referenced_key=*/"", referenced_data_size, + lookup_data_block_context.num_keys_in_block, + does_referenced_key_exist); + // TODO: Should handle status here? + block_cache_tracer_ + ->WriteBlockAccess(access_record, + lookup_data_block_context.block_key, + rep_->cf_name_for_tracing(), referenced_key) + .PermitUncheckedError(); + } + + if (done) { + // Avoid the extra Next which is expensive in two-level indexes + break; + } + } + if (matched && filter != nullptr && !filter->IsBlockBased()) { + RecordTick(rep_->ioptions.stats, BLOOM_FILTER_FULL_TRUE_POSITIVE); + PERF_COUNTER_BY_LEVEL_ADD(bloom_filter_full_true_positive, 1, + rep_->level); + } + if (s.ok() && !iiter->status().IsNotFound()) { + s = iiter->status(); + } + } + + co_return s; +} + using MultiGetRange = MultiGetContext::Range; void BlockBasedTable::MultiGet(const ReadOptions& read_options, const MultiGetRange* mget_range, diff --git a/table/block_based/block_based_table_reader.h b/table/block_based/block_based_table_reader.h index 9488b80133d1..aa8b2d6116ba 100644 --- a/table/block_based/block_based_table_reader.h +++ b/table/block_based/block_based_table_reader.h @@ -14,6 +14,7 @@ #include "cache/cache_key.h" #include "db/range_tombstone_fragmenter.h" #include "file/filename.h" +#include "rocksdb/async_future.h" #include "rocksdb/slice_transform.h" #include "rocksdb/table_properties.h" #include "table/block_based/block.h" @@ -136,6 +137,11 @@ class BlockBasedTable : public TableReader { GetContext* get_context, const SliceTransform* prefix_extractor, bool skip_filters = false) override; + Async_future AsyncGet(const ReadOptions& readOptions, const Slice& key, + GetContext* get_context, + const SliceTransform* prefix_extractor, + bool skip_filters = false) override; + void MultiGet(const ReadOptions& readOptions, const MultiGetContext::Range* mget_range, const SliceTransform* prefix_extractor, @@ -274,6 +280,14 @@ class BlockBasedTable : public TableReader { BlockCacheLookupContext* lookup_context, Status s, FilePrefetchBuffer* prefetch_buffer, bool for_compaction = false) const; + template + Async_future AsyncNewDataBlockIterator( + const ReadOptions& ro, const BlockHandle& block_handle, + TBlockIter* input_iter, BlockType block_type, GetContext* get_context, + BlockCacheLookupContext* lookup_context, Status s, + TBlockIter** result_iterator, FilePrefetchBuffer* prefetch_buffer, + bool for_compaction = false) const; + // input_iter: if it is not null, update this one and return it as Iterator template TBlockIter* NewDataBlockIterator(const ReadOptions& ro, @@ -348,6 +362,15 @@ class BlockBasedTable : public TableReader { GetContext* get_context, BlockCacheLookupContext* lookup_context, BlockContents* contents) const; + template + Async_future AsyncMaybeReadBlockAndLoadToCache( + FilePrefetchBuffer* prefetch_buffer, const ReadOptions& ro, + const BlockHandle& handle, const UncompressionDict& uncompression_dict, + const bool wait, const bool for_compaction, + CachableEntry* block_entry, BlockType block_type, + GetContext* get_context, BlockCacheLookupContext* lookup_context, + BlockContents* contents) const; + // Similar to the above, with one crucial difference: it will retrieve the // block from the file even if there are no caches configured (assuming the // read options allow I/O). @@ -361,6 +384,14 @@ class BlockBasedTable : public TableReader { bool for_compaction, bool use_cache, bool wait_for_cache) const; + template + Async_future AsyncRetrieveBlock( + FilePrefetchBuffer* prefetch_buffer, const ReadOptions& ro, + const BlockHandle& handle, const UncompressionDict& uncompression_dict, + CachableEntry* block_entry, BlockType block_type, + GetContext* get_context, BlockCacheLookupContext* lookup_context, + bool for_compaction, bool use_cache, bool wait_for_cache) const; + void RetrieveMultipleBlocks( const ReadOptions& options, const MultiGetRange* batch, const autovector* handles, diff --git a/table/block_based/block_based_table_reader_impl.h b/table/block_based/block_based_table_reader_impl.h index 39930e719210..34be94b8f91e 100644 --- a/table/block_based/block_based_table_reader_impl.h +++ b/table/block_based/block_based_table_reader_impl.h @@ -104,6 +104,92 @@ TBlockIter* BlockBasedTable::NewDataBlockIterator( return iter; } +template +Async_future BlockBasedTable::AsyncNewDataBlockIterator( + const ReadOptions& ro, const BlockHandle& handle, TBlockIter* input_iter, + BlockType block_type, GetContext* get_context, + BlockCacheLookupContext* lookup_context, Status s, + TBlockIter** result_iterator, FilePrefetchBuffer* prefetch_buffer, + bool for_compaction) const { + PERF_TIMER_GUARD(new_table_block_iter_nanos); + + TBlockIter* iter = input_iter != nullptr ? input_iter : new TBlockIter; + *result_iterator = iter; + if (!s.ok()) { + iter->Invalidate(s); + co_return Status::OK(); + } + + CachableEntry uncompression_dict; + if (rep_->uncompression_dict_reader) { + const bool no_io = (ro.read_tier == kBlockCacheTier); + s = rep_->uncompression_dict_reader->GetOrReadUncompressionDictionary( + prefetch_buffer, no_io, get_context, lookup_context, + &uncompression_dict); + if (!s.ok()) { + iter->Invalidate(s); + co_return Status::OK(); + } + } + + const UncompressionDict& dict = uncompression_dict.GetValue() + ? *uncompression_dict.GetValue() + : UncompressionDict::GetEmptyDict(); + + CachableEntry block; + auto result = AsyncRetrieveBlock(prefetch_buffer, ro, handle, dict, &block, block_type, + get_context, lookup_context, for_compaction, + /* use_cache */ true, /* wait_for_cache */ true); + co_await result; + s = result.status(); + + if (!s.ok()) { + assert(block.IsEmpty()); + iter->Invalidate(s); + co_return Status::OK(); + } + + assert(block.GetValue() != nullptr); + + // Block contents are pinned and it is still pinned after the iterator + // is destroyed as long as cleanup functions are moved to another object, + // when: + // 1. block cache handle is set to be released in cleanup function, or + // 2. it's pointing to immortal source. If own_bytes is true then we are + // not reading data from the original source, whether immortal or not. + // Otherwise, the block is pinned iff the source is immortal. + const bool block_contents_pinned = + block.IsCached() || + (!block.GetValue()->own_bytes() && rep_->immortal_table); + iter = InitBlockIterator(rep_, block.GetValue(), block_type, iter, + block_contents_pinned); + + if (!block.IsCached()) { + if (!ro.fill_cache) { + Cache* const block_cache = rep_->table_options.block_cache.get(); + if (block_cache) { + // insert a dummy record to block cache to track the memory usage + Cache::Handle* cache_handle = nullptr; + CacheKey key = CacheKey::CreateUniqueForCacheLifetime(block_cache); + s = block_cache->Insert(key.AsSlice(), nullptr, + block.GetValue()->ApproximateMemoryUsage(), + nullptr, &cache_handle); + + if (s.ok()) { + assert(cache_handle != nullptr); + iter->RegisterCleanup(&ForceReleaseCachedEntry, block_cache, + cache_handle); + } + } + } + } else { + iter->SetCacheHandle(block.GetCacheHandle()); + } + + block.TransferTo(iter); + co_return Status::OK(); +} + // Convert an uncompressed data block (i.e CachableEntry) // into an iterator over the contents of the corresponding block. // If input_iter is null, new a iterator diff --git a/table/block_fetcher.cc b/table/block_fetcher.cc index 54604238cf3f..6bbfc0bd8bea 100644 --- a/table/block_fetcher.cc +++ b/table/block_fetcher.cc @@ -330,4 +330,112 @@ IOStatus BlockFetcher::ReadBlockContents() { return io_status_; } +Async_future BlockFetcher::AsyncReadBlockContents() { + if (TryGetUncompressBlockFromPersistentCache()) { + compression_type_ = kNoCompression; +#ifndef NDEBUG + contents_->is_raw_block = true; +#endif // NDEBUG + co_return Status::OK(); + } + if (TryGetFromPrefetchBuffer()) { + if (!io_status_.ok()) { + co_return io_status_; + } + } else if (!TryGetCompressedBlockFromPersistentCache()) { + IOOptions opts; + io_status_ = file_->PrepareIOOptions(read_options_, opts); + // Actual file read + if (io_status_.ok()) { + if (file_->use_direct_io()) { + PERF_TIMER_GUARD(block_read_time); + auto a_result = + file_->AsyncRead(opts, handle_.offset(), block_size_with_trailer_, + &slice_, nullptr, &direct_io_buf_, for_compaction_); + co_await a_result; + io_status_ = a_result.io_result(); + PERF_COUNTER_ADD(block_read_count, 1); + used_buf_ = const_cast(slice_.data()); + } else { + PrepareBufferForBlockFromFile(); + PERF_TIMER_GUARD(block_read_time); + auto a_result = + file_->AsyncRead(opts, handle_.offset(), block_size_with_trailer_, + &slice_, used_buf_, nullptr, for_compaction_); + co_await a_result; + io_status_ = a_result.io_result(); + PERF_COUNTER_ADD(block_read_count, 1); +#ifndef NDEBUG + if (slice_.data() == &stack_buf_[0]) { + num_stack_buf_memcpy_++; + } else if (slice_.data() == heap_buf_.get()) { + num_heap_buf_memcpy_++; + } else if (slice_.data() == compressed_buf_.get()) { + num_compressed_buf_memcpy_++; + } +#endif + } + } + + // TODO: introduce dedicated perf counter for range tombstones + switch (block_type_) { + case BlockType::kFilter: + PERF_COUNTER_ADD(filter_block_read_count, 1); + break; + + case BlockType::kCompressionDictionary: + PERF_COUNTER_ADD(compression_dict_block_read_count, 1); + break; + + case BlockType::kIndex: + PERF_COUNTER_ADD(index_block_read_count, 1); + break; + + // Nothing to do here as we don't have counters for the other types. + default: + break; + } + + PERF_COUNTER_ADD(block_read_byte, block_size_with_trailer_); + if (!io_status_.ok()) { + co_return io_status_; + } + + if (slice_.size() != block_size_with_trailer_) { + co_return IOStatus::Corruption( + "truncated block read from " + file_->file_name() + " offset " + + ToString(handle_.offset()) + ", expected " + + ToString(block_size_with_trailer_) + " bytes, got " + + ToString(slice_.size())); + } + + ProcessTrailerIfPresent(); + if (io_status_.ok()) { + InsertCompressedBlockToPersistentCacheIfNeeded(); + } else { + co_return io_status_; + } + } + + if (do_uncompress_ && compression_type_ != kNoCompression) { + PERF_TIMER_GUARD(block_decompress_time); + // compressed page, uncompress, update cache + UncompressionContext context(compression_type_); + UncompressionInfo info(context, uncompression_dict_, compression_type_); + io_status_ = status_to_io_status(UncompressBlockContents( + info, slice_.data(), block_size_, contents_, footer_.format_version(), + ioptions_, memory_allocator_)); +#ifndef NDEBUG + num_heap_buf_memcpy_++; +#endif + compression_type_ = kNoCompression; + } else { + GetBlockContents(); + } + + InsertUncompressedBlockToPersistentCacheIfNeeded(); + + co_return io_status_; +} + } // namespace ROCKSDB_NAMESPACE diff --git a/table/block_fetcher.h b/table/block_fetcher.h index 355cb53d0d74..73165574af41 100644 --- a/table/block_fetcher.h +++ b/table/block_fetcher.h @@ -71,6 +71,7 @@ class BlockFetcher { } IOStatus ReadBlockContents(); + Async_future AsyncReadBlockContents(); inline CompressionType get_compression_type() const { return compression_type_; } diff --git a/table/plain/plain_table_builder.cc b/table/plain/plain_table_builder.cc index 2c93516011ec..387d59e2c57b 100644 --- a/table/plain/plain_table_builder.cc +++ b/table/plain/plain_table_builder.cc @@ -277,7 +277,7 @@ Status PlainTableBuilder::Finish() { IOStatus s = WriteBlock(property_block_builder.Finish(), file_, &offset_, &property_block_handle); if (!s.ok()) { - return std::move(s); + return s; } meta_index_builer.Add(kPropertiesBlockName, property_block_handle); diff --git a/table/table_reader.h b/table/table_reader.h index 3631705c4bfc..a447aede74c7 100644 --- a/table/table_reader.h +++ b/table/table_reader.h @@ -110,6 +110,20 @@ class TableReader { const SliceTransform* prefix_extractor, bool skip_filters = false) = 0; + virtual Async_future AsyncGet(const ReadOptions& readOptions, + const Slice& key, GetContext* get_context, + const SliceTransform* prefix_extractor, + bool skip_filters = false) { + assert(readOptions.fill_cache || key.size_ != 0 || get_context != nullptr || + prefix_extractor != nullptr || !skip_filters); + (void)readOptions; + (void)key; + (void)get_context; + (void)prefix_extractor; + (void)skip_filters; + throw "Not implemented"; + } + virtual void MultiGet(const ReadOptions& readOptions, const MultiGetContext::Range* mget_range, const SliceTransform* prefix_extractor, diff --git a/test_util/testutil.cc b/test_util/testutil.cc index e359aa022c95..3114797dfebd 100644 --- a/test_util/testutil.cc +++ b/test_util/testutil.cc @@ -563,7 +563,8 @@ void DeleteDir(Env* env, const std::string& dirname) { Status CreateEnvFromSystem(const ConfigOptions& config_options, Env** result, std::shared_ptr* guard) { const char* env_uri = getenv("TEST_ENV_URI"); - const char* fs_uri = getenv("TEST_FS_URI"); + char* fs_uri = getenv("TEST_FS_URI"); + if (env_uri || fs_uri) { return Env::CreateFromUri(config_options, (env_uri != nullptr) ? env_uri : "", diff --git a/test_util/testutil.h b/test_util/testutil.h index 8d364bc7c62e..817324a546c6 100644 --- a/test_util/testutil.h +++ b/test_util/testutil.h @@ -217,6 +217,19 @@ class StringSink : public FSWritableFile { contents_.append(slice.data(), slice.size()); return IOStatus::OK(); } + Async_future AsyncAppend(const Slice& slice, const IOOptions& /*opts*/, + IODebugContext* /*dbg*/) override { + (void)slice; + throw "Not implemented"; + } + Async_future AsyncAppend(const Slice& data, const IOOptions& options, + const DataVerificationInfo& /* verification_info */, + IODebugContext* dbg) override { + (void)data; + (void)options; + (void)dbg; + throw "Not implemented"; + } void Drop(size_t bytes) { if (reader_contents_ != nullptr) { contents_.resize(contents_.size() - bytes); @@ -317,6 +330,19 @@ class OverwritingStringSink : public FSWritableFile { contents_.append(slice.data(), slice.size()); return IOStatus::OK(); } + Async_future AsyncAppend(const Slice& slice, const IOOptions& /*opts*/, + IODebugContext* /*dbg*/) override { + (void)slice; + throw "Not implemented"; + } + Async_future AsyncAppend(const Slice& data, const IOOptions& options, + const DataVerificationInfo& /* verification_info */, + IODebugContext* dbg) override { + (void)data; + (void)options; + (void)dbg; + throw "Not implemented"; + } void Drop(size_t bytes) { contents_.resize(contents_.size() - bytes); if (last_flush_ > contents_.size()) last_flush_ = contents_.size(); @@ -371,6 +397,18 @@ class StringSource : public FSRandomAccessFile { return IOStatus::OK(); } + Async_future AsyncRead(uint64_t offset, size_t n, const IOOptions& options, + Slice* result, char* scratch, + IODebugContext* dbg) const override { + (void)offset; + (void)n; + (void)options; + (void)result; + (void)scratch; + (void)dbg; + throw "Not implemented"; + } + size_t GetUniqueId(char* id, size_t max_size) const override { if (max_size < 20) { return 0; @@ -590,6 +628,25 @@ class StringFS : public FileSystemWrapper { return IOStatus::OK(); } + virtual Async_future AsyncAppend(const Slice& data, + const IOOptions& options, + IODebugContext* dbg) { + (void)data; + (void)options; + (void)dbg; + throw "not implemented"; + } + + virtual Async_future AsyncAppend( + const Slice& data, const IOOptions& opts, + const DataVerificationInfo& /* verification_info */, + IODebugContext* dbg) override { + (void)data; + (void)opts; + (void)dbg; + throw "not implemented"; + } + private: std::string* contents_; }; diff --git a/third-party/folly/folly/synchronization/DistributedMutex-inl.h b/third-party/folly/folly/synchronization/DistributedMutex-inl.h index 6e250c94d25e..c2cb7e41b42a 100644 --- a/third-party/folly/folly/synchronization/DistributedMutex-inl.h +++ b/third-party/folly/folly/synchronization/DistributedMutex-inl.h @@ -1043,7 +1043,7 @@ inline void recordTimedWaiterAndClearTimedBit( template void wakeTimedWaiters(Atomic* state, bool timedWaiters) { if ((timedWaiters)) { - atomic_notify_one(state); + folly::atomic_notify_one(state); } } diff --git a/third-party/gtest-1.8.1/fused-src/gtest/gtest-all.cc b/third-party/gtest-1.8.1/fused-src/gtest/gtest-all.cc index 9f2b3d565309..b34493115a56 100644 --- a/third-party/gtest-1.8.1/fused-src/gtest/gtest-all.cc +++ b/third-party/gtest-1.8.1/fused-src/gtest/gtest-all.cc @@ -8676,8 +8676,8 @@ static void StackLowerThanAddress(const void* ptr, bool* result) { // Make sure AddressSanitizer does not tamper with the stack here. GTEST_ATTRIBUTE_NO_SANITIZE_ADDRESS_ static bool StackGrowsDown() { - int dummy; - bool result; + int dummy{}; + bool result{}; StackLowerThanAddress(&dummy, &result); return result; } diff --git a/tools/rdb/binding.gyp b/tools/rdb/binding.gyp index 89145541ce58..f1a37a03732d 100644 --- a/tools/rdb/binding.gyp +++ b/tools/rdb/binding.gyp @@ -11,7 +11,7 @@ "-fno-exceptions" ], "cflags_cc+": [ - "-std=c++11", + "-std=c++20", ], "include_dirs+": [ "../../include" diff --git a/util/file_reader_writer_test.cc b/util/file_reader_writer_test.cc index edd71b899461..f12744959623 100644 --- a/util/file_reader_writer_test.cc +++ b/util/file_reader_writer_test.cc @@ -39,6 +39,16 @@ TEST_F(WritableFileWriterTest, RangeSync) { size_ += data.size(); return IOStatus::OK(); } + using FSWritableFile::AsyncAppend; + virtual Async_future AsyncAppend(const Slice& data, + const IOOptions& options, + IODebugContext* dbg) override { + (void)data; + (void)options; + (void)dbg; + throw "Not implemented"; + } + IOStatus Truncate(uint64_t /*size*/, const IOOptions& /*options*/, IODebugContext* /*dbg*/) override { return IOStatus::OK(); @@ -143,6 +153,16 @@ TEST_F(WritableFileWriterTest, IncrementalBuffer) { size_ += data.size(); return IOStatus::OK(); } + using FSWritableFile::AsyncAppend; + virtual Async_future AsyncAppend(const Slice& data, + const IOOptions& options, + IODebugContext* dbg) override { + (void)data; + (void)options; + (void)dbg; + throw "Not implemented"; + } + using FSWritableFile::PositionedAppend; IOStatus PositionedAppend(const Slice& data, uint64_t pos, const IOOptions& /*options*/, @@ -432,6 +452,15 @@ TEST_F(WritableFileWriterTest, AppendStatusReturn) { } return IOStatus::OK(); } + using FSWritableFile::AsyncAppend; + Async_future AsyncAppend(const Slice& data, const IOOptions& options, + IODebugContext* dbg) override { + (void)data; + (void)options; + (void)dbg; + throw "Not implemented"; + } + IOStatus Close(const IOOptions& /*options*/, IODebugContext* /*dbg*/) override { return IOStatus::OK(); @@ -809,6 +838,16 @@ TEST_F(DBWritableFileWriterTest, IOErrorNotification) { return IOStatus::OK(); } + using FSWritableFile::AsyncAppend; + virtual Async_future AsyncAppend(const Slice& data, + const IOOptions& options, + IODebugContext* dbg) override { + (void)data; + (void)options; + (void)dbg; + throw "Not implemented"; + } + using FSWritableFile::PositionedAppend; IOStatus PositionedAppend(const Slice& /*data*/, uint64_t, const IOOptions& /*options*/, diff --git a/utilities/fault_injection_fs.h b/utilities/fault_injection_fs.h index b33964489831..8d19549e0d6a 100644 --- a/utilities/fault_injection_fs.h +++ b/utilities/fault_injection_fs.h @@ -65,6 +65,17 @@ class TestFSWritableFile : public FSWritableFile { virtual ~TestFSWritableFile(); virtual IOStatus Append(const Slice& data, const IOOptions&, IODebugContext*) override; + Async_future AsyncAppend(const Slice& data, const IOOptions&, + IODebugContext*) override { + (void)data; + throw "Not implemented"; + } + Async_future AsyncAppend(const Slice& data, const IOOptions& /*options*/, + const DataVerificationInfo& /*verification_info*/, + IODebugContext* /*dbg*/) override { + (void)data; + throw "Not implemented"; + } virtual IOStatus Append(const Slice& data, const IOOptions& options, const DataVerificationInfo& verification_info, IODebugContext* dbg) override; @@ -138,8 +149,22 @@ class TestFSRandomAccessFile : public FSRandomAccessFile { IOStatus Read(uint64_t offset, size_t n, const IOOptions& options, Slice* result, char* scratch, IODebugContext* dbg) const override; + + Async_future AsyncRead(uint64_t offset, size_t n, const IOOptions& options, + Slice* result, char* scratch, + IODebugContext* dbg) const override { + (void)offset; + (void)n; + (void)options; + (void)result; + (void)scratch; + (void)dbg; + throw "Not implemented"; + } + IOStatus MultiRead(FSReadRequest* reqs, size_t num_reqs, const IOOptions& options, IODebugContext* dbg) override; + size_t GetRequiredBufferAlignment() const override { return target_->GetRequiredBufferAlignment(); }