mirror of
https://github.com/apple/foundationdb.git
synced 2025-05-15 02:18:39 +08:00
Add support for returning whether a key-backed type range read has more results or not.
This commit is contained in:
parent
b85fbaef52
commit
91949439d2
@ -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;
|
||||||
|
@ -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",
|
||||||
|
@ -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,
|
||||||
|
@ -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>
|
||||||
|
@ -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;
|
||||||
}
|
}
|
||||||
|
@ -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();
|
||||||
|
@ -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;
|
||||||
|
Loading…
x
Reference in New Issue
Block a user