@ -13,6 +13,7 @@
# include <inttypes.h>
# include <inttypes.h>
# include <map>
# include <map>
# include <set>
# include "db/column_family.h"
# include "db/column_family.h"
# include "db/db_impl.h"
# include "db/db_impl.h"
@ -30,9 +31,7 @@ WritePreparedTxn::WritePreparedTxn(WritePreparedTxnDB* txn_db,
const WriteOptions & write_options ,
const WriteOptions & write_options ,
const TransactionOptions & txn_options )
const TransactionOptions & txn_options )
: PessimisticTransaction ( txn_db , write_options , txn_options ) ,
: PessimisticTransaction ( txn_db , write_options , txn_options ) ,
wpt_db_ ( txn_db ) {
wpt_db_ ( txn_db ) { }
GetWriteBatch ( ) - > DisableDuplicateMergeKeys ( ) ;
}
Status WritePreparedTxn : : Get ( const ReadOptions & read_options ,
Status WritePreparedTxn : : Get ( const ReadOptions & read_options ,
ColumnFamilyHandle * column_family ,
ColumnFamilyHandle * column_family ,
@ -63,6 +62,69 @@ Iterator* WritePreparedTxn::GetIterator(const ReadOptions& options,
return write_batch_ . NewIteratorWithBase ( column_family , db_iter ) ;
return write_batch_ . NewIteratorWithBase ( column_family , db_iter ) ;
}
}
namespace {
// A wrapper around Comparator to make it usable in std::set
struct SetComparator {
explicit SetComparator ( ) : user_comparator_ ( BytewiseComparator ( ) ) { }
explicit SetComparator ( const Comparator * user_comparator )
: user_comparator_ ( user_comparator ? user_comparator
: BytewiseComparator ( ) ) { }
bool operator ( ) ( const Slice & lhs , const Slice & rhs ) const {
return user_comparator_ - > Compare ( lhs , rhs ) < 0 ;
}
private :
const Comparator * user_comparator_ ;
} ;
// Count the number of sub-batches inside a batch. A sub-batch does not have
// duplicate keys.
struct SubBatchCounter : public WriteBatch : : Handler {
explicit SubBatchCounter ( std : : map < uint32_t , const Comparator * > & comparators )
: comparators_ ( comparators ) , batches_ ( 1 ) { }
std : : map < uint32_t , const Comparator * > & comparators_ ;
using CFKeys = std : : set < Slice , SetComparator > ;
std : : map < uint32_t , CFKeys > keys_ ;
size_t batches_ ;
size_t BatchCount ( ) { return batches_ ; }
void AddKey ( uint32_t cf , const Slice & key ) {
CFKeys & cf_keys = keys_ [ cf ] ;
if ( cf_keys . size ( ) = = 0 ) { // just inserted
auto cmp = comparators_ [ cf ] ;
keys_ [ cf ] = CFKeys ( SetComparator ( cmp ) ) ;
}
auto it = cf_keys . insert ( key ) ;
if ( it . second = = false ) { // second is false if a element already existed.
batches_ + + ;
keys_ . clear ( ) ;
keys_ [ cf ] . insert ( key ) ;
}
}
Status MarkNoop ( bool ) override { return Status : : OK ( ) ; }
Status MarkEndPrepare ( const Slice & ) override { return Status : : OK ( ) ; }
Status MarkCommit ( const Slice & ) override { return Status : : OK ( ) ; }
Status PutCF ( uint32_t cf , const Slice & key , const Slice & ) override {
AddKey ( cf , key ) ;
return Status : : OK ( ) ;
}
Status DeleteCF ( uint32_t cf , const Slice & key ) override {
AddKey ( cf , key ) ;
return Status : : OK ( ) ;
}
Status SingleDeleteCF ( uint32_t cf , const Slice & key ) override {
AddKey ( cf , key ) ;
return Status : : OK ( ) ;
}
Status MergeCF ( uint32_t cf , const Slice & key , const Slice & ) override {
AddKey ( cf , key ) ;
return Status : : OK ( ) ;
}
Status MarkBeginPrepare ( ) override { return Status : : OK ( ) ; }
Status MarkRollback ( const Slice & ) override { return Status : : OK ( ) ; }
bool WriteAfterCommit ( ) const override { return false ; }
} ;
} // namespace
Status WritePreparedTxn : : PrepareInternal ( ) {
Status WritePreparedTxn : : PrepareInternal ( ) {
WriteOptions write_options = write_options_ ;
WriteOptions write_options = write_options_ ;
write_options . disableWAL = false ;
write_options . disableWAL = false ;
@ -71,15 +133,18 @@ Status WritePreparedTxn::PrepareInternal() {
! WRITE_AFTER_COMMIT ) ;
! WRITE_AFTER_COMMIT ) ;
const bool DISABLE_MEMTABLE = true ;
const bool DISABLE_MEMTABLE = true ;
uint64_t seq_used = kMaxSequenceNumber ;
uint64_t seq_used = kMaxSequenceNumber ;
bool collapsed = GetWriteBatch ( ) - > Collapse ( ) ;
// For each duplicate key we account for a new sub-batch
if ( collapsed ) {
prepare_batch_cnt_ = 1 ;
ROCKS_LOG_WARN ( db_impl_ - > immutable_db_options ( ) . info_log ,
if ( GetWriteBatch ( ) - > HasDuplicateKeys ( ) ) {
" Collapse overhead due to duplicate keys " ) ;
SubBatchCounter counter ( * wpt_db_ - > GetCFComparatorMap ( ) ) ;
auto s = GetWriteBatch ( ) - > GetWriteBatch ( ) - > Iterate ( & counter ) ;
assert ( s . ok ( ) ) ;
prepare_batch_cnt_ = counter . BatchCount ( ) ;
}
}
Status s =
Status s =
db_impl_ - > WriteImpl ( write_options , GetWriteBatch ( ) - > GetWriteBatch ( ) ,
db_impl_ - > WriteImpl ( write_options , GetWriteBatch ( ) - > GetWriteBatch ( ) ,
/*callback*/ nullptr , & log_number_ , /*log ref*/ 0 ,
/*callback*/ nullptr , & log_number_ , /*log ref*/ 0 ,
! DISABLE_MEMTABLE , & seq_used ) ;
! DISABLE_MEMTABLE , & seq_used , prepare_batch_cnt_ ) ;
assert ( ! s . ok ( ) | | seq_used ! = kMaxSequenceNumber ) ;
assert ( ! s . ok ( ) | | seq_used ! = kMaxSequenceNumber ) ;
auto prepare_seq = seq_used ;
auto prepare_seq = seq_used ;
SetId ( prepare_seq ) ;
SetId ( prepare_seq ) ;
@ -93,18 +158,32 @@ Status WritePreparedTxn::PrepareInternal() {
}
}
Status WritePreparedTxn : : CommitWithoutPrepareInternal ( ) {
Status WritePreparedTxn : : CommitWithoutPrepareInternal ( ) {
bool collapsed = GetWriteBatch ( ) - > Collapse ( ) ;
// For each duplicate key we account for a new sub-batch
if ( collapsed ) {
size_t batch_cnt = 1 ;
ROCKS_LOG_WARN ( db_impl_ - > immutable_db_options ( ) . info_log ,
if ( GetWriteBatch ( ) - > HasDuplicateKeys ( ) ) {
" Collapse overhead due to duplicate keys " ) ;
batch_cnt = 0 ; // this will trigger a batch cnt compute
}
}
return CommitBatchInternal ( GetWriteBatch ( ) - > GetWriteBatch ( ) ) ;
return CommitBatchInternal ( GetWriteBatch ( ) - > GetWriteBatch ( ) , batch_cnt ) ;
}
}
Status WritePreparedTxn : : CommitBatchInternal ( WriteBatch * batch ) {
Status WritePreparedTxn : : CommitBatchInternal ( WriteBatch * batch ,
size_t batch_cnt ) {
ROCKS_LOG_DETAILS ( db_impl_ - > immutable_db_options ( ) . info_log ,
ROCKS_LOG_DETAILS ( db_impl_ - > immutable_db_options ( ) . info_log ,
" CommitBatchInternal " ) ;
" CommitBatchInternal " ) ;
// TODO(myabandeh): handle the duplicate keys in the batch
if ( batch - > Count ( ) = = 0 ) {
// Otherwise our 1 seq per batch logic will break since there is no seq
// increased for this batch.
return Status : : OK ( ) ;
}
if ( batch_cnt = = 0 ) { // not provided, then compute it
// TODO(myabandeh): add an option to allow user skipping this cost
SubBatchCounter counter ( * wpt_db_ - > GetCFComparatorMap ( ) ) ;
auto s = batch - > Iterate ( & counter ) ;
assert ( s . ok ( ) ) ;
batch_cnt = counter . BatchCount ( ) ;
}
assert ( batch_cnt ) ;
bool do_one_write = ! db_impl_ - > immutable_db_options ( ) . two_write_queues ;
bool do_one_write = ! db_impl_ - > immutable_db_options ( ) . two_write_queues ;
bool sync = write_options_ . sync ;
bool sync = write_options_ . sync ;
if ( ! do_one_write ) {
if ( ! do_one_write ) {
@ -116,12 +195,12 @@ Status WritePreparedTxn::CommitBatchInternal(WriteBatch* batch) {
const bool DISABLE_MEMTABLE = true ;
const bool DISABLE_MEMTABLE = true ;
const uint64_t no_log_ref = 0 ;
const uint64_t no_log_ref = 0 ;
uint64_t seq_used = kMaxSequenceNumber ;
uint64_t seq_used = kMaxSequenceNumber ;
const bool INCLUDES_DATA = true ;
const size_t ZERO_PREPARES = 0 ;
WritePreparedCommitEntryPreReleaseCallback update_commit_map (
WritePreparedCommitEntryPreReleaseCallback update_commit_map (
wpt_db_ , db_impl_ , kMaxSequenceNumber , INCLUDES_DATA ) ;
wpt_db_ , db_impl_ , kMaxSequenceNumber , ZERO_PREPARES , batch_cnt ) ;
auto s = db_impl_ - > WriteImpl ( write_options_ , batch , nullptr , nullptr ,
auto s = db_impl_ - > WriteImpl (
no_log_ref , ! DISABLE_MEMTABLE , & seq_used ,
write_options_ , batch , nullptr , nullptr , no_log_ref , ! DISABLE_MEMTABLE ,
do_one_write ? & update_commit_map : nullptr ) ;
& seq_used , batch_cnt , do_one_write ? & update_commit_map : nullptr ) ;
assert ( ! s . ok ( ) | | seq_used ! = kMaxSequenceNumber ) ;
assert ( ! s . ok ( ) | | seq_used ! = kMaxSequenceNumber ) ;
uint64_t & prepare_seq = seq_used ;
uint64_t & prepare_seq = seq_used ;
SetId ( prepare_seq ) ;
SetId ( prepare_seq ) ;
@ -144,13 +223,14 @@ Status WritePreparedTxn::CommitBatchInternal(WriteBatch* batch) {
// Commit the batch by writing an empty batch to the 2nd queue that will
// Commit the batch by writing an empty batch to the 2nd queue that will
// release the commit sequence number to readers.
// release the commit sequence number to readers.
WritePreparedCommitEntryPreReleaseCallback update_commit_map_with_prepare (
WritePreparedCommitEntryPreReleaseCallback update_commit_map_with_prepare (
wpt_db_ , db_impl_ , prepare_seq ) ;
wpt_db_ , db_impl_ , prepare_seq , batch_cnt ) ;
WriteBatch empty_batch ;
WriteBatch empty_batch ;
empty_batch . PutLogData ( Slice ( ) ) ;
empty_batch . PutLogData ( Slice ( ) ) ;
const size_t ONE_BATCH = 1 ;
// In the absence of Prepare markers, use Noop as a batch separator
// In the absence of Prepare markers, use Noop as a batch separator
WriteBatchInternal : : InsertNoop ( & empty_batch ) ;
WriteBatchInternal : : InsertNoop ( & empty_batch ) ;
s = db_impl_ - > WriteImpl ( write_options_ , & empty_batch , nullptr , nullptr ,
s = db_impl_ - > WriteImpl ( write_options_ , & empty_batch , nullptr , nullptr ,
no_log_ref , DISABLE_MEMTABLE , & seq_used ,
no_log_ref , DISABLE_MEMTABLE , & seq_used , ONE_BATCH ,
& update_commit_map_with_prepare ) ;
& update_commit_map_with_prepare ) ;
assert ( ! s . ok ( ) | | seq_used ! = kMaxSequenceNumber ) ;
assert ( ! s . ok ( ) | | seq_used ! = kMaxSequenceNumber ) ;
return s ;
return s ;
@ -175,17 +255,26 @@ Status WritePreparedTxn::CommitInternal() {
auto prepare_seq = GetId ( ) ;
auto prepare_seq = GetId ( ) ;
const bool includes_data = ! empty & & ! for_recovery ;
const bool includes_data = ! empty & & ! for_recovery ;
assert ( prepare_batch_cnt_ ) ;
size_t commit_batch_cnt = 0 ;
if ( includes_data ) {
SubBatchCounter counter ( * wpt_db_ - > GetCFComparatorMap ( ) ) ;
auto s = working_batch - > Iterate ( & counter ) ;
assert ( s . ok ( ) ) ;
commit_batch_cnt = counter . BatchCount ( ) ;
}
WritePreparedCommitEntryPreReleaseCallback update_commit_map (
WritePreparedCommitEntryPreReleaseCallback update_commit_map (
wpt_db_ , db_impl_ , prepare_seq , includes_data ) ;
wpt_db_ , db_impl_ , prepare_seq , prepare_batch_cnt_ , commit_batch_cnt ) ;
const bool disable_memtable = ! includes_data ;
const bool disable_memtable = ! includes_data ;
uint64_t seq_used = kMaxSequenceNumber ;
uint64_t seq_used = kMaxSequenceNumber ;
// Since the prepared batch is directly written to memtable, there is already
// Since the prepared batch is directly written to memtable, there is already
// a connection between the memtable and its WAL, so there is no need to
// a connection between the memtable and its WAL, so there is no need to
// redundantly reference the log that contains the prepared data.
// redundantly reference the log that contains the prepared data.
const uint64_t zero_log_number = 0ull ;
const uint64_t zero_log_number = 0ull ;
size_t batch_cnt = commit_batch_cnt ? commit_batch_cnt : 1 ;
auto s = db_impl_ - > WriteImpl ( write_options_ , working_batch , nullptr , nullptr ,
auto s = db_impl_ - > WriteImpl ( write_options_ , working_batch , nullptr , nullptr ,
zero_log_number , disable_memtable , & seq_used ,
zero_log_number , disable_memtable , & seq_used ,
& update_commit_map ) ;
batch_cnt , & update_commit_map ) ;
assert ( ! s . ok ( ) | | seq_used ! = kMaxSequenceNumber ) ;
assert ( ! s . ok ( ) | | seq_used ! = kMaxSequenceNumber ) ;
return s ;
return s ;
}
}
@ -203,16 +292,36 @@ Status WritePreparedTxn::RollbackInternal() {
ReadOptions roptions ;
ReadOptions roptions ;
WritePreparedTxnReadCallback callback ;
WritePreparedTxnReadCallback callback ;
WriteBatch * rollback_batch_ ;
WriteBatch * rollback_batch_ ;
RollbackWriteBatchBuilder ( DBImpl * db , WritePreparedTxnDB * wpt_db ,
std : : map < uint32_t , const Comparator * > & comparators_ ;
SequenceNumber snap_seq , WriteBatch * dst_batch )
using CFKeys = std : : set < Slice , SetComparator > ;
: db_ ( db ) , callback ( wpt_db , snap_seq ) , rollback_batch_ ( dst_batch ) { }
std : : map < uint32_t , CFKeys > keys_ ;
RollbackWriteBatchBuilder (
DBImpl * db , WritePreparedTxnDB * wpt_db , SequenceNumber snap_seq ,
WriteBatch * dst_batch ,
std : : map < uint32_t , const Comparator * > & comparators )
: db_ ( db ) ,
callback ( wpt_db , snap_seq ) ,
rollback_batch_ ( dst_batch ) ,
comparators_ ( comparators ) { }
Status Rollback ( uint32_t cf , const Slice & key ) {
Status Rollback ( uint32_t cf , const Slice & key ) {
Status s ;
CFKeys & cf_keys = keys_ [ cf ] ;
if ( cf_keys . size ( ) = = 0 ) { // just inserted
auto cmp = comparators_ [ cf ] ;
keys_ [ cf ] = CFKeys ( SetComparator ( cmp ) ) ;
}
auto it = cf_keys . insert ( key ) ;
if ( it . second = =
false ) { // second is false if a element already existed.
return s ;
}
PinnableSlice pinnable_val ;
PinnableSlice pinnable_val ;
bool not_used ;
bool not_used ;
auto cf_handle = db_ - > GetColumnFamilyHandle ( cf ) ;
auto cf_handle = db_ - > GetColumnFamilyHandle ( cf ) ;
auto s = db_ - > GetImpl ( roptions , cf_handle , key , & pinnable_val , & not_used ,
s = db_ - > GetImpl ( roptions , cf_handle , key , & pinnable_val , & not_used ,
& callback ) ;
& callback ) ;
assert ( s . ok ( ) | | s . IsNotFound ( ) ) ;
assert ( s . ok ( ) | | s . IsNotFound ( ) ) ;
if ( s . ok ( ) ) {
if ( s . ok ( ) ) {
s = rollback_batch_ - > Put ( cf_handle , key , pinnable_val ) ;
s = rollback_batch_ - > Put ( cf_handle , key , pinnable_val ) ;
@ -254,7 +363,8 @@ Status WritePreparedTxn::RollbackInternal() {
protected :
protected :
virtual bool WriteAfterCommit ( ) const override { return false ; }
virtual bool WriteAfterCommit ( ) const override { return false ; }
} rollback_handler ( db_impl_ , wpt_db_ , last_visible_txn , & rollback_batch ) ;
} rollback_handler ( db_impl_ , wpt_db_ , last_visible_txn , & rollback_batch ,
* wpt_db_ - > GetCFComparatorMap ( ) ) ;
auto s = GetWriteBatch ( ) - > GetWriteBatch ( ) - > Iterate ( & rollback_handler ) ;
auto s = GetWriteBatch ( ) - > GetWriteBatch ( ) - > Iterate ( & rollback_handler ) ;
assert ( s . ok ( ) ) ;
assert ( s . ok ( ) ) ;
if ( ! s . ok ( ) ) {
if ( ! s . ok ( ) ) {
@ -266,11 +376,12 @@ Status WritePreparedTxn::RollbackInternal() {
const bool DISABLE_MEMTABLE = true ;
const bool DISABLE_MEMTABLE = true ;
const uint64_t no_log_ref = 0 ;
const uint64_t no_log_ref = 0 ;
uint64_t seq_used = kMaxSequenceNumber ;
uint64_t seq_used = kMaxSequenceNumber ;
const bool INCLUDES_DATA = true ;
const size_t ZERO_PREPARES = 0 ;
const size_t ONE_BATCH = 1 ;
WritePreparedCommitEntryPreReleaseCallback update_commit_map (
WritePreparedCommitEntryPreReleaseCallback update_commit_map (
wpt_db_ , db_impl_ , kMaxSequenceNumber , INCLUDES_DATA ) ;
wpt_db_ , db_impl_ , kMaxSequenceNumber , ZERO_PREPARES , ONE_BATCH ) ;
s = db_impl_ - > WriteImpl ( write_options_ , & rollback_batch , nullptr , nullptr ,
s = db_impl_ - > WriteImpl ( write_options_ , & rollback_batch , nullptr , nullptr ,
no_log_ref , ! DISABLE_MEMTABLE , & seq_used ,
no_log_ref , ! DISABLE_MEMTABLE , & seq_used , ONE_BATCH ,
do_one_write ? & update_commit_map : nullptr ) ;
do_one_write ? & update_commit_map : nullptr ) ;
assert ( ! s . ok ( ) | | seq_used ! = kMaxSequenceNumber ) ;
assert ( ! s . ok ( ) | | seq_used ! = kMaxSequenceNumber ) ;
if ( ! s . ok ( ) ) {
if ( ! s . ok ( ) ) {
@ -289,13 +400,13 @@ Status WritePreparedTxn::RollbackInternal() {
// Commit the batch by writing an empty batch to the queue that will release
// Commit the batch by writing an empty batch to the queue that will release
// the commit sequence number to readers.
// the commit sequence number to readers.
WritePreparedCommitEntryPreReleaseCallback update_commit_map_with_prepare (
WritePreparedCommitEntryPreReleaseCallback update_commit_map_with_prepare (
wpt_db_ , db_impl_ , prepare_seq ) ;
wpt_db_ , db_impl_ , prepare_seq , ONE_BATCH ) ;
WriteBatch empty_batch ;
WriteBatch empty_batch ;
empty_batch . PutLogData ( Slice ( ) ) ;
empty_batch . PutLogData ( Slice ( ) ) ;
// In the absence of Prepare markers, use Noop as a batch separator
// In the absence of Prepare markers, use Noop as a batch separator
WriteBatchInternal : : InsertNoop ( & empty_batch ) ;
WriteBatchInternal : : InsertNoop ( & empty_batch ) ;
s = db_impl_ - > WriteImpl ( write_options_ , & empty_batch , nullptr , nullptr ,
s = db_impl_ - > WriteImpl ( write_options_ , & empty_batch , nullptr , nullptr ,
no_log_ref , DISABLE_MEMTABLE , & seq_used ,
no_log_ref , DISABLE_MEMTABLE , & seq_used , ONE_BATCH ,
& update_commit_map_with_prepare ) ;
& update_commit_map_with_prepare ) ;
assert ( ! s . ok ( ) | | seq_used ! = kMaxSequenceNumber ) ;
assert ( ! s . ok ( ) | | seq_used ! = kMaxSequenceNumber ) ;
// Mark the txn as rolled back
// Mark the txn as rolled back
@ -334,6 +445,18 @@ Status WritePreparedTxn::ValidateSnapshot(ColumnFamilyHandle* column_family,
& snap_checker ) ;
& snap_checker ) ;
}
}
Status WritePreparedTxn : : RebuildFromWriteBatch ( WriteBatch * src_batch ) {
auto ret = PessimisticTransaction : : RebuildFromWriteBatch ( src_batch ) ;
prepare_batch_cnt_ = 1 ;
if ( GetWriteBatch ( ) - > HasDuplicateKeys ( ) ) {
SubBatchCounter counter ( * wpt_db_ - > GetCFComparatorMap ( ) ) ;
auto s = GetWriteBatch ( ) - > GetWriteBatch ( ) - > Iterate ( & counter ) ;
assert ( s . ok ( ) ) ;
prepare_batch_cnt_ = counter . BatchCount ( ) ;
}
return ret ;
}
} // namespace rocksdb
} // namespace rocksdb
# endif // ROCKSDB_LITE
# endif // ROCKSDB_LITE