Improved const-correctness

Using the recently added IndexedSet::const_iterator
(https://github.com/apple/foundationdb/pull/3185), we can improve the
const-correctness of many functions. In this PR const is added where
applicable. Also, wherever I came across the following while adding
const, I made the following changes:

- virtual function overrides are marked as override
- NULL is replaced with nullptr
- git clang-format is applied
This commit is contained in:
sfc-gh-tclinkenbeard 2020-06-17 14:45:38 -07:00
parent 1e1c7a5a85
commit 2a393633b6
14 changed files with 181 additions and 158 deletions

View File

@ -32,18 +32,28 @@ const RYWIterator::SEGMENT_TYPE RYWIterator::typeMap[12] = {
// DEPENDENT_WRITE
RYWIterator::UNKNOWN_RANGE, RYWIterator::KV, RYWIterator::KV };
RYWIterator::SEGMENT_TYPE RYWIterator::type() {
RYWIterator::SEGMENT_TYPE RYWIterator::type() const {
if (is_unreadable())
throw accessed_unreadable();
return typeMap[ writes.type()*3 + cache.type() ];
}
bool RYWIterator::is_kv() { return type() == KV; }
bool RYWIterator::is_unknown_range() { return type() == UNKNOWN_RANGE; }
bool RYWIterator::is_empty_range() { return type() == EMPTY_RANGE; }
bool RYWIterator::is_dependent() { return writes.type() == WriteMap::iterator::DEPENDENT_WRITE; }
bool RYWIterator::is_unreadable() { return writes.is_unreadable(); }
bool RYWIterator::is_kv() const {
return type() == KV;
}
bool RYWIterator::is_unknown_range() const {
return type() == UNKNOWN_RANGE;
}
bool RYWIterator::is_empty_range() const {
return type() == EMPTY_RANGE;
}
bool RYWIterator::is_dependent() const {
return writes.type() == WriteMap::iterator::DEPENDENT_WRITE;
}
bool RYWIterator::is_unreadable() const {
return writes.is_unreadable();
}
ExtStringRef RYWIterator::beginKey() { return begin_key_cmp <= 0 ? writes.beginKey() : cache.beginKey(); }
ExtStringRef RYWIterator::endKey() { return end_key_cmp <= 0 ? cache.endKey() : writes.endKey(); }

View File

@ -32,13 +32,13 @@ public:
enum SEGMENT_TYPE { UNKNOWN_RANGE, EMPTY_RANGE, KV };
static const SEGMENT_TYPE typeMap[12];
SEGMENT_TYPE type();
SEGMENT_TYPE type() const;
bool is_kv();
bool is_unknown_range();
bool is_empty_range();
bool is_unreadable();
bool is_dependent();
bool is_kv() const;
bool is_unknown_range() const;
bool is_empty_range() const;
bool is_unreadable() const;
bool is_dependent() const;
ExtStringRef beginKey();
ExtStringRef endKey();

View File

@ -31,7 +31,7 @@ struct ExtStringRef {
ExtStringRef() : extra_zero_bytes(0) {}
ExtStringRef( StringRef const& s, int extra_zero_bytes=0 ) : base( s ), extra_zero_bytes(extra_zero_bytes) {}
Standalone<StringRef> toStandaloneStringRef() {
Standalone<StringRef> toStandaloneStringRef() const {
auto s = makeString( size() );
if (base.size() > 0) {
memcpy(mutateString(s), base.begin(), base.size());
@ -40,7 +40,7 @@ struct ExtStringRef {
return s;
};
StringRef toArenaOrRef( Arena& a ) {
StringRef toArenaOrRef(Arena& a) const {
if (extra_zero_bytes) {
StringRef dest = StringRef( new(a) uint8_t[ size() ], size() );
if (base.size() > 0) {
@ -52,12 +52,12 @@ struct ExtStringRef {
return base;
}
StringRef assertRef() {
StringRef assertRef() const {
ASSERT( extra_zero_bytes == 0 );
return base;
}
StringRef toArena( Arena& a ) {
StringRef toArena(Arena& a) const {
if (extra_zero_bytes) {
StringRef dest = StringRef( new(a) uint8_t[ size() ], size() );
if (base.size() > 0) {
@ -186,19 +186,19 @@ public:
enum SEGMENT_TYPE { UNKNOWN_RANGE, EMPTY_RANGE, KV };
SEGMENT_TYPE type() {
SEGMENT_TYPE type() const {
if (!offset) return UNKNOWN_RANGE;
if (offset&1) return EMPTY_RANGE;
return KV;
}
bool is_kv() { return type() == KV; }
bool is_unknown_range() { return type() == UNKNOWN_RANGE; }
bool is_empty_range() { return type() == EMPTY_RANGE; }
bool is_dependent() { return false; }
bool is_unreadable() { return false; }
bool is_kv() const { return type() == KV; }
bool is_unknown_range() const { return type() == UNKNOWN_RANGE; }
bool is_empty_range() const { return type() == EMPTY_RANGE; }
bool is_dependent() const { return false; }
bool is_unreadable() const { return false; }
ExtStringRef beginKey() {
ExtStringRef beginKey() const {
if (offset == 0) {
auto prev = it;
prev.decrementNonEnd();
@ -208,7 +208,7 @@ public:
else
return ExtStringRef( it->values[ (offset-2)>>1 ].key, offset&1 );
}
ExtStringRef endKey() {
ExtStringRef endKey() const {
if (offset == 0)
return it->beginKey;
else if (offset == it->segments()-1)
@ -217,7 +217,7 @@ public:
return ExtStringRef( it->values[ (offset-1)>>1 ].key, 1-(offset&1) );
}
const KeyValueRef* kv(Arena& arena) { // only if is_kv()
const KeyValueRef* kv(Arena& arena) const { // only if is_kv()
return &it->values[(offset - 2) >> 1];
}

View File

@ -333,24 +333,26 @@ public:
enum SEGMENT_TYPE { UNMODIFIED_RANGE, CLEARED_RANGE, INDEPENDENT_WRITE, DEPENDENT_WRITE };
SEGMENT_TYPE type() {
SEGMENT_TYPE type() const {
if (offset)
return entry().following_keys_cleared ? CLEARED_RANGE : UNMODIFIED_RANGE;
else
return entry().stack.isDependent() ? DEPENDENT_WRITE : INDEPENDENT_WRITE;
}
bool is_cleared_range() { return offset && entry().following_keys_cleared; }
bool is_unmodified_range() { return offset && !entry().following_keys_cleared; }
bool is_operation() { return !offset; }
bool is_conflict_range() { return offset ? entry().following_keys_conflict : entry().is_conflict; }
bool is_unreadable() { return offset ? entry().following_keys_unreadable : entry().is_unreadable; }
bool is_cleared_range() const { return offset && entry().following_keys_cleared; }
bool is_unmodified_range() const { return offset && !entry().following_keys_cleared; }
bool is_operation() const { return !offset; }
bool is_conflict_range() const { return offset ? entry().following_keys_conflict : entry().is_conflict; }
bool is_unreadable() const { return offset ? entry().following_keys_unreadable : entry().is_unreadable; }
bool is_independent() { return entry().following_keys_cleared || !entry().stack.isDependent(); } // Defined if is_operation()
bool is_independent() const {
return entry().following_keys_cleared || !entry().stack.isDependent();
} // Defined if is_operation()
ExtStringRef beginKey() { return ExtStringRef( entry().key, offset && entry().stack.size() ); }
ExtStringRef endKey() { return offset ? nextEntry().key : ExtStringRef( entry().key, 1 ); }
ExtStringRef beginKey() const { return ExtStringRef(entry().key, offset && entry().stack.size()); }
ExtStringRef endKey() const { return offset ? nextEntry().key : ExtStringRef(entry().key, 1); }
OperationStack const& op() { return entry().stack; } // Only if is_operation()
OperationStack const& op() const { return entry().stack; } // Only if is_operation()
iterator& operator++() {
if (!offset && !equalsKeyAfter( entry().key, nextEntry().key )) {
@ -392,8 +394,8 @@ public:
friend class WriteMap;
void reset( Tree const& tree, Version ver ) { this->tree = tree; this->at = ver; this->finger.clear(); beginLen=endLen=0; offset = false; }
WriteMapEntry const& entry() { return finger[beginLen-1]->data; }
WriteMapEntry const& nextEntry() { return finger[endLen-1]->data; }
WriteMapEntry const& entry() const { return finger[beginLen - 1]->data; }
WriteMapEntry const& nextEntry() const { return finger[endLen - 1]->data; }
bool keyAtBegin() { return !offset || !entry().stack.size(); }

View File

@ -37,9 +37,9 @@ public:
class IKeyValueStore : public IClosable {
public:
virtual KeyValueStoreType getType() = 0;
virtual void set( KeyValueRef keyValue, const Arena* arena = NULL ) = 0;
virtual void clear( KeyRangeRef range, const Arena* arena = NULL ) = 0;
virtual KeyValueStoreType getType() const = 0;
virtual void set(KeyValueRef keyValue, const Arena* arena = nullptr) = 0;
virtual void clear(KeyRangeRef range, const Arena* arena = nullptr) = 0;
virtual Future<Void> commit(bool sequential = false) = 0; // returns when prior sets and clears are (atomically) durable
virtual Future<Optional<Value>> readValue( KeyRef key, Optional<UID> debugID = Optional<UID>() ) = 0;
@ -54,7 +54,7 @@ public:
// To debug MEMORY_RADIXTREE type ONLY
// Returns (1) how many key & value pairs have been inserted (2) how many nodes have been created (3) how many
// key size is less than 12 bytes
virtual std::tuple<size_t, size_t, size_t> getSize() { return std::make_tuple(0, 0, 0); }
virtual std::tuple<size_t, size_t, size_t> getSize() const { return std::make_tuple(0, 0, 0); }
// Returns the amount of free and total space for this store, in bytes
virtual StorageBytes getStorageBytes() = 0;

View File

@ -85,7 +85,7 @@ public:
// Returns the usable size of pages returned by the pager (i.e. the size of the page that isn't pager overhead).
// For a given pager instance, separate calls to this function must return the same value.
// Only valid to call after recovery is complete.
virtual int getUsablePageSize() = 0;
virtual int getUsablePageSize() const = 0;
// Allocate a new page ID for a subsequent write. The page will be considered in-use after the next commit
// regardless of whether or not it was written to.
@ -131,7 +131,7 @@ public:
// Sets the next commit version
virtual void setCommitVersion(Version v) = 0;
virtual StorageBytes getStorageBytes() = 0;
virtual StorageBytes getStorageBytes() const = 0;
// Count of pages in use by the pager client
virtual Future<int64_t> getUserPageCount() = 0;
@ -142,10 +142,10 @@ public:
virtual Future<Void> init() = 0;
// Returns latest committed version
virtual Version getLatestVersion() = 0;
virtual Version getLatestVersion() const = 0;
// Returns the oldest readable version as of the most recent committed version
virtual Version getOldestVersion() = 0;
virtual Version getOldestVersion() const = 0;
// Sets the oldest readable version to be put into affect at the next commit.
// The pager can reuse pages that were freed at a version less than v.

View File

@ -45,9 +45,9 @@ public:
class IVersionedStore : public IClosable {
public:
virtual KeyValueStoreType getType() = 0;
virtual bool supportsMutation(int op) = 0; // If this returns true, then mutate(op, ...) may be called
virtual StorageBytes getStorageBytes() = 0;
virtual KeyValueStoreType getType() const = 0;
virtual bool supportsMutation(int op) const = 0; // If this returns true, then mutate(op, ...) may be called
virtual StorageBytes getStorageBytes() const = 0;
// Writes are provided in an ordered stream.
// A write is considered part of (a change leading to) the version determined by the previous call to
@ -58,11 +58,11 @@ public:
virtual void mutate(int op, StringRef param1, StringRef param2) = 0;
virtual void setWriteVersion(Version) = 0; // The write version must be nondecreasing
virtual void setOldestVersion(Version v) = 0; // Set oldest readable version to be used in next commit
virtual Version getOldestVersion() = 0; // Get oldest readable version
virtual Version getOldestVersion() const = 0; // Get oldest readable version
virtual Future<Void> commit() = 0;
virtual Future<Void> init() = 0;
virtual Version getLatestVersion() = 0;
virtual Version getLatestVersion() const = 0;
// readAtVersion() may only be called on a version which has previously been passed to setWriteVersion() and never
// previously passed

View File

@ -35,35 +35,52 @@ struct KeyValueStoreCompressTestData : IKeyValueStore {
KeyValueStoreCompressTestData(IKeyValueStore* store) : store(store) {}
virtual Future<Void> getError() { return store->getError(); }
virtual Future<Void> onClosed() { return store->onClosed(); }
virtual void dispose() { store->dispose(); delete this; }
virtual void close() { store->close(); delete this; }
virtual Future<Void> getError() override { return store->getError(); }
virtual Future<Void> onClosed() override { return store->onClosed(); }
virtual void dispose() override {
store->dispose();
delete this;
}
virtual void close() override {
store->close();
delete this;
}
virtual KeyValueStoreType getType() { return store->getType(); }
virtual StorageBytes getStorageBytes() { return store->getStorageBytes(); }
virtual KeyValueStoreType getType() const override { return store->getType(); }
virtual StorageBytes getStorageBytes() override { return store->getStorageBytes(); }
virtual void set( KeyValueRef keyValue, const Arena* arena = NULL ) {
virtual void set(KeyValueRef keyValue, const Arena* arena = nullptr) override {
store->set( KeyValueRef( keyValue.key, pack(keyValue.value) ), arena );
}
virtual void clear( KeyRangeRef range, const Arena* arena = NULL ) { store->clear( range, arena ); }
virtual void clear(KeyRangeRef range, const Arena* arena = nullptr) override { store->clear(range, arena); }
virtual Future<Void> commit(bool sequential = false) { return store->commit(sequential); }
virtual Future<Optional<Value>> readValue( KeyRef key, Optional<UID> debugID = Optional<UID>() ) {
virtual Future<Optional<Value>> readValue(KeyRef key, Optional<UID> debugID = Optional<UID>()) override {
return doReadValue(store, key, debugID);
}
ACTOR static Future<Optional<Value>> doReadValue( IKeyValueStore* store, Key key, Optional<UID> debugID ) {
Optional<Value> v = wait( store->readValue(key, debugID) );
if (!v.present()) return v;
return unpack(v.get());
}
// Note that readValuePrefix doesn't do anything in this implementation of IKeyValueStore, so the "atomic bomb" problem is still
// present if you are using this storage interface, but this storage interface is not used by customers ever. However, if you want
// to try to test malicious atomic op workloads with compressed values for some reason, you will need to fix this.
virtual Future<Optional<Value>> readValuePrefix( KeyRef key, int maxLength, Optional<UID> debugID = Optional<UID>() ) {
virtual Future<Optional<Value>> readValuePrefix(KeyRef key, int maxLength,
Optional<UID> debugID = Optional<UID>()) override {
return doReadValuePrefix( store, key, maxLength, debugID );
}
// If rowLimit>=0, reads first rows sorted ascending, otherwise reads last rows sorted descending
// The total size of the returned value (less the last entry) will be less than byteLimit
virtual Future<Standalone<RangeResultRef>> readRange(KeyRangeRef keys, int rowLimit = 1 << 30,
int byteLimit = 1 << 30) override {
return doReadRange(store, keys, rowLimit, byteLimit);
}
private:
ACTOR static Future<Optional<Value>> doReadValue(IKeyValueStore* store, Key key, Optional<UID> debugID) {
Optional<Value> v = wait(store->readValue(key, debugID));
if (!v.present()) return v;
return unpack(v.get());
}
ACTOR static Future<Optional<Value>> doReadValuePrefix( IKeyValueStore* store, Key key, int maxLength, Optional<UID> debugID ) {
Optional<Value> v = wait( doReadValue(store, key, debugID) );
if (!v.present()) return v;
@ -74,12 +91,6 @@ struct KeyValueStoreCompressTestData : IKeyValueStore {
return v;
}
}
// If rowLimit>=0, reads first rows sorted ascending, otherwise reads last rows sorted descending
// The total size of the returned value (less the last entry) will be less than byteLimit
virtual Future<Standalone<RangeResultRef>> readRange( KeyRangeRef keys, int rowLimit = 1<<30, int byteLimit = 1<<30 ) {
return doReadRange(store, keys, rowLimit, byteLimit);
}
ACTOR Future<Standalone<RangeResultRef>> doReadRange( IKeyValueStore* store, KeyRangeRef keys, int rowLimit, int byteLimit ) {
Standalone<RangeResultRef> _vs = wait( store->readRange(keys, rowLimit, byteLimit) );
Standalone<RangeResultRef> vs = _vs; // Get rid of implicit const& from wait statement
@ -89,7 +100,6 @@ struct KeyValueStoreCompressTestData : IKeyValueStore {
return vs;
}
private:
// These implement the actual "compression" scheme
static Value pack( Value val ) {
if (!val.size()) return val;

View File

@ -39,9 +39,9 @@ public:
bool replaceContent, bool exactRecovery);
// IClosable
virtual Future<Void> getError() { return log->getError(); }
virtual Future<Void> onClosed() { return log->onClosed(); }
virtual void dispose() {
virtual Future<Void> getError() override { return log->getError(); }
virtual Future<Void> onClosed() override { return log->onClosed(); }
virtual void dispose() override {
recovering.cancel();
log->dispose();
if (reserved_buffer != nullptr) {
@ -50,7 +50,7 @@ public:
}
delete this;
}
virtual void close() {
virtual void close() override {
recovering.cancel();
log->close();
if (reserved_buffer != nullptr) {
@ -61,9 +61,9 @@ public:
}
// IKeyValueStore
virtual KeyValueStoreType getType() { return type; }
virtual KeyValueStoreType getType() const override { return type; }
virtual std::tuple<size_t, size_t, size_t> getSize() { return data.size(); }
virtual std::tuple<size_t, size_t, size_t> getSize() const override { return data.size(); }
int64_t getAvailableSize() {
int64_t residentSize = data.sumTo(data.end()) + queue.totalSize() + // doesn't account for overhead in queue
@ -72,7 +72,7 @@ public:
return memoryLimit - residentSize;
}
virtual StorageBytes getStorageBytes() {
virtual StorageBytes getStorageBytes() override {
StorageBytes diskQueueBytes = log->getStorageBytes();
// Try to bound how many in-memory bytes we might need to write to disk if we commit() now
@ -103,7 +103,7 @@ public:
committedWriteBytes += bytesWritten;
}
virtual void set(KeyValueRef keyValue, const Arena* arena) {
virtual void set(KeyValueRef keyValue, const Arena* arena) override {
// A commit that occurs with no available space returns Never, so we can throw out all modifications
if (getAvailableSize() <= 0) return;
@ -117,7 +117,7 @@ public:
}
}
virtual void clear(KeyRangeRef range, const Arena* arena) {
virtual void clear(KeyRangeRef range, const Arena* arena) override {
// A commit that occurs with no available space returns Never, so we can throw out all modifications
if (getAvailableSize() <= 0) return;
@ -131,7 +131,7 @@ public:
}
}
virtual Future<Void> commit(bool sequential) {
virtual Future<Void> commit(bool sequential) override {
if(getAvailableSize() <= 0) {
TraceEvent(SevError, "KeyValueStoreMemory_OutOfSpace", id);
return Never();
@ -184,7 +184,7 @@ public:
return c;
}
virtual Future<Optional<Value>> readValue(KeyRef key, Optional<UID> debugID = Optional<UID>()) {
virtual Future<Optional<Value>> readValue(KeyRef key, Optional<UID> debugID = Optional<UID>()) override {
if (recovering.isError()) throw recovering.getError();
if (!recovering.isReady()) return waitAndReadValue(this, key);
@ -194,7 +194,7 @@ public:
}
virtual Future<Optional<Value>> readValuePrefix(KeyRef key, int maxLength,
Optional<UID> debugID = Optional<UID>()) {
Optional<UID> debugID = Optional<UID>()) override {
if (recovering.isError()) throw recovering.getError();
if (!recovering.isReady()) return waitAndReadValuePrefix(this, key, maxLength);
@ -210,7 +210,8 @@ public:
// If rowLimit>=0, reads first rows sorted ascending, otherwise reads last rows sorted descending
// The total size of the returned value (less the last entry) will be less than byteLimit
virtual Future<Standalone<RangeResultRef>> readRange( KeyRangeRef keys, int rowLimit = 1<<30, int byteLimit = 1<<30 ) {
virtual Future<Standalone<RangeResultRef>> readRange(KeyRangeRef keys, int rowLimit = 1 << 30,
int byteLimit = 1 << 30) override {
if(recovering.isError()) throw recovering.getError();
if (!recovering.isReady()) return waitAndReadRange(this, keys, rowLimit, byteLimit);
@ -252,13 +253,13 @@ public:
return result;
}
virtual void resyncLog() {
virtual void resyncLog() override {
ASSERT(recovering.isReady());
resetSnapshot = true;
log_op(OpSnapshotAbort, StringRef(), StringRef());
}
virtual void enableSnapshot() { disableSnapshot = false; }
virtual void enableSnapshot() override { disableSnapshot = false; }
private:
enum OpType {

View File

@ -1445,26 +1445,23 @@ struct ThreadSafeCounter {
class KeyValueStoreSQLite : public IKeyValueStore {
public:
virtual void dispose() {
doClose(this, true);
}
virtual void close() {
doClose(this, false);
}
virtual void dispose() override { doClose(this, true); }
virtual void close() override { doClose(this, false); }
virtual Future<Void> getError() { return delayed( readThreads->getError() || writeThread->getError() ); }
virtual Future<Void> onClosed() { return stopped.getFuture(); }
virtual Future<Void> getError() override { return delayed(readThreads->getError() || writeThread->getError()); }
virtual Future<Void> onClosed() override { return stopped.getFuture(); }
virtual KeyValueStoreType getType() { return type; }
virtual StorageBytes getStorageBytes();
virtual KeyValueStoreType getType() const override { return type; }
virtual StorageBytes getStorageBytes() override;
virtual void set( KeyValueRef keyValue, const Arena* arena = NULL );
virtual void clear( KeyRangeRef range, const Arena* arena = NULL );
virtual Future<Void> commit(bool sequential = false);
virtual void set(KeyValueRef keyValue, const Arena* arena = nullptr) override;
virtual void clear(KeyRangeRef range, const Arena* arena = nullptr) override;
virtual Future<Void> commit(bool sequential = false) override;
virtual Future<Optional<Value>> readValue( KeyRef key, Optional<UID> debugID );
virtual Future<Optional<Value>> readValuePrefix( KeyRef key, int maxLength, Optional<UID> debugID );
virtual Future<Standalone<RangeResultRef>> readRange( KeyRangeRef keys, int rowLimit = 1<<30, int byteLimit = 1<<30 );
virtual Future<Optional<Value>> readValue(KeyRef key, Optional<UID> debugID) override;
virtual Future<Optional<Value>> readValuePrefix(KeyRef key, int maxLength, Optional<UID> debugID) override;
virtual Future<Standalone<RangeResultRef>> readRange(KeyRangeRef keys, int rowLimit = 1 << 30,
int byteLimit = 1 << 30) override;
KeyValueStoreSQLite(std::string const& filename, UID logID, KeyValueStoreType type, bool checkChecksums, bool checkIntegrity);
~KeyValueStoreSQLite();
@ -2081,4 +2078,3 @@ ACTOR Future<Void> KVFileCheck(std::string filename, bool integrity) {
return Void();
}

View File

@ -100,9 +100,6 @@ struct TransientStorageMetricSample : StorageMetricSample {
TransientStorageMetricSample( int64_t metricUnitsPerSample ) : StorageMetricSample(metricUnitsPerSample) {}
bool roll( KeyRef key, int64_t metric ) {
return deterministicRandom()->random01() < (double)metric / metricUnitsPerSample; //< SOMEDAY: Better randomInt64?
}
// Returns the sampled metric value (possibly 0, possibly increased by the sampling factor)
int64_t addAndExpire( KeyRef key, int64_t metric, double expiration ) {
@ -164,6 +161,11 @@ struct TransientStorageMetricSample : StorageMetricSample {
}
private:
bool roll(KeyRef key, int64_t metric) const {
return deterministicRandom()->random01() <
(double)metric / metricUnitsPerSample; //< SOMEDAY: Better randomInt64?
}
int64_t add( KeyRef key, int64_t metric ) {
if (!metric) return 0;
int64_t mag = metric<0 ? -metric : metric;
@ -195,7 +197,7 @@ struct StorageServerMetrics {
bytesReadSample(SERVER_KNOBS->BYTES_READ_UNITS_PER_SAMPLE) {}
// Get the current estimated metrics for the given keys
StorageMetrics getMetrics( KeyRangeRef const& keys ) {
StorageMetrics getMetrics(KeyRangeRef const& keys) const {
StorageMetrics result;
result.bytes = byteSample.getEstimate( keys );
result.bytesPerKSecond = bandwidthSample.getEstimate( keys ) * SERVER_KNOBS->STORAGE_METRICS_AVERAGE_INTERVAL_PER_KSECONDS;
@ -305,9 +307,9 @@ struct StorageServerMetrics {
//static void waitMetrics( StorageServerMetrics* const& self, WaitMetricsRequest const& req );
// This function can run on untrusted user data. We must validate all divisions carefully.
KeyRef getSplitKey( int64_t remaining, int64_t estimated, int64_t limits, int64_t used, int64_t infinity,
bool isLastShard, StorageMetricSample& sample, double divisor, KeyRef const& lastKey, KeyRef const& key, bool hasUsed )
{
KeyRef getSplitKey(int64_t remaining, int64_t estimated, int64_t limits, int64_t used, int64_t infinity,
bool isLastShard, const StorageMetricSample& sample, double divisor, KeyRef const& lastKey,
KeyRef const& key, bool hasUsed) const {
ASSERT(remaining >= 0);
ASSERT(limits > 0);
ASSERT(divisor > 0);
@ -335,7 +337,7 @@ struct StorageServerMetrics {
return key;
}
void splitMetrics( SplitMetricsRequest req ) {
void splitMetrics(SplitMetricsRequest req) const {
try {
SplitMetricsReply reply;
KeyRef lastKey = req.keys.begin;
@ -378,7 +380,8 @@ struct StorageServerMetrics {
}
}
void getStorageMetrics( GetStorageMetricsRequest req, StorageBytes sb, double bytesInputRate, int64_t versionLag, double lastUpdate ){
void getStorageMetrics(GetStorageMetricsRequest req, StorageBytes sb, double bytesInputRate, int64_t versionLag,
double lastUpdate) const {
GetStorageMetricsReply rep;
// SOMEDAY: make bytes dynamic with hard disk space
@ -417,7 +420,7 @@ struct StorageServerMetrics {
// readBytes/sizeBytes exceeds the `readDensityRatio`. Please make sure to run unit tests
// `StorageMetricsSampleTests.txt` after change made.
std::vector<KeyRangeRef> getReadHotRanges(KeyRangeRef shard, double readDensityRatio, int64_t baseChunkSize,
int64_t minShardReadBandwidthPerKSeconds) {
int64_t minShardReadBandwidthPerKSeconds) const {
std::vector<KeyRangeRef> toReturn;
double shardSize = (double)byteSample.getEstimate(shard);
int64_t shardReadBandwidth = bytesReadSample.getEstimate(shard);
@ -433,7 +436,7 @@ struct StorageServerMetrics {
return toReturn;
}
KeyRef beginKey = shard.begin;
IndexedSet<Key, int64_t>::iterator endKey =
auto endKey =
byteSample.sample.index(byteSample.sample.sumTo(byteSample.sample.lower_bound(beginKey)) + baseChunkSize);
while (endKey != byteSample.sample.end()) {
if (*endKey > shard.end) {
@ -466,7 +469,7 @@ struct StorageServerMetrics {
return toReturn;
}
void getReadHotRanges(ReadHotSubRangeRequest req) {
void getReadHotRanges(ReadHotSubRangeRequest req) const {
ReadHotSubRangeReply reply;
std::vector<KeyRangeRef> v = getReadHotRanges(req.keys, SERVER_KNOBS->SHARD_MAX_READ_DENSITY_RATIO,
SERVER_KNOBS->READ_HOT_SUB_RANGE_CHUNK_SIZE,

View File

@ -1347,7 +1347,7 @@ public:
// Returns the usable size of pages returned by the pager (i.e. the size of the page that isn't pager overhead).
// For a given pager instance, separate calls to this function must return the same value.
int getUsablePageSize() override { return logicalPageSize - sizeof(FastAllocatedPage::Checksum); }
int getUsablePageSize() const override { return logicalPageSize - sizeof(FastAllocatedPage::Checksum); }
// Get a new, previously available page ID. The page will be considered in-use after the next commit
// regardless of whether or not it was written to, until it is returned to the pager via freePage()
@ -1613,7 +1613,7 @@ public:
// Get the oldest *readable* version, which is not the same as the oldest retained version as the version
// returned could have been set as the oldest version in the pending commit
Version getOldestVersion() override { return pHeader->oldestVersion; };
Version getOldestVersion() const override { return pHeader->oldestVersion; };
// Calculate the *effective* oldest version, which can be older than the one set in the last commit since we
// are allowing active snapshots to temporarily delay page reuse.
@ -1884,7 +1884,7 @@ public:
Future<Void> onClosed() override { return closedPromise.getFuture(); }
StorageBytes getStorageBytes() override {
StorageBytes getStorageBytes() const override {
ASSERT(recoverFuture.isReady());
int64_t free;
int64_t total;
@ -1931,7 +1931,7 @@ public:
Future<Void> init() override { return recoverFuture; }
Version getLatestVersion() override { return pLastCommittedHeader->committedVersion; }
Version getLatestVersion() const override { return pLastCommittedHeader->committedVersion; }
private:
~DWALPager() {}
@ -2833,8 +2833,7 @@ struct BoundaryRefAndPage {
}
};
#define NOT_IMPLEMENTED \
{ UNSTOPPABLE_ASSERT(false); }
#define NOT_IMPLEMENTED UNSTOPPABLE_ASSERT(false)
#pragma pack(push, 1)
template <typename T, typename SizeT = int8_t>
@ -2939,23 +2938,22 @@ public:
void close() { return close_impl(false); }
KeyValueStoreType getType() NOT_IMPLEMENTED bool supportsMutation(int op) NOT_IMPLEMENTED StorageBytes
getStorageBytes() {
return m_pager->getStorageBytes();
}
KeyValueStoreType getType() const override { NOT_IMPLEMENTED; }
bool supportsMutation(int op) const override { NOT_IMPLEMENTED; }
StorageBytes getStorageBytes() const override { return m_pager->getStorageBytes(); }
// Writes are provided in an ordered stream.
// A write is considered part of (a change leading to) the version determined by the previous call to
// setWriteVersion() A write shall not become durable until the following call to commit() begins, and shall be
// durable once the following call to commit() returns
void set(KeyValueRef keyValue) {
void set(KeyValueRef keyValue) override {
++g_redwoodMetrics.opSet;
++g_redwoodMetrics.opSetKeyBytes += keyValue.key.size();
++g_redwoodMetrics.opSetValueBytes += keyValue.value.size();
m_pBuffer->insert(keyValue.key).mutation().setBoundaryValue(m_pBuffer->copyToArena(keyValue.value));
}
void clear(KeyRangeRef clearedRange) {
void clear(KeyRangeRef clearedRange) override {
// Optimization for single key clears to create just one mutation boundary instead of two
if (clearedRange.begin.size() == clearedRange.end.size() - 1 &&
clearedRange.end[clearedRange.end.size() - 1] == 0 && clearedRange.end.startsWith(clearedRange.begin)) {
@ -2974,22 +2972,20 @@ public:
m_pBuffer->erase(iBegin, iEnd);
}
void mutate(int op, StringRef param1, StringRef param2) NOT_IMPLEMENTED
void mutate(int op, StringRef param1, StringRef param2) override { NOT_IMPLEMENTED; }
void setOldestVersion(Version v) {
m_newOldestVersion = v;
}
void setOldestVersion(Version v) override { m_newOldestVersion = v; }
Version getOldestVersion() { return m_pager->getOldestVersion(); }
Version getOldestVersion() const override { return m_pager->getOldestVersion(); }
Version getLatestVersion() {
Version getLatestVersion() const override {
if (m_writeVersion != invalidVersion) return m_writeVersion;
return m_pager->getLatestVersion();
}
Version getWriteVersion() { return m_writeVersion; }
Version getWriteVersion() override { return m_writeVersion; }
Version getLastCommittedVersion() { return m_lastCommittedVersion; }
Version getLastCommittedVersion() override { return m_lastCommittedVersion; }
VersionedBTree(IPager2* pager, std::string name)
: m_pager(pager), m_writeVersion(invalidVersion), m_lastCommittedVersion(invalidVersion), m_pBuffer(nullptr),
@ -3170,7 +3166,7 @@ public:
m_writeVersion = v;
}
Future<Void> commit() {
Future<Void> commit() override {
if (m_pBuffer == nullptr) return m_latestCommit;
return commit_impl(this);
}
@ -5481,22 +5477,22 @@ public:
delete self;
}
void close() { shutdown(this, false); }
void close() override { shutdown(this, false); }
void dispose() { shutdown(this, true); }
void dispose() override { shutdown(this, true); }
Future<Void> onClosed() { return m_closed.getFuture(); }
Future<Void> onClosed() override { return m_closed.getFuture(); }
Future<Void> commit(bool sequential = false) {
Future<Void> commit(bool sequential = false) override {
Future<Void> c = m_tree->commit();
m_tree->setOldestVersion(m_tree->getLatestVersion());
m_tree->setWriteVersion(m_tree->getWriteVersion() + 1);
return catchError(c);
}
KeyValueStoreType getType() { return KeyValueStoreType::SSD_REDWOOD_V1; }
KeyValueStoreType getType() const override { return KeyValueStoreType::SSD_REDWOOD_V1; }
StorageBytes getStorageBytes() { return m_tree->getStorageBytes(); }
StorageBytes getStorageBytes() override { return m_tree->getStorageBytes(); }
Future<Void> getError() { return delayed(m_error.getFuture()); };
@ -5505,12 +5501,13 @@ public:
m_tree->clear(range);
}
void set(KeyValueRef keyValue, const Arena* arena = NULL) {
void set(KeyValueRef keyValue, const Arena* arena = nullptr) override {
debug_printf("SET %s\n", printable(keyValue).c_str());
m_tree->set(keyValue);
}
Future<Standalone<RangeResultRef>> readRange(KeyRangeRef keys, int rowLimit = 1 << 30, int byteLimit = 1 << 30) {
Future<Standalone<RangeResultRef>> readRange(KeyRangeRef keys, int rowLimit = 1 << 30,
int byteLimit = 1 << 30) override {
debug_printf("READRANGE %s\n", printable(keys).c_str());
return catchError(readRange_impl(this, keys, rowLimit, byteLimit));
}

View File

@ -31,7 +31,7 @@ struct MetricSample {
explicit MetricSample(int64_t metricUnitsPerSample) : metricUnitsPerSample(metricUnitsPerSample) {}
int64_t getMetric(const T& Key) {
int64_t getMetric(const T& Key) const {
auto i = sample.find(Key);
if (i == sample.end())
return 0;
@ -46,10 +46,6 @@ struct TransientMetricSample : MetricSample<T> {
explicit TransientMetricSample(int64_t metricUnitsPerSample) : MetricSample<T>(metricUnitsPerSample) { }
bool roll(int64_t metric) {
return nondeterministicRandom()->random01() < (double)metric / this->metricUnitsPerSample; //< SOMEDAY: Better randomInt64?
}
// Returns the sampled metric value (possibly 0, possibly increased by the sampling factor)
int64_t addAndExpire(const T& key, int64_t metric, double expiration) {
int64_t x = add(key, metric);
@ -75,6 +71,11 @@ struct TransientMetricSample : MetricSample<T> {
}
private:
bool roll(int64_t metric) const {
return nondeterministicRandom()->random01() <
(double)metric / this->metricUnitsPerSample; //< SOMEDAY: Better randomInt64?
}
int64_t add(const T& key, int64_t metric) {
if (!metric) return 0;
int64_t mag = std::abs(metric);
@ -101,12 +102,8 @@ struct TransientThresholdMetricSample : MetricSample<T> {
TransientThresholdMetricSample(int64_t metricUnitsPerSample, int64_t threshold) : MetricSample<T>(metricUnitsPerSample), thresholdLimit(threshold) { }
bool roll(int64_t metric) {
return nondeterministicRandom()->random01() < (double)metric / this->metricUnitsPerSample; //< SOMEDAY: Better randomInt64?
}
template <class U>
bool isAboveThreshold(const U& key) {
bool isAboveThreshold(const U& key) const {
auto i = thresholdCrossedSet.find(key);
if (i == thresholdCrossedSet.end())
return false;
@ -146,6 +143,11 @@ struct TransientThresholdMetricSample : MetricSample<T> {
}
private:
bool roll(int64_t metric) const {
return nondeterministicRandom()->random01() <
(double)metric / this->metricUnitsPerSample; //< SOMEDAY: Better randomInt64?
}
template <class T_>
int64_t add(T_&& key, int64_t metric) {
if (!metric) return 0;

View File

@ -257,7 +257,9 @@ public:
radix_tree(const radix_tree& other) = delete; // delete
radix_tree& operator=(const radix_tree other) = delete; // delete
inline std::tuple<size_type, size_type, size_type> size() { return std::make_tuple(m_size, m_node, inline_keys); }
inline std::tuple<size_type, size_type, size_type> size() const {
return std::make_tuple(m_size, m_node, inline_keys);
}
// Return the amount of memory used by an entry in the RadixTree
static int getElementBytes(node* node) {