Add support for returning whether a key-backed type range read has more results or not.

This commit is contained in:
A.J. Beamon 2022-07-13 15:46:10 -07:00
parent b85fbaef52
commit 91949439d2
7 changed files with 109 additions and 92 deletions

View File

@ -393,21 +393,21 @@ ACTOR Future<Version> timeKeeperVersionFromDatetime(std::string datetime, Databa
try { try {
tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
tr->setOption(FDBTransactionOptions::LOCK_AWARE); tr->setOption(FDBTransactionOptions::LOCK_AWARE);
state std::vector<std::pair<int64_t, Version>> results = state KeyBackedRangeResult<std::pair<int64_t, Version>> rangeResult =
wait(versionMap.getRange(tr, 0, time, 1, Snapshot::False, Reverse::True)); wait(versionMap.getRange(tr, 0, time, 1, Snapshot::False, Reverse::True));
if (results.size() != 1) { if (rangeResult.results.size() != 1) {
// No key less than time was found in the database // No key less than time was found in the database
// Look for a key >= time. // Look for a key >= time.
wait(store(results, versionMap.getRange(tr, time, std::numeric_limits<int64_t>::max(), 1))); wait(store(rangeResult, versionMap.getRange(tr, time, std::numeric_limits<int64_t>::max(), 1)));
if (results.size() != 1) { if (rangeResult.results.size() != 1) {
fprintf(stderr, "ERROR: Unable to calculate a version for given date/time.\n"); fprintf(stderr, "ERROR: Unable to calculate a version for given date/time.\n");
throw backup_error(); throw backup_error();
} }
} }
// Adjust version found by the delta between time and the time found and min with 0. // Adjust version found by the delta between time and the time found and min with 0.
auto& result = results[0]; auto& result = rangeResult.results[0];
return std::max<Version>(0, result.second + (time - result.first) * CLIENT_KNOBS->CORE_VERSIONSPERSECOND); return std::max<Version>(0, result.second + (time - result.first) * CLIENT_KNOBS->CORE_VERSIONSPERSECOND);
} catch (Error& e) { } catch (Error& e) {
@ -432,21 +432,21 @@ ACTOR Future<Optional<int64_t>> timeKeeperEpochsFromVersion(Version v, Reference
mid = (min + max + 1) / 2; // ceiling mid = (min + max + 1) / 2; // ceiling
// Find the highest time < mid // Find the highest time < mid
state std::vector<std::pair<int64_t, Version>> results = state KeyBackedRangeResult<std::pair<int64_t, Version>> rangeResult =
wait(versionMap.getRange(tr, min, mid, 1, Snapshot::False, Reverse::True)); wait(versionMap.getRange(tr, min, mid, 1, Snapshot::False, Reverse::True));
if (results.size() != 1) { if (rangeResult.results.size() != 1) {
if (mid == min) { if (mid == min) {
// There aren't any records having a version < v, so just look for any record having a time < now // There aren't any records having a version < v, so just look for any record having a time < now
// and base a result on it // and base a result on it
wait(store(results, versionMap.getRange(tr, 0, (int64_t)now(), 1))); wait(store(rangeResult, versionMap.getRange(tr, 0, (int64_t)now(), 1)));
if (results.size() != 1) { if (rangeResult.results.size() != 1) {
// There aren't any timekeeper records to base a result on so return nothing // There aren't any timekeeper records to base a result on so return nothing
return Optional<int64_t>(); return Optional<int64_t>();
} }
found = results[0]; found = rangeResult.results[0];
break; break;
} }
@ -454,7 +454,7 @@ ACTOR Future<Optional<int64_t>> timeKeeperEpochsFromVersion(Version v, Reference
continue; continue;
} }
found = results[0]; found = rangeResult.results[0];
if (v < found.second) { if (v < found.second) {
max = found.first; max = found.first;

View File

@ -123,9 +123,9 @@ ACTOR Future<std::vector<KeyBackedTag>> TagUidMap::getAll_impl(TagUidMap* tagsMa
Reference<ReadYourWritesTransaction> tr, Reference<ReadYourWritesTransaction> tr,
Snapshot snapshot) { Snapshot snapshot) {
state Key prefix = tagsMap->prefix; // Copying it here as tagsMap lifetime is not tied to this actor state Key prefix = tagsMap->prefix; // Copying it here as tagsMap lifetime is not tied to this actor
TagMap::PairsType tagPairs = wait(tagsMap->getRange(tr, std::string(), {}, 1e6, snapshot)); TagMap::RangeResultType tagPairs = wait(tagsMap->getRange(tr, std::string(), {}, 1e6, snapshot));
std::vector<KeyBackedTag> results; std::vector<KeyBackedTag> results;
for (auto& p : tagPairs) for (auto& p : tagPairs.results)
results.push_back(KeyBackedTag(p.first, prefix)); results.push_back(KeyBackedTag(p.first, prefix));
return results; return results;
} }
@ -1564,18 +1564,22 @@ struct BackupSnapshotDispatchTask : BackupTaskFuncBase {
tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
tr->setOption(FDBTransactionOptions::LOCK_AWARE); tr->setOption(FDBTransactionOptions::LOCK_AWARE);
state Future<std::vector<std::pair<Key, bool>>> bounds = config.snapshotRangeDispatchMap().getRange( state Future<BackupConfig::RangeDispatchMapT::RangeResultType> bounds =
tr, beginKey, keyAfter(normalKeys.end), CLIENT_KNOBS->TOO_MANY); config.snapshotRangeDispatchMap().getRange(
tr, beginKey, keyAfter(normalKeys.end), CLIENT_KNOBS->TOO_MANY);
wait(success(bounds) && taskBucket->keepRunning(tr, task) && wait(success(bounds) && taskBucket->keepRunning(tr, task) &&
store(recentReadVersion, tr->getReadVersion())); store(recentReadVersion, tr->getReadVersion()));
if (bounds.get().empty()) if (!bounds.get().results.empty()) {
dispatchBoundaries.reserve(dispatchBoundaries.size() + bounds.get().results.size());
dispatchBoundaries.insert(
dispatchBoundaries.end(), bounds.get().results.begin(), bounds.get().results.end());
}
if (!bounds.get().more)
break; break;
dispatchBoundaries.reserve(dispatchBoundaries.size() + bounds.get().size()); beginKey = keyAfter(bounds.get().results.back().first);
dispatchBoundaries.insert(dispatchBoundaries.end(), bounds.get().begin(), bounds.get().end());
beginKey = keyAfter(bounds.get().back().first);
tr->reset(); tr->reset();
} catch (Error& e) { } catch (Error& e) {
wait(tr->onError(e)); wait(tr->onError(e));
@ -2543,17 +2547,17 @@ struct BackupSnapshotManifest : BackupTaskFuncBase {
wait(store(bc, config.backupContainer().getOrThrow(tr))); wait(store(bc, config.backupContainer().getOrThrow(tr)));
} }
BackupConfig::RangeFileMapT::PairsType rangeresults = BackupConfig::RangeFileMapT::RangeResultType rangeresults =
wait(config.snapshotRangeFileMap().getRange(tr, startKey, {}, batchSize)); wait(config.snapshotRangeFileMap().getRange(tr, startKey, {}, batchSize));
for (auto& p : rangeresults) { for (auto& p : rangeresults.results) {
localmap.insert(p); localmap.insert(p);
} }
if (rangeresults.size() < batchSize) if (!rangeresults.more)
break; break;
startKey = keyAfter(rangeresults.back().first); startKey = keyAfter(rangeresults.results.back().first);
tr->reset(); tr->reset();
} catch (Error& e) { } catch (Error& e) {
wait(tr->onError(e)); wait(tr->onError(e));
@ -3617,7 +3621,7 @@ struct RestoreDispatchTaskFunc : RestoreTaskFuncBase {
// Get a batch of files. We're targeting batchSize blocks being dispatched so query for batchSize files (each // Get a batch of files. We're targeting batchSize blocks being dispatched so query for batchSize files (each
// of which is 0 or more blocks). // of which is 0 or more blocks).
state int taskBatchSize = BUGGIFY ? 1 : CLIENT_KNOBS->RESTORE_DISPATCH_ADDTASK_SIZE; state int taskBatchSize = BUGGIFY ? 1 : CLIENT_KNOBS->RESTORE_DISPATCH_ADDTASK_SIZE;
state RestoreConfig::FileSetT::Values files = wait(restore.fileSet().getRange( state RestoreConfig::FileSetT::RangeResultType files = wait(restore.fileSet().getRange(
tr, Optional<RestoreConfig::RestoreFile>({ beginVersion, beginFile }), {}, taskBatchSize)); tr, Optional<RestoreConfig::RestoreFile>({ beginVersion, beginFile }), {}, taskBatchSize));
// allPartsDone will be set once all block tasks in the current batch are finished. // allPartsDone will be set once all block tasks in the current batch are finished.
@ -3636,7 +3640,7 @@ struct RestoreDispatchTaskFunc : RestoreTaskFuncBase {
} }
// If there were no files to load then this batch is done and restore is almost done. // If there were no files to load then this batch is done and restore is almost done.
if (files.size() == 0) { if (files.results.size() == 0) {
// If adding to existing batch then blocks could be in progress so create a new Dispatch task that waits for // If adding to existing batch then blocks could be in progress so create a new Dispatch task that waits for
// them to finish // them to finish
if (addingToExistingBatch) { if (addingToExistingBatch) {
@ -3714,13 +3718,13 @@ struct RestoreDispatchTaskFunc : RestoreTaskFuncBase {
// blocks per Dispatch task and target batchSize total per batch but a batch must end on a complete version // blocks per Dispatch task and target batchSize total per batch but a batch must end on a complete version
// boundary so exceed the limit if necessary to reach the end of a version of files. // boundary so exceed the limit if necessary to reach the end of a version of files.
state std::vector<Future<Key>> addTaskFutures; state std::vector<Future<Key>> addTaskFutures;
state Version endVersion = files[0].version; state Version endVersion = files.results[0].version;
state int blocksDispatched = 0; state int blocksDispatched = 0;
state int64_t beginBlock = Params.beginBlock().getOrDefault(task); state int64_t beginBlock = Params.beginBlock().getOrDefault(task);
state int i = 0; state int i = 0;
for (; i < files.size(); ++i) { for (; i < files.results.size(); ++i) {
RestoreConfig::RestoreFile& f = files[i]; RestoreConfig::RestoreFile& f = files.results[i];
// Here we are "between versions" (prior to adding the first block of the first file of a new version) so // Here we are "between versions" (prior to adding the first block of the first file of a new version) so
// this is an opportunity to end the current dispatch batch (which must end on a version boundary) if the // this is an opportunity to end the current dispatch batch (which must end on a version boundary) if the
@ -5057,11 +5061,11 @@ public:
doc.setKey("CurrentSnapshot", snapshot); doc.setKey("CurrentSnapshot", snapshot);
} }
KeyBackedMap<int64_t, std::pair<std::string, Version>>::PairsType errors = KeyBackedMap<int64_t, std::pair<std::string, Version>>::RangeResultType errors =
wait(config.lastErrorPerType().getRange( wait(config.lastErrorPerType().getRange(
tr, 0, std::numeric_limits<int>::max(), CLIENT_KNOBS->TOO_MANY)); tr, 0, std::numeric_limits<int>::max(), CLIENT_KNOBS->TOO_MANY));
JsonBuilderArray errorList; JsonBuilderArray errorList;
for (auto& e : errors) { for (auto& e : errors.results) {
std::string msg = e.second.first; std::string msg = e.second.first;
Version ver = e.second.second; Version ver = e.second.second;
@ -5209,13 +5213,13 @@ public:
// Append the errors, if requested // Append the errors, if requested
if (showErrors) { if (showErrors) {
KeyBackedMap<int64_t, std::pair<std::string, Version>>::PairsType errors = KeyBackedMap<int64_t, std::pair<std::string, Version>>::RangeResultType errors =
wait(config.lastErrorPerType().getRange( wait(config.lastErrorPerType().getRange(
tr, 0, std::numeric_limits<int>::max(), CLIENT_KNOBS->TOO_MANY)); tr, 0, std::numeric_limits<int>::max(), CLIENT_KNOBS->TOO_MANY));
std::string recentErrors; std::string recentErrors;
std::string pastErrors; std::string pastErrors;
for (auto& e : errors) { for (auto& e : errors.results) {
Version v = e.second.second; Version v = e.second.second;
std::string msg = format( std::string msg = format(
"%s ago : %s\n", "%s ago : %s\n",

View File

@ -7948,7 +7948,8 @@ ACTOR Future<std::vector<std::pair<UID, StorageWiggleValue>>> readStorageWiggleV
state KeyBackedObjectMap<UID, StorageWiggleValue, decltype(IncludeVersion())> metadataMap(readKey, state KeyBackedObjectMap<UID, StorageWiggleValue, decltype(IncludeVersion())> metadataMap(readKey,
IncludeVersion()); IncludeVersion());
state Reference<ReadYourWritesTransaction> tr(new ReadYourWritesTransaction(cx)); state Reference<ReadYourWritesTransaction> tr(new ReadYourWritesTransaction(cx));
state std::vector<std::pair<UID, StorageWiggleValue>> res; state KeyBackedRangeResult<std::pair<UID, StorageWiggleValue>> res;
// read the wiggling pairs // read the wiggling pairs
loop { loop {
try { try {
@ -7964,7 +7965,7 @@ ACTOR Future<std::vector<std::pair<UID, StorageWiggleValue>>> readStorageWiggleV
wait(tr->onError(e)); wait(tr->onError(e));
} }
} }
return res; return res.results;
} }
ACTOR Future<Void> splitStorageMetricsStream(PromiseStream<Key> resultStream, ACTOR Future<Void> splitStorageMetricsStream(PromiseStream<Key> resultStream,

View File

@ -151,6 +151,12 @@ inline KeyRange TupleCodec<KeyRange>::unpack(Standalone<StringRef> const& val) {
return KeyRangeRef(t.getString(0), t.getString(1)); return KeyRangeRef(t.getString(0), t.getString(1));
} }
template <typename ResultType>
struct KeyBackedRangeResult {
std::vector<ResultType> results;
bool more;
};
// Convenient read/write access to a single value of type T stored at key // Convenient read/write access to a single value of type T stored at key
// Even though 'this' is not actually mutated, methods that change the db key are not const. // Even though 'this' is not actually mutated, methods that change the db key are not const.
template <typename T, typename Codec = TupleCodec<T>> template <typename T, typename Codec = TupleCodec<T>>
@ -308,16 +314,16 @@ public:
typedef _KeyType KeyType; typedef _KeyType KeyType;
typedef _ValueType ValueType; typedef _ValueType ValueType;
typedef std::pair<KeyType, ValueType> PairType; typedef std::pair<KeyType, ValueType> PairType;
typedef std::vector<PairType> PairsType; typedef KeyBackedRangeResult<PairType> RangeResultType;
// If end is not present one key past the end of the map is used. // If end is not present one key past the end of the map is used.
template <class Transaction> template <class Transaction>
Future<PairsType> getRange(Transaction tr, Future<RangeResultType> getRange(Transaction tr,
Optional<KeyType> const& begin, Optional<KeyType> const& begin,
Optional<KeyType> const& end, Optional<KeyType> const& end,
int limit, int limit,
Snapshot snapshot = Snapshot::False, Snapshot snapshot = Snapshot::False,
Reverse reverse = Reverse::False) const { Reverse reverse = Reverse::False) const {
Key prefix = subspace.begin; // 'this' could be invalid inside lambda Key prefix = subspace.begin; // 'this' could be invalid inside lambda
Key beginKey = begin.present() ? prefix.withSuffix(KeyCodec::pack(begin.get())) : subspace.begin; Key beginKey = begin.present() ? prefix.withSuffix(KeyCodec::pack(begin.get())) : subspace.begin;
@ -326,16 +332,18 @@ public:
typename transaction_future_type<Transaction, RangeResult>::type getRangeFuture = typename transaction_future_type<Transaction, RangeResult>::type getRangeFuture =
tr->getRange(KeyRangeRef(beginKey, endKey), GetRangeLimits(limit), snapshot, reverse); tr->getRange(KeyRangeRef(beginKey, endKey), GetRangeLimits(limit), snapshot, reverse);
return holdWhile(getRangeFuture, return holdWhile(
map(safeThreadFutureToFuture(getRangeFuture), [prefix](RangeResult const& kvs) -> PairsType { getRangeFuture,
PairsType results; map(safeThreadFutureToFuture(getRangeFuture), [prefix](RangeResult const& kvs) -> RangeResultType {
for (int i = 0; i < kvs.size(); ++i) { RangeResultType rangeResult;
KeyType key = KeyCodec::unpack(kvs[i].key.removePrefix(prefix)); for (int i = 0; i < kvs.size(); ++i) {
ValueType val = ValueCodec::unpack(kvs[i].value); KeyType key = KeyCodec::unpack(kvs[i].key.removePrefix(prefix));
results.push_back(PairType(key, val)); ValueType val = ValueCodec::unpack(kvs[i].value);
} rangeResult.results.push_back(PairType(key, val));
return results; }
})); rangeResult.more = kvs.more;
return rangeResult;
}));
} }
template <class Transaction> template <class Transaction>
@ -497,15 +505,15 @@ public:
typedef _KeyType KeyType; typedef _KeyType KeyType;
typedef _ValueType ValueType; typedef _ValueType ValueType;
typedef std::pair<KeyType, ValueType> PairType; typedef std::pair<KeyType, ValueType> PairType;
typedef std::vector<PairType> PairsType; typedef KeyBackedRangeResult<PairType> RangeResultType;
template <class Transaction> template <class Transaction>
Future<PairsType> getRange(Transaction tr, Future<RangeResultType> getRange(Transaction tr,
Optional<KeyType> const& begin, Optional<KeyType> const& begin,
Optional<KeyType> const& end, Optional<KeyType> const& end,
int limit, int limit,
Snapshot snapshot = Snapshot::False, Snapshot snapshot = Snapshot::False,
Reverse reverse = Reverse::False) const { Reverse reverse = Reverse::False) const {
Key beginKey = begin.present() ? subspace.begin.withSuffix(KeyCodec::pack(begin.get())) : subspace.begin; Key beginKey = begin.present() ? subspace.begin.withSuffix(KeyCodec::pack(begin.get())) : subspace.begin;
Key endKey = end.present() ? subspace.begin.withSuffix(KeyCodec::pack(end.get())) : subspace.end; Key endKey = end.present() ? subspace.begin.withSuffix(KeyCodec::pack(end.get())) : subspace.end;
@ -514,14 +522,15 @@ public:
return holdWhile( return holdWhile(
getRangeFuture, getRangeFuture,
map(safeThreadFutureToFuture(getRangeFuture), [self = *this](RangeResult const& kvs) -> PairsType { map(safeThreadFutureToFuture(getRangeFuture), [self = *this](RangeResult const& kvs) -> RangeResultType {
PairsType results; RangeResultType rangeResult;
for (int i = 0; i < kvs.size(); ++i) { for (int i = 0; i < kvs.size(); ++i) {
KeyType key = KeyCodec::unpack(kvs[i].key.removePrefix(self.subspace.begin)); KeyType key = KeyCodec::unpack(kvs[i].key.removePrefix(self.subspace.begin));
ValueType val = ObjectReader::fromStringRef<ValueType>(kvs[i].value, self.versionOptions); ValueType val = ObjectReader::fromStringRef<ValueType>(kvs[i].value, self.versionOptions);
results.push_back(PairType(key, val)); rangeResult.results.push_back(PairType(key, val));
} }
return results; rangeResult.more = kvs.more;
return rangeResult;
})); }));
} }
@ -584,15 +593,15 @@ public:
KeyBackedSet(KeyRef key) : subspace(prefixRange(key)) {} KeyBackedSet(KeyRef key) : subspace(prefixRange(key)) {}
typedef _ValueType ValueType; typedef _ValueType ValueType;
typedef std::vector<ValueType> Values; typedef KeyBackedRangeResult<ValueType> RangeResultType;
template <class Transaction> template <class Transaction>
Future<Values> getRange(Transaction tr, Future<RangeResultType> getRange(Transaction tr,
Optional<ValueType> const& begin, Optional<ValueType> const& begin,
Optional<ValueType> const& end, Optional<ValueType> const& end,
int limit, int limit,
Snapshot snapshot = Snapshot::False, Snapshot snapshot = Snapshot::False,
Reverse reverse = Reverse::False) const { Reverse reverse = Reverse::False) const {
Key prefix = subspace.begin; // 'this' could be invalid inside lambda Key prefix = subspace.begin; // 'this' could be invalid inside lambda
Key beginKey = begin.present() ? prefix.withSuffix(Codec::pack(begin.get())) : subspace.begin; Key beginKey = begin.present() ? prefix.withSuffix(Codec::pack(begin.get())) : subspace.begin;
Key endKey = end.present() ? prefix.withSuffix(Codec::pack(end.get())) : subspace.end; Key endKey = end.present() ? prefix.withSuffix(Codec::pack(end.get())) : subspace.end;
@ -600,14 +609,16 @@ public:
typename transaction_future_type<Transaction, RangeResult>::type getRangeFuture = typename transaction_future_type<Transaction, RangeResult>::type getRangeFuture =
tr->getRange(KeyRangeRef(beginKey, endKey), GetRangeLimits(limit), snapshot, reverse); tr->getRange(KeyRangeRef(beginKey, endKey), GetRangeLimits(limit), snapshot, reverse);
return holdWhile(getRangeFuture, return holdWhile(
map(safeThreadFutureToFuture(getRangeFuture), [prefix](RangeResult const& kvs) -> Values { getRangeFuture,
Values results; map(safeThreadFutureToFuture(getRangeFuture), [prefix](RangeResult const& kvs) -> RangeResultType {
for (int i = 0; i < kvs.size(); ++i) { RangeResultType rangeResult;
results.push_back(Codec::unpack(kvs[i].key.removePrefix(prefix))); for (int i = 0; i < kvs.size(); ++i) {
} rangeResult.results.push_back(Codec::unpack(kvs[i].key.removePrefix(prefix)));
return results; }
})); rangeResult.more = kvs.more;
return rangeResult;
}));
} }
template <class Transaction> template <class Transaction>

View File

@ -106,7 +106,7 @@ struct MetricsConfig {
typedef KeyBackedMap<int64_t, MetricsRule> RuleMapT; typedef KeyBackedMap<int64_t, MetricsRule> RuleMapT;
RuleMapT ruleMap; RuleMapT ruleMap;
RuleMapT::PairsType rules; RuleMapT::RangeResultType rules;
KeyBackedMap<Key, int64_t> addressMap; KeyBackedMap<Key, int64_t> addressMap;
KeyBackedMap<std::pair<Key, Key>, int64_t> nameAndTypeMap; KeyBackedMap<std::pair<Key, Key>, int64_t> nameAndTypeMap;
@ -152,11 +152,11 @@ ACTOR Future<Void> metricRuleUpdater(Database cx, MetricsConfig* config, TDMetri
state Future<Void> newMetric = collection->metricAdded.onTrigger(); state Future<Void> newMetric = collection->metricAdded.onTrigger();
try { try {
tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
MetricsConfig::RuleMapT::PairsType rules = wait(config->ruleMap.getRange(tr, 0, {}, 1e6)); MetricsConfig::RuleMapT::RangeResultType rules = wait(config->ruleMap.getRange(tr, 0, {}, 1e6));
for (auto& it : collection->metricMap) { for (auto& it : collection->metricMap) {
it.value->setConfig(false); it.value->setConfig(false);
for (auto i = rules.rbegin(); !(i == rules.rend()); ++i) for (auto i = rules.results.rbegin(); !(i == rules.results.rend()); ++i)
if (i->second.applyTo(it.value.getPtr(), collection->address)) if (i->second.applyTo(it.value.getPtr(), collection->address))
break; break;
} }

View File

@ -26,14 +26,15 @@
ACTOR Future<Void> readTSSMappingRYW(Reference<ReadYourWritesTransaction> tr, ACTOR Future<Void> readTSSMappingRYW(Reference<ReadYourWritesTransaction> tr,
std::map<UID, StorageServerInterface>* tssMapping) { std::map<UID, StorageServerInterface>* tssMapping) {
KeyBackedMap<UID, UID> tssMapDB = KeyBackedMap<UID, UID>(tssMappingKeys.begin); KeyBackedMap<UID, UID> tssMapDB = KeyBackedMap<UID, UID>(tssMappingKeys.begin);
state std::vector<std::pair<UID, UID>> uidMapping = state KeyBackedMap<UID, UID>::RangeResultType uidMapping =
wait(tssMapDB.getRange(tr, UID(), Optional<UID>(), CLIENT_KNOBS->TOO_MANY)); wait(tssMapDB.getRange(tr, UID(), Optional<UID>(), CLIENT_KNOBS->TOO_MANY));
ASSERT(uidMapping.size() < CLIENT_KNOBS->TOO_MANY); ASSERT(uidMapping.results.size() < CLIENT_KNOBS->TOO_MANY);
state std::map<UID, StorageServerInterface> mapping; state std::map<UID, StorageServerInterface> mapping;
for (auto& it : uidMapping) { state std::vector<std::pair<UID, UID>>::iterator mapItr;
state UID ssId = it.first; for (mapItr = uidMapping.results.begin(); mapItr != uidMapping.results.end(); ++mapItr) {
Optional<Value> v = wait(tr->get(serverListKeyFor(it.second))); state UID ssId = mapItr->first;
Optional<Value> v = wait(tr->get(serverListKeyFor(mapItr->second)));
(*tssMapping)[ssId] = decodeServerListValue(v.get()); (*tssMapping)[ssId] = decodeServerListValue(v.get());
} }
return Void(); return Void();

View File

@ -79,23 +79,23 @@ struct TimeKeeperCorrectnessWorkload : TestWorkload {
tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
tr->setOption(FDBTransactionOptions::LOCK_AWARE); tr->setOption(FDBTransactionOptions::LOCK_AWARE);
std::vector<std::pair<int64_t, Version>> allItems = KeyBackedRangeResult<std::pair<int64_t, Version>> allItems =
wait(dbTimeKeeper.getRange(tr, 0, Optional<int64_t>(), self->inMemTimeKeeper.size() + 2)); wait(dbTimeKeeper.getRange(tr, 0, Optional<int64_t>(), self->inMemTimeKeeper.size() + 2));
if (allItems.size() > SERVER_KNOBS->TIME_KEEPER_MAX_ENTRIES + 1) { if (allItems.results.size() > SERVER_KNOBS->TIME_KEEPER_MAX_ENTRIES + 1) {
TraceEvent(SevError, "TKCorrectness_TooManyEntries") TraceEvent(SevError, "TKCorrectness_TooManyEntries")
.detail("Expected", SERVER_KNOBS->TIME_KEEPER_MAX_ENTRIES + 1) .detail("Expected", SERVER_KNOBS->TIME_KEEPER_MAX_ENTRIES + 1)
.detail("Found", allItems.size()); .detail("Found", allItems.results.size());
return false; return false;
} }
if (allItems.size() < self->testDuration / SERVER_KNOBS->TIME_KEEPER_DELAY) { if (allItems.results.size() < self->testDuration / SERVER_KNOBS->TIME_KEEPER_DELAY) {
TraceEvent(SevWarnAlways, "TKCorrectness_TooFewEntries") TraceEvent(SevWarnAlways, "TKCorrectness_TooFewEntries")
.detail("Expected", self->testDuration / SERVER_KNOBS->TIME_KEEPER_DELAY) .detail("Expected", self->testDuration / SERVER_KNOBS->TIME_KEEPER_DELAY)
.detail("Found", allItems.size()); .detail("Found", allItems.results.size());
} }
for (auto item : allItems) { for (auto item : allItems.results) {
auto it = self->inMemTimeKeeper.lower_bound(item.first); auto it = self->inMemTimeKeeper.lower_bound(item.first);
if (it == self->inMemTimeKeeper.end()) { if (it == self->inMemTimeKeeper.end()) {
continue; continue;