mirror of
https://github.com/apple/foundationdb.git
synced 2025-05-14 01:42:37 +08:00
Add a simple workload, ReportConflictingKeysWorkload, to test correctness of the API and performance overhead added to the resovler.
This commit is contained in:
parent
edf52e8c97
commit
eacfdf6da3
@ -1584,7 +1584,6 @@ struct UnitTestsFunc : InstructionFunc {
|
||||
data->db->setDatabaseOption(FDBDatabaseOption::FDB_DB_OPTION_TRANSACTION_RETRY_LIMIT, Optional<StringRef>(StringRef((const uint8_t*)&noRetryLimit, 8)));
|
||||
data->db->setDatabaseOption(FDBDatabaseOption::FDB_DB_OPTION_TRANSACTION_CAUSAL_READ_RISKY);
|
||||
data->db->setDatabaseOption(FDBDatabaseOption::FDB_DB_OPTION_TRANSACTION_INCLUDE_PORT_IN_ADDRESS);
|
||||
data->db->setDatabaseOption(FDBDatabaseOption::FDB_DB_OPTION_TRANSACTION_REPORT_CONFLICTING_KEYS);
|
||||
|
||||
state Reference<Transaction> tr = data->db->createTransaction();
|
||||
tr->setOption(FDBTransactionOption::FDB_TR_OPTION_PRIORITY_SYSTEM_IMMEDIATE);
|
||||
|
@ -163,7 +163,6 @@ struct CommitTransactionRef {
|
||||
}
|
||||
|
||||
void clear( Arena& arena, KeyRangeRef const& keys ) {
|
||||
// TODO: check do I need to clear flag here
|
||||
mutations.push_back_deep(arena, MutationRef(MutationRef::ClearRange, keys.begin, keys.end));
|
||||
write_conflict_ranges.push_back_deep(arena, keys);
|
||||
}
|
||||
|
@ -103,8 +103,7 @@ struct CommitID {
|
||||
constexpr static FileIdentifier file_identifier = 14254927;
|
||||
Version version; // returns invalidVersion if transaction conflicts
|
||||
uint16_t txnBatchId;
|
||||
Optional<Value> metadataVersion;
|
||||
// TODO : data structure okay here ?
|
||||
Optional<Value> metadataVersion;
|
||||
Optional<Standalone<VectorRef<KeyRangeRef>>> conflictingKeyRanges;
|
||||
|
||||
template <class Ar>
|
||||
@ -120,13 +119,11 @@ struct CommitTransactionRequest : TimedRequest {
|
||||
constexpr static FileIdentifier file_identifier = 93948;
|
||||
enum {
|
||||
FLAG_IS_LOCK_AWARE = 0x1,
|
||||
FLAG_FIRST_IN_BATCH = 0x2,
|
||||
FLAG_REPORT_CONFLICTING_KEYS = 0x4
|
||||
FLAG_FIRST_IN_BATCH = 0x2
|
||||
};
|
||||
|
||||
bool isLockAware() const { return (flags & FLAG_IS_LOCK_AWARE) != 0; }
|
||||
bool firstInBatch() const { return (flags & FLAG_FIRST_IN_BATCH) != 0; }
|
||||
bool isReportConflictingKeys() const { return (flags & FLAG_REPORT_CONFLICTING_KEYS) != 0; }
|
||||
|
||||
Arena arena;
|
||||
CommitTransactionRef transaction;
|
||||
|
@ -2789,8 +2789,6 @@ Future<Void> Transaction::commitMutations() {
|
||||
tr.flags = tr.flags | CommitTransactionRequest::FLAG_FIRST_IN_BATCH;
|
||||
}
|
||||
if(options.reportConflictingKeys) {
|
||||
// TODO : Is it better to keep it as a flag?
|
||||
tr.flags = tr.flags | CommitTransactionRequest::FLAG_REPORT_CONFLICTING_KEYS;
|
||||
tr.reportConflictingKeys();
|
||||
}
|
||||
|
||||
|
@ -23,7 +23,6 @@
|
||||
#include "fdbclient/DatabaseContext.h"
|
||||
#include "fdbclient/StatusClient.h"
|
||||
#include "fdbclient/MonitorLeader.h"
|
||||
#include "fdbclient/JsonBuilder.h"
|
||||
#include "flow/Util.h"
|
||||
#include "flow/actorcompiler.h" // This must be the last #include.
|
||||
|
||||
@ -1229,17 +1228,23 @@ Future< Optional<Value> > ReadYourWritesTransaction::get( const Key& key, bool s
|
||||
return Optional<Value>();
|
||||
}
|
||||
|
||||
// TODO : add conflict keys to special key space
|
||||
// Add conflicting keys to special key space
|
||||
if (key == LiteralStringRef("\xff\xff/conflicting_keys/json")){
|
||||
if (!tr.info.conflictingKeyRanges.empty()){
|
||||
// TODO : return a json value which represents all the values
|
||||
JsonBuilderArray conflictingKeysArray;
|
||||
for (auto & cKR : tr.info.conflictingKeyRanges) {
|
||||
for (auto & kr : cKR) {
|
||||
conflictingKeysArray.push_back(format("[%s, %s)", kr.begin.toString().c_str(), kr.end.toString().c_str()));
|
||||
json_spirit::mArray root;
|
||||
json_spirit::mArray keyranges;
|
||||
json_spirit::mObject keyrange;
|
||||
for (int i = 0; i < tr.info.conflictingKeyRanges.size(); ++i) {
|
||||
for (const auto & kr : tr.info.conflictingKeyRanges[i]) {
|
||||
keyrange["begin"] = kr.begin.toString();
|
||||
keyrange["end"] = kr.end.toString();
|
||||
keyranges.push_back(keyrange);
|
||||
keyrange.clear();
|
||||
}
|
||||
root.push_back(keyranges);
|
||||
keyranges.clear();
|
||||
}
|
||||
Optional<Value> output = StringRef(conflictingKeysArray.getJson());
|
||||
Optional<Value> output = StringRef(json_spirit::write_string(json_spirit::mValue(root), json_spirit::Output_options::raw_utf8).c_str());
|
||||
return output;
|
||||
} else {
|
||||
return Optional<Value>();
|
||||
|
@ -174,9 +174,6 @@ description is not currently required but encouraged.
|
||||
<Option name="transaction_include_port_in_address" code="505"
|
||||
description="Addresses returned by get_addresses_for_key include the port when enabled. This will be enabled by default in api version 700, and this option will be deprecated."
|
||||
defaultFor="23"/>
|
||||
<Option name="transaction_report_conflicting_keys" code="506"
|
||||
description="Report conflicting keys for transactions that are determined as conflicts by resolvers."
|
||||
defaultFor="712"/>
|
||||
</Scope>
|
||||
|
||||
<Scope name="TransactionOption">
|
||||
|
@ -162,6 +162,7 @@ set(FDBSERVER_SRCS
|
||||
workloads/RandomSelector.actor.cpp
|
||||
workloads/ReadWrite.actor.cpp
|
||||
workloads/RemoveServersSafely.actor.cpp
|
||||
workloads/ReportConflictingKeys.actor.cpp
|
||||
workloads/Rollback.actor.cpp
|
||||
workloads/RyowCorrectness.actor.cpp
|
||||
workloads/RYWDisable.actor.cpp
|
||||
|
@ -705,7 +705,9 @@ ACTOR Future<Void> commitBatch(
|
||||
// Determine which transactions actually committed (conservatively) by combining results from the resolvers
|
||||
state vector<uint8_t> committed(trs.size());
|
||||
ASSERT(transactionResolverMap.size() == committed.size());
|
||||
vector<int> nextTr(resolution.size());
|
||||
// For each commitTransactionRef, it is only sent to resolvers specified in transactionResolverMap
|
||||
// Thus, we use this nextTr to track the correct transaction index on each resolver.
|
||||
state vector<int> nextTr(resolution.size());
|
||||
for (int t = 0; t<trs.size(); t++) {
|
||||
uint8_t commit = ConflictBatch::TransactionCommitted;
|
||||
for (int r : transactionResolverMap[t])
|
||||
@ -1019,6 +1021,8 @@ ACTOR Future<Void> commitBatch(
|
||||
|
||||
// Send replies to clients
|
||||
double endTime = timer();
|
||||
// Reset all to zero, used to track the correct index of each commitTransacitonRef on each resolver
|
||||
std::fill(nextTr.begin(), nextTr.end(), 0);
|
||||
for (int t = 0; t < trs.size(); t++) {
|
||||
if (committed[t] == ConflictBatch::TransactionCommitted && (!locked || trs[t].isLockAware())) {
|
||||
ASSERT_WE_THINK(commitVersion != invalidVersion);
|
||||
@ -1029,15 +1033,16 @@ ACTOR Future<Void> commitBatch(
|
||||
}
|
||||
else {
|
||||
// If enable the option to report conflicting keys from resolvers, we union all conflicting key ranges here and send back through CommitID
|
||||
if (trs[t].isReportConflictingKeys()) {
|
||||
if (trs[t].transaction.report_conflicting_keys) {
|
||||
Standalone<VectorRef<KeyRangeRef>> conflictingEntries;
|
||||
std::vector<KeyRange> unmergedConflictingKRs;
|
||||
for (int resolverInd = 0; resolverInd < resolution.size(); ++resolverInd){
|
||||
for (const KeyRangeRef & kr : resolution[resolverInd].conflictingKeyRangeMap[t]) {
|
||||
for (int resolverInd : transactionResolverMap[t]) {
|
||||
for (const KeyRangeRef & kr : resolution[resolverInd].conflictingKeyRangeMap[nextTr[resolverInd]]){
|
||||
unmergedConflictingKRs.emplace_back(kr);
|
||||
// TraceEvent("ConflictingKeyRange").detail("ResolverIndex", resolverInd).detail("TrasactionIndex", t).detail("StartKey", kr.begin.toString()).detail("EndKey", kr.end.toString());
|
||||
}
|
||||
}
|
||||
ASSERT(unmergedConflictingKRs.size());
|
||||
// Sort the keyranges by begin key, then union overlap ranges from left to right
|
||||
std::sort(unmergedConflictingKRs.begin(), unmergedConflictingKRs.end(), [](KeyRange a, KeyRange b){
|
||||
return a.begin < b.begin;
|
||||
@ -1056,10 +1061,15 @@ ACTOR Future<Void> commitBatch(
|
||||
}
|
||||
conflictingEntries.push_back_deep(conflictingEntries.arena(), curr);
|
||||
trs[t].reply.send(CommitID(invalidVersion, t, Optional<Value>(), Optional<Standalone<VectorRef<KeyRangeRef>>>(conflictingEntries)));
|
||||
} else
|
||||
} else {
|
||||
trs[t].reply.sendError(not_committed());
|
||||
}
|
||||
}
|
||||
|
||||
// Update corresponding transaction index on each resolver
|
||||
for (int resolverInd : transactionResolverMap[t])
|
||||
nextTr[resolverInd]++;
|
||||
|
||||
// TODO: filter if pipelined with large commit
|
||||
if(self->latencyBandConfig.present()) {
|
||||
bool filter = maxTransactionBytes > self->latencyBandConfig.get().commitConfig.maxCommitBytes.orDefault(std::numeric_limits<int>::max());
|
||||
|
@ -77,7 +77,7 @@ struct ResolveTransactionBatchReply {
|
||||
VectorRef<uint8_t> committed;
|
||||
Optional<UID> debugID;
|
||||
VectorRef<VectorRef<StateTransactionRef>> stateMutations; // [version][transaction#] -> (committed, [mutation#])
|
||||
std::map<int, Standalone<VectorRef<KeyRangeRef>>> conflictingKeyRangeMap; // resolver index -> conflicting keyRanges given by that resolver
|
||||
std::map<int, Standalone<VectorRef<KeyRangeRef>>> conflictingKeyRangeMap; // transaction index -> conflicting keyRanges given by the resolver
|
||||
|
||||
template <class Archive>
|
||||
void serialize(Archive& ar) {
|
||||
|
@ -169,6 +169,7 @@
|
||||
<ActorCompiler Include="workloads\SnapTest.actor.cpp" />
|
||||
<ActorCompiler Include="workloads\Mako.actor.cpp" />
|
||||
<ActorCompiler Include="workloads\ExternalWorkload.actor.cpp" />
|
||||
<ActorCompiler Include="workloads\ReportConflictingKeys.actor.cpp" />
|
||||
</ItemGroup>
|
||||
<ItemGroup>
|
||||
<ClInclude Include="ApplyMetadataMutation.h" />
|
||||
|
@ -298,6 +298,9 @@
|
||||
<ActorCompiler Include="workloads\Mako.actor.cpp">
|
||||
<Filter>workloads</Filter>
|
||||
</ActorCompiler>
|
||||
<ActorCompiler Include="workloads\ReportConflictingKeys.actor.cpp">
|
||||
<Filter>workloads</Filter>
|
||||
</ActorCompiler>
|
||||
<ActorCompiler Include="OldTLogServer.actor.cpp" />
|
||||
</ItemGroup>
|
||||
<ItemGroup>
|
||||
|
@ -14,7 +14,7 @@ constexpr int RANGELIMIT = 10000;
|
||||
struct MakoWorkload : TestWorkload {
|
||||
uint64_t rowCount, seqNumLen, sampleSize, actorCountPerClient, keyBytes, maxValueBytes, minValueBytes;
|
||||
double testDuration, loadTime, warmingDelay, maxInsertRate, transactionsPerSecond, allowedLatency, periodicLoggingInterval, zipfConstant;
|
||||
bool enableLogging, commitGet, populateData, runBenchmark, preserveData, zipf, reportConflictingKeys;
|
||||
bool enableLogging, commitGet, populateData, runBenchmark, preserveData, zipf;
|
||||
PerfIntCounter xacts, retries, conflicts, commits, totalOps;
|
||||
std::vector<PerfIntCounter> opCounters;
|
||||
std::vector<uint64_t> insertionCountsToMeasure;
|
||||
@ -61,8 +61,6 @@ struct MakoWorkload : TestWorkload {
|
||||
// If true, the workload will picking up keys which are zipfian distributed
|
||||
zipf = getOption(options, LiteralStringRef("zipf"), false);
|
||||
zipfConstant = getOption(options, LiteralStringRef("zipfConstant"), 0.99);
|
||||
// If true, return key ranges conflicting with other txs
|
||||
reportConflictingKeys = getOption(options, LiteralStringRef("reportConflictingKeys"), false);
|
||||
// Specified length of keys and length range of values
|
||||
keyBytes = std::max( getOption( options, LiteralStringRef("keyBytes"), 16 ), 16);
|
||||
maxValueBytes = getOption( options, LiteralStringRef("valueBytes"), 16 );
|
||||
@ -292,8 +290,6 @@ struct MakoWorkload : TestWorkload {
|
||||
// used for throttling
|
||||
wait(poisson(&lastTime, delay));
|
||||
try{
|
||||
if (self->reportConflictingKeys)
|
||||
tr.setOption(FDBTransactionOptions::REPORT_CONFLICTING_KEYS);
|
||||
// user-defined value: whether commit read-only ops or not; default is false
|
||||
doCommit = self->commitGet;
|
||||
for (i = 0; i < MAX_OP; ++i) {
|
||||
@ -398,19 +394,11 @@ struct MakoWorkload : TestWorkload {
|
||||
TraceEvent("FailedToExecOperations").error(e);
|
||||
if (e.code() == error_code_operation_cancelled)
|
||||
throw;
|
||||
else if (e.code() == error_code_not_committed){
|
||||
else if (e.code() == error_code_not_committed)
|
||||
++self->conflicts;
|
||||
}
|
||||
|
||||
wait(tr.onError(e));
|
||||
++self->retries;
|
||||
if (self->reportConflictingKeys && deterministicRandom()->random01() < 0.01){
|
||||
// Standalone<VectorRef<KeyRangeRef>> rCKs;
|
||||
// tr.extractConflictingKeys(rCKs);
|
||||
// TraceEvent("ReportConflictingKeys").detail("KeySize", rCKs.size()).detail("KeyStart", rCKs[0].begin.toString()).detail("KeyEnd", rCKs[0].end.toString());
|
||||
Optional<Standalone<StringRef>> temp = wait(tr.get(LiteralStringRef("\xff\xff/conflicting_keys/json")));
|
||||
TraceEvent("ReportConflictingKeys").detail("Log", temp.present() ? temp.get().toString() : "DEBUG_NO_KEYS");
|
||||
}
|
||||
}
|
||||
// reset all the operations' counters to 0
|
||||
std::fill(perOpCount.begin(), perOpCount.end(), 0);
|
||||
|
188
fdbserver/workloads/ReportConflictingKeys.actor.cpp
Normal file
188
fdbserver/workloads/ReportConflictingKeys.actor.cpp
Normal file
@ -0,0 +1,188 @@
|
||||
/*
|
||||
* ReportConflictingKeys.actor.cpp
|
||||
*
|
||||
* This source file is part of the FoundationDB open source project
|
||||
*
|
||||
* Copyright 2013-2018 Apple Inc. and the FoundationDB project authors
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
#include "fdbclient/NativeAPI.actor.h"
|
||||
#include "fdbserver/TesterInterface.actor.h"
|
||||
#include "fdbserver/workloads/workloads.actor.h"
|
||||
#include "fdbserver/workloads/BulkSetup.actor.h"
|
||||
#include "fdbclient/ReadYourWrites.h"
|
||||
#include "flow/actorcompiler.h" // This must be the last #include.
|
||||
|
||||
struct ReportConflictingKeysWorkload : TestWorkload {
|
||||
|
||||
double testDuration, transactionsPerSecond, addReadConflictRangeProb, addWriteConflictRangeProb;
|
||||
Key keyPrefix;
|
||||
|
||||
int nodeCount, actorCount, keyBytes, valueBytes, readConflictRangeCount, writeConflictRangeCount;
|
||||
bool reportConflictingKeys, skipCorrectnessCheck;
|
||||
|
||||
PerfIntCounter invalidReports, commits, conflicts, retries, xacts;
|
||||
|
||||
ReportConflictingKeysWorkload(WorkloadContext const& wcx)
|
||||
: TestWorkload(wcx), invalidReports("InvalidReports"), conflicts("Conflicts"), retries("Retries"),
|
||||
commits("Commits"), xacts("Transactions") {
|
||||
testDuration = getOption(options, LiteralStringRef("testDuration"), 10.0);
|
||||
transactionsPerSecond = getOption(options, LiteralStringRef("transactionsPerSecond"), 5000.0) / clientCount;
|
||||
actorCount = getOption(options, LiteralStringRef("actorsPerClient"), transactionsPerSecond / 5);
|
||||
nodeCount = getOption(options, LiteralStringRef("nodeCount"), transactionsPerSecond * clientCount);
|
||||
keyPrefix = unprintable(
|
||||
getOption(options, LiteralStringRef("keyPrefix"), LiteralStringRef("ReportConflictingKeysWorkload"))
|
||||
.toString());
|
||||
keyBytes = getOption(options, LiteralStringRef("keyBytes"), 16);
|
||||
ASSERT(keyPrefix.size() + 16 <= keyBytes); // make sure the string format is valid
|
||||
readConflictRangeCount = getOption(options, LiteralStringRef("readConflictRangeCountPerTx"), 1);
|
||||
writeConflictRangeCount = getOption(options, LiteralStringRef("writeConflictRangeCountPerTx"), 1);
|
||||
// modeled by geometric distribution: (1 - prob) / prob = mean
|
||||
addReadConflictRangeProb = readConflictRangeCount / (readConflictRangeCount + 1.0);
|
||||
addWriteConflictRangeProb = writeConflictRangeCount / (writeConflictRangeCount + 1.0);
|
||||
// If true, store key ranges conflicting with other txs
|
||||
reportConflictingKeys = getOption(options, LiteralStringRef("reportConflictingKeys"), false);
|
||||
skipCorrectnessCheck = getOption(options, LiteralStringRef("skipCorrectnessCheck"), false);
|
||||
}
|
||||
|
||||
std::string description() override { return "ReportConflictingKeysWorkload"; }
|
||||
|
||||
Future<Void> setup(Database const& cx) override { return Void(); }
|
||||
|
||||
Future<Void> start(const Database& cx) override { return _start(cx->clone(), this); }
|
||||
|
||||
ACTOR Future<Void> _start(Database cx, ReportConflictingKeysWorkload* self) {
|
||||
std::vector<Future<Void>> clients;
|
||||
for (int c = 0; c < self->actorCount; ++c) {
|
||||
clients.push_back(self->conflictingClient(cx, self, self->actorCount / self->transactionsPerSecond, c));
|
||||
}
|
||||
|
||||
wait(timeout(waitForAll(clients), self->testDuration, Void()));
|
||||
return Void();
|
||||
}
|
||||
|
||||
Future<bool> check(Database const& cx) override { return invalidReports.getValue() == 0; }
|
||||
|
||||
void getMetrics(vector<PerfMetric>& m) override {
|
||||
m.push_back(PerfMetric("Measured Duration", testDuration, true));
|
||||
m.push_back(xacts.getMetric());
|
||||
m.push_back(PerfMetric("Transactions/sec", xacts.getValue() / testDuration, true));
|
||||
m.push_back(commits.getMetric());
|
||||
m.push_back(PerfMetric("Commits/sec", commits.getValue() / testDuration, true));
|
||||
m.push_back(conflicts.getMetric());
|
||||
m.push_back(PerfMetric("Conflicts/sec", conflicts.getValue() / testDuration, true));
|
||||
m.push_back(retries.getMetric());
|
||||
m.push_back(PerfMetric("Retries/sec", retries.getValue() / testDuration, true));
|
||||
}
|
||||
|
||||
// disable the default timeout setting
|
||||
double getCheckTimeout() override { return std::numeric_limits<double>::max(); }
|
||||
|
||||
// Copied from tester.actor.cpp, added parameter to determine the key's length
|
||||
Key keyForIndex(int n) {
|
||||
double p = (double)n / nodeCount;
|
||||
int paddingLen = keyBytes - 16 - keyPrefix.size();
|
||||
// left padding by zero
|
||||
return StringRef(format("%0*llx", paddingLen, *(uint64_t*)&p)).withPrefix(keyPrefix);
|
||||
}
|
||||
|
||||
void addRandomReadConflictRange(ReadYourWritesTransaction* tr, std::vector<KeyRange>& readConflictRanges) {
|
||||
int startIdx, endIdx;
|
||||
Key startKey, endKey;
|
||||
while (deterministicRandom()->random01() < addReadConflictRangeProb) {
|
||||
startIdx = deterministicRandom()->randomInt(0, nodeCount);
|
||||
endIdx = deterministicRandom()->randomInt(startIdx, nodeCount);
|
||||
startKey = keyForIndex(startIdx);
|
||||
endKey = keyForIndex(endIdx);
|
||||
tr->addReadConflictRange(KeyRangeRef(startKey, endKey));
|
||||
readConflictRanges.push_back(KeyRangeRef(startKey, endKey));
|
||||
}
|
||||
}
|
||||
|
||||
void addRandomWriteConflictRange(ReadYourWritesTransaction* tr) {
|
||||
int startIdx, endIdx;
|
||||
Key startKey, endKey;
|
||||
while (deterministicRandom()->random01() < addWriteConflictRangeProb) {
|
||||
startIdx = deterministicRandom()->randomInt(0, nodeCount);
|
||||
endIdx = deterministicRandom()->randomInt(startIdx, nodeCount);
|
||||
startKey = keyForIndex(startIdx);
|
||||
endKey = keyForIndex(endIdx);
|
||||
tr->addWriteConflictRange(KeyRangeRef(startKey, endKey));
|
||||
}
|
||||
}
|
||||
|
||||
ACTOR Future<Void> conflictingClient(Database cx, ReportConflictingKeysWorkload* self, double delay,
|
||||
int actorIndex) {
|
||||
|
||||
state ReadYourWritesTransaction tr(cx);
|
||||
state double lastTime = now();
|
||||
state std::vector<KeyRange> readConflictRanges;
|
||||
|
||||
loop {
|
||||
try {
|
||||
// used for throttling
|
||||
wait(poisson(&lastTime, delay));
|
||||
if (self->reportConflictingKeys) tr.setOption(FDBTransactionOptions::REPORT_CONFLICTING_KEYS);
|
||||
self->addRandomReadConflictRange(&tr, readConflictRanges);
|
||||
self->addRandomWriteConflictRange(&tr);
|
||||
++self->commits;
|
||||
wait(tr.commit());
|
||||
++self->xacts;
|
||||
} catch (Error& e) {
|
||||
TraceEvent("FailedToCommitTx").error(e);
|
||||
state bool isConflict = false;
|
||||
if (e.code() == error_code_operation_cancelled)
|
||||
throw;
|
||||
else if (e.code() == error_code_not_committed) {
|
||||
++self->conflicts;
|
||||
isConflict = true;
|
||||
}
|
||||
wait(tr.onError(e));
|
||||
// check API correctness
|
||||
if (!self->skipCorrectnessCheck && self->reportConflictingKeys && isConflict) {
|
||||
Optional<Standalone<StringRef>> temp =
|
||||
wait(tr.get(LiteralStringRef("\xff\xff/conflicting_keys/json")));
|
||||
auto jsonStr = temp.get().toString();
|
||||
json_spirit::mValue val;
|
||||
if (json_spirit::read_string(jsonStr, val)) {
|
||||
auto root = val.get_array();
|
||||
ASSERT(root.size() > 0);
|
||||
// Only use the last entry which contains the read_conflict_ranges corresponding to current
|
||||
// conflicts
|
||||
for (const auto& pair : root.back().get_array()) {
|
||||
json_spirit::mObject kr_obj = pair.get_obj();
|
||||
KeyRange kr = KeyRangeRef(kr_obj["begin"].get_str(), kr_obj["end"].get_str());
|
||||
if (!std::any_of(readConflictRanges.begin(), readConflictRanges.end(), [&kr](KeyRange rCR) {
|
||||
// Read_conflict_range remains same in the resolver.
|
||||
// Thus, the returned keyrange is either the original read_conflict_range or merged
|
||||
// by several overlapped ones In either case, it contains at least one original
|
||||
// read_conflict_range
|
||||
return kr.contains(rCR);
|
||||
})) {
|
||||
++self->invalidReports;
|
||||
TraceEvent(SevError, "TestFailure").detail("Reason", "InvalidKeyRangeReturned");
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
++self->retries;
|
||||
}
|
||||
readConflictRanges.clear();
|
||||
tr.reset();
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
WorkloadFactory<ReportConflictingKeysWorkload> ReportConflictingKeysWorkload("ReportConflictingKeys");
|
@ -65,6 +65,7 @@ add_fdb_test(TEST_FILES ReadHalfAbsent.txt IGNORE)
|
||||
add_fdb_test(TEST_FILES RedwoodCorrectnessUnits.txt IGNORE)
|
||||
add_fdb_test(TEST_FILES RedwoodCorrectnessBTree.txt IGNORE)
|
||||
add_fdb_test(TEST_FILES RedwoodCorrectnessPager.txt IGNORE)
|
||||
add_fdb_test(TEST_FILES ReportConflictingKeys.txt IGNORE)
|
||||
add_fdb_test(TEST_FILES fast/RedwoodCorrectnessBTree.txt IGNORE)
|
||||
add_fdb_test(TEST_FILES RedwoodCorrectness.txt IGNORE)
|
||||
add_fdb_test(TEST_FILES RedwoodPerfTests.txt IGNORE)
|
||||
|
@ -1,15 +1,15 @@
|
||||
testTitle=MakoTest
|
||||
testName=Mako
|
||||
testDuration=20.0
|
||||
testDuration=10.0
|
||||
transactionsPerSecond=100000
|
||||
rows=1000000
|
||||
sampleSize=100
|
||||
valueBytes=16
|
||||
keyBytes=16
|
||||
operations=u8
|
||||
operations=g5gr5:10i5ir5:10grv5
|
||||
actorCountPerClient=256
|
||||
enableLogging=false
|
||||
commitGet=false
|
||||
populateData=true
|
||||
runBenchmark=true
|
||||
preserveData=false
|
||||
preserveData=true
|
||||
|
11
tests/ReportConflictingKeys.txt
Normal file
11
tests/ReportConflictingKeys.txt
Normal file
@ -0,0 +1,11 @@
|
||||
testTitle=ReportConflictingKeysTest
|
||||
testName=ReportConflictingKeys
|
||||
testDuration=10.0
|
||||
transactionsPerSecond=100000
|
||||
nodeCount=10000
|
||||
actorsPerClient=16
|
||||
keyPrefix=RCK
|
||||
keyBytes=64
|
||||
readConflictRangeCountPerTx=1
|
||||
writeConflictRangeCountPerTx=1
|
||||
reportConflictingKeys=true
|
Loading…
x
Reference in New Issue
Block a user