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
main
Shylock Hg 5 years ago committed by Facebook Github Bot
parent fcda80fc33
commit 9eb3e1f77d
  1. 8
      db/compacted_db_impl.h
  2. 8
      db/db_impl/db_impl.h
  3. 8
      db/db_impl/db_impl_readonly.h
  4. 14
      db/db_iter.cc
  5. 7
      db/dbformat.h
  6. 8
      db/log_reader.h
  7. 8
      db/log_writer.h
  8. 7
      db/memtable.cc
  9. 7
      db/memtable.h
  10. 18
      db/version_set.h
  11. 7
      env/mock_env.cc
  12. 7
      include/rocksdb/cache.h
  13. 4
      include/rocksdb/cleanable.h
  14. 9
      include/rocksdb/db.h
  15. 33
      include/rocksdb/env.h
  16. 9
      include/rocksdb/iterator.h
  17. 7
      include/rocksdb/utilities/transaction.h
  18. 6
      include/rocksdb/utilities/transaction_db.h
  19. 8
      include/rocksdb/write_buffer_manager.h
  20. 8
      memory/allocator.h
  21. 7
      memtable/inlineskiplist.h
  22. 7
      memtable/skiplist.h
  23. 14
      monitoring/in_memory_stats_history.h
  24. 16
      port/port_posix.h
  25. 6
      port/win/port_win.h
  26. 7
      table/block_based/block.h
  27. 10
      table/block_based/block_based_filter_block.h
  28. 6
      table/block_based/block_based_table_builder.h
  29. 7
      table/block_based/block_based_table_reader.h
  30. 9
      table/block_based/filter_block.h
  31. 8
      table/block_based/full_filter_block.h
  32. 7
      table/cuckoo/cuckoo_table_builder.h
  33. 6
      table/cuckoo/cuckoo_table_reader.cc
  34. 8
      table/full_filter_bits_builder.h
  35. 10
      table/internal_iterator.h
  36. 7
      table/plain/plain_table_builder.h
  37. 7
      table/plain/plain_table_reader.cc
  38. 7
      util/bloom.cc
  39. 9
      util/concurrent_task_limiter_impl.h
  40. 28
      util/mutexlock.h
  41. 5
      utilities/blob_db/blob_log_reader.h
  42. 5
      utilities/blob_db/blob_log_writer.h
  43. 9
      utilities/transactions/optimistic_transaction.h
  44. 14
      utilities/transactions/pessimistic_transaction.h
  45. 7
      utilities/transactions/transaction_lock_mgr.h
  46. 7
      utilities/transactions/write_prepared_txn.h

@ -14,6 +14,10 @@ namespace rocksdb {
class CompactedDBImpl : public DBImpl { class CompactedDBImpl : public DBImpl {
public: public:
CompactedDBImpl(const DBOptions& options, const std::string& dbname); CompactedDBImpl(const DBOptions& options, const std::string& dbname);
// No copying allowed
CompactedDBImpl(const CompactedDBImpl&) = delete;
void operator=(const CompactedDBImpl&) = delete;
virtual ~CompactedDBImpl(); virtual ~CompactedDBImpl();
static Status Open(const Options& options, const std::string& dbname, static Status Open(const Options& options, const std::string& dbname,
@ -104,10 +108,6 @@ class CompactedDBImpl : public DBImpl {
Version* version_; Version* version_;
const Comparator* user_comparator_; const Comparator* user_comparator_;
LevelFilesBrief files_; LevelFilesBrief files_;
// No copying allowed
CompactedDBImpl(const CompactedDBImpl&);
void operator=(const CompactedDBImpl&);
}; };
} }
#endif // ROCKSDB_LITE #endif // ROCKSDB_LITE

@ -128,6 +128,10 @@ class DBImpl : public DB {
public: public:
DBImpl(const DBOptions& options, const std::string& dbname, DBImpl(const DBOptions& options, const std::string& dbname,
const bool seq_per_batch = false, const bool batch_per_txn = true); 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(); virtual ~DBImpl();
// ---- Implementations of the DB interface ---- // ---- Implementations of the DB interface ----
@ -1563,10 +1567,6 @@ class DBImpl : public DB {
void WaitForBackgroundWork(); void WaitForBackgroundWork();
// No copying allowed
DBImpl(const DBImpl&);
void operator=(const DBImpl&);
// Background threads call this function, which is just a wrapper around // Background threads call this function, which is just a wrapper around
// the InstallSuperVersion() function. Background threads carry // the InstallSuperVersion() function. Background threads carry
// sv_context which can have new_superversion already // sv_context which can have new_superversion already

@ -16,6 +16,10 @@ namespace rocksdb {
class DBImplReadOnly : public DBImpl { class DBImplReadOnly : public DBImpl {
public: public:
DBImplReadOnly(const DBOptions& options, const std::string& dbname); DBImplReadOnly(const DBOptions& options, const std::string& dbname);
// No copying allowed
DBImplReadOnly(const DBImplReadOnly&) = delete;
void operator=(const DBImplReadOnly&) = delete;
virtual ~DBImplReadOnly(); virtual ~DBImplReadOnly();
// Implementations of the DB interface // Implementations of the DB interface
@ -127,10 +131,6 @@ class DBImplReadOnly : public DBImpl {
private: private:
friend class DB; friend class DB;
// No copying allowed
DBImplReadOnly(const DBImplReadOnly&);
void operator=(const DBImplReadOnly&);
}; };
} // namespace rocksdb } // namespace rocksdb

@ -155,6 +155,10 @@ class DBIter final: public Iterator {
iter_.iter()->SetPinnedItersMgr(&pinned_iters_mgr_); iter_.iter()->SetPinnedItersMgr(&pinned_iters_mgr_);
} }
} }
// No copying allowed
DBIter(const DBIter&) = delete;
void operator=(const DBIter&) = delete;
~DBIter() override { ~DBIter() override {
// Release pinned data if any // Release pinned data if any
if (pinned_iters_mgr_.PinningEnabled()) { if (pinned_iters_mgr_.PinningEnabled()) {
@ -345,15 +349,17 @@ class DBIter final: public Iterator {
ReadRangeDelAggregator range_del_agg_; ReadRangeDelAggregator range_del_agg_;
LocalStatistics local_stats_; LocalStatistics local_stats_;
PinnedIteratorsManager pinned_iters_mgr_; PinnedIteratorsManager pinned_iters_mgr_;
#ifdef ROCKSDB_LITE
ROCKSDB_FIELD_UNUSED
#endif
DBImpl* db_impl_; DBImpl* db_impl_;
#ifdef ROCKSDB_LITE
ROCKSDB_FIELD_UNUSED
#endif
ColumnFamilyData* cfd_; ColumnFamilyData* cfd_;
// for diff snapshots we want the lower bound on the seqnum; // for diff snapshots we want the lower bound on the seqnum;
// if this value > 0 iterator will return internal keys // if this value > 0 iterator will return internal keys
SequenceNumber start_seqnum_; SequenceNumber start_seqnum_;
// No copying allowed
DBIter(const DBIter&);
void operator=(const DBIter&);
}; };
inline bool DBIter::ParseKey(ParsedInternalKey* ikey) { inline bool DBIter::ParseKey(ParsedInternalKey* ikey) {

@ -326,6 +326,9 @@ class IterKey {
key_size_(0), key_size_(0),
buf_size_(sizeof(space_)), buf_size_(sizeof(space_)),
is_user_key_(true) {} is_user_key_(true) {}
// No copying allowed
IterKey(const IterKey&) = delete;
void operator=(const IterKey&) = delete;
~IterKey() { ResetBuffer(); } ~IterKey() { ResetBuffer(); }
@ -523,10 +526,6 @@ class IterKey {
} }
void EnlargeBuffer(size_t key_size); 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 // Convert from a SliceTranform of user keys, to a SliceTransform of

@ -53,6 +53,9 @@ class Reader {
// @lint-ignore TXT2 T25377293 Grandfathered in // @lint-ignore TXT2 T25377293 Grandfathered in
std::unique_ptr<SequentialFileReader>&& file, Reporter* reporter, std::unique_ptr<SequentialFileReader>&& file, Reporter* reporter,
bool checksum, uint64_t log_num); bool checksum, uint64_t log_num);
// No copying allowed
Reader(const Reader&) = delete;
void operator=(const Reader&) = delete;
virtual ~Reader(); virtual ~Reader();
@ -148,11 +151,6 @@ class Reader {
// buffer_ must be updated to remove the dropped bytes prior to invocation. // buffer_ must be updated to remove the dropped bytes prior to invocation.
void ReportCorruption(size_t bytes, const char* reason); void ReportCorruption(size_t bytes, const char* reason);
void ReportDrop(size_t bytes, const Status& reason); void ReportDrop(size_t bytes, const Status& reason);
private:
// No copying allowed
Reader(const Reader&);
void operator=(const Reader&);
}; };
class FragmentBufferedReader : public Reader { class FragmentBufferedReader : public Reader {

@ -73,6 +73,10 @@ class Writer {
explicit Writer(std::unique_ptr<WritableFileWriter>&& dest, explicit Writer(std::unique_ptr<WritableFileWriter>&& dest,
uint64_t log_number, bool recycle_log_files, uint64_t log_number, bool recycle_log_files,
bool manual_flush = false); bool manual_flush = false);
// No copying allowed
Writer(const Writer&) = delete;
void operator=(const Writer&) = delete;
~Writer(); ~Writer();
Status AddRecord(const Slice& slice); 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 // If true, it does not flush after each write. Instead it relies on the upper
// layer to manually does the flush by calling ::WriteBuffer() // layer to manually does the flush by calling ::WriteBuffer()
bool manual_flush_; bool manual_flush_;
// No copying allowed
Writer(const Writer&);
void operator=(const Writer&);
}; };
} // namespace log } // namespace log

@ -295,6 +295,9 @@ class MemTableIterator : public InternalIterator {
iter_ = mem.table_->GetIterator(arena); iter_ = mem.table_->GetIterator(arena);
} }
} }
// No copying allowed
MemTableIterator(const MemTableIterator&) = delete;
void operator=(const MemTableIterator&) = delete;
~MemTableIterator() override { ~MemTableIterator() override {
#ifndef NDEBUG #ifndef NDEBUG
@ -408,10 +411,6 @@ class MemTableIterator : public InternalIterator {
bool valid_; bool valid_;
bool arena_mode_; bool arena_mode_;
bool value_pinned_; bool value_pinned_;
// No copying allowed
MemTableIterator(const MemTableIterator&);
void operator=(const MemTableIterator&);
}; };
InternalIterator* MemTable::NewIterator(const ReadOptions& read_options, InternalIterator* MemTable::NewIterator(const ReadOptions& read_options,

@ -101,6 +101,9 @@ class MemTable {
const MutableCFOptions& mutable_cf_options, const MutableCFOptions& mutable_cf_options,
WriteBufferManager* write_buffer_manager, WriteBufferManager* write_buffer_manager,
SequenceNumber earliest_seq, uint32_t column_family_id); 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. // Do not delete this MemTable unless Unref() indicates it not in use.
~MemTable(); ~MemTable();
@ -503,10 +506,6 @@ class MemTable {
void UpdateFlushState(); void UpdateFlushState();
void UpdateOldestKeyTime(); void UpdateOldestKeyTime();
// No copying allowed
MemTable(const MemTable&);
MemTable& operator=(const MemTable&);
}; };
extern const char* EncodeKey(std::string* scratch, const Slice& target); extern const char* EncodeKey(std::string* scratch, const Slice& target);

@ -109,6 +109,9 @@ class VersionStorageInfo {
CompactionStyle compaction_style, CompactionStyle compaction_style,
VersionStorageInfo* src_vstorage, VersionStorageInfo* src_vstorage,
bool _force_consistency_checks); bool _force_consistency_checks);
// No copying allowed
VersionStorageInfo(const VersionStorageInfo&) = delete;
void operator=(const VersionStorageInfo&) = delete;
~VersionStorageInfo(); ~VersionStorageInfo();
void Reserve(int level, size_t size) { files_[level].reserve(size); } void Reserve(int level, size_t size) { files_[level].reserve(size); }
@ -542,9 +545,6 @@ class VersionStorageInfo {
friend class Version; friend class Version;
friend class VersionSet; friend class VersionSet;
// No copying allowed
VersionStorageInfo(const VersionStorageInfo&) = delete;
void operator=(const VersionStorageInfo&) = delete;
}; };
using MultiGetRange = MultiGetContext::Range; using MultiGetRange = MultiGetContext::Range;
@ -734,8 +734,8 @@ class Version {
~Version(); ~Version();
// No copying allowed // No copying allowed
Version(const Version&); Version(const Version&) = delete;
void operator=(const Version&); void operator=(const Version&) = delete;
}; };
struct ObsoleteFileInfo { struct ObsoleteFileInfo {
@ -797,6 +797,10 @@ class VersionSet {
WriteBufferManager* write_buffer_manager, WriteBufferManager* write_buffer_manager,
WriteController* write_controller, WriteController* write_controller,
BlockCacheTracer* const block_cache_tracer); BlockCacheTracer* const block_cache_tracer);
// No copying allowed
VersionSet(const VersionSet&) = delete;
void operator=(const VersionSet&) = delete;
virtual ~VersionSet(); virtual ~VersionSet();
// Apply *edit to the current version to form a new descriptor that // Apply *edit to the current version to form a new descriptor that
@ -1143,10 +1147,6 @@ class VersionSet {
BlockCacheTracer* const block_cache_tracer_; BlockCacheTracer* const block_cache_tracer_;
private: private:
// No copying allowed
VersionSet(const VersionSet&);
void operator=(const VersionSet&);
// REQUIRES db mutex at beginning. may release and re-acquire db mutex // REQUIRES db mutex at beginning. may release and re-acquire db mutex
Status ProcessManifestWrites(std::deque<ManifestWriter>& writers, Status ProcessManifestWrites(std::deque<ManifestWriter>& writers,
InstrumentedMutex* mu, Directory* db_directory, InstrumentedMutex* mu, Directory* db_directory,

7
env/mock_env.cc vendored

@ -31,6 +31,9 @@ class MemFile {
rnd_(static_cast<uint32_t>( rnd_(static_cast<uint32_t>(
MurmurHash(fn.data(), static_cast<int>(fn.size()), 0))), MurmurHash(fn.data(), static_cast<int>(fn.size()), 0))),
fsynced_bytes_(0) {} fsynced_bytes_(0) {}
// No copying allowed.
MemFile(const MemFile&) = delete;
void operator=(const MemFile&) = delete;
void Ref() { void Ref() {
MutexLock lock(&mutex_); MutexLock lock(&mutex_);
@ -154,10 +157,6 @@ class MemFile {
// Private since only Unref() should be used to delete it. // Private since only Unref() should be used to delete it.
~MemFile() { assert(refs_ == 0); } ~MemFile() { assert(refs_ == 0); }
// No copying allowed.
MemFile(const MemFile&);
void operator=(const MemFile&);
Env* env_; Env* env_;
const std::string fn_; const std::string fn_;
mutable port::Mutex mutex_; mutable port::Mutex mutex_;

@ -122,6 +122,9 @@ class Cache {
Cache(std::shared_ptr<MemoryAllocator> allocator = nullptr) Cache(std::shared_ptr<MemoryAllocator> allocator = nullptr)
: memory_allocator_(std::move(allocator)) {} : 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" // Destroys all existing entries by calling the "deleter"
// function that was passed via the Insert() function. // function that was passed via the Insert() function.
@ -253,10 +256,6 @@ class Cache {
MemoryAllocator* memory_allocator() const { return memory_allocator_.get(); } MemoryAllocator* memory_allocator() const { return memory_allocator_.get(); }
private: private:
// No copying allowed
Cache(const Cache&);
Cache& operator=(const Cache&);
std::shared_ptr<MemoryAllocator> memory_allocator_; std::shared_ptr<MemoryAllocator> memory_allocator_;
}; };

@ -23,12 +23,12 @@ namespace rocksdb {
class Cleanable { class Cleanable {
public: public:
Cleanable(); Cleanable();
~Cleanable();
// No copy constructor and copy assignment allowed. // No copy constructor and copy assignment allowed.
Cleanable(Cleanable&) = delete; Cleanable(Cleanable&) = delete;
Cleanable& operator=(Cleanable&) = delete; Cleanable& operator=(Cleanable&) = delete;
~Cleanable();
// Move constructor and move assignment is allowed. // Move constructor and move assignment is allowed.
Cleanable(Cleanable&&); Cleanable(Cleanable&&);
Cleanable& operator=(Cleanable&&); Cleanable& operator=(Cleanable&&);

@ -255,6 +255,10 @@ class DB {
std::vector<std::string>* column_families); std::vector<std::string>* column_families);
DB() {} DB() {}
// No copying allowed
DB(const DB&) = delete;
void operator=(const DB&) = delete;
virtual ~DB(); virtual ~DB();
// Create a column_family and return the handle of column family // 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"); return Status::NotSupported("Supported only by secondary instance");
} }
#endif // !ROCKSDB_LITE #endif // !ROCKSDB_LITE
private:
// No copying allowed
DB(const DB&);
void operator=(const DB&);
}; };
// Destroy the contents of the specified database. // Destroy the contents of the specified database.

@ -141,6 +141,9 @@ class Env {
}; };
Env() : thread_status_updater_(nullptr) {} Env() : thread_status_updater_(nullptr) {}
// No copying allowed
Env(const Env&) = delete;
void operator=(const Env&) = delete;
virtual ~Env(); virtual ~Env();
@ -527,11 +530,6 @@ class Env {
// The pointer to an internal structure that will update the // The pointer to an internal structure that will update the
// status of each thread. // status of each thread.
ThreadStatusUpdater* thread_status_updater_; ThreadStatusUpdater* thread_status_updater_;
private:
// No copying allowed
Env(const Env&);
void operator=(const Env&);
}; };
// The factory function to construct a ThreadStatusUpdater. Any Env // The factory function to construct a ThreadStatusUpdater. Any Env
@ -711,6 +709,9 @@ class WritableFile {
io_priority_(Env::IO_TOTAL), io_priority_(Env::IO_TOTAL),
write_hint_(Env::WLTH_NOT_SET), write_hint_(Env::WLTH_NOT_SET),
strict_bytes_per_sync_(options.strict_bytes_per_sync) {} strict_bytes_per_sync_(options.strict_bytes_per_sync) {}
// No copying allowed
WritableFile(const WritableFile&) = delete;
void operator=(const WritableFile&) = delete;
virtual ~WritableFile(); virtual ~WritableFile();
@ -870,9 +871,6 @@ class WritableFile {
private: private:
size_t last_preallocated_block_; size_t last_preallocated_block_;
size_t preallocation_block_size_; size_t preallocation_block_size_;
// No copying allowed
WritableFile(const WritableFile&);
void operator=(const WritableFile&);
protected: protected:
Env::IOPriority io_priority_; Env::IOPriority io_priority_;
@ -884,6 +882,10 @@ class WritableFile {
class RandomRWFile { class RandomRWFile {
public: public:
RandomRWFile() {} RandomRWFile() {}
// No copying allowed
RandomRWFile(const RandomRWFile&) = delete;
RandomRWFile& operator=(const RandomRWFile&) = delete;
virtual ~RandomRWFile() {} virtual ~RandomRWFile() {}
// Indicates if the class makes use of direct I/O // 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 // If you're adding methods here, remember to add them to
// RandomRWFileWrapper too. // RandomRWFileWrapper too.
// No copying allowed
RandomRWFile(const RandomRWFile&) = delete;
RandomRWFile& operator=(const RandomRWFile&) = delete;
}; };
// MemoryMappedFileBuffer object represents a memory-mapped file's raw buffer. // 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) explicit Logger(const InfoLogLevel log_level = InfoLogLevel::INFO_LEVEL)
: closed_(false), log_level_(log_level) {} : closed_(false), log_level_(log_level) {}
// No copying allowed
Logger(const Logger&) = delete;
void operator=(const Logger&) = delete;
virtual ~Logger(); virtual ~Logger();
// Close the log file. Must be called before destructor. If the return // Close the log file. Must be called before destructor. If the return
@ -1016,9 +1018,6 @@ class Logger {
bool closed_; bool closed_;
private: private:
// No copying allowed
Logger(const Logger&);
void operator=(const Logger&);
InfoLogLevel log_level_; InfoLogLevel log_level_;
}; };
@ -1030,8 +1029,8 @@ class FileLock {
private: private:
// No copying allowed // No copying allowed
FileLock(const FileLock&); FileLock(const FileLock&) = delete;
void operator=(const FileLock&); void operator=(const FileLock&) = delete;
}; };
class DynamicLibrary { class DynamicLibrary {

@ -28,6 +28,10 @@ namespace rocksdb {
class Iterator : public Cleanable { class Iterator : public Cleanable {
public: public:
Iterator() {} Iterator() {}
// No copying allowed
Iterator(const Iterator&) = delete;
void operator=(const Iterator&) = delete;
virtual ~Iterator() {} virtual ~Iterator() {}
// An iterator is either positioned at a key/value pair, or // 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 // Get the user-key portion of the internal key at which the iteration
// stopped. // stopped.
virtual Status GetProperty(std::string prop_name, std::string* prop); 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). // Return an empty iterator (yields nothing).

@ -52,6 +52,10 @@ class TransactionNotifier {
// -Support for using Transactions with DBWithTTL // -Support for using Transactions with DBWithTTL
class Transaction { class Transaction {
public: public:
// No copying allowed
Transaction(const Transaction&) = delete;
void operator=(const Transaction&) = delete;
virtual ~Transaction() {} virtual ~Transaction() {}
// If a transaction has a snapshot set, the transaction will ensure that // If a transaction has a snapshot set, the transaction will ensure that
@ -529,9 +533,6 @@ class Transaction {
friend class WriteUnpreparedTxnDB; friend class WriteUnpreparedTxnDB;
friend class TransactionTest_TwoPhaseLogRollingTest_Test; friend class TransactionTest_TwoPhaseLogRollingTest_Test;
friend class TransactionTest_TwoPhaseLogRollingTest2_Test; friend class TransactionTest_TwoPhaseLogRollingTest2_Test;
// No copying allowed
Transaction(const Transaction&);
void operator=(const Transaction&);
}; };
} // namespace rocksdb } // namespace rocksdb

@ -301,11 +301,9 @@ class TransactionDB : public StackableDB {
// To Create an TransactionDB, call Open() // To Create an TransactionDB, call Open()
// The ownership of db is transferred to the base StackableDB // The ownership of db is transferred to the base StackableDB
explicit TransactionDB(DB* db) : StackableDB(db) {} explicit TransactionDB(DB* db) : StackableDB(db) {}
private:
// No copying allowed // No copying allowed
TransactionDB(const TransactionDB&); TransactionDB(const TransactionDB&) = delete;
void operator=(const TransactionDB&); void operator=(const TransactionDB&) = delete;
}; };
} // namespace rocksdb } // namespace rocksdb

@ -26,6 +26,10 @@ class WriteBufferManager {
// the memory allocated to the cache. It can be used even if _buffer_size = 0. // the memory allocated to the cache. It can be used even if _buffer_size = 0.
explicit WriteBufferManager(size_t _buffer_size, explicit WriteBufferManager(size_t _buffer_size,
std::shared_ptr<Cache> cache = {}); std::shared_ptr<Cache> cache = {});
// No copying allowed
WriteBufferManager(const WriteBufferManager&) = delete;
WriteBufferManager& operator=(const WriteBufferManager&) = delete;
~WriteBufferManager(); ~WriteBufferManager();
bool enabled() const { return buffer_size_ != 0; } bool enabled() const { return buffer_size_ != 0; }
@ -94,9 +98,5 @@ class WriteBufferManager {
void ReserveMemWithCache(size_t mem); void ReserveMemWithCache(size_t mem);
void FreeMemWithCache(size_t mem); void FreeMemWithCache(size_t mem);
// No copying allowed
WriteBufferManager(const WriteBufferManager&) = delete;
WriteBufferManager& operator=(const WriteBufferManager&) = delete;
}; };
} // namespace rocksdb } // namespace rocksdb

@ -33,6 +33,10 @@ class Allocator {
class AllocTracker { class AllocTracker {
public: public:
explicit AllocTracker(WriteBufferManager* write_buffer_manager); explicit AllocTracker(WriteBufferManager* write_buffer_manager);
// No copying allowed
AllocTracker(const AllocTracker&) = delete;
void operator=(const AllocTracker&) = delete;
~AllocTracker(); ~AllocTracker();
void Allocate(size_t bytes); void Allocate(size_t bytes);
// Call when we're finished allocating memory so we can free it from // Call when we're finished allocating memory so we can free it from
@ -48,10 +52,6 @@ class AllocTracker {
std::atomic<size_t> bytes_allocated_; std::atomic<size_t> bytes_allocated_;
bool done_allocating_; bool done_allocating_;
bool freed_; bool freed_;
// No copying allowed
AllocTracker(const AllocTracker&);
void operator=(const AllocTracker&);
}; };
} // namespace rocksdb } // namespace rocksdb

@ -74,6 +74,9 @@ class InlineSkipList {
explicit InlineSkipList(Comparator cmp, Allocator* allocator, explicit InlineSkipList(Comparator cmp, Allocator* allocator,
int32_t max_height = 12, int32_t max_height = 12,
int32_t branching_factor = 4); 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 // 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 // portion of the node. This method is thread-safe if the allocator
@ -254,10 +257,6 @@ class InlineSkipList {
// lowest_level (inclusive). // lowest_level (inclusive).
void RecomputeSpliceLevels(const DecodedKey& key, Splice* splice, void RecomputeSpliceLevels(const DecodedKey& key, Splice* splice,
int recompute_level); int recompute_level);
// No copying allowed
InlineSkipList(const InlineSkipList&);
InlineSkipList& operator=(const InlineSkipList&);
}; };
// Implementation details follow // Implementation details follow

@ -51,6 +51,9 @@ class SkipList {
// allocator must remain allocated for the lifetime of the skiplist object. // allocator must remain allocated for the lifetime of the skiplist object.
explicit SkipList(Comparator cmp, Allocator* allocator, explicit SkipList(Comparator cmp, Allocator* allocator,
int32_t max_height = 12, int32_t branching_factor = 4); 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. // Insert key into the list.
// REQUIRES: nothing that compares equal to key is currently in 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 the last node in the list.
// Return head_ if list is empty. // Return head_ if list is empty.
Node* FindLast() const; Node* FindLast() const;
// No copying allowed
SkipList(const SkipList&);
void operator=(const SkipList&);
}; };
// Implementation details follow // Implementation details follow

@ -34,6 +34,13 @@ class InMemoryStatsHistoryIterator final : public StatsHistoryIterator {
db_impl_(db_impl) { db_impl_(db_impl) {
AdvanceIteratorByTime(start_time_, end_time_); 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; ~InMemoryStatsHistoryIterator() override;
bool Valid() const override; bool Valid() const override;
Status status() const override; Status status() const override;
@ -55,13 +62,6 @@ class InMemoryStatsHistoryIterator final : public StatsHistoryIterator {
// between [start_time, end_time) // between [start_time, end_time)
void AdvanceIteratorByTime(uint64_t start_time, uint64_t 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 time_;
uint64_t start_time_; uint64_t start_time_;
uint64_t end_time_; uint64_t end_time_;

@ -104,6 +104,10 @@ class CondVar;
class Mutex { class Mutex {
public: public:
explicit Mutex(bool adaptive = kDefaultToAdaptiveMutex); explicit Mutex(bool adaptive = kDefaultToAdaptiveMutex);
// No copying
Mutex(const Mutex&) = delete;
void operator=(const Mutex&) = delete;
~Mutex(); ~Mutex();
void Lock(); void Lock();
@ -118,15 +122,15 @@ class Mutex {
#ifndef NDEBUG #ifndef NDEBUG
bool locked_; bool locked_;
#endif #endif
// No copying
Mutex(const Mutex&);
void operator=(const Mutex&);
}; };
class RWMutex { class RWMutex {
public: public:
RWMutex(); RWMutex();
// No copying allowed
RWMutex(const RWMutex&) = delete;
void operator=(const RWMutex&) = delete;
~RWMutex(); ~RWMutex();
void ReadLock(); void ReadLock();
@ -137,10 +141,6 @@ class RWMutex {
private: private:
pthread_rwlock_t mu_; // the underlying platform mutex pthread_rwlock_t mu_; // the underlying platform mutex
// No copying allowed
RWMutex(const RWMutex&);
void operator=(const RWMutex&);
}; };
class CondVar { class CondVar {

@ -180,6 +180,9 @@ class Mutex {
class RWMutex { class RWMutex {
public: public:
RWMutex() { InitializeSRWLock(&srwLock_); } RWMutex() { InitializeSRWLock(&srwLock_); }
// No copying allowed
RWMutex(const RWMutex&) = delete;
void operator=(const RWMutex&) = delete;
void ReadLock() { AcquireSRWLockShared(&srwLock_); } void ReadLock() { AcquireSRWLockShared(&srwLock_); }
@ -194,9 +197,6 @@ class RWMutex {
private: private:
SRWLOCK srwLock_; SRWLOCK srwLock_;
// No copying allowed
RWMutex(const RWMutex&);
void operator=(const RWMutex&);
}; };
class CondVar { class CondVar {

@ -147,6 +147,9 @@ class Block {
explicit Block(BlockContents&& contents, SequenceNumber _global_seqno, explicit Block(BlockContents&& contents, SequenceNumber _global_seqno,
size_t read_amp_bytes_per_bit = 0, size_t read_amp_bytes_per_bit = 0,
Statistics* statistics = nullptr); Statistics* statistics = nullptr);
// No copying allowed
Block(const Block&) = delete;
void operator=(const Block&) = delete;
~Block(); ~Block();
@ -222,10 +225,6 @@ class Block {
const SequenceNumber global_seqno_; const SequenceNumber global_seqno_;
DataBlockHashIndex data_block_hash_index_; DataBlockHashIndex data_block_hash_index_;
// No copying allowed
Block(const Block&) = delete;
void operator=(const Block&) = delete;
}; };
template <class TValue> template <class TValue>

@ -38,6 +38,9 @@ class BlockBasedFilterBlockBuilder : public FilterBlockBuilder {
public: public:
BlockBasedFilterBlockBuilder(const SliceTransform* prefix_extractor, BlockBasedFilterBlockBuilder(const SliceTransform* prefix_extractor,
const BlockBasedTableOptions& table_opt); const BlockBasedTableOptions& table_opt);
// No copying allowed
BlockBasedFilterBlockBuilder(const BlockBasedFilterBlockBuilder&) = delete;
void operator=(const BlockBasedFilterBlockBuilder&) = delete;
virtual bool IsBlockBased() override { return true; } virtual bool IsBlockBased() override { return true; }
virtual void StartBlock(uint64_t block_offset) override; virtual void StartBlock(uint64_t block_offset) override;
@ -68,10 +71,6 @@ class BlockBasedFilterBlockBuilder : public FilterBlockBuilder {
std::vector<Slice> tmp_entries_; // policy_->CreateFilter() argument std::vector<Slice> tmp_entries_; // policy_->CreateFilter() argument
std::vector<uint32_t> filter_offsets_; std::vector<uint32_t> filter_offsets_;
size_t num_added_; // Number of keys added 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. // A FilterBlockReader is used to parse filter from SST table.
@ -81,6 +80,9 @@ class BlockBasedFilterBlockReader
public: public:
BlockBasedFilterBlockReader(const BlockBasedTable* t, BlockBasedFilterBlockReader(const BlockBasedTable* t,
CachableEntry<BlockContents>&& filter_block); CachableEntry<BlockContents>&& filter_block);
// No copying allowed
BlockBasedFilterBlockReader(const BlockBasedFilterBlockReader&) = delete;
void operator=(const BlockBasedFilterBlockReader&) = delete;
static std::unique_ptr<FilterBlockReader> Create( static std::unique_ptr<FilterBlockReader> Create(
const BlockBasedTable* table, FilePrefetchBuffer* prefetch_buffer, const BlockBasedTable* table, FilePrefetchBuffer* prefetch_buffer,

@ -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 oldest_key_time = 0, const uint64_t target_file_size = 0,
const uint64_t file_creation_time = 0); const uint64_t file_creation_time = 0);
// REQUIRES: Either Finish() or Abandon() has been called.
~BlockBasedTableBuilder();
// No copying allowed // No copying allowed
BlockBasedTableBuilder(const BlockBasedTableBuilder&) = delete; BlockBasedTableBuilder(const BlockBasedTableBuilder&) = delete;
BlockBasedTableBuilder& operator=(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. // Add key,value to the table being constructed.
// REQUIRES: key is after any previously added key according to comparator. // REQUIRES: key is after any previously added key according to comparator.
// REQUIRES: Finish(), Abandon() have not been called // REQUIRES: Finish(), Abandon() have not been called

@ -265,6 +265,9 @@ class BlockBasedTable : public TableReader {
Rep* rep_; Rep* rep_;
explicit BlockBasedTable(Rep* rep, BlockCacheTracer* const block_cache_tracer) explicit BlockBasedTable(Rep* rep, BlockCacheTracer* const block_cache_tracer)
: rep_(rep), block_cache_tracer_(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: private:
friend class MockedBlockBasedTable; friend class MockedBlockBasedTable;
@ -458,10 +461,6 @@ class BlockBasedTable : public TableReader {
void DumpKeyValue(const Slice& key, const Slice& value, void DumpKeyValue(const Slice& key, const Slice& value,
WritableFile* out_file); WritableFile* out_file);
// No copying allowed
explicit BlockBasedTable(const TableReader&) = delete;
void operator=(const TableReader&) = delete;
friend class PartitionedFilterBlockReader; friend class PartitionedFilterBlockReader;
friend class PartitionedFilterBlockTest; friend class PartitionedFilterBlockTest;
}; };

@ -52,6 +52,10 @@ using MultiGetRange = MultiGetContext::Range;
class FilterBlockBuilder { class FilterBlockBuilder {
public: public:
explicit FilterBlockBuilder() {} explicit FilterBlockBuilder() {}
// No copying allowed
FilterBlockBuilder(const FilterBlockBuilder&) = delete;
void operator=(const FilterBlockBuilder&) = delete;
virtual ~FilterBlockBuilder() {} virtual ~FilterBlockBuilder() {}
virtual bool IsBlockBased() = 0; // If is blockbased filter virtual bool IsBlockBased() = 0; // If is blockbased filter
@ -66,11 +70,6 @@ class FilterBlockBuilder {
return ret; return ret;
} }
virtual Slice Finish(const BlockHandle& tmp, Status* status) = 0; 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. // A FilterBlockReader is used to parse filter from SST table.

@ -40,6 +40,10 @@ class FullFilterBlockBuilder : public FilterBlockBuilder {
explicit FullFilterBlockBuilder(const SliceTransform* prefix_extractor, explicit FullFilterBlockBuilder(const SliceTransform* prefix_extractor,
bool whole_key_filtering, bool whole_key_filtering,
FilterBitsBuilder* filter_bits_builder); 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 // bits_builder is created in filter_policy, it should be passed in here
// directly. and be deleted here // directly. and be deleted here
~FullFilterBlockBuilder() {} ~FullFilterBlockBuilder() {}
@ -71,10 +75,6 @@ class FullFilterBlockBuilder : public FilterBlockBuilder {
std::unique_ptr<const char[]> filter_data_; std::unique_ptr<const char[]> filter_data_;
void AddPrefix(const Slice& key); 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. // A FilterBlockReader is used to parse filter from SST table.

@ -30,6 +30,9 @@ class CuckooTableBuilder: public TableBuilder {
uint64_t), uint64_t),
uint32_t column_family_id, uint32_t column_family_id,
const std::string& column_family_name); 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. // REQUIRES: Either Finish() or Abandon() has been called.
~CuckooTableBuilder() {} ~CuckooTableBuilder() {}
@ -116,10 +119,6 @@ class CuckooTableBuilder: public TableBuilder {
std::string smallest_user_key_ = ""; std::string smallest_user_key_ = "";
bool closed_; // Either Finish() or Abandon() has been called. bool closed_; // Either Finish() or Abandon() has been called.
// No copying allowed
CuckooTableBuilder(const CuckooTableBuilder&) = delete;
void operator=(const CuckooTableBuilder&) = delete;
}; };
} // namespace rocksdb } // namespace rocksdb

@ -197,6 +197,9 @@ void CuckooTableReader::Prepare(const Slice& key) {
class CuckooTableIterator : public InternalIterator { class CuckooTableIterator : public InternalIterator {
public: public:
explicit CuckooTableIterator(CuckooTableReader* reader); explicit CuckooTableIterator(CuckooTableReader* reader);
// No copying allowed
CuckooTableIterator(const CuckooTableIterator&) = delete;
void operator=(const Iterator&) = delete;
~CuckooTableIterator() override {} ~CuckooTableIterator() override {}
bool Valid() const override; bool Valid() const override;
void SeekToFirst() override; void SeekToFirst() override;
@ -248,9 +251,6 @@ class CuckooTableIterator : public InternalIterator {
uint32_t curr_key_idx_; uint32_t curr_key_idx_;
Slice curr_value_; Slice curr_value_;
IterKey curr_key_; IterKey curr_key_;
// No copying allowed
CuckooTableIterator(const CuckooTableIterator&) = delete;
void operator=(const Iterator&) = delete;
}; };
CuckooTableIterator::CuckooTableIterator(CuckooTableReader* reader) CuckooTableIterator::CuckooTableIterator(CuckooTableReader* reader)

@ -23,6 +23,10 @@ class FullFilterBitsBuilder : public FilterBitsBuilder {
explicit FullFilterBitsBuilder(const size_t bits_per_key, explicit FullFilterBitsBuilder(const size_t bits_per_key,
const size_t num_probes); const size_t num_probes);
// No Copy allowed
FullFilterBitsBuilder(const FullFilterBitsBuilder&) = delete;
void operator=(const FullFilterBitsBuilder&) = delete;
~FullFilterBitsBuilder(); ~FullFilterBitsBuilder();
virtual void AddKey(const Slice& key) override; virtual void AddKey(const Slice& key) override;
@ -65,10 +69,6 @@ class FullFilterBitsBuilder : public FilterBitsBuilder {
// Assuming single threaded access to this function. // Assuming single threaded access to this function.
void AddHash(uint32_t h, char* data, uint32_t num_lines, uint32_t total_bits); 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 } // namespace rocksdb

@ -27,6 +27,10 @@ class InternalIteratorBase : public Cleanable {
public: public:
InternalIteratorBase() : is_mutable_(true) {} InternalIteratorBase() : is_mutable_(true) {}
InternalIteratorBase(bool _is_mutable) : is_mutable_(_is_mutable) {} InternalIteratorBase(bool _is_mutable) : is_mutable_(_is_mutable) {}
// No copying allowed
InternalIteratorBase(const InternalIteratorBase&) = delete;
InternalIteratorBase& operator=(const InternalIteratorBase&) = delete;
virtual ~InternalIteratorBase() {} virtual ~InternalIteratorBase() {}
// An iterator is either positioned at a key/value pair, or // An iterator is either positioned at a key/value pair, or
@ -156,12 +160,8 @@ class InternalIteratorBase : public Cleanable {
Prev(); Prev();
} }
} }
bool is_mutable_;
private: bool is_mutable_;
// No copying allowed
InternalIteratorBase(const InternalIteratorBase&) = delete;
InternalIteratorBase& operator=(const InternalIteratorBase&) = delete;
}; };
using InternalIterator = InternalIteratorBase<Slice>; using InternalIterator = InternalIteratorBase<Slice>;

@ -45,6 +45,9 @@ class PlainTableBuilder: public TableBuilder {
const std::string& column_family_name, uint32_t num_probes = 6, const std::string& column_family_name, uint32_t num_probes = 6,
size_t huge_page_tlb_size = 0, double hash_table_ratio = 0, size_t huge_page_tlb_size = 0, double hash_table_ratio = 0,
bool store_index_in_file = false); 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. // REQUIRES: Either Finish() or Abandon() has been called.
~PlainTableBuilder(); ~PlainTableBuilder();
@ -131,10 +134,6 @@ class PlainTableBuilder: public TableBuilder {
} }
bool IsTotalOrderMode() const { return (prefix_extractor_ == nullptr); } bool IsTotalOrderMode() const { return (prefix_extractor_ == nullptr); }
// No copying allowed
PlainTableBuilder(const PlainTableBuilder&) = delete;
void operator=(const PlainTableBuilder&) = delete;
}; };
} // namespace rocksdb } // namespace rocksdb

@ -55,6 +55,10 @@ inline uint32_t GetFixed32Element(const char* base, size_t offset) {
class PlainTableIterator : public InternalIterator { class PlainTableIterator : public InternalIterator {
public: public:
explicit PlainTableIterator(PlainTableReader* table, bool use_prefix_seek); explicit PlainTableIterator(PlainTableReader* table, bool use_prefix_seek);
// No copying allowed
PlainTableIterator(const PlainTableIterator&) = delete;
void operator=(const Iterator&) = delete;
~PlainTableIterator() override; ~PlainTableIterator() override;
bool Valid() const override; bool Valid() const override;
@ -86,9 +90,6 @@ class PlainTableIterator : public InternalIterator {
Slice key_; Slice key_;
Slice value_; Slice value_;
Status status_; Status status_;
// No copying allowed
PlainTableIterator(const PlainTableIterator&) = delete;
void operator=(const Iterator&) = delete;
}; };
extern const uint64_t kPlainTableMagicNumber; extern const uint64_t kPlainTableMagicNumber;

@ -171,6 +171,9 @@ class FullFilterBitsReader : public FilterBitsReader {
} }
} }
} }
// No Copy allowed
FullFilterBitsReader(const FullFilterBitsReader&) = delete;
void operator=(const FullFilterBitsReader&) = delete;
~FullFilterBitsReader() override {} ~FullFilterBitsReader() override {}
@ -244,10 +247,6 @@ class FullFilterBitsReader : public FilterBitsReader {
void FilterPrepare(const uint32_t& hash, const Slice& filter, void FilterPrepare(const uint32_t& hash, const Slice& filter,
const uint32_t& num_lines, uint32_t* bit_offset); 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, void FullFilterBitsReader::GetFilterMeta(const Slice& filter,

@ -22,6 +22,10 @@ class ConcurrentTaskLimiterImpl : public ConcurrentTaskLimiter {
public: public:
explicit ConcurrentTaskLimiterImpl(const std::string& name, explicit ConcurrentTaskLimiterImpl(const std::string& name,
int32_t max_outstanding_task); int32_t max_outstanding_task);
// No copying allowed
ConcurrentTaskLimiterImpl(const ConcurrentTaskLimiterImpl&) = delete;
ConcurrentTaskLimiterImpl& operator=(
const ConcurrentTaskLimiterImpl&) = delete;
virtual ~ConcurrentTaskLimiterImpl(); virtual ~ConcurrentTaskLimiterImpl();
@ -44,11 +48,6 @@ class ConcurrentTaskLimiterImpl : public ConcurrentTaskLimiter {
std::string name_; std::string name_;
std::atomic<int32_t> max_outstanding_tasks_; std::atomic<int32_t> max_outstanding_tasks_;
std::atomic<int32_t> outstanding_tasks_; std::atomic<int32_t> outstanding_tasks_;
// No copying allowed
ConcurrentTaskLimiterImpl(const ConcurrentTaskLimiterImpl&) = delete;
ConcurrentTaskLimiterImpl& operator=(
const ConcurrentTaskLimiterImpl&) = delete;
}; };
class TaskLimiterToken { class TaskLimiterToken {

@ -31,13 +31,14 @@ class MutexLock {
explicit MutexLock(port::Mutex *mu) : mu_(mu) { explicit MutexLock(port::Mutex *mu) : mu_(mu) {
this->mu_->Lock(); this->mu_->Lock();
} }
// No copying allowed
MutexLock(const MutexLock&) = delete;
void operator=(const MutexLock&) = delete;
~MutexLock() { this->mu_->Unlock(); } ~MutexLock() { this->mu_->Unlock(); }
private: private:
port::Mutex *const mu_; 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) { explicit ReadLock(port::RWMutex *mu) : mu_(mu) {
this->mu_->ReadLock(); this->mu_->ReadLock();
} }
// No copying allowed
ReadLock(const ReadLock&) = delete;
void operator=(const ReadLock&) = delete;
~ReadLock() { this->mu_->ReadUnlock(); } ~ReadLock() { this->mu_->ReadUnlock(); }
private: private:
port::RWMutex *const mu_; port::RWMutex *const mu_;
// No copying allowed
ReadLock(const ReadLock&);
void operator=(const ReadLock&);
}; };
// //
@ -65,13 +67,14 @@ class ReadLock {
class ReadUnlock { class ReadUnlock {
public: public:
explicit ReadUnlock(port::RWMutex *mu) : mu_(mu) { mu->AssertHeld(); } explicit ReadUnlock(port::RWMutex *mu) : mu_(mu) { mu->AssertHeld(); }
// No copying allowed
ReadUnlock(const ReadUnlock &) = delete;
ReadUnlock &operator=(const ReadUnlock &) = delete;
~ReadUnlock() { mu_->ReadUnlock(); } ~ReadUnlock() { mu_->ReadUnlock(); }
private: private:
port::RWMutex *const mu_; 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) { explicit WriteLock(port::RWMutex *mu) : mu_(mu) {
this->mu_->WriteLock(); this->mu_->WriteLock();
} }
// No copying allowed
WriteLock(const WriteLock&) = delete;
void operator=(const WriteLock&) = delete;
~WriteLock() { this->mu_->WriteUnlock(); } ~WriteLock() { this->mu_->WriteUnlock(); }
private: private:
port::RWMutex *const mu_; port::RWMutex *const mu_;
// No copying allowed
WriteLock(const WriteLock&);
void operator=(const WriteLock&);
}; };
// //

@ -42,13 +42,12 @@ class Reader {
// "*file" must remain live while this Reader is in use. // "*file" must remain live while this Reader is in use.
Reader(std::unique_ptr<RandomAccessFileReader>&& file_reader, Env* env, Reader(std::unique_ptr<RandomAccessFileReader>&& file_reader, Env* env,
Statistics* statistics); Statistics* statistics);
~Reader() = default;
// No copying allowed // No copying allowed
Reader(const Reader&) = delete; Reader(const Reader&) = delete;
Reader& operator=(const Reader&) = delete; Reader& operator=(const Reader&) = delete;
~Reader() = default;
Status ReadHeader(BlobLogHeader* header); Status ReadHeader(BlobLogHeader* header);
// Read the next record into *record. Returns true if read // Read the next record into *record. Returns true if read

@ -39,13 +39,12 @@ class Writer {
Writer(std::unique_ptr<WritableFileWriter>&& dest, Env* env, Writer(std::unique_ptr<WritableFileWriter>&& dest, Env* env,
Statistics* statistics, uint64_t log_number, uint64_t bpsync, Statistics* statistics, uint64_t log_number, uint64_t bpsync,
bool use_fsync, uint64_t boffset = 0); bool use_fsync, uint64_t boffset = 0);
~Writer() = default;
// No copying allowed // No copying allowed
Writer(const Writer&) = delete; Writer(const Writer&) = delete;
Writer& operator=(const Writer&) = delete; Writer& operator=(const Writer&) = delete;
~Writer() = default;
static void ConstructBlobHeader(std::string* buf, const Slice& key, static void ConstructBlobHeader(std::string* buf, const Slice& key,
const Slice& val, uint64_t expiration); const Slice& val, uint64_t expiration);

@ -31,6 +31,9 @@ class OptimisticTransaction : public TransactionBaseImpl {
OptimisticTransaction(OptimisticTransactionDB* db, OptimisticTransaction(OptimisticTransactionDB* db,
const WriteOptions& write_options, const WriteOptions& write_options,
const OptimisticTransactionOptions& txn_options); const OptimisticTransactionOptions& txn_options);
// No copying allowed
OptimisticTransaction(const OptimisticTransaction&) = delete;
void operator=(const OptimisticTransaction&) = delete;
virtual ~OptimisticTransaction(); virtual ~OptimisticTransaction();
@ -52,7 +55,7 @@ class OptimisticTransaction : public TransactionBaseImpl {
const bool assume_tracked = false) override; const bool assume_tracked = false) override;
private: private:
OptimisticTransactionDB* const txn_db_; ROCKSDB_FIELD_UNUSED OptimisticTransactionDB* const txn_db_;
friend class OptimisticTransactionCallback; friend class OptimisticTransactionCallback;
@ -71,10 +74,6 @@ class OptimisticTransaction : public TransactionBaseImpl {
const Slice& /* unused */) override { const Slice& /* unused */) override {
// Nothing to unlock. // Nothing to unlock.
} }
// No copying allowed
OptimisticTransaction(const OptimisticTransaction&);
void operator=(const OptimisticTransaction&);
}; };
// Used at commit time to trigger transaction validation // Used at commit time to trigger transaction validation

@ -40,6 +40,9 @@ class PessimisticTransaction : public TransactionBaseImpl {
PessimisticTransaction(TransactionDB* db, const WriteOptions& write_options, PessimisticTransaction(TransactionDB* db, const WriteOptions& write_options,
const TransactionOptions& txn_options, const TransactionOptions& txn_options,
const bool init = true); const bool init = true);
// No copying allowed
PessimisticTransaction(const PessimisticTransaction&) = delete;
void operator=(const PessimisticTransaction&) = delete;
virtual ~PessimisticTransaction(); virtual ~PessimisticTransaction();
@ -193,16 +196,15 @@ class PessimisticTransaction : public TransactionBaseImpl {
void UnlockGetForUpdate(ColumnFamilyHandle* column_family, void UnlockGetForUpdate(ColumnFamilyHandle* column_family,
const Slice& key) override; const Slice& key) override;
// No copying allowed
PessimisticTransaction(const PessimisticTransaction&);
void operator=(const PessimisticTransaction&);
}; };
class WriteCommittedTxn : public PessimisticTransaction { class WriteCommittedTxn : public PessimisticTransaction {
public: public:
WriteCommittedTxn(TransactionDB* db, const WriteOptions& write_options, WriteCommittedTxn(TransactionDB* db, const WriteOptions& write_options,
const TransactionOptions& txn_options); const TransactionOptions& txn_options);
// No copying allowed
WriteCommittedTxn(const WriteCommittedTxn&) = delete;
void operator=(const WriteCommittedTxn&) = delete;
virtual ~WriteCommittedTxn() {} virtual ~WriteCommittedTxn() {}
@ -216,10 +218,6 @@ class WriteCommittedTxn : public PessimisticTransaction {
Status CommitInternal() override; Status CommitInternal() override;
Status RollbackInternal() override; Status RollbackInternal() override;
// No copying allowed
WriteCommittedTxn(const WriteCommittedTxn&);
void operator=(const WriteCommittedTxn&);
}; };
} // namespace rocksdb } // namespace rocksdb

@ -57,6 +57,9 @@ class TransactionLockMgr {
TransactionLockMgr(TransactionDB* txn_db, size_t default_num_stripes, TransactionLockMgr(TransactionDB* txn_db, size_t default_num_stripes,
int64_t max_num_locks, uint32_t max_num_deadlocks, int64_t max_num_locks, uint32_t max_num_deadlocks,
std::shared_ptr<TransactionDBMutexFactory> factory); std::shared_ptr<TransactionDBMutexFactory> factory);
// No copying allowed
TransactionLockMgr(const TransactionLockMgr&) = delete;
void operator=(const TransactionLockMgr&) = delete;
~TransactionLockMgr(); ~TransactionLockMgr();
@ -149,10 +152,6 @@ class TransactionLockMgr {
const autovector<TransactionID>& wait_ids); const autovector<TransactionID>& wait_ids);
void DecrementWaitersImpl(const PessimisticTransaction* txn, void DecrementWaitersImpl(const PessimisticTransaction* txn,
const autovector<TransactionID>& wait_ids); const autovector<TransactionID>& wait_ids);
// No copying allowed
TransactionLockMgr(const TransactionLockMgr&);
void operator=(const TransactionLockMgr&);
}; };
} // namespace rocksdb } // namespace rocksdb

@ -42,6 +42,9 @@ class WritePreparedTxn : public PessimisticTransaction {
public: public:
WritePreparedTxn(WritePreparedTxnDB* db, const WriteOptions& write_options, WritePreparedTxn(WritePreparedTxnDB* db, const WriteOptions& write_options,
const TransactionOptions& txn_options); const TransactionOptions& txn_options);
// No copying allowed
WritePreparedTxn(const WritePreparedTxn&) = delete;
void operator=(const WritePreparedTxn&) = delete;
virtual ~WritePreparedTxn() {} virtual ~WritePreparedTxn() {}
@ -106,10 +109,6 @@ class WritePreparedTxn : public PessimisticTransaction {
virtual Status RebuildFromWriteBatch(WriteBatch* src_batch) override; virtual Status RebuildFromWriteBatch(WriteBatch* src_batch) override;
// No copying allowed
WritePreparedTxn(const WritePreparedTxn&);
void operator=(const WritePreparedTxn&);
WritePreparedTxnDB* wpt_db_; WritePreparedTxnDB* wpt_db_;
// Number of sub-batches in prepare // Number of sub-batches in prepare
size_t prepare_batch_cnt_ = 0; size_t prepare_batch_cnt_ = 0;

Loading…
Cancel
Save