@ -21,6 +21,7 @@
# include "rocksdb/db.h"
# include "rocksdb/options.h"
# include "rocksdb/utilities/transaction_db.h"
# include "util/cast_util.h"
# include "util/mutexlock.h"
# include "util/string_util.h"
# include "util/sync_point.h"
@ -149,11 +150,22 @@ Status WritePreparedTxnDB::WriteInternal(const WriteOptions& write_options_orig,
const uint64_t no_log_ref = 0 ;
uint64_t seq_used = kMaxSequenceNumber ;
const size_t ZERO_PREPARES = 0 ;
// Since this is not 2pc, there is no need for AddPrepared but having it in
// the PreReleaseCallback enables an optimization. Refer to
// SmallestUnCommittedSeq for more details.
AddPreparedCallback add_prepared_callback (
this , batch_cnt , db_impl_ - > immutable_db_options ( ) . two_write_queues ) ;
WritePreparedCommitEntryPreReleaseCallback update_commit_map (
this , db_impl_ , kMaxSequenceNumber , ZERO_PREPARES , batch_cnt ) ;
auto s = db_impl_ - > WriteImpl (
write_options , batch , nullptr , nullptr , no_log_ref , ! DISABLE_MEMTABLE ,
& seq_used , batch_cnt , do_one_write ? & update_commit_map : nullptr ) ;
PreReleaseCallback * pre_release_callback ;
if ( do_one_write ) {
pre_release_callback = & update_commit_map ;
} else {
pre_release_callback = & add_prepared_callback ;
}
auto s = db_impl_ - > WriteImpl ( write_options , batch , nullptr , nullptr ,
no_log_ref , ! DISABLE_MEMTABLE , & seq_used ,
batch_cnt , pre_release_callback ) ;
assert ( ! s . ok ( ) | | seq_used ! = kMaxSequenceNumber ) ;
uint64_t & prepare_seq = seq_used ;
if ( txn ! = nullptr ) {
@ -170,15 +182,12 @@ Status WritePreparedTxnDB::WriteInternal(const WriteOptions& write_options_orig,
ROCKS_LOG_DETAILS ( db_impl_ - > immutable_db_options ( ) . info_log ,
" CommitBatchInternal 2nd write prepare_seq: % " PRIu64 ,
prepare_seq ) ;
// TODO(myabandeh): What if max advances the prepare_seq_ in the meanwhile and
// readers assume the prepared data as committed? Almost zero probability.
// Commit the batch by writing an empty batch to the 2nd queue that will
// release the commit sequence number to readers.
const size_t ZERO_COMMITS = 0 ;
const bool PREP_HEAP_SKIPPED = true ;
WritePreparedCommitEntryPreReleaseCallback update_commit_map_with_prepare (
this , db_impl_ , prepare_seq , batch_cnt , ZERO_COMMITS , PREP_HEAP_SKIPPED ) ;
this , db_impl_ , prepare_seq , batch_cnt , ZERO_COMMITS , ! PREP_HEAP_SKIPPED ) ;
WriteBatch empty_batch ;
empty_batch . PutLogData ( Slice ( ) ) ;
const size_t ONE_BATCH = 1 ;
@ -197,10 +206,16 @@ Status WritePreparedTxnDB::Get(const ReadOptions& options,
// We are fine with the latest committed value. This could be done by
// specifying the snapshot as kMaxSequenceNumber.
SequenceNumber seq = kMaxSequenceNumber ;
SequenceNumber min_uncommitted = 0 ;
if ( options . snapshot ! = nullptr ) {
seq = options . snapshot - > GetSequenceNumber ( ) ;
min_uncommitted = static_cast_with_check < const SnapshotImpl , const Snapshot > (
options . snapshot )
- > min_uncommitted_ ;
} else {
min_uncommitted = SmallestUnCommittedSeq ( ) ;
}
WritePreparedTxnReadCallback callback ( this , seq ) ;
WritePreparedTxnReadCallback callback ( this , seq , min_uncommitted ) ;
bool * dont_care = nullptr ;
// Note: no need to specify a snapshot for read options as no specific
// snapshot is requested by the user.
@ -252,8 +267,9 @@ std::vector<Status> WritePreparedTxnDB::MultiGet(
// Struct to hold ownership of snapshot and read callback for iterator cleanup.
struct WritePreparedTxnDB : : IteratorState {
IteratorState ( WritePreparedTxnDB * txn_db , SequenceNumber sequence ,
std : : shared_ptr < ManagedSnapshot > s )
: callback ( txn_db , sequence ) , snapshot ( s ) { }
std : : shared_ptr < ManagedSnapshot > s ,
SequenceNumber min_uncommitted )
: callback ( txn_db , sequence , min_uncommitted ) , snapshot ( s ) { }
WritePreparedTxnReadCallback callback ;
std : : shared_ptr < ManagedSnapshot > snapshot ;
@ -271,18 +287,26 @@ Iterator* WritePreparedTxnDB::NewIterator(const ReadOptions& options,
constexpr bool ALLOW_REFRESH = true ;
std : : shared_ptr < ManagedSnapshot > own_snapshot = nullptr ;
SequenceNumber snapshot_seq = kMaxSequenceNumber ;
SequenceNumber min_uncommitted = 0 ;
if ( options . snapshot ! = nullptr ) {
snapshot_seq = options . snapshot - > GetSequenceNumber ( ) ;
min_uncommitted = static_cast_with_check < const SnapshotImpl , const Snapshot > (
options . snapshot )
- > min_uncommitted_ ;
} else {
auto * snapshot = db_impl_ - > GetSnapshot ( ) ;
auto * snapshot = GetSnapshot ( ) ;
// We take a snapshot to make sure that the related data in the commit map
// are not deleted.
snapshot_seq = snapshot - > GetSequenceNumber ( ) ;
min_uncommitted =
static_cast_with_check < const SnapshotImpl , const Snapshot > ( snapshot )
- > min_uncommitted_ ;
own_snapshot = std : : make_shared < ManagedSnapshot > ( db_impl_ , snapshot ) ;
}
assert ( snapshot_seq ! = kMaxSequenceNumber ) ;
auto * cfd = reinterpret_cast < ColumnFamilyHandleImpl * > ( column_family ) - > cfd ( ) ;
auto * state = new IteratorState ( this , snapshot_seq , own_snapshot ) ;
auto * state =
new IteratorState ( this , snapshot_seq , own_snapshot , min_uncommitted ) ;
auto * db_iter =
db_impl_ - > NewIteratorImpl ( options , cfd , snapshot_seq , & state - > callback ,
! ALLOW_BLOB , ! ALLOW_REFRESH ) ;
@ -298,20 +322,28 @@ Status WritePreparedTxnDB::NewIterators(
constexpr bool ALLOW_REFRESH = true ;
std : : shared_ptr < ManagedSnapshot > own_snapshot = nullptr ;
SequenceNumber snapshot_seq = kMaxSequenceNumber ;
SequenceNumber min_uncommitted = 0 ;
if ( options . snapshot ! = nullptr ) {
snapshot_seq = options . snapshot - > GetSequenceNumber ( ) ;
min_uncommitted = static_cast_with_check < const SnapshotImpl , const Snapshot > (
options . snapshot )
- > min_uncommitted_ ;
} else {
auto * snapshot = db_impl_ - > GetSnapshot ( ) ;
auto * snapshot = GetSnapshot ( ) ;
// We take a snapshot to make sure that the related data in the commit map
// are not deleted.
snapshot_seq = snapshot - > GetSequenceNumber ( ) ;
own_snapshot = std : : make_shared < ManagedSnapshot > ( db_impl_ , snapshot ) ;
min_uncommitted =
static_cast_with_check < const SnapshotImpl , const Snapshot > ( snapshot )
- > min_uncommitted_ ;
}
iterators - > clear ( ) ;
iterators - > reserve ( column_families . size ( ) ) ;
for ( auto * column_family : column_families ) {
auto * cfd = reinterpret_cast < ColumnFamilyHandleImpl * > ( column_family ) - > cfd ( ) ;
auto * state = new IteratorState ( this , snapshot_seq , own_snapshot ) ;
auto * state =
new IteratorState ( this , snapshot_seq , own_snapshot , min_uncommitted ) ;
auto * db_iter =
db_impl_ - > NewIteratorImpl ( options , cfd , snapshot_seq , & state - > callback ,
! ALLOW_BLOB , ! ALLOW_REFRESH ) ;
@ -332,118 +364,6 @@ void WritePreparedTxnDB::Init(const TransactionDBOptions& /* unused */) {
new std : : atomic < CommitEntry64b > [ COMMIT_CACHE_SIZE ] { } ) ;
}
// Returns true if commit_seq <= snapshot_seq
bool WritePreparedTxnDB : : IsInSnapshot ( uint64_t prep_seq ,
uint64_t snapshot_seq ) const {
// Here we try to infer the return value without looking into prepare list.
// This would help avoiding synchronization over a shared map.
// TODO(myabandeh): optimize this. This sequence of checks must be correct but
// not necessary efficient
if ( prep_seq = = 0 ) {
// Compaction will output keys to bottom-level with sequence number 0 if
// it is visible to the earliest snapshot.
ROCKS_LOG_DETAILS (
info_log_ , " IsInSnapshot % " PRIu64 " in % " PRIu64 " returns % " PRId32 ,
prep_seq , snapshot_seq , 1 ) ;
return true ;
}
if ( snapshot_seq < prep_seq ) {
// snapshot_seq < prep_seq <= commit_seq => snapshot_seq < commit_seq
ROCKS_LOG_DETAILS (
info_log_ , " IsInSnapshot % " PRIu64 " in % " PRIu64 " returns % " PRId32 ,
prep_seq , snapshot_seq , 0 ) ;
return false ;
}
if ( ! delayed_prepared_empty_ . load ( std : : memory_order_acquire ) ) {
// We should not normally reach here
ReadLock rl ( & prepared_mutex_ ) ;
// TODO(myabandeh): also add a stat
ROCKS_LOG_WARN ( info_log_ , " prepared_mutex_ overhead % " PRIu64 ,
static_cast < uint64_t > ( delayed_prepared_ . size ( ) ) ) ;
if ( delayed_prepared_ . find ( prep_seq ) ! = delayed_prepared_ . end ( ) ) {
// Then it is not committed yet
ROCKS_LOG_DETAILS (
info_log_ , " IsInSnapshot % " PRIu64 " in % " PRIu64 " returns % " PRId32 ,
prep_seq , snapshot_seq , 0 ) ;
return false ;
}
}
auto indexed_seq = prep_seq % COMMIT_CACHE_SIZE ;
CommitEntry64b dont_care ;
CommitEntry cached ;
bool exist = GetCommitEntry ( indexed_seq , & dont_care , & cached ) ;
if ( exist & & prep_seq = = cached . prep_seq ) {
// It is committed and also not evicted from commit cache
ROCKS_LOG_DETAILS (
info_log_ , " IsInSnapshot % " PRIu64 " in % " PRIu64 " returns % " PRId32 ,
prep_seq , snapshot_seq , cached . commit_seq < = snapshot_seq ) ;
return cached . commit_seq < = snapshot_seq ;
}
// else it could be committed but not inserted in the map which could happen
// after recovery, or it could be committed and evicted by another commit, or
// never committed.
// At this point we dont know if it was committed or it is still prepared
auto max_evicted_seq = max_evicted_seq_ . load ( std : : memory_order_acquire ) ;
// max_evicted_seq_ when we did GetCommitEntry <= max_evicted_seq now
if ( max_evicted_seq < prep_seq ) {
// Not evicted from cache and also not present, so must be still prepared
ROCKS_LOG_DETAILS (
info_log_ , " IsInSnapshot % " PRIu64 " in % " PRIu64 " returns % " PRId32 ,
prep_seq , snapshot_seq , 0 ) ;
return false ;
}
// When advancing max_evicted_seq_, we move older entires from prepared to
// delayed_prepared_. Also we move evicted entries from commit cache to
// old_commit_map_ if it overlaps with any snapshot. Since prep_seq <=
// max_evicted_seq_, we have three cases: i) in delayed_prepared_, ii) in
// old_commit_map_, iii) committed with no conflict with any snapshot. Case
// (i) delayed_prepared_ is checked above
if ( max_evicted_seq < snapshot_seq ) { // then (ii) cannot be the case
// only (iii) is the case: committed
// commit_seq <= max_evicted_seq_ < snapshot_seq => commit_seq <
// snapshot_seq
ROCKS_LOG_DETAILS (
info_log_ , " IsInSnapshot % " PRIu64 " in % " PRIu64 " returns % " PRId32 ,
prep_seq , snapshot_seq , 1 ) ;
return true ;
}
// else (ii) might be the case: check the commit data saved for this snapshot.
// If there was no overlapping commit entry, then it is committed with a
// commit_seq lower than any live snapshot, including snapshot_seq.
if ( old_commit_map_empty_ . load ( std : : memory_order_acquire ) ) {
ROCKS_LOG_DETAILS (
info_log_ , " IsInSnapshot % " PRIu64 " in % " PRIu64 " returns % " PRId32 ,
prep_seq , snapshot_seq , 1 ) ;
return true ;
}
{
// We should not normally reach here unless sapshot_seq is old. This is a
// rare case and it is ok to pay the cost of mutex ReadLock for such old,
// reading transactions.
// TODO(myabandeh): also add a stat
ROCKS_LOG_WARN ( info_log_ , " old_commit_map_mutex_ overhead " ) ;
ReadLock rl ( & old_commit_map_mutex_ ) ;
auto prep_set_entry = old_commit_map_ . find ( snapshot_seq ) ;
bool found = prep_set_entry ! = old_commit_map_ . end ( ) ;
if ( found ) {
auto & vec = prep_set_entry - > second ;
found = std : : binary_search ( vec . begin ( ) , vec . end ( ) , prep_seq ) ;
}
if ( ! found ) {
ROCKS_LOG_DETAILS (
info_log_ , " IsInSnapshot % " PRIu64 " in % " PRIu64 " returns % " PRId32 ,
prep_seq , snapshot_seq , 1 ) ;
return true ;
}
}
// (ii) it the case: it is committed but after the snapshot_seq
ROCKS_LOG_DETAILS ( info_log_ ,
" IsInSnapshot % " PRIu64 " in % " PRIu64 " returns % " PRId32 ,
prep_seq , snapshot_seq , 0 ) ;
return false ;
}
void WritePreparedTxnDB : : AddPrepared ( uint64_t seq ) {
ROCKS_LOG_DETAILS ( info_log_ , " Txn % " PRIu64 " Prepareing " , seq ) ;
assert ( seq > max_evicted_seq_ ) ;
@ -622,6 +542,14 @@ void WritePreparedTxnDB::AdvanceMaxEvictedSeq(const SequenceNumber& prev_max,
} ;
}
const Snapshot * WritePreparedTxnDB : : GetSnapshot ( ) {
const bool FOR_WW_CONFLICT_CHECK = true ;
SnapshotImpl * snap_impl = db_impl_ - > GetSnapshotImpl ( ! FOR_WW_CONFLICT_CHECK ) ;
assert ( snap_impl ) ;
EnhanceSnapshot ( snap_impl ) ;
return snap_impl ;
}
const std : : vector < SequenceNumber > WritePreparedTxnDB : : GetSnapshotListFromDB (
SequenceNumber max ) {
ROCKS_LOG_DETAILS ( info_log_ , " GetSnapshotListFromDB with max % " PRIu64 , max ) ;