Add a simple workload, ReportConflictingKeysWorkload, to test correctness of the API and performance overhead added to the resovler.

This commit is contained in:
chaoguang 2019-11-26 00:25:37 -08:00
parent edf52e8c97
commit eacfdf6da3
16 changed files with 241 additions and 43 deletions

View File

@ -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);

View File

@ -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);
}

View File

@ -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;

View File

@ -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();
}

View File

@ -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>();

View File

@ -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">

View File

@ -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

View File

@ -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());

View File

@ -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) {

View File

@ -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" />

View File

@ -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>

View File

@ -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);

View 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");

View File

@ -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)

View File

@ -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

View 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