From 9eb3e1f77d287baca3499f8c291be6a2c7c45c6e Mon Sep 17 00:00:00 2001 From: Shylock Hg Date: Wed, 11 Sep 2019 18:07:12 -0700 Subject: [PATCH] Use delete to disable automatic generated methods. (#5009) Summary: Use delete to disable automatic generated methods instead of private, and put the constructor together for more clear.This modification cause the unused field warning, so add unused attribute to disable this warning. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5009 Differential Revision: D17288733 fbshipit-source-id: 8a767ce096f185f1db01bd28fc88fef1cdd921f3 --- db/compacted_db_impl.h | 8 ++--- db/db_impl/db_impl.h | 8 ++--- db/db_impl/db_impl_readonly.h | 8 ++--- db/db_iter.cc | 14 +++++--- db/dbformat.h | 7 ++-- db/log_reader.h | 8 ++--- db/log_writer.h | 8 ++--- db/memtable.cc | 7 ++-- db/memtable.h | 7 ++-- db/version_set.h | 18 +++++----- env/mock_env.cc | 7 ++-- include/rocksdb/cache.h | 7 ++-- include/rocksdb/cleanable.h | 4 +-- include/rocksdb/db.h | 9 +++-- include/rocksdb/env.h | 33 +++++++++---------- include/rocksdb/iterator.h | 9 +++-- include/rocksdb/utilities/transaction.h | 7 ++-- include/rocksdb/utilities/transaction_db.h | 6 ++-- include/rocksdb/write_buffer_manager.h | 8 ++--- memory/allocator.h | 8 ++--- memtable/inlineskiplist.h | 7 ++-- memtable/skiplist.h | 7 ++-- monitoring/in_memory_stats_history.h | 14 ++++---- port/port_posix.h | 16 ++++----- port/win/port_win.h | 6 ++-- table/block_based/block.h | 7 ++-- table/block_based/block_based_filter_block.h | 10 +++--- table/block_based/block_based_table_builder.h | 6 ++-- table/block_based/block_based_table_reader.h | 7 ++-- table/block_based/filter_block.h | 9 +++-- table/block_based/full_filter_block.h | 8 ++--- table/cuckoo/cuckoo_table_builder.h | 7 ++-- table/cuckoo/cuckoo_table_reader.cc | 6 ++-- table/full_filter_bits_builder.h | 8 ++--- table/internal_iterator.h | 10 +++--- table/plain/plain_table_builder.h | 7 ++-- table/plain/plain_table_reader.cc | 7 ++-- util/bloom.cc | 7 ++-- util/concurrent_task_limiter_impl.h | 9 +++-- util/mutexlock.h | 28 +++++++++------- utilities/blob_db/blob_log_reader.h | 5 ++- utilities/blob_db/blob_log_writer.h | 5 ++- .../transactions/optimistic_transaction.h | 9 +++-- .../transactions/pessimistic_transaction.h | 14 ++++---- utilities/transactions/transaction_lock_mgr.h | 7 ++-- utilities/transactions/write_prepared_txn.h | 7 ++-- 46 files changed, 205 insertions(+), 219 deletions(-) diff --git a/db/compacted_db_impl.h b/db/compacted_db_impl.h index e71ce2494..8a57c5b77 100644 --- a/db/compacted_db_impl.h +++ b/db/compacted_db_impl.h @@ -14,6 +14,10 @@ namespace rocksdb { class CompactedDBImpl : public DBImpl { public: CompactedDBImpl(const DBOptions& options, const std::string& dbname); + // No copying allowed + CompactedDBImpl(const CompactedDBImpl&) = delete; + void operator=(const CompactedDBImpl&) = delete; + virtual ~CompactedDBImpl(); static Status Open(const Options& options, const std::string& dbname, @@ -104,10 +108,6 @@ class CompactedDBImpl : public DBImpl { Version* version_; const Comparator* user_comparator_; LevelFilesBrief files_; - - // No copying allowed - CompactedDBImpl(const CompactedDBImpl&); - void operator=(const CompactedDBImpl&); }; } #endif // ROCKSDB_LITE diff --git a/db/db_impl/db_impl.h b/db/db_impl/db_impl.h index 1942f0979..3e45442ca 100644 --- a/db/db_impl/db_impl.h +++ b/db/db_impl/db_impl.h @@ -128,6 +128,10 @@ class DBImpl : public DB { public: DBImpl(const DBOptions& options, const std::string& dbname, const bool seq_per_batch = false, const bool batch_per_txn = true); + // No copying allowed + DBImpl(const DBImpl&) = delete; + void operator=(const DBImpl&) = delete; + virtual ~DBImpl(); // ---- Implementations of the DB interface ---- @@ -1563,10 +1567,6 @@ class DBImpl : public DB { void WaitForBackgroundWork(); - // No copying allowed - DBImpl(const DBImpl&); - void operator=(const DBImpl&); - // Background threads call this function, which is just a wrapper around // the InstallSuperVersion() function. Background threads carry // sv_context which can have new_superversion already diff --git a/db/db_impl/db_impl_readonly.h b/db/db_impl/db_impl_readonly.h index ad307677c..9f7ad17a4 100644 --- a/db/db_impl/db_impl_readonly.h +++ b/db/db_impl/db_impl_readonly.h @@ -16,6 +16,10 @@ namespace rocksdb { class DBImplReadOnly : public DBImpl { public: DBImplReadOnly(const DBOptions& options, const std::string& dbname); + // No copying allowed + DBImplReadOnly(const DBImplReadOnly&) = delete; + void operator=(const DBImplReadOnly&) = delete; + virtual ~DBImplReadOnly(); // Implementations of the DB interface @@ -127,10 +131,6 @@ class DBImplReadOnly : public DBImpl { private: friend class DB; - - // No copying allowed - DBImplReadOnly(const DBImplReadOnly&); - void operator=(const DBImplReadOnly&); }; } // namespace rocksdb diff --git a/db/db_iter.cc b/db/db_iter.cc index 060138fd6..961431611 100644 --- a/db/db_iter.cc +++ b/db/db_iter.cc @@ -155,6 +155,10 @@ class DBIter final: public Iterator { iter_.iter()->SetPinnedItersMgr(&pinned_iters_mgr_); } } + // No copying allowed + DBIter(const DBIter&) = delete; + void operator=(const DBIter&) = delete; + ~DBIter() override { // Release pinned data if any if (pinned_iters_mgr_.PinningEnabled()) { @@ -345,15 +349,17 @@ class DBIter final: public Iterator { ReadRangeDelAggregator range_del_agg_; LocalStatistics local_stats_; PinnedIteratorsManager pinned_iters_mgr_; +#ifdef ROCKSDB_LITE + ROCKSDB_FIELD_UNUSED +#endif DBImpl* db_impl_; +#ifdef ROCKSDB_LITE + ROCKSDB_FIELD_UNUSED +#endif ColumnFamilyData* cfd_; // for diff snapshots we want the lower bound on the seqnum; // if this value > 0 iterator will return internal keys SequenceNumber start_seqnum_; - - // No copying allowed - DBIter(const DBIter&); - void operator=(const DBIter&); }; inline bool DBIter::ParseKey(ParsedInternalKey* ikey) { diff --git a/db/dbformat.h b/db/dbformat.h index 1d9b7ef7e..090d8c133 100644 --- a/db/dbformat.h +++ b/db/dbformat.h @@ -326,6 +326,9 @@ class IterKey { key_size_(0), buf_size_(sizeof(space_)), is_user_key_(true) {} + // No copying allowed + IterKey(const IterKey&) = delete; + void operator=(const IterKey&) = delete; ~IterKey() { ResetBuffer(); } @@ -523,10 +526,6 @@ class IterKey { } void EnlargeBuffer(size_t key_size); - - // No copying allowed - IterKey(const IterKey&) = delete; - void operator=(const IterKey&) = delete; }; // Convert from a SliceTranform of user keys, to a SliceTransform of diff --git a/db/log_reader.h b/db/log_reader.h index bda9ac8bb..efeb270e2 100644 --- a/db/log_reader.h +++ b/db/log_reader.h @@ -53,6 +53,9 @@ class Reader { // @lint-ignore TXT2 T25377293 Grandfathered in std::unique_ptr&& file, Reporter* reporter, bool checksum, uint64_t log_num); + // No copying allowed + Reader(const Reader&) = delete; + void operator=(const Reader&) = delete; virtual ~Reader(); @@ -148,11 +151,6 @@ class Reader { // buffer_ must be updated to remove the dropped bytes prior to invocation. void ReportCorruption(size_t bytes, const char* reason); void ReportDrop(size_t bytes, const Status& reason); - - private: - // No copying allowed - Reader(const Reader&); - void operator=(const Reader&); }; class FragmentBufferedReader : public Reader { diff --git a/db/log_writer.h b/db/log_writer.h index 116d03358..e5ed71a76 100644 --- a/db/log_writer.h +++ b/db/log_writer.h @@ -73,6 +73,10 @@ class Writer { explicit Writer(std::unique_ptr&& dest, uint64_t log_number, bool recycle_log_files, bool manual_flush = false); + // No copying allowed + Writer(const Writer&) = delete; + void operator=(const Writer&) = delete; + ~Writer(); Status AddRecord(const Slice& slice); @@ -104,10 +108,6 @@ class Writer { // If true, it does not flush after each write. Instead it relies on the upper // layer to manually does the flush by calling ::WriteBuffer() bool manual_flush_; - - // No copying allowed - Writer(const Writer&); - void operator=(const Writer&); }; } // namespace log diff --git a/db/memtable.cc b/db/memtable.cc index 21d3e347b..33036ad98 100644 --- a/db/memtable.cc +++ b/db/memtable.cc @@ -295,6 +295,9 @@ class MemTableIterator : public InternalIterator { iter_ = mem.table_->GetIterator(arena); } } + // No copying allowed + MemTableIterator(const MemTableIterator&) = delete; + void operator=(const MemTableIterator&) = delete; ~MemTableIterator() override { #ifndef NDEBUG @@ -408,10 +411,6 @@ class MemTableIterator : public InternalIterator { bool valid_; bool arena_mode_; bool value_pinned_; - - // No copying allowed - MemTableIterator(const MemTableIterator&); - void operator=(const MemTableIterator&); }; InternalIterator* MemTable::NewIterator(const ReadOptions& read_options, diff --git a/db/memtable.h b/db/memtable.h index c0baa9e17..ed837e945 100644 --- a/db/memtable.h +++ b/db/memtable.h @@ -101,6 +101,9 @@ class MemTable { const MutableCFOptions& mutable_cf_options, WriteBufferManager* write_buffer_manager, SequenceNumber earliest_seq, uint32_t column_family_id); + // No copying allowed + MemTable(const MemTable&) = delete; + MemTable& operator=(const MemTable&) = delete; // Do not delete this MemTable unless Unref() indicates it not in use. ~MemTable(); @@ -503,10 +506,6 @@ class MemTable { void UpdateFlushState(); void UpdateOldestKeyTime(); - - // No copying allowed - MemTable(const MemTable&); - MemTable& operator=(const MemTable&); }; extern const char* EncodeKey(std::string* scratch, const Slice& target); diff --git a/db/version_set.h b/db/version_set.h index c5b391a15..24919a602 100644 --- a/db/version_set.h +++ b/db/version_set.h @@ -109,6 +109,9 @@ class VersionStorageInfo { CompactionStyle compaction_style, VersionStorageInfo* src_vstorage, bool _force_consistency_checks); + // No copying allowed + VersionStorageInfo(const VersionStorageInfo&) = delete; + void operator=(const VersionStorageInfo&) = delete; ~VersionStorageInfo(); void Reserve(int level, size_t size) { files_[level].reserve(size); } @@ -542,9 +545,6 @@ class VersionStorageInfo { friend class Version; friend class VersionSet; - // No copying allowed - VersionStorageInfo(const VersionStorageInfo&) = delete; - void operator=(const VersionStorageInfo&) = delete; }; using MultiGetRange = MultiGetContext::Range; @@ -734,8 +734,8 @@ class Version { ~Version(); // No copying allowed - Version(const Version&); - void operator=(const Version&); + Version(const Version&) = delete; + void operator=(const Version&) = delete; }; struct ObsoleteFileInfo { @@ -797,6 +797,10 @@ class VersionSet { WriteBufferManager* write_buffer_manager, WriteController* write_controller, BlockCacheTracer* const block_cache_tracer); + // No copying allowed + VersionSet(const VersionSet&) = delete; + void operator=(const VersionSet&) = delete; + virtual ~VersionSet(); // Apply *edit to the current version to form a new descriptor that @@ -1143,10 +1147,6 @@ class VersionSet { BlockCacheTracer* const block_cache_tracer_; private: - // No copying allowed - VersionSet(const VersionSet&); - void operator=(const VersionSet&); - // REQUIRES db mutex at beginning. may release and re-acquire db mutex Status ProcessManifestWrites(std::deque& writers, InstrumentedMutex* mu, Directory* db_directory, diff --git a/env/mock_env.cc b/env/mock_env.cc index 793a0837a..6d3adc808 100644 --- a/env/mock_env.cc +++ b/env/mock_env.cc @@ -31,6 +31,9 @@ class MemFile { rnd_(static_cast( MurmurHash(fn.data(), static_cast(fn.size()), 0))), fsynced_bytes_(0) {} + // No copying allowed. + MemFile(const MemFile&) = delete; + void operator=(const MemFile&) = delete; void Ref() { MutexLock lock(&mutex_); @@ -154,10 +157,6 @@ class MemFile { // Private since only Unref() should be used to delete it. ~MemFile() { assert(refs_ == 0); } - // No copying allowed. - MemFile(const MemFile&); - void operator=(const MemFile&); - Env* env_; const std::string fn_; mutable port::Mutex mutex_; diff --git a/include/rocksdb/cache.h b/include/rocksdb/cache.h index 6bde575e0..d8093c7ea 100644 --- a/include/rocksdb/cache.h +++ b/include/rocksdb/cache.h @@ -122,6 +122,9 @@ class Cache { Cache(std::shared_ptr allocator = nullptr) : memory_allocator_(std::move(allocator)) {} + // No copying allowed + Cache(const Cache&) = delete; + Cache& operator=(const Cache&) = delete; // Destroys all existing entries by calling the "deleter" // function that was passed via the Insert() function. @@ -253,10 +256,6 @@ class Cache { MemoryAllocator* memory_allocator() const { return memory_allocator_.get(); } private: - // No copying allowed - Cache(const Cache&); - Cache& operator=(const Cache&); - std::shared_ptr memory_allocator_; }; diff --git a/include/rocksdb/cleanable.h b/include/rocksdb/cleanable.h index 6dba8d953..3a111d545 100644 --- a/include/rocksdb/cleanable.h +++ b/include/rocksdb/cleanable.h @@ -23,12 +23,12 @@ namespace rocksdb { class Cleanable { public: Cleanable(); - ~Cleanable(); - // No copy constructor and copy assignment allowed. Cleanable(Cleanable&) = delete; Cleanable& operator=(Cleanable&) = delete; + ~Cleanable(); + // Move constructor and move assignment is allowed. Cleanable(Cleanable&&); Cleanable& operator=(Cleanable&&); diff --git a/include/rocksdb/db.h b/include/rocksdb/db.h index 6aa05baae..a2961792b 100644 --- a/include/rocksdb/db.h +++ b/include/rocksdb/db.h @@ -255,6 +255,10 @@ class DB { std::vector* column_families); DB() {} + // No copying allowed + DB(const DB&) = delete; + void operator=(const DB&) = delete; + virtual ~DB(); // Create a column_family and return the handle of column family @@ -1421,11 +1425,6 @@ class DB { return Status::NotSupported("Supported only by secondary instance"); } #endif // !ROCKSDB_LITE - - private: - // No copying allowed - DB(const DB&); - void operator=(const DB&); }; // Destroy the contents of the specified database. diff --git a/include/rocksdb/env.h b/include/rocksdb/env.h index 398a7ff51..b3b30b5ac 100644 --- a/include/rocksdb/env.h +++ b/include/rocksdb/env.h @@ -141,6 +141,9 @@ class Env { }; Env() : thread_status_updater_(nullptr) {} + // No copying allowed + Env(const Env&) = delete; + void operator=(const Env&) = delete; virtual ~Env(); @@ -527,11 +530,6 @@ class Env { // The pointer to an internal structure that will update the // status of each thread. ThreadStatusUpdater* thread_status_updater_; - - private: - // No copying allowed - Env(const Env&); - void operator=(const Env&); }; // The factory function to construct a ThreadStatusUpdater. Any Env @@ -711,6 +709,9 @@ class WritableFile { io_priority_(Env::IO_TOTAL), write_hint_(Env::WLTH_NOT_SET), strict_bytes_per_sync_(options.strict_bytes_per_sync) {} + // No copying allowed + WritableFile(const WritableFile&) = delete; + void operator=(const WritableFile&) = delete; virtual ~WritableFile(); @@ -870,9 +871,6 @@ class WritableFile { private: size_t last_preallocated_block_; size_t preallocation_block_size_; - // No copying allowed - WritableFile(const WritableFile&); - void operator=(const WritableFile&); protected: Env::IOPriority io_priority_; @@ -884,6 +882,10 @@ class WritableFile { class RandomRWFile { public: RandomRWFile() {} + // No copying allowed + RandomRWFile(const RandomRWFile&) = delete; + RandomRWFile& operator=(const RandomRWFile&) = delete; + virtual ~RandomRWFile() {} // Indicates if the class makes use of direct I/O @@ -914,10 +916,6 @@ class RandomRWFile { // If you're adding methods here, remember to add them to // RandomRWFileWrapper too. - - // No copying allowed - RandomRWFile(const RandomRWFile&) = delete; - RandomRWFile& operator=(const RandomRWFile&) = delete; }; // MemoryMappedFileBuffer object represents a memory-mapped file's raw buffer. @@ -975,6 +973,10 @@ class Logger { explicit Logger(const InfoLogLevel log_level = InfoLogLevel::INFO_LEVEL) : closed_(false), log_level_(log_level) {} + // No copying allowed + Logger(const Logger&) = delete; + void operator=(const Logger&) = delete; + virtual ~Logger(); // Close the log file. Must be called before destructor. If the return @@ -1016,9 +1018,6 @@ class Logger { bool closed_; private: - // No copying allowed - Logger(const Logger&); - void operator=(const Logger&); InfoLogLevel log_level_; }; @@ -1030,8 +1029,8 @@ class FileLock { private: // No copying allowed - FileLock(const FileLock&); - void operator=(const FileLock&); + FileLock(const FileLock&) = delete; + void operator=(const FileLock&) = delete; }; class DynamicLibrary { diff --git a/include/rocksdb/iterator.h b/include/rocksdb/iterator.h index e99b434a0..162e262e3 100644 --- a/include/rocksdb/iterator.h +++ b/include/rocksdb/iterator.h @@ -28,6 +28,10 @@ namespace rocksdb { class Iterator : public Cleanable { public: Iterator() {} + // No copying allowed + Iterator(const Iterator&) = delete; + void operator=(const Iterator&) = delete; + virtual ~Iterator() {} // An iterator is either positioned at a key/value pair, or @@ -104,11 +108,6 @@ class Iterator : public Cleanable { // Get the user-key portion of the internal key at which the iteration // stopped. virtual Status GetProperty(std::string prop_name, std::string* prop); - - private: - // No copying allowed - Iterator(const Iterator&); - void operator=(const Iterator&); }; // Return an empty iterator (yields nothing). diff --git a/include/rocksdb/utilities/transaction.h b/include/rocksdb/utilities/transaction.h index 95d299c1b..44ce28019 100644 --- a/include/rocksdb/utilities/transaction.h +++ b/include/rocksdb/utilities/transaction.h @@ -52,6 +52,10 @@ class TransactionNotifier { // -Support for using Transactions with DBWithTTL class Transaction { public: + // No copying allowed + Transaction(const Transaction&) = delete; + void operator=(const Transaction&) = delete; + virtual ~Transaction() {} // If a transaction has a snapshot set, the transaction will ensure that @@ -529,9 +533,6 @@ class Transaction { friend class WriteUnpreparedTxnDB; friend class TransactionTest_TwoPhaseLogRollingTest_Test; friend class TransactionTest_TwoPhaseLogRollingTest2_Test; - // No copying allowed - Transaction(const Transaction&); - void operator=(const Transaction&); }; } // namespace rocksdb diff --git a/include/rocksdb/utilities/transaction_db.h b/include/rocksdb/utilities/transaction_db.h index fa271c35d..91a9cec28 100644 --- a/include/rocksdb/utilities/transaction_db.h +++ b/include/rocksdb/utilities/transaction_db.h @@ -301,11 +301,9 @@ class TransactionDB : public StackableDB { // To Create an TransactionDB, call Open() // The ownership of db is transferred to the base StackableDB explicit TransactionDB(DB* db) : StackableDB(db) {} - - private: // No copying allowed - TransactionDB(const TransactionDB&); - void operator=(const TransactionDB&); + TransactionDB(const TransactionDB&) = delete; + void operator=(const TransactionDB&) = delete; }; } // namespace rocksdb diff --git a/include/rocksdb/write_buffer_manager.h b/include/rocksdb/write_buffer_manager.h index dea904c18..a6c204a63 100644 --- a/include/rocksdb/write_buffer_manager.h +++ b/include/rocksdb/write_buffer_manager.h @@ -26,6 +26,10 @@ class WriteBufferManager { // the memory allocated to the cache. It can be used even if _buffer_size = 0. explicit WriteBufferManager(size_t _buffer_size, std::shared_ptr cache = {}); + // No copying allowed + WriteBufferManager(const WriteBufferManager&) = delete; + WriteBufferManager& operator=(const WriteBufferManager&) = delete; + ~WriteBufferManager(); bool enabled() const { return buffer_size_ != 0; } @@ -94,9 +98,5 @@ class WriteBufferManager { void ReserveMemWithCache(size_t mem); void FreeMemWithCache(size_t mem); - - // No copying allowed - WriteBufferManager(const WriteBufferManager&) = delete; - WriteBufferManager& operator=(const WriteBufferManager&) = delete; }; } // namespace rocksdb diff --git a/memory/allocator.h b/memory/allocator.h index 505d6ba2b..619cd66a5 100644 --- a/memory/allocator.h +++ b/memory/allocator.h @@ -33,6 +33,10 @@ class Allocator { class AllocTracker { public: explicit AllocTracker(WriteBufferManager* write_buffer_manager); + // No copying allowed + AllocTracker(const AllocTracker&) = delete; + void operator=(const AllocTracker&) = delete; + ~AllocTracker(); void Allocate(size_t bytes); // Call when we're finished allocating memory so we can free it from @@ -48,10 +52,6 @@ class AllocTracker { std::atomic bytes_allocated_; bool done_allocating_; bool freed_; - - // No copying allowed - AllocTracker(const AllocTracker&); - void operator=(const AllocTracker&); }; } // namespace rocksdb diff --git a/memtable/inlineskiplist.h b/memtable/inlineskiplist.h index c3adb2ddb..faebad63e 100644 --- a/memtable/inlineskiplist.h +++ b/memtable/inlineskiplist.h @@ -74,6 +74,9 @@ class InlineSkipList { explicit InlineSkipList(Comparator cmp, Allocator* allocator, int32_t max_height = 12, int32_t branching_factor = 4); + // No copying allowed + InlineSkipList(const InlineSkipList&) = delete; + InlineSkipList& operator=(const InlineSkipList&) = delete; // Allocates a key and a skip-list node, returning a pointer to the key // portion of the node. This method is thread-safe if the allocator @@ -254,10 +257,6 @@ class InlineSkipList { // lowest_level (inclusive). void RecomputeSpliceLevels(const DecodedKey& key, Splice* splice, int recompute_level); - - // No copying allowed - InlineSkipList(const InlineSkipList&); - InlineSkipList& operator=(const InlineSkipList&); }; // Implementation details follow diff --git a/memtable/skiplist.h b/memtable/skiplist.h index 275daa794..5edfc10b7 100644 --- a/memtable/skiplist.h +++ b/memtable/skiplist.h @@ -51,6 +51,9 @@ class SkipList { // allocator must remain allocated for the lifetime of the skiplist object. explicit SkipList(Comparator cmp, Allocator* allocator, int32_t max_height = 12, int32_t branching_factor = 4); + // No copying allowed + SkipList(const SkipList&) = delete; + void operator=(const SkipList&) = delete; // Insert key into the list. // REQUIRES: nothing that compares equal to key is currently in the list. @@ -158,10 +161,6 @@ class SkipList { // Return the last node in the list. // Return head_ if list is empty. Node* FindLast() const; - - // No copying allowed - SkipList(const SkipList&); - void operator=(const SkipList&); }; // Implementation details follow diff --git a/monitoring/in_memory_stats_history.h b/monitoring/in_memory_stats_history.h index 8ccec146a..37b50ca06 100644 --- a/monitoring/in_memory_stats_history.h +++ b/monitoring/in_memory_stats_history.h @@ -34,6 +34,13 @@ class InMemoryStatsHistoryIterator final : public StatsHistoryIterator { db_impl_(db_impl) { AdvanceIteratorByTime(start_time_, end_time_); } + // no copying allowed + InMemoryStatsHistoryIterator(const InMemoryStatsHistoryIterator&) = delete; + void operator=(const InMemoryStatsHistoryIterator&) = delete; + InMemoryStatsHistoryIterator(InMemoryStatsHistoryIterator&&) = delete; + InMemoryStatsHistoryIterator& operator=(InMemoryStatsHistoryIterator&&) = + delete; + ~InMemoryStatsHistoryIterator() override; bool Valid() const override; Status status() const override; @@ -55,13 +62,6 @@ class InMemoryStatsHistoryIterator final : public StatsHistoryIterator { // between [start_time, end_time) void AdvanceIteratorByTime(uint64_t start_time, uint64_t end_time); - // No copying allowed - InMemoryStatsHistoryIterator(const InMemoryStatsHistoryIterator&) = delete; - void operator=(const InMemoryStatsHistoryIterator&) = delete; - InMemoryStatsHistoryIterator(InMemoryStatsHistoryIterator&&) = delete; - InMemoryStatsHistoryIterator& operator=(InMemoryStatsHistoryIterator&&) = - delete; - uint64_t time_; uint64_t start_time_; uint64_t end_time_; diff --git a/port/port_posix.h b/port/port_posix.h index 63d7239fe..51eb24162 100644 --- a/port/port_posix.h +++ b/port/port_posix.h @@ -104,6 +104,10 @@ class CondVar; class Mutex { public: explicit Mutex(bool adaptive = kDefaultToAdaptiveMutex); + // No copying + Mutex(const Mutex&) = delete; + void operator=(const Mutex&) = delete; + ~Mutex(); void Lock(); @@ -118,15 +122,15 @@ class Mutex { #ifndef NDEBUG bool locked_; #endif - - // No copying - Mutex(const Mutex&); - void operator=(const Mutex&); }; class RWMutex { public: RWMutex(); + // No copying allowed + RWMutex(const RWMutex&) = delete; + void operator=(const RWMutex&) = delete; + ~RWMutex(); void ReadLock(); @@ -137,10 +141,6 @@ class RWMutex { private: pthread_rwlock_t mu_; // the underlying platform mutex - - // No copying allowed - RWMutex(const RWMutex&); - void operator=(const RWMutex&); }; class CondVar { diff --git a/port/win/port_win.h b/port/win/port_win.h index de41cdc7f..1b302b3d2 100644 --- a/port/win/port_win.h +++ b/port/win/port_win.h @@ -180,6 +180,9 @@ class Mutex { class RWMutex { public: RWMutex() { InitializeSRWLock(&srwLock_); } + // No copying allowed + RWMutex(const RWMutex&) = delete; + void operator=(const RWMutex&) = delete; void ReadLock() { AcquireSRWLockShared(&srwLock_); } @@ -194,9 +197,6 @@ class RWMutex { private: SRWLOCK srwLock_; - // No copying allowed - RWMutex(const RWMutex&); - void operator=(const RWMutex&); }; class CondVar { diff --git a/table/block_based/block.h b/table/block_based/block.h index 3af92b6a2..3e19f9fdc 100644 --- a/table/block_based/block.h +++ b/table/block_based/block.h @@ -147,6 +147,9 @@ class Block { explicit Block(BlockContents&& contents, SequenceNumber _global_seqno, size_t read_amp_bytes_per_bit = 0, Statistics* statistics = nullptr); + // No copying allowed + Block(const Block&) = delete; + void operator=(const Block&) = delete; ~Block(); @@ -222,10 +225,6 @@ class Block { const SequenceNumber global_seqno_; DataBlockHashIndex data_block_hash_index_; - - // No copying allowed - Block(const Block&) = delete; - void operator=(const Block&) = delete; }; template diff --git a/table/block_based/block_based_filter_block.h b/table/block_based/block_based_filter_block.h index 43dbc4f4f..ed409e041 100644 --- a/table/block_based/block_based_filter_block.h +++ b/table/block_based/block_based_filter_block.h @@ -38,6 +38,9 @@ class BlockBasedFilterBlockBuilder : public FilterBlockBuilder { public: BlockBasedFilterBlockBuilder(const SliceTransform* prefix_extractor, const BlockBasedTableOptions& table_opt); + // No copying allowed + BlockBasedFilterBlockBuilder(const BlockBasedFilterBlockBuilder&) = delete; + void operator=(const BlockBasedFilterBlockBuilder&) = delete; virtual bool IsBlockBased() override { return true; } virtual void StartBlock(uint64_t block_offset) override; @@ -68,10 +71,6 @@ class BlockBasedFilterBlockBuilder : public FilterBlockBuilder { std::vector tmp_entries_; // policy_->CreateFilter() argument std::vector filter_offsets_; size_t num_added_; // Number of keys added - - // No copying allowed - BlockBasedFilterBlockBuilder(const BlockBasedFilterBlockBuilder&); - void operator=(const BlockBasedFilterBlockBuilder&); }; // A FilterBlockReader is used to parse filter from SST table. @@ -81,6 +80,9 @@ class BlockBasedFilterBlockReader public: BlockBasedFilterBlockReader(const BlockBasedTable* t, CachableEntry&& filter_block); + // No copying allowed + BlockBasedFilterBlockReader(const BlockBasedFilterBlockReader&) = delete; + void operator=(const BlockBasedFilterBlockReader&) = delete; static std::unique_ptr Create( const BlockBasedTable* table, FilePrefetchBuffer* prefetch_buffer, diff --git a/table/block_based/block_based_table_builder.h b/table/block_based/block_based_table_builder.h index a1ef38891..bd099ab3d 100644 --- a/table/block_based/block_based_table_builder.h +++ b/table/block_based/block_based_table_builder.h @@ -51,13 +51,13 @@ class BlockBasedTableBuilder : public TableBuilder { const uint64_t oldest_key_time = 0, const uint64_t target_file_size = 0, const uint64_t file_creation_time = 0); - // REQUIRES: Either Finish() or Abandon() has been called. - ~BlockBasedTableBuilder(); - // No copying allowed BlockBasedTableBuilder(const BlockBasedTableBuilder&) = delete; BlockBasedTableBuilder& operator=(const BlockBasedTableBuilder&) = delete; + // REQUIRES: Either Finish() or Abandon() has been called. + ~BlockBasedTableBuilder(); + // Add key,value to the table being constructed. // REQUIRES: key is after any previously added key according to comparator. // REQUIRES: Finish(), Abandon() have not been called diff --git a/table/block_based/block_based_table_reader.h b/table/block_based/block_based_table_reader.h index 59575899a..c8e8ea006 100644 --- a/table/block_based/block_based_table_reader.h +++ b/table/block_based/block_based_table_reader.h @@ -265,6 +265,9 @@ class BlockBasedTable : public TableReader { Rep* rep_; explicit BlockBasedTable(Rep* rep, BlockCacheTracer* const block_cache_tracer) : rep_(rep), block_cache_tracer_(block_cache_tracer) {} + // No copying allowed + explicit BlockBasedTable(const TableReader&) = delete; + void operator=(const TableReader&) = delete; private: friend class MockedBlockBasedTable; @@ -458,10 +461,6 @@ class BlockBasedTable : public TableReader { void DumpKeyValue(const Slice& key, const Slice& value, WritableFile* out_file); - // No copying allowed - explicit BlockBasedTable(const TableReader&) = delete; - void operator=(const TableReader&) = delete; - friend class PartitionedFilterBlockReader; friend class PartitionedFilterBlockTest; }; diff --git a/table/block_based/filter_block.h b/table/block_based/filter_block.h index a2871e6c8..38c3cc05f 100644 --- a/table/block_based/filter_block.h +++ b/table/block_based/filter_block.h @@ -52,6 +52,10 @@ using MultiGetRange = MultiGetContext::Range; class FilterBlockBuilder { public: explicit FilterBlockBuilder() {} + // No copying allowed + FilterBlockBuilder(const FilterBlockBuilder&) = delete; + void operator=(const FilterBlockBuilder&) = delete; + virtual ~FilterBlockBuilder() {} virtual bool IsBlockBased() = 0; // If is blockbased filter @@ -66,11 +70,6 @@ class FilterBlockBuilder { return ret; } virtual Slice Finish(const BlockHandle& tmp, Status* status) = 0; - - private: - // No copying allowed - FilterBlockBuilder(const FilterBlockBuilder&); - void operator=(const FilterBlockBuilder&); }; // A FilterBlockReader is used to parse filter from SST table. diff --git a/table/block_based/full_filter_block.h b/table/block_based/full_filter_block.h index 4d10a5a33..ae1e974f4 100644 --- a/table/block_based/full_filter_block.h +++ b/table/block_based/full_filter_block.h @@ -40,6 +40,10 @@ class FullFilterBlockBuilder : public FilterBlockBuilder { explicit FullFilterBlockBuilder(const SliceTransform* prefix_extractor, bool whole_key_filtering, FilterBitsBuilder* filter_bits_builder); + // No copying allowed + FullFilterBlockBuilder(const FullFilterBlockBuilder&) = delete; + void operator=(const FullFilterBlockBuilder&) = delete; + // bits_builder is created in filter_policy, it should be passed in here // directly. and be deleted here ~FullFilterBlockBuilder() {} @@ -71,10 +75,6 @@ class FullFilterBlockBuilder : public FilterBlockBuilder { std::unique_ptr filter_data_; void AddPrefix(const Slice& key); - - // No copying allowed - FullFilterBlockBuilder(const FullFilterBlockBuilder&); - void operator=(const FullFilterBlockBuilder&); }; // A FilterBlockReader is used to parse filter from SST table. diff --git a/table/cuckoo/cuckoo_table_builder.h b/table/cuckoo/cuckoo_table_builder.h index 3829541b3..c42744de0 100644 --- a/table/cuckoo/cuckoo_table_builder.h +++ b/table/cuckoo/cuckoo_table_builder.h @@ -30,6 +30,9 @@ class CuckooTableBuilder: public TableBuilder { uint64_t), uint32_t column_family_id, const std::string& column_family_name); + // No copying allowed + CuckooTableBuilder(const CuckooTableBuilder&) = delete; + void operator=(const CuckooTableBuilder&) = delete; // REQUIRES: Either Finish() or Abandon() has been called. ~CuckooTableBuilder() {} @@ -116,10 +119,6 @@ class CuckooTableBuilder: public TableBuilder { std::string smallest_user_key_ = ""; bool closed_; // Either Finish() or Abandon() has been called. - - // No copying allowed - CuckooTableBuilder(const CuckooTableBuilder&) = delete; - void operator=(const CuckooTableBuilder&) = delete; }; } // namespace rocksdb diff --git a/table/cuckoo/cuckoo_table_reader.cc b/table/cuckoo/cuckoo_table_reader.cc index 30109ece6..982b14763 100644 --- a/table/cuckoo/cuckoo_table_reader.cc +++ b/table/cuckoo/cuckoo_table_reader.cc @@ -197,6 +197,9 @@ void CuckooTableReader::Prepare(const Slice& key) { class CuckooTableIterator : public InternalIterator { public: explicit CuckooTableIterator(CuckooTableReader* reader); + // No copying allowed + CuckooTableIterator(const CuckooTableIterator&) = delete; + void operator=(const Iterator&) = delete; ~CuckooTableIterator() override {} bool Valid() const override; void SeekToFirst() override; @@ -248,9 +251,6 @@ class CuckooTableIterator : public InternalIterator { uint32_t curr_key_idx_; Slice curr_value_; IterKey curr_key_; - // No copying allowed - CuckooTableIterator(const CuckooTableIterator&) = delete; - void operator=(const Iterator&) = delete; }; CuckooTableIterator::CuckooTableIterator(CuckooTableReader* reader) diff --git a/table/full_filter_bits_builder.h b/table/full_filter_bits_builder.h index 851ed1e2a..c719c698a 100644 --- a/table/full_filter_bits_builder.h +++ b/table/full_filter_bits_builder.h @@ -23,6 +23,10 @@ class FullFilterBitsBuilder : public FilterBitsBuilder { explicit FullFilterBitsBuilder(const size_t bits_per_key, const size_t num_probes); + // No Copy allowed + FullFilterBitsBuilder(const FullFilterBitsBuilder&) = delete; + void operator=(const FullFilterBitsBuilder&) = delete; + ~FullFilterBitsBuilder(); virtual void AddKey(const Slice& key) override; @@ -65,10 +69,6 @@ class FullFilterBitsBuilder : public FilterBitsBuilder { // Assuming single threaded access to this function. void AddHash(uint32_t h, char* data, uint32_t num_lines, uint32_t total_bits); - - // No Copy allowed - FullFilterBitsBuilder(const FullFilterBitsBuilder&); - void operator=(const FullFilterBitsBuilder&); }; } // namespace rocksdb diff --git a/table/internal_iterator.h b/table/internal_iterator.h index 426ff3965..adcccf795 100644 --- a/table/internal_iterator.h +++ b/table/internal_iterator.h @@ -27,6 +27,10 @@ class InternalIteratorBase : public Cleanable { public: InternalIteratorBase() : is_mutable_(true) {} InternalIteratorBase(bool _is_mutable) : is_mutable_(_is_mutable) {} + // No copying allowed + InternalIteratorBase(const InternalIteratorBase&) = delete; + InternalIteratorBase& operator=(const InternalIteratorBase&) = delete; + virtual ~InternalIteratorBase() {} // An iterator is either positioned at a key/value pair, or @@ -156,12 +160,8 @@ class InternalIteratorBase : public Cleanable { Prev(); } } - bool is_mutable_; - private: - // No copying allowed - InternalIteratorBase(const InternalIteratorBase&) = delete; - InternalIteratorBase& operator=(const InternalIteratorBase&) = delete; + bool is_mutable_; }; using InternalIterator = InternalIteratorBase; diff --git a/table/plain/plain_table_builder.h b/table/plain/plain_table_builder.h index ce2169a38..f2cd6009e 100644 --- a/table/plain/plain_table_builder.h +++ b/table/plain/plain_table_builder.h @@ -45,6 +45,9 @@ class PlainTableBuilder: public TableBuilder { const std::string& column_family_name, uint32_t num_probes = 6, size_t huge_page_tlb_size = 0, double hash_table_ratio = 0, bool store_index_in_file = false); + // No copying allowed + PlainTableBuilder(const PlainTableBuilder&) = delete; + void operator=(const PlainTableBuilder&) = delete; // REQUIRES: Either Finish() or Abandon() has been called. ~PlainTableBuilder(); @@ -131,10 +134,6 @@ class PlainTableBuilder: public TableBuilder { } bool IsTotalOrderMode() const { return (prefix_extractor_ == nullptr); } - - // No copying allowed - PlainTableBuilder(const PlainTableBuilder&) = delete; - void operator=(const PlainTableBuilder&) = delete; }; } // namespace rocksdb diff --git a/table/plain/plain_table_reader.cc b/table/plain/plain_table_reader.cc index 3d5c4f2db..2ac7cf2e3 100644 --- a/table/plain/plain_table_reader.cc +++ b/table/plain/plain_table_reader.cc @@ -55,6 +55,10 @@ inline uint32_t GetFixed32Element(const char* base, size_t offset) { class PlainTableIterator : public InternalIterator { public: explicit PlainTableIterator(PlainTableReader* table, bool use_prefix_seek); + // No copying allowed + PlainTableIterator(const PlainTableIterator&) = delete; + void operator=(const Iterator&) = delete; + ~PlainTableIterator() override; bool Valid() const override; @@ -86,9 +90,6 @@ class PlainTableIterator : public InternalIterator { Slice key_; Slice value_; Status status_; - // No copying allowed - PlainTableIterator(const PlainTableIterator&) = delete; - void operator=(const Iterator&) = delete; }; extern const uint64_t kPlainTableMagicNumber; diff --git a/util/bloom.cc b/util/bloom.cc index f859ab7dd..23607a51e 100644 --- a/util/bloom.cc +++ b/util/bloom.cc @@ -171,6 +171,9 @@ class FullFilterBitsReader : public FilterBitsReader { } } } + // No Copy allowed + FullFilterBitsReader(const FullFilterBitsReader&) = delete; + void operator=(const FullFilterBitsReader&) = delete; ~FullFilterBitsReader() override {} @@ -244,10 +247,6 @@ class FullFilterBitsReader : public FilterBitsReader { void FilterPrepare(const uint32_t& hash, const Slice& filter, const uint32_t& num_lines, uint32_t* bit_offset); - - // No Copy allowed - FullFilterBitsReader(const FullFilterBitsReader&); - void operator=(const FullFilterBitsReader&); }; void FullFilterBitsReader::GetFilterMeta(const Slice& filter, diff --git a/util/concurrent_task_limiter_impl.h b/util/concurrent_task_limiter_impl.h index 515f1481e..91b7bbe3d 100644 --- a/util/concurrent_task_limiter_impl.h +++ b/util/concurrent_task_limiter_impl.h @@ -22,6 +22,10 @@ class ConcurrentTaskLimiterImpl : public ConcurrentTaskLimiter { public: explicit ConcurrentTaskLimiterImpl(const std::string& name, int32_t max_outstanding_task); + // No copying allowed + ConcurrentTaskLimiterImpl(const ConcurrentTaskLimiterImpl&) = delete; + ConcurrentTaskLimiterImpl& operator=( + const ConcurrentTaskLimiterImpl&) = delete; virtual ~ConcurrentTaskLimiterImpl(); @@ -44,11 +48,6 @@ class ConcurrentTaskLimiterImpl : public ConcurrentTaskLimiter { std::string name_; std::atomic max_outstanding_tasks_; std::atomic outstanding_tasks_; - - // No copying allowed - ConcurrentTaskLimiterImpl(const ConcurrentTaskLimiterImpl&) = delete; - ConcurrentTaskLimiterImpl& operator=( - const ConcurrentTaskLimiterImpl&) = delete; }; class TaskLimiterToken { diff --git a/util/mutexlock.h b/util/mutexlock.h index 640cef3da..90e6c8b99 100644 --- a/util/mutexlock.h +++ b/util/mutexlock.h @@ -31,13 +31,14 @@ class MutexLock { explicit MutexLock(port::Mutex *mu) : mu_(mu) { this->mu_->Lock(); } + // No copying allowed + MutexLock(const MutexLock&) = delete; + void operator=(const MutexLock&) = delete; + ~MutexLock() { this->mu_->Unlock(); } private: port::Mutex *const mu_; - // No copying allowed - MutexLock(const MutexLock&); - void operator=(const MutexLock&); }; // @@ -50,13 +51,14 @@ class ReadLock { explicit ReadLock(port::RWMutex *mu) : mu_(mu) { this->mu_->ReadLock(); } + // No copying allowed + ReadLock(const ReadLock&) = delete; + void operator=(const ReadLock&) = delete; + ~ReadLock() { this->mu_->ReadUnlock(); } private: port::RWMutex *const mu_; - // No copying allowed - ReadLock(const ReadLock&); - void operator=(const ReadLock&); }; // @@ -65,13 +67,14 @@ class ReadLock { class ReadUnlock { public: explicit ReadUnlock(port::RWMutex *mu) : mu_(mu) { mu->AssertHeld(); } + // No copying allowed + ReadUnlock(const ReadUnlock &) = delete; + ReadUnlock &operator=(const ReadUnlock &) = delete; + ~ReadUnlock() { mu_->ReadUnlock(); } private: port::RWMutex *const mu_; - // No copying allowed - ReadUnlock(const ReadUnlock &) = delete; - ReadUnlock &operator=(const ReadUnlock &) = delete; }; // @@ -84,13 +87,14 @@ class WriteLock { explicit WriteLock(port::RWMutex *mu) : mu_(mu) { this->mu_->WriteLock(); } + // No copying allowed + WriteLock(const WriteLock&) = delete; + void operator=(const WriteLock&) = delete; + ~WriteLock() { this->mu_->WriteUnlock(); } private: port::RWMutex *const mu_; - // No copying allowed - WriteLock(const WriteLock&); - void operator=(const WriteLock&); }; // diff --git a/utilities/blob_db/blob_log_reader.h b/utilities/blob_db/blob_log_reader.h index 45e2e9551..af7971554 100644 --- a/utilities/blob_db/blob_log_reader.h +++ b/utilities/blob_db/blob_log_reader.h @@ -42,13 +42,12 @@ class Reader { // "*file" must remain live while this Reader is in use. Reader(std::unique_ptr&& file_reader, Env* env, Statistics* statistics); - - ~Reader() = default; - // No copying allowed Reader(const Reader&) = delete; Reader& operator=(const Reader&) = delete; + ~Reader() = default; + Status ReadHeader(BlobLogHeader* header); // Read the next record into *record. Returns true if read diff --git a/utilities/blob_db/blob_log_writer.h b/utilities/blob_db/blob_log_writer.h index dccac355c..faf2cfd26 100644 --- a/utilities/blob_db/blob_log_writer.h +++ b/utilities/blob_db/blob_log_writer.h @@ -39,13 +39,12 @@ class Writer { Writer(std::unique_ptr&& dest, Env* env, Statistics* statistics, uint64_t log_number, uint64_t bpsync, bool use_fsync, uint64_t boffset = 0); - - ~Writer() = default; - // No copying allowed Writer(const Writer&) = delete; Writer& operator=(const Writer&) = delete; + ~Writer() = default; + static void ConstructBlobHeader(std::string* buf, const Slice& key, const Slice& val, uint64_t expiration); diff --git a/utilities/transactions/optimistic_transaction.h b/utilities/transactions/optimistic_transaction.h index 445979b96..b6e249f3c 100644 --- a/utilities/transactions/optimistic_transaction.h +++ b/utilities/transactions/optimistic_transaction.h @@ -31,6 +31,9 @@ class OptimisticTransaction : public TransactionBaseImpl { OptimisticTransaction(OptimisticTransactionDB* db, const WriteOptions& write_options, const OptimisticTransactionOptions& txn_options); + // No copying allowed + OptimisticTransaction(const OptimisticTransaction&) = delete; + void operator=(const OptimisticTransaction&) = delete; virtual ~OptimisticTransaction(); @@ -52,7 +55,7 @@ class OptimisticTransaction : public TransactionBaseImpl { const bool assume_tracked = false) override; private: - OptimisticTransactionDB* const txn_db_; + ROCKSDB_FIELD_UNUSED OptimisticTransactionDB* const txn_db_; friend class OptimisticTransactionCallback; @@ -71,10 +74,6 @@ class OptimisticTransaction : public TransactionBaseImpl { const Slice& /* unused */) override { // Nothing to unlock. } - - // No copying allowed - OptimisticTransaction(const OptimisticTransaction&); - void operator=(const OptimisticTransaction&); }; // Used at commit time to trigger transaction validation diff --git a/utilities/transactions/pessimistic_transaction.h b/utilities/transactions/pessimistic_transaction.h index 1f851818e..c91d9217b 100644 --- a/utilities/transactions/pessimistic_transaction.h +++ b/utilities/transactions/pessimistic_transaction.h @@ -40,6 +40,9 @@ class PessimisticTransaction : public TransactionBaseImpl { PessimisticTransaction(TransactionDB* db, const WriteOptions& write_options, const TransactionOptions& txn_options, const bool init = true); + // No copying allowed + PessimisticTransaction(const PessimisticTransaction&) = delete; + void operator=(const PessimisticTransaction&) = delete; virtual ~PessimisticTransaction(); @@ -193,16 +196,15 @@ class PessimisticTransaction : public TransactionBaseImpl { void UnlockGetForUpdate(ColumnFamilyHandle* column_family, const Slice& key) override; - - // No copying allowed - PessimisticTransaction(const PessimisticTransaction&); - void operator=(const PessimisticTransaction&); }; class WriteCommittedTxn : public PessimisticTransaction { public: WriteCommittedTxn(TransactionDB* db, const WriteOptions& write_options, const TransactionOptions& txn_options); + // No copying allowed + WriteCommittedTxn(const WriteCommittedTxn&) = delete; + void operator=(const WriteCommittedTxn&) = delete; virtual ~WriteCommittedTxn() {} @@ -216,10 +218,6 @@ class WriteCommittedTxn : public PessimisticTransaction { Status CommitInternal() override; Status RollbackInternal() override; - - // No copying allowed - WriteCommittedTxn(const WriteCommittedTxn&); - void operator=(const WriteCommittedTxn&); }; } // namespace rocksdb diff --git a/utilities/transactions/transaction_lock_mgr.h b/utilities/transactions/transaction_lock_mgr.h index b12357604..16a40d7b7 100644 --- a/utilities/transactions/transaction_lock_mgr.h +++ b/utilities/transactions/transaction_lock_mgr.h @@ -57,6 +57,9 @@ class TransactionLockMgr { TransactionLockMgr(TransactionDB* txn_db, size_t default_num_stripes, int64_t max_num_locks, uint32_t max_num_deadlocks, std::shared_ptr factory); + // No copying allowed + TransactionLockMgr(const TransactionLockMgr&) = delete; + void operator=(const TransactionLockMgr&) = delete; ~TransactionLockMgr(); @@ -149,10 +152,6 @@ class TransactionLockMgr { const autovector& wait_ids); void DecrementWaitersImpl(const PessimisticTransaction* txn, const autovector& wait_ids); - - // No copying allowed - TransactionLockMgr(const TransactionLockMgr&); - void operator=(const TransactionLockMgr&); }; } // namespace rocksdb diff --git a/utilities/transactions/write_prepared_txn.h b/utilities/transactions/write_prepared_txn.h index c574f6231..e5dadabc4 100644 --- a/utilities/transactions/write_prepared_txn.h +++ b/utilities/transactions/write_prepared_txn.h @@ -42,6 +42,9 @@ class WritePreparedTxn : public PessimisticTransaction { public: WritePreparedTxn(WritePreparedTxnDB* db, const WriteOptions& write_options, const TransactionOptions& txn_options); + // No copying allowed + WritePreparedTxn(const WritePreparedTxn&) = delete; + void operator=(const WritePreparedTxn&) = delete; virtual ~WritePreparedTxn() {} @@ -106,10 +109,6 @@ class WritePreparedTxn : public PessimisticTransaction { virtual Status RebuildFromWriteBatch(WriteBatch* src_batch) override; - // No copying allowed - WritePreparedTxn(const WritePreparedTxn&); - void operator=(const WritePreparedTxn&); - WritePreparedTxnDB* wpt_db_; // Number of sub-batches in prepare size_t prepare_batch_cnt_ = 0;