From bc757f3e4b3708f6057c97fba034a2ce2c4c876e Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Thu, 4 Mar 2021 19:14:54 -0800 Subject: [PATCH 01/72] basic framework for range feed support --- fdbcli/fdbcli.actor.cpp | 38 ++++++++++++++++++++ fdbclient/DatabaseContext.h | 2 ++ fdbclient/NativeAPI.actor.cpp | 49 ++++++++++++++++++++++++++ fdbclient/NativeAPI.actor.h | 2 ++ fdbclient/StorageServerInterface.h | 39 +++++++++++++++++++++ fdbclient/SystemData.cpp | 19 ++++++++++ fdbclient/SystemData.h | 6 ++++ fdbserver/ApplyMetadataMutation.cpp | 22 ++++++++++++ fdbserver/storageserver.actor.cpp | 54 ++++++++++++++++++++++++++--- flow/ProtocolVersion.h | 1 + 10 files changed, 228 insertions(+), 4 deletions(-) diff --git a/fdbcli/fdbcli.actor.cpp b/fdbcli/fdbcli.actor.cpp index 12a6a0a11d..1d7689bc10 100644 --- a/fdbcli/fdbcli.actor.cpp +++ b/fdbcli/fdbcli.actor.cpp @@ -617,6 +617,10 @@ void initHelp() { helpMap["triggerddteaminfolog"] = CommandHelp("triggerddteaminfolog", "trigger the data distributor teams logging", "Trigger the data distributor to log detailed information about its teams."); + helpMap["rangefeed"] = CommandHelp( + "rangefeed ", + "", + ""); hiddenCommands.insert("expensive_data_check"); hiddenCommands.insert("datadistribution"); @@ -3267,6 +3271,40 @@ ACTOR Future cli(CLIOptions opt, LineNoise* plinenoise) { continue; } + if (tokencmp(tokens[0], "rangefeed")) { + if(tokens.size() == 1) { + printUsage(tokens[0]); + is_error = true; + continue; + } + if(tokencmp(tokens[1], "register")) { + if(tokens.size() != 5) { + printUsage(tokens[0]); + is_error = true; + continue; + } + state Transaction trx(db); + loop { + try { + wait(trx.registerRangeFeed(tokens[2], KeyRangeRef(tokens[3], tokens[4]))); + wait(trx.commit()); + } catch( Error &e ) { + wait(trx.onError(e)); + } + } + } else if(tokencmp(tokens[1], "get")) { + if(tokens.size() != 3) { + printUsage(tokens[0]); + is_error = true; + continue; + } + Standalone res = wait(db->getRangeFeedMutations(tokens[2])); + for(auto& it : res) { + printf("%lld %s\n", it.version, it.mutation.toString().c_str()); + } + } + } + if (tokencmp(tokens[0], "configure")) { bool err = wait(configure(db, tokens, db->getConnectionFile(), &linenoise, warn)); if (err) is_error = true; diff --git a/fdbclient/DatabaseContext.h b/fdbclient/DatabaseContext.h index 831bf5d80a..dd4ae2d186 100644 --- a/fdbclient/DatabaseContext.h +++ b/fdbclient/DatabaseContext.h @@ -236,6 +236,8 @@ public: // Management API, create snapshot Future createSnapshot(StringRef uid, StringRef snapshot_command); + Future> getRangeFeedMutations(StringRef rangeID); + //private: explicit DatabaseContext( Reference>> connectionFile, Reference> clientDBInfo, Future clientInfoMonitor, TaskPriority taskID, LocalityData const& clientLocality, diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index 32e6df4acf..20294fe4bd 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -3093,6 +3093,24 @@ Future< Standalone< VectorRef< const char*>>> Transaction::getAddressesForKey( c return getAddressesForKeyActor(key, ver, cx, info, options); } +ACTOR Future registerRangeFeedActor(StringRef rangeID, KeyRangeRef range, Future ver, Database cx, + TransactionInfo info, + TransactionOptions options) { + state Key rangeIDKey = rangeID.withPrefix(rangeFeedPrefix); + Optional val = wait( getValue(ver, rangeIDKey, cx, info, trLogInfo, options.readTags) ); + if(!val.present()) { + set(rangeIDKey, rangeFeedValue(range)); + } else if(decodeRangeFeedValue(val.get()) != range) { + throw unsupported_operation(); + } + return Void(); +} + +Future Transaction::registerRangeFeed( const StringRef& rangeID, const KeyRangeRef& range ) { + auto ver = getReadVersion(); + return registerRangeFeedActor(rangeID, range, ver, cx, info, options); +} + ACTOR Future< Key > getKeyAndConflictRange( Database cx, KeySelector k, Future version, Promise> conflictRange, TransactionInfo info, TagSet tags) { @@ -5025,3 +5043,34 @@ Future DatabaseContext::createSnapshot(StringRef uid, } return createSnapshotActor(this, UID::fromString(uid_str), snapshot_command); } + +ACTOR Future>> getRangeFeedMutationsActor(Reference db, StringRef rangeID) { + state Database cx(db); + state Transaction tr(cx); + state Key rangeIDKey = rangeID.withPrefix(rangeFeedPrefix); + state Span span("NAPI:GetRangeFeedMutations"_loc); + Optional val = wait( tr.get(rangeIDKey) ); + if(!val.present()) { + throw unsupported_operation(); + } + KeyRange keys = decodeRangeFeedValue(val.get()); + state vector< pair> > locations = wait( getKeyRangeLocations( cx, keys, 100, + false, &StorageServerInterface::rangeFeed, TransactionInfo(TaskPriority::DefaultEndpoint, span.context) ) ); + + if(locations.size() > 1) { + throw unsupported_operation(); + } + + state RangeFeedRequest req; + req.rangeID = rangeID; + + RangeFeedReply rep = + wait(loadBalance(cx.getPtr(), locations[0].second, &StorageServerInterface::rangeFeed, req, + TaskPriority::DefaultPromiseEndpoint, false, + cx->enableLocalityLoadBalance ? &cx->queueModel : nullptr)); + return Standalone>(rep.mutations, rep.arena); +} + +Future>> DatabaseContext::getRangeFeedMutations(StringRef rangeID) { + return getRangeFeedMutationsActor(Reference::addRef(this), rangeID); +} diff --git a/fdbclient/NativeAPI.actor.h b/fdbclient/NativeAPI.actor.h index 1be191e16c..385f106ca3 100644 --- a/fdbclient/NativeAPI.actor.h +++ b/fdbclient/NativeAPI.actor.h @@ -255,6 +255,8 @@ public: [[nodiscard]] Future>> getAddressesForKey(const Key& key); + Future registerRangeFeed(const Key& rangeID, const KeyRange& range); + void enableCheckWrites(); void addReadConflictRange( KeyRangeRef const& keys ); void addWriteConflictRange( KeyRangeRef const& keys ); diff --git a/fdbclient/StorageServerInterface.h b/fdbclient/StorageServerInterface.h index e6a4e18f89..8c74e554e3 100644 --- a/fdbclient/StorageServerInterface.h +++ b/fdbclient/StorageServerInterface.h @@ -73,6 +73,7 @@ struct StorageServerInterface { RequestStream watchValue; RequestStream getReadHotRanges; RequestStream getRangeSplitPoints; + RequestStream rangeFeed; explicit StorageServerInterface(UID uid) : uniqueID( uid ) {} StorageServerInterface() : uniqueID( deterministicRandom()->randomUniqueID() ) {} @@ -101,6 +102,7 @@ struct StorageServerInterface { watchValue = RequestStream( getValue.getEndpoint().getAdjustedEndpoint(10) ); getReadHotRanges = RequestStream( getValue.getEndpoint().getAdjustedEndpoint(11) ); getRangeSplitPoints = RequestStream(getValue.getEndpoint().getAdjustedEndpoint(12)); + rangeFeed = RequestStream(getValue.getEndpoint().getAdjustedEndpoint(13)); } } else { ASSERT(Ar::isDeserializing); @@ -129,6 +131,7 @@ struct StorageServerInterface { streams.push_back(watchValue.getReceiver()); streams.push_back(getReadHotRanges.getReceiver()); streams.push_back(getRangeSplitPoints.getReceiver()); + streams.push_back(rangeFeed.getReceiver()); FlowTransport::transport().addEndpoints(streams); } }; @@ -516,6 +519,42 @@ struct SplitRangeRequest { } }; +struct MutationRefAndVersion { + MutationRef mutation; + Version version; + + MutationRefAndVersion(MutationRef mutation, Version version, Arena arena) : mutation(mutation), version(version) {} + + template + void serialize(Ar& ar) { + serializer(ar, mutation, version); + } +}; + +struct RangeFeedReply { + constexpr static FileIdentifier file_identifier = 11815134; + VectorRef mutations; + Arena arena; + + template + void serialize(Ar& ar) { + serializer(ar, mutations, arena); + } +}; +struct RangeFeedRequest { + constexpr static FileIdentifier file_identifier = 10726174; + Key rangeID; + ReplyPromise reply; + + RangeFeedRequest() {} + RangeFeedRequest(Key const& rangeID) : rangeID(rangeID) {} + + template + void serialize(Ar& ar) { + serializer(ar, rangeID, reply); + } +}; + struct GetStorageMetricsReply { constexpr static FileIdentifier file_identifier = 15491478; StorageMetrics load; diff --git a/fdbclient/SystemData.cpp b/fdbclient/SystemData.cpp index 54bb0003e4..94f8e3293b 100644 --- a/fdbclient/SystemData.cpp +++ b/fdbclient/SystemData.cpp @@ -1062,3 +1062,22 @@ const KeyRangeRef testOnlyTxnStateStorePrefixRange( const KeyRef writeRecoveryKey = LiteralStringRef("\xff/writeRecovery"); const ValueRef writeRecoveryKeyTrue = LiteralStringRef("1"); const KeyRef snapshotEndVersionKey = LiteralStringRef("\xff/snapshotEndVersion"); + +const KeyRangeRef rangeFeedKeys( + LiteralStringRef("\xff\x02/feed/"), + LiteralStringRef("\xff\x02/feed0") +); +const KeyRef rangeFeedPrefix = rangeFeedKeys.begin; +const KeyRef rangeFeedPrivatePrefix = LiteralStringRef("\xff\xff\x02/feed/"); + +const Value rangeFeedValue( KeyRangeRef const& range ) { + BinaryWriter wr(IncludeVersion(ProtocolVersion::withRangeFeed())); + wr << range; + return wr.toValue(); +} +KeyRange decodeFeedValue( ValueRef const& value ) { + KeyRange range; + BinaryReader reader( value, IncludeVersion() ); + reader >> range; + return range; +} diff --git a/fdbclient/SystemData.h b/fdbclient/SystemData.h index 4a5c5c5a19..89e8ff3bf0 100644 --- a/fdbclient/SystemData.h +++ b/fdbclient/SystemData.h @@ -461,6 +461,12 @@ extern const ValueRef writeRecoveryKeyTrue; // Allows incremental restore to read and set starting version for consistency. extern const KeyRef snapshotEndVersionKey; +extern const KeyRangeRef rangeFeedKeys; +const Value rangeFeedValue( KeyRangeRef const& range ); +KeyRange decodeRangeFeedValue( ValueRef const& value ); +extern const KeyRef rangeFeedPrefix; +extern const KeyRef rangeFeedPrivatePrefix; + #pragma clang diagnostic pop #endif diff --git a/fdbserver/ApplyMetadataMutation.cpp b/fdbserver/ApplyMetadataMutation.cpp index e6cabc8d1a..cf804870af 100644 --- a/fdbserver/ApplyMetadataMutation.cpp +++ b/fdbserver/ApplyMetadataMutation.cpp @@ -308,6 +308,28 @@ void applyMetadataMutations(SpanID const& spanContext, UID const& dbgid, Arena& TraceEvent("WriteRecoveryKeySet", dbgid); if (!initialCommit) txnStateStore->set(KeyValueRef(m.param1, m.param2)); TEST(true); // Snapshot created, setting writeRecoveryKey in txnStateStore + } else if (m.param1.startsWith(rangeFeedPrefix)) { + if(toCommit && keyInfo) { + KeyRange r = decodeRangeFeedValue( m.param2 ); + MutationRef privatized = m; + privatized.param1 = m.param1.withPrefix(systemKeys.begin, arena); + auto ranges = keyInfo->intersectingRanges(r); + auto firstRange = ranges.begin(); + ++firstRange; + if (firstRange == ranges.end()) { + ranges.begin().value().populateTags(); + toCommit->addTags(ranges.begin().value().tags); + } + else { + std::set allSources; + for (auto r : ranges) { + r.value().populateTags(); + allSources.insert(r.value().tags.begin(), r.value().tags.end()); + } + toCommit->addTags(allSources); + } + toCommit->writeTypedMessage(privatized); + } } } else if (m.param2.size() > 1 && m.param2[0] == systemKeys.begin[0] && m.type == MutationRef::ClearRange) { KeyRangeRef range(m.param1, m.param2); diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index d111ef076e..70b08a594c 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -259,6 +259,12 @@ struct FetchInjectionInfo { vector changes; }; +struct RangeFeedInfo : ReferenceCounted { + std::deque mutations; + KeyRange range; + Key id; +}; + struct StorageServer { typedef VersionedMap VersionedData; @@ -436,7 +442,9 @@ public: KeyRangeMap< Reference > shards; uint64_t shardChangeCounter; // max( shards->changecounter ) - KeyRangeMap cachedRangeMap; // indicates if a key-range is being cached + KeyRangeMap cachedRangeMap; // indicates if a key-range is being cached + KeyRangeMap>> keyRangeFeed; + std::unordered_map> uidRangeFeed; // newestAvailableVersion[k] // == invalidVersion -> k is unavailable at all versions @@ -492,7 +500,6 @@ public: FlowLock durableVersionLock; FlowLock fetchKeysParallelismLock; vector< Promise > readyFetchKeys; - int64_t instanceID; Promise otherError; @@ -1206,6 +1213,16 @@ ACTOR Future watchValueQ( StorageServer* data, WatchValueRequest req ) { } } +ACTOR Future rangeFeedQ( StorageServer* data, RangeFeedRequest req ) { + wait(delay(0)); + RangeFeedReply reply; + for(auto& it : data->uidRangeFeed[req.rangeID]->mutations) { + reply.mutations.push_back(reply.arena, it); + } + req.reply.send(reply); + return Void(); +} + ACTOR Future getShardState_impl( StorageServer* data, GetShardStateRequest req ) { ASSERT( req.mode != GetShardStateRequest::NO_WAIT ); @@ -2001,7 +2018,7 @@ bool expandMutation( MutationRef& m, StorageServer::VersionedData const& data, U return true; } -void applyMutation( StorageServer *self, MutationRef const& m, Arena& arena, StorageServer::VersionedData &data ) { +void applyMutation( StorageServer *self, MutationRef const& m, Arena& arena, StorageServer::VersionedData &data, Version version ) { // m is expected to be in arena already // Clear split keys are added to arena StorageMetrics metrics; @@ -2027,12 +2044,21 @@ void applyMutation( StorageServer *self, MutationRef const& m, Arena& arena, Sto } data.insert( m.param1, ValueOrClearToRef::value(m.param2) ); self->watches.trigger( m.param1 ); + + for(auto& it : self->keyRangeFeed[m.param1]) { + it->mutations.emplace_back(m,version); + } } else if (m.type == MutationRef::ClearRange) { data.erase( m.param1, m.param2 ); ASSERT( m.param2 > m.param1 ); ASSERT( !data.isClearContaining( data.atLatest(), m.param1 ) ); data.insert( m.param1, ValueOrClearToRef::clearTo(m.param2) ); self->watches.triggerRange( m.param1, m.param2 ); + + auto ranges = self->keyRangeFeed.intersectingRanges(KeyRangeRef(m.param1, m.param2)); + for(auto &it : ranges) { + it.value()->mutations.emplace_back(m,version); + } } } @@ -2720,7 +2746,7 @@ void StorageServer::addMutation(Version version, MutationRef const& mutation, Ke } expanded = addMutationToMutationLog(mLog, expanded); DEBUG_MUTATION("applyMutation", version, expanded).detail("UID", thisServerID).detail("ShardBegin", shard.begin).detail("ShardEnd", shard.end); - applyMutation( this, expanded, mLog.arena(), mutableData() ); + applyMutation( this, expanded, mLog.arena(), mutableData(), version ); //printf("\nSSUpdate: Printing versioned tree after applying mutation\n"); //mutableData().printTree(version); } @@ -2850,6 +2876,18 @@ private: data->primaryLocality = BinaryReader::fromStringRef(m.param2, Unversioned()); auto& mLV = data->addVersionToMutationLog( data->data().getLatestVersion() ); data->addMutationToMutationLog( mLV, MutationRef(MutationRef::SetValue, persistPrimaryLocality, m.param2) ); + } else if (m.type == MutationRef::SetValue && m.param1.startsWith(rangeFeedPrivatePrefix)) { + Key rangeFeedId = m.param1.removePrefix(rangeFeedPrivatePrefix); + KeyRange rangeFeedRange = decodeRangeFeedValue( m.param2 ); + Reference rangeFeedInfo( new RangeFeedInfo() ); + rangeFeedInfo->range = rangeFeedRange; + rangeFeedInfo->id = rangeFeedId; + data->uidRangeFeed[rangeFeedId] = rangeFeedInfo; + auto rs = data->keyRangeFeed.modify( rangeFeedRange ); + for(auto r = rs.begin(); r != rs.end(); ++r) { + r->value().push_back( rangeFeedInfo ); + } + data->uidRangeFeed.coalesce( rangeFeedRange ); } else { ASSERT(false); // Unknown private mutation } @@ -3900,6 +3938,13 @@ ACTOR Future serveWatchValueRequests( StorageServer* self, FutureStream serveRangeFeedRequests( StorageServer* self, FutureStream rangeFeed ) { + loop { + RangeFeedRequest req = waitNext(rangeFeed); + self->actors.add(self->readGuard(req, rangeFeedQ)); + } +} + ACTOR Future reportStorageServerState(StorageServer* self) { if (!SERVER_KNOBS->REPORT_DD_METRICS) { return Void(); @@ -3948,6 +3993,7 @@ ACTOR Future storageServerCore( StorageServer* self, StorageServerInterfac self->actors.add(serveGetKeyValuesRequests(self, ssi.getKeyValues.getFuture())); self->actors.add(serveGetKeyRequests(self, ssi.getKey.getFuture())); self->actors.add(serveWatchValueRequests(self, ssi.watchValue.getFuture())); + self->actors.add(serveRangeFeedRequests(self, ssi.rangeFeed.getFuture())); self->actors.add(traceRole(Role::STORAGE_SERVER, ssi.id())); self->actors.add(reportStorageServerState(self)); diff --git a/flow/ProtocolVersion.h b/flow/ProtocolVersion.h index 3ee31959ab..4578f40d39 100644 --- a/flow/ProtocolVersion.h +++ b/flow/ProtocolVersion.h @@ -132,6 +132,7 @@ public: // introduced features PROTOCOL_VERSION_FEATURE(0x0FDB00B070010000LL, StableInterfaces); PROTOCOL_VERSION_FEATURE(0x0FDB00B070010001LL, TagThrottleValueReason); PROTOCOL_VERSION_FEATURE(0x0FDB00B070010001LL, SpanContext); + PROTOCOL_VERSION_FEATURE(0x0FDB00B070010001LL, RangeFeed); }; // These impact both communications and the deserialization of certain database and IKeyValueStore keys. From 7c3403c3dd622f4222619bbebf13f8af8ec7cb05 Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Thu, 4 Mar 2021 19:29:18 -0800 Subject: [PATCH 02/72] added missing include --- fdbclient/StorageServerInterface.h | 1 + 1 file changed, 1 insertion(+) diff --git a/fdbclient/StorageServerInterface.h b/fdbclient/StorageServerInterface.h index 8c74e554e3..25b067cb8a 100644 --- a/fdbclient/StorageServerInterface.h +++ b/fdbclient/StorageServerInterface.h @@ -30,6 +30,7 @@ #include "fdbrpc/Stats.h" #include "fdbrpc/TimedRequest.h" #include "fdbclient/TagThrottle.h" +#include "fdbclient/CommitTransaction.h" // Dead code, removed in the next protocol version struct VersionReply { From bfc5ec92419afd6ead5992db9bc2def8993280a9 Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Thu, 4 Mar 2021 19:32:16 -0800 Subject: [PATCH 03/72] fixed mutationRefAnfVersion constructor --- fdbclient/StorageServerInterface.h | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/fdbclient/StorageServerInterface.h b/fdbclient/StorageServerInterface.h index 25b067cb8a..d128cdf45f 100644 --- a/fdbclient/StorageServerInterface.h +++ b/fdbclient/StorageServerInterface.h @@ -524,7 +524,8 @@ struct MutationRefAndVersion { MutationRef mutation; Version version; - MutationRefAndVersion(MutationRef mutation, Version version, Arena arena) : mutation(mutation), version(version) {} + MutationRefAndVersion() {} + MutationRefAndVersion(MutationRef mutation, Version version) : mutation(mutation), version(version) {} template void serialize(Ar& ar) { From 77769aabee9013386d0e67986efd9593b4b7beb3 Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Thu, 4 Mar 2021 19:33:32 -0800 Subject: [PATCH 04/72] fixed compile error --- fdbclient/DatabaseContext.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/fdbclient/DatabaseContext.h b/fdbclient/DatabaseContext.h index dd4ae2d186..863c382dcd 100644 --- a/fdbclient/DatabaseContext.h +++ b/fdbclient/DatabaseContext.h @@ -236,7 +236,7 @@ public: // Management API, create snapshot Future createSnapshot(StringRef uid, StringRef snapshot_command); - Future> getRangeFeedMutations(StringRef rangeID); + Future>> getRangeFeedMutations(StringRef rangeID); //private: explicit DatabaseContext( Reference>> connectionFile, Reference> clientDBInfo, From adaa67df6317af9fdb81aa1172d83a609a5239f0 Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Thu, 4 Mar 2021 19:42:36 -0800 Subject: [PATCH 05/72] fix registerRangeFeedActor --- fdbclient/NativeAPI.actor.cpp | 11 ++++------- 1 file changed, 4 insertions(+), 7 deletions(-) diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index 20294fe4bd..c6354b1fc3 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -3093,13 +3093,11 @@ Future< Standalone< VectorRef< const char*>>> Transaction::getAddressesForKey( c return getAddressesForKeyActor(key, ver, cx, info, options); } -ACTOR Future registerRangeFeedActor(StringRef rangeID, KeyRangeRef range, Future ver, Database cx, - TransactionInfo info, - TransactionOptions options) { +ACTOR Future registerRangeFeedActor(Transaction *tr, StringRef rangeID, KeyRangeRef range) { state Key rangeIDKey = rangeID.withPrefix(rangeFeedPrefix); - Optional val = wait( getValue(ver, rangeIDKey, cx, info, trLogInfo, options.readTags) ); + Optional val = wait( tr->get(rangeIDKey) ); if(!val.present()) { - set(rangeIDKey, rangeFeedValue(range)); + tr->set(rangeIDKey, rangeFeedValue(range)); } else if(decodeRangeFeedValue(val.get()) != range) { throw unsupported_operation(); } @@ -3107,8 +3105,7 @@ ACTOR Future registerRangeFeedActor(StringRef rangeID, KeyRangeRef range, } Future Transaction::registerRangeFeed( const StringRef& rangeID, const KeyRangeRef& range ) { - auto ver = getReadVersion(); - return registerRangeFeedActor(rangeID, range, ver, cx, info, options); + return registerRangeFeedActor(this, rangeID, range); } ACTOR Future< Key > getKeyAndConflictRange( From 21822e49498d9aee2f30c43149b60cafc59cb537 Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Thu, 4 Mar 2021 19:46:32 -0800 Subject: [PATCH 06/72] more fixes --- fdbclient/NativeAPI.actor.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index c6354b1fc3..13a06e25bd 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -3093,7 +3093,7 @@ Future< Standalone< VectorRef< const char*>>> Transaction::getAddressesForKey( c return getAddressesForKeyActor(key, ver, cx, info, options); } -ACTOR Future registerRangeFeedActor(Transaction *tr, StringRef rangeID, KeyRangeRef range) { +ACTOR Future registerRangeFeedActor(Transaction *tr, Key rangeID, KeyRange range) { state Key rangeIDKey = rangeID.withPrefix(rangeFeedPrefix); Optional val = wait( tr->get(rangeIDKey) ); if(!val.present()) { @@ -3104,7 +3104,7 @@ ACTOR Future registerRangeFeedActor(Transaction *tr, StringRef rangeID, Ke return Void(); } -Future Transaction::registerRangeFeed( const StringRef& rangeID, const KeyRangeRef& range ) { +Future Transaction::registerRangeFeed( const Key& rangeID, const KeyRange& range ) { return registerRangeFeedActor(this, rangeID, range); } From 83187f9c0e43c7379ae5803dcc22a134a7cd275e Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Thu, 4 Mar 2021 19:48:48 -0800 Subject: [PATCH 07/72] rangeID is a key --- fdbserver/storageserver.actor.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index 70b08a594c..4ea7f3f902 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -444,7 +444,7 @@ public: KeyRangeMap cachedRangeMap; // indicates if a key-range is being cached KeyRangeMap>> keyRangeFeed; - std::unordered_map> uidRangeFeed; + std::unordered_map> uidRangeFeed; // newestAvailableVersion[k] // == invalidVersion -> k is unavailable at all versions From d071f4871682a708c51822f25ee84125b09e4e49 Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Thu, 4 Mar 2021 19:53:24 -0800 Subject: [PATCH 08/72] added missing load balance field --- fdbclient/StorageServerInterface.h | 3 +++ 1 file changed, 3 insertions(+) diff --git a/fdbclient/StorageServerInterface.h b/fdbclient/StorageServerInterface.h index d128cdf45f..2fdaeaf080 100644 --- a/fdbclient/StorageServerInterface.h +++ b/fdbclient/StorageServerInterface.h @@ -536,8 +536,11 @@ struct MutationRefAndVersion { struct RangeFeedReply { constexpr static FileIdentifier file_identifier = 11815134; VectorRef mutations; + bool cached; Arena arena; + RangeFeedReply() : cached(false) {} + template void serialize(Ar& ar) { serializer(ar, mutations, arena); From b7df90ec66987d9f05b8eb923f7508253bf84e9b Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Thu, 4 Mar 2021 19:57:33 -0800 Subject: [PATCH 09/72] change to an ordered map for now --- fdbserver/storageserver.actor.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index 4ea7f3f902..63a9b58870 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -444,7 +444,7 @@ public: KeyRangeMap cachedRangeMap; // indicates if a key-range is being cached KeyRangeMap>> keyRangeFeed; - std::unordered_map> uidRangeFeed; + std::map> uidRangeFeed; // newestAvailableVersion[k] // == invalidVersion -> k is unavailable at all versions From 14ea90b34b39eff7f21f667e80393ffd594f34ee Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Thu, 4 Mar 2021 20:04:37 -0800 Subject: [PATCH 10/72] more compile fixes --- fdbserver/storageserver.actor.cpp | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index 63a9b58870..49155ed637 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -2056,8 +2056,10 @@ void applyMutation( StorageServer *self, MutationRef const& m, Arena& arena, Sto self->watches.triggerRange( m.param1, m.param2 ); auto ranges = self->keyRangeFeed.intersectingRanges(KeyRangeRef(m.param1, m.param2)); - for(auto &it : ranges) { - it.value()->mutations.emplace_back(m,version); + for(auto &r : ranges) { + for(auto& it : r.value()) { + it->mutations.emplace_back(m,version); + } } } @@ -2887,7 +2889,7 @@ private: for(auto r = rs.begin(); r != rs.end(); ++r) { r->value().push_back( rangeFeedInfo ); } - data->uidRangeFeed.coalesce( rangeFeedRange ); + data->keyRangeFeed.coalesce( rangeFeedRange ); } else { ASSERT(false); // Unknown private mutation } From 371f11861d8327364cd9217ea9eb59082ef946bb Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Thu, 4 Mar 2021 20:11:22 -0800 Subject: [PATCH 11/72] compile fixes --- fdbserver/storageserver.actor.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index 49155ed637..5a6b07e379 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -2889,7 +2889,7 @@ private: for(auto r = rs.begin(); r != rs.end(); ++r) { r->value().push_back( rangeFeedInfo ); } - data->keyRangeFeed.coalesce( rangeFeedRange ); + data->keyRangeFeed.coalesce( rangeFeedRange.contents() ); } else { ASSERT(false); // Unknown private mutation } From 0c72e5a9cb9186ac9baa3c34cec5bed6878d77b8 Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Thu, 4 Mar 2021 20:22:42 -0800 Subject: [PATCH 12/72] fixed name of function --- fdbclient/SystemData.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/fdbclient/SystemData.cpp b/fdbclient/SystemData.cpp index 94f8e3293b..9495bd6207 100644 --- a/fdbclient/SystemData.cpp +++ b/fdbclient/SystemData.cpp @@ -1075,7 +1075,7 @@ const Value rangeFeedValue( KeyRangeRef const& range ) { wr << range; return wr.toValue(); } -KeyRange decodeFeedValue( ValueRef const& value ) { +KeyRange decodeRangeFeedValue( ValueRef const& value ) { KeyRange range; BinaryReader reader( value, IncludeVersion() ); reader >> range; From 7d0a86395dbafdf48f6e4fb3563eb6e7a1fd6c35 Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Thu, 4 Mar 2021 20:31:16 -0800 Subject: [PATCH 13/72] compile fix --- fdbcli/fdbcli.actor.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/fdbcli/fdbcli.actor.cpp b/fdbcli/fdbcli.actor.cpp index 1d7689bc10..1872c0f160 100644 --- a/fdbcli/fdbcli.actor.cpp +++ b/fdbcli/fdbcli.actor.cpp @@ -3298,7 +3298,7 @@ ACTOR Future cli(CLIOptions opt, LineNoise* plinenoise) { is_error = true; continue; } - Standalone res = wait(db->getRangeFeedMutations(tokens[2])); + Standalone> res = wait(db->getRangeFeedMutations(tokens[2])); for(auto& it : res) { printf("%lld %s\n", it.version, it.mutation.toString().c_str()); } From fa39ea35b4f24c7ce5822c950238aea4e2a2a5f4 Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Thu, 4 Mar 2021 20:42:32 -0800 Subject: [PATCH 14/72] added missing break --- fdbcli/fdbcli.actor.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/fdbcli/fdbcli.actor.cpp b/fdbcli/fdbcli.actor.cpp index 1872c0f160..0555849784 100644 --- a/fdbcli/fdbcli.actor.cpp +++ b/fdbcli/fdbcli.actor.cpp @@ -3288,6 +3288,7 @@ ACTOR Future cli(CLIOptions opt, LineNoise* plinenoise) { try { wait(trx.registerRangeFeed(tokens[2], KeyRangeRef(tokens[3], tokens[4]))); wait(trx.commit()); + break; } catch( Error &e ) { wait(trx.onError(e)); } From 78a74aab8ee10580f3d6f9084f891193af9db0fa Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Thu, 4 Mar 2021 21:17:29 -0800 Subject: [PATCH 15/72] added logging --- fdbserver/storageserver.actor.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index 5a6b07e379..74540ff442 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -1219,6 +1219,7 @@ ACTOR Future rangeFeedQ( StorageServer* data, RangeFeedRequest req ) { for(auto& it : data->uidRangeFeed[req.rangeID]->mutations) { reply.mutations.push_back(reply.arena, it); } + TraceEvent("RangeFeedQuery", data->thisServerID).detail("RangeID", req.rangeID.printable()).detail("Mutations", reply.mutations.size()); req.reply.send(reply); return Void(); } @@ -2890,6 +2891,7 @@ private: r->value().push_back( rangeFeedInfo ); } data->keyRangeFeed.coalesce( rangeFeedRange.contents() ); + TraceEvent("AddingRangeFeed", data->thisServerID).detail("RangeID", rangeFeedId.printable()).detail("Range", rangeFeedRange.toString()); } else { ASSERT(false); // Unknown private mutation } From e25aac49880a5e00d0d563fc328789f284bb74bb Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Thu, 4 Mar 2021 21:31:21 -0800 Subject: [PATCH 16/72] added missing continue --- fdbcli/fdbcli.actor.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/fdbcli/fdbcli.actor.cpp b/fdbcli/fdbcli.actor.cpp index 0555849784..4ae1be7715 100644 --- a/fdbcli/fdbcli.actor.cpp +++ b/fdbcli/fdbcli.actor.cpp @@ -3304,6 +3304,7 @@ ACTOR Future cli(CLIOptions opt, LineNoise* plinenoise) { printf("%lld %s\n", it.version, it.mutation.toString().c_str()); } } + continue; } if (tokencmp(tokens[0], "configure")) { From 4a7358b0c4c083a15b409f02f18862229c5df46b Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Thu, 4 Mar 2021 21:44:02 -0800 Subject: [PATCH 17/72] disable range feeds temporarily --- fdbserver/storageserver.actor.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index 74540ff442..ebbf4e68de 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -2882,6 +2882,8 @@ private: } else if (m.type == MutationRef::SetValue && m.param1.startsWith(rangeFeedPrivatePrefix)) { Key rangeFeedId = m.param1.removePrefix(rangeFeedPrivatePrefix); KeyRange rangeFeedRange = decodeRangeFeedValue( m.param2 ); + TraceEvent("AddingRangeFeed", data->thisServerID).detail("RangeID", rangeFeedId.printable()).detail("Range", rangeFeedRange.toString()); + /* Reference rangeFeedInfo( new RangeFeedInfo() ); rangeFeedInfo->range = rangeFeedRange; rangeFeedInfo->id = rangeFeedId; @@ -2891,7 +2893,7 @@ private: r->value().push_back( rangeFeedInfo ); } data->keyRangeFeed.coalesce( rangeFeedRange.contents() ); - TraceEvent("AddingRangeFeed", data->thisServerID).detail("RangeID", rangeFeedId.printable()).detail("Range", rangeFeedRange.toString()); + */ } else { ASSERT(false); // Unknown private mutation } From 077d5cb7744068ebd82d98f642660a52485d7dce Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Thu, 4 Mar 2021 22:05:00 -0800 Subject: [PATCH 18/72] more logging --- fdbserver/ApplyMetadataMutation.cpp | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/fdbserver/ApplyMetadataMutation.cpp b/fdbserver/ApplyMetadataMutation.cpp index cf804870af..1bfb29bb57 100644 --- a/fdbserver/ApplyMetadataMutation.cpp +++ b/fdbserver/ApplyMetadataMutation.cpp @@ -318,7 +318,8 @@ void applyMetadataMutations(SpanID const& spanContext, UID const& dbgid, Arena& ++firstRange; if (firstRange == ranges.end()) { ranges.begin().value().populateTags(); - toCommit->addTags(ranges.begin().value().tags); + TraceEvent("RangeFeedTags1").detail("Tags", describe(ranges.begin().value().tags)); + //toCommit->addTags(ranges.begin().value().tags); } else { std::set allSources; @@ -326,9 +327,10 @@ void applyMetadataMutations(SpanID const& spanContext, UID const& dbgid, Arena& r.value().populateTags(); allSources.insert(r.value().tags.begin(), r.value().tags.end()); } - toCommit->addTags(allSources); + TraceEvent("RangeFeedTags2").detail("Tags", describe(allSources)); + //toCommit->addTags(allSources); } - toCommit->writeTypedMessage(privatized); + //toCommit->writeTypedMessage(privatized); } } } else if (m.param2.size() > 1 && m.param2[0] == systemKeys.begin[0] && m.type == MutationRef::ClearRange) { From fcdfb608c3982ed16a994f9c72af8f95d00c5326 Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Thu, 4 Mar 2021 22:45:58 -0800 Subject: [PATCH 19/72] re-enabled range feeds --- fdbserver/ApplyMetadataMutation.cpp | 6 +++--- fdbserver/storageserver.actor.cpp | 2 -- 2 files changed, 3 insertions(+), 5 deletions(-) diff --git a/fdbserver/ApplyMetadataMutation.cpp b/fdbserver/ApplyMetadataMutation.cpp index 1bfb29bb57..2ab63d9413 100644 --- a/fdbserver/ApplyMetadataMutation.cpp +++ b/fdbserver/ApplyMetadataMutation.cpp @@ -319,7 +319,7 @@ void applyMetadataMutations(SpanID const& spanContext, UID const& dbgid, Arena& if (firstRange == ranges.end()) { ranges.begin().value().populateTags(); TraceEvent("RangeFeedTags1").detail("Tags", describe(ranges.begin().value().tags)); - //toCommit->addTags(ranges.begin().value().tags); + toCommit->addTags(ranges.begin().value().tags); } else { std::set allSources; @@ -328,9 +328,9 @@ void applyMetadataMutations(SpanID const& spanContext, UID const& dbgid, Arena& allSources.insert(r.value().tags.begin(), r.value().tags.end()); } TraceEvent("RangeFeedTags2").detail("Tags", describe(allSources)); - //toCommit->addTags(allSources); + toCommit->addTags(allSources); } - //toCommit->writeTypedMessage(privatized); + toCommit->writeTypedMessage(privatized); } } } else if (m.param2.size() > 1 && m.param2[0] == systemKeys.begin[0] && m.type == MutationRef::ClearRange) { diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index ebbf4e68de..7f6e5dd9d2 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -2883,7 +2883,6 @@ private: Key rangeFeedId = m.param1.removePrefix(rangeFeedPrivatePrefix); KeyRange rangeFeedRange = decodeRangeFeedValue( m.param2 ); TraceEvent("AddingRangeFeed", data->thisServerID).detail("RangeID", rangeFeedId.printable()).detail("Range", rangeFeedRange.toString()); - /* Reference rangeFeedInfo( new RangeFeedInfo() ); rangeFeedInfo->range = rangeFeedRange; rangeFeedInfo->id = rangeFeedId; @@ -2893,7 +2892,6 @@ private: r->value().push_back( rangeFeedInfo ); } data->keyRangeFeed.coalesce( rangeFeedRange.contents() ); - */ } else { ASSERT(false); // Unknown private mutation } From 0865b1c70c80166884b31f86682a1060fa06cd2a Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Thu, 4 Mar 2021 23:27:49 -0800 Subject: [PATCH 20/72] prevent pointing to released memory --- fdbserver/storageserver.actor.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index 7f6e5dd9d2..a827d60765 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -260,7 +260,7 @@ struct FetchInjectionInfo { }; struct RangeFeedInfo : ReferenceCounted { - std::deque mutations; + std::deque> mutations; KeyRange range; Key id; }; From 6d1878113220befa3be4389e0f2f5ff5fb2166e3 Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Thu, 4 Mar 2021 23:30:26 -0800 Subject: [PATCH 21/72] fix compile error --- fdbserver/storageserver.actor.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index a827d60765..d13f751817 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -2047,7 +2047,7 @@ void applyMutation( StorageServer *self, MutationRef const& m, Arena& arena, Sto self->watches.trigger( m.param1 ); for(auto& it : self->keyRangeFeed[m.param1]) { - it->mutations.emplace_back(m,version); + it->mutations.push_back(MutationRefAndVersion(m,version)); } } else if (m.type == MutationRef::ClearRange) { data.erase( m.param1, m.param2 ); @@ -2059,7 +2059,7 @@ void applyMutation( StorageServer *self, MutationRef const& m, Arena& arena, Sto auto ranges = self->keyRangeFeed.intersectingRanges(KeyRangeRef(m.param1, m.param2)); for(auto &r : ranges) { for(auto& it : r.value()) { - it->mutations.emplace_back(m,version); + it->mutations.push_back(MutationRefAndVersion(m,version)); } } } From a6c9b75aa5d508b1b01cd6bcbaf2abe0013f4914 Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Thu, 4 Mar 2021 23:36:41 -0800 Subject: [PATCH 22/72] add Standalone support for MutationRefAndVersion --- fdbclient/StorageServerInterface.h | 3 +++ 1 file changed, 3 insertions(+) diff --git a/fdbclient/StorageServerInterface.h b/fdbclient/StorageServerInterface.h index 2fdaeaf080..ee3b445185 100644 --- a/fdbclient/StorageServerInterface.h +++ b/fdbclient/StorageServerInterface.h @@ -526,6 +526,9 @@ struct MutationRefAndVersion { MutationRefAndVersion() {} MutationRefAndVersion(MutationRef mutation, Version version) : mutation(mutation), version(version) {} + MutationRefAndVersion( Arena& to, MutationRef mutation, Version version ) : mutation(to, mutation), version(version) {} + MutationRefAndVersion( Arena& to, const MutationRefAndVersion& from ) : mutation(to, from.mutation), version(from.version) {} + int expectedSize() const { return mutation.expectedSize(); } template void serialize(Ar& ar) { From 6cfba6e54be22b63590edfa91bdba2f0eff80e05 Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Fri, 5 Mar 2021 11:46:33 -0800 Subject: [PATCH 23/72] added support for popping rangefeeds --- fdbcli/fdbcli.actor.cpp | 16 +++++++++++++++- fdbclient/DatabaseContext.h | 1 + fdbclient/NativeAPI.actor.cpp | 29 +++++++++++++++++++++++++++++ fdbclient/StorageServerInterface.h | 18 ++++++++++++++++++ fdbserver/storageserver.actor.cpp | 12 ++++++++++++ 5 files changed, 75 insertions(+), 1 deletion(-) diff --git a/fdbcli/fdbcli.actor.cpp b/fdbcli/fdbcli.actor.cpp index 4ae1be7715..a351eda2fb 100644 --- a/fdbcli/fdbcli.actor.cpp +++ b/fdbcli/fdbcli.actor.cpp @@ -618,7 +618,7 @@ void initHelp() { CommandHelp("triggerddteaminfolog", "trigger the data distributor teams logging", "Trigger the data distributor to log detailed information about its teams."); helpMap["rangefeed"] = CommandHelp( - "rangefeed ", + "rangefeed ", "", ""); @@ -3303,6 +3303,20 @@ ACTOR Future cli(CLIOptions opt, LineNoise* plinenoise) { for(auto& it : res) { printf("%lld %s\n", it.version, it.mutation.toString().c_str()); } + } else if(tokencmp(tokens[1], "pop")) { + if(tokens.size() != 4) { + printUsage(tokens[0]); + is_error = true; + continue; + } + Version v; + int n = 0; + if (sscanf(tokens[3].toString().c_str(), "%ld%n", &v, &n) != 1 || n != tokens[3].size()) { + printUsage(tokens[0]); + is_error = true; + } else { + wait(db->popRangeFeedMutations(tokens[2], v)); + } } continue; } diff --git a/fdbclient/DatabaseContext.h b/fdbclient/DatabaseContext.h index 863c382dcd..412915589c 100644 --- a/fdbclient/DatabaseContext.h +++ b/fdbclient/DatabaseContext.h @@ -237,6 +237,7 @@ public: Future createSnapshot(StringRef uid, StringRef snapshot_command); Future>> getRangeFeedMutations(StringRef rangeID); + Future popRangeFeedMutations(StringRef rangeID, Version version); //private: explicit DatabaseContext( Reference>> connectionFile, Reference> clientDBInfo, diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index 13a06e25bd..6d7fe8a99c 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -5071,3 +5071,32 @@ ACTOR Future>> getRangeFeedMutations Future>> DatabaseContext::getRangeFeedMutations(StringRef rangeID) { return getRangeFeedMutationsActor(Reference::addRef(this), rangeID); } + +ACTOR Future popRangeFeedMutationsActor(Reference db, StringRef rangeID, Version version) { + state Database cx(db); + state Transaction tr(cx); + state Key rangeIDKey = rangeID.withPrefix(rangeFeedPrefix); + state Span span("NAPI:PopRangeFeedMutations"_loc); + Optional val = wait( tr.get(rangeIDKey) ); + if(!val.present()) { + throw unsupported_operation(); + } + KeyRange keys = decodeRangeFeedValue(val.get()); + state vector< pair> > locations = wait( getKeyRangeLocations( cx, keys, 100, + false, &StorageServerInterface::rangeFeed, TransactionInfo(TaskPriority::DefaultEndpoint, span.context) ) ); + + if(locations.size() > 1) { + throw unsupported_operation(); + } + + state std::vector> popRequests; + for(int i = 0; i < locations[0].second->size(); i++) { + popRequests.push_back(locations[0].second->getInterface(i).rangeFeedPop.getReply(RangeFeedPopRequest(rangeID, version))); + } + wait(waitForAll(popRequests)); + return Void(); +} + +Future DatabaseContext::popRangeFeedMutations(StringRef rangeID, Version version) { + return popRangeFeedMutationsActor(Reference::addRef(this), rangeID, version); +} \ No newline at end of file diff --git a/fdbclient/StorageServerInterface.h b/fdbclient/StorageServerInterface.h index ee3b445185..eed882dc22 100644 --- a/fdbclient/StorageServerInterface.h +++ b/fdbclient/StorageServerInterface.h @@ -75,6 +75,7 @@ struct StorageServerInterface { RequestStream getReadHotRanges; RequestStream getRangeSplitPoints; RequestStream rangeFeed; + RequestStream rangeFeedPop; explicit StorageServerInterface(UID uid) : uniqueID( uid ) {} StorageServerInterface() : uniqueID( deterministicRandom()->randomUniqueID() ) {} @@ -104,6 +105,7 @@ struct StorageServerInterface { getReadHotRanges = RequestStream( getValue.getEndpoint().getAdjustedEndpoint(11) ); getRangeSplitPoints = RequestStream(getValue.getEndpoint().getAdjustedEndpoint(12)); rangeFeed = RequestStream(getValue.getEndpoint().getAdjustedEndpoint(13)); + rangeFeedPop = RequestStream(getValue.getEndpoint().getAdjustedEndpoint(14)); } } else { ASSERT(Ar::isDeserializing); @@ -133,6 +135,7 @@ struct StorageServerInterface { streams.push_back(getReadHotRanges.getReceiver()); streams.push_back(getRangeSplitPoints.getReceiver()); streams.push_back(rangeFeed.getReceiver()); + streams.push_back(rangeFeedPop.getReceiver()); FlowTransport::transport().addEndpoints(streams); } }; @@ -563,6 +566,21 @@ struct RangeFeedRequest { } }; +struct RangeFeedPopRequest { + constexpr static FileIdentifier file_identifier = 10726174; + Key rangeID; + Version version; + ReplyPromise reply; + + RangeFeedPopRequest() {} + RangeFeedPopRequest(Key const& rangeID, Version version) : rangeID(rangeID), version(version) {} + + template + void serialize(Ar& ar) { + serializer(ar, rangeID, version, reply); + } +}; + struct GetStorageMetricsReply { constexpr static FileIdentifier file_identifier = 15491478; StorageMetrics load; diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index d13f751817..25bfec2922 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -3949,6 +3949,17 @@ ACTOR Future serveRangeFeedRequests( StorageServer* self, FutureStream serveRangeFeedPopRequests( StorageServer* self, FutureStream rangeFeedPops ) { + loop { + RangeFeedPopRequest req = waitNext(rangeFeedPops); + while(data->uidRangeFeed[req.rangeID]->mutations.front().version < req.version) { + data->uidRangeFeed[req.rangeID]->mutations.pop_front(); + } + TraceEvent("RangeFeedPopQuery", data->thisServerID).detail("RangeID", req.rangeID.printable()).detail("Version", req.version); + req.reply.send(Void()); + } +} + ACTOR Future reportStorageServerState(StorageServer* self) { if (!SERVER_KNOBS->REPORT_DD_METRICS) { return Void(); @@ -3998,6 +4009,7 @@ ACTOR Future storageServerCore( StorageServer* self, StorageServerInterfac self->actors.add(serveGetKeyRequests(self, ssi.getKey.getFuture())); self->actors.add(serveWatchValueRequests(self, ssi.watchValue.getFuture())); self->actors.add(serveRangeFeedRequests(self, ssi.rangeFeed.getFuture())); + self->actors.add(serveRangeFeedPopRequests(self, ssi.rangeFeedPop.getFuture())); self->actors.add(traceRole(Role::STORAGE_SERVER, ssi.id())); self->actors.add(reportStorageServerState(self)); From 163e44b6b0b635ef63901f9457988084beba6c56 Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Fri, 5 Mar 2021 11:50:08 -0800 Subject: [PATCH 24/72] rename data to self --- fdbserver/storageserver.actor.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index 25bfec2922..ed77fe7882 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -3952,10 +3952,10 @@ ACTOR Future serveRangeFeedRequests( StorageServer* self, FutureStream serveRangeFeedPopRequests( StorageServer* self, FutureStream rangeFeedPops ) { loop { RangeFeedPopRequest req = waitNext(rangeFeedPops); - while(data->uidRangeFeed[req.rangeID]->mutations.front().version < req.version) { - data->uidRangeFeed[req.rangeID]->mutations.pop_front(); + while(self->uidRangeFeed[req.rangeID]->mutations.front().version < req.version) { + self->uidRangeFeed[req.rangeID]->mutations.pop_front(); } - TraceEvent("RangeFeedPopQuery", data->thisServerID).detail("RangeID", req.rangeID.printable()).detail("Version", req.version); + TraceEvent("RangeFeedPopQuery", self->thisServerID).detail("RangeID", req.rangeID.printable()).detail("Version", req.version); req.reply.send(Void()); } } From 2621c7153a244e21fdbb949bc4e2a82439f5c466 Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Fri, 30 Apr 2021 10:41:35 -0700 Subject: [PATCH 25/72] clang format PR --- documentation/tutorial/tutorial.actor.cpp | 36 +++++----- fdbclient/NativeAPI.actor.cpp | 67 ++++++++++++------- fdbclient/StorageServerInterface.h | 9 ++- fdbrpc/FlowTransport.actor.cpp | 2 +- fdbserver/storageserver.actor.cpp | 81 +++++++++++++---------- 5 files changed, 114 insertions(+), 81 deletions(-) diff --git a/documentation/tutorial/tutorial.actor.cpp b/documentation/tutorial/tutorial.actor.cpp index 5ec749b1cb..dfc684e922 100644 --- a/documentation/tutorial/tutorial.actor.cpp +++ b/documentation/tutorial/tutorial.actor.cpp @@ -1,23 +1,23 @@ /* - * tutorial.actor.cpp +* tutorial.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. - */ +* +* 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 "flow/flow.h" #include "flow/Platform.h" diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index b96b60da96..c95ed7ba29 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -1054,14 +1054,16 @@ DatabaseContext::DatabaseContext(Reference( KeyRangeRef(LiteralStringRef("profiling/"), LiteralStringRef("profiling0")) - .withPrefix(SpecialKeySpace::getModuleRange(SpecialKeySpace::MODULE::MANAGEMENT).begin))); + .withPrefix(SpecialKeySpace::getModuleRange(SpecialKeySpace::MODULE::MANAGEMENT).begin))); registerSpecialKeySpaceModule( - SpecialKeySpace::MODULE::MANAGEMENT, SpecialKeySpace::IMPLTYPE::READWRITE, + SpecialKeySpace::MODULE::MANAGEMENT, + SpecialKeySpace::IMPLTYPE::READWRITE, std::make_unique( KeyRangeRef(LiteralStringRef("maintenance/"), LiteralStringRef("maintenance0")) .withPrefix(SpecialKeySpace::getModuleRange(SpecialKeySpace::MODULE::MANAGEMENT).begin))); registerSpecialKeySpaceModule( - SpecialKeySpace::MODULE::MANAGEMENT, SpecialKeySpace::IMPLTYPE::READWRITE, + SpecialKeySpace::MODULE::MANAGEMENT, + SpecialKeySpace::IMPLTYPE::READWRITE, std::make_unique( KeyRangeRef(LiteralStringRef("data_distribution/"), LiteralStringRef("data_distribution0")) .withPrefix(SpecialKeySpace::getModuleRange(SpecialKeySpace::MODULE::MANAGEMENT).begin))); @@ -3531,18 +3533,18 @@ Future>> Transaction::getAddressesForKey(const return getAddressesForKeyActor(key, ver, cx, info, options); } -ACTOR Future registerRangeFeedActor(Transaction *tr, Key rangeID, KeyRange range) { +ACTOR Future registerRangeFeedActor(Transaction* tr, Key rangeID, KeyRange range) { state Key rangeIDKey = rangeID.withPrefix(rangeFeedPrefix); - Optional val = wait( tr->get(rangeIDKey) ); - if(!val.present()) { + Optional val = wait(tr->get(rangeIDKey)); + if (!val.present()) { tr->set(rangeIDKey, rangeFeedValue(range)); - } else if(decodeRangeFeedValue(val.get()) != range) { + } else if (decodeRangeFeedValue(val.get()) != range) { throw unsupported_operation(); } return Void(); } -Future Transaction::registerRangeFeed( const Key& rangeID, const KeyRange& range ) { +Future Transaction::registerRangeFeed(const Key& rangeID, const KeyRange& range) { return registerRangeFeedActor(this, rangeID, range); } @@ -5678,30 +5680,39 @@ Future DatabaseContext::createSnapshot(StringRef uid, StringRef snapshot_c return createSnapshotActor(this, UID::fromString(uid_str), snapshot_command); } -ACTOR Future>> getRangeFeedMutationsActor(Reference db, StringRef rangeID) { +ACTOR Future>> getRangeFeedMutationsActor(Reference db, + StringRef rangeID) { state Database cx(db); state Transaction tr(cx); state Key rangeIDKey = rangeID.withPrefix(rangeFeedPrefix); state Span span("NAPI:GetRangeFeedMutations"_loc); - Optional val = wait( tr.get(rangeIDKey) ); - if(!val.present()) { + Optional val = wait(tr.get(rangeIDKey)); + if (!val.present()) { throw unsupported_operation(); } KeyRange keys = decodeRangeFeedValue(val.get()); - state vector< pair> > locations = wait( getKeyRangeLocations( cx, keys, 100, - false, &StorageServerInterface::rangeFeed, TransactionInfo(TaskPriority::DefaultEndpoint, span.context) ) ); + state vector>> locations = + wait(getKeyRangeLocations(cx, + keys, + 100, + false, + &StorageServerInterface::rangeFeed, + TransactionInfo(TaskPriority::DefaultEndpoint, span.context))); - if(locations.size() > 1) { + if (locations.size() > 1) { throw unsupported_operation(); } state RangeFeedRequest req; req.rangeID = rangeID; - RangeFeedReply rep = - wait(loadBalance(cx.getPtr(), locations[0].second, &StorageServerInterface::rangeFeed, req, - TaskPriority::DefaultPromiseEndpoint, false, - cx->enableLocalityLoadBalance ? &cx->queueModel : nullptr)); + RangeFeedReply rep = wait(loadBalance(cx.getPtr(), + locations[0].second, + &StorageServerInterface::rangeFeed, + req, + TaskPriority::DefaultPromiseEndpoint, + false, + cx->enableLocalityLoadBalance ? &cx->queueModel : nullptr)); return Standalone>(rep.mutations, rep.arena); } @@ -5714,21 +5725,27 @@ ACTOR Future popRangeFeedMutationsActor(Reference db, Str state Transaction tr(cx); state Key rangeIDKey = rangeID.withPrefix(rangeFeedPrefix); state Span span("NAPI:PopRangeFeedMutations"_loc); - Optional val = wait( tr.get(rangeIDKey) ); - if(!val.present()) { + Optional val = wait(tr.get(rangeIDKey)); + if (!val.present()) { throw unsupported_operation(); } KeyRange keys = decodeRangeFeedValue(val.get()); - state vector< pair> > locations = wait( getKeyRangeLocations( cx, keys, 100, - false, &StorageServerInterface::rangeFeed, TransactionInfo(TaskPriority::DefaultEndpoint, span.context) ) ); + state vector>> locations = + wait(getKeyRangeLocations(cx, + keys, + 100, + false, + &StorageServerInterface::rangeFeed, + TransactionInfo(TaskPriority::DefaultEndpoint, span.context))); - if(locations.size() > 1) { + if (locations.size() > 1) { throw unsupported_operation(); } state std::vector> popRequests; - for(int i = 0; i < locations[0].second->size(); i++) { - popRequests.push_back(locations[0].second->getInterface(i).rangeFeedPop.getReply(RangeFeedPopRequest(rangeID, version))); + for (int i = 0; i < locations[0].second->size(); i++) { + popRequests.push_back( + locations[0].second->getInterface(i).rangeFeedPop.getReply(RangeFeedPopRequest(rangeID, version))); } wait(waitForAll(popRequests)); return Void(); diff --git a/fdbclient/StorageServerInterface.h b/fdbclient/StorageServerInterface.h index 566908e63c..84c2e6c14f 100644 --- a/fdbclient/StorageServerInterface.h +++ b/fdbclient/StorageServerInterface.h @@ -112,7 +112,8 @@ struct StorageServerInterface { getRangeSplitPoints = RequestStream(getValue.getEndpoint().getAdjustedEndpoint(12)); rangeFeed = RequestStream(getValue.getEndpoint().getAdjustedEndpoint(13)); - rangeFeedPop = RequestStream(getValue.getEndpoint().getAdjustedEndpoint(14)); + rangeFeedPop = + RequestStream(getValue.getEndpoint().getAdjustedEndpoint(14)); } } else { ASSERT(Ar::isDeserializing); @@ -576,8 +577,10 @@ struct MutationRefAndVersion { MutationRefAndVersion() {} MutationRefAndVersion(MutationRef mutation, Version version) : mutation(mutation), version(version) {} - MutationRefAndVersion( Arena& to, MutationRef mutation, Version version ) : mutation(to, mutation), version(version) {} - MutationRefAndVersion( Arena& to, const MutationRefAndVersion& from ) : mutation(to, from.mutation), version(from.version) {} + MutationRefAndVersion(Arena& to, MutationRef mutation, Version version) + : mutation(to, mutation), version(version) {} + MutationRefAndVersion(Arena& to, const MutationRefAndVersion& from) + : mutation(to, from.mutation), version(from.version) {} int expectedSize() const { return mutation.expectedSize(); } template diff --git a/fdbrpc/FlowTransport.actor.cpp b/fdbrpc/FlowTransport.actor.cpp index 8cc9d0d8e6..bf119496cf 100644 --- a/fdbrpc/FlowTransport.actor.cpp +++ b/fdbrpc/FlowTransport.actor.cpp @@ -1215,7 +1215,7 @@ ACTOR static Future connectionReader(TransportData* transport, } compatible = false; if (!protocolVersion.hasInexpensiveMultiVersionClient()) { - if(peer) { + if (peer) { peer->protocolVersion->set(protocolVersion); } diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index 811d24f22b..a7d9191fac 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -522,7 +522,7 @@ public: uint64_t shardChangeCounter; // max( shards->changecounter ) KeyRangeMap cachedRangeMap; // indicates if a key-range is being cached - + KeyRangeMap>> keyRangeFeed; std::map> uidRangeFeed; @@ -1394,13 +1394,15 @@ ACTOR Future watchValueSendReply(StorageServer* data, } } -ACTOR Future rangeFeedQ( StorageServer* data, RangeFeedRequest req ) { +ACTOR Future rangeFeedQ(StorageServer* data, RangeFeedRequest req) { wait(delay(0)); RangeFeedReply reply; - for(auto& it : data->uidRangeFeed[req.rangeID]->mutations) { + for (auto& it : data->uidRangeFeed[req.rangeID]->mutations) { reply.mutations.push_back(reply.arena, it); } - TraceEvent("RangeFeedQuery", data->thisServerID).detail("RangeID", req.rangeID.printable()).detail("Mutations", reply.mutations.size()); + TraceEvent("RangeFeedQuery", data->thisServerID) + .detail("RangeID", req.rangeID.printable()) + .detail("Mutations", reply.mutations.size()); req.reply.send(reply); return Void(); } @@ -2292,7 +2294,11 @@ bool expandMutation(MutationRef& m, return true; } -void applyMutation( StorageServer *self, MutationRef const& m, Arena& arena, StorageServer::VersionedData &data, Version version ) { +void applyMutation(StorageServer* self, + MutationRef const& m, + Arena& arena, + StorageServer::VersionedData& data, + Version version) { // m is expected to be in arena already // Clear split keys are added to arena StorageMetrics metrics; @@ -2321,23 +2327,23 @@ void applyMutation( StorageServer *self, MutationRef const& m, Arena& arena, Sto data.insert(nextKey, ValueOrClearToRef::clearTo(KeyRef(arena, end))); } } - data.insert( m.param1, ValueOrClearToRef::value(m.param2) ); - self->watches.trigger( m.param1 ); + data.insert(m.param1, ValueOrClearToRef::value(m.param2)); + self->watches.trigger(m.param1); - for(auto& it : self->keyRangeFeed[m.param1]) { - it->mutations.push_back(MutationRefAndVersion(m,version)); + for (auto& it : self->keyRangeFeed[m.param1]) { + it->mutations.push_back(MutationRefAndVersion(m, version)); } } else if (m.type == MutationRef::ClearRange) { - data.erase( m.param1, m.param2 ); - ASSERT( m.param2 > m.param1 ); - ASSERT( !data.isClearContaining( data.atLatest(), m.param1 ) ); - data.insert( m.param1, ValueOrClearToRef::clearTo(m.param2) ); - self->watches.triggerRange( m.param1, m.param2 ); + data.erase(m.param1, m.param2); + ASSERT(m.param2 > m.param1); + ASSERT(!data.isClearContaining(data.atLatest(), m.param1)); + data.insert(m.param1, ValueOrClearToRef::clearTo(m.param2)); + self->watches.triggerRange(m.param1, m.param2); auto ranges = self->keyRangeFeed.intersectingRanges(KeyRangeRef(m.param1, m.param2)); - for(auto &r : ranges) { - for(auto& it : r.value()) { - it->mutations.push_back(MutationRefAndVersion(m,version)); + for (auto& r : ranges) { + for (auto& it : r.value()) { + it->mutations.push_back(MutationRefAndVersion(m, version)); } } } @@ -3106,10 +3112,13 @@ void StorageServer::addMutation(Version version, return; } expanded = addMutationToMutationLog(mLog, expanded); - DEBUG_MUTATION("applyMutation", version, expanded).detail("UID", thisServerID).detail("ShardBegin", shard.begin).detail("ShardEnd", shard.end); - applyMutation( this, expanded, mLog.arena(), mutableData(), version ); - //printf("\nSSUpdate: Printing versioned tree after applying mutation\n"); - //mutableData().printTree(version); + DEBUG_MUTATION("applyMutation", version, expanded) + .detail("UID", thisServerID) + .detail("ShardBegin", shard.begin) + .detail("ShardEnd", shard.end); + applyMutation(this, expanded, mLog.arena(), mutableData(), version); + // printf("\nSSUpdate: Printing versioned tree after applying mutation\n"); + // mutableData().printTree(version); } struct OrderByVersion { @@ -3256,21 +3265,23 @@ private: .detail("RebootAfterDurableVersion", data->rebootAfterDurableVersion); } else if (m.type == MutationRef::SetValue && m.param1 == primaryLocalityPrivateKey) { data->primaryLocality = BinaryReader::fromStringRef(m.param2, Unversioned()); - auto& mLV = data->addVersionToMutationLog( data->data().getLatestVersion() ); - data->addMutationToMutationLog( mLV, MutationRef(MutationRef::SetValue, persistPrimaryLocality, m.param2) ); + auto& mLV = data->addVersionToMutationLog(data->data().getLatestVersion()); + data->addMutationToMutationLog(mLV, MutationRef(MutationRef::SetValue, persistPrimaryLocality, m.param2)); } else if (m.type == MutationRef::SetValue && m.param1.startsWith(rangeFeedPrivatePrefix)) { Key rangeFeedId = m.param1.removePrefix(rangeFeedPrivatePrefix); - KeyRange rangeFeedRange = decodeRangeFeedValue( m.param2 ); - TraceEvent("AddingRangeFeed", data->thisServerID).detail("RangeID", rangeFeedId.printable()).detail("Range", rangeFeedRange.toString()); - Reference rangeFeedInfo( new RangeFeedInfo() ); + KeyRange rangeFeedRange = decodeRangeFeedValue(m.param2); + TraceEvent("AddingRangeFeed", data->thisServerID) + .detail("RangeID", rangeFeedId.printable()) + .detail("Range", rangeFeedRange.toString()); + Reference rangeFeedInfo(new RangeFeedInfo()); rangeFeedInfo->range = rangeFeedRange; rangeFeedInfo->id = rangeFeedId; data->uidRangeFeed[rangeFeedId] = rangeFeedInfo; - auto rs = data->keyRangeFeed.modify( rangeFeedRange ); - for(auto r = rs.begin(); r != rs.end(); ++r) { - r->value().push_back( rangeFeedInfo ); + auto rs = data->keyRangeFeed.modify(rangeFeedRange); + for (auto r = rs.begin(); r != rs.end(); ++r) { + r->value().push_back(rangeFeedInfo); } - data->keyRangeFeed.coalesce( rangeFeedRange.contents() ); + data->keyRangeFeed.coalesce(rangeFeedRange.contents()); } else { ASSERT(false); // Unknown private mutation } @@ -4508,20 +4519,22 @@ ACTOR Future serveWatchValueRequests(StorageServer* self, FutureStream serveRangeFeedRequests( StorageServer* self, FutureStream rangeFeed ) { +ACTOR Future serveRangeFeedRequests(StorageServer* self, FutureStream rangeFeed) { loop { RangeFeedRequest req = waitNext(rangeFeed); self->actors.add(self->readGuard(req, rangeFeedQ)); } } -ACTOR Future serveRangeFeedPopRequests( StorageServer* self, FutureStream rangeFeedPops ) { +ACTOR Future serveRangeFeedPopRequests(StorageServer* self, FutureStream rangeFeedPops) { loop { RangeFeedPopRequest req = waitNext(rangeFeedPops); - while(self->uidRangeFeed[req.rangeID]->mutations.front().version < req.version) { + while (self->uidRangeFeed[req.rangeID]->mutations.front().version < req.version) { self->uidRangeFeed[req.rangeID]->mutations.pop_front(); } - TraceEvent("RangeFeedPopQuery", self->thisServerID).detail("RangeID", req.rangeID.printable()).detail("Version", req.version); + TraceEvent("RangeFeedPopQuery", self->thisServerID) + .detail("RangeID", req.rangeID.printable()) + .detail("Version", req.version); req.reply.send(Void()); } } From 78697003ea290dbcb60063efb32e52aa862ccf31 Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Fri, 30 Apr 2021 10:51:35 -0700 Subject: [PATCH 26/72] more formatting --- fdbcli/fdbcli.actor.cpp | 23 ++++++++++------------- fdbclient/SystemData.cpp | 19 +++++++++---------- fdbclient/SystemData.h | 4 ++-- fdbserver/ApplyMetadataMutation.cpp | 7 +++---- 4 files changed, 24 insertions(+), 29 deletions(-) diff --git a/fdbcli/fdbcli.actor.cpp b/fdbcli/fdbcli.actor.cpp index 930eeaf639..9eb5f974a4 100644 --- a/fdbcli/fdbcli.actor.cpp +++ b/fdbcli/fdbcli.actor.cpp @@ -678,10 +678,7 @@ void initHelp() { CommandHelp("triggerddteaminfolog", "trigger the data distributor teams logging", "Trigger the data distributor to log detailed information about its teams."); - helpMap["rangefeed"] = CommandHelp( - "rangefeed ", - "", - ""); + helpMap["rangefeed"] = CommandHelp("rangefeed ", "", ""); hiddenCommands.insert("expensive_data_check"); hiddenCommands.insert("datadistribution"); @@ -3398,13 +3395,13 @@ ACTOR Future cli(CLIOptions opt, LineNoise* plinenoise) { } if (tokencmp(tokens[0], "rangefeed")) { - if(tokens.size() == 1) { + if (tokens.size() == 1) { printUsage(tokens[0]); is_error = true; continue; } - if(tokencmp(tokens[1], "register")) { - if(tokens.size() != 5) { + if (tokencmp(tokens[1], "register")) { + if (tokens.size() != 5) { printUsage(tokens[0]); is_error = true; continue; @@ -3415,22 +3412,22 @@ ACTOR Future cli(CLIOptions opt, LineNoise* plinenoise) { wait(trx.registerRangeFeed(tokens[2], KeyRangeRef(tokens[3], tokens[4]))); wait(trx.commit()); break; - } catch( Error &e ) { + } catch (Error& e) { wait(trx.onError(e)); } } - } else if(tokencmp(tokens[1], "get")) { - if(tokens.size() != 3) { + } else if (tokencmp(tokens[1], "get")) { + if (tokens.size() != 3) { printUsage(tokens[0]); is_error = true; continue; } Standalone> res = wait(db->getRangeFeedMutations(tokens[2])); - for(auto& it : res) { + for (auto& it : res) { printf("%lld %s\n", it.version, it.mutation.toString().c_str()); } - } else if(tokencmp(tokens[1], "pop")) { - if(tokens.size() != 4) { + } else if (tokencmp(tokens[1], "pop")) { + if (tokens.size() != 4) { printUsage(tokens[0]); is_error = true; continue; diff --git a/fdbclient/SystemData.cpp b/fdbclient/SystemData.cpp index 3a8646d486..734728d331 100644 --- a/fdbclient/SystemData.cpp +++ b/fdbclient/SystemData.cpp @@ -635,15 +635,17 @@ std::string encodeFailedServersKey(AddressExclusion const& addr) { // const KeyRangeRef globalConfigKeys( LiteralStringRef("\xff/globalConfig/"), LiteralStringRef("\xff/globalConfig0") ); // const KeyRef globalConfigPrefix = globalConfigKeys.begin; -const KeyRangeRef globalConfigDataKeys( LiteralStringRef("\xff/globalConfig/k/"), LiteralStringRef("\xff/globalConfig/k0") ); +const KeyRangeRef globalConfigDataKeys(LiteralStringRef("\xff/globalConfig/k/"), + LiteralStringRef("\xff/globalConfig/k0")); const KeyRef globalConfigKeysPrefix = globalConfigDataKeys.begin; -const KeyRangeRef globalConfigHistoryKeys( LiteralStringRef("\xff/globalConfig/h/"), LiteralStringRef("\xff/globalConfig/h0") ); +const KeyRangeRef globalConfigHistoryKeys(LiteralStringRef("\xff/globalConfig/h/"), + LiteralStringRef("\xff/globalConfig/h0")); const KeyRef globalConfigHistoryPrefix = globalConfigHistoryKeys.begin; const KeyRef globalConfigVersionKey = LiteralStringRef("\xff/globalConfig/v"); -const KeyRangeRef workerListKeys( LiteralStringRef("\xff/worker/"), LiteralStringRef("\xff/worker0") ); +const KeyRangeRef workerListKeys(LiteralStringRef("\xff/worker/"), LiteralStringRef("\xff/worker0")); const KeyRef workerListPrefix = workerListKeys.begin; const Key workerListKeyFor(StringRef processID) { @@ -1085,21 +1087,18 @@ const KeyRef writeRecoveryKey = LiteralStringRef("\xff/writeRecovery"); const ValueRef writeRecoveryKeyTrue = LiteralStringRef("1"); const KeyRef snapshotEndVersionKey = LiteralStringRef("\xff/snapshotEndVersion"); -const KeyRangeRef rangeFeedKeys( - LiteralStringRef("\xff\x02/feed/"), - LiteralStringRef("\xff\x02/feed0") -); +const KeyRangeRef rangeFeedKeys(LiteralStringRef("\xff\x02/feed/"), LiteralStringRef("\xff\x02/feed0")); const KeyRef rangeFeedPrefix = rangeFeedKeys.begin; const KeyRef rangeFeedPrivatePrefix = LiteralStringRef("\xff\xff\x02/feed/"); -const Value rangeFeedValue( KeyRangeRef const& range ) { +const Value rangeFeedValue(KeyRangeRef const& range) { BinaryWriter wr(IncludeVersion(ProtocolVersion::withRangeFeed())); wr << range; return wr.toValue(); } -KeyRange decodeRangeFeedValue( ValueRef const& value ) { +KeyRange decodeRangeFeedValue(ValueRef const& value) { KeyRange range; - BinaryReader reader( value, IncludeVersion() ); + BinaryReader reader(value, IncludeVersion()); reader >> range; return range; } diff --git a/fdbclient/SystemData.h b/fdbclient/SystemData.h index 9f061963ac..d948ea4da3 100644 --- a/fdbclient/SystemData.h +++ b/fdbclient/SystemData.h @@ -492,8 +492,8 @@ extern const ValueRef writeRecoveryKeyTrue; extern const KeyRef snapshotEndVersionKey; extern const KeyRangeRef rangeFeedKeys; -const Value rangeFeedValue( KeyRangeRef const& range ); -KeyRange decodeRangeFeedValue( ValueRef const& value ); +const Value rangeFeedValue(KeyRangeRef const& range); +KeyRange decodeRangeFeedValue(ValueRef const& value); extern const KeyRef rangeFeedPrefix; extern const KeyRef rangeFeedPrivatePrefix; diff --git a/fdbserver/ApplyMetadataMutation.cpp b/fdbserver/ApplyMetadataMutation.cpp index 3879339074..aa4f4cec93 100644 --- a/fdbserver/ApplyMetadataMutation.cpp +++ b/fdbserver/ApplyMetadataMutation.cpp @@ -353,8 +353,8 @@ void applyMetadataMutations(SpanID const& spanContext, txnStateStore->set(KeyValueRef(m.param1, m.param2)); TEST(true); // Snapshot created, setting writeRecoveryKey in txnStateStore } else if (m.param1.startsWith(rangeFeedPrefix)) { - if(toCommit && keyInfo) { - KeyRange r = decodeRangeFeedValue( m.param2 ); + if (toCommit && keyInfo) { + KeyRange r = decodeRangeFeedValue(m.param2); MutationRef privatized = m; privatized.param1 = m.param1.withPrefix(systemKeys.begin, arena); auto ranges = keyInfo->intersectingRanges(r); @@ -364,8 +364,7 @@ void applyMetadataMutations(SpanID const& spanContext, ranges.begin().value().populateTags(); TraceEvent("RangeFeedTags1").detail("Tags", describe(ranges.begin().value().tags)); toCommit->addTags(ranges.begin().value().tags); - } - else { + } else { std::set allSources; for (auto r : ranges) { r.value().populateTags(); From e9409b02fe2136ddd73be1c96c3ebfab26c20624 Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Wed, 28 Jul 2021 14:02:12 -0700 Subject: [PATCH 27/72] fixed the build --- fdbclient/NativeAPI.actor.cpp | 6 +++--- fdbclient/StorageServerInterface.cpp | 24 ++++++++++++++++++++++++ 2 files changed, 27 insertions(+), 3 deletions(-) diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index dc281b5c61..a509f7f6c4 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -6532,7 +6532,7 @@ ACTOR Future>> getRangeFeedMutations wait(getKeyRangeLocations(cx, keys, 100, - false, + Reverse::False, &StorageServerInterface::rangeFeed, TransactionInfo(TaskPriority::DefaultEndpoint, span.context))); @@ -6548,7 +6548,7 @@ ACTOR Future>> getRangeFeedMutations &StorageServerInterface::rangeFeed, req, TaskPriority::DefaultPromiseEndpoint, - false, + AtMostOnce::False, cx->enableLocalityLoadBalance ? &cx->queueModel : nullptr)); return Standalone>(rep.mutations, rep.arena); } @@ -6571,7 +6571,7 @@ ACTOR Future popRangeFeedMutationsActor(Reference db, Str wait(getKeyRangeLocations(cx, keys, 100, - false, + Reverse::False, &StorageServerInterface::rangeFeed, TransactionInfo(TaskPriority::DefaultEndpoint, span.context))); diff --git a/fdbclient/StorageServerInterface.cpp b/fdbclient/StorageServerInterface.cpp index d379a0fa69..b52ca0a8cd 100644 --- a/fdbclient/StorageServerInterface.cpp +++ b/fdbclient/StorageServerInterface.cpp @@ -296,6 +296,27 @@ void TSS_traceMismatch(TraceEvent& event, ASSERT(false); } +// split range +template <> +bool TSS_doCompare(const RangeFeedReply& src, const RangeFeedReply& tss) { + ASSERT(false); + return true; +} + +template <> +const char* TSS_mismatchTraceName(const RangeFeedRequest& req) { + ASSERT(false); + return ""; +} + +template <> +void TSS_traceMismatch(TraceEvent& event, + const RangeFeedRequest& req, + const RangeFeedReply& src, + const RangeFeedReply& tss) { + ASSERT(false); +} + // only record metrics for data reads template <> @@ -334,6 +355,9 @@ void TSSMetrics::recordLatency(const SplitRangeRequest& req, double ssLatency, d template <> void TSSMetrics::recordLatency(const GetKeyValuesStreamRequest& req, double ssLatency, double tssLatency) {} +template <> +void TSSMetrics::recordLatency(const RangeFeedRequest& req, double ssLatency, double tssLatency) {} + // ------------------- TEST_CASE("/StorageServerInterface/TSSCompare/TestComparison") { From 0989c28a6b191cfe4d95c22889900d0507259f38 Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Fri, 30 Jul 2021 15:23:42 -0700 Subject: [PATCH 28/72] made range feeds durable on the storage server --- fdbcli/fdbcli.actor.cpp | 6 +- fdbclient/DatabaseContext.h | 2 +- fdbclient/NativeAPI.actor.cpp | 8 +- fdbclient/StorageServerInterface.h | 24 +++--- fdbclient/SystemData.cpp | 30 ++++++++ fdbclient/SystemData.h | 8 ++ fdbserver/storageserver.actor.cpp | 115 +++++++++++++++++++++++++++-- 7 files changed, 167 insertions(+), 26 deletions(-) diff --git a/fdbcli/fdbcli.actor.cpp b/fdbcli/fdbcli.actor.cpp index 142850c7fc..4c57fbc3f8 100644 --- a/fdbcli/fdbcli.actor.cpp +++ b/fdbcli/fdbcli.actor.cpp @@ -3585,9 +3585,11 @@ ACTOR Future cli(CLIOptions opt, LineNoise* plinenoise) { is_error = true; continue; } - Standalone> res = wait(db->getRangeFeedMutations(tokens[2])); + Standalone> res = wait(db->getRangeFeedMutations(tokens[2])); for (auto& it : res) { - printf("%lld %s\n", it.version, it.mutation.toString().c_str()); + for (auto& it2 : it.mutations) { + printf("%lld %s\n", it.version, it2.toString().c_str()); + } } } else if (tokencmp(tokens[1], "pop")) { if (tokens.size() != 4) { diff --git a/fdbclient/DatabaseContext.h b/fdbclient/DatabaseContext.h index afb4f3a299..e28cff582c 100644 --- a/fdbclient/DatabaseContext.h +++ b/fdbclient/DatabaseContext.h @@ -252,7 +252,7 @@ public: // Management API, create snapshot Future createSnapshot(StringRef uid, StringRef snapshot_command); - Future>> getRangeFeedMutations(StringRef rangeID); + Future>> getRangeFeedMutations(StringRef rangeID); Future popRangeFeedMutations(StringRef rangeID, Version version); // private: diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index a509f7f6c4..fb292cf361 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -6517,8 +6517,8 @@ Future DatabaseContext::createSnapshot(StringRef uid, StringRef snapshot_c return createSnapshotActor(this, UID::fromString(uid_str), snapshot_command); } -ACTOR Future>> getRangeFeedMutationsActor(Reference db, - StringRef rangeID) { +ACTOR Future>> getRangeFeedMutationsActor(Reference db, + StringRef rangeID) { state Database cx(db); state Transaction tr(cx); state Key rangeIDKey = rangeID.withPrefix(rangeFeedPrefix); @@ -6550,10 +6550,10 @@ ACTOR Future>> getRangeFeedMutations TaskPriority::DefaultPromiseEndpoint, AtMostOnce::False, cx->enableLocalityLoadBalance ? &cx->queueModel : nullptr)); - return Standalone>(rep.mutations, rep.arena); + return Standalone>(rep.mutations, rep.arena); } -Future>> DatabaseContext::getRangeFeedMutations(StringRef rangeID) { +Future>> DatabaseContext::getRangeFeedMutations(StringRef rangeID) { return getRangeFeedMutationsActor(Reference::addRef(this), rangeID); } diff --git a/fdbclient/StorageServerInterface.h b/fdbclient/StorageServerInterface.h index 86745cb182..999bbb4054 100644 --- a/fdbclient/StorageServerInterface.h +++ b/fdbclient/StorageServerInterface.h @@ -623,27 +623,29 @@ struct SplitRangeRequest { } }; -struct MutationRefAndVersion { - MutationRef mutation; +struct MutationsAndVersionRef { + VectorRef mutations; Version version; - MutationRefAndVersion() {} - MutationRefAndVersion(MutationRef mutation, Version version) : mutation(mutation), version(version) {} - MutationRefAndVersion(Arena& to, MutationRef mutation, Version version) - : mutation(to, mutation), version(version) {} - MutationRefAndVersion(Arena& to, const MutationRefAndVersion& from) - : mutation(to, from.mutation), version(from.version) {} - int expectedSize() const { return mutation.expectedSize(); } + MutationsAndVersionRef() {} + explicit MutationsAndVersionRef(Version version) : version(version) {} + MutationsAndVersionRef(VectorRef mutations, Version version) + : mutations(mutations), version(version) {} + MutationsAndVersionRef(Arena& to, VectorRef mutations, Version version) + : mutations(to, mutations), version(version) {} + MutationsAndVersionRef(Arena& to, const MutationsAndVersionRef& from) + : mutations(to, from.mutations), version(from.version) {} + int expectedSize() const { return mutations.expectedSize(); } template void serialize(Ar& ar) { - serializer(ar, mutation, version); + serializer(ar, mutations, version); } }; struct RangeFeedReply { constexpr static FileIdentifier file_identifier = 11815134; - VectorRef mutations; + VectorRef mutations; bool cached; Arena arena; diff --git a/fdbclient/SystemData.cpp b/fdbclient/SystemData.cpp index 02c7f358c0..1a0fbc662b 100644 --- a/fdbclient/SystemData.cpp +++ b/fdbclient/SystemData.cpp @@ -1046,6 +1046,36 @@ KeyRange decodeRangeFeedValue(ValueRef const& value) { return range; } +const KeyRangeRef rangeFeedDurableKeys(LiteralStringRef("\xff\xff/rf/"), LiteralStringRef("\xff\xff/rf0")); +const KeyRef rangeFeedDurablePrefix = rangeFeedDurableKeys.begin; + +const Value rangeFeedDurableKey(Key const& feed, Version const& version) { + BinaryWriter wr(Unversioned()); + wr.serializeBytes(rangeFeedDurablePrefix); + wr << feed; + wr << version; + return wr.toValue(); +} +std::pair decodeRangeFeedDurableKey(ValueRef const& key) { + Key feed; + Version version; + BinaryReader reader(key.removePrefix(rangeFeedDurablePrefix), Unversioned()); + reader >> feed; + reader >> version; + return std::make_pair(feed, version); +} +const Value rangeFeedDurableValue(Standalone> const& mutations) { + BinaryWriter wr(IncludeVersion(ProtocolVersion::withRangeFeed())); + wr << mutations; + return wr.toValue(); +} +Standalone> decodeRangeFeedDurableValue(ValueRef const& value) { + Standalone> mutations; + BinaryReader reader(value, IncludeVersion()); + reader >> mutations; + return mutations; +} + const KeyRef configTransactionDescriptionKey = "\xff\xff/description"_sr; const KeyRange globalConfigKnobKeys = singleKeyRange("\xff\xff/globalKnobs"_sr); const KeyRangeRef configKnobKeys("\xff\xff/knobs/"_sr, "\xff\xff/knobs0"_sr); diff --git a/fdbclient/SystemData.h b/fdbclient/SystemData.h index ddc82287e4..316c528405 100644 --- a/fdbclient/SystemData.h +++ b/fdbclient/SystemData.h @@ -501,6 +501,14 @@ KeyRange decodeRangeFeedValue(ValueRef const& value); extern const KeyRef rangeFeedPrefix; extern const KeyRef rangeFeedPrivatePrefix; +extern const KeyRangeRef rangeFeedDurableKeys; +extern const KeyRef rangeFeedDurablePrefix; + +const Value rangeFeedDurableKey(Key const& feed, Version const& version); +std::pair decodeRangeFeedDurableKey(ValueRef const& key); +const Value rangeFeedDurableValue(Standalone> const& mutations); +Standalone> decodeRangeFeedDurableValue(ValueRef const& value); + // Configuration database special keys extern const KeyRef configTransactionDescriptionKey; extern const KeyRange globalConfigKnobKeys; diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index 8c8919b180..3ee30ce625 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -310,7 +310,8 @@ struct FetchInjectionInfo { }; struct RangeFeedInfo : ReferenceCounted { - std::deque> mutations; + std::deque> mutations; + Version durableVersion = invalidVersion; KeyRange range; Key id; }; @@ -581,6 +582,8 @@ public: KeyRangeMap>> keyRangeFeed; std::map> uidRangeFeed; + Deque, Version>> rangeFeedVersions; + std::set currentRangeFeeds; // newestAvailableVersion[k] // == invalidVersion -> k is unavailable at all versions @@ -1504,11 +1507,37 @@ ACTOR Future watchValueSendReply(StorageServer* data, } ACTOR Future rangeFeedQ(StorageServer* data, RangeFeedRequest req) { + state RangeFeedReply reply; wait(delay(0)); - RangeFeedReply reply; - for (auto& it : data->uidRangeFeed[req.rangeID]->mutations) { - reply.mutations.push_back(reply.arena, it); + auto& feedInfo = data->uidRangeFeed[req.rangeID]; + if (feedInfo->durableVersion == invalidVersion) { + for (auto& it : data->uidRangeFeed[req.rangeID]->mutations) { + reply.mutations.push_back(reply.arena, it); + } + } else { + state std::deque> mutationsDeque = + data->uidRangeFeed[req.rangeID]->mutations; + RangeResult res = wait(data->storage.readRange( + KeyRangeRef(rangeFeedDurableKey(req.rangeID, 0), rangeFeedDurableKey(req.rangeID, data->version.get())))); + if (res.empty()) { + data->uidRangeFeed[req.rangeID]->durableVersion = invalidVersion; + } + Version lastVersion = invalidVersion; + for (auto& kv : res) { + Key id; + Version version; + std::tie(id, version) = decodeRangeFeedDurableKey(kv.key); + auto mutations = decodeRangeFeedDurableValue(kv.value); + reply.mutations.push_back(reply.arena, MutationsAndVersionRef(mutations, version)); + lastVersion = version; + } + for (auto& it : mutationsDeque) { + if (it.version > lastVersion) { + reply.mutations.push_back(reply.arena, it); + } + } } + TraceEvent("RangeFeedQuery", data->thisServerID) .detail("RangeID", req.rangeID.printable()) .detail("Mutations", reply.mutations.size()); @@ -2628,7 +2657,11 @@ void applyMutation(StorageServer* self, self->watches.trigger(m.param1); for (auto& it : self->keyRangeFeed[m.param1]) { - it->mutations.push_back(MutationRefAndVersion(m, version)); + if (it->mutations.empty() || it->mutations.back().version != version) { + it->mutations.push_back(MutationsAndVersionRef(version)); + } + it->mutations.back().mutations.push_back_deep(it->mutations.back().arena(), m); + self->currentRangeFeeds.insert(it->id); } } else if (m.type == MutationRef::ClearRange) { data.erase(m.param1, m.param2); @@ -2640,7 +2673,11 @@ void applyMutation(StorageServer* self, auto ranges = self->keyRangeFeed.intersectingRanges(KeyRangeRef(m.param1, m.param2)); for (auto& r : ranges) { for (auto& it : r.value()) { - it->mutations.push_back(MutationRefAndVersion(m, version)); + if (it->mutations.empty() || it->mutations.back().version != version) { + it->mutations.push_back(MutationsAndVersionRef(version)); + } + it->mutations.back().mutations.push_back_deep(it->mutations.back().arena(), m); + self->currentRangeFeeds.insert(it->id); } } } @@ -3434,6 +3471,8 @@ static const KeyRangeRef persistByteSampleSampleKeys = LiteralStringRef(PERSIST_PREFIX "BS/" PERSIST_PREFIX "BS0")); static const KeyRef persistLogProtocol = LiteralStringRef(PERSIST_PREFIX "LogProtocol"); static const KeyRef persistPrimaryLocality = LiteralStringRef(PERSIST_PREFIX "PrimaryLocality"); +static const KeyRangeRef persistRangeFeedKeys = + KeyRangeRef(LiteralStringRef(PERSIST_PREFIX "RF/"), LiteralStringRef(PERSIST_PREFIX "RF0")); // data keys are unmangled (but never start with PERSIST_PREFIX because they are always in allKeys) class StorageUpdater { @@ -3575,6 +3614,11 @@ private: r->value().push_back(rangeFeedInfo); } data->keyRangeFeed.coalesce(rangeFeedRange.contents()); + auto& mLV = data->addVersionToMutationLog(data->data().getLatestVersion()); + data->addMutationToMutationLog(mLV, + MutationRef(MutationRef::SetValue, + persistRangeFeedKeys.begin.toString() + rangeFeedId.toString(), + m.param2)); } else if (m.param1.substr(1).startsWith(tssMappingKeys.begin) && (m.type == MutationRef::SetValue || m.type == MutationRef::ClearRange)) { if (!data->isTss()) { @@ -3962,6 +4006,12 @@ ACTOR Future update(StorageServer* data, bool* pReceivedUpdate) { .trackLatest(data->thisServerID.toString() + "/StorageServerSourceTLogID"); } + if (data->currentRangeFeeds.size()) { + data->rangeFeedVersions.push_back(std::make_pair( + std::vector(data->currentRangeFeeds.begin(), data->currentRangeFeeds.end()), ver)); + data->currentRangeFeeds.clear(); + } + data->noRecentUpdates.set(false); data->lastUpdate = now(); data->version.set(ver); // Triggers replies to waiting gets for new version(s) @@ -4060,6 +4110,26 @@ ACTOR Future updateStorage(StorageServer* data) { break; } + std::set modifiedRangeFeeds; + while (data->rangeFeedVersions.front().second < newOldestVersion) { + modifiedRangeFeeds.insert(data->rangeFeedVersions.front().first.begin(), + data->rangeFeedVersions.front().first.end()); + data->rangeFeedVersions.pop_front(); + } + + state std::vector updatedRangeFeeds(modifiedRangeFeeds.begin(), modifiedRangeFeeds.end()); + state int curFeed = 0; + while (curFeed < updatedRangeFeeds.size()) { + auto info = data->uidRangeFeed[updatedRangeFeeds[curFeed]]; + while (info->mutations.front().version < newOldestVersion) { + data->storage.writeKeyValue(KeyValueRef(rangeFeedDurableKey(info->id, info->mutations.front().version), + rangeFeedDurableValue(info->mutations.front().mutations))); + info->durableVersion = info->mutations.front().version; + info->mutations.pop_front(); + } + wait(yield(TaskPriority::UpdateStorage)); + } + // Set the new durable version as part of the outstanding change set, before commit if (startOldestVersion != newOldestVersion) data->storage.makeVersionDurable(newOldestVersion); @@ -4379,6 +4449,7 @@ ACTOR Future restoreDurableState(StorageServer* data, IKeyValueStore* stor state Future> fPrimaryLocality = storage->readValue(persistPrimaryLocality); state Future fShardAssigned = storage->readRange(persistShardAssignedKeys); state Future fShardAvailable = storage->readRange(persistShardAvailableKeys); + state Future fRangeFeeds = storage->readRange(persistRangeFeedKeys); state Promise byteSampleSampleRecovered; state Promise startByteSampleRestore; @@ -4387,7 +4458,7 @@ ACTOR Future restoreDurableState(StorageServer* data, IKeyValueStore* stor TraceEvent("ReadingDurableState", data->thisServerID).log(); wait(waitForAll(std::vector{ fFormat, fID, ftssPairID, fTssQuarantine, fVersion, fLogProtocol, fPrimaryLocality })); - wait(waitForAll(std::vector{ fShardAssigned, fShardAvailable })); + wait(waitForAll(std::vector{ fShardAssigned, fShardAvailable, fRangeFeeds })); wait(byteSampleSampleRecovered.getFuture()); TraceEvent("RestoringDurableState", data->thisServerID).log(); @@ -4465,6 +4536,26 @@ ACTOR Future restoreDurableState(StorageServer* data, IKeyValueStore* stor wait(yield()); } + state RangeResult rangeFeeds = fRangeFeeds.get(); + state int feedLoc; + for (feedLoc = 0; feedLoc < rangeFeeds.size(); feedLoc++) { + Key rangeFeedId = rangeFeeds[feedLoc].key.removePrefix(persistRangeFeedKeys.begin); + KeyRange rangeFeedRange = decodeRangeFeedValue(rangeFeeds[feedLoc].value); + TraceEvent("RestoringRangeFeed", data->thisServerID) + .detail("RangeID", rangeFeedId.printable()) + .detail("Range", rangeFeedRange.toString()); + Reference rangeFeedInfo(new RangeFeedInfo()); + rangeFeedInfo->range = rangeFeedRange; + rangeFeedInfo->id = rangeFeedId; + rangeFeedInfo->durableVersion = version; + data->uidRangeFeed[rangeFeedId] = rangeFeedInfo; + auto rs = data->keyRangeFeed.modify(rangeFeedRange); + for (auto r = rs.begin(); r != rs.end(); ++r) { + r->value().push_back(rangeFeedInfo); + } + wait(yield()); + } + data->keyRangeFeed.coalesce(allKeys); // TODO: why is this seemingly random delay here? wait(delay(0.0001)); @@ -4982,9 +5073,17 @@ ACTOR Future serveRangeFeedRequests(StorageServer* self, FutureStream serveRangeFeedPopRequests(StorageServer* self, FutureStream rangeFeedPops) { loop { RangeFeedPopRequest req = waitNext(rangeFeedPops); - while (self->uidRangeFeed[req.rangeID]->mutations.front().version < req.version) { + auto& feed = self->uidRangeFeed[req.rangeID]; + while (feed->mutations.front().version < req.version) { self->uidRangeFeed[req.rangeID]->mutations.pop_front(); } + if (feed->durableVersion != invalidVersion) { + self->storage.clearRange( + KeyRangeRef(rangeFeedDurableKey(feed->id, 0), rangeFeedDurableKey(feed->id, req.version))); + if (req.version > feed->durableVersion) { + feed->durableVersion = invalidVersion; + } + } TraceEvent("RangeFeedPopQuery", self->thisServerID) .detail("RangeID", req.rangeID.printable()) .detail("Version", req.version); From 125241743deb37d284f3b4a33cd95e8283300a4b Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Fri, 30 Jul 2021 16:01:46 -0700 Subject: [PATCH 29/72] added support for removing range feeds --- fdbserver/storageserver.actor.cpp | 52 ++++++++++++++++++++----------- 1 file changed, 33 insertions(+), 19 deletions(-) diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index 3ee30ce625..46646d968b 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -3599,26 +3599,40 @@ private: data->primaryLocality = BinaryReader::fromStringRef(m.param2, Unversioned()); auto& mLV = data->addVersionToMutationLog(data->data().getLatestVersion()); data->addMutationToMutationLog(mLV, MutationRef(MutationRef::SetValue, persistPrimaryLocality, m.param2)); - } else if (m.type == MutationRef::SetValue && m.param1.startsWith(rangeFeedPrivatePrefix)) { - Key rangeFeedId = m.param1.removePrefix(rangeFeedPrivatePrefix); - KeyRange rangeFeedRange = decodeRangeFeedValue(m.param2); - TraceEvent("AddingRangeFeed", data->thisServerID) - .detail("RangeID", rangeFeedId.printable()) - .detail("Range", rangeFeedRange.toString()); - Reference rangeFeedInfo(new RangeFeedInfo()); - rangeFeedInfo->range = rangeFeedRange; - rangeFeedInfo->id = rangeFeedId; - data->uidRangeFeed[rangeFeedId] = rangeFeedInfo; - auto rs = data->keyRangeFeed.modify(rangeFeedRange); - for (auto r = rs.begin(); r != rs.end(); ++r) { - r->value().push_back(rangeFeedInfo); + } else if ((m.type == MutationRef::SetValue || m.type == MutationRef::ClearRange) && + m.param1.startsWith(rangeFeedPrivatePrefix)) { + if (m.type == MutationRef::SetValue) { + Key rangeFeedId = m.param1.removePrefix(rangeFeedPrivatePrefix); + KeyRange rangeFeedRange = decodeRangeFeedValue(m.param2); + TraceEvent("AddingRangeFeed", data->thisServerID) + .detail("RangeID", rangeFeedId.printable()) + .detail("Range", rangeFeedRange.toString()); + Reference rangeFeedInfo(new RangeFeedInfo()); + rangeFeedInfo->range = rangeFeedRange; + rangeFeedInfo->id = rangeFeedId; + data->uidRangeFeed[rangeFeedId] = rangeFeedInfo; + auto rs = data->keyRangeFeed.modify(rangeFeedRange); + for (auto r = rs.begin(); r != rs.end(); ++r) { + r->value().push_back(rangeFeedInfo); + } + data->keyRangeFeed.coalesce(rangeFeedRange.contents()); + auto& mLV = data->addVersionToMutationLog(data->data().getLatestVersion()); + data->addMutationToMutationLog( + mLV, + MutationRef(MutationRef::SetValue, + persistRangeFeedKeys.begin.toString() + rangeFeedId.toString(), + m.param2)); + } else { + auto& mLV = data->addVersionToMutationLog(data->data().getLatestVersion()); + auto beginFeed = m.param1.removePrefix(rangeFeedPrivatePrefix); + auto endFeed = m.param2.removePrefix(rangeFeedPrivatePrefix); + data->addMutationToMutationLog(mLV, + MutationRef(MutationRef::ClearRange, + persistRangeFeedKeys.begin.toString() + beginFeed.toString(), + persistRangeFeedKeys.begin.toString() + endFeed.toString())); + data->uidRangeFeed.erase(data->uidRangeFeed.lower_bound(beginFeed), + data->uidRangeFeed.lower_bound(endFeed)); } - data->keyRangeFeed.coalesce(rangeFeedRange.contents()); - auto& mLV = data->addVersionToMutationLog(data->data().getLatestVersion()); - data->addMutationToMutationLog(mLV, - MutationRef(MutationRef::SetValue, - persistRangeFeedKeys.begin.toString() + rangeFeedId.toString(), - m.param2)); } else if (m.param1.substr(1).startsWith(tssMappingKeys.begin) && (m.type == MutationRef::SetValue || m.type == MutationRef::ClearRange)) { if (!data->isTss()) { From bc9a0e1315c0b2a40a1f99dc6d396963c8f18492 Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Mon, 9 Aug 2021 10:05:56 -0700 Subject: [PATCH 30/72] first attempt to add data distribution support for range feeds --- fdbclient/DatabaseContext.h | 4 +- fdbclient/NativeAPI.actor.cpp | 44 ++++- fdbclient/StorageServerInterface.h | 47 ++++- fdbserver/TLogServer.actor.cpp | 11 +- fdbserver/storageserver.actor.cpp | 287 ++++++++++++++++++++++------- 5 files changed, 308 insertions(+), 85 deletions(-) diff --git a/fdbclient/DatabaseContext.h b/fdbclient/DatabaseContext.h index e28cff582c..e6d98cef6e 100644 --- a/fdbclient/DatabaseContext.h +++ b/fdbclient/DatabaseContext.h @@ -252,7 +252,9 @@ public: // Management API, create snapshot Future createSnapshot(StringRef uid, StringRef snapshot_command); - Future>> getRangeFeedMutations(StringRef rangeID); + Future>> getRangeFeedMutations(StringRef rangeID, + KeyRangeRef range = allKeys); + Future>> getOverlappingRangeFeeds(KeyRangeRef ranges, Version minVersion); Future popRangeFeedMutations(StringRef rangeID, Version version); // private: diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index fb292cf361..c198994f48 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -6518,7 +6518,8 @@ Future DatabaseContext::createSnapshot(StringRef uid, StringRef snapshot_c } ACTOR Future>> getRangeFeedMutationsActor(Reference db, - StringRef rangeID) { + StringRef rangeID, + KeyRangeRef range) { state Database cx(db); state Transaction tr(cx); state Key rangeIDKey = rangeID.withPrefix(rangeFeedPrefix); @@ -6553,8 +6554,45 @@ ACTOR Future>> getRangeFeedMutation return Standalone>(rep.mutations, rep.arena); } -Future>> DatabaseContext::getRangeFeedMutations(StringRef rangeID) { - return getRangeFeedMutationsActor(Reference::addRef(this), rangeID); +Future>> DatabaseContext::getRangeFeedMutations(StringRef rangeID, + KeyRangeRef range) { + return getRangeFeedMutationsActor(Reference::addRef(this), rangeID, range); +} + +ACTOR Future>> getOverlappingRangeFeedsActor(Reference db, + KeyRangeRef range, + Version minVersion) { + state Database cx(db); + state Transaction tr(cx); + state Span span("NAPI:GetOverlappingRangeFeeds"_loc); + state vector>> locations = + wait(getKeyRangeLocations(cx, + range, + 100, + Reverse::False, + &StorageServerInterface::rangeFeed, + TransactionInfo(TaskPriority::DefaultEndpoint, span.context))); + + if (locations.size() > 1) { + throw unsupported_operation(); + } + + state OverlappingRangeFeedsRequest req; + req.range = range; + + OverlappingRangeFeedsReply rep = wait(loadBalance(cx.getPtr(), + locations[0].second, + &StorageServerInterface::overlappingRangeFeeds, + req, + TaskPriority::DefaultPromiseEndpoint, + AtMostOnce::False, + cx->enableLocalityLoadBalance ? &cx->queueModel : nullptr)); + return rep.rangeIds; +} + +Future>> DatabaseContext::getOverlappingRangeFeeds(KeyRangeRef range, + Version minVersion) { + return getOverlappingRangeFeedsActor(Reference::addRef(this), range, minVersion); } ACTOR Future popRangeFeedMutationsActor(Reference db, StringRef rangeID, Version version) { diff --git a/fdbclient/StorageServerInterface.h b/fdbclient/StorageServerInterface.h index 999bbb4054..9efea0e732 100644 --- a/fdbclient/StorageServerInterface.h +++ b/fdbclient/StorageServerInterface.h @@ -78,6 +78,7 @@ struct StorageServerInterface { RequestStream getReadHotRanges; RequestStream getRangeSplitPoints; RequestStream rangeFeed; + RequestStream overlappingRangeFeeds; RequestStream rangeFeedPop; RequestStream getKeyValuesStream; @@ -120,11 +121,13 @@ struct StorageServerInterface { RequestStream(getValue.getEndpoint().getAdjustedEndpoint(11)); getRangeSplitPoints = RequestStream(getValue.getEndpoint().getAdjustedEndpoint(12)); - rangeFeed = RequestStream(getValue.getEndpoint().getAdjustedEndpoint(13)); - rangeFeedPop = - RequestStream(getValue.getEndpoint().getAdjustedEndpoint(14)); getKeyValuesStream = - RequestStream(getValue.getEndpoint().getAdjustedEndpoint(15)); + RequestStream(getValue.getEndpoint().getAdjustedEndpoint(13)); + rangeFeed = RequestStream(getValue.getEndpoint().getAdjustedEndpoint(14)); + overlappingRangeFeeds = + RequestStream(getValue.getEndpoint().getAdjustedEndpoint(15)); + rangeFeedPop = + RequestStream(getValue.getEndpoint().getAdjustedEndpoint(16)); } } else { ASSERT(Ar::isDeserializing); @@ -166,9 +169,10 @@ struct StorageServerInterface { streams.push_back(watchValue.getReceiver()); streams.push_back(getReadHotRanges.getReceiver()); streams.push_back(getRangeSplitPoints.getReceiver()); - streams.push_back(rangeFeed.getReceiver()); - streams.push_back(rangeFeedPop.getReceiver()); streams.push_back(getKeyValuesStream.getReceiver(TaskPriority::LoadBalancedEndpoint)); + streams.push_back(rangeFeed.getReceiver()); + streams.push_back(overlappingRangeFeeds.getReceiver()); + streams.push_back(rangeFeedPop.getReceiver()); FlowTransport::transport().addEndpoints(streams); } }; @@ -662,7 +666,7 @@ struct RangeFeedRequest { ReplyPromise reply; RangeFeedRequest() {} - RangeFeedRequest(Key const& rangeID) : rangeID(rangeID) {} + explicit RangeFeedRequest(Key const& rangeID) : rangeID(rangeID) {} template void serialize(Ar& ar) { @@ -685,6 +689,35 @@ struct RangeFeedPopRequest { } }; +struct OverlappingRangeFeedsReply { + constexpr static FileIdentifier file_identifier = 11815134; + std::vector> rangeIds; + bool cached; + Arena arena; + + OverlappingRangeFeedsReply() : cached(false) {} + explicit OverlappingRangeFeedsReply(std::vector> const& rangeIds) + : rangeIds(rangeIds), cached(false) {} + + template + void serialize(Ar& ar) { + serializer(ar, rangeIds, arena); + } +}; +struct OverlappingRangeFeedsRequest { + constexpr static FileIdentifier file_identifier = 10726174; + KeyRange range; + ReplyPromise reply; + + OverlappingRangeFeedsRequest() {} + explicit OverlappingRangeFeedsRequest(KeyRange const& range) : range(range) {} + + template + void serialize(Ar& ar) { + serializer(ar, range, reply); + } +}; + struct GetStorageMetricsReply { constexpr static FileIdentifier file_identifier = 15491478; StorageMetrics load; diff --git a/fdbserver/TLogServer.actor.cpp b/fdbserver/TLogServer.actor.cpp index d8e27a78ba..2643d02e01 100644 --- a/fdbserver/TLogServer.actor.cpp +++ b/fdbserver/TLogServer.actor.cpp @@ -1861,10 +1861,13 @@ ACTOR Future tLogPeekMessages(TLogData* self, TLogPeekRequest req, Referen reply.end = endVersion; reply.onlySpilled = onlySpilled; - //TraceEvent("TlogPeek", self->dbgid).detail("LogId", logData->logId).detail("Tag", req.tag.toString()). - // detail("BeginVer", req.begin).detail("EndVer", reply.end). - // detail("MsgBytes", reply.messages.expectedSize()). - // detail("ForAddress", req.reply.getEndpoint().getPrimaryAddress()); + TraceEvent("TlogPeek", self->dbgid) + .detail("LogId", logData->logId) + .detail("Tag", req.tag.toString()) + .detail("BeginVer", req.begin) + .detail("EndVer", reply.end) + .detail("MsgBytes", reply.messages.expectedSize()) + .detail("ForAddress", req.reply.getEndpoint().getPrimaryAddress()); if (req.sequence.present()) { auto& trackerData = logData->peekTracker[peekId]; diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index 46646d968b..b4cce58fb3 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -94,6 +94,7 @@ struct AddingShard : NonCopyable { Future fetchClient; // holds FetchKeys() actor Promise fetchComplete; Promise readWrite; + PromiseStream rangeFeedRemovals; // During the Fetching phase, it saves newer mutations whose version is greater or equal to fetchClient's // fetchVersion, while the shard is still busy catching up with fetchClient. It applies these updates after fetching @@ -130,7 +131,7 @@ struct AddingShard : NonCopyable { readWrite.send(Void()); } - void addMutation(Version version, MutationRef const& mutation); + void addMutation(Version version, bool fromFetch, MutationRef const& mutation); bool isTransferred() const { return phase == Waiting; } }; @@ -159,7 +160,7 @@ public: bool notAssigned() const { return !readWrite && !adding; } bool assigned() const { return readWrite || adding; } bool isInVersionedData() const { return readWrite || (adding && adding->isTransferred()); } - void addMutation(Version version, MutationRef const& mutation); + void addMutation(Version version, bool fromFetch, MutationRef const& mutation); bool isFetched() const { return readWrite || (adding && adding->fetchComplete.isSet()); } const char* debugDescribeState() const { @@ -306,6 +307,7 @@ static int mvccStorageBytes(MutationRef const& m) { struct FetchInjectionInfo { Arena arena; + Version transferredVersion; vector changes; }; @@ -583,6 +585,7 @@ public: KeyRangeMap>> keyRangeFeed; std::map> uidRangeFeed; Deque, Version>> rangeFeedVersions; + std::map> rangeFeedRemovals; std::set currentRangeFeeds; // newestAvailableVersion[k] @@ -882,6 +885,7 @@ public: shards.insert(newShard->keys, Reference(newShard)); } void addMutation(Version version, + bool fromFetch, MutationRef const& mutation, KeyRangeRef const& shard, UpdateEagerReadInfo* eagerReads); @@ -1506,7 +1510,21 @@ ACTOR Future watchValueSendReply(StorageServer* data, } } -ACTOR Future rangeFeedQ(StorageServer* data, RangeFeedRequest req) { +ACTOR Future overlappingRangeFeedsQ(StorageServer* data, OverlappingRangeFeedsRequest req) { + wait(delay(0)); + auto ranges = data->keyRangeFeed.intersectingRanges(req.range); + std::set> rangeIds; + for (auto r : ranges) { + for (auto& it : r.value()) { + rangeIds.insert(std::make_pair(it->id, it->range)); + } + } + OverlappingRangeFeedsReply reply(std::vector>(rangeIds.begin(), rangeIds.end())); + req.reply.send(reply); + return Void(); +} + +ACTOR Future getRangeFeedMutations(StorageServer* data, RangeFeedRequest req) { state RangeFeedReply reply; wait(delay(0)); auto& feedInfo = data->uidRangeFeed[req.rangeID]; @@ -1537,11 +1555,12 @@ ACTOR Future rangeFeedQ(StorageServer* data, RangeFeedRequest req) { } } } + return reply; +} - TraceEvent("RangeFeedQuery", data->thisServerID) - .detail("RangeID", req.rangeID.printable()) - .detail("Mutations", reply.mutations.size()); - req.reply.send(reply); +ACTOR Future rangeFeedQ(StorageServer* data, RangeFeedRequest req) { + RangeFeedReply rep = wait(getRangeFeedMutations(data, req)); + req.reply.send(rep); return Void(); } @@ -2622,7 +2641,8 @@ void applyMutation(StorageServer* self, MutationRef const& m, Arena& arena, StorageServer::VersionedData& data, - Version version) { + Version version, + bool fromFetch) { // m is expected to be in arena already // Clear split keys are added to arena StorageMetrics metrics; @@ -2656,12 +2676,14 @@ void applyMutation(StorageServer* self, data.insert(m.param1, ValueOrClearToRef::value(m.param2)); self->watches.trigger(m.param1); - for (auto& it : self->keyRangeFeed[m.param1]) { - if (it->mutations.empty() || it->mutations.back().version != version) { - it->mutations.push_back(MutationsAndVersionRef(version)); + if (!fromFetch) { + for (auto& it : self->keyRangeFeed[m.param1]) { + if (it->mutations.empty() || it->mutations.back().version != version) { + it->mutations.push_back(MutationsAndVersionRef(version)); + } + it->mutations.back().mutations.push_back_deep(it->mutations.back().arena(), m); + self->currentRangeFeeds.insert(it->id); } - it->mutations.back().mutations.push_back_deep(it->mutations.back().arena(), m); - self->currentRangeFeeds.insert(it->id); } } else if (m.type == MutationRef::ClearRange) { data.erase(m.param1, m.param2); @@ -2670,14 +2692,16 @@ void applyMutation(StorageServer* self, data.insert(m.param1, ValueOrClearToRef::clearTo(m.param2)); self->watches.triggerRange(m.param1, m.param2); - auto ranges = self->keyRangeFeed.intersectingRanges(KeyRangeRef(m.param1, m.param2)); - for (auto& r : ranges) { - for (auto& it : r.value()) { - if (it->mutations.empty() || it->mutations.back().version != version) { - it->mutations.push_back(MutationsAndVersionRef(version)); + if (!fromFetch) { + auto ranges = self->keyRangeFeed.intersectingRanges(KeyRangeRef(m.param1, m.param2)); + for (auto& r : ranges) { + for (auto& it : r.value()) { + if (it->mutations.empty() || it->mutations.back().version != version) { + it->mutations.push_back(MutationsAndVersionRef(version)); + } + it->mutations.back().mutations.push_back_deep(it->mutations.back().arena(), m); + self->currentRangeFeeds.insert(it->id); } - it->mutations.back().mutations.push_back_deep(it->mutations.back().arena(), m); - self->currentRangeFeeds.insert(it->id); } } } @@ -2760,34 +2784,34 @@ void coalesceShards(StorageServer* data, KeyRangeRef keys) { } template -void addMutation(T& target, Version version, MutationRef const& mutation) { - target.addMutation(version, mutation); +void addMutation(T& target, Version version, bool fromFetch, MutationRef const& mutation) { + target.addMutation(version, fromFetch, mutation); } template -void addMutation(Reference& target, Version version, MutationRef const& mutation) { - addMutation(*target, version, mutation); +void addMutation(Reference& target, Version version, bool fromFetch, MutationRef const& mutation) { + addMutation(*target, version, fromFetch, mutation); } template void splitMutations(StorageServer* data, KeyRangeMap& map, VerUpdateRef const& update) { for (int i = 0; i < update.mutations.size(); i++) { - splitMutation(data, map, update.mutations[i], update.version); + splitMutation(data, map, update.mutations[i], update.version, update.version); } } template -void splitMutation(StorageServer* data, KeyRangeMap& map, MutationRef const& m, Version ver) { +void splitMutation(StorageServer* data, KeyRangeMap& map, MutationRef const& m, Version ver, bool fromFetch) { if (isSingleKeyMutation((MutationRef::Type)m.type)) { if (!SHORT_CIRCUT_ACTUAL_STORAGE || !normalKeys.contains(m.param1)) - addMutation(map.rangeContaining(m.param1)->value(), ver, m); + addMutation(map.rangeContaining(m.param1)->value(), ver, fromFetch, m); } else if (m.type == MutationRef::ClearRange) { KeyRangeRef mKeys(m.param1, m.param2); if (!SHORT_CIRCUT_ACTUAL_STORAGE || !normalKeys.contains(mKeys)) { auto r = map.intersectingRanges(mKeys); for (auto i = r.begin(); i != r.end(); ++i) { KeyRangeRef k = mKeys & i->range(); - addMutation(i->value(), ver, MutationRef((MutationRef::Type)m.type, k.begin, k.end)); + addMutation(i->value(), ver, fromFetch, MutationRef((MutationRef::Type)m.type, k.begin, k.end)); } } } else @@ -2890,12 +2914,129 @@ ACTOR Future tryGetRange(PromiseStream results, Transaction* } } +#define PERSIST_PREFIX "\xff\xff" + +// Immutable +static const KeyValueRef persistFormat(LiteralStringRef(PERSIST_PREFIX "Format"), + LiteralStringRef("FoundationDB/StorageServer/1/4")); +static const KeyRangeRef persistFormatReadableRange(LiteralStringRef("FoundationDB/StorageServer/1/2"), + LiteralStringRef("FoundationDB/StorageServer/1/5")); +static const KeyRef persistID = LiteralStringRef(PERSIST_PREFIX "ID"); +static const KeyRef persistTssPairID = LiteralStringRef(PERSIST_PREFIX "tssPairID"); +static const KeyRef persistTssQuarantine = LiteralStringRef(PERSIST_PREFIX "tssQ"); + +// (Potentially) change with the durable version or when fetchKeys completes +static const KeyRef persistVersion = LiteralStringRef(PERSIST_PREFIX "Version"); +static const KeyRangeRef persistShardAssignedKeys = + KeyRangeRef(LiteralStringRef(PERSIST_PREFIX "ShardAssigned/"), LiteralStringRef(PERSIST_PREFIX "ShardAssigned0")); +static const KeyRangeRef persistShardAvailableKeys = + KeyRangeRef(LiteralStringRef(PERSIST_PREFIX "ShardAvailable/"), LiteralStringRef(PERSIST_PREFIX "ShardAvailable0")); +static const KeyRangeRef persistByteSampleKeys = + KeyRangeRef(LiteralStringRef(PERSIST_PREFIX "BS/"), LiteralStringRef(PERSIST_PREFIX "BS0")); +static const KeyRangeRef persistByteSampleSampleKeys = + KeyRangeRef(LiteralStringRef(PERSIST_PREFIX "BS/" PERSIST_PREFIX "BS/"), + LiteralStringRef(PERSIST_PREFIX "BS/" PERSIST_PREFIX "BS0")); +static const KeyRef persistLogProtocol = LiteralStringRef(PERSIST_PREFIX "LogProtocol"); +static const KeyRef persistPrimaryLocality = LiteralStringRef(PERSIST_PREFIX "PrimaryLocality"); +static const KeyRangeRef persistRangeFeedKeys = + KeyRangeRef(LiteralStringRef(PERSIST_PREFIX "RF/"), LiteralStringRef(PERSIST_PREFIX "RF0")); +// data keys are unmangled (but never start with PERSIST_PREFIX because they are always in allKeys) + +ACTOR Future fetchRangeFeed(StorageServer* data, Key rangeId, KeyRange range) { + + TraceEvent("FetchRangeFeed", data->thisServerID) + .detail("RangeID", rangeId.printable()) + .detail("Range", range.toString()); + Reference rangeFeedInfo(new RangeFeedInfo()); + rangeFeedInfo->range = range; + rangeFeedInfo->id = rangeId; + data->uidRangeFeed[rangeId] = rangeFeedInfo; + auto rs = data->keyRangeFeed.modify(range); + for (auto r = rs.begin(); r != rs.end(); ++r) { + r->value().push_back(rangeFeedInfo); + } + data->keyRangeFeed.coalesce(range.contents()); + auto& mLV = data->addVersionToMutationLog(data->data().getLatestVersion()); + data->addMutationToMutationLog(mLV, + MutationRef(MutationRef::SetValue, + persistRangeFeedKeys.begin.toString() + rangeId.toString(), + rangeFeedValue(range))); + + state Standalone> mutations = + wait(data->cx->getRangeFeedMutations(rangeId, range)); + state RangeFeedRequest req; + req.rangeID = rangeId; + state RangeFeedReply rep = wait(getRangeFeedMutations(data, req)); + state int mLoc = 0; + state int rLoc = 0; + while (mLoc < mutations.size() && rLoc < rep.mutations.size()) { + if (mutations[mLoc].version < rep.mutations[rLoc].version) { + // write mutation to disk + mLoc++; + } else if (mutations[mLoc].version == rep.mutations[rLoc].version) { + // merge mutations and write to disk + mLoc++; + rLoc++; + } else { + rLoc++; + } + } + while (mLoc < mutations.size()) { + // write mutation to disk + mLoc++; + } +} + +ACTOR Future dispatchRangeFeeds(StorageServer* data, UID fetchKeysID, KeyRange keys, Version fetchVersion) { + // find overlapping range feeds + state std::map> feedFetches; + state PromiseStream removals; + data->rangeFeedRemovals[fetchKeysID] = removals; + try { + state std::vector> feeds = + wait(data->cx->getOverlappingRangeFeeds(keys, fetchVersion)); + for (auto& feed : feeds) { + feedFetches[feed.first] = fetchRangeFeed(data, feed.first, feed.second); + } + + loop { + Future nextFeed = Never(); + if (!removals.getFuture().isReady()) { + bool done = true; + while (!feedFetches.empty()) { + if (feedFetches.begin()->second.isReady()) { + feedFetches.erase(feedFetches.begin()); + } else { + nextFeed = feedFetches.begin()->second; + done = false; + } + } + if (done) { + data->rangeFeedRemovals.erase(fetchKeysID); + return Void(); + } + } + choose { + when(Key remove = waitNext(removals.getFuture())) { feedFetches.erase(remove); } + when(wait(nextFeed)) {} + } + } + + } catch (Error& e) { + if (!data->shuttingDown) { + data->rangeFeedRemovals.erase(fetchKeysID); + } + throw; + } +} + ACTOR Future fetchKeys(StorageServer* data, AddingShard* shard) { state const UID fetchKeysID = deterministicRandom()->randomUniqueID(); state TraceInterval interval("FetchKeys"); state KeyRange keys = shard->keys; state Future warningLogger = logFetchKeysWarning(shard); state const double startTime = now(); + state Version fetchVersion = invalidVersion; state FetchKeysMetricReporter metricReporter(fetchKeysID, startTime, keys, @@ -2957,7 +3098,6 @@ ACTOR Future fetchKeys(StorageServer* data, AddingShard* shard) { // Get the history state int debug_getRangeRetries = 0; state int debug_nextRetryToLog = 1; - state bool isTooOld = false; // FIXME: The client cache does not notice when servers are added to a team. To read from a local storage server // we must refresh the cache manually. @@ -2965,7 +3105,7 @@ ACTOR Future fetchKeys(StorageServer* data, AddingShard* shard) { loop { state Transaction tr(data->cx); - state Version fetchVersion = data->version.get(); + fetchVersion = data->version.get(); TraceEvent(SevDebug, "FetchKeysUnblocked", data->thisServerID) .detail("FKID", interval.pairID) @@ -3107,8 +3247,10 @@ ACTOR Future fetchKeys(StorageServer* data, AddingShard* shard) { // being recovered. Instead we wait for the updateStorage loop to commit something (and consequently also what // we have written) + state Future fetchDurable = data->durableVersion.whenAtLeast(data->storageVersion() + 1); + holdingFKPL.release(); - wait(data->durableVersion.whenAtLeast(data->storageVersion() + 1)); + wait(fetchDurable); TraceEvent(SevDebug, "FKAfterFinalCommit", data->thisServerID) .detail("FKID", interval.pairID) @@ -3150,8 +3292,9 @@ ACTOR Future fetchKeys(StorageServer* data, AddingShard* shard) { // Eager reads will be done for them by update(), and the mutations will come back through // AddingShard::addMutations and be applied to versionedMap and mutationLog as normal. The lie about their // version is acceptable because this shard will never be read at versions < transferredVersion + + batch->transferredVersion = shard->transferredVersion; for (auto i = shard->updates.begin(); i != shard->updates.end(); ++i) { - i->version = shard->transferredVersion; batch->arena.dependsOn(i->arena()); } @@ -3232,7 +3375,8 @@ AddingShard::AddingShard(StorageServer* server, KeyRangeRef const& keys) fetchClient = fetchKeys(server, this); } -void AddingShard::addMutation(Version version, MutationRef const& mutation) { +void AddingShard::addMutation(Version version, bool fromFetch, MutationRef const& mutation) { + ASSERT(!fromFetch); if (mutation.type == mutation.ClearRange) { ASSERT(keys.begin <= mutation.param1 && mutation.param2 <= keys.end); } else if (isSingleKeyMutation((MutationRef::Type)mutation.type)) { @@ -3255,19 +3399,39 @@ void AddingShard::addMutation(Version version, MutationRef const& mutation) { } // Add the mutation to the version. updates.back().mutations.push_back_deep(updates.back().arena(), mutation); + if (mutation.type == MutationRef::SetValue) { + for (auto& it : server->keyRangeFeed[mutation.param1]) { + if (it->mutations.empty() || it->mutations.back().version != version) { + it->mutations.push_back(MutationsAndVersionRef(version)); + } + it->mutations.back().mutations.push_back_deep(it->mutations.back().arena(), mutation); + server->currentRangeFeeds.insert(it->id); + } + } else if (mutation.type == MutationRef::ClearRange) { + auto ranges = server->keyRangeFeed.intersectingRanges(KeyRangeRef(mutation.param1, mutation.param2)); + for (auto& r : ranges) { + for (auto& it : r.value()) { + if (it->mutations.empty() || it->mutations.back().version != version) { + it->mutations.push_back(MutationsAndVersionRef(version)); + } + it->mutations.back().mutations.push_back_deep(it->mutations.back().arena(), mutation); + server->currentRangeFeeds.insert(it->id); + } + } + } } else if (phase == Waiting) { - server->addMutation(version, mutation, keys, server->updateEagerReads); + server->addMutation(version, fromFetch, mutation, keys, server->updateEagerReads); } else ASSERT(false); } -void ShardInfo::addMutation(Version version, MutationRef const& mutation) { +void ShardInfo::addMutation(Version version, bool fromFetch, MutationRef const& mutation) { ASSERT((void*)this); ASSERT(keys.contains(mutation.param1)); if (adding) - adding->addMutation(version, mutation); + adding->addMutation(version, fromFetch, mutation); else if (readWrite) - readWrite->addMutation(version, mutation, this->keys, readWrite->updateEagerReads); + readWrite->addMutation(version, fromFetch, mutation, this->keys, readWrite->updateEagerReads); else if (mutation.type != MutationRef::ClearRange) { TraceEvent(SevError, "DeliveredToNotAssigned") .detail("Version", version) @@ -3418,6 +3582,7 @@ void rollback(StorageServer* data, Version rollbackVersion, Version nextVersion) } void StorageServer::addMutation(Version version, + bool fromFetch, MutationRef const& mutation, KeyRangeRef const& shard, UpdateEagerReadInfo* eagerReads) { @@ -3432,7 +3597,7 @@ void StorageServer::addMutation(Version version, .detail("UID", thisServerID) .detail("ShardBegin", shard.begin) .detail("ShardEnd", shard.end); - applyMutation(this, expanded, mLog.arena(), mutableData(), version); + applyMutation(this, expanded, mLog.arena(), mutableData(), version, fromFetch); // printf("\nSSUpdate: Printing versioned tree after applying mutation\n"); // mutableData().printTree(version); } @@ -3447,34 +3612,6 @@ struct OrderByVersion { } }; -#define PERSIST_PREFIX "\xff\xff" - -// Immutable -static const KeyValueRef persistFormat(LiteralStringRef(PERSIST_PREFIX "Format"), - LiteralStringRef("FoundationDB/StorageServer/1/4")); -static const KeyRangeRef persistFormatReadableRange(LiteralStringRef("FoundationDB/StorageServer/1/2"), - LiteralStringRef("FoundationDB/StorageServer/1/5")); -static const KeyRef persistID = LiteralStringRef(PERSIST_PREFIX "ID"); -static const KeyRef persistTssPairID = LiteralStringRef(PERSIST_PREFIX "tssPairID"); -static const KeyRef persistTssQuarantine = LiteralStringRef(PERSIST_PREFIX "tssQ"); - -// (Potentially) change with the durable version or when fetchKeys completes -static const KeyRef persistVersion = LiteralStringRef(PERSIST_PREFIX "Version"); -static const KeyRangeRef persistShardAssignedKeys = - KeyRangeRef(LiteralStringRef(PERSIST_PREFIX "ShardAssigned/"), LiteralStringRef(PERSIST_PREFIX "ShardAssigned0")); -static const KeyRangeRef persistShardAvailableKeys = - KeyRangeRef(LiteralStringRef(PERSIST_PREFIX "ShardAvailable/"), LiteralStringRef(PERSIST_PREFIX "ShardAvailable0")); -static const KeyRangeRef persistByteSampleKeys = - KeyRangeRef(LiteralStringRef(PERSIST_PREFIX "BS/"), LiteralStringRef(PERSIST_PREFIX "BS0")); -static const KeyRangeRef persistByteSampleSampleKeys = - KeyRangeRef(LiteralStringRef(PERSIST_PREFIX "BS/" PERSIST_PREFIX "BS/"), - LiteralStringRef(PERSIST_PREFIX "BS/" PERSIST_PREFIX "BS0")); -static const KeyRef persistLogProtocol = LiteralStringRef(PERSIST_PREFIX "LogProtocol"); -static const KeyRef persistPrimaryLocality = LiteralStringRef(PERSIST_PREFIX "PrimaryLocality"); -static const KeyRangeRef persistRangeFeedKeys = - KeyRangeRef(LiteralStringRef(PERSIST_PREFIX "RF/"), LiteralStringRef(PERSIST_PREFIX "RF0")); -// data keys are unmangled (but never start with PERSIST_PREFIX because they are always in allKeys) - class StorageUpdater { public: StorageUpdater() @@ -3484,7 +3621,7 @@ public: : fromVersion(fromVersion), currentVersion(fromVersion), restoredVersion(restoredVersion), processedStartKey(false), processedCacheStartKey(false) {} - void applyMutation(StorageServer* data, MutationRef const& m, Version ver) { + void applyMutation(StorageServer* data, MutationRef const& m, Version ver, bool fromFetch) { //TraceEvent("SSNewVersion", data->thisServerID).detail("VerWas", data->mutableData().latestVersion).detail("ChVer", ver); if (currentVersion != ver) { @@ -3505,7 +3642,7 @@ public: // DEBUG_MUTATION("SSUpdateMutation", changes[c].version, *m); //} - splitMutation(data, data->shards, m, ver); + splitMutation(data, data->shards, m, ver, fromFetch); } if (data->otherError.getFuture().isReady()) @@ -3873,7 +4010,7 @@ ACTOR Future update(StorageServer* data, bool* pReceivedUpdate) { state int mutationNum = 0; state VerUpdateRef* pUpdate = &fii.changes[changeNum]; for (; mutationNum < pUpdate->mutations.size(); mutationNum++) { - updater.applyMutation(data, pUpdate->mutations[mutationNum], pUpdate->version); + updater.applyMutation(data, pUpdate->mutations[mutationNum], fii.transferredVersion, true); mutationBytes += pUpdate->mutations[mutationNum].totalSize(); // data->counters.mutationBytes or data->counters.mutations should not be updated because they should // have counted when the mutations arrive from cursor initially. @@ -3950,7 +4087,7 @@ ACTOR Future update(StorageServer* data, bool* pReceivedUpdate) { //TraceEvent("SSPeekMutation", data->thisServerID).detail("Mutation", msg.toString()).detail("Version", cloneCursor2->version().toString()); } - updater.applyMutation(data, msg, ver); + updater.applyMutation(data, msg, ver, false); mutationBytes += msg.totalSize(); data->counters.mutationBytes += msg.totalSize(); ++data->counters.mutations; @@ -5084,6 +5221,15 @@ ACTOR Future serveRangeFeedRequests(StorageServer* self, FutureStream serveOverlappingRangeFeedsRequests( + StorageServer* self, + FutureStream overlappingRangeFeeds) { + loop { + OverlappingRangeFeedsRequest req = waitNext(overlappingRangeFeeds); + self->actors.add(self->readGuard(req, overlappingRangeFeedsQ)); + } +} + ACTOR Future serveRangeFeedPopRequests(StorageServer* self, FutureStream rangeFeedPops) { loop { RangeFeedPopRequest req = waitNext(rangeFeedPops); @@ -5155,6 +5301,7 @@ ACTOR Future storageServerCore(StorageServer* self, StorageServerInterface self->actors.add(serveGetKeyRequests(self, ssi.getKey.getFuture())); self->actors.add(serveWatchValueRequests(self, ssi.watchValue.getFuture())); self->actors.add(serveRangeFeedRequests(self, ssi.rangeFeed.getFuture())); + self->actors.add(serveOverlappingRangeFeedsRequests(self, ssi.overlappingRangeFeeds.getFuture())); self->actors.add(serveRangeFeedPopRequests(self, ssi.rangeFeedPop.getFuture())); self->actors.add(traceRole(Role::STORAGE_SERVER, ssi.id())); self->actors.add(reportStorageServerState(self)); From 27f87471ab3049f470c5d07d77cab3a7c90a5652 Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Mon, 9 Aug 2021 12:48:38 -0700 Subject: [PATCH 31/72] fixed compile errors --- fdbclient/StorageServerInterface.cpp | 25 ++++++++++++++++++++++++- fdbserver/storageserver.actor.cpp | 16 ++++++++++------ 2 files changed, 34 insertions(+), 7 deletions(-) diff --git a/fdbclient/StorageServerInterface.cpp b/fdbclient/StorageServerInterface.cpp index b52ca0a8cd..417fe03f36 100644 --- a/fdbclient/StorageServerInterface.cpp +++ b/fdbclient/StorageServerInterface.cpp @@ -296,7 +296,7 @@ void TSS_traceMismatch(TraceEvent& event, ASSERT(false); } -// split range +// range feed template <> bool TSS_doCompare(const RangeFeedReply& src, const RangeFeedReply& tss) { ASSERT(false); @@ -317,6 +317,26 @@ void TSS_traceMismatch(TraceEvent& event, ASSERT(false); } +template <> +bool TSS_doCompare(const OverlappingRangeFeedsReply& src, const OverlappingRangeFeedsReply& tss) { + ASSERT(false); + return true; +} + +template <> +const char* TSS_mismatchTraceName(const OverlappingRangeFeedsRequest& req) { + ASSERT(false); + return ""; +} + +template <> +void TSS_traceMismatch(TraceEvent& event, + const OverlappingRangeFeedsRequest& req, + const OverlappingRangeFeedsReply& src, + const OverlappingRangeFeedsReply& tss) { + ASSERT(false); +} + // only record metrics for data reads template <> @@ -358,6 +378,9 @@ void TSSMetrics::recordLatency(const GetKeyValuesStreamRequest& req, double ssLa template <> void TSSMetrics::recordLatency(const RangeFeedRequest& req, double ssLatency, double tssLatency) {} +template <> +void TSSMetrics::recordLatency(const OverlappingRangeFeedsRequest& req, double ssLatency, double tssLatency) {} + // ------------------- TEST_CASE("/StorageServerInterface/TSSCompare/TestComparison") { diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index b4cce58fb3..a832adc43c 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -307,7 +307,6 @@ static int mvccStorageBytes(MutationRef const& m) { struct FetchInjectionInfo { Arena arena; - Version transferredVersion; vector changes; }; @@ -1513,12 +1512,16 @@ ACTOR Future watchValueSendReply(StorageServer* data, ACTOR Future overlappingRangeFeedsQ(StorageServer* data, OverlappingRangeFeedsRequest req) { wait(delay(0)); auto ranges = data->keyRangeFeed.intersectingRanges(req.range); - std::set> rangeIds; + std::map rangeIds; for (auto r : ranges) { for (auto& it : r.value()) { - rangeIds.insert(std::make_pair(it->id, it->range)); + rangeIds[it->id] = it->range; } } + std::vector> result; + for (auto& it : rangeIds) { + result.push_back(std::make_pair(it.first, it.second)); + } OverlappingRangeFeedsReply reply(std::vector>(rangeIds.begin(), rangeIds.end())); req.reply.send(reply); return Void(); @@ -2985,6 +2988,7 @@ ACTOR Future fetchRangeFeed(StorageServer* data, Key rangeId, KeyRange ran // write mutation to disk mLoc++; } + return Void(); } ACTOR Future dispatchRangeFeeds(StorageServer* data, UID fetchKeysID, KeyRange keys, Version fetchVersion) { @@ -3293,8 +3297,8 @@ ACTOR Future fetchKeys(StorageServer* data, AddingShard* shard) { // AddingShard::addMutations and be applied to versionedMap and mutationLog as normal. The lie about their // version is acceptable because this shard will never be read at versions < transferredVersion - batch->transferredVersion = shard->transferredVersion; for (auto i = shard->updates.begin(); i != shard->updates.end(); ++i) { + i->version = shard->transferredVersion; batch->arena.dependsOn(i->arena()); } @@ -4010,7 +4014,7 @@ ACTOR Future update(StorageServer* data, bool* pReceivedUpdate) { state int mutationNum = 0; state VerUpdateRef* pUpdate = &fii.changes[changeNum]; for (; mutationNum < pUpdate->mutations.size(); mutationNum++) { - updater.applyMutation(data, pUpdate->mutations[mutationNum], fii.transferredVersion, true); + updater.applyMutation(data, pUpdate->mutations[mutationNum], pUpdate->version, true); mutationBytes += pUpdate->mutations[mutationNum].totalSize(); // data->counters.mutationBytes or data->counters.mutations should not be updated because they should // have counted when the mutations arrive from cursor initially. @@ -4262,7 +4266,7 @@ ACTOR Future updateStorage(StorageServer* data) { } std::set modifiedRangeFeeds; - while (data->rangeFeedVersions.front().second < newOldestVersion) { + while (!data->rangeFeedVersions.empty() && data->rangeFeedVersions.front().second < newOldestVersion) { modifiedRangeFeeds.insert(data->rangeFeedVersions.front().first.begin(), data->rangeFeedVersions.front().first.end()); data->rangeFeedVersions.pop_front(); From b03649d627fbb07ff1d4d15d6c247c3324c45627 Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Mon, 9 Aug 2021 12:55:57 -0700 Subject: [PATCH 32/72] fixed assertion error --- fdbserver/storageserver.actor.cpp | 27 ++++++++++++++------------- 1 file changed, 14 insertions(+), 13 deletions(-) diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index a832adc43c..a101b0052e 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -3380,7 +3380,6 @@ AddingShard::AddingShard(StorageServer* server, KeyRangeRef const& keys) } void AddingShard::addMutation(Version version, bool fromFetch, MutationRef const& mutation) { - ASSERT(!fromFetch); if (mutation.type == mutation.ClearRange) { ASSERT(keys.begin <= mutation.param1 && mutation.param2 <= keys.end); } else if (isSingleKeyMutation((MutationRef::Type)mutation.type)) { @@ -3403,24 +3402,26 @@ void AddingShard::addMutation(Version version, bool fromFetch, MutationRef const } // Add the mutation to the version. updates.back().mutations.push_back_deep(updates.back().arena(), mutation); - if (mutation.type == MutationRef::SetValue) { - for (auto& it : server->keyRangeFeed[mutation.param1]) { - if (it->mutations.empty() || it->mutations.back().version != version) { - it->mutations.push_back(MutationsAndVersionRef(version)); - } - it->mutations.back().mutations.push_back_deep(it->mutations.back().arena(), mutation); - server->currentRangeFeeds.insert(it->id); - } - } else if (mutation.type == MutationRef::ClearRange) { - auto ranges = server->keyRangeFeed.intersectingRanges(KeyRangeRef(mutation.param1, mutation.param2)); - for (auto& r : ranges) { - for (auto& it : r.value()) { + if (!fromFetch) { + if (mutation.type == MutationRef::SetValue) { + for (auto& it : server->keyRangeFeed[mutation.param1]) { if (it->mutations.empty() || it->mutations.back().version != version) { it->mutations.push_back(MutationsAndVersionRef(version)); } it->mutations.back().mutations.push_back_deep(it->mutations.back().arena(), mutation); server->currentRangeFeeds.insert(it->id); } + } else if (mutation.type == MutationRef::ClearRange) { + auto ranges = server->keyRangeFeed.intersectingRanges(KeyRangeRef(mutation.param1, mutation.param2)); + for (auto& r : ranges) { + for (auto& it : r.value()) { + if (it->mutations.empty() || it->mutations.back().version != version) { + it->mutations.push_back(MutationsAndVersionRef(version)); + } + it->mutations.back().mutations.push_back_deep(it->mutations.back().arena(), mutation); + server->currentRangeFeeds.insert(it->id); + } + } } } } else if (phase == Waiting) { From f1eedf49108f058a0a6eb70917ff5bd6e37d639f Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Mon, 9 Aug 2021 17:15:51 -0700 Subject: [PATCH 33/72] fixed serialization of rangefeed durable keys --- fdbclient/SystemData.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/fdbclient/SystemData.cpp b/fdbclient/SystemData.cpp index 1a0fbc662b..bd81802ebd 100644 --- a/fdbclient/SystemData.cpp +++ b/fdbclient/SystemData.cpp @@ -1050,19 +1050,19 @@ const KeyRangeRef rangeFeedDurableKeys(LiteralStringRef("\xff\xff/rf/"), Literal const KeyRef rangeFeedDurablePrefix = rangeFeedDurableKeys.begin; const Value rangeFeedDurableKey(Key const& feed, Version const& version) { - BinaryWriter wr(Unversioned()); + BinaryWriter wr(AssumeVersion(ProtocolVersion::withRangeFeed())); wr.serializeBytes(rangeFeedDurablePrefix); wr << feed; - wr << version; + wr << littleEndian64(version); return wr.toValue(); } std::pair decodeRangeFeedDurableKey(ValueRef const& key) { Key feed; Version version; - BinaryReader reader(key.removePrefix(rangeFeedDurablePrefix), Unversioned()); + BinaryReader reader(key.removePrefix(rangeFeedDurablePrefix), AssumeVersion(ProtocolVersion::withRangeFeed())); reader >> feed; reader >> version; - return std::make_pair(feed, version); + return std::make_pair(feed, littleEndian64(version)); } const Value rangeFeedDurableValue(Standalone> const& mutations) { BinaryWriter wr(IncludeVersion(ProtocolVersion::withRangeFeed())); From 52fcf3f5653bbd6eff9ed9ea3eedf929dd8861da Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Mon, 9 Aug 2021 17:16:53 -0700 Subject: [PATCH 34/72] fixed a few bugs --- fdbcli/fdbcli.actor.cpp | 1 + fdbserver/storageserver.actor.cpp | 5 +++-- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/fdbcli/fdbcli.actor.cpp b/fdbcli/fdbcli.actor.cpp index 4c57fbc3f8..608324cd98 100644 --- a/fdbcli/fdbcli.actor.cpp +++ b/fdbcli/fdbcli.actor.cpp @@ -3606,6 +3606,7 @@ ACTOR Future cli(CLIOptions opt, LineNoise* plinenoise) { wait(db->popRangeFeedMutations(tokens[2], v)); } } + continue; } if (tokencmp(tokens[0], "tssq")) { if (tokens.size() == 2) { diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index a101b0052e..3bf85cfd93 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -1549,7 +1549,7 @@ ACTOR Future getRangeFeedMutations(StorageServer* data, RangeFee Version version; std::tie(id, version) = decodeRangeFeedDurableKey(kv.key); auto mutations = decodeRangeFeedDurableValue(kv.value); - reply.mutations.push_back(reply.arena, MutationsAndVersionRef(mutations, version)); + reply.mutations.push_back_deep(reply.arena, MutationsAndVersionRef(mutations, version)); lastVersion = version; } for (auto& it : mutationsDeque) { @@ -4284,6 +4284,7 @@ ACTOR Future updateStorage(StorageServer* data) { info->mutations.pop_front(); } wait(yield(TaskPriority::UpdateStorage)); + curFeed++; } // Set the new durable version as part of the outstanding change set, before commit @@ -5239,7 +5240,7 @@ ACTOR Future serveRangeFeedPopRequests(StorageServer* self, FutureStreamuidRangeFeed[req.rangeID]; - while (feed->mutations.front().version < req.version) { + while (!feed->mutations.empty() && feed->mutations.front().version < req.version) { self->uidRangeFeed[req.rangeID]->mutations.pop_front(); } if (feed->durableVersion != invalidVersion) { From 42ae870c843d72ea65bb4c703d52d40e7dfd87ba Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Mon, 9 Aug 2021 20:39:28 -0700 Subject: [PATCH 35/72] added support for querying specific range feed versions --- fdbcli/fdbcli.actor.cpp | 24 ++++++++- fdbclient/DatabaseContext.h | 7 ++- fdbclient/NativeAPI.actor.cpp | 12 +++-- fdbclient/StorageServerInterface.h | 4 +- fdbserver/storageserver.actor.cpp | 78 +++++++++++++++++++++++------- 5 files changed, 99 insertions(+), 26 deletions(-) diff --git a/fdbcli/fdbcli.actor.cpp b/fdbcli/fdbcli.actor.cpp index 608324cd98..3265126bd4 100644 --- a/fdbcli/fdbcli.actor.cpp +++ b/fdbcli/fdbcli.actor.cpp @@ -3580,12 +3580,32 @@ ACTOR Future cli(CLIOptions opt, LineNoise* plinenoise) { } } } else if (tokencmp(tokens[1], "get")) { - if (tokens.size() != 3) { + if (tokens.size() < 3 || tokens.size() > 5) { printUsage(tokens[0]); is_error = true; continue; } - Standalone> res = wait(db->getRangeFeedMutations(tokens[2])); + Version begin = 0; + Version end = std::numeric_limits::max(); + if (tokens.size() > 3) { + int n = 0; + if (sscanf(tokens[3].toString().c_str(), "%ld%n", &begin, &n) != 1 || + n != tokens[3].size()) { + printUsage(tokens[0]); + is_error = true; + continue; + } + } + if (tokens.size() > 4) { + int n = 0; + if (sscanf(tokens[4].toString().c_str(), "%ld%n", &end, &n) != 1 || n != tokens[4].size()) { + printUsage(tokens[0]); + is_error = true; + continue; + } + } + Standalone> res = + wait(db->getRangeFeedMutations(tokens[2], begin, end)); for (auto& it : res) { for (auto& it2 : it.mutations) { printf("%lld %s\n", it.version, it2.toString().c_str()); diff --git a/fdbclient/DatabaseContext.h b/fdbclient/DatabaseContext.h index e6d98cef6e..8a7fe4021f 100644 --- a/fdbclient/DatabaseContext.h +++ b/fdbclient/DatabaseContext.h @@ -252,8 +252,11 @@ public: // Management API, create snapshot Future createSnapshot(StringRef uid, StringRef snapshot_command); - Future>> getRangeFeedMutations(StringRef rangeID, - KeyRangeRef range = allKeys); + Future>> getRangeFeedMutations( + StringRef rangeID, + Version begin = 0, + Version end = std::numeric_limits::max(), + KeyRange range = allKeys); Future>> getOverlappingRangeFeeds(KeyRangeRef ranges, Version minVersion); Future popRangeFeedMutations(StringRef rangeID, Version version); diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index c198994f48..09b8070cc2 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -6519,7 +6519,9 @@ Future DatabaseContext::createSnapshot(StringRef uid, StringRef snapshot_c ACTOR Future>> getRangeFeedMutationsActor(Reference db, StringRef rangeID, - KeyRangeRef range) { + Version begin, + Version end, + KeyRange range) { state Database cx(db); state Transaction tr(cx); state Key rangeIDKey = rangeID.withPrefix(rangeFeedPrefix); @@ -6543,6 +6545,8 @@ ACTOR Future>> getRangeFeedMutation state RangeFeedRequest req; req.rangeID = rangeID; + req.begin = begin; + req.end = end; RangeFeedReply rep = wait(loadBalance(cx.getPtr(), locations[0].second, @@ -6555,8 +6559,10 @@ ACTOR Future>> getRangeFeedMutation } Future>> DatabaseContext::getRangeFeedMutations(StringRef rangeID, - KeyRangeRef range) { - return getRangeFeedMutationsActor(Reference::addRef(this), rangeID, range); + Version begin, + Version end, + KeyRange range) { + return getRangeFeedMutationsActor(Reference::addRef(this), rangeID, begin, end, range); } ACTOR Future>> getOverlappingRangeFeedsActor(Reference db, diff --git a/fdbclient/StorageServerInterface.h b/fdbclient/StorageServerInterface.h index 9efea0e732..7661398a58 100644 --- a/fdbclient/StorageServerInterface.h +++ b/fdbclient/StorageServerInterface.h @@ -663,6 +663,8 @@ struct RangeFeedReply { struct RangeFeedRequest { constexpr static FileIdentifier file_identifier = 10726174; Key rangeID; + Version begin = 0; + Version end = 0; ReplyPromise reply; RangeFeedRequest() {} @@ -670,7 +672,7 @@ struct RangeFeedRequest { template void serialize(Ar& ar) { - serializer(ar, rangeID, reply); + serializer(ar, rangeID, begin, end, reply); } }; diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index 3bf85cfd93..b6827d00d8 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -313,6 +313,7 @@ struct FetchInjectionInfo { struct RangeFeedInfo : ReferenceCounted { std::deque> mutations; Version durableVersion = invalidVersion; + Version emptyVersion = 0; KeyRange range; Key id; }; @@ -1531,7 +1532,8 @@ ACTOR Future getRangeFeedMutations(StorageServer* data, RangeFee state RangeFeedReply reply; wait(delay(0)); auto& feedInfo = data->uidRangeFeed[req.rangeID]; - if (feedInfo->durableVersion == invalidVersion) { + if (req.end <= feedInfo->emptyVersion + 1) { + } else if (feedInfo->durableVersion == invalidVersion || req.begin > feedInfo->durableVersion) { for (auto& it : data->uidRangeFeed[req.rangeID]->mutations) { reply.mutations.push_back(reply.arena, it); } @@ -1539,10 +1541,7 @@ ACTOR Future getRangeFeedMutations(StorageServer* data, RangeFee state std::deque> mutationsDeque = data->uidRangeFeed[req.rangeID]->mutations; RangeResult res = wait(data->storage.readRange( - KeyRangeRef(rangeFeedDurableKey(req.rangeID, 0), rangeFeedDurableKey(req.rangeID, data->version.get())))); - if (res.empty()) { - data->uidRangeFeed[req.rangeID]->durableVersion = invalidVersion; - } + KeyRangeRef(rangeFeedDurableKey(req.rangeID, req.begin), rangeFeedDurableKey(req.rangeID, req.end)))); Version lastVersion = invalidVersion; for (auto& kv : res) { Key id; @@ -1553,10 +1552,35 @@ ACTOR Future getRangeFeedMutations(StorageServer* data, RangeFee lastVersion = version; } for (auto& it : mutationsDeque) { + if (it.version >= req.end) { + break; + } if (it.version > lastVersion) { reply.mutations.push_back(reply.arena, it); } } + if (res.empty()) { + auto& feedInfo = data->uidRangeFeed[req.rangeID]; + if (req.end > feedInfo->durableVersion) { + if (req.begin == 0) { + feedInfo->durableVersion = req.end > data->storageVersion() ? invalidVersion : req.end; + } else { + RangeResult emp = wait(data->storage.readRange( + KeyRangeRef(rangeFeedDurableKey(req.rangeID, 0), rangeFeedDurableKey(req.rangeID, req.end)), + -1)); + + auto& feedInfo = data->uidRangeFeed[req.rangeID]; + if (emp.empty()) { + feedInfo->durableVersion = req.end > data->storageVersion() ? invalidVersion : req.end; + } else { + Key id; + Version version; + std::tie(id, version) = decodeRangeFeedDurableKey(emp[0].key); + feedInfo->durableVersion = version; + } + } + } + } } return reply; } @@ -2945,7 +2969,7 @@ static const KeyRangeRef persistRangeFeedKeys = KeyRangeRef(LiteralStringRef(PERSIST_PREFIX "RF/"), LiteralStringRef(PERSIST_PREFIX "RF0")); // data keys are unmangled (but never start with PERSIST_PREFIX because they are always in allKeys) -ACTOR Future fetchRangeFeed(StorageServer* data, Key rangeId, KeyRange range) { +ACTOR Future fetchRangeFeed(StorageServer* data, Key rangeId, KeyRange range, Version fetchVersion) { TraceEvent("FetchRangeFeed", data->thisServerID) .detail("RangeID", rangeId.printable()) @@ -2966,9 +2990,11 @@ ACTOR Future fetchRangeFeed(StorageServer* data, Key rangeId, KeyRange ran rangeFeedValue(range))); state Standalone> mutations = - wait(data->cx->getRangeFeedMutations(rangeId, range)); + wait(data->cx->getRangeFeedMutations(rangeId, 0, fetchVersion, range)); state RangeFeedRequest req; req.rangeID = rangeId; + req.begin = 0; + req.end = fetchVersion; state RangeFeedReply rep = wait(getRangeFeedMutations(data, req)); state int mLoc = 0; state int rLoc = 0; @@ -3000,7 +3026,7 @@ ACTOR Future dispatchRangeFeeds(StorageServer* data, UID fetchKeysID, KeyR state std::vector> feeds = wait(data->cx->getOverlappingRangeFeeds(keys, fetchVersion)); for (auto& feed : feeds) { - feedFetches[feed.first] = fetchRangeFeed(data, feed.first, feed.second); + feedFetches[feed.first] = fetchRangeFeed(data, feed.first, feed.second, fetchVersion); } loop { @@ -3752,6 +3778,7 @@ private: Reference rangeFeedInfo(new RangeFeedInfo()); rangeFeedInfo->range = rangeFeedRange; rangeFeedInfo->id = rangeFeedId; + rangeFeedInfo->emptyVersion = currentVersion - 1; data->uidRangeFeed[rangeFeedId] = rangeFeedInfo; auto rs = data->keyRangeFeed.modify(rangeFeedRange); for (auto r = rs.begin(); r != rs.end(); ++r) { @@ -4277,11 +4304,13 @@ ACTOR Future updateStorage(StorageServer* data) { state int curFeed = 0; while (curFeed < updatedRangeFeeds.size()) { auto info = data->uidRangeFeed[updatedRangeFeeds[curFeed]]; - while (info->mutations.front().version < newOldestVersion) { + for (auto& it : info->mutations) { + if (it.version >= newOldestVersion) { + break; + } data->storage.writeKeyValue(KeyValueRef(rangeFeedDurableKey(info->id, info->mutations.front().version), rangeFeedDurableValue(info->mutations.front().mutations))); info->durableVersion = info->mutations.front().version; - info->mutations.pop_front(); } wait(yield(TaskPriority::UpdateStorage)); curFeed++; @@ -4320,6 +4349,16 @@ ACTOR Future updateStorage(StorageServer* data) { throw please_reboot(); } + curFeed = 0; + while (curFeed < updatedRangeFeeds.size()) { + auto info = data->uidRangeFeed[updatedRangeFeeds[curFeed]]; + while (info->mutations.front().version < newOldestVersion) { + info->mutations.pop_front(); + } + wait(yield(TaskPriority::UpdateStorage)); + curFeed++; + } + durableInProgress.send(Void()); wait(delay(0, TaskPriority::UpdateStorage)); // Setting durableInProgess could cause the storage server to shut // down, so delay to check for cancellation @@ -5240,14 +5279,17 @@ ACTOR Future serveRangeFeedPopRequests(StorageServer* self, FutureStreamuidRangeFeed[req.rangeID]; - while (!feed->mutations.empty() && feed->mutations.front().version < req.version) { - self->uidRangeFeed[req.rangeID]->mutations.pop_front(); - } - if (feed->durableVersion != invalidVersion) { - self->storage.clearRange( - KeyRangeRef(rangeFeedDurableKey(feed->id, 0), rangeFeedDurableKey(feed->id, req.version))); - if (req.version > feed->durableVersion) { - feed->durableVersion = invalidVersion; + if (req.version - 1 > feed->emptyVersion) { + feed->emptyVersion = req.version - 1; + while (!feed->mutations.empty() && feed->mutations.front().version < req.version) { + self->uidRangeFeed[req.rangeID]->mutations.pop_front(); + } + if (feed->durableVersion != invalidVersion) { + self->storage.clearRange( + KeyRangeRef(rangeFeedDurableKey(feed->id, 0), rangeFeedDurableKey(feed->id, req.version))); + if (req.version > feed->durableVersion) { + feed->durableVersion = invalidVersion; + } } } TraceEvent("RangeFeedPopQuery", self->thisServerID) From 208a5790ad486df04f897ac13deb75f516d5df63 Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Mon, 9 Aug 2021 21:58:44 -0700 Subject: [PATCH 36/72] fixed usage of durable version --- fdbserver/TLogServer.actor.cpp | 14 +++++++------- fdbserver/storageserver.actor.cpp | 31 +++++++++++++++++++------------ 2 files changed, 26 insertions(+), 19 deletions(-) diff --git a/fdbserver/TLogServer.actor.cpp b/fdbserver/TLogServer.actor.cpp index 3cf9d55546..a0f8bc58b0 100644 --- a/fdbserver/TLogServer.actor.cpp +++ b/fdbserver/TLogServer.actor.cpp @@ -1860,13 +1860,13 @@ ACTOR Future tLogPeekMessages(TLogData* self, TLogPeekRequest req, Referen reply.end = endVersion; reply.onlySpilled = onlySpilled; - TraceEvent("TlogPeek", self->dbgid) - .detail("LogId", logData->logId) - .detail("Tag", req.tag.toString()) - .detail("BeginVer", req.begin) - .detail("EndVer", reply.end) - .detail("MsgBytes", reply.messages.expectedSize()) - .detail("ForAddress", req.reply.getEndpoint().getPrimaryAddress()); + // TraceEvent("TlogPeek", self->dbgid) + // .detail("LogId", logData->logId) + // .detail("Tag", req.tag.toString()) + // .detail("BeginVer", req.begin) + // .detail("EndVer", reply.end) + // .detail("MsgBytes", reply.messages.expectedSize()) + // .detail("ForAddress", req.reply.getEndpoint().getPrimaryAddress()); if (req.sequence.present()) { auto& trackerData = logData->peekTracker[peekId]; diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index c1500b6e3e..c58816de59 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -312,6 +312,7 @@ struct FetchInjectionInfo { struct RangeFeedInfo : ReferenceCounted { std::deque> mutations; + Version storageVersion = invalidVersion; Version durableVersion = invalidVersion; Version emptyVersion = 0; KeyRange range; @@ -1540,6 +1541,7 @@ ACTOR Future getRangeFeedMutations(StorageServer* data, RangeFee } else { state std::deque> mutationsDeque = data->uidRangeFeed[req.rangeID]->mutations; + state Version startingDurableVersion = feedInfo->durableVersion; RangeResult res = wait(data->storage.readRange( KeyRangeRef(rangeFeedDurableKey(req.rangeID, req.begin), rangeFeedDurableKey(req.rangeID, req.end)))); Version lastVersion = invalidVersion; @@ -1559,9 +1561,9 @@ ACTOR Future getRangeFeedMutations(StorageServer* data, RangeFee reply.mutations.push_back(reply.arena, it); } } - if (res.empty()) { + if (reply.mutations.empty()) { auto& feedInfo = data->uidRangeFeed[req.rangeID]; - if (req.end > feedInfo->durableVersion) { + if (startingDurableVersion == feedInfo->storageVersion && req.end > startingDurableVersion) { if (req.begin == 0) { feedInfo->durableVersion = req.end > data->storageVersion() ? invalidVersion : req.end; } else { @@ -1570,13 +1572,15 @@ ACTOR Future getRangeFeedMutations(StorageServer* data, RangeFee -1)); auto& feedInfo = data->uidRangeFeed[req.rangeID]; - if (emp.empty()) { - feedInfo->durableVersion = req.end > data->storageVersion() ? invalidVersion : req.end; - } else { - Key id; - Version version; - std::tie(id, version) = decodeRangeFeedDurableKey(emp[0].key); - feedInfo->durableVersion = version; + if (startingDurableVersion == feedInfo->storageVersion) { + if (emp.empty()) { + feedInfo->durableVersion = req.end > data->storageVersion() ? invalidVersion : req.end; + } else { + Key id; + Version version; + std::tie(id, version) = decodeRangeFeedDurableKey(emp[0].key); + feedInfo->durableVersion = version; + } } } } @@ -4310,7 +4314,7 @@ ACTOR Future updateStorage(StorageServer* data) { } data->storage.writeKeyValue(KeyValueRef(rangeFeedDurableKey(info->id, info->mutations.front().version), rangeFeedDurableValue(info->mutations.front().mutations))); - info->durableVersion = info->mutations.front().version; + info->storageVersion = info->mutations.front().version; } wait(yield(TaskPriority::UpdateStorage)); curFeed++; @@ -4355,6 +4359,7 @@ ACTOR Future updateStorage(StorageServer* data) { while (info->mutations.front().version < newOldestVersion) { info->mutations.pop_front(); } + info->durableVersion = info->mutations.front().version; wait(yield(TaskPriority::UpdateStorage)); curFeed++; } @@ -4744,6 +4749,7 @@ ACTOR Future restoreDurableState(StorageServer* data, IKeyValueStore* stor rangeFeedInfo->range = rangeFeedRange; rangeFeedInfo->id = rangeFeedId; rangeFeedInfo->durableVersion = version; + rangeFeedInfo->storageVersion = version; data->uidRangeFeed[rangeFeedId] = rangeFeedInfo; auto rs = data->keyRangeFeed.modify(rangeFeedRange); for (auto r = rs.begin(); r != rs.end(); ++r) { @@ -5284,10 +5290,11 @@ ACTOR Future serveRangeFeedPopRequests(StorageServer* self, FutureStreammutations.empty() && feed->mutations.front().version < req.version) { self->uidRangeFeed[req.rangeID]->mutations.pop_front(); } - if (feed->durableVersion != invalidVersion) { + if (feed->storageVersion != invalidVersion) { self->storage.clearRange( KeyRangeRef(rangeFeedDurableKey(feed->id, 0), rangeFeedDurableKey(feed->id, req.version))); - if (req.version > feed->durableVersion) { + if (req.version > feed->storageVersion) { + feed->storageVersion = invalidVersion; feed->durableVersion = invalidVersion; } } From a1b0053b572faf7f477003b76aa684e89140a63f Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Mon, 9 Aug 2021 22:06:53 -0700 Subject: [PATCH 37/72] do not return send a reply to a pop request until the clear has been made durable --- fdbserver/storageserver.actor.cpp | 48 ++++++++++++++++++------------- 1 file changed, 28 insertions(+), 20 deletions(-) diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index c58816de59..c357ca9162 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -1511,6 +1511,30 @@ ACTOR Future watchValueSendReply(StorageServer* data, } } +ACTOR Future rangeFeedPopQ(StorageServer* self, RangeFeedPopRequest req) { + auto& feed = self->uidRangeFeed[req.rangeID]; + if (req.version - 1 > feed->emptyVersion) { + feed->emptyVersion = req.version - 1; + while (!feed->mutations.empty() && feed->mutations.front().version < req.version) { + self->uidRangeFeed[req.rangeID]->mutations.pop_front(); + } + if (feed->storageVersion != invalidVersion) { + self->storage.clearRange( + KeyRangeRef(rangeFeedDurableKey(feed->id, 0), rangeFeedDurableKey(feed->id, req.version))); + if (req.version > feed->storageVersion) { + feed->storageVersion = invalidVersion; + feed->durableVersion = invalidVersion; + } + wait(self->durableVersion.whenAtLeast(self->storageVersion() + 1)); + } + } + TraceEvent("RangeFeedPopQuery", self->thisServerID) + .detail("RangeID", req.rangeID.printable()) + .detail("Version", req.version); + req.reply.send(Void()); + return Void(); +} + ACTOR Future overlappingRangeFeedsQ(StorageServer* data, OverlappingRangeFeedsRequest req) { wait(delay(0)); auto ranges = data->keyRangeFeed.intersectingRanges(req.range); @@ -4359,7 +4383,9 @@ ACTOR Future updateStorage(StorageServer* data) { while (info->mutations.front().version < newOldestVersion) { info->mutations.pop_front(); } - info->durableVersion = info->mutations.front().version; + if (info->storageVersion != invalidVersion) { + info->durableVersion = info->mutations.front().version; + } wait(yield(TaskPriority::UpdateStorage)); curFeed++; } @@ -5284,25 +5310,7 @@ ACTOR Future serveOverlappingRangeFeedsRequests( ACTOR Future serveRangeFeedPopRequests(StorageServer* self, FutureStream rangeFeedPops) { loop { RangeFeedPopRequest req = waitNext(rangeFeedPops); - auto& feed = self->uidRangeFeed[req.rangeID]; - if (req.version - 1 > feed->emptyVersion) { - feed->emptyVersion = req.version - 1; - while (!feed->mutations.empty() && feed->mutations.front().version < req.version) { - self->uidRangeFeed[req.rangeID]->mutations.pop_front(); - } - if (feed->storageVersion != invalidVersion) { - self->storage.clearRange( - KeyRangeRef(rangeFeedDurableKey(feed->id, 0), rangeFeedDurableKey(feed->id, req.version))); - if (req.version > feed->storageVersion) { - feed->storageVersion = invalidVersion; - feed->durableVersion = invalidVersion; - } - } - } - TraceEvent("RangeFeedPopQuery", self->thisServerID) - .detail("RangeID", req.rangeID.printable()) - .detail("Version", req.version); - req.reply.send(Void()); + self->actors.add(self->readGuard(req, rangeFeedPopQ)); } } From df6bbb6a087d6c779e7913d5043b2db78e442a97 Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Tue, 10 Aug 2021 14:37:05 -0700 Subject: [PATCH 38/72] fix: needed bigEndian instead of littleEndian --- fdbclient/SystemData.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/fdbclient/SystemData.cpp b/fdbclient/SystemData.cpp index bd81802ebd..faba5828ce 100644 --- a/fdbclient/SystemData.cpp +++ b/fdbclient/SystemData.cpp @@ -1053,7 +1053,7 @@ const Value rangeFeedDurableKey(Key const& feed, Version const& version) { BinaryWriter wr(AssumeVersion(ProtocolVersion::withRangeFeed())); wr.serializeBytes(rangeFeedDurablePrefix); wr << feed; - wr << littleEndian64(version); + wr << bigEndian64(version); return wr.toValue(); } std::pair decodeRangeFeedDurableKey(ValueRef const& key) { @@ -1062,7 +1062,7 @@ std::pair decodeRangeFeedDurableKey(ValueRef const& key) { BinaryReader reader(key.removePrefix(rangeFeedDurablePrefix), AssumeVersion(ProtocolVersion::withRangeFeed())); reader >> feed; reader >> version; - return std::make_pair(feed, littleEndian64(version)); + return std::make_pair(feed, bigEndian64(version)); } const Value rangeFeedDurableValue(Standalone> const& mutations) { BinaryWriter wr(IncludeVersion(ProtocolVersion::withRangeFeed())); From db274c243125af905d498d413ebdb9787c6a1d19 Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Tue, 10 Aug 2021 16:47:01 -0700 Subject: [PATCH 39/72] a number of bug fixes --- fdbserver/storageserver.actor.cpp | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index c357ca9162..650bbc1727 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -1560,7 +1560,9 @@ ACTOR Future getRangeFeedMutations(StorageServer* data, RangeFee if (req.end <= feedInfo->emptyVersion + 1) { } else if (feedInfo->durableVersion == invalidVersion || req.begin > feedInfo->durableVersion) { for (auto& it : data->uidRangeFeed[req.rangeID]->mutations) { - reply.mutations.push_back(reply.arena, it); + if (it.version >= req.begin) { + reply.mutations.push_back(reply.arena, it); + } } } else { state std::deque> mutationsDeque = @@ -1568,7 +1570,7 @@ ACTOR Future getRangeFeedMutations(StorageServer* data, RangeFee state Version startingDurableVersion = feedInfo->durableVersion; RangeResult res = wait(data->storage.readRange( KeyRangeRef(rangeFeedDurableKey(req.rangeID, req.begin), rangeFeedDurableKey(req.rangeID, req.end)))); - Version lastVersion = invalidVersion; + Version lastVersion = req.begin - 1; for (auto& kv : res) { Key id; Version version; @@ -4336,9 +4338,9 @@ ACTOR Future updateStorage(StorageServer* data) { if (it.version >= newOldestVersion) { break; } - data->storage.writeKeyValue(KeyValueRef(rangeFeedDurableKey(info->id, info->mutations.front().version), - rangeFeedDurableValue(info->mutations.front().mutations))); - info->storageVersion = info->mutations.front().version; + data->storage.writeKeyValue( + KeyValueRef(rangeFeedDurableKey(info->id, it.version), rangeFeedDurableValue(it.mutations))); + info->storageVersion = it.version; } wait(yield(TaskPriority::UpdateStorage)); curFeed++; @@ -4383,9 +4385,7 @@ ACTOR Future updateStorage(StorageServer* data) { while (info->mutations.front().version < newOldestVersion) { info->mutations.pop_front(); } - if (info->storageVersion != invalidVersion) { - info->durableVersion = info->mutations.front().version; - } + info->durableVersion = info->storageVersion; wait(yield(TaskPriority::UpdateStorage)); curFeed++; } From 7891477a7c221e4f275bb7ee159f6daa57ae3a98 Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Tue, 10 Aug 2021 21:28:25 -0700 Subject: [PATCH 40/72] added support for streaming range feeds --- fdbcli/fdbcli.actor.cpp | 49 +++++++++++- fdbclient/DatabaseContext.h | 7 ++ fdbclient/NativeAPI.actor.cpp | 121 +++++++++++++++++++++++++++++ fdbclient/StorageServerInterface.h | 42 +++++++++- fdbserver/storageserver.actor.cpp | 94 +++++++++++++++++++--- 5 files changed, 299 insertions(+), 14 deletions(-) diff --git a/fdbcli/fdbcli.actor.cpp b/fdbcli/fdbcli.actor.cpp index f4364b6855..faedc44764 100644 --- a/fdbcli/fdbcli.actor.cpp +++ b/fdbcli/fdbcli.actor.cpp @@ -657,7 +657,7 @@ void initHelp() { CommandHelp("triggerddteaminfolog", "trigger the data distributor teams logging", "Trigger the data distributor to log detailed information about its teams."); - helpMap["rangefeed"] = CommandHelp("rangefeed ", "", ""); + helpMap["rangefeed"] = CommandHelp("rangefeed ", "", ""); helpMap["tssq"] = CommandHelp("tssq start|stop ", "start/stop tss quarantine", @@ -3553,11 +3553,58 @@ ACTOR Future cli(CLIOptions opt, LineNoise* plinenoise) { } Standalone> res = wait(db->getRangeFeedMutations(tokens[2], begin, end)); + printf("\n"); for (auto& it : res) { for (auto& it2 : it.mutations) { printf("%lld %s\n", it.version, it2.toString().c_str()); } } + } else if (tokencmp(tokens[1], "stream")) { + if (tokens.size() < 3 || tokens.size() > 5) { + printUsage(tokens[0]); + is_error = true; + continue; + } + Version begin = 0; + Version end = std::numeric_limits::max(); + if (tokens.size() > 3) { + int n = 0; + if (sscanf(tokens[3].toString().c_str(), "%ld%n", &begin, &n) != 1 || + n != tokens[3].size()) { + printUsage(tokens[0]); + is_error = true; + continue; + } + } + if (tokens.size() > 4) { + int n = 0; + if (sscanf(tokens[4].toString().c_str(), "%ld%n", &end, &n) != 1 || n != tokens[4].size()) { + printUsage(tokens[0]); + is_error = true; + continue; + } + } + if (warn.isValid()) { + warn.cancel(); + } + state PromiseStream>> feedResults; + state Future feed = db->getRangeFeedStream(feedResults, tokens[2], begin, end); + printf("\n"); + try { + loop { + Standalone> res = waitNext(feedResults.getFuture()); + for (auto& it : res) { + for (auto& it2 : it.mutations) { + printf("%lld %s\n", it.version, it2.toString().c_str()); + } + } + } + } catch (Error& e) { + if (e.code() == error_code_end_of_stream) { + continue; + } + throw; + } } else if (tokencmp(tokens[1], "pop")) { if (tokens.size() != 4) { printUsage(tokens[0]); diff --git a/fdbclient/DatabaseContext.h b/fdbclient/DatabaseContext.h index cd9c233c63..6c597d58fc 100644 --- a/fdbclient/DatabaseContext.h +++ b/fdbclient/DatabaseContext.h @@ -257,6 +257,13 @@ public: Version begin = 0, Version end = std::numeric_limits::max(), KeyRange range = allKeys); + + Future getRangeFeedStream(const PromiseStream>>& results, + StringRef rangeID, + Version begin = 0, + Version end = std::numeric_limits::max(), + KeyRange range = allKeys); + Future>> getOverlappingRangeFeeds(KeyRangeRef ranges, Version minVersion); Future popRangeFeedMutations(StringRef rangeID, Version version); diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index 7df277bb0b..06c6f2d38f 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -6575,6 +6575,126 @@ Future>> DatabaseContext::getRangeF return getRangeFeedMutationsActor(Reference::addRef(this), rangeID, begin, end, range); } +ACTOR Future getRangeFeedStreamActor(Reference db, + PromiseStream>> results, + StringRef rangeID, + Version begin, + Version end, + KeyRange range) { + state Database cx(db); + state Transaction tr(cx); + state Key rangeIDKey = rangeID.withPrefix(rangeFeedPrefix); + state Span span("NAPI:GetRangeFeedStream"_loc); + state KeyRange keys; + loop { + try { + Optional val = wait(tr.get(rangeIDKey)); + if (!val.present()) { + results.sendError(unsupported_operation()); + return Void(); + } + keys = decodeRangeFeedValue(val.get()); + break; + } catch (Error& e) { + wait(tr.onError(e)); + } + } + + loop { + try { + state vector>> locations = + wait(getKeyRangeLocations(cx, + keys, + 100, + Reverse::False, + &StorageServerInterface::rangeFeed, + TransactionInfo(TaskPriority::DefaultEndpoint, span.context))); + + if (locations.size() > 1) { + results.sendError(unsupported_operation()); + return Void(); + } + + state int useIdx = -1; + + loop { + // FIXME: create a load balance function for this code so future users of reply streams do not have + // to duplicate this code + int count = 0; + for (int i = 0; i < locations[0].second->size(); i++) { + if (!IFailureMonitor::failureMonitor() + .getState( + locations[0].second->get(i, &StorageServerInterface::rangeFeedStream).getEndpoint()) + .failed) { + if (deterministicRandom()->random01() <= 1.0 / ++count) { + useIdx = i; + } + } + } + + if (useIdx >= 0) { + break; + } + + vector> ok(locations[0].second->size()); + for (int i = 0; i < ok.size(); i++) { + ok[i] = IFailureMonitor::failureMonitor().onStateEqual( + locations[0].second->get(i, &StorageServerInterface::rangeFeedStream).getEndpoint(), + FailureStatus(false)); + } + + // Making this SevWarn means a lot of clutter + if (now() - g_network->networkInfo.newestAlternativesFailure > 1 || + deterministicRandom()->random01() < 0.01) { + TraceEvent("AllAlternativesFailed").detail("Alternatives", locations[0].second->description()); + } + + wait(allAlternativesFailedDelay(quorum(ok, 1))); + } + + state RangeFeedStreamRequest req; + req.rangeID = rangeID; + req.begin = begin; + req.end = end; + + state ReplyPromiseStream replyStream = + locations[0].second->get(useIdx, &StorageServerInterface::rangeFeedStream).getReplyStream(req); + + loop { + wait(results.onEmpty()); + choose { + when(wait(cx->connectionFileChanged())) { break; } + when(RangeFeedStreamReply rep = waitNext(replyStream.getFuture())) { + begin = rep.mutations.back().version + 1; + results.send(Standalone>(rep.mutations, rep.arena)); + } + } + } + } catch (Error& e) { + if (e.code() == error_code_actor_cancelled) { + throw; + } + if (e.code() == error_code_wrong_shard_server || e.code() == error_code_all_alternatives_failed || + e.code() == error_code_connection_failed) { + cx->invalidateCache(keys); + wait(delay(CLIENT_KNOBS->WRONG_SHARD_SERVER_DELAY)); + } else { + results.sendError(e); + return Void(); + } + } + } +} + +Future DatabaseContext::getRangeFeedStream( + const PromiseStream>>& results, + StringRef rangeID, + Version begin, + Version end, + KeyRange range) { + return getRangeFeedStreamActor(Reference::addRef(this), results, rangeID, begin, end, range); +} + ACTOR Future>> getOverlappingRangeFeedsActor(Reference db, KeyRangeRef range, Version minVersion) { @@ -6633,6 +6753,7 @@ ACTOR Future popRangeFeedMutationsActor(Reference db, Str throw unsupported_operation(); } + // FIXME: lookup both the src and dest shards as of the pop version to ensure all locations are popped state std::vector> popRequests; for (int i = 0; i < locations[0].second->size(); i++) { popRequests.push_back( diff --git a/fdbclient/StorageServerInterface.h b/fdbclient/StorageServerInterface.h index 7661398a58..14b4991938 100644 --- a/fdbclient/StorageServerInterface.h +++ b/fdbclient/StorageServerInterface.h @@ -77,10 +77,11 @@ struct StorageServerInterface { RequestStream watchValue; RequestStream getReadHotRanges; RequestStream getRangeSplitPoints; + RequestStream getKeyValuesStream; RequestStream rangeFeed; + RequestStream rangeFeedStream; RequestStream overlappingRangeFeeds; RequestStream rangeFeedPop; - RequestStream getKeyValuesStream; explicit StorageServerInterface(UID uid) : uniqueID(uid) {} StorageServerInterface() : uniqueID(deterministicRandom()->randomUniqueID()) {} @@ -124,10 +125,12 @@ struct StorageServerInterface { getKeyValuesStream = RequestStream(getValue.getEndpoint().getAdjustedEndpoint(13)); rangeFeed = RequestStream(getValue.getEndpoint().getAdjustedEndpoint(14)); + rangeFeedStream = + RequestStream(getValue.getEndpoint().getAdjustedEndpoint(15)); overlappingRangeFeeds = - RequestStream(getValue.getEndpoint().getAdjustedEndpoint(15)); + RequestStream(getValue.getEndpoint().getAdjustedEndpoint(16)); rangeFeedPop = - RequestStream(getValue.getEndpoint().getAdjustedEndpoint(16)); + RequestStream(getValue.getEndpoint().getAdjustedEndpoint(17)); } } else { ASSERT(Ar::isDeserializing); @@ -171,6 +174,7 @@ struct StorageServerInterface { streams.push_back(getRangeSplitPoints.getReceiver()); streams.push_back(getKeyValuesStream.getReceiver(TaskPriority::LoadBalancedEndpoint)); streams.push_back(rangeFeed.getReceiver()); + streams.push_back(rangeFeedStream.getReceiver()); streams.push_back(overlappingRangeFeeds.getReceiver()); streams.push_back(rangeFeedPop.getReceiver()); FlowTransport::transport().addEndpoints(streams); @@ -676,6 +680,38 @@ struct RangeFeedRequest { } }; +struct RangeFeedStreamReply : public ReplyPromiseStreamReply { + constexpr static FileIdentifier file_identifier = 1783066; + Arena arena; + VectorRef mutations; + + RangeFeedStreamReply() {} + RangeFeedStreamReply(RangeFeedReply r) : arena(r.arena), mutations(r.mutations) {} + + int expectedSize() const { return sizeof(RangeFeedStreamReply) + mutations.expectedSize(); } + + template + void serialize(Ar& ar) { + serializer(ar, ReplyPromiseStreamReply::acknowledgeToken, mutations, arena); + } +}; + +struct RangeFeedStreamRequest { + constexpr static FileIdentifier file_identifier = 6795746; + SpanID spanContext; + Arena arena; + Key rangeID; + Version begin = 0; + Version end = 0; + ReplyPromiseStream reply; + + RangeFeedStreamRequest() {} + template + void serialize(Ar& ar) { + serializer(ar, rangeID, begin, end, reply, spanContext, arena); + } +}; + struct RangeFeedPopRequest { constexpr static FileIdentifier file_identifier = 10726174; Key rangeID; diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index 650bbc1727..9ea3c2be5a 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -317,6 +317,7 @@ struct RangeFeedInfo : ReferenceCounted { Version emptyVersion = 0; KeyRange range; Key id; + AsyncTrigger newMutations; }; class ServerWatchMetadata : public ReferenceCounted { @@ -1555,13 +1556,21 @@ ACTOR Future overlappingRangeFeedsQ(StorageServer* data, OverlappingRangeF ACTOR Future getRangeFeedMutations(StorageServer* data, RangeFeedRequest req) { state RangeFeedReply reply; - wait(delay(0)); + state int remainingLimitBytes = CLIENT_KNOBS->REPLY_BYTE_LIMIT; + wait(delay(0, TaskPriority::DefaultEndpoint)); + if (data->version.get() < req.begin) { + wait(data->version.whenAtLeast(req.begin)); + } auto& feedInfo = data->uidRangeFeed[req.rangeID]; if (req.end <= feedInfo->emptyVersion + 1) { } else if (feedInfo->durableVersion == invalidVersion || req.begin > feedInfo->durableVersion) { for (auto& it : data->uidRangeFeed[req.rangeID]->mutations) { + if (it.version >= req.end || remainingLimitBytes <= 0) { + break; + } if (it.version >= req.begin) { reply.mutations.push_back(reply.arena, it); + remainingLimitBytes -= sizeof(MutationsAndVersionRef) + it.expectedSize(); } } } else { @@ -1569,7 +1578,10 @@ ACTOR Future getRangeFeedMutations(StorageServer* data, RangeFee data->uidRangeFeed[req.rangeID]->mutations; state Version startingDurableVersion = feedInfo->durableVersion; RangeResult res = wait(data->storage.readRange( - KeyRangeRef(rangeFeedDurableKey(req.rangeID, req.begin), rangeFeedDurableKey(req.rangeID, req.end)))); + KeyRangeRef(rangeFeedDurableKey(req.rangeID, req.begin), rangeFeedDurableKey(req.rangeID, req.end)), + 1 << 30, + remainingLimitBytes)); + Version lastVersion = req.begin - 1; for (auto& kv : res) { Key id; @@ -1577,14 +1589,19 @@ ACTOR Future getRangeFeedMutations(StorageServer* data, RangeFee std::tie(id, version) = decodeRangeFeedDurableKey(kv.key); auto mutations = decodeRangeFeedDurableValue(kv.value); reply.mutations.push_back_deep(reply.arena, MutationsAndVersionRef(mutations, version)); + remainingLimitBytes -= + sizeof(KeyValueRef) + + kv.expectedSize(); // FIXME: this is currently tracking the size on disk rather than the reply size + // because we cannot add mutaitons from memory if there are potentially more on disk lastVersion = version; } for (auto& it : mutationsDeque) { - if (it.version >= req.end) { + if (it.version >= req.end || remainingLimitBytes <= 0) { break; } if (it.version > lastVersion) { reply.mutations.push_back(reply.arena, it); + remainingLimitBytes -= sizeof(MutationsAndVersionRef) + it.expectedSize(); } } if (reply.mutations.empty()) { @@ -1612,6 +1629,10 @@ ACTOR Future getRangeFeedMutations(StorageServer* data, RangeFee } } } + Version finalVersion = std::min(req.end - 1, data->version.get()); + if ((reply.mutations.empty() || reply.mutations.back().version) < finalVersion && remainingLimitBytes > 0) { + reply.mutations.push_back(reply.arena, MutationsAndVersionRef(finalVersion)); + } return reply; } @@ -1621,6 +1642,47 @@ ACTOR Future rangeFeedQ(StorageServer* data, RangeFeedRequest req) { return Void(); } +ACTOR Future rangeFeedStreamQ(StorageServer* data, RangeFeedStreamRequest req) +// Throws a wrong_shard_server if the keys in the request or result depend on data outside this server OR if a large +// selector offset prevents all data from being read in one range read +{ + state Span span("SS:getRangeFeedStream"_loc, { req.spanContext }); + state Version begin = req.begin; + req.reply.setByteLimit(SERVER_KNOBS->RANGESTREAM_LIMIT_BYTES); + + wait(delay(0, TaskPriority::DefaultEndpoint)); + + try { + loop { + wait(req.reply.onReady()); + state RangeFeedRequest feedRequest; + feedRequest.rangeID = req.rangeID; + feedRequest.begin = begin; + feedRequest.end = req.end; + RangeFeedReply feedReply = wait(getRangeFeedMutations(data, feedRequest)); + begin = feedReply.mutations.back().version + 1; + req.reply.send(RangeFeedStreamReply(feedReply)); + if (feedReply.mutations.back().version == req.end - 1) { + req.reply.sendError(end_of_stream()); + return Void(); + } + if (feedReply.mutations.back().mutations.empty()) { + choose { + when(wait(delay(5.0, TaskPriority::DefaultEndpoint))) {} + when(wait(data->uidRangeFeed[req.rangeID]->newMutations.onTrigger())) {} + } + } + } + } catch (Error& e) { + if (e.code() != error_code_operation_obsolete) { + if (!canReplyWith(e)) + throw; + req.reply.sendError(e); + } + } + return Void(); +} + #ifdef NO_INTELLISENSE size_t WATCH_OVERHEAD_WATCHQ = sizeof(WatchValueSendReplyActorState) + sizeof(WatchValueSendReplyActor); @@ -4219,12 +4281,6 @@ ACTOR Future update(StorageServer* data, bool* pReceivedUpdate) { .trackLatest(data->thisServerID.toString() + "/StorageServerSourceTLogID"); } - if (data->currentRangeFeeds.size()) { - data->rangeFeedVersions.push_back(std::make_pair( - std::vector(data->currentRangeFeeds.begin(), data->currentRangeFeeds.end()), ver)); - data->currentRangeFeeds.clear(); - } - data->noRecentUpdates.set(false); data->lastUpdate = now(); data->version.set(ver); // Triggers replies to waiting gets for new version(s) @@ -4260,6 +4316,15 @@ ACTOR Future update(StorageServer* data, bool* pReceivedUpdate) { data->recoveryVersionSkips.pop_front(); } data->desiredOldestVersion.set(proposedOldestVersion); + + if (data->currentRangeFeeds.size()) { + data->rangeFeedVersions.push_back(std::make_pair( + std::vector(data->currentRangeFeeds.begin(), data->currentRangeFeeds.end()), ver)); + for (auto& it : data->currentRangeFeeds) { + data->uidRangeFeed[it]->newMutations.trigger(); + } + data->currentRangeFeeds.clear(); + } } validate(data); @@ -4382,7 +4447,7 @@ ACTOR Future updateStorage(StorageServer* data) { curFeed = 0; while (curFeed < updatedRangeFeeds.size()) { auto info = data->uidRangeFeed[updatedRangeFeeds[curFeed]]; - while (info->mutations.front().version < newOldestVersion) { + while (!info->mutations.empty() && info->mutations.front().version < newOldestVersion) { info->mutations.pop_front(); } info->durableVersion = info->storageVersion; @@ -5298,6 +5363,14 @@ ACTOR Future serveRangeFeedRequests(StorageServer* self, FutureStream serveRangeFeedStreamRequests(StorageServer* self, + FutureStream rangeFeedStream) { + loop { + RangeFeedStreamRequest req = waitNext(rangeFeedStream); + self->actors.add(rangeFeedStreamQ(self, req)); + } +} + ACTOR Future serveOverlappingRangeFeedsRequests( StorageServer* self, FutureStream overlappingRangeFeeds) { @@ -5364,6 +5437,7 @@ ACTOR Future storageServerCore(StorageServer* self, StorageServerInterface self->actors.add(serveGetKeyRequests(self, ssi.getKey.getFuture())); self->actors.add(serveWatchValueRequests(self, ssi.watchValue.getFuture())); self->actors.add(serveRangeFeedRequests(self, ssi.rangeFeed.getFuture())); + self->actors.add(serveRangeFeedStreamRequests(self, ssi.rangeFeedStream.getFuture())); self->actors.add(serveOverlappingRangeFeedsRequests(self, ssi.overlappingRangeFeeds.getFuture())); self->actors.add(serveRangeFeedPopRequests(self, ssi.rangeFeedPop.getFuture())); self->actors.add(traceRole(Role::STORAGE_SERVER, ssi.id())); From 94239098a50b89c1836ae51a0ecc04300bcc31f1 Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Tue, 10 Aug 2021 23:59:05 -0700 Subject: [PATCH 41/72] added additional rangefeed commands to fdbcli --- fdbcli/fdbcli.actor.cpp | 78 +++++++++++++++++++++++++++++++---- fdbclient/NativeAPI.actor.cpp | 4 ++ fdbclient/NativeAPI.actor.h | 1 + 3 files changed, 76 insertions(+), 7 deletions(-) diff --git a/fdbcli/fdbcli.actor.cpp b/fdbcli/fdbcli.actor.cpp index faedc44764..bef169ecd3 100644 --- a/fdbcli/fdbcli.actor.cpp +++ b/fdbcli/fdbcli.actor.cpp @@ -36,6 +36,7 @@ #include "fdbclient/Schemas.h" #include "fdbclient/CoordinationInterface.h" #include "fdbclient/FDBOptions.g.h" +#include "fdbclient/SystemData.h" #include "fdbclient/TagThrottle.h" #include "fdbclient/Tuple.h" @@ -657,7 +658,8 @@ void initHelp() { CommandHelp("triggerddteaminfolog", "trigger the data distributor teams logging", "Trigger the data distributor to log detailed information about its teams."); - helpMap["rangefeed"] = CommandHelp("rangefeed ", "", ""); + helpMap["rangefeed"] = + CommandHelp("rangefeed ", "", ""); helpMap["tssq"] = CommandHelp("tssq start|stop ", "start/stop tss quarantine", @@ -1969,6 +1971,31 @@ ACTOR Future commitTransaction(Reference tr) { return Void(); } +ACTOR Future rangeFeedList(Database db) { + state ReadYourWritesTransaction tr(db); + loop { + try { + tr.setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); + tr.setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); + + RangeResult result = wait(tr.getRange(rangeFeedKeys, CLIENT_KNOBS->TOO_MANY)); + // shouldn't have many quarantined TSSes + ASSERT(!result.more); + printf("Found %d range feeds%s\n", result.size(), result.size() == 0 ? "." : ":"); + for (auto& it : result) { + auto range = decodeRangeFeedValue(it.value); + printf(" %s: %s - %s\n", + it.key.removePrefix(rangeFeedPrefix).toString().c_str(), + range.begin.toString().c_str(), + range.end.toString().c_str()); + } + return Void(); + } catch (Error& e) { + wait(tr.onError(e)); + } + } +} + ACTOR Future configure(Database db, std::vector tokens, Reference ccf, @@ -3240,6 +3267,7 @@ ACTOR Future cli(CLIOptions opt, LineNoise* plinenoise) { state Database db; state Reference tr; + state Transaction trx; // TODO: refactoring work, will replace db, tr when we have all commands through the general fdb interface state Reference db2; state Reference tr2; @@ -3510,13 +3538,21 @@ ACTOR Future cli(CLIOptions opt, LineNoise* plinenoise) { is_error = true; continue; } - if (tokencmp(tokens[1], "register")) { + if (tokencmp(tokens[1], "list")) { + if (tokens.size() != 2) { + printUsage(tokens[0]); + is_error = true; + continue; + } + wait(rangeFeedList(db)); + continue; + } else if (tokencmp(tokens[1], "register")) { if (tokens.size() != 5) { printUsage(tokens[0]); is_error = true; continue; } - state Transaction trx(db); + trx = Transaction(db); loop { try { wait(trx.registerRangeFeed(tokens[2], KeyRangeRef(tokens[3], tokens[4]))); @@ -3526,6 +3562,22 @@ ACTOR Future cli(CLIOptions opt, LineNoise* plinenoise) { wait(trx.onError(e)); } } + } else if (tokencmp(tokens[1], "destroy")) { + if (tokens.size() != 3) { + printUsage(tokens[0]); + is_error = true; + continue; + } + trx = Transaction(db); + loop { + try { + trx.destroyRangeFeed(tokens[2]); + wait(trx.commit()); + break; + } catch (Error& e) { + wait(trx.onError(e)); + } + } } else if (tokencmp(tokens[1], "get")) { if (tokens.size() < 3 || tokens.size() > 5) { printUsage(tokens[0]); @@ -3591,14 +3643,26 @@ ACTOR Future cli(CLIOptions opt, LineNoise* plinenoise) { state Future feed = db->getRangeFeedStream(feedResults, tokens[2], begin, end); printf("\n"); try { + state Future feedInterrupt = LineNoise::onKeyboardInterrupt(); loop { - Standalone> res = waitNext(feedResults.getFuture()); - for (auto& it : res) { - for (auto& it2 : it.mutations) { - printf("%lld %s\n", it.version, it2.toString().c_str()); + choose { + when(Standalone> res = + waitNext(feedResults.getFuture())) { + for (auto& it : res) { + for (auto& it2 : it.mutations) { + printf("%lld %s\n", it.version, it2.toString().c_str()); + } + } + } + when(wait(feedInterrupt)) { + feedInterrupt = Future(); + feed.cancel(); + feedResults = PromiseStream>>(); + break; } } } + continue; } catch (Error& e) { if (e.code() == error_code_end_of_stream) { continue; diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index 06c6f2d38f..41b875a5e3 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -4330,6 +4330,10 @@ Future Transaction::registerRangeFeed(const Key& rangeID, const KeyRange& return registerRangeFeedActor(this, rangeID, range); } +void Transaction::destroyRangeFeed(const Key& rangeID) { + clear(rangeID.withPrefix(rangeFeedPrefix)); +} + ACTOR Future getKeyAndConflictRange(Database cx, KeySelector k, Future version, diff --git a/fdbclient/NativeAPI.actor.h b/fdbclient/NativeAPI.actor.h index 60e51762ad..c9cf10daf5 100644 --- a/fdbclient/NativeAPI.actor.h +++ b/fdbclient/NativeAPI.actor.h @@ -327,6 +327,7 @@ public: [[nodiscard]] Future>> getAddressesForKey(const Key& key); Future registerRangeFeed(const Key& rangeID, const KeyRange& range); + void destroyRangeFeed(const Key& rangeID); void enableCheckWrites(); void addReadConflictRange(KeyRangeRef const& keys); From a278d2977a27fc93e573a32a62b9ac869ac73825 Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Fri, 13 Aug 2021 14:27:15 -0700 Subject: [PATCH 42/72] renamed range feeds to change feeds, data distribution support almost complete --- fdbcli/fdbcli.actor.cpp | 26 +- fdbclient/DatabaseContext.h | 16 +- fdbclient/NativeAPI.actor.cpp | 144 +++++----- fdbclient/NativeAPI.actor.h | 4 +- fdbclient/StorageServerInterface.cpp | 24 +- fdbclient/StorageServerInterface.h | 82 +++--- fdbclient/SystemData.cpp | 32 +-- fdbclient/SystemData.h | 22 +- fdbserver/ApplyMetadataMutation.cpp | 8 +- fdbserver/storageserver.actor.cpp | 407 ++++++++++++++++----------- flow/ProtocolVersion.h | 2 +- 11 files changed, 425 insertions(+), 342 deletions(-) diff --git a/fdbcli/fdbcli.actor.cpp b/fdbcli/fdbcli.actor.cpp index bef169ecd3..5dcd85be2c 100644 --- a/fdbcli/fdbcli.actor.cpp +++ b/fdbcli/fdbcli.actor.cpp @@ -658,8 +658,8 @@ void initHelp() { CommandHelp("triggerddteaminfolog", "trigger the data distributor teams logging", "Trigger the data distributor to log detailed information about its teams."); - helpMap["rangefeed"] = - CommandHelp("rangefeed ", "", ""); + helpMap["changefeed"] = + CommandHelp("changefeed ", "", ""); helpMap["tssq"] = CommandHelp("tssq start|stop ", "start/stop tss quarantine", @@ -1971,21 +1971,21 @@ ACTOR Future commitTransaction(Reference tr) { return Void(); } -ACTOR Future rangeFeedList(Database db) { +ACTOR Future changeFeedList(Database db) { state ReadYourWritesTransaction tr(db); loop { try { tr.setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); tr.setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); - RangeResult result = wait(tr.getRange(rangeFeedKeys, CLIENT_KNOBS->TOO_MANY)); + RangeResult result = wait(tr.getRange(changeFeedKeys, CLIENT_KNOBS->TOO_MANY)); // shouldn't have many quarantined TSSes ASSERT(!result.more); printf("Found %d range feeds%s\n", result.size(), result.size() == 0 ? "." : ":"); for (auto& it : result) { - auto range = decodeRangeFeedValue(it.value); + auto range = decodeChangeFeedValue(it.value); printf(" %s: %s - %s\n", - it.key.removePrefix(rangeFeedPrefix).toString().c_str(), + it.key.removePrefix(changeFeedPrefix).toString().c_str(), range.begin.toString().c_str(), range.end.toString().c_str()); } @@ -3532,7 +3532,7 @@ ACTOR Future cli(CLIOptions opt, LineNoise* plinenoise) { continue; } - if (tokencmp(tokens[0], "rangefeed")) { + if (tokencmp(tokens[0], "changefeed")) { if (tokens.size() == 1) { printUsage(tokens[0]); is_error = true; @@ -3544,7 +3544,7 @@ ACTOR Future cli(CLIOptions opt, LineNoise* plinenoise) { is_error = true; continue; } - wait(rangeFeedList(db)); + wait(changeFeedList(db)); continue; } else if (tokencmp(tokens[1], "register")) { if (tokens.size() != 5) { @@ -3555,7 +3555,7 @@ ACTOR Future cli(CLIOptions opt, LineNoise* plinenoise) { trx = Transaction(db); loop { try { - wait(trx.registerRangeFeed(tokens[2], KeyRangeRef(tokens[3], tokens[4]))); + wait(trx.registerChangeFeed(tokens[2], KeyRangeRef(tokens[3], tokens[4]))); wait(trx.commit()); break; } catch (Error& e) { @@ -3571,7 +3571,7 @@ ACTOR Future cli(CLIOptions opt, LineNoise* plinenoise) { trx = Transaction(db); loop { try { - trx.destroyRangeFeed(tokens[2]); + trx.destroyChangeFeed(tokens[2]); wait(trx.commit()); break; } catch (Error& e) { @@ -3604,7 +3604,7 @@ ACTOR Future cli(CLIOptions opt, LineNoise* plinenoise) { } } Standalone> res = - wait(db->getRangeFeedMutations(tokens[2], begin, end)); + wait(db->getChangeFeedMutations(tokens[2], begin, end)); printf("\n"); for (auto& it : res) { for (auto& it2 : it.mutations) { @@ -3640,7 +3640,7 @@ ACTOR Future cli(CLIOptions opt, LineNoise* plinenoise) { warn.cancel(); } state PromiseStream>> feedResults; - state Future feed = db->getRangeFeedStream(feedResults, tokens[2], begin, end); + state Future feed = db->getChangeFeedStream(feedResults, tokens[2], begin, end); printf("\n"); try { state Future feedInterrupt = LineNoise::onKeyboardInterrupt(); @@ -3681,7 +3681,7 @@ ACTOR Future cli(CLIOptions opt, LineNoise* plinenoise) { printUsage(tokens[0]); is_error = true; } else { - wait(db->popRangeFeedMutations(tokens[2], v)); + wait(db->popChangeFeedMutations(tokens[2], v)); } } continue; diff --git a/fdbclient/DatabaseContext.h b/fdbclient/DatabaseContext.h index 6c597d58fc..2b0ea30fd7 100644 --- a/fdbclient/DatabaseContext.h +++ b/fdbclient/DatabaseContext.h @@ -252,20 +252,20 @@ public: // Management API, create snapshot Future createSnapshot(StringRef uid, StringRef snapshot_command); - Future>> getRangeFeedMutations( + Future>> getChangeFeedMutations( StringRef rangeID, Version begin = 0, Version end = std::numeric_limits::max(), KeyRange range = allKeys); - Future getRangeFeedStream(const PromiseStream>>& results, - StringRef rangeID, - Version begin = 0, - Version end = std::numeric_limits::max(), - KeyRange range = allKeys); + Future getChangeFeedStream(const PromiseStream>>& results, + StringRef rangeID, + Version begin = 0, + Version end = std::numeric_limits::max(), + KeyRange range = allKeys); - Future>> getOverlappingRangeFeeds(KeyRangeRef ranges, Version minVersion); - Future popRangeFeedMutations(StringRef rangeID, Version version); + Future>> getOverlappingChangeFeeds(KeyRangeRef ranges, Version minVersion); + Future popChangeFeedMutations(StringRef rangeID, Version version); // private: explicit DatabaseContext(Reference>> connectionFile, diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index 41b875a5e3..78ecd5c3f8 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -4315,23 +4315,23 @@ Future>> Transaction::getAddressesForKey(const return getAddressesForKeyActor(key, ver, cx, info, options); } -ACTOR Future registerRangeFeedActor(Transaction* tr, Key rangeID, KeyRange range) { - state Key rangeIDKey = rangeID.withPrefix(rangeFeedPrefix); +ACTOR Future registerChangeFeedActor(Transaction* tr, Key rangeID, KeyRange range) { + state Key rangeIDKey = rangeID.withPrefix(changeFeedPrefix); Optional val = wait(tr->get(rangeIDKey)); if (!val.present()) { - tr->set(rangeIDKey, rangeFeedValue(range)); - } else if (decodeRangeFeedValue(val.get()) != range) { + tr->set(rangeIDKey, changeFeedValue(range)); + } else if (decodeChangeFeedValue(val.get()) != range) { throw unsupported_operation(); } return Void(); } -Future Transaction::registerRangeFeed(const Key& rangeID, const KeyRange& range) { - return registerRangeFeedActor(this, rangeID, range); +Future Transaction::registerChangeFeed(const Key& rangeID, const KeyRange& range) { + return registerChangeFeedActor(this, rangeID, range); } -void Transaction::destroyRangeFeed(const Key& rangeID) { - clear(rangeID.withPrefix(rangeFeedPrefix)); +void Transaction::destroyChangeFeed(const Key& rangeID) { + clear(rangeID.withPrefix(changeFeedPrefix)); } ACTOR Future getKeyAndConflictRange(Database cx, @@ -6531,64 +6531,64 @@ Future DatabaseContext::createSnapshot(StringRef uid, StringRef snapshot_c return createSnapshotActor(this, UID::fromString(uid_str), snapshot_command); } -ACTOR Future>> getRangeFeedMutationsActor(Reference db, - StringRef rangeID, - Version begin, - Version end, - KeyRange range) { +ACTOR Future>> getChangeFeedMutationsActor(Reference db, + StringRef rangeID, + Version begin, + Version end, + KeyRange range) { state Database cx(db); state Transaction tr(cx); - state Key rangeIDKey = rangeID.withPrefix(rangeFeedPrefix); - state Span span("NAPI:GetRangeFeedMutations"_loc); + state Key rangeIDKey = rangeID.withPrefix(changeFeedPrefix); + state Span span("NAPI:GetChangeFeedMutations"_loc); Optional val = wait(tr.get(rangeIDKey)); if (!val.present()) { throw unsupported_operation(); } - KeyRange keys = decodeRangeFeedValue(val.get()); + KeyRange keys = decodeChangeFeedValue(val.get()); state vector>> locations = wait(getKeyRangeLocations(cx, keys, 100, Reverse::False, - &StorageServerInterface::rangeFeed, + &StorageServerInterface::changeFeed, TransactionInfo(TaskPriority::DefaultEndpoint, span.context))); if (locations.size() > 1) { throw unsupported_operation(); } - state RangeFeedRequest req; + state ChangeFeedRequest req; req.rangeID = rangeID; req.begin = begin; req.end = end; - RangeFeedReply rep = wait(loadBalance(cx.getPtr(), - locations[0].second, - &StorageServerInterface::rangeFeed, - req, - TaskPriority::DefaultPromiseEndpoint, - AtMostOnce::False, - cx->enableLocalityLoadBalance ? &cx->queueModel : nullptr)); + ChangeFeedReply rep = wait(loadBalance(cx.getPtr(), + locations[0].second, + &StorageServerInterface::changeFeed, + req, + TaskPriority::DefaultPromiseEndpoint, + AtMostOnce::False, + cx->enableLocalityLoadBalance ? &cx->queueModel : nullptr)); return Standalone>(rep.mutations, rep.arena); } -Future>> DatabaseContext::getRangeFeedMutations(StringRef rangeID, - Version begin, - Version end, - KeyRange range) { - return getRangeFeedMutationsActor(Reference::addRef(this), rangeID, begin, end, range); +Future>> DatabaseContext::getChangeFeedMutations(StringRef rangeID, + Version begin, + Version end, + KeyRange range) { + return getChangeFeedMutationsActor(Reference::addRef(this), rangeID, begin, end, range); } -ACTOR Future getRangeFeedStreamActor(Reference db, - PromiseStream>> results, - StringRef rangeID, - Version begin, - Version end, - KeyRange range) { +ACTOR Future getChangeFeedStreamActor(Reference db, + PromiseStream>> results, + StringRef rangeID, + Version begin, + Version end, + KeyRange range) { state Database cx(db); state Transaction tr(cx); - state Key rangeIDKey = rangeID.withPrefix(rangeFeedPrefix); - state Span span("NAPI:GetRangeFeedStream"_loc); + state Key rangeIDKey = rangeID.withPrefix(changeFeedPrefix); + state Span span("NAPI:GetChangeFeedStream"_loc); state KeyRange keys; loop { try { @@ -6597,7 +6597,7 @@ ACTOR Future getRangeFeedStreamActor(Reference db, results.sendError(unsupported_operation()); return Void(); } - keys = decodeRangeFeedValue(val.get()); + keys = decodeChangeFeedValue(val.get()); break; } catch (Error& e) { wait(tr.onError(e)); @@ -6611,7 +6611,7 @@ ACTOR Future getRangeFeedStreamActor(Reference db, keys, 100, Reverse::False, - &StorageServerInterface::rangeFeed, + &StorageServerInterface::changeFeed, TransactionInfo(TaskPriority::DefaultEndpoint, span.context))); if (locations.size() > 1) { @@ -6628,7 +6628,7 @@ ACTOR Future getRangeFeedStreamActor(Reference db, for (int i = 0; i < locations[0].second->size(); i++) { if (!IFailureMonitor::failureMonitor() .getState( - locations[0].second->get(i, &StorageServerInterface::rangeFeedStream).getEndpoint()) + locations[0].second->get(i, &StorageServerInterface::changeFeedStream).getEndpoint()) .failed) { if (deterministicRandom()->random01() <= 1.0 / ++count) { useIdx = i; @@ -6643,7 +6643,7 @@ ACTOR Future getRangeFeedStreamActor(Reference db, vector> ok(locations[0].second->size()); for (int i = 0; i < ok.size(); i++) { ok[i] = IFailureMonitor::failureMonitor().onStateEqual( - locations[0].second->get(i, &StorageServerInterface::rangeFeedStream).getEndpoint(), + locations[0].second->get(i, &StorageServerInterface::changeFeedStream).getEndpoint(), FailureStatus(false)); } @@ -6656,19 +6656,19 @@ ACTOR Future getRangeFeedStreamActor(Reference db, wait(allAlternativesFailedDelay(quorum(ok, 1))); } - state RangeFeedStreamRequest req; + state ChangeFeedStreamRequest req; req.rangeID = rangeID; req.begin = begin; req.end = end; - state ReplyPromiseStream replyStream = - locations[0].second->get(useIdx, &StorageServerInterface::rangeFeedStream).getReplyStream(req); + state ReplyPromiseStream replyStream = + locations[0].second->get(useIdx, &StorageServerInterface::changeFeedStream).getReplyStream(req); loop { wait(results.onEmpty()); choose { when(wait(cx->connectionFileChanged())) { break; } - when(RangeFeedStreamReply rep = waitNext(replyStream.getFuture())) { + when(ChangeFeedStreamReply rep = waitNext(replyStream.getFuture())) { begin = rep.mutations.back().version + 1; results.send(Standalone>(rep.mutations, rep.arena)); } @@ -6690,67 +6690,67 @@ ACTOR Future getRangeFeedStreamActor(Reference db, } } -Future DatabaseContext::getRangeFeedStream( +Future DatabaseContext::getChangeFeedStream( const PromiseStream>>& results, StringRef rangeID, Version begin, Version end, KeyRange range) { - return getRangeFeedStreamActor(Reference::addRef(this), results, rangeID, begin, end, range); + return getChangeFeedStreamActor(Reference::addRef(this), results, rangeID, begin, end, range); } -ACTOR Future>> getOverlappingRangeFeedsActor(Reference db, - KeyRangeRef range, - Version minVersion) { +ACTOR Future>> getOverlappingChangeFeedsActor(Reference db, + KeyRangeRef range, + Version minVersion) { state Database cx(db); state Transaction tr(cx); - state Span span("NAPI:GetOverlappingRangeFeeds"_loc); + state Span span("NAPI:GetOverlappingChangeFeeds"_loc); state vector>> locations = wait(getKeyRangeLocations(cx, range, 100, Reverse::False, - &StorageServerInterface::rangeFeed, + &StorageServerInterface::changeFeed, TransactionInfo(TaskPriority::DefaultEndpoint, span.context))); if (locations.size() > 1) { throw unsupported_operation(); } - state OverlappingRangeFeedsRequest req; + state OverlappingChangeFeedsRequest req; req.range = range; - OverlappingRangeFeedsReply rep = wait(loadBalance(cx.getPtr(), - locations[0].second, - &StorageServerInterface::overlappingRangeFeeds, - req, - TaskPriority::DefaultPromiseEndpoint, - AtMostOnce::False, - cx->enableLocalityLoadBalance ? &cx->queueModel : nullptr)); + OverlappingChangeFeedsReply rep = wait(loadBalance(cx.getPtr(), + locations[0].second, + &StorageServerInterface::overlappingChangeFeeds, + req, + TaskPriority::DefaultPromiseEndpoint, + AtMostOnce::False, + cx->enableLocalityLoadBalance ? &cx->queueModel : nullptr)); return rep.rangeIds; } -Future>> DatabaseContext::getOverlappingRangeFeeds(KeyRangeRef range, - Version minVersion) { - return getOverlappingRangeFeedsActor(Reference::addRef(this), range, minVersion); +Future>> DatabaseContext::getOverlappingChangeFeeds(KeyRangeRef range, + Version minVersion) { + return getOverlappingChangeFeedsActor(Reference::addRef(this), range, minVersion); } -ACTOR Future popRangeFeedMutationsActor(Reference db, StringRef rangeID, Version version) { +ACTOR Future popChangeFeedMutationsActor(Reference db, StringRef rangeID, Version version) { state Database cx(db); state Transaction tr(cx); - state Key rangeIDKey = rangeID.withPrefix(rangeFeedPrefix); - state Span span("NAPI:PopRangeFeedMutations"_loc); + state Key rangeIDKey = rangeID.withPrefix(changeFeedPrefix); + state Span span("NAPI:PopChangeFeedMutations"_loc); Optional val = wait(tr.get(rangeIDKey)); if (!val.present()) { throw unsupported_operation(); } - KeyRange keys = decodeRangeFeedValue(val.get()); + KeyRange keys = decodeChangeFeedValue(val.get()); state vector>> locations = wait(getKeyRangeLocations(cx, keys, 100, Reverse::False, - &StorageServerInterface::rangeFeed, + &StorageServerInterface::changeFeed, TransactionInfo(TaskPriority::DefaultEndpoint, span.context))); if (locations.size() > 1) { @@ -6761,14 +6761,14 @@ ACTOR Future popRangeFeedMutationsActor(Reference db, Str state std::vector> popRequests; for (int i = 0; i < locations[0].second->size(); i++) { popRequests.push_back( - locations[0].second->getInterface(i).rangeFeedPop.getReply(RangeFeedPopRequest(rangeID, version))); + locations[0].second->getInterface(i).changeFeedPop.getReply(ChangeFeedPopRequest(rangeID, version))); } wait(waitForAll(popRequests)); return Void(); } -Future DatabaseContext::popRangeFeedMutations(StringRef rangeID, Version version) { - return popRangeFeedMutationsActor(Reference::addRef(this), rangeID, version); +Future DatabaseContext::popChangeFeedMutations(StringRef rangeID, Version version) { + return popChangeFeedMutationsActor(Reference::addRef(this), rangeID, version); } ACTOR Future setPerpetualStorageWiggle(Database cx, bool enable, LockAware lockAware) { state ReadYourWritesTransaction tr(cx); diff --git a/fdbclient/NativeAPI.actor.h b/fdbclient/NativeAPI.actor.h index c9cf10daf5..8ebbe40722 100644 --- a/fdbclient/NativeAPI.actor.h +++ b/fdbclient/NativeAPI.actor.h @@ -326,8 +326,8 @@ public: [[nodiscard]] Future>> getAddressesForKey(const Key& key); - Future registerRangeFeed(const Key& rangeID, const KeyRange& range); - void destroyRangeFeed(const Key& rangeID); + Future registerChangeFeed(const Key& rangeID, const KeyRange& range); + void destroyChangeFeed(const Key& rangeID); void enableCheckWrites(); void addReadConflictRange(KeyRangeRef const& keys); diff --git a/fdbclient/StorageServerInterface.cpp b/fdbclient/StorageServerInterface.cpp index 417fe03f36..3301917eb0 100644 --- a/fdbclient/StorageServerInterface.cpp +++ b/fdbclient/StorageServerInterface.cpp @@ -298,42 +298,42 @@ void TSS_traceMismatch(TraceEvent& event, // range feed template <> -bool TSS_doCompare(const RangeFeedReply& src, const RangeFeedReply& tss) { +bool TSS_doCompare(const ChangeFeedReply& src, const ChangeFeedReply& tss) { ASSERT(false); return true; } template <> -const char* TSS_mismatchTraceName(const RangeFeedRequest& req) { +const char* TSS_mismatchTraceName(const ChangeFeedRequest& req) { ASSERT(false); return ""; } template <> void TSS_traceMismatch(TraceEvent& event, - const RangeFeedRequest& req, - const RangeFeedReply& src, - const RangeFeedReply& tss) { + const ChangeFeedRequest& req, + const ChangeFeedReply& src, + const ChangeFeedReply& tss) { ASSERT(false); } template <> -bool TSS_doCompare(const OverlappingRangeFeedsReply& src, const OverlappingRangeFeedsReply& tss) { +bool TSS_doCompare(const OverlappingChangeFeedsReply& src, const OverlappingChangeFeedsReply& tss) { ASSERT(false); return true; } template <> -const char* TSS_mismatchTraceName(const OverlappingRangeFeedsRequest& req) { +const char* TSS_mismatchTraceName(const OverlappingChangeFeedsRequest& req) { ASSERT(false); return ""; } template <> void TSS_traceMismatch(TraceEvent& event, - const OverlappingRangeFeedsRequest& req, - const OverlappingRangeFeedsReply& src, - const OverlappingRangeFeedsReply& tss) { + const OverlappingChangeFeedsRequest& req, + const OverlappingChangeFeedsReply& src, + const OverlappingChangeFeedsReply& tss) { ASSERT(false); } @@ -376,10 +376,10 @@ template <> void TSSMetrics::recordLatency(const GetKeyValuesStreamRequest& req, double ssLatency, double tssLatency) {} template <> -void TSSMetrics::recordLatency(const RangeFeedRequest& req, double ssLatency, double tssLatency) {} +void TSSMetrics::recordLatency(const ChangeFeedRequest& req, double ssLatency, double tssLatency) {} template <> -void TSSMetrics::recordLatency(const OverlappingRangeFeedsRequest& req, double ssLatency, double tssLatency) {} +void TSSMetrics::recordLatency(const OverlappingChangeFeedsRequest& req, double ssLatency, double tssLatency) {} // ------------------- diff --git a/fdbclient/StorageServerInterface.h b/fdbclient/StorageServerInterface.h index 14b4991938..e11491e0b2 100644 --- a/fdbclient/StorageServerInterface.h +++ b/fdbclient/StorageServerInterface.h @@ -78,10 +78,10 @@ struct StorageServerInterface { RequestStream getReadHotRanges; RequestStream getRangeSplitPoints; RequestStream getKeyValuesStream; - RequestStream rangeFeed; - RequestStream rangeFeedStream; - RequestStream overlappingRangeFeeds; - RequestStream rangeFeedPop; + RequestStream changeFeed; + RequestStream changeFeedStream; + RequestStream overlappingChangeFeeds; + RequestStream changeFeedPop; explicit StorageServerInterface(UID uid) : uniqueID(uid) {} StorageServerInterface() : uniqueID(deterministicRandom()->randomUniqueID()) {} @@ -124,13 +124,13 @@ struct StorageServerInterface { RequestStream(getValue.getEndpoint().getAdjustedEndpoint(12)); getKeyValuesStream = RequestStream(getValue.getEndpoint().getAdjustedEndpoint(13)); - rangeFeed = RequestStream(getValue.getEndpoint().getAdjustedEndpoint(14)); - rangeFeedStream = - RequestStream(getValue.getEndpoint().getAdjustedEndpoint(15)); - overlappingRangeFeeds = - RequestStream(getValue.getEndpoint().getAdjustedEndpoint(16)); - rangeFeedPop = - RequestStream(getValue.getEndpoint().getAdjustedEndpoint(17)); + changeFeed = RequestStream(getValue.getEndpoint().getAdjustedEndpoint(14)); + changeFeedStream = + RequestStream(getValue.getEndpoint().getAdjustedEndpoint(15)); + overlappingChangeFeeds = + RequestStream(getValue.getEndpoint().getAdjustedEndpoint(16)); + changeFeedPop = + RequestStream(getValue.getEndpoint().getAdjustedEndpoint(17)); } } else { ASSERT(Ar::isDeserializing); @@ -173,10 +173,10 @@ struct StorageServerInterface { streams.push_back(getReadHotRanges.getReceiver()); streams.push_back(getRangeSplitPoints.getReceiver()); streams.push_back(getKeyValuesStream.getReceiver(TaskPriority::LoadBalancedEndpoint)); - streams.push_back(rangeFeed.getReceiver()); - streams.push_back(rangeFeedStream.getReceiver()); - streams.push_back(overlappingRangeFeeds.getReceiver()); - streams.push_back(rangeFeedPop.getReceiver()); + streams.push_back(changeFeed.getReceiver()); + streams.push_back(changeFeedStream.getReceiver()); + streams.push_back(overlappingChangeFeeds.getReceiver()); + streams.push_back(changeFeedPop.getReceiver()); FlowTransport::transport().addEndpoints(streams); } }; @@ -645,34 +645,40 @@ struct MutationsAndVersionRef { : mutations(to, from.mutations), version(from.version) {} int expectedSize() const { return mutations.expectedSize(); } + struct OrderByVersion { + bool operator()(MutationsAndVersionRef const& a, MutationsAndVersionRef const& b) const { + return a.version < b.version; + } + }; + template void serialize(Ar& ar) { serializer(ar, mutations, version); } }; -struct RangeFeedReply { +struct ChangeFeedReply { constexpr static FileIdentifier file_identifier = 11815134; VectorRef mutations; bool cached; Arena arena; - RangeFeedReply() : cached(false) {} + ChangeFeedReply() : cached(false) {} template void serialize(Ar& ar) { serializer(ar, mutations, arena); } }; -struct RangeFeedRequest { +struct ChangeFeedRequest { constexpr static FileIdentifier file_identifier = 10726174; Key rangeID; Version begin = 0; Version end = 0; - ReplyPromise reply; + ReplyPromise reply; - RangeFeedRequest() {} - explicit RangeFeedRequest(Key const& rangeID) : rangeID(rangeID) {} + ChangeFeedRequest() {} + explicit ChangeFeedRequest(Key const& rangeID) : rangeID(rangeID) {} template void serialize(Ar& ar) { @@ -680,15 +686,15 @@ struct RangeFeedRequest { } }; -struct RangeFeedStreamReply : public ReplyPromiseStreamReply { +struct ChangeFeedStreamReply : public ReplyPromiseStreamReply { constexpr static FileIdentifier file_identifier = 1783066; Arena arena; VectorRef mutations; - RangeFeedStreamReply() {} - RangeFeedStreamReply(RangeFeedReply r) : arena(r.arena), mutations(r.mutations) {} + ChangeFeedStreamReply() {} + ChangeFeedStreamReply(ChangeFeedReply r) : arena(r.arena), mutations(r.mutations) {} - int expectedSize() const { return sizeof(RangeFeedStreamReply) + mutations.expectedSize(); } + int expectedSize() const { return sizeof(ChangeFeedStreamReply) + mutations.expectedSize(); } template void serialize(Ar& ar) { @@ -696,30 +702,30 @@ struct RangeFeedStreamReply : public ReplyPromiseStreamReply { } }; -struct RangeFeedStreamRequest { +struct ChangeFeedStreamRequest { constexpr static FileIdentifier file_identifier = 6795746; SpanID spanContext; Arena arena; Key rangeID; Version begin = 0; Version end = 0; - ReplyPromiseStream reply; + ReplyPromiseStream reply; - RangeFeedStreamRequest() {} + ChangeFeedStreamRequest() {} template void serialize(Ar& ar) { serializer(ar, rangeID, begin, end, reply, spanContext, arena); } }; -struct RangeFeedPopRequest { +struct ChangeFeedPopRequest { constexpr static FileIdentifier file_identifier = 10726174; Key rangeID; Version version; ReplyPromise reply; - RangeFeedPopRequest() {} - RangeFeedPopRequest(Key const& rangeID, Version version) : rangeID(rangeID), version(version) {} + ChangeFeedPopRequest() {} + ChangeFeedPopRequest(Key const& rangeID, Version version) : rangeID(rangeID), version(version) {} template void serialize(Ar& ar) { @@ -727,14 +733,14 @@ struct RangeFeedPopRequest { } }; -struct OverlappingRangeFeedsReply { +struct OverlappingChangeFeedsReply { constexpr static FileIdentifier file_identifier = 11815134; std::vector> rangeIds; bool cached; Arena arena; - OverlappingRangeFeedsReply() : cached(false) {} - explicit OverlappingRangeFeedsReply(std::vector> const& rangeIds) + OverlappingChangeFeedsReply() : cached(false) {} + explicit OverlappingChangeFeedsReply(std::vector> const& rangeIds) : rangeIds(rangeIds), cached(false) {} template @@ -742,13 +748,13 @@ struct OverlappingRangeFeedsReply { serializer(ar, rangeIds, arena); } }; -struct OverlappingRangeFeedsRequest { +struct OverlappingChangeFeedsRequest { constexpr static FileIdentifier file_identifier = 10726174; KeyRange range; - ReplyPromise reply; + ReplyPromise reply; - OverlappingRangeFeedsRequest() {} - explicit OverlappingRangeFeedsRequest(KeyRange const& range) : range(range) {} + OverlappingChangeFeedsRequest() {} + explicit OverlappingChangeFeedsRequest(KeyRange const& range) : range(range) {} template void serialize(Ar& ar) { diff --git a/fdbclient/SystemData.cpp b/fdbclient/SystemData.cpp index faba5828ce..342dc9ab88 100644 --- a/fdbclient/SystemData.cpp +++ b/fdbclient/SystemData.cpp @@ -1030,46 +1030,46 @@ const KeyRef writeRecoveryKey = LiteralStringRef("\xff/writeRecovery"); const ValueRef writeRecoveryKeyTrue = LiteralStringRef("1"); const KeyRef snapshotEndVersionKey = LiteralStringRef("\xff/snapshotEndVersion"); -const KeyRangeRef rangeFeedKeys(LiteralStringRef("\xff\x02/feed/"), LiteralStringRef("\xff\x02/feed0")); -const KeyRef rangeFeedPrefix = rangeFeedKeys.begin; -const KeyRef rangeFeedPrivatePrefix = LiteralStringRef("\xff\xff\x02/feed/"); +const KeyRangeRef changeFeedKeys(LiteralStringRef("\xff\x02/feed/"), LiteralStringRef("\xff\x02/feed0")); +const KeyRef changeFeedPrefix = changeFeedKeys.begin; +const KeyRef changeFeedPrivatePrefix = LiteralStringRef("\xff\xff\x02/feed/"); -const Value rangeFeedValue(KeyRangeRef const& range) { - BinaryWriter wr(IncludeVersion(ProtocolVersion::withRangeFeed())); +const Value changeFeedValue(KeyRangeRef const& range) { + BinaryWriter wr(IncludeVersion(ProtocolVersion::withChangeFeed())); wr << range; return wr.toValue(); } -KeyRange decodeRangeFeedValue(ValueRef const& value) { +KeyRange decodeChangeFeedValue(ValueRef const& value) { KeyRange range; BinaryReader reader(value, IncludeVersion()); reader >> range; return range; } -const KeyRangeRef rangeFeedDurableKeys(LiteralStringRef("\xff\xff/rf/"), LiteralStringRef("\xff\xff/rf0")); -const KeyRef rangeFeedDurablePrefix = rangeFeedDurableKeys.begin; +const KeyRangeRef changeFeedDurableKeys(LiteralStringRef("\xff\xff/rf/"), LiteralStringRef("\xff\xff/rf0")); +const KeyRef changeFeedDurablePrefix = changeFeedDurableKeys.begin; -const Value rangeFeedDurableKey(Key const& feed, Version const& version) { - BinaryWriter wr(AssumeVersion(ProtocolVersion::withRangeFeed())); - wr.serializeBytes(rangeFeedDurablePrefix); +const Value changeFeedDurableKey(Key const& feed, Version const& version) { + BinaryWriter wr(AssumeVersion(ProtocolVersion::withChangeFeed())); + wr.serializeBytes(changeFeedDurablePrefix); wr << feed; wr << bigEndian64(version); return wr.toValue(); } -std::pair decodeRangeFeedDurableKey(ValueRef const& key) { +std::pair decodeChangeFeedDurableKey(ValueRef const& key) { Key feed; Version version; - BinaryReader reader(key.removePrefix(rangeFeedDurablePrefix), AssumeVersion(ProtocolVersion::withRangeFeed())); + BinaryReader reader(key.removePrefix(changeFeedDurablePrefix), AssumeVersion(ProtocolVersion::withChangeFeed())); reader >> feed; reader >> version; return std::make_pair(feed, bigEndian64(version)); } -const Value rangeFeedDurableValue(Standalone> const& mutations) { - BinaryWriter wr(IncludeVersion(ProtocolVersion::withRangeFeed())); +const Value changeFeedDurableValue(Standalone> const& mutations) { + BinaryWriter wr(IncludeVersion(ProtocolVersion::withChangeFeed())); wr << mutations; return wr.toValue(); } -Standalone> decodeRangeFeedDurableValue(ValueRef const& value) { +Standalone> decodeChangeFeedDurableValue(ValueRef const& value) { Standalone> mutations; BinaryReader reader(value, IncludeVersion()); reader >> mutations; diff --git a/fdbclient/SystemData.h b/fdbclient/SystemData.h index 316c528405..7a0b3c4dfd 100644 --- a/fdbclient/SystemData.h +++ b/fdbclient/SystemData.h @@ -495,19 +495,19 @@ extern const ValueRef writeRecoveryKeyTrue; // Allows incremental restore to read and set starting version for consistency. extern const KeyRef snapshotEndVersionKey; -extern const KeyRangeRef rangeFeedKeys; -const Value rangeFeedValue(KeyRangeRef const& range); -KeyRange decodeRangeFeedValue(ValueRef const& value); -extern const KeyRef rangeFeedPrefix; -extern const KeyRef rangeFeedPrivatePrefix; +extern const KeyRangeRef changeFeedKeys; +const Value changeFeedValue(KeyRangeRef const& range); +KeyRange decodeChangeFeedValue(ValueRef const& value); +extern const KeyRef changeFeedPrefix; +extern const KeyRef changeFeedPrivatePrefix; -extern const KeyRangeRef rangeFeedDurableKeys; -extern const KeyRef rangeFeedDurablePrefix; +extern const KeyRangeRef changeFeedDurableKeys; +extern const KeyRef changeFeedDurablePrefix; -const Value rangeFeedDurableKey(Key const& feed, Version const& version); -std::pair decodeRangeFeedDurableKey(ValueRef const& key); -const Value rangeFeedDurableValue(Standalone> const& mutations); -Standalone> decodeRangeFeedDurableValue(ValueRef const& value); +const Value changeFeedDurableKey(Key const& feed, Version const& version); +std::pair decodeChangeFeedDurableKey(ValueRef const& key); +const Value changeFeedDurableValue(Standalone> const& mutations); +Standalone> decodeChangeFeedDurableValue(ValueRef const& value); // Configuration database special keys extern const KeyRef configTransactionDescriptionKey; diff --git a/fdbserver/ApplyMetadataMutation.cpp b/fdbserver/ApplyMetadataMutation.cpp index f5ec26386a..693eaa2603 100644 --- a/fdbserver/ApplyMetadataMutation.cpp +++ b/fdbserver/ApplyMetadataMutation.cpp @@ -408,9 +408,9 @@ void applyMetadataMutations(SpanID const& spanContext, if (!initialCommit) txnStateStore->set(KeyValueRef(m.param1, m.param2)); TEST(true); // Snapshot created, setting writeRecoveryKey in txnStateStore - } else if (m.param1.startsWith(rangeFeedPrefix)) { + } else if (m.param1.startsWith(changeFeedPrefix)) { if (toCommit && keyInfo) { - KeyRange r = decodeRangeFeedValue(m.param2); + KeyRange r = decodeChangeFeedValue(m.param2); MutationRef privatized = m; privatized.param1 = m.param1.withPrefix(systemKeys.begin, arena); auto ranges = keyInfo->intersectingRanges(r); @@ -418,7 +418,7 @@ void applyMetadataMutations(SpanID const& spanContext, ++firstRange; if (firstRange == ranges.end()) { ranges.begin().value().populateTags(); - TraceEvent("RangeFeedTags1").detail("Tags", describe(ranges.begin().value().tags)); + TraceEvent("ChangeFeedTags1").detail("Tags", describe(ranges.begin().value().tags)); toCommit->addTags(ranges.begin().value().tags); } else { std::set allSources; @@ -426,7 +426,7 @@ void applyMetadataMutations(SpanID const& spanContext, r.value().populateTags(); allSources.insert(r.value().tags.begin(), r.value().tags.end()); } - TraceEvent("RangeFeedTags2").detail("Tags", describe(allSources)); + TraceEvent("ChangeFeedTags2").detail("Tags", describe(allSources)); toCommit->addTags(allSources); } toCommit->writeTypedMessage(privatized); diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index 9ea3c2be5a..3e8c26b527 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -94,7 +94,7 @@ struct AddingShard : NonCopyable { Future fetchClient; // holds FetchKeys() actor Promise fetchComplete; Promise readWrite; - PromiseStream rangeFeedRemovals; + PromiseStream changeFeedRemovals; // During the Fetching phase, it saves newer mutations whose version is greater or equal to fetchClient's // fetchVersion, while the shard is still busy catching up with fetchClient. It applies these updates after fetching @@ -310,7 +310,7 @@ struct FetchInjectionInfo { vector changes; }; -struct RangeFeedInfo : ReferenceCounted { +struct ChangeFeedInfo : ReferenceCounted { std::deque> mutations; Version storageVersion = invalidVersion; Version durableVersion = invalidVersion; @@ -584,11 +584,11 @@ public: KeyRangeMap cachedRangeMap; // indicates if a key-range is being cached - KeyRangeMap>> keyRangeFeed; - std::map> uidRangeFeed; - Deque, Version>> rangeFeedVersions; - std::map> rangeFeedRemovals; - std::set currentRangeFeeds; + KeyRangeMap>> keyChangeFeed; + std::map> uidChangeFeed; + Deque, Version>> changeFeedVersions; + std::map> changeFeedRemovals; + std::set currentChangeFeeds; // newestAvailableVersion[k] // == invalidVersion -> k is unavailable at all versions @@ -1512,16 +1512,16 @@ ACTOR Future watchValueSendReply(StorageServer* data, } } -ACTOR Future rangeFeedPopQ(StorageServer* self, RangeFeedPopRequest req) { - auto& feed = self->uidRangeFeed[req.rangeID]; +ACTOR Future changeFeedPopQ(StorageServer* self, ChangeFeedPopRequest req) { + auto& feed = self->uidChangeFeed[req.rangeID]; if (req.version - 1 > feed->emptyVersion) { feed->emptyVersion = req.version - 1; while (!feed->mutations.empty() && feed->mutations.front().version < req.version) { - self->uidRangeFeed[req.rangeID]->mutations.pop_front(); + self->uidChangeFeed[req.rangeID]->mutations.pop_front(); } if (feed->storageVersion != invalidVersion) { self->storage.clearRange( - KeyRangeRef(rangeFeedDurableKey(feed->id, 0), rangeFeedDurableKey(feed->id, req.version))); + KeyRangeRef(changeFeedDurableKey(feed->id, 0), changeFeedDurableKey(feed->id, req.version))); if (req.version > feed->storageVersion) { feed->storageVersion = invalidVersion; feed->durableVersion = invalidVersion; @@ -1529,16 +1529,16 @@ ACTOR Future rangeFeedPopQ(StorageServer* self, RangeFeedPopRequest req) { wait(self->durableVersion.whenAtLeast(self->storageVersion() + 1)); } } - TraceEvent("RangeFeedPopQuery", self->thisServerID) + TraceEvent("ChangeFeedPopQuery", self->thisServerID) .detail("RangeID", req.rangeID.printable()) .detail("Version", req.version); req.reply.send(Void()); return Void(); } -ACTOR Future overlappingRangeFeedsQ(StorageServer* data, OverlappingRangeFeedsRequest req) { +ACTOR Future overlappingChangeFeedsQ(StorageServer* data, OverlappingChangeFeedsRequest req) { wait(delay(0)); - auto ranges = data->keyRangeFeed.intersectingRanges(req.range); + auto ranges = data->keyChangeFeed.intersectingRanges(req.range); std::map rangeIds; for (auto r : ranges) { for (auto& it : r.value()) { @@ -1549,22 +1549,22 @@ ACTOR Future overlappingRangeFeedsQ(StorageServer* data, OverlappingRangeF for (auto& it : rangeIds) { result.push_back(std::make_pair(it.first, it.second)); } - OverlappingRangeFeedsReply reply(std::vector>(rangeIds.begin(), rangeIds.end())); + OverlappingChangeFeedsReply reply(std::vector>(rangeIds.begin(), rangeIds.end())); req.reply.send(reply); return Void(); } -ACTOR Future getRangeFeedMutations(StorageServer* data, RangeFeedRequest req) { - state RangeFeedReply reply; +ACTOR Future getChangeFeedMutations(StorageServer* data, ChangeFeedRequest req) { + state ChangeFeedReply reply; state int remainingLimitBytes = CLIENT_KNOBS->REPLY_BYTE_LIMIT; wait(delay(0, TaskPriority::DefaultEndpoint)); if (data->version.get() < req.begin) { wait(data->version.whenAtLeast(req.begin)); } - auto& feedInfo = data->uidRangeFeed[req.rangeID]; + auto& feedInfo = data->uidChangeFeed[req.rangeID]; if (req.end <= feedInfo->emptyVersion + 1) { } else if (feedInfo->durableVersion == invalidVersion || req.begin > feedInfo->durableVersion) { - for (auto& it : data->uidRangeFeed[req.rangeID]->mutations) { + for (auto& it : data->uidChangeFeed[req.rangeID]->mutations) { if (it.version >= req.end || remainingLimitBytes <= 0) { break; } @@ -1575,10 +1575,10 @@ ACTOR Future getRangeFeedMutations(StorageServer* data, RangeFee } } else { state std::deque> mutationsDeque = - data->uidRangeFeed[req.rangeID]->mutations; + data->uidChangeFeed[req.rangeID]->mutations; state Version startingDurableVersion = feedInfo->durableVersion; RangeResult res = wait(data->storage.readRange( - KeyRangeRef(rangeFeedDurableKey(req.rangeID, req.begin), rangeFeedDurableKey(req.rangeID, req.end)), + KeyRangeRef(changeFeedDurableKey(req.rangeID, req.begin), changeFeedDurableKey(req.rangeID, req.end)), 1 << 30, remainingLimitBytes)); @@ -1586,8 +1586,8 @@ ACTOR Future getRangeFeedMutations(StorageServer* data, RangeFee for (auto& kv : res) { Key id; Version version; - std::tie(id, version) = decodeRangeFeedDurableKey(kv.key); - auto mutations = decodeRangeFeedDurableValue(kv.value); + std::tie(id, version) = decodeChangeFeedDurableKey(kv.key); + auto mutations = decodeChangeFeedDurableValue(kv.value); reply.mutations.push_back_deep(reply.arena, MutationsAndVersionRef(mutations, version)); remainingLimitBytes -= sizeof(KeyValueRef) + @@ -1605,23 +1605,23 @@ ACTOR Future getRangeFeedMutations(StorageServer* data, RangeFee } } if (reply.mutations.empty()) { - auto& feedInfo = data->uidRangeFeed[req.rangeID]; + auto& feedInfo = data->uidChangeFeed[req.rangeID]; if (startingDurableVersion == feedInfo->storageVersion && req.end > startingDurableVersion) { if (req.begin == 0) { feedInfo->durableVersion = req.end > data->storageVersion() ? invalidVersion : req.end; } else { RangeResult emp = wait(data->storage.readRange( - KeyRangeRef(rangeFeedDurableKey(req.rangeID, 0), rangeFeedDurableKey(req.rangeID, req.end)), + KeyRangeRef(changeFeedDurableKey(req.rangeID, 0), changeFeedDurableKey(req.rangeID, req.end)), -1)); - auto& feedInfo = data->uidRangeFeed[req.rangeID]; + auto& feedInfo = data->uidChangeFeed[req.rangeID]; if (startingDurableVersion == feedInfo->storageVersion) { if (emp.empty()) { feedInfo->durableVersion = req.end > data->storageVersion() ? invalidVersion : req.end; } else { Key id; Version version; - std::tie(id, version) = decodeRangeFeedDurableKey(emp[0].key); + std::tie(id, version) = decodeChangeFeedDurableKey(emp[0].key); feedInfo->durableVersion = version; } } @@ -1636,17 +1636,45 @@ ACTOR Future getRangeFeedMutations(StorageServer* data, RangeFee return reply; } -ACTOR Future rangeFeedQ(StorageServer* data, RangeFeedRequest req) { - RangeFeedReply rep = wait(getRangeFeedMutations(data, req)); +ACTOR Future localChangeFeedStream(StorageServer* data, + PromiseStream> results, + Key rangeID, + Version begin, + Version end, + KeyRange range) { + loop { + state ChangeFeedRequest feedRequest; + feedRequest.rangeID = rangeID; + feedRequest.begin = begin; + feedRequest.end = end; + state ChangeFeedReply feedReply = wait(getChangeFeedMutations(data, feedRequest)); + begin = feedReply.mutations.back().version + 1; + state int resultLoc = 0; + while (resultLoc < feedReply.mutations.size()) { + if (feedReply.mutations[resultLoc].mutations.size() || feedReply.mutations[resultLoc].version == end) { + wait(results.onEmpty()); + results.send(feedReply.mutations[resultLoc]); + } + resultLoc++; + } + + if (begin == end) { + return Void(); + } + } +} + +ACTOR Future changeFeedQ(StorageServer* data, ChangeFeedRequest req) { + ChangeFeedReply rep = wait(getChangeFeedMutations(data, req)); req.reply.send(rep); return Void(); } -ACTOR Future rangeFeedStreamQ(StorageServer* data, RangeFeedStreamRequest req) +ACTOR Future changeFeedStreamQ(StorageServer* data, ChangeFeedStreamRequest req) // Throws a wrong_shard_server if the keys in the request or result depend on data outside this server OR if a large // selector offset prevents all data from being read in one range read { - state Span span("SS:getRangeFeedStream"_loc, { req.spanContext }); + state Span span("SS:getChangeFeedStream"_loc, { req.spanContext }); state Version begin = req.begin; req.reply.setByteLimit(SERVER_KNOBS->RANGESTREAM_LIMIT_BYTES); @@ -1655,13 +1683,13 @@ ACTOR Future rangeFeedStreamQ(StorageServer* data, RangeFeedStreamRequest try { loop { wait(req.reply.onReady()); - state RangeFeedRequest feedRequest; + state ChangeFeedRequest feedRequest; feedRequest.rangeID = req.rangeID; feedRequest.begin = begin; feedRequest.end = req.end; - RangeFeedReply feedReply = wait(getRangeFeedMutations(data, feedRequest)); + ChangeFeedReply feedReply = wait(getChangeFeedMutations(data, feedRequest)); begin = feedReply.mutations.back().version + 1; - req.reply.send(RangeFeedStreamReply(feedReply)); + req.reply.send(ChangeFeedStreamReply(feedReply)); if (feedReply.mutations.back().version == req.end - 1) { req.reply.sendError(end_of_stream()); return Void(); @@ -1669,7 +1697,7 @@ ACTOR Future rangeFeedStreamQ(StorageServer* data, RangeFeedStreamRequest if (feedReply.mutations.back().mutations.empty()) { choose { when(wait(delay(5.0, TaskPriority::DefaultEndpoint))) {} - when(wait(data->uidRangeFeed[req.rangeID]->newMutations.onTrigger())) {} + when(wait(data->uidChangeFeed[req.rangeID]->newMutations.onTrigger())) {} } } } @@ -2796,12 +2824,12 @@ void applyMutation(StorageServer* self, self->watches.trigger(m.param1); if (!fromFetch) { - for (auto& it : self->keyRangeFeed[m.param1]) { + for (auto& it : self->keyChangeFeed[m.param1]) { if (it->mutations.empty() || it->mutations.back().version != version) { it->mutations.push_back(MutationsAndVersionRef(version)); } it->mutations.back().mutations.push_back_deep(it->mutations.back().arena(), m); - self->currentRangeFeeds.insert(it->id); + self->currentChangeFeeds.insert(it->id); } } } else if (m.type == MutationRef::ClearRange) { @@ -2812,14 +2840,14 @@ void applyMutation(StorageServer* self, self->watches.triggerRange(m.param1, m.param2); if (!fromFetch) { - auto ranges = self->keyRangeFeed.intersectingRanges(KeyRangeRef(m.param1, m.param2)); + auto ranges = self->keyChangeFeed.intersectingRanges(KeyRangeRef(m.param1, m.param2)); for (auto& r : ranges) { for (auto& it : r.value()) { if (it->mutations.empty() || it->mutations.back().version != version) { it->mutations.push_back(MutationsAndVersionRef(version)); } it->mutations.back().mutations.push_back_deep(it->mutations.back().arena(), m); - self->currentRangeFeeds.insert(it->id); + self->currentChangeFeeds.insert(it->id); } } } @@ -3057,68 +3085,115 @@ static const KeyRangeRef persistByteSampleSampleKeys = LiteralStringRef(PERSIST_PREFIX "BS/" PERSIST_PREFIX "BS0")); static const KeyRef persistLogProtocol = LiteralStringRef(PERSIST_PREFIX "LogProtocol"); static const KeyRef persistPrimaryLocality = LiteralStringRef(PERSIST_PREFIX "PrimaryLocality"); -static const KeyRangeRef persistRangeFeedKeys = +static const KeyRangeRef persistChangeFeedKeys = KeyRangeRef(LiteralStringRef(PERSIST_PREFIX "RF/"), LiteralStringRef(PERSIST_PREFIX "RF0")); // data keys are unmangled (but never start with PERSIST_PREFIX because they are always in allKeys) -ACTOR Future fetchRangeFeed(StorageServer* data, Key rangeId, KeyRange range, Version fetchVersion) { +ACTOR Future fetchChangeFeed(StorageServer* data, Key rangeId, KeyRange range, Version fetchVersion) { + state bool existing = data->uidChangeFeed.count(rangeId); + state Reference changeFeedInfo; + wait(delay(0)); // allow this actor to be cancelled by removals - TraceEvent("FetchRangeFeed", data->thisServerID) + TraceEvent("FetchChangeFeed", data->thisServerID) .detail("RangeID", rangeId.printable()) - .detail("Range", range.toString()); - Reference rangeFeedInfo(new RangeFeedInfo()); - rangeFeedInfo->range = range; - rangeFeedInfo->id = rangeId; - data->uidRangeFeed[rangeId] = rangeFeedInfo; - auto rs = data->keyRangeFeed.modify(range); - for (auto r = rs.begin(); r != rs.end(); ++r) { - r->value().push_back(rangeFeedInfo); - } - data->keyRangeFeed.coalesce(range.contents()); - auto& mLV = data->addVersionToMutationLog(data->data().getLatestVersion()); - data->addMutationToMutationLog(mLV, - MutationRef(MutationRef::SetValue, - persistRangeFeedKeys.begin.toString() + rangeId.toString(), - rangeFeedValue(range))); + .detail("Range", range.toString()) + .detail("Existing", existing); - state Standalone> mutations = - wait(data->cx->getRangeFeedMutations(rangeId, 0, fetchVersion, range)); - state RangeFeedRequest req; - req.rangeID = rangeId; - req.begin = 0; - req.end = fetchVersion; - state RangeFeedReply rep = wait(getRangeFeedMutations(data, req)); - state int mLoc = 0; - state int rLoc = 0; - while (mLoc < mutations.size() && rLoc < rep.mutations.size()) { - if (mutations[mLoc].version < rep.mutations[rLoc].version) { - // write mutation to disk - mLoc++; - } else if (mutations[mLoc].version == rep.mutations[rLoc].version) { - // merge mutations and write to disk - mLoc++; - rLoc++; - } else { - rLoc++; + if (!existing) { + changeFeedInfo = Reference(new ChangeFeedInfo()); + changeFeedInfo->range = range; + changeFeedInfo->id = rangeId; + data->uidChangeFeed[rangeId] = changeFeedInfo; + auto rs = data->keyChangeFeed.modify(range); + for (auto r = rs.begin(); r != rs.end(); ++r) { + r->value().push_back(changeFeedInfo); + } + data->keyChangeFeed.coalesce(range.contents()); + auto& mLV = data->addVersionToMutationLog(data->data().getLatestVersion()); + data->addMutationToMutationLog(mLV, + MutationRef(MutationRef::SetValue, + persistChangeFeedKeys.begin.toString() + rangeId.toString(), + changeFeedValue(range))); + } else { + changeFeedInfo = data->uidChangeFeed[rangeId]; + } + + state PromiseStream>> feedResults; + state Future feed = data->cx->getChangeFeedStream( + feedResults, rangeId, 0, existing ? fetchVersion + 1 : data->version.get() + 1, range); + + if (!existing) { + try { + loop { + Standalone> res = waitNext(feedResults.getFuture()); + for (auto& it : res) { + data->storage.writeKeyValue( + KeyValueRef(changeFeedDurableKey(rangeId, it.version), changeFeedDurableValue(it.mutations))); + } + wait(yield()); + } + } catch (Error& e) { + if (e.code() != error_code_end_of_stream) { + throw; + } } } - while (mLoc < mutations.size()) { - // write mutation to disk - mLoc++; + + state PromiseStream> localResults; + state Future localStream = localChangeFeedStream(data, localResults, rangeId, 0, fetchVersion + 2, range); + state Standalone localResult; + + Standalone _localResult = waitNext(localResults.getFuture()); + localResult = _localResult; + try { + loop { + state Standalone> remoteResult = waitNext(feedResults.getFuture()); + state int remoteLoc = 0; + while (remoteLoc < remoteResult.size()) { + if (remoteResult[remoteLoc].version < localResult.version) { + if (remoteResult[remoteLoc].mutations.size()) { + data->storage.writeKeyValue( + KeyValueRef(changeFeedDurableKey(rangeId, remoteResult[remoteLoc].version), + changeFeedDurableValue(remoteResult[remoteLoc].mutations))); + } + remoteLoc++; + } else if (remoteResult[remoteLoc].version == localResult.version) { + if (remoteResult[remoteLoc].mutations.size()) { + ASSERT(localResult.mutations.size()); + remoteResult[remoteLoc].mutations.append( + remoteResult.arena(), localResult.mutations.begin(), localResult.mutations.size()); + data->storage.writeKeyValue( + KeyValueRef(changeFeedDurableKey(rangeId, remoteResult[remoteLoc].version), + changeFeedDurableValue(remoteResult[remoteLoc].mutations))); + } + remoteLoc++; + Standalone _localResult = waitNext(localResults.getFuture()); + localResult = _localResult; + } else { + Standalone _localResult = waitNext(localResults.getFuture()); + localResult = _localResult; + } + } + wait(yield()); + } + } catch (Error& e) { + if (e.code() != error_code_end_of_stream) { + throw; + } } return Void(); } -ACTOR Future dispatchRangeFeeds(StorageServer* data, UID fetchKeysID, KeyRange keys, Version fetchVersion) { +ACTOR Future dispatchChangeFeeds(StorageServer* data, UID fetchKeysID, KeyRange keys, Version fetchVersion) { // find overlapping range feeds state std::map> feedFetches; state PromiseStream removals; - data->rangeFeedRemovals[fetchKeysID] = removals; + data->changeFeedRemovals[fetchKeysID] = removals; try { state std::vector> feeds = - wait(data->cx->getOverlappingRangeFeeds(keys, fetchVersion)); + wait(data->cx->getOverlappingChangeFeeds(keys, fetchVersion + 1)); for (auto& feed : feeds) { - feedFetches[feed.first] = fetchRangeFeed(data, feed.first, feed.second, fetchVersion); + feedFetches[feed.first] = fetchChangeFeed(data, feed.first, feed.second, fetchVersion); } loop { @@ -3134,7 +3209,7 @@ ACTOR Future dispatchRangeFeeds(StorageServer* data, UID fetchKeysID, KeyR } } if (done) { - data->rangeFeedRemovals.erase(fetchKeysID); + data->changeFeedRemovals.erase(fetchKeysID); return Void(); } } @@ -3146,7 +3221,7 @@ ACTOR Future dispatchRangeFeeds(StorageServer* data, UID fetchKeysID, KeyR } catch (Error& e) { if (!data->shuttingDown) { - data->rangeFeedRemovals.erase(fetchKeysID); + data->changeFeedRemovals.erase(fetchKeysID); } throw; } @@ -3370,6 +3445,7 @@ ACTOR Future fetchKeys(StorageServer* data, AddingShard* shard) { // we have written) state Future fetchDurable = data->durableVersion.whenAtLeast(data->storageVersion() + 1); + // wait(dispatchChangeFeeds(data, fetchKeysID, keys, fetchVersion)); holdingFKPL.release(); wait(fetchDurable); @@ -3522,22 +3598,22 @@ void AddingShard::addMutation(Version version, bool fromFetch, MutationRef const updates.back().mutations.push_back_deep(updates.back().arena(), mutation); if (!fromFetch) { if (mutation.type == MutationRef::SetValue) { - for (auto& it : server->keyRangeFeed[mutation.param1]) { + for (auto& it : server->keyChangeFeed[mutation.param1]) { if (it->mutations.empty() || it->mutations.back().version != version) { it->mutations.push_back(MutationsAndVersionRef(version)); } it->mutations.back().mutations.push_back_deep(it->mutations.back().arena(), mutation); - server->currentRangeFeeds.insert(it->id); + server->currentChangeFeeds.insert(it->id); } } else if (mutation.type == MutationRef::ClearRange) { - auto ranges = server->keyRangeFeed.intersectingRanges(KeyRangeRef(mutation.param1, mutation.param2)); + auto ranges = server->keyChangeFeed.intersectingRanges(KeyRangeRef(mutation.param1, mutation.param2)); for (auto& r : ranges) { for (auto& it : r.value()) { if (it->mutations.empty() || it->mutations.back().version != version) { it->mutations.push_back(MutationsAndVersionRef(version)); } it->mutations.back().mutations.push_back_deep(it->mutations.back().arena(), mutation); - server->currentRangeFeeds.insert(it->id); + server->currentChangeFeeds.insert(it->id); } } } @@ -3860,39 +3936,40 @@ private: auto& mLV = data->addVersionToMutationLog(data->data().getLatestVersion()); data->addMutationToMutationLog(mLV, MutationRef(MutationRef::SetValue, persistPrimaryLocality, m.param2)); } else if ((m.type == MutationRef::SetValue || m.type == MutationRef::ClearRange) && - m.param1.startsWith(rangeFeedPrivatePrefix)) { + m.param1.startsWith(changeFeedPrivatePrefix)) { if (m.type == MutationRef::SetValue) { - Key rangeFeedId = m.param1.removePrefix(rangeFeedPrivatePrefix); - KeyRange rangeFeedRange = decodeRangeFeedValue(m.param2); - TraceEvent("AddingRangeFeed", data->thisServerID) - .detail("RangeID", rangeFeedId.printable()) - .detail("Range", rangeFeedRange.toString()); - Reference rangeFeedInfo(new RangeFeedInfo()); - rangeFeedInfo->range = rangeFeedRange; - rangeFeedInfo->id = rangeFeedId; - rangeFeedInfo->emptyVersion = currentVersion - 1; - data->uidRangeFeed[rangeFeedId] = rangeFeedInfo; - auto rs = data->keyRangeFeed.modify(rangeFeedRange); + Key changeFeedId = m.param1.removePrefix(changeFeedPrivatePrefix); + KeyRange changeFeedRange = decodeChangeFeedValue(m.param2); + TraceEvent("AddingChangeFeed", data->thisServerID) + .detail("RangeID", changeFeedId.printable()) + .detail("Range", changeFeedRange.toString()); + Reference changeFeedInfo(new ChangeFeedInfo()); + changeFeedInfo->range = changeFeedRange; + changeFeedInfo->id = changeFeedId; + changeFeedInfo->emptyVersion = currentVersion - 1; + data->uidChangeFeed[changeFeedId] = changeFeedInfo; + auto rs = data->keyChangeFeed.modify(changeFeedRange); for (auto r = rs.begin(); r != rs.end(); ++r) { - r->value().push_back(rangeFeedInfo); + r->value().push_back(changeFeedInfo); } - data->keyRangeFeed.coalesce(rangeFeedRange.contents()); + data->keyChangeFeed.coalesce(changeFeedRange.contents()); auto& mLV = data->addVersionToMutationLog(data->data().getLatestVersion()); data->addMutationToMutationLog( mLV, MutationRef(MutationRef::SetValue, - persistRangeFeedKeys.begin.toString() + rangeFeedId.toString(), + persistChangeFeedKeys.begin.toString() + changeFeedId.toString(), m.param2)); } else { auto& mLV = data->addVersionToMutationLog(data->data().getLatestVersion()); - auto beginFeed = m.param1.removePrefix(rangeFeedPrivatePrefix); - auto endFeed = m.param2.removePrefix(rangeFeedPrivatePrefix); - data->addMutationToMutationLog(mLV, - MutationRef(MutationRef::ClearRange, - persistRangeFeedKeys.begin.toString() + beginFeed.toString(), - persistRangeFeedKeys.begin.toString() + endFeed.toString())); - data->uidRangeFeed.erase(data->uidRangeFeed.lower_bound(beginFeed), - data->uidRangeFeed.lower_bound(endFeed)); + auto beginFeed = m.param1.removePrefix(changeFeedPrivatePrefix); + auto endFeed = m.param2.removePrefix(changeFeedPrivatePrefix); + data->addMutationToMutationLog( + mLV, + MutationRef(MutationRef::ClearRange, + persistChangeFeedKeys.begin.toString() + beginFeed.toString(), + persistChangeFeedKeys.begin.toString() + endFeed.toString())); + data->uidChangeFeed.erase(data->uidChangeFeed.lower_bound(beginFeed), + data->uidChangeFeed.lower_bound(endFeed)); } } else if (m.param1.substr(1).startsWith(tssMappingKeys.begin) && (m.type == MutationRef::SetValue || m.type == MutationRef::ClearRange)) { @@ -4317,13 +4394,13 @@ ACTOR Future update(StorageServer* data, bool* pReceivedUpdate) { } data->desiredOldestVersion.set(proposedOldestVersion); - if (data->currentRangeFeeds.size()) { - data->rangeFeedVersions.push_back(std::make_pair( - std::vector(data->currentRangeFeeds.begin(), data->currentRangeFeeds.end()), ver)); - for (auto& it : data->currentRangeFeeds) { - data->uidRangeFeed[it]->newMutations.trigger(); + if (data->currentChangeFeeds.size()) { + data->changeFeedVersions.push_back(std::make_pair( + std::vector(data->currentChangeFeeds.begin(), data->currentChangeFeeds.end()), ver)); + for (auto& it : data->currentChangeFeeds) { + data->uidChangeFeed[it]->newMutations.trigger(); } - data->currentRangeFeeds.clear(); + data->currentChangeFeeds.clear(); } } @@ -4388,23 +4465,23 @@ ACTOR Future updateStorage(StorageServer* data) { break; } - std::set modifiedRangeFeeds; - while (!data->rangeFeedVersions.empty() && data->rangeFeedVersions.front().second < newOldestVersion) { - modifiedRangeFeeds.insert(data->rangeFeedVersions.front().first.begin(), - data->rangeFeedVersions.front().first.end()); - data->rangeFeedVersions.pop_front(); + std::set modifiedChangeFeeds; + while (!data->changeFeedVersions.empty() && data->changeFeedVersions.front().second < newOldestVersion) { + modifiedChangeFeeds.insert(data->changeFeedVersions.front().first.begin(), + data->changeFeedVersions.front().first.end()); + data->changeFeedVersions.pop_front(); } - state std::vector updatedRangeFeeds(modifiedRangeFeeds.begin(), modifiedRangeFeeds.end()); + state std::vector updatedChangeFeeds(modifiedChangeFeeds.begin(), modifiedChangeFeeds.end()); state int curFeed = 0; - while (curFeed < updatedRangeFeeds.size()) { - auto info = data->uidRangeFeed[updatedRangeFeeds[curFeed]]; + while (curFeed < updatedChangeFeeds.size()) { + auto info = data->uidChangeFeed[updatedChangeFeeds[curFeed]]; for (auto& it : info->mutations) { if (it.version >= newOldestVersion) { break; } data->storage.writeKeyValue( - KeyValueRef(rangeFeedDurableKey(info->id, it.version), rangeFeedDurableValue(it.mutations))); + KeyValueRef(changeFeedDurableKey(info->id, it.version), changeFeedDurableValue(it.mutations))); info->storageVersion = it.version; } wait(yield(TaskPriority::UpdateStorage)); @@ -4445,8 +4522,8 @@ ACTOR Future updateStorage(StorageServer* data) { } curFeed = 0; - while (curFeed < updatedRangeFeeds.size()) { - auto info = data->uidRangeFeed[updatedRangeFeeds[curFeed]]; + while (curFeed < updatedChangeFeeds.size()) { + auto info = data->uidChangeFeed[updatedChangeFeeds[curFeed]]; while (!info->mutations.empty() && info->mutations.front().version < newOldestVersion) { info->mutations.pop_front(); } @@ -4741,7 +4818,7 @@ ACTOR Future restoreDurableState(StorageServer* data, IKeyValueStore* stor state Future> fPrimaryLocality = storage->readValue(persistPrimaryLocality); state Future fShardAssigned = storage->readRange(persistShardAssignedKeys); state Future fShardAvailable = storage->readRange(persistShardAvailableKeys); - state Future fRangeFeeds = storage->readRange(persistRangeFeedKeys); + state Future fChangeFeeds = storage->readRange(persistChangeFeedKeys); state Promise byteSampleSampleRecovered; state Promise startByteSampleRestore; @@ -4750,7 +4827,7 @@ ACTOR Future restoreDurableState(StorageServer* data, IKeyValueStore* stor TraceEvent("ReadingDurableState", data->thisServerID).log(); wait(waitForAll(std::vector{ fFormat, fID, ftssPairID, fTssQuarantine, fVersion, fLogProtocol, fPrimaryLocality })); - wait(waitForAll(std::vector{ fShardAssigned, fShardAvailable, fRangeFeeds })); + wait(waitForAll(std::vector{ fShardAssigned, fShardAvailable, fChangeFeeds })); wait(byteSampleSampleRecovered.getFuture()); TraceEvent("RestoringDurableState", data->thisServerID).log(); @@ -4828,27 +4905,27 @@ ACTOR Future restoreDurableState(StorageServer* data, IKeyValueStore* stor wait(yield()); } - state RangeResult rangeFeeds = fRangeFeeds.get(); + state RangeResult changeFeeds = fChangeFeeds.get(); state int feedLoc; - for (feedLoc = 0; feedLoc < rangeFeeds.size(); feedLoc++) { - Key rangeFeedId = rangeFeeds[feedLoc].key.removePrefix(persistRangeFeedKeys.begin); - KeyRange rangeFeedRange = decodeRangeFeedValue(rangeFeeds[feedLoc].value); - TraceEvent("RestoringRangeFeed", data->thisServerID) - .detail("RangeID", rangeFeedId.printable()) - .detail("Range", rangeFeedRange.toString()); - Reference rangeFeedInfo(new RangeFeedInfo()); - rangeFeedInfo->range = rangeFeedRange; - rangeFeedInfo->id = rangeFeedId; - rangeFeedInfo->durableVersion = version; - rangeFeedInfo->storageVersion = version; - data->uidRangeFeed[rangeFeedId] = rangeFeedInfo; - auto rs = data->keyRangeFeed.modify(rangeFeedRange); + for (feedLoc = 0; feedLoc < changeFeeds.size(); feedLoc++) { + Key changeFeedId = changeFeeds[feedLoc].key.removePrefix(persistChangeFeedKeys.begin); + KeyRange changeFeedRange = decodeChangeFeedValue(changeFeeds[feedLoc].value); + TraceEvent("RestoringChangeFeed", data->thisServerID) + .detail("RangeID", changeFeedId.printable()) + .detail("Range", changeFeedRange.toString()); + Reference changeFeedInfo(new ChangeFeedInfo()); + changeFeedInfo->range = changeFeedRange; + changeFeedInfo->id = changeFeedId; + changeFeedInfo->durableVersion = version; + changeFeedInfo->storageVersion = version; + data->uidChangeFeed[changeFeedId] = changeFeedInfo; + auto rs = data->keyChangeFeed.modify(changeFeedRange); for (auto r = rs.begin(); r != rs.end(); ++r) { - r->value().push_back(rangeFeedInfo); + r->value().push_back(changeFeedInfo); } wait(yield()); } - data->keyRangeFeed.coalesce(allKeys); + data->keyChangeFeed.coalesce(allKeys); // TODO: why is this seemingly random delay here? wait(delay(0.0001)); @@ -5356,34 +5433,34 @@ ACTOR Future serveWatchValueRequests(StorageServer* self, FutureStream serveRangeFeedRequests(StorageServer* self, FutureStream rangeFeed) { +ACTOR Future serveChangeFeedRequests(StorageServer* self, FutureStream changeFeed) { loop { - RangeFeedRequest req = waitNext(rangeFeed); - self->actors.add(self->readGuard(req, rangeFeedQ)); + ChangeFeedRequest req = waitNext(changeFeed); + self->actors.add(self->readGuard(req, changeFeedQ)); } } -ACTOR Future serveRangeFeedStreamRequests(StorageServer* self, - FutureStream rangeFeedStream) { +ACTOR Future serveChangeFeedStreamRequests(StorageServer* self, + FutureStream changeFeedStream) { loop { - RangeFeedStreamRequest req = waitNext(rangeFeedStream); - self->actors.add(rangeFeedStreamQ(self, req)); + ChangeFeedStreamRequest req = waitNext(changeFeedStream); + self->actors.add(changeFeedStreamQ(self, req)); } } -ACTOR Future serveOverlappingRangeFeedsRequests( +ACTOR Future serveOverlappingChangeFeedsRequests( StorageServer* self, - FutureStream overlappingRangeFeeds) { + FutureStream overlappingChangeFeeds) { loop { - OverlappingRangeFeedsRequest req = waitNext(overlappingRangeFeeds); - self->actors.add(self->readGuard(req, overlappingRangeFeedsQ)); + OverlappingChangeFeedsRequest req = waitNext(overlappingChangeFeeds); + self->actors.add(self->readGuard(req, overlappingChangeFeedsQ)); } } -ACTOR Future serveRangeFeedPopRequests(StorageServer* self, FutureStream rangeFeedPops) { +ACTOR Future serveChangeFeedPopRequests(StorageServer* self, FutureStream changeFeedPops) { loop { - RangeFeedPopRequest req = waitNext(rangeFeedPops); - self->actors.add(self->readGuard(req, rangeFeedPopQ)); + ChangeFeedPopRequest req = waitNext(changeFeedPops); + self->actors.add(self->readGuard(req, changeFeedPopQ)); } } @@ -5436,10 +5513,10 @@ ACTOR Future storageServerCore(StorageServer* self, StorageServerInterface self->actors.add(serveGetKeyValuesStreamRequests(self, ssi.getKeyValuesStream.getFuture())); self->actors.add(serveGetKeyRequests(self, ssi.getKey.getFuture())); self->actors.add(serveWatchValueRequests(self, ssi.watchValue.getFuture())); - self->actors.add(serveRangeFeedRequests(self, ssi.rangeFeed.getFuture())); - self->actors.add(serveRangeFeedStreamRequests(self, ssi.rangeFeedStream.getFuture())); - self->actors.add(serveOverlappingRangeFeedsRequests(self, ssi.overlappingRangeFeeds.getFuture())); - self->actors.add(serveRangeFeedPopRequests(self, ssi.rangeFeedPop.getFuture())); + self->actors.add(serveChangeFeedRequests(self, ssi.changeFeed.getFuture())); + self->actors.add(serveChangeFeedStreamRequests(self, ssi.changeFeedStream.getFuture())); + self->actors.add(serveOverlappingChangeFeedsRequests(self, ssi.overlappingChangeFeeds.getFuture())); + self->actors.add(serveChangeFeedPopRequests(self, ssi.changeFeedPop.getFuture())); self->actors.add(traceRole(Role::STORAGE_SERVER, ssi.id())); self->actors.add(reportStorageServerState(self)); diff --git a/flow/ProtocolVersion.h b/flow/ProtocolVersion.h index e53a90664b..ebf8df96bf 100644 --- a/flow/ProtocolVersion.h +++ b/flow/ProtocolVersion.h @@ -138,7 +138,7 @@ public: // introduced features PROTOCOL_VERSION_FEATURE(0x0FDB00B070010000LL, StableInterfaces); PROTOCOL_VERSION_FEATURE(0x0FDB00B070010001LL, TagThrottleValueReason); PROTOCOL_VERSION_FEATURE(0x0FDB00B070010001LL, SpanContext); - PROTOCOL_VERSION_FEATURE(0x0FDB00B070010001LL, RangeFeed); + PROTOCOL_VERSION_FEATURE(0x0FDB00B070010001LL, ChangeFeed); PROTOCOL_VERSION_FEATURE(0x0FDB00B070010001LL, TSS); }; From 27151b08312a8ab469f0743d6f82f1db5085d88b Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Wed, 1 Sep 2021 14:35:51 -0700 Subject: [PATCH 43/72] support reads from range feeds that span multiple storage servers --- fdbclient/NativeAPI.actor.cpp | 237 ++++++++++++++++++++++++----- fdbclient/StorageServerInterface.h | 6 +- fdbserver/storageserver.actor.cpp | 11 +- 3 files changed, 210 insertions(+), 44 deletions(-) diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index 78ecd5c3f8..993391b237 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -6579,6 +6579,115 @@ Future>> DatabaseContext::getChange return getChangeFeedMutationsActor(Reference::addRef(this), rangeID, begin, end, range); } +ACTOR Future singleChangeFeedStream(StorageServerInterface interf, + PromiseStream> results, + Key rangeID, + Version begin, + Version end, + KeyRange range) { + loop { + try { + state ChangeFeedStreamRequest req; + req.rangeID = rangeID; + req.begin = begin; + req.end = end; + req.range = range; + + state ReplyPromiseStream replyStream = interf.changeFeedStream.getReplyStream(req); + + loop { + state ChangeFeedStreamReply rep = waitNext(replyStream.getFuture()); + begin = rep.mutations.back().version + 1; + state int resultLoc = 0; + while (resultLoc < rep.mutations.size()) { + if (rep.mutations[resultLoc].mutations.size() || rep.mutations[resultLoc].version + 1 == end) { + wait(results.onEmpty()); + results.send(rep.mutations[resultLoc]); + } + resultLoc++; + } + if (begin == end) { + return Void(); + } + } + } catch (Error& e) { + if (e.code() == error_code_actor_cancelled) { + throw; + } + if (e.code() == error_code_wrong_shard_server || e.code() == error_code_all_alternatives_failed || + e.code() == error_code_connection_failed) { + wait(delay(CLIENT_KNOBS->WRONG_SHARD_SERVER_DELAY)); + } else { + results.sendError(e); + return Void(); + } + } + } +} + +struct MutationAndVersionStream { + Standalone next; + PromiseStream> results; + + bool operator<(MutationAndVersionStream const& rhs) const { return next.version < rhs.next.version; } +}; + +ACTOR Future mergeChangeFeedStream(std::vector> interfs, + PromiseStream>> results, + Key rangeID, + Version* begin, + Version end) { + state std::priority_queue> mutations; + state std::vector> fetchers(interfs.size()); + state std::vector streams(interfs.size()); + for (int i = 0; i < interfs.size(); i++) { + fetchers[i] = + singleChangeFeedStream(interfs[i].first, streams[i].results, rangeID, *begin, end, interfs[i].second); + } + state int interfNum = 0; + while (interfNum < interfs.size()) { + try { + Standalone res = waitNext(streams[interfNum].results.getFuture()); + streams[interfNum].next = res; + mutations.push(streams[interfNum]); + } catch (Error& e) { + if (e.code() != error_code_end_of_stream) { + throw e; + } + } + interfNum++; + } + state Version checkVersion = invalidVersion; + state Standalone> nextOut; + while (mutations.size()) { + state MutationAndVersionStream nextStream = mutations.top(); + mutations.pop(); + ASSERT(nextStream.next.version >= checkVersion); + if (nextStream.next.version != checkVersion) { + if (nextOut.size()) { + *begin = checkVersion + 1; + results.send(nextOut); + nextOut = Standalone>(); + } + checkVersion = nextStream.next.version; + } + nextOut.push_back_deep(nextOut.arena(), nextStream.next); + try { + Standalone res = waitNext(nextStream.results.getFuture()); + nextStream.next = res; + mutations.push(nextStream); + } catch (Error& e) { + if (e.code() != error_code_end_of_stream) { + throw e; + } + } + } + if (nextOut.size()) { + results.send(nextOut); + } + throw end_of_stream(); +} + ACTOR Future getChangeFeedStreamActor(Reference db, PromiseStream>> results, StringRef rangeID, @@ -6609,26 +6718,27 @@ ACTOR Future getChangeFeedStreamActor(Reference db, state vector>> locations = wait(getKeyRangeLocations(cx, keys, - 100, + 1000, Reverse::False, &StorageServerInterface::changeFeed, TransactionInfo(TaskPriority::DefaultEndpoint, span.context))); - if (locations.size() > 1) { + if (locations.size() >= 1000) { results.sendError(unsupported_operation()); return Void(); } - state int useIdx = -1; - - loop { + state std::vector chosenLocations(locations.size()); + state int loc = 0; + while (loc < locations.size()) { // FIXME: create a load balance function for this code so future users of reply streams do not have // to duplicate this code int count = 0; - for (int i = 0; i < locations[0].second->size(); i++) { + int useIdx = -1; + for (int i = 0; i < locations[loc].second->size(); i++) { if (!IFailureMonitor::failureMonitor() .getState( - locations[0].second->get(i, &StorageServerInterface::changeFeedStream).getEndpoint()) + locations[loc].second->get(i, &StorageServerInterface::changeFeedStream).getEndpoint()) .failed) { if (deterministicRandom()->random01() <= 1.0 / ++count) { useIdx = i; @@ -6637,13 +6747,15 @@ ACTOR Future getChangeFeedStreamActor(Reference db, } if (useIdx >= 0) { - break; + chosenLocations[loc] = useIdx; + loc++; + continue; } - vector> ok(locations[0].second->size()); + vector> ok(locations[loc].second->size()); for (int i = 0; i < ok.size(); i++) { ok[i] = IFailureMonitor::failureMonitor().onStateEqual( - locations[0].second->get(i, &StorageServerInterface::changeFeedStream).getEndpoint(), + locations[loc].second->get(i, &StorageServerInterface::changeFeedStream).getEndpoint(), FailureStatus(false)); } @@ -6654,23 +6766,36 @@ ACTOR Future getChangeFeedStreamActor(Reference db, } wait(allAlternativesFailedDelay(quorum(ok, 1))); + loc = 0; } - state ChangeFeedStreamRequest req; - req.rangeID = rangeID; - req.begin = begin; - req.end = end; + if (locations.size() > 1) { + std::vector> interfs; + for (int i = 0; i < locations.size(); i++) { + interfs.push_back( + std::make_pair(locations[i].second->getInterface(chosenLocations[i]), locations[i].first)); + } + wait(mergeChangeFeedStream(interfs, results, rangeID, &begin, end) || cx->connectionFileChanged()); + } else { + state ChangeFeedStreamRequest req; + req.rangeID = rangeID; + req.begin = begin; + req.end = end; + req.range = range; - state ReplyPromiseStream replyStream = - locations[0].second->get(useIdx, &StorageServerInterface::changeFeedStream).getReplyStream(req); + state ReplyPromiseStream replyStream = + locations[0] + .second->get(chosenLocations[0], &StorageServerInterface::changeFeedStream) + .getReplyStream(req); - loop { - wait(results.onEmpty()); - choose { - when(wait(cx->connectionFileChanged())) { break; } - when(ChangeFeedStreamReply rep = waitNext(replyStream.getFuture())) { - begin = rep.mutations.back().version + 1; - results.send(Standalone>(rep.mutations, rep.arena)); + loop { + wait(results.onEmpty()); + choose { + when(wait(cx->connectionFileChanged())) { break; } + when(ChangeFeedStreamReply rep = waitNext(replyStream.getFuture())) { + begin = rep.mutations.back().version + 1; + results.send(Standalone>(rep.mutations, rep.arena)); + } } } } @@ -6699,6 +6824,29 @@ Future DatabaseContext::getChangeFeedStream( return getChangeFeedStreamActor(Reference::addRef(this), results, rangeID, begin, end, range); } +ACTOR Future>> singleLocationOverlappingChangeFeeds( + Database cx, + Reference location, + KeyRangeRef range, + Version minVersion) { + state OverlappingChangeFeedsRequest req; + req.range = range; + req.minVersion = minVersion; + + OverlappingChangeFeedsReply rep = wait(loadBalance(cx.getPtr(), + location, + &StorageServerInterface::overlappingChangeFeeds, + req, + TaskPriority::DefaultPromiseEndpoint, + AtMostOnce::False, + cx->enableLocalityLoadBalance ? &cx->queueModel : nullptr)); + return rep.rangeIds; +} + +bool compareChangeFeedResult(const std::pair& i, const std::pair& j) { + return i.first < j.first; +} + ACTOR Future>> getOverlappingChangeFeedsActor(Reference db, KeyRangeRef range, Version minVersion) { @@ -6708,26 +6856,28 @@ ACTOR Future>> getOverlappingChangeFeedsAct state vector>> locations = wait(getKeyRangeLocations(cx, range, - 100, + 1000, Reverse::False, &StorageServerInterface::changeFeed, TransactionInfo(TaskPriority::DefaultEndpoint, span.context))); - if (locations.size() > 1) { + if (locations.size() >= 1000) { throw unsupported_operation(); } - state OverlappingChangeFeedsRequest req; - req.range = range; + state std::vector>>> allOverlappingRequests; + for (auto& it : locations) { + allOverlappingRequests.push_back(singleLocationOverlappingChangeFeeds(cx, it.second, range, minVersion)); + } + wait(waitForAll(allOverlappingRequests)); - OverlappingChangeFeedsReply rep = wait(loadBalance(cx.getPtr(), - locations[0].second, - &StorageServerInterface::overlappingChangeFeeds, - req, - TaskPriority::DefaultPromiseEndpoint, - AtMostOnce::False, - cx->enableLocalityLoadBalance ? &cx->queueModel : nullptr)); - return rep.rangeIds; + std::vector> result; + for (auto& it : allOverlappingRequests) { + result.insert(result.end(), it.get().begin(), it.get().end()); + } + std::sort(result.begin(), result.end(), compareChangeFeedResult); + result.resize(std::unique(result.begin(), result.end()) - result.begin()); + return result; } Future>> DatabaseContext::getOverlappingChangeFeeds(KeyRangeRef range, @@ -6748,20 +6898,27 @@ ACTOR Future popChangeFeedMutationsActor(Reference db, St state vector>> locations = wait(getKeyRangeLocations(cx, keys, - 100, + 1000, Reverse::False, &StorageServerInterface::changeFeed, TransactionInfo(TaskPriority::DefaultEndpoint, span.context))); - if (locations.size() > 1) { + if (locations.size() >= 1000) { throw unsupported_operation(); } + state std::vector allInterfs; + for (auto& it : locations) { + for (int i = 0; i < it.second->size(); i++) { + allInterfs.push_back(it.second->getInterface(i)); + } + } + uniquify(allInterfs); + // FIXME: lookup both the src and dest shards as of the pop version to ensure all locations are popped state std::vector> popRequests; - for (int i = 0; i < locations[0].second->size(); i++) { - popRequests.push_back( - locations[0].second->getInterface(i).changeFeedPop.getReply(ChangeFeedPopRequest(rangeID, version))); + for (int i = 0; i < allInterfs.size(); i++) { + popRequests.push_back(allInterfs[i].changeFeedPop.getReply(ChangeFeedPopRequest(rangeID, version))); } wait(waitForAll(popRequests)); return Void(); diff --git a/fdbclient/StorageServerInterface.h b/fdbclient/StorageServerInterface.h index e11491e0b2..f644c7cee3 100644 --- a/fdbclient/StorageServerInterface.h +++ b/fdbclient/StorageServerInterface.h @@ -709,12 +709,13 @@ struct ChangeFeedStreamRequest { Key rangeID; Version begin = 0; Version end = 0; + KeyRange range; ReplyPromiseStream reply; ChangeFeedStreamRequest() {} template void serialize(Ar& ar) { - serializer(ar, rangeID, begin, end, reply, spanContext, arena); + serializer(ar, rangeID, begin, end, range, reply, spanContext, arena); } }; @@ -751,6 +752,7 @@ struct OverlappingChangeFeedsReply { struct OverlappingChangeFeedsRequest { constexpr static FileIdentifier file_identifier = 10726174; KeyRange range; + Version minVersion; ReplyPromise reply; OverlappingChangeFeedsRequest() {} @@ -758,7 +760,7 @@ struct OverlappingChangeFeedsRequest { template void serialize(Ar& ar) { - serializer(ar, range, reply); + serializer(ar, range, minVersion, reply); } }; diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index 3e8c26b527..2d22dbce57 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -1538,6 +1538,7 @@ ACTOR Future changeFeedPopQ(StorageServer* self, ChangeFeedPopRequest req) ACTOR Future overlappingChangeFeedsQ(StorageServer* data, OverlappingChangeFeedsRequest req) { wait(delay(0)); + wait(data->version.whenAtLeast(req.minVersion)); auto ranges = data->keyChangeFeed.intersectingRanges(req.range); std::map rangeIds; for (auto r : ranges) { @@ -1671,8 +1672,6 @@ ACTOR Future changeFeedQ(StorageServer* data, ChangeFeedRequest req) { } ACTOR Future changeFeedStreamQ(StorageServer* data, ChangeFeedStreamRequest req) -// Throws a wrong_shard_server if the keys in the request or result depend on data outside this server OR if a large -// selector offset prevents all data from being read in one range read { state Span span("SS:getChangeFeedStream"_loc, { req.spanContext }); state Version begin = req.begin; @@ -1680,6 +1679,11 @@ ACTOR Future changeFeedStreamQ(StorageServer* data, ChangeFeedStreamReques wait(delay(0, TaskPriority::DefaultEndpoint)); + state uint64_t changeCounter = data->shardChangeCounter; + if (!data->isReadable(req.range)) { + throw wrong_shard_server(); + } + try { loop { wait(req.reply.onReady()); @@ -1688,6 +1692,9 @@ ACTOR Future changeFeedStreamQ(StorageServer* data, ChangeFeedStreamReques feedRequest.begin = begin; feedRequest.end = req.end; ChangeFeedReply feedReply = wait(getChangeFeedMutations(data, feedRequest)); + + data->checkChangeCounter(changeCounter, req.range); + begin = feedReply.mutations.back().version + 1; req.reply.send(ChangeFeedStreamReply(feedReply)); if (feedReply.mutations.back().version == req.end - 1) { From 4bbae59bb0715819ca43fb835f510b3440df47d6 Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Thu, 2 Sep 2021 21:11:44 -0700 Subject: [PATCH 44/72] Fixed a few bugs and added a backup popping mechanism --- fdbcli/fdbcli.actor.cpp | 2 +- fdbclient/NativeAPI.actor.cpp | 118 ++++++++++++++++++---------- fdbclient/StorageServerInterface.h | 6 +- fdbclient/SystemData.cpp | 15 +++- fdbclient/SystemData.h | 4 +- fdbserver/ApplyMetadataMutation.cpp | 2 +- fdbserver/storageserver.actor.cpp | 85 ++++++++++++++------ 7 files changed, 157 insertions(+), 75 deletions(-) diff --git a/fdbcli/fdbcli.actor.cpp b/fdbcli/fdbcli.actor.cpp index 5dcd85be2c..b9dd1a0712 100644 --- a/fdbcli/fdbcli.actor.cpp +++ b/fdbcli/fdbcli.actor.cpp @@ -1983,7 +1983,7 @@ ACTOR Future changeFeedList(Database db) { ASSERT(!result.more); printf("Found %d range feeds%s\n", result.size(), result.size() == 0 ? "." : ":"); for (auto& it : result) { - auto range = decodeChangeFeedValue(it.value); + auto range = std::get<0>(decodeChangeFeedValue(it.value)); printf(" %s: %s - %s\n", it.key.removePrefix(changeFeedPrefix).toString().c_str(), range.begin.toString().c_str(), diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index 993391b237..104723f6ff 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -4319,8 +4319,8 @@ ACTOR Future registerChangeFeedActor(Transaction* tr, Key rangeID, KeyRang state Key rangeIDKey = rangeID.withPrefix(changeFeedPrefix); Optional val = wait(tr->get(rangeIDKey)); if (!val.present()) { - tr->set(rangeIDKey, changeFeedValue(range)); - } else if (decodeChangeFeedValue(val.get()) != range) { + tr->set(rangeIDKey, changeFeedValue(range, invalidVersion, false)); + } else if (std::get<0>(decodeChangeFeedValue(val.get())) != range) { throw unsupported_operation(); } return Void(); @@ -6544,7 +6544,7 @@ ACTOR Future>> getChangeFeedMutatio if (!val.present()) { throw unsupported_operation(); } - KeyRange keys = decodeChangeFeedValue(val.get()); + KeyRange keys = std::get<0>(decodeChangeFeedValue(val.get())); state vector>> locations = wait(getKeyRangeLocations(cx, keys, @@ -6706,7 +6706,7 @@ ACTOR Future getChangeFeedStreamActor(Reference db, results.sendError(unsupported_operation()); return Void(); } - keys = decodeChangeFeedValue(val.get()); + keys = std::get<0>(decodeChangeFeedValue(val.get())); break; } catch (Error& e) { wait(tr.onError(e)); @@ -6853,31 +6853,44 @@ ACTOR Future>> getOverlappingChangeFeedsAct state Database cx(db); state Transaction tr(cx); state Span span("NAPI:GetOverlappingChangeFeeds"_loc); - state vector>> locations = - wait(getKeyRangeLocations(cx, - range, - 1000, - Reverse::False, - &StorageServerInterface::changeFeed, - TransactionInfo(TaskPriority::DefaultEndpoint, span.context))); - if (locations.size() >= 1000) { - throw unsupported_operation(); - } + loop { + try { + state vector>> locations = + wait(getKeyRangeLocations(cx, + range, + 1000, + Reverse::False, + &StorageServerInterface::overlappingChangeFeeds, + TransactionInfo(TaskPriority::DefaultEndpoint, span.context))); - state std::vector>>> allOverlappingRequests; - for (auto& it : locations) { - allOverlappingRequests.push_back(singleLocationOverlappingChangeFeeds(cx, it.second, range, minVersion)); - } - wait(waitForAll(allOverlappingRequests)); + if (locations.size() >= 1000) { + throw unsupported_operation(); + } - std::vector> result; - for (auto& it : allOverlappingRequests) { - result.insert(result.end(), it.get().begin(), it.get().end()); + state std::vector>>> allOverlappingRequests; + for (auto& it : locations) { + allOverlappingRequests.push_back( + singleLocationOverlappingChangeFeeds(cx, it.second, it.first & range, minVersion)); + } + wait(waitForAll(allOverlappingRequests)); + + std::vector> result; + for (auto& it : allOverlappingRequests) { + result.insert(result.end(), it.get().begin(), it.get().end()); + } + std::sort(result.begin(), result.end(), compareChangeFeedResult); + result.resize(std::unique(result.begin(), result.end()) - result.begin()); + return result; + } catch (Error& e) { + if (e.code() == error_code_wrong_shard_server || e.code() == error_code_all_alternatives_failed) { + cx->invalidateCache(range); + wait(delay(CLIENT_KNOBS->WRONG_SHARD_SERVER_DELAY)); + } else { + throw e; + } + } } - std::sort(result.begin(), result.end(), compareChangeFeedResult); - result.resize(std::unique(result.begin(), result.end()) - result.begin()); - return result; } Future>> DatabaseContext::getOverlappingChangeFeeds(KeyRangeRef range, @@ -6885,6 +6898,31 @@ Future>> DatabaseContext::getOverlappingCha return getOverlappingChangeFeedsActor(Reference::addRef(this), range, minVersion); } +ACTOR static Future popChangeFeedBackup(Database cx, StringRef rangeID, Version version) { + state Transaction tr(cx); + loop { + try { + state Key rangeIDKey = rangeID.withPrefix(changeFeedPrefix); + Optional val = wait(tr.get(rangeIDKey)); + if (val.present()) { + KeyRange range; + Version popVersion; + bool stopped; + std::tie(range, popVersion, stopped) = decodeChangeFeedValue(val.get()); + if (version > popVersion) { + tr.set(rangeIDKey, changeFeedValue(range, invalidVersion, stopped)); + } + } else { + throw unsupported_operation(); + } + wait(tr.commit()); + return Void(); + } catch (Error& e) { + wait(tr.onError(e)); + } + } +} + ACTOR Future popChangeFeedMutationsActor(Reference db, StringRef rangeID, Version version) { state Database cx(db); state Transaction tr(cx); @@ -6894,33 +6932,33 @@ ACTOR Future popChangeFeedMutationsActor(Reference db, St if (!val.present()) { throw unsupported_operation(); } - KeyRange keys = decodeChangeFeedValue(val.get()); + KeyRange keys = std::get<0>(decodeChangeFeedValue(val.get())); state vector>> locations = wait(getKeyRangeLocations(cx, keys, - 1000, + 3, Reverse::False, &StorageServerInterface::changeFeed, TransactionInfo(TaskPriority::DefaultEndpoint, span.context))); - if (locations.size() >= 1000) { - throw unsupported_operation(); + if (locations.size() > 2) { + wait(popChangeFeedBackup(cx, rangeID, version)); + return Void(); } - state std::vector allInterfs; - for (auto& it : locations) { - for (int i = 0; i < it.second->size(); i++) { - allInterfs.push_back(it.second->getInterface(i)); - } - } - uniquify(allInterfs); - // FIXME: lookup both the src and dest shards as of the pop version to ensure all locations are popped state std::vector> popRequests; - for (int i = 0; i < allInterfs.size(); i++) { - popRequests.push_back(allInterfs[i].changeFeedPop.getReply(ChangeFeedPopRequest(rangeID, version))); + for (int i = 0; i < locations.size(); i++) { + for (int j = 0; j < locations[i].second->size(); j++) { + popRequests.push_back(locations[i].second->getInterface(j).changeFeedPop.getReply( + ChangeFeedPopRequest(rangeID, version, locations[i].first))); + } + } + + choose { + when(wait(waitForAll(popRequests))) {} + when(wait(delay(5.0))) { wait(popChangeFeedBackup(cx, rangeID, version)); } } - wait(waitForAll(popRequests)); return Void(); } diff --git a/fdbclient/StorageServerInterface.h b/fdbclient/StorageServerInterface.h index f644c7cee3..6e127b2326 100644 --- a/fdbclient/StorageServerInterface.h +++ b/fdbclient/StorageServerInterface.h @@ -723,14 +723,16 @@ struct ChangeFeedPopRequest { constexpr static FileIdentifier file_identifier = 10726174; Key rangeID; Version version; + KeyRange range; ReplyPromise reply; ChangeFeedPopRequest() {} - ChangeFeedPopRequest(Key const& rangeID, Version version) : rangeID(rangeID), version(version) {} + ChangeFeedPopRequest(Key const& rangeID, Version version, KeyRange const& range) + : rangeID(rangeID), version(version), range(range) {} template void serialize(Ar& ar) { - serializer(ar, rangeID, version, reply); + serializer(ar, rangeID, version, range, reply); } }; diff --git a/fdbclient/SystemData.cpp b/fdbclient/SystemData.cpp index 342dc9ab88..c3d5a9d467 100644 --- a/fdbclient/SystemData.cpp +++ b/fdbclient/SystemData.cpp @@ -1034,19 +1034,26 @@ const KeyRangeRef changeFeedKeys(LiteralStringRef("\xff\x02/feed/"), LiteralStri const KeyRef changeFeedPrefix = changeFeedKeys.begin; const KeyRef changeFeedPrivatePrefix = LiteralStringRef("\xff\xff\x02/feed/"); -const Value changeFeedValue(KeyRangeRef const& range) { +const Value changeFeedValue(KeyRangeRef const& range, Version popVersion, bool stopped) { BinaryWriter wr(IncludeVersion(ProtocolVersion::withChangeFeed())); wr << range; + wr << popVersion; + wr << stopped; return wr.toValue(); } -KeyRange decodeChangeFeedValue(ValueRef const& value) { + +std::tuple decodeChangeFeedValue(ValueRef const& value) { KeyRange range; + Version version; + bool stopped; BinaryReader reader(value, IncludeVersion()); reader >> range; - return range; + reader >> version; + reader >> stopped; + return std::make_tuple(range, version, stopped); } -const KeyRangeRef changeFeedDurableKeys(LiteralStringRef("\xff\xff/rf/"), LiteralStringRef("\xff\xff/rf0")); +const KeyRangeRef changeFeedDurableKeys(LiteralStringRef("\xff\xff/cf/"), LiteralStringRef("\xff\xff/cf0")); const KeyRef changeFeedDurablePrefix = changeFeedDurableKeys.begin; const Value changeFeedDurableKey(Key const& feed, Version const& version) { diff --git a/fdbclient/SystemData.h b/fdbclient/SystemData.h index 7a0b3c4dfd..c1e66eac3c 100644 --- a/fdbclient/SystemData.h +++ b/fdbclient/SystemData.h @@ -496,8 +496,8 @@ extern const ValueRef writeRecoveryKeyTrue; extern const KeyRef snapshotEndVersionKey; extern const KeyRangeRef changeFeedKeys; -const Value changeFeedValue(KeyRangeRef const& range); -KeyRange decodeChangeFeedValue(ValueRef const& value); +const Value changeFeedValue(KeyRangeRef const& range, Version popVersion, bool stopped); +std::tuple decodeChangeFeedValue(ValueRef const& value); extern const KeyRef changeFeedPrefix; extern const KeyRef changeFeedPrivatePrefix; diff --git a/fdbserver/ApplyMetadataMutation.cpp b/fdbserver/ApplyMetadataMutation.cpp index 693eaa2603..a095728004 100644 --- a/fdbserver/ApplyMetadataMutation.cpp +++ b/fdbserver/ApplyMetadataMutation.cpp @@ -410,7 +410,7 @@ void applyMetadataMutations(SpanID const& spanContext, TEST(true); // Snapshot created, setting writeRecoveryKey in txnStateStore } else if (m.param1.startsWith(changeFeedPrefix)) { if (toCommit && keyInfo) { - KeyRange r = decodeChangeFeedValue(m.param2); + KeyRange r = std::get<0>(decodeChangeFeedValue(m.param2)); MutationRef privatized = m; privatized.param1 = m.param1.withPrefix(systemKeys.begin, arena); auto ranges = keyInfo->intersectingRanges(r); diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index 2d22dbce57..4ca7042033 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -1513,11 +1513,16 @@ ACTOR Future watchValueSendReply(StorageServer* data, } ACTOR Future changeFeedPopQ(StorageServer* self, ChangeFeedPopRequest req) { + if (!self->isReadable(req.range)) { + req.reply.sendError(wrong_shard_server()); + return Void(); + } + auto& feed = self->uidChangeFeed[req.rangeID]; if (req.version - 1 > feed->emptyVersion) { feed->emptyVersion = req.version - 1; while (!feed->mutations.empty() && feed->mutations.front().version < req.version) { - self->uidChangeFeed[req.rangeID]->mutations.pop_front(); + feed->mutations.pop_front(); } if (feed->storageVersion != invalidVersion) { self->storage.clearRange( @@ -1539,6 +1544,12 @@ ACTOR Future changeFeedPopQ(StorageServer* self, ChangeFeedPopRequest req) ACTOR Future overlappingChangeFeedsQ(StorageServer* data, OverlappingChangeFeedsRequest req) { wait(delay(0)); wait(data->version.whenAtLeast(req.minVersion)); + + if (!data->isReadable(req.range)) { + req.reply.sendError(wrong_shard_server()); + return Void(); + } + auto ranges = data->keyChangeFeed.intersectingRanges(req.range); std::map rangeIds; for (auto r : ranges) { @@ -1680,11 +1691,12 @@ ACTOR Future changeFeedStreamQ(StorageServer* data, ChangeFeedStreamReques wait(delay(0, TaskPriority::DefaultEndpoint)); state uint64_t changeCounter = data->shardChangeCounter; - if (!data->isReadable(req.range)) { - throw wrong_shard_server(); - } try { + if (!data->isReadable(req.range)) { + throw wrong_shard_server(); + } + loop { wait(req.reply.onReady()); state ChangeFeedRequest feedRequest; @@ -3120,7 +3132,7 @@ ACTOR Future fetchChangeFeed(StorageServer* data, Key rangeId, KeyRange ra data->addMutationToMutationLog(mLV, MutationRef(MutationRef::SetValue, persistChangeFeedKeys.begin.toString() + rangeId.toString(), - changeFeedValue(range))); + changeFeedValue(range, invalidVersion, false))); } else { changeFeedInfo = data->uidChangeFeed[rangeId]; } @@ -3946,26 +3958,49 @@ private: m.param1.startsWith(changeFeedPrivatePrefix)) { if (m.type == MutationRef::SetValue) { Key changeFeedId = m.param1.removePrefix(changeFeedPrivatePrefix); - KeyRange changeFeedRange = decodeChangeFeedValue(m.param2); - TraceEvent("AddingChangeFeed", data->thisServerID) - .detail("RangeID", changeFeedId.printable()) - .detail("Range", changeFeedRange.toString()); - Reference changeFeedInfo(new ChangeFeedInfo()); - changeFeedInfo->range = changeFeedRange; - changeFeedInfo->id = changeFeedId; - changeFeedInfo->emptyVersion = currentVersion - 1; - data->uidChangeFeed[changeFeedId] = changeFeedInfo; - auto rs = data->keyChangeFeed.modify(changeFeedRange); - for (auto r = rs.begin(); r != rs.end(); ++r) { - r->value().push_back(changeFeedInfo); + KeyRange changeFeedRange; + Version popVersion; + bool stopped; + std::tie(changeFeedRange, popVersion, stopped) = decodeChangeFeedValue(m.param2); + auto feed = data->uidChangeFeed.find(changeFeedId); + if (feed == data->uidChangeFeed.end()) { + TraceEvent("AddingChangeFeed", data->thisServerID) + .detail("RangeID", changeFeedId.printable()) + .detail("Range", changeFeedRange.toString()); + Reference changeFeedInfo(new ChangeFeedInfo()); + changeFeedInfo->range = changeFeedRange; + changeFeedInfo->id = changeFeedId; + changeFeedInfo->emptyVersion = currentVersion - 1; + data->uidChangeFeed[changeFeedId] = changeFeedInfo; + + auto rs = data->keyChangeFeed.modify(changeFeedRange); + for (auto r = rs.begin(); r != rs.end(); ++r) { + r->value().push_back(changeFeedInfo); + } + data->keyChangeFeed.coalesce(changeFeedRange.contents()); + auto& mLV = data->addVersionToMutationLog(data->data().getLatestVersion()); + data->addMutationToMutationLog( + mLV, + MutationRef(MutationRef::SetValue, + persistChangeFeedKeys.begin.toString() + changeFeedId.toString(), + m.param2)); + } else { + if (popVersion != invalidVersion && popVersion - 1 > feed->second->emptyVersion) { + feed->second->emptyVersion = popVersion - 1; + while (!feed->second->mutations.empty() && + feed->second->mutations.front().version < popVersion) { + feed->second->mutations.pop_front(); + } + if (feed->second->storageVersion != invalidVersion) { + data->storage.clearRange(KeyRangeRef(changeFeedDurableKey(feed->second->id, 0), + changeFeedDurableKey(feed->second->id, popVersion))); + if (popVersion > feed->second->storageVersion) { + feed->second->storageVersion = invalidVersion; + feed->second->durableVersion = invalidVersion; + } + } + } } - data->keyChangeFeed.coalesce(changeFeedRange.contents()); - auto& mLV = data->addVersionToMutationLog(data->data().getLatestVersion()); - data->addMutationToMutationLog( - mLV, - MutationRef(MutationRef::SetValue, - persistChangeFeedKeys.begin.toString() + changeFeedId.toString(), - m.param2)); } else { auto& mLV = data->addVersionToMutationLog(data->data().getLatestVersion()); auto beginFeed = m.param1.removePrefix(changeFeedPrivatePrefix); @@ -4916,7 +4951,7 @@ ACTOR Future restoreDurableState(StorageServer* data, IKeyValueStore* stor state int feedLoc; for (feedLoc = 0; feedLoc < changeFeeds.size(); feedLoc++) { Key changeFeedId = changeFeeds[feedLoc].key.removePrefix(persistChangeFeedKeys.begin); - KeyRange changeFeedRange = decodeChangeFeedValue(changeFeeds[feedLoc].value); + KeyRange changeFeedRange = std::get<0>(decodeChangeFeedValue(changeFeeds[feedLoc].value)); TraceEvent("RestoringChangeFeed", data->thisServerID) .detail("RangeID", changeFeedId.printable()) .detail("Range", changeFeedRange.toString()); From 8309c2ce267b3a5a2882e9124e28d5a515ece36e Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Fri, 3 Sep 2021 12:06:13 -0700 Subject: [PATCH 45/72] filter mutations in change feeds to the requested range --- fdbclient/NativeAPI.actor.cpp | 1 + fdbclient/StorageServerInterface.h | 3 +- fdbserver/CMakeLists.txt | 1 + fdbserver/storageserver.actor.cpp | 71 ++++++++++++++++++---- fdbserver/workloads/ChangeFeeds.actor.cpp | 72 +++++++++++++++++++++++ 5 files changed, 134 insertions(+), 14 deletions(-) create mode 100644 fdbserver/workloads/ChangeFeeds.actor.cpp diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index 104723f6ff..5d5e8ad9fc 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -6561,6 +6561,7 @@ ACTOR Future>> getChangeFeedMutatio req.rangeID = rangeID; req.begin = begin; req.end = end; + req.range = keys; ChangeFeedReply rep = wait(loadBalance(cx.getPtr(), locations[0].second, diff --git a/fdbclient/StorageServerInterface.h b/fdbclient/StorageServerInterface.h index 6e127b2326..1365903c3d 100644 --- a/fdbclient/StorageServerInterface.h +++ b/fdbclient/StorageServerInterface.h @@ -675,6 +675,7 @@ struct ChangeFeedRequest { Key rangeID; Version begin = 0; Version end = 0; + KeyRange range; ReplyPromise reply; ChangeFeedRequest() {} @@ -682,7 +683,7 @@ struct ChangeFeedRequest { template void serialize(Ar& ar) { - serializer(ar, rangeID, begin, end, reply); + serializer(ar, rangeID, begin, end, range, reply); } }; diff --git a/fdbserver/CMakeLists.txt b/fdbserver/CMakeLists.txt index c4234f84b4..6319ad646a 100644 --- a/fdbserver/CMakeLists.txt +++ b/fdbserver/CMakeLists.txt @@ -162,6 +162,7 @@ set(FDBSERVER_SRCS workloads/ConsistencyCheck.actor.cpp workloads/CpuProfiler.actor.cpp workloads/Cycle.actor.cpp + workloads/ChangeFeeds.actor.cpp workloads/DataDistributionMetrics.actor.cpp workloads/DDBalance.actor.cpp workloads/DDMetrics.actor.cpp diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index 4ca7042033..b04d245438 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -1566,6 +1566,41 @@ ACTOR Future overlappingChangeFeedsQ(StorageServer* data, OverlappingChang return Void(); } +void filterMutations(Arena& arena, VectorRef& mutations, KeyRange const& range) { + Optional> modifiedMutations; + for (int i = 0; i < mutations.size(); i++) { + if (mutations[i].type == MutationRef::SetValue) { + if (modifiedMutations.present() && range.contains(mutations[i].param1)) { + modifiedMutations.get().push_back(arena, mutations[i]); + } + if (!modifiedMutations.present() && !range.contains(mutations[i].param1)) { + modifiedMutations = mutations; + modifiedMutations.get().resize(arena, i); + arena.dependsOn(range.arena()); + } + } else { + ASSERT(mutations[i].type == MutationRef::ClearRange); + if (!modifiedMutations.present() && + (mutations[i].param1 < range.begin || mutations[i].param2 > range.end)) { + modifiedMutations = mutations; + modifiedMutations.get().resize(arena, i); + arena.dependsOn(range.arena()); + } + if (modifiedMutations.present()) { + if (mutations[i].param1 < range.end && range.begin < mutations[i].param2) { + modifiedMutations.get().push_back(arena, + MutationRef(MutationRef::ClearRange, + std::max(range.begin, mutations[i].param1), + std::min(range.end, mutations[i].param2))); + } + } + } + } + if (modifiedMutations.present()) { + mutations = modifiedMutations.get(); + } +} + ACTOR Future getChangeFeedMutations(StorageServer* data, ChangeFeedRequest req) { state ChangeFeedReply reply; state int remainingLimitBytes = CLIENT_KNOBS->REPLY_BYTE_LIMIT; @@ -1573,14 +1608,20 @@ ACTOR Future getChangeFeedMutations(StorageServer* data, Change if (data->version.get() < req.begin) { wait(data->version.whenAtLeast(req.begin)); } + state uint64_t changeCounter = data->shardChangeCounter; + if (!data->isReadable(req.range)) { + throw wrong_shard_server(); + } auto& feedInfo = data->uidChangeFeed[req.rangeID]; if (req.end <= feedInfo->emptyVersion + 1) { } else if (feedInfo->durableVersion == invalidVersion || req.begin > feedInfo->durableVersion) { - for (auto& it : data->uidChangeFeed[req.rangeID]->mutations) { + for (auto it : data->uidChangeFeed[req.rangeID]->mutations) { if (it.version >= req.end || remainingLimitBytes <= 0) { break; } if (it.version >= req.begin) { + reply.arena.dependsOn(it.arena()); + filterMutations(reply.arena, it.mutations, req.range); reply.mutations.push_back(reply.arena, it); remainingLimitBytes -= sizeof(MutationsAndVersionRef) + it.expectedSize(); } @@ -1594,13 +1635,18 @@ ACTOR Future getChangeFeedMutations(StorageServer* data, Change 1 << 30, remainingLimitBytes)); + data->checkChangeCounter(changeCounter, req.range); + + bool isEmpty = res.size() == 0; Version lastVersion = req.begin - 1; for (auto& kv : res) { Key id; Version version; std::tie(id, version) = decodeChangeFeedDurableKey(kv.key); auto mutations = decodeChangeFeedDurableValue(kv.value); - reply.mutations.push_back_deep(reply.arena, MutationsAndVersionRef(mutations, version)); + reply.arena.dependsOn(mutations.arena()); + filterMutations(reply.arena, mutations, req.range); + reply.mutations.push_back(reply.arena, MutationsAndVersionRef(mutations, version)); remainingLimitBytes -= sizeof(KeyValueRef) + kv.expectedSize(); // FIXME: this is currently tracking the size on disk rather than the reply size @@ -1612,11 +1658,16 @@ ACTOR Future getChangeFeedMutations(StorageServer* data, Change break; } if (it.version > lastVersion) { + if (it.mutations.size()) { + isEmpty = false; + } + reply.arena.dependsOn(it.arena()); + filterMutations(reply.arena, it.mutations, req.range); reply.mutations.push_back(reply.arena, it); remainingLimitBytes -= sizeof(MutationsAndVersionRef) + it.expectedSize(); } } - if (reply.mutations.empty()) { + if (isEmpty) { auto& feedInfo = data->uidChangeFeed[req.rangeID]; if (startingDurableVersion == feedInfo->storageVersion && req.end > startingDurableVersion) { if (req.begin == 0) { @@ -1625,7 +1676,7 @@ ACTOR Future getChangeFeedMutations(StorageServer* data, Change RangeResult emp = wait(data->storage.readRange( KeyRangeRef(changeFeedDurableKey(req.rangeID, 0), changeFeedDurableKey(req.rangeID, req.end)), -1)); - + data->checkChangeCounter(changeCounter, req.range); auto& feedInfo = data->uidChangeFeed[req.rangeID]; if (startingDurableVersion == feedInfo->storageVersion) { if (emp.empty()) { @@ -1659,6 +1710,7 @@ ACTOR Future localChangeFeedStream(StorageServer* data, feedRequest.rangeID = rangeID; feedRequest.begin = begin; feedRequest.end = end; + feedRequest.range = range; state ChangeFeedReply feedReply = wait(getChangeFeedMutations(data, feedRequest)); begin = feedReply.mutations.back().version + 1; state int resultLoc = 0; @@ -1690,23 +1742,16 @@ ACTOR Future changeFeedStreamQ(StorageServer* data, ChangeFeedStreamReques wait(delay(0, TaskPriority::DefaultEndpoint)); - state uint64_t changeCounter = data->shardChangeCounter; - try { - if (!data->isReadable(req.range)) { - throw wrong_shard_server(); - } - loop { wait(req.reply.onReady()); state ChangeFeedRequest feedRequest; feedRequest.rangeID = req.rangeID; feedRequest.begin = begin; feedRequest.end = req.end; + feedRequest.range = req.range; ChangeFeedReply feedReply = wait(getChangeFeedMutations(data, feedRequest)); - data->checkChangeCounter(changeCounter, req.range); - begin = feedReply.mutations.back().version + 1; req.reply.send(ChangeFeedStreamReply(feedReply)); if (feedReply.mutations.back().version == req.end - 1) { @@ -3464,7 +3509,7 @@ ACTOR Future fetchKeys(StorageServer* data, AddingShard* shard) { // we have written) state Future fetchDurable = data->durableVersion.whenAtLeast(data->storageVersion() + 1); - // wait(dispatchChangeFeeds(data, fetchKeysID, keys, fetchVersion)); + wait(dispatchChangeFeeds(data, fetchKeysID, keys, fetchVersion)); holdingFKPL.release(); wait(fetchDurable); diff --git a/fdbserver/workloads/ChangeFeeds.actor.cpp b/fdbserver/workloads/ChangeFeeds.actor.cpp new file mode 100644 index 0000000000..757a5723f7 --- /dev/null +++ b/fdbserver/workloads/ChangeFeeds.actor.cpp @@ -0,0 +1,72 @@ +/* + * ChangeFeeds.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/FDBOptions.g.h" +#include "fdbclient/NativeAPI.actor.h" +#include "fdbserver/TesterInterface.actor.h" +#include "fdbserver/workloads/workloads.actor.h" +#include "fdbserver/workloads/BulkSetup.actor.h" +#include "flow/Arena.h" +#include "flow/IRandom.h" +#include "flow/Trace.h" +#include "flow/actorcompiler.h" // This must be the last #include. +#include "flow/serialize.h" +#include + +struct ChangeFeedsWorkload : TestWorkload { + double testDuration; + + ChangeFeedsWorkload(WorkloadContext const& wcx) : TestWorkload(wcx) { + testDuration = getOption(options, "testDuration"_sr, 10.0); + } + + std::string description() const override { return "ChangeFeedsWorkload"; } + Future setup(Database const& cx) override { return Void(); } + Future start(Database const& cx) override { return Void(); /*changeFeedClient(cx->clone(), this, testDuration);*/ } + Future check(Database const& cx) override { return true; } + void getMetrics(vector& m) override {} + + /* + ACTOR Future changeFeedClient(Database cx, ChangeFeedsWorkload* self, double duration) { + // Enable change feed for a key range + state UID rangeUID = deterministicRandom()->randomUniqueID(); + state Key rangeID = StringRef(rangeUID.toString()); + state Key logPath = LiteralStringRef("\xff\x02/cftest/"); + state Transaction tr(cx); + loop { + try { + wait(tr.registerChangeFeed(rangeID, normalKeys)); + wait(tr.commit()); + break; + } catch (Error& e) { + wait(tr.onError(e)); + } + } + + // Periodically read from both and compare results (merging clears) + state PromiseStream>> results; + cx->getChangeFeedStream(results, rangeID); + + // Pop from both + return Void(); + }*/ +}; + +WorkloadFactory ChangeFeedsWorkloadFactory("ChangeFeeds"); From b02e8d99c64b0801c5a7f69e07d3b66a957eff03 Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Fri, 3 Sep 2021 14:19:27 -0700 Subject: [PATCH 46/72] added a workload to test change feeds --- fdbclient/NativeAPI.actor.cpp | 8 +- fdbserver/workloads/ChangeFeeds.actor.cpp | 153 ++++++++++++++++++++-- tests/CMakeLists.txt | 1 + tests/fast/ChangeFeeds.toml | 10 ++ 4 files changed, 154 insertions(+), 18 deletions(-) create mode 100644 tests/fast/ChangeFeeds.toml diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index 5d5e8ad9fc..65adddccd5 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -6544,7 +6544,7 @@ ACTOR Future>> getChangeFeedMutatio if (!val.present()) { throw unsupported_operation(); } - KeyRange keys = std::get<0>(decodeChangeFeedValue(val.get())); + state KeyRange keys = std::get<0>(decodeChangeFeedValue(val.get())) & range; state vector>> locations = wait(getKeyRangeLocations(cx, keys, @@ -6707,7 +6707,7 @@ ACTOR Future getChangeFeedStreamActor(Reference db, results.sendError(unsupported_operation()); return Void(); } - keys = std::get<0>(decodeChangeFeedValue(val.get())); + keys = std::get<0>(decodeChangeFeedValue(val.get())) & range; break; } catch (Error& e) { wait(tr.onError(e)); @@ -6773,8 +6773,8 @@ ACTOR Future getChangeFeedStreamActor(Reference db, if (locations.size() > 1) { std::vector> interfs; for (int i = 0; i < locations.size(); i++) { - interfs.push_back( - std::make_pair(locations[i].second->getInterface(chosenLocations[i]), locations[i].first)); + interfs.push_back(std::make_pair(locations[i].second->getInterface(chosenLocations[i]), + locations[i].first & range)); } wait(mergeChangeFeedStream(interfs, results, rangeID, &begin, end) || cx->connectionFileChanged()); } else { diff --git a/fdbserver/workloads/ChangeFeeds.actor.cpp b/fdbserver/workloads/ChangeFeeds.actor.cpp index 757a5723f7..ae2ea664e1 100644 --- a/fdbserver/workloads/ChangeFeeds.actor.cpp +++ b/fdbserver/workloads/ChangeFeeds.actor.cpp @@ -30,8 +30,103 @@ #include "flow/serialize.h" #include +ACTOR Future>, Version>> readDatabase(Database cx) { + state Transaction tr(cx); + loop { + state Standalone> output; + state Version readVersion; + try { + Version ver = wait(tr.getReadVersion()); + readVersion = ver; + + state PromiseStream> results; + state Future stream = tr.getRangeStream(results, normalKeys, 1e6); + + loop { + Standalone res = waitNext(results.getFuture()); + output.append(output.arena(), res.begin(), res.size()); + } + } catch (Error& e) { + if (e.code() == error_code_end_of_stream) { + return std::make_pair(output, readVersion); + } + wait(tr.onError(e)); + } + } +} + +ACTOR Future>> readMutations(Database cx, + Key rangeID, + Version begin, + Version end) { + state Standalone> output; + + loop { + try { + state PromiseStream>> results; + state Future stream = cx->getChangeFeedStream(results, rangeID, begin, end, normalKeys); + loop { + Standalone> res = waitNext(results.getFuture()); + output.arena().dependsOn(res.arena()); + output.append(output.arena(), res.begin(), res.size()); + begin = res.back().version + 1; + } + } catch (Error& e) { + if (e.code() == error_code_end_of_stream) { + return output; + } + throw; + } + } +} + +Standalone> advanceData(Standalone> source, + Standalone> mutations) { + std::map data; + for (auto& kv : source) { + data[kv.key] = kv.value; + } + for (auto& it : mutations) { + for (auto& m : it.mutations) { + if (m.type == MutationRef::SetValue) { + data[m.param1] = m.param2; + } else { + ASSERT(m.type == MutationRef::ClearRange); + data.erase(data.lower_bound(m.param1), data.lower_bound(m.param2)); + } + } + } + Standalone> output; + output.arena().dependsOn(source.arena()); + output.arena().dependsOn(mutations.arena()); + for (auto& kv : data) { + output.push_back(output.arena(), KeyValueRef(kv.first, kv.second)); + } + return output; +} + +bool compareData(Standalone> source, Standalone> dest) { + if (source.size() != dest.size()) { + TraceEvent(SevError, "ChangeFeedSizeMismatch").detail("SrcSize", source.size()).detail("DestSize", dest.size()); + return false; + } + for (int i = 0; i < source.size(); i++) { + if (source[i] != dest[i]) { + TraceEvent("ChangeFeedMutationMismatch") + .detail("Index", i) + .detail("SrcKey", source[i].key) + .detail("DestKey", dest[i].key) + .detail("SrcValue", source[i].value) + .detail("DestValue", dest[i].value); + return false; + } + } + return true; +} + struct ChangeFeedsWorkload : TestWorkload { double testDuration; + Future client; ChangeFeedsWorkload(WorkloadContext const& wcx) : TestWorkload(wcx) { testDuration = getOption(options, "testDuration"_sr, 10.0); @@ -39,16 +134,19 @@ struct ChangeFeedsWorkload : TestWorkload { std::string description() const override { return "ChangeFeedsWorkload"; } Future setup(Database const& cx) override { return Void(); } - Future start(Database const& cx) override { return Void(); /*changeFeedClient(cx->clone(), this, testDuration);*/ } - Future check(Database const& cx) override { return true; } + Future start(Database const& cx) override { + client = changeFeedClient(cx->clone(), this); + return delay(testDuration); + } + Future check(Database const& cx) override { + client = Future(); + return true; + } void getMetrics(vector& m) override {} - /* - ACTOR Future changeFeedClient(Database cx, ChangeFeedsWorkload* self, double duration) { - // Enable change feed for a key range - state UID rangeUID = deterministicRandom()->randomUniqueID(); - state Key rangeID = StringRef(rangeUID.toString()); - state Key logPath = LiteralStringRef("\xff\x02/cftest/"); + ACTOR Future changeFeedClient(Database cx, ChangeFeedsWorkload* self) { + // Enable change feeds for a key range + state Key rangeID = StringRef(deterministicRandom()->randomUniqueID().toString()); state Transaction tr(cx); loop { try { @@ -60,13 +158,40 @@ struct ChangeFeedsWorkload : TestWorkload { } } - // Periodically read from both and compare results (merging clears) - state PromiseStream>> results; - cx->getChangeFeedStream(results, rangeID); + loop { + wait(delay(deterministicRandom()->random01())); - // Pop from both - return Void(); - }*/ + state std::pair>, Version> firstResults = wait(readDatabase(cx)); + + wait(delay(10 * deterministicRandom()->random01())); + + state std::pair>, Version> secondResults = wait(readDatabase(cx)); + state Standalone> mutations = + wait(readMutations(cx, rangeID, firstResults.second, secondResults.second)); + + Standalone> advancedResults = advanceData(firstResults.first, mutations); + + if (!compareData(secondResults.first, advancedResults)) { + TraceEvent(SevError, "ChangeFeedMismatch") + .detail("FirstVersion", firstResults.second) + .detail("SecondVersion", secondResults.second); + for (int i = 0; i < secondResults.first.size(); i++) { + TraceEvent("ChangeFeedBase") + .detail("Index", i) + .detail("K", secondResults.first[i].key) + .detail("V", secondResults.first[i].value); + } + for (int i = 0; i < advancedResults.size(); i++) { + TraceEvent("ChangeFeedAdvanced") + .detail("Index", i) + .detail("K", advancedResults[i].key) + .detail("V", advancedResults[i].value); + } + } + + wait(cx->popChangeFeedMutations(rangeID, secondResults.second)); + } + } }; WorkloadFactory ChangeFeedsWorkloadFactory("ChangeFeeds"); diff --git a/tests/CMakeLists.txt b/tests/CMakeLists.txt index 996d4d6322..d764e5c00b 100644 --- a/tests/CMakeLists.txt +++ b/tests/CMakeLists.txt @@ -125,6 +125,7 @@ if(WITH_PYTHON) add_fdb_test(TEST_FILES fast/ConstrainedRandomSelector.toml) add_fdb_test(TEST_FILES fast/CycleAndLock.toml) add_fdb_test(TEST_FILES fast/CycleTest.toml) + add_fdb_test(TEST_FILES fast/ChangeFeeds.toml) add_fdb_test(TEST_FILES fast/FuzzApiCorrectness.toml) add_fdb_test(TEST_FILES fast/FuzzApiCorrectnessClean.toml) add_fdb_test(TEST_FILES fast/IncrementalBackup.toml) diff --git a/tests/fast/ChangeFeeds.toml b/tests/fast/ChangeFeeds.toml new file mode 100644 index 0000000000..24635b03ec --- /dev/null +++ b/tests/fast/ChangeFeeds.toml @@ -0,0 +1,10 @@ +[[test]] + [[test.workload]] + testName = 'Cycle' + transactionsPerSecond = 2500.0 + testDuration = 60.0 + expectedRate = 0 + + [[test.workload]] + testName = 'ChangeFeeds' + testDuration = 60.0 From 367c7cda926c60f4dc5d87a5965edb4ece6fcdb2 Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Fri, 3 Sep 2021 15:14:28 -0700 Subject: [PATCH 47/72] fixed problems with the change feed workload --- fdbserver/workloads/ChangeFeeds.actor.cpp | 25 +++++++++++++++++------ tests/fast/ChangeFeeds.toml | 4 +++- 2 files changed, 22 insertions(+), 7 deletions(-) diff --git a/fdbserver/workloads/ChangeFeeds.actor.cpp b/fdbserver/workloads/ChangeFeeds.actor.cpp index ae2ea664e1..b75e728e83 100644 --- a/fdbserver/workloads/ChangeFeeds.actor.cpp +++ b/fdbserver/workloads/ChangeFeeds.actor.cpp @@ -40,10 +40,11 @@ ACTOR Future>, Version>> readDatabas readVersion = ver; state PromiseStream> results; - state Future stream = tr.getRangeStream(results, normalKeys, 1e6); + state Future stream = tr.getRangeStream(results, normalKeys, GetRangeLimits()); loop { Standalone res = waitNext(results.getFuture()); + output.arena().dependsOn(res.arena()); output.append(output.arena(), res.begin(), res.size()); } } catch (Error& e) { @@ -60,7 +61,6 @@ ACTOR Future>> readMutations(Databa Version begin, Version end) { state Standalone> output; - loop { try { state PromiseStream>> results; @@ -82,16 +82,29 @@ ACTOR Future>> readMutations(Databa Standalone> advanceData(Standalone> source, Standalone> mutations) { + StringRef dbgKey = LiteralStringRef(""); std::map data; for (auto& kv : source) { + if (kv.key == dbgKey) + TraceEvent("ChangeFeedDbgStart").detail("K", kv.key).detail("V", kv.value); data[kv.key] = kv.value; } for (auto& it : mutations) { for (auto& m : it.mutations) { if (m.type == MutationRef::SetValue) { + if (m.param1 == dbgKey) + TraceEvent("ChangeFeedDbgSet") + .detail("Ver", it.version) + .detail("K", m.param1) + .detail("V", m.param2); data[m.param1] = m.param2; } else { ASSERT(m.type == MutationRef::ClearRange); + if (KeyRangeRef(m.param1, m.param2).contains(dbgKey)) + TraceEvent("ChangeFeedDbgClear") + .detail("Ver", it.version) + .detail("Begin", m.param1) + .detail("End", m.param2); data.erase(data.lower_bound(m.param1), data.lower_bound(m.param2)); } } @@ -108,9 +121,8 @@ Standalone> advanceData(Standalone bool compareData(Standalone> source, Standalone> dest) { if (source.size() != dest.size()) { TraceEvent(SevError, "ChangeFeedSizeMismatch").detail("SrcSize", source.size()).detail("DestSize", dest.size()); - return false; } - for (int i = 0; i < source.size(); i++) { + for (int i = 0; i < std::min(source.size(), dest.size()); i++) { if (source[i] != dest[i]) { TraceEvent("ChangeFeedMutationMismatch") .detail("Index", i) @@ -121,7 +133,7 @@ bool compareData(Standalone> source, Standalone>, Version> secondResults = wait(readDatabase(cx)); state Standalone> mutations = - wait(readMutations(cx, rangeID, firstResults.second, secondResults.second)); + wait(readMutations(cx, rangeID, firstResults.second, secondResults.second + 1)); Standalone> advancedResults = advanceData(firstResults.first, mutations); @@ -187,6 +199,7 @@ struct ChangeFeedsWorkload : TestWorkload { .detail("K", advancedResults[i].key) .detail("V", advancedResults[i].value); } + ASSERT(false); } wait(cx->popChangeFeedMutations(rangeID, secondResults.second)); diff --git a/tests/fast/ChangeFeeds.toml b/tests/fast/ChangeFeeds.toml index 24635b03ec..5cd26d1883 100644 --- a/tests/fast/ChangeFeeds.toml +++ b/tests/fast/ChangeFeeds.toml @@ -1,7 +1,9 @@ [[test]] +testTitle = 'ChangeFeed' + [[test.workload]] testName = 'Cycle' - transactionsPerSecond = 2500.0 + transactionsPerSecond = 250.0 testDuration = 60.0 expectedRate = 0 From af0792b1f5f3748805c410d9c31a45e970762889 Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Fri, 3 Sep 2021 17:24:33 -0700 Subject: [PATCH 48/72] fixes that allow change feeds to pass correctness in tests with no failures --- fdbclient/NativeAPI.actor.cpp | 60 ++++++----- fdbserver/storageserver.actor.cpp | 162 ++++++++++++++++++------------ flow/error_definitions.h | 1 + 3 files changed, 133 insertions(+), 90 deletions(-) diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index 65adddccd5..bdb91e902d 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -6536,6 +6536,7 @@ ACTOR Future>> getChangeFeedMutatio Version begin, Version end, KeyRange range) { + // FIXME: this function is out of date! state Database cx(db); state Transaction tr(cx); state Key rangeIDKey = rangeID.withPrefix(changeFeedPrefix); @@ -6612,16 +6613,13 @@ ACTOR Future singleChangeFeedStream(StorageServerInterface interf, } } } catch (Error& e) { - if (e.code() == error_code_actor_cancelled) { + if (e.code() == error_code_wrong_shard_server || e.code() == error_code_all_alternatives_failed || + e.code() == error_code_connection_failed || e.code() == error_code_unknown_change_feed || + e.code() == error_code_actor_cancelled) { throw; } - if (e.code() == error_code_wrong_shard_server || e.code() == error_code_all_alternatives_failed || - e.code() == error_code_connection_failed) { - wait(delay(CLIENT_KNOBS->WRONG_SHARD_SERVER_DELAY)); - } else { - results.sendError(e); - return Void(); - } + results.sendError(e); + return Void(); } } } @@ -6700,21 +6698,22 @@ ACTOR Future getChangeFeedStreamActor(Reference db, state Key rangeIDKey = rangeID.withPrefix(changeFeedPrefix); state Span span("NAPI:GetChangeFeedStream"_loc); state KeyRange keys; - loop { - try { - Optional val = wait(tr.get(rangeIDKey)); - if (!val.present()) { - results.sendError(unsupported_operation()); - return Void(); - } - keys = std::get<0>(decodeChangeFeedValue(val.get())) & range; - break; - } catch (Error& e) { - wait(tr.onError(e)); - } - } loop { + loop { + try { + Optional val = wait(tr.get(rangeIDKey)); + if (!val.present()) { + results.sendError(unsupported_operation()); + return Void(); + } + keys = std::get<0>(decodeChangeFeedValue(val.get())) & range; + break; + } catch (Error& e) { + wait(tr.onError(e)); + } + } + try { state vector>> locations = wait(getKeyRangeLocations(cx, @@ -6805,7 +6804,7 @@ ACTOR Future getChangeFeedStreamActor(Reference db, throw; } if (e.code() == error_code_wrong_shard_server || e.code() == error_code_all_alternatives_failed || - e.code() == error_code_connection_failed) { + e.code() == error_code_connection_failed || e.code() == error_code_unknown_change_feed) { cx->invalidateCache(keys); wait(delay(CLIENT_KNOBS->WRONG_SHARD_SERVER_DELAY)); } else { @@ -6933,7 +6932,7 @@ ACTOR Future popChangeFeedMutationsActor(Reference db, St if (!val.present()) { throw unsupported_operation(); } - KeyRange keys = std::get<0>(decodeChangeFeedValue(val.get())); + state KeyRange keys = std::get<0>(decodeChangeFeedValue(val.get())); state vector>> locations = wait(getKeyRangeLocations(cx, keys, @@ -6956,9 +6955,18 @@ ACTOR Future popChangeFeedMutationsActor(Reference db, St } } - choose { - when(wait(waitForAll(popRequests))) {} - when(wait(delay(5.0))) { wait(popChangeFeedBackup(cx, rangeID, version)); } + try { + choose { + when(wait(waitForAll(popRequests))) {} + when(wait(delay(5.0))) { wait(popChangeFeedBackup(cx, rangeID, version)); } + } + } catch (Error& e) { + if (e.code() != error_code_unknown_change_feed && e.code() != error_code_wrong_shard_server && + e.code() != error_code_all_alternatives_failed) { + throw; + } + cx->invalidateCache(keys); + wait(popChangeFeedBackup(cx, rangeID, version)); } return Void(); } diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index b04d245438..7a6aaea47b 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -81,6 +81,7 @@ bool canReplyWith(Error e) { case error_code_wrong_shard_server: case error_code_process_behind: case error_code_watch_cancelled: + case error_code_unknown_change_feed: // case error_code_all_alternatives_failed: return true; default: @@ -1513,30 +1514,37 @@ ACTOR Future watchValueSendReply(StorageServer* data, } ACTOR Future changeFeedPopQ(StorageServer* self, ChangeFeedPopRequest req) { + wait(delay(0)); + + TraceEvent("ChangeFeedPopQuery", self->thisServerID) + .detail("RangeID", req.rangeID.printable()) + .detail("Version", req.version) + .detail("Range", req.range.toString()); + if (!self->isReadable(req.range)) { req.reply.sendError(wrong_shard_server()); return Void(); } - - auto& feed = self->uidChangeFeed[req.rangeID]; - if (req.version - 1 > feed->emptyVersion) { - feed->emptyVersion = req.version - 1; - while (!feed->mutations.empty() && feed->mutations.front().version < req.version) { - feed->mutations.pop_front(); + auto feed = self->uidChangeFeed.find(req.rangeID); + if (feed == self->uidChangeFeed.end()) { + req.reply.sendError(unknown_change_feed()); + return Void(); + } + if (req.version - 1 > feed->second->emptyVersion) { + feed->second->emptyVersion = req.version - 1; + while (!feed->second->mutations.empty() && feed->second->mutations.front().version < req.version) { + feed->second->mutations.pop_front(); } - if (feed->storageVersion != invalidVersion) { - self->storage.clearRange( - KeyRangeRef(changeFeedDurableKey(feed->id, 0), changeFeedDurableKey(feed->id, req.version))); - if (req.version > feed->storageVersion) { - feed->storageVersion = invalidVersion; - feed->durableVersion = invalidVersion; + if (feed->second->storageVersion != invalidVersion) { + self->storage.clearRange(KeyRangeRef(changeFeedDurableKey(feed->second->id, 0), + changeFeedDurableKey(feed->second->id, req.version))); + if (req.version > feed->second->storageVersion) { + feed->second->storageVersion = invalidVersion; + feed->second->durableVersion = invalidVersion; } wait(self->durableVersion.whenAtLeast(self->storageVersion() + 1)); } } - TraceEvent("ChangeFeedPopQuery", self->thisServerID) - .detail("RangeID", req.rangeID.printable()) - .detail("Version", req.version); req.reply.send(Void()); return Void(); } @@ -1612,10 +1620,14 @@ ACTOR Future getChangeFeedMutations(StorageServer* data, Change if (!data->isReadable(req.range)) { throw wrong_shard_server(); } - auto& feedInfo = data->uidChangeFeed[req.rangeID]; - if (req.end <= feedInfo->emptyVersion + 1) { - } else if (feedInfo->durableVersion == invalidVersion || req.begin > feedInfo->durableVersion) { - for (auto it : data->uidChangeFeed[req.rangeID]->mutations) { + + auto feed = data->uidChangeFeed.find(req.rangeID); + if (feed == data->uidChangeFeed.end()) { + throw unknown_change_feed(); + } + if (req.end <= feed->second->emptyVersion + 1) { + } else if (feed->second->durableVersion == invalidVersion || req.begin > feed->second->durableVersion) { + for (auto it : feed->second->mutations) { if (it.version >= req.end || remainingLimitBytes <= 0) { break; } @@ -1627,9 +1639,8 @@ ACTOR Future getChangeFeedMutations(StorageServer* data, Change } } } else { - state std::deque> mutationsDeque = - data->uidChangeFeed[req.rangeID]->mutations; - state Version startingDurableVersion = feedInfo->durableVersion; + state std::deque> mutationsDeque = feed->second->mutations; + state Version startingDurableVersion = feed->second->durableVersion; RangeResult res = wait(data->storage.readRange( KeyRangeRef(changeFeedDurableKey(req.rangeID, req.begin), changeFeedDurableKey(req.rangeID, req.end)), 1 << 30, @@ -1668,24 +1679,25 @@ ACTOR Future getChangeFeedMutations(StorageServer* data, Change } } if (isEmpty) { - auto& feedInfo = data->uidChangeFeed[req.rangeID]; - if (startingDurableVersion == feedInfo->storageVersion && req.end > startingDurableVersion) { + auto feed = data->uidChangeFeed.find(req.rangeID); + if (feed != data->uidChangeFeed.end() && startingDurableVersion == feed->second->storageVersion && + req.end > startingDurableVersion) { if (req.begin == 0) { - feedInfo->durableVersion = req.end > data->storageVersion() ? invalidVersion : req.end; + feed->second->durableVersion = req.end > data->storageVersion() ? invalidVersion : req.end; } else { RangeResult emp = wait(data->storage.readRange( KeyRangeRef(changeFeedDurableKey(req.rangeID, 0), changeFeedDurableKey(req.rangeID, req.end)), -1)); data->checkChangeCounter(changeCounter, req.range); - auto& feedInfo = data->uidChangeFeed[req.rangeID]; - if (startingDurableVersion == feedInfo->storageVersion) { + auto feed = data->uidChangeFeed.find(req.rangeID); + if (feed != data->uidChangeFeed.end() && startingDurableVersion == feed->second->storageVersion) { if (emp.empty()) { - feedInfo->durableVersion = req.end > data->storageVersion() ? invalidVersion : req.end; + feed->second->durableVersion = req.end > data->storageVersion() ? invalidVersion : req.end; } else { Key id; Version version; std::tie(id, version) = decodeChangeFeedDurableKey(emp[0].key); - feedInfo->durableVersion = version; + feed->second->durableVersion = version; } } } @@ -1705,32 +1717,43 @@ ACTOR Future localChangeFeedStream(StorageServer* data, Version begin, Version end, KeyRange range) { - loop { - state ChangeFeedRequest feedRequest; - feedRequest.rangeID = rangeID; - feedRequest.begin = begin; - feedRequest.end = end; - feedRequest.range = range; - state ChangeFeedReply feedReply = wait(getChangeFeedMutations(data, feedRequest)); - begin = feedReply.mutations.back().version + 1; - state int resultLoc = 0; - while (resultLoc < feedReply.mutations.size()) { - if (feedReply.mutations[resultLoc].mutations.size() || feedReply.mutations[resultLoc].version == end) { - wait(results.onEmpty()); - results.send(feedReply.mutations[resultLoc]); + try { + loop { + state ChangeFeedRequest feedRequest; + feedRequest.rangeID = rangeID; + feedRequest.begin = begin; + feedRequest.end = end; + feedRequest.range = range; + state ChangeFeedReply feedReply = wait(getChangeFeedMutations(data, feedRequest)); + begin = feedReply.mutations.back().version + 1; + state int resultLoc = 0; + while (resultLoc < feedReply.mutations.size()) { + if (feedReply.mutations[resultLoc].mutations.size() || feedReply.mutations[resultLoc].version == end) { + wait(results.onEmpty()); + results.send(feedReply.mutations[resultLoc]); + } + resultLoc++; } - resultLoc++; - } - if (begin == end) { - return Void(); + if (begin == end) { + return Void(); + } } + } catch (Error& e) { + TraceEvent(SevError, "LocalChangeFeedError", data->thisServerID).error(e); + throw; } } ACTOR Future changeFeedQ(StorageServer* data, ChangeFeedRequest req) { - ChangeFeedReply rep = wait(getChangeFeedMutations(data, req)); - req.reply.send(rep); + try { + ChangeFeedReply rep = wait(getChangeFeedMutations(data, req)); + req.reply.send(rep); + } catch (Error& e) { + if (!canReplyWith(e)) + throw; + req.reply.sendError(e); + } return Void(); } @@ -1759,9 +1782,13 @@ ACTOR Future changeFeedStreamQ(StorageServer* data, ChangeFeedStreamReques return Void(); } if (feedReply.mutations.back().mutations.empty()) { + auto feed = data->uidChangeFeed.find(req.rangeID); + if (feed == data->uidChangeFeed.end()) { + throw unknown_change_feed(); + } choose { when(wait(delay(5.0, TaskPriority::DefaultEndpoint))) {} - when(wait(data->uidChangeFeed[req.rangeID]->newMutations.onTrigger())) {} + when(wait(feed->second->newMutations.onTrigger())) {} } } } @@ -3154,9 +3181,9 @@ static const KeyRangeRef persistChangeFeedKeys = // data keys are unmangled (but never start with PERSIST_PREFIX because they are always in allKeys) ACTOR Future fetchChangeFeed(StorageServer* data, Key rangeId, KeyRange range, Version fetchVersion) { - state bool existing = data->uidChangeFeed.count(rangeId); state Reference changeFeedInfo; wait(delay(0)); // allow this actor to be cancelled by removals + bool existing = data->uidChangeFeed.count(rangeId); TraceEvent("FetchChangeFeed", data->thisServerID) .detail("RangeID", rangeId.printable()) @@ -4485,7 +4512,10 @@ ACTOR Future update(StorageServer* data, bool* pReceivedUpdate) { data->changeFeedVersions.push_back(std::make_pair( std::vector(data->currentChangeFeeds.begin(), data->currentChangeFeeds.end()), ver)); for (auto& it : data->currentChangeFeeds) { - data->uidChangeFeed[it]->newMutations.trigger(); + auto feed = data->uidChangeFeed.find(it); + if (feed != data->uidChangeFeed.end()) { + feed->second->newMutations.trigger(); + } } data->currentChangeFeeds.clear(); } @@ -4562,16 +4592,18 @@ ACTOR Future updateStorage(StorageServer* data) { state std::vector updatedChangeFeeds(modifiedChangeFeeds.begin(), modifiedChangeFeeds.end()); state int curFeed = 0; while (curFeed < updatedChangeFeeds.size()) { - auto info = data->uidChangeFeed[updatedChangeFeeds[curFeed]]; - for (auto& it : info->mutations) { - if (it.version >= newOldestVersion) { - break; + auto info = data->uidChangeFeed.find(updatedChangeFeeds[curFeed]); + if (info != data->uidChangeFeed.end()) { + for (auto& it : info->second->mutations) { + if (it.version >= newOldestVersion) { + break; + } + data->storage.writeKeyValue(KeyValueRef(changeFeedDurableKey(info->second->id, it.version), + changeFeedDurableValue(it.mutations))); + info->second->storageVersion = it.version; } - data->storage.writeKeyValue( - KeyValueRef(changeFeedDurableKey(info->id, it.version), changeFeedDurableValue(it.mutations))); - info->storageVersion = it.version; + wait(yield(TaskPriority::UpdateStorage)); } - wait(yield(TaskPriority::UpdateStorage)); curFeed++; } @@ -4610,12 +4642,14 @@ ACTOR Future updateStorage(StorageServer* data) { curFeed = 0; while (curFeed < updatedChangeFeeds.size()) { - auto info = data->uidChangeFeed[updatedChangeFeeds[curFeed]]; - while (!info->mutations.empty() && info->mutations.front().version < newOldestVersion) { - info->mutations.pop_front(); + auto info = data->uidChangeFeed.find(updatedChangeFeeds[curFeed]); + if (info != data->uidChangeFeed.end()) { + while (!info->second->mutations.empty() && info->second->mutations.front().version < newOldestVersion) { + info->second->mutations.pop_front(); + } + info->second->durableVersion = info->second->storageVersion; + wait(yield(TaskPriority::UpdateStorage)); } - info->durableVersion = info->storageVersion; - wait(yield(TaskPriority::UpdateStorage)); curFeed++; } diff --git a/flow/error_definitions.h b/flow/error_definitions.h index b69801cfd7..b6083f3c08 100755 --- a/flow/error_definitions.h +++ b/flow/error_definitions.h @@ -77,6 +77,7 @@ ERROR( dd_not_found, 1053, "Data distributor not found") ERROR( wrong_connection_file, 1054, "Connection file mismatch") ERROR( version_already_compacted, 1055, "The requested changes have been compacted away") ERROR( local_config_changed, 1056, "Local configuration file has changed. Restart and apply these changes" ) +ERROR( unknown_change_feed, 1057, "Change feed not found" ) ERROR( broken_promise, 1100, "Broken promise" ) ERROR( operation_cancelled, 1101, "Asynchronous operation cancelled" ) From 9fefd59076f3ffd24b644315812032ff8511a097 Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Sun, 5 Sep 2021 18:17:23 -0700 Subject: [PATCH 49/72] Fixed a number of bugs, change feeds are correctness clean with the current workload --- fdbserver/storageserver.actor.cpp | 47 ++++++++++++++++------- fdbserver/workloads/ChangeFeeds.actor.cpp | 27 +++++++++++-- tests/fast/ChangeFeeds.toml | 23 +++++++++++ 3 files changed, 80 insertions(+), 17 deletions(-) diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index 7a6aaea47b..cae7d4bf6a 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -1575,6 +1575,10 @@ ACTOR Future overlappingChangeFeedsQ(StorageServer* data, OverlappingChang } void filterMutations(Arena& arena, VectorRef& mutations, KeyRange const& range) { + if (mutations.size() == 1 && mutations.back().param1 == lastEpochEndPrivateKey) { + return; + } + Optional> modifiedMutations; for (int i = 0; i < mutations.size(); i++) { if (mutations[i].type == MutationRef::SetValue) { @@ -1705,7 +1709,7 @@ ACTOR Future getChangeFeedMutations(StorageServer* data, Change } } Version finalVersion = std::min(req.end - 1, data->version.get()); - if ((reply.mutations.empty() || reply.mutations.back().version) < finalVersion && remainingLimitBytes > 0) { + if ((reply.mutations.empty() || reply.mutations.back().version < finalVersion) && remainingLimitBytes > 0) { reply.mutations.push_back(reply.arena, MutationsAndVersionRef(finalVersion)); } return reply; @@ -3999,6 +4003,11 @@ private: ASSERT(rollbackVersion >= data->storageVersion()); rollback(data, rollbackVersion, currentVersion); } + for (auto& it : data->uidChangeFeed) { + it.second->mutations.push_back(MutationsAndVersionRef(currentVersion)); + it.second->mutations.back().mutations.push_back_deep(it.second->mutations.back().arena(), m); + data->currentChangeFeeds.insert(it.first); + } data->recoveryVersionSkips.emplace_back(rollbackVersion, currentVersion - rollbackVersion); } else if (m.type == MutationRef::SetValue && m.param1 == killStoragePrivateKey) { @@ -4357,6 +4366,17 @@ ACTOR Future update(StorageServer* data, bool* pReceivedUpdate) { if (cloneCursor2->version().version > ver && cloneCursor2->version().version > data->version.get()) { ++data->counters.updateVersions; + if (data->currentChangeFeeds.size()) { + data->changeFeedVersions.push_back(std::make_pair( + std::vector(data->currentChangeFeeds.begin(), data->currentChangeFeeds.end()), ver)); + for (auto& it : data->currentChangeFeeds) { + auto feed = data->uidChangeFeed.find(it); + if (feed != data->uidChangeFeed.end()) { + feed->second->newMutations.trigger(); + } + } + data->currentChangeFeeds.clear(); + } ver = cloneCursor2->version().version; } @@ -4435,6 +4455,17 @@ ACTOR Future update(StorageServer* data, bool* pReceivedUpdate) { } } data->tLogMsgsPTreeUpdatesLatencyHistogram->sampleSeconds(now() - beforeTLogMsgsUpdates); + if (data->currentChangeFeeds.size()) { + data->changeFeedVersions.push_back(std::make_pair( + std::vector(data->currentChangeFeeds.begin(), data->currentChangeFeeds.end()), ver)); + for (auto& it : data->currentChangeFeeds) { + auto feed = data->uidChangeFeed.find(it); + if (feed != data->uidChangeFeed.end()) { + feed->second->newMutations.trigger(); + } + } + data->currentChangeFeeds.clear(); + } if (ver != invalidVersion) { data->lastVersionWithData = ver; @@ -4507,18 +4538,6 @@ ACTOR Future update(StorageServer* data, bool* pReceivedUpdate) { data->recoveryVersionSkips.pop_front(); } data->desiredOldestVersion.set(proposedOldestVersion); - - if (data->currentChangeFeeds.size()) { - data->changeFeedVersions.push_back(std::make_pair( - std::vector(data->currentChangeFeeds.begin(), data->currentChangeFeeds.end()), ver)); - for (auto& it : data->currentChangeFeeds) { - auto feed = data->uidChangeFeed.find(it); - if (feed != data->uidChangeFeed.end()) { - feed->second->newMutations.trigger(); - } - } - data->currentChangeFeeds.clear(); - } } validate(data); @@ -4595,7 +4614,7 @@ ACTOR Future updateStorage(StorageServer* data) { auto info = data->uidChangeFeed.find(updatedChangeFeeds[curFeed]); if (info != data->uidChangeFeed.end()) { for (auto& it : info->second->mutations) { - if (it.version >= newOldestVersion) { + if (it.version > newOldestVersion) { break; } data->storage.writeKeyValue(KeyValueRef(changeFeedDurableKey(info->second->id, it.version), diff --git a/fdbserver/workloads/ChangeFeeds.actor.cpp b/fdbserver/workloads/ChangeFeeds.actor.cpp index b75e728e83..0dce7c990d 100644 --- a/fdbserver/workloads/ChangeFeeds.actor.cpp +++ b/fdbserver/workloads/ChangeFeeds.actor.cpp @@ -20,6 +20,7 @@ #include "fdbclient/FDBOptions.g.h" #include "fdbclient/NativeAPI.actor.h" +#include "fdbclient/SystemData.h" #include "fdbserver/TesterInterface.actor.h" #include "fdbserver/workloads/workloads.actor.h" #include "fdbserver/workloads/BulkSetup.actor.h" @@ -68,7 +69,22 @@ ACTOR Future>> readMutations(Databa loop { Standalone> res = waitNext(results.getFuture()); output.arena().dependsOn(res.arena()); - output.append(output.arena(), res.begin(), res.size()); + for (auto& it : res) { + if (it.mutations.size() == 1 && it.mutations.back().param1 == lastEpochEndPrivateKey) { + Version rollbackVersion; + BinaryReader br(it.mutations.back().param2, Unversioned()); + br >> rollbackVersion; + TraceEvent("ChangeFeedRollback") + .detail("Ver", it.version) + .detail("RollbackVer", rollbackVersion); + while (output.size() && output.back().version > rollbackVersion) { + TraceEvent("ChangeFeedRollbackVer").detail("Ver", output.back().version); + output.pop_back(); + } + } else { + output.push_back(output.arena(), it); + } + } begin = res.back().version + 1; } } catch (Error& e) { @@ -147,8 +163,11 @@ struct ChangeFeedsWorkload : TestWorkload { std::string description() const override { return "ChangeFeedsWorkload"; } Future setup(Database const& cx) override { return Void(); } Future start(Database const& cx) override { - client = changeFeedClient(cx->clone(), this); - return delay(testDuration); + if (clientId == 0) { + client = changeFeedClient(cx->clone(), this); + return delay(testDuration); + } + return Void(); } Future check(Database const& cx) override { client = Future(); @@ -174,10 +193,12 @@ struct ChangeFeedsWorkload : TestWorkload { wait(delay(deterministicRandom()->random01())); state std::pair>, Version> firstResults = wait(readDatabase(cx)); + TraceEvent("ChangeFeedReadDB").detail("Ver1", firstResults.second); wait(delay(10 * deterministicRandom()->random01())); state std::pair>, Version> secondResults = wait(readDatabase(cx)); + TraceEvent("ChangeFeedReadDB").detail("Ver2", secondResults.second); state Standalone> mutations = wait(readMutations(cx, rangeID, firstResults.second, secondResults.second + 1)); diff --git a/tests/fast/ChangeFeeds.toml b/tests/fast/ChangeFeeds.toml index 5cd26d1883..d88341be6c 100644 --- a/tests/fast/ChangeFeeds.toml +++ b/tests/fast/ChangeFeeds.toml @@ -10,3 +10,26 @@ testTitle = 'ChangeFeed' [[test.workload]] testName = 'ChangeFeeds' testDuration = 60.0 + + [[test.workload]] + testName = 'RandomClogging' + testDuration = 60.0 + + [[test.workload]] + testName = 'Rollback' + meanDelay = 60.0 + testDuration = 60.0 + + [[test.workload]] + testName = 'Attrition' + machinesToKill = 10 + machinesToLeave = 3 + reboot = true + testDuration = 60.0 + + [[test.workload]] + testName = 'Attrition' + machinesToKill = 10 + machinesToLeave = 3 + reboot = true + testDuration = 60.0 From 7b238940393f179ab2faf70e243870ab81bb78f6 Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Sun, 5 Sep 2021 20:33:12 -0700 Subject: [PATCH 50/72] fixed a few bugs related to change feed fetches --- fdbserver/storageserver.actor.cpp | 27 +++++++++++++++++---------- 1 file changed, 17 insertions(+), 10 deletions(-) diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index cae7d4bf6a..c5fb3036cf 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -1621,7 +1621,7 @@ ACTOR Future getChangeFeedMutations(StorageServer* data, Change wait(data->version.whenAtLeast(req.begin)); } state uint64_t changeCounter = data->shardChangeCounter; - if (!data->isReadable(req.range)) { + if (!req.range.empty() && !data->isReadable(req.range)) { throw wrong_shard_server(); } @@ -1650,7 +1650,9 @@ ACTOR Future getChangeFeedMutations(StorageServer* data, Change 1 << 30, remainingLimitBytes)); - data->checkChangeCounter(changeCounter, req.range); + if (!req.range.empty()) { + data->checkChangeCounter(changeCounter, req.range); + } bool isEmpty = res.size() == 0; Version lastVersion = req.begin - 1; @@ -1692,7 +1694,9 @@ ACTOR Future getChangeFeedMutations(StorageServer* data, Change RangeResult emp = wait(data->storage.readRange( KeyRangeRef(changeFeedDurableKey(req.rangeID, 0), changeFeedDurableKey(req.rangeID, req.end)), -1)); - data->checkChangeCounter(changeCounter, req.range); + if (!req.range.empty()) { + data->checkChangeCounter(changeCounter, req.range); + } auto feed = data->uidChangeFeed.find(req.rangeID); if (feed != data->uidChangeFeed.end() && startingDurableVersion == feed->second->storageVersion) { if (emp.empty()) { @@ -1719,20 +1723,19 @@ ACTOR Future localChangeFeedStream(StorageServer* data, PromiseStream> results, Key rangeID, Version begin, - Version end, - KeyRange range) { + Version end) { try { loop { state ChangeFeedRequest feedRequest; feedRequest.rangeID = rangeID; feedRequest.begin = begin; feedRequest.end = end; - feedRequest.range = range; state ChangeFeedReply feedReply = wait(getChangeFeedMutations(data, feedRequest)); begin = feedReply.mutations.back().version + 1; state int resultLoc = 0; while (resultLoc < feedReply.mutations.size()) { - if (feedReply.mutations[resultLoc].mutations.size() || feedReply.mutations[resultLoc].version == end) { + if (feedReply.mutations[resultLoc].mutations.size() || + feedReply.mutations[resultLoc].version == end - 1) { wait(results.onEmpty()); results.send(feedReply.mutations[resultLoc]); } @@ -3222,8 +3225,10 @@ ACTOR Future fetchChangeFeed(StorageServer* data, Key rangeId, KeyRange ra loop { Standalone> res = waitNext(feedResults.getFuture()); for (auto& it : res) { - data->storage.writeKeyValue( - KeyValueRef(changeFeedDurableKey(rangeId, it.version), changeFeedDurableValue(it.mutations))); + if (it.mutations.size()) { + data->storage.writeKeyValue(KeyValueRef(changeFeedDurableKey(rangeId, it.version), + changeFeedDurableValue(it.mutations))); + } } wait(yield()); } @@ -3231,11 +3236,12 @@ ACTOR Future fetchChangeFeed(StorageServer* data, Key rangeId, KeyRange ra if (e.code() != error_code_end_of_stream) { throw; } + return Void(); } } state PromiseStream> localResults; - state Future localStream = localChangeFeedStream(data, localResults, rangeId, 0, fetchVersion + 2, range); + state Future localStream = localChangeFeedStream(data, localResults, rangeId, 0, fetchVersion + 2); state Standalone localResult; Standalone _localResult = waitNext(localResults.getFuture()); @@ -3301,6 +3307,7 @@ ACTOR Future dispatchChangeFeeds(StorageServer* data, UID fetchKeysID, Key } else { nextFeed = feedFetches.begin()->second; done = false; + break; } } if (done) { From f53ddf67205dc4744dd096ca6bc56301a5254c95 Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Mon, 6 Sep 2021 13:51:53 -0700 Subject: [PATCH 51/72] fix: handle the case where I change feed is added while a fetch is in progress --- fdbclient/NativeAPI.actor.cpp | 18 ++++++++++++------ fdbserver/storageserver.actor.cpp | 4 ++-- 2 files changed, 14 insertions(+), 8 deletions(-) diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index bdb91e902d..e8231ccee8 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -6702,13 +6702,18 @@ ACTOR Future getChangeFeedStreamActor(Reference db, loop { loop { try { - Optional val = wait(tr.get(rangeIDKey)); - if (!val.present()) { - results.sendError(unsupported_operation()); - return Void(); + Version readVer = wait(tr.getReadVersion()); + if (readVer < begin) { + wait(delay(FLOW_KNOBS->PREVENT_FAST_SPIN_DELAY)); + } else { + Optional val = wait(tr.get(rangeIDKey)); + if (!val.present()) { + results.sendError(unsupported_operation()); + return Void(); + } + keys = std::get<0>(decodeChangeFeedValue(val.get())) & range; + break; } - keys = std::get<0>(decodeChangeFeedValue(val.get())) & range; - break; } catch (Error& e) { wait(tr.onError(e)); } @@ -6724,6 +6729,7 @@ ACTOR Future getChangeFeedStreamActor(Reference db, TransactionInfo(TaskPriority::DefaultEndpoint, span.context))); if (locations.size() >= 1000) { + ASSERT(false); results.sendError(unsupported_operation()); return Void(); } diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index c5fb3036cf..bdf0e944da 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -3233,7 +3233,7 @@ ACTOR Future fetchChangeFeed(StorageServer* data, Key rangeId, KeyRange ra wait(yield()); } } catch (Error& e) { - if (e.code() != error_code_end_of_stream) { + if (e.code() != error_code_end_of_stream && e.code() != error_code_unsupported_operation) { throw; } return Void(); @@ -3278,7 +3278,7 @@ ACTOR Future fetchChangeFeed(StorageServer* data, Key rangeId, KeyRange ra wait(yield()); } } catch (Error& e) { - if (e.code() != error_code_end_of_stream) { + if (e.code() != error_code_end_of_stream && e.code() != error_code_unsupported_operation) { throw; } } From 590da022ee1e615f8dd7ff39ed1631b6a42e8b60 Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Tue, 7 Sep 2021 16:20:49 -0700 Subject: [PATCH 52/72] fix: missing mutations committed at newOldestVersion --- fdbserver/storageserver.actor.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index bdf0e944da..7144082d75 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -4609,7 +4609,7 @@ ACTOR Future updateStorage(StorageServer* data) { } std::set modifiedChangeFeeds; - while (!data->changeFeedVersions.empty() && data->changeFeedVersions.front().second < newOldestVersion) { + while (!data->changeFeedVersions.empty() && data->changeFeedVersions.front().second <= newOldestVersion) { modifiedChangeFeeds.insert(data->changeFeedVersions.front().first.begin(), data->changeFeedVersions.front().first.end()); data->changeFeedVersions.pop_front(); From d9782bc29d989cca99d6f12cf2d5671f09d024b3 Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Tue, 7 Sep 2021 21:35:00 -0700 Subject: [PATCH 53/72] fixed a few bugs with merging change feeds on the client --- fdbclient/NativeAPI.actor.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index e8231ccee8..4fd67506f5 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -6609,6 +6609,7 @@ ACTOR Future singleChangeFeedStream(StorageServerInterface interf, resultLoc++; } if (begin == end) { + results.sendError(end_of_stream()); return Void(); } } @@ -6628,7 +6629,7 @@ struct MutationAndVersionStream { Standalone next; PromiseStream> results; - bool operator<(MutationAndVersionStream const& rhs) const { return next.version < rhs.next.version; } + bool operator<(MutationAndVersionStream const& rhs) const { return next.version > rhs.next.version; } }; ACTOR Future mergeChangeFeedStream(std::vector> interfs, From efc4cec53fa7eee0498e3a510ac7211f6c3d7f3f Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Sat, 9 Oct 2021 19:24:01 -0700 Subject: [PATCH 54/72] fixed a variety of bugs with change feeds --- fdbclient/NativeAPI.actor.cpp | 25 +++++++++------ fdbserver/storageserver.actor.cpp | 53 ++++++++++++++++--------------- 2 files changed, 44 insertions(+), 34 deletions(-) diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index 8466a8f906..34afdc7b5b 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -6665,9 +6665,7 @@ ACTOR Future singleChangeFeedStream(StorageServerInterface interf, } } } catch (Error& e) { - if (e.code() == error_code_wrong_shard_server || e.code() == error_code_all_alternatives_failed || - e.code() == error_code_connection_failed || e.code() == error_code_unknown_change_feed || - e.code() == error_code_actor_cancelled) { + if (e.code() == error_code_actor_cancelled) { throw; } results.sendError(e); @@ -6746,12 +6744,12 @@ ACTOR Future getChangeFeedStreamActor(Reference db, Version end, KeyRange range) { state Database cx(db); - state Transaction tr(cx); state Key rangeIDKey = rangeID.withPrefix(changeFeedPrefix); state Span span("NAPI:GetChangeFeedStream"_loc); state KeyRange keys; loop { + state Transaction tr(cx); loop { try { Version readVer = wait(tr.getReadVersion()); @@ -6983,14 +6981,23 @@ ACTOR static Future popChangeFeedBackup(Database cx, StringRef rangeID, Ve ACTOR Future popChangeFeedMutationsActor(Reference db, StringRef rangeID, Version version) { state Database cx(db); - state Transaction tr(cx); state Key rangeIDKey = rangeID.withPrefix(changeFeedPrefix); state Span span("NAPI:PopChangeFeedMutations"_loc); - Optional val = wait(tr.get(rangeIDKey)); - if (!val.present()) { - throw unsupported_operation(); + + state Transaction tr(cx); + state KeyRange keys; + loop { + try { + Optional val = wait(tr.get(rangeIDKey)); + if (!val.present()) { + throw unsupported_operation(); + } + keys = std::get<0>(decodeChangeFeedValue(val.get())); + break; + } catch (Error& e) { + wait(tr.onError(e)); + } } - state KeyRange keys = std::get<0>(decodeChangeFeedValue(val.get())); state vector>> locations = wait(getKeyRangeLocations(cx, keys, diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index b9ce4214b4..c47d794252 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -1591,43 +1591,44 @@ ACTOR Future overlappingChangeFeedsQ(StorageServer* data, OverlappingChang return Void(); } -void filterMutations(Arena& arena, VectorRef& mutations, KeyRange const& range) { - if (mutations.size() == 1 && mutations.back().param1 == lastEpochEndPrivateKey) { - return; +MutationsAndVersionRef filterMutations(Arena& arena, MutationsAndVersionRef const& m, KeyRange const& range) { + if (m.mutations.size() == 1 && m.mutations.back().param1 == lastEpochEndPrivateKey) { + return m; } Optional> modifiedMutations; - for (int i = 0; i < mutations.size(); i++) { - if (mutations[i].type == MutationRef::SetValue) { - if (modifiedMutations.present() && range.contains(mutations[i].param1)) { - modifiedMutations.get().push_back(arena, mutations[i]); + for (int i = 0; i < m.mutations.size(); i++) { + if (m.mutations[i].type == MutationRef::SetValue) { + if (modifiedMutations.present() && range.contains(m.mutations[i].param1)) { + modifiedMutations.get().push_back(arena, m.mutations[i]); } - if (!modifiedMutations.present() && !range.contains(mutations[i].param1)) { - modifiedMutations = mutations; + if (!modifiedMutations.present() && !range.contains(m.mutations[i].param1)) { + modifiedMutations = m.mutations; modifiedMutations.get().resize(arena, i); arena.dependsOn(range.arena()); } } else { - ASSERT(mutations[i].type == MutationRef::ClearRange); + ASSERT(m.mutations[i].type == MutationRef::ClearRange); if (!modifiedMutations.present() && - (mutations[i].param1 < range.begin || mutations[i].param2 > range.end)) { - modifiedMutations = mutations; + (m.mutations[i].param1 < range.begin || m.mutations[i].param2 > range.end)) { + modifiedMutations = m.mutations; modifiedMutations.get().resize(arena, i); arena.dependsOn(range.arena()); } if (modifiedMutations.present()) { - if (mutations[i].param1 < range.end && range.begin < mutations[i].param2) { + if (m.mutations[i].param1 < range.end && range.begin < m.mutations[i].param2) { modifiedMutations.get().push_back(arena, MutationRef(MutationRef::ClearRange, - std::max(range.begin, mutations[i].param1), - std::min(range.end, mutations[i].param2))); + std::max(range.begin, m.mutations[i].param1), + std::min(range.end, m.mutations[i].param2))); } } } } if (modifiedMutations.present()) { - mutations = modifiedMutations.get(); + return MutationsAndVersionRef(modifiedMutations.get(), m.version); } + return m; } ACTOR Future getChangeFeedMutations(StorageServer* data, ChangeFeedRequest req) { @@ -1646,6 +1647,7 @@ ACTOR Future getChangeFeedMutations(StorageServer* data, Change if (feed == data->uidChangeFeed.end()) { throw unknown_change_feed(); } + state Version dequeVersion = data->version.get(); if (req.end <= feed->second->emptyVersion + 1) { } else if (feed->second->durableVersion == invalidVersion || req.begin > feed->second->durableVersion) { for (auto it : feed->second->mutations) { @@ -1654,14 +1656,15 @@ ACTOR Future getChangeFeedMutations(StorageServer* data, Change } if (it.version >= req.begin) { reply.arena.dependsOn(it.arena()); - filterMutations(reply.arena, it.mutations, req.range); - reply.mutations.push_back(reply.arena, it); - remainingLimitBytes -= sizeof(MutationsAndVersionRef) + it.expectedSize(); + auto m = filterMutations(reply.arena, it, req.range); + reply.mutations.push_back(reply.arena, m); + remainingLimitBytes -= sizeof(MutationsAndVersionRef) + m.expectedSize(); } } } else { state std::deque> mutationsDeque = feed->second->mutations; state Version startingDurableVersion = feed->second->durableVersion; + RangeResult res = wait(data->storage.readRange( KeyRangeRef(changeFeedDurableKey(req.rangeID, req.begin), changeFeedDurableKey(req.rangeID, req.end)), 1 << 30, @@ -1679,8 +1682,8 @@ ACTOR Future getChangeFeedMutations(StorageServer* data, Change std::tie(id, version) = decodeChangeFeedDurableKey(kv.key); auto mutations = decodeChangeFeedDurableValue(kv.value); reply.arena.dependsOn(mutations.arena()); - filterMutations(reply.arena, mutations, req.range); - reply.mutations.push_back(reply.arena, MutationsAndVersionRef(mutations, version)); + auto m = filterMutations(reply.arena, MutationsAndVersionRef(mutations, version), req.range); + reply.mutations.push_back(reply.arena, m); remainingLimitBytes -= sizeof(KeyValueRef) + kv.expectedSize(); // FIXME: this is currently tracking the size on disk rather than the reply size @@ -1696,9 +1699,9 @@ ACTOR Future getChangeFeedMutations(StorageServer* data, Change isEmpty = false; } reply.arena.dependsOn(it.arena()); - filterMutations(reply.arena, it.mutations, req.range); - reply.mutations.push_back(reply.arena, it); - remainingLimitBytes -= sizeof(MutationsAndVersionRef) + it.expectedSize(); + auto m = filterMutations(reply.arena, it, req.range); + reply.mutations.push_back(reply.arena, m); + remainingLimitBytes -= sizeof(MutationsAndVersionRef) + m.expectedSize(); } } if (isEmpty) { @@ -1729,7 +1732,7 @@ ACTOR Future getChangeFeedMutations(StorageServer* data, Change } } } - Version finalVersion = std::min(req.end - 1, data->version.get()); + Version finalVersion = std::min(req.end - 1, dequeVersion); if ((reply.mutations.empty() || reply.mutations.back().version < finalVersion) && remainingLimitBytes > 0) { reply.mutations.push_back(reply.arena, MutationsAndVersionRef(finalVersion)); } From d51edf18dc88c181a37f99c894d58e8457e7a791 Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Sat, 9 Oct 2021 19:47:24 -0700 Subject: [PATCH 55/72] fixed merge conflicts --- fdbcli/fdbcli.actor.cpp | 12 ++++++------ fdbclient/NativeAPI.actor.cpp | 2 +- fdbserver/workloads/ChangeFeeds.actor.cpp | 2 +- 3 files changed, 8 insertions(+), 8 deletions(-) diff --git a/fdbcli/fdbcli.actor.cpp b/fdbcli/fdbcli.actor.cpp index 1add9272a3..7f839c7b4f 100644 --- a/fdbcli/fdbcli.actor.cpp +++ b/fdbcli/fdbcli.actor.cpp @@ -1852,7 +1852,7 @@ ACTOR Future cli(CLIOptions opt, LineNoise* plinenoise) { is_error = true; continue; } - wait(changeFeedList(db)); + wait(changeFeedList(localDb)); continue; } else if (tokencmp(tokens[1], "register")) { if (tokens.size() != 5) { @@ -1860,7 +1860,7 @@ ACTOR Future cli(CLIOptions opt, LineNoise* plinenoise) { is_error = true; continue; } - trx = Transaction(db); + trx = Transaction(localDb); loop { try { wait(trx.registerChangeFeed(tokens[2], KeyRangeRef(tokens[3], tokens[4]))); @@ -1876,7 +1876,7 @@ ACTOR Future cli(CLIOptions opt, LineNoise* plinenoise) { is_error = true; continue; } - trx = Transaction(db); + trx = Transaction(localDb); loop { try { trx.destroyChangeFeed(tokens[2]); @@ -1912,7 +1912,7 @@ ACTOR Future cli(CLIOptions opt, LineNoise* plinenoise) { } } Standalone> res = - wait(db->getChangeFeedMutations(tokens[2], begin, end)); + wait(localDb->getChangeFeedMutations(tokens[2], begin, end)); printf("\n"); for (auto& it : res) { for (auto& it2 : it.mutations) { @@ -1948,7 +1948,7 @@ ACTOR Future cli(CLIOptions opt, LineNoise* plinenoise) { warn.cancel(); } state PromiseStream>> feedResults; - state Future feed = db->getChangeFeedStream(feedResults, tokens[2], begin, end); + state Future feed = localDb->getChangeFeedStream(feedResults, tokens[2], begin, end); printf("\n"); try { state Future feedInterrupt = LineNoise::onKeyboardInterrupt(); @@ -1989,7 +1989,7 @@ ACTOR Future cli(CLIOptions opt, LineNoise* plinenoise) { printUsage(tokens[0]); is_error = true; } else { - wait(db->popChangeFeedMutations(tokens[2], v)); + wait(localDb->popChangeFeedMutations(tokens[2], v)); } } continue; diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index dbbec3e7ad..1119f41e35 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -6898,7 +6898,7 @@ ACTOR Future getChangeFeedStreamActor(Reference db, continue; } - vector> ok(locations[loc].second->size()); + std::vector> ok(locations[loc].second->size()); for (int i = 0; i < ok.size(); i++) { ok[i] = IFailureMonitor::failureMonitor().onStateEqual( locations[loc].second->get(i, &StorageServerInterface::changeFeedStream).getEndpoint(), diff --git a/fdbserver/workloads/ChangeFeeds.actor.cpp b/fdbserver/workloads/ChangeFeeds.actor.cpp index 0dce7c990d..fa382f68c9 100644 --- a/fdbserver/workloads/ChangeFeeds.actor.cpp +++ b/fdbserver/workloads/ChangeFeeds.actor.cpp @@ -173,7 +173,7 @@ struct ChangeFeedsWorkload : TestWorkload { client = Future(); return true; } - void getMetrics(vector& m) override {} + void getMetrics(std::vector& m) override {} ACTOR Future changeFeedClient(Database cx, ChangeFeedsWorkload* self) { // Enable change feeds for a key range From effeb1dcd7bd062ef7f112bd5fd369d11eec4e95 Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Sat, 9 Oct 2021 21:18:13 -0700 Subject: [PATCH 56/72] fix: must serialize the sequence --- fdbclient/StorageServerInterface.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/fdbclient/StorageServerInterface.h b/fdbclient/StorageServerInterface.h index 7e79794104..0502cef70a 100644 --- a/fdbclient/StorageServerInterface.h +++ b/fdbclient/StorageServerInterface.h @@ -706,7 +706,7 @@ struct ChangeFeedStreamReply : public ReplyPromiseStreamReply { template void serialize(Ar& ar) { - serializer(ar, ReplyPromiseStreamReply::acknowledgeToken, mutations, arena); + serializer(ar, ReplyPromiseStreamReply::acknowledgeToken, ReplyPromiseStreamReply::sequence, mutations, arena); } }; From 26b6f9a3f10f64abe5298e93ffbf95570e2ff1ff Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Sun, 10 Oct 2021 11:32:49 -0700 Subject: [PATCH 57/72] fix: resize changes to source vector even though m is a copy --- fdbserver/storageserver.actor.cpp | 8 +++----- 1 file changed, 3 insertions(+), 5 deletions(-) diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index 577c009509..576faf6d0b 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -1616,16 +1616,14 @@ MutationsAndVersionRef filterMutations(Arena& arena, MutationsAndVersionRef cons modifiedMutations.get().push_back(arena, m.mutations[i]); } if (!modifiedMutations.present() && !range.contains(m.mutations[i].param1)) { - modifiedMutations = m.mutations; - modifiedMutations.get().resize(arena, i); + modifiedMutations = m.mutations.slice(0, i); arena.dependsOn(range.arena()); } } else { ASSERT(m.mutations[i].type == MutationRef::ClearRange); if (!modifiedMutations.present() && (m.mutations[i].param1 < range.begin || m.mutations[i].param2 > range.end)) { - modifiedMutations = m.mutations; - modifiedMutations.get().resize(arena, i); + modifiedMutations = m.mutations.slice(0, i); arena.dependsOn(range.arena()); } if (modifiedMutations.present()) { @@ -1663,7 +1661,7 @@ ACTOR Future getChangeFeedMutations(StorageServer* data, Change state Version dequeVersion = data->version.get(); if (req.end <= feed->second->emptyVersion + 1) { } else if (feed->second->durableVersion == invalidVersion || req.begin > feed->second->durableVersion) { - for (auto it : feed->second->mutations) { + for (auto& it : feed->second->mutations) { if (it.version >= req.end || remainingLimitBytes <= 0) { break; } From 5b6f8b2abb7b3a113ae04c807c518377738aec72 Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Mon, 11 Oct 2021 13:53:36 -0700 Subject: [PATCH 58/72] added the known committed version to change feeds --- fdbclient/StorageServerInterface.h | 16 +++++----- fdbclient/SystemData.cpp | 11 ++++--- fdbclient/SystemData.h | 6 ++-- fdbserver/storageserver.actor.cpp | 50 ++++++++++++++++++------------ 4 files changed, 49 insertions(+), 34 deletions(-) diff --git a/fdbclient/StorageServerInterface.h b/fdbclient/StorageServerInterface.h index 0502cef70a..ed4b2a5dbd 100644 --- a/fdbclient/StorageServerInterface.h +++ b/fdbclient/StorageServerInterface.h @@ -641,15 +641,17 @@ struct SplitRangeRequest { struct MutationsAndVersionRef { VectorRef mutations; Version version; + Version knownCommittedVersion; MutationsAndVersionRef() {} - explicit MutationsAndVersionRef(Version version) : version(version) {} - MutationsAndVersionRef(VectorRef mutations, Version version) - : mutations(mutations), version(version) {} - MutationsAndVersionRef(Arena& to, VectorRef mutations, Version version) - : mutations(to, mutations), version(version) {} + explicit MutationsAndVersionRef(Version version, Version knownCommittedVersion) + : version(version), knownCommittedVersion(knownCommittedVersion) {} + MutationsAndVersionRef(VectorRef mutations, Version version, Version knownCommittedVersion) + : mutations(mutations), version(version), knownCommittedVersion(knownCommittedVersion) {} + MutationsAndVersionRef(Arena& to, VectorRef mutations, Version version, Version knownCommittedVersion) + : mutations(to, mutations), version(version), knownCommittedVersion(knownCommittedVersion) {} MutationsAndVersionRef(Arena& to, const MutationsAndVersionRef& from) - : mutations(to, from.mutations), version(from.version) {} + : mutations(to, from.mutations), version(from.version), knownCommittedVersion(from.knownCommittedVersion) {} int expectedSize() const { return mutations.expectedSize(); } struct OrderByVersion { @@ -660,7 +662,7 @@ struct MutationsAndVersionRef { template void serialize(Ar& ar) { - serializer(ar, mutations, version); + serializer(ar, mutations, version, knownCommittedVersion); } }; diff --git a/fdbclient/SystemData.cpp b/fdbclient/SystemData.cpp index 72e3e6e558..7676dea4a7 100644 --- a/fdbclient/SystemData.cpp +++ b/fdbclient/SystemData.cpp @@ -1058,7 +1058,7 @@ std::tuple decodeChangeFeedValue(ValueRef const& value) const KeyRangeRef changeFeedDurableKeys(LiteralStringRef("\xff\xff/cf/"), LiteralStringRef("\xff\xff/cf0")); const KeyRef changeFeedDurablePrefix = changeFeedDurableKeys.begin; -const Value changeFeedDurableKey(Key const& feed, Version const& version) { +const Value changeFeedDurableKey(Key const& feed, Version version) { BinaryWriter wr(AssumeVersion(ProtocolVersion::withChangeFeed())); wr.serializeBytes(changeFeedDurablePrefix); wr << feed; @@ -1073,16 +1073,19 @@ std::pair decodeChangeFeedDurableKey(ValueRef const& key) { reader >> version; return std::make_pair(feed, bigEndian64(version)); } -const Value changeFeedDurableValue(Standalone> const& mutations) { +const Value changeFeedDurableValue(Standalone> const& mutations, Version knownCommittedVersion) { BinaryWriter wr(IncludeVersion(ProtocolVersion::withChangeFeed())); wr << mutations; + wr << knownCommittedVersion; return wr.toValue(); } -Standalone> decodeChangeFeedDurableValue(ValueRef const& value) { +std::pair>, Version> decodeChangeFeedDurableValue(ValueRef const& value) { Standalone> mutations; + Version knownCommittedVersion; BinaryReader reader(value, IncludeVersion()); reader >> mutations; - return mutations; + reader >> knownCommittedVersion; + return std::make_pair(mutations, knownCommittedVersion); } const KeyRef configTransactionDescriptionKey = "\xff\xff/description"_sr; diff --git a/fdbclient/SystemData.h b/fdbclient/SystemData.h index ecf5550f0d..e7039764bd 100644 --- a/fdbclient/SystemData.h +++ b/fdbclient/SystemData.h @@ -505,10 +505,10 @@ extern const KeyRef changeFeedPrivatePrefix; extern const KeyRangeRef changeFeedDurableKeys; extern const KeyRef changeFeedDurablePrefix; -const Value changeFeedDurableKey(Key const& feed, Version const& version); +const Value changeFeedDurableKey(Key const& feed, Version version); std::pair decodeChangeFeedDurableKey(ValueRef const& key); -const Value changeFeedDurableValue(Standalone> const& mutations); -Standalone> decodeChangeFeedDurableValue(ValueRef const& value); +const Value changeFeedDurableValue(Standalone> const& mutations, Version knownCommittedVersion); +std::pair>, Version> decodeChangeFeedDurableValue(ValueRef const& value); // Configuration database special keys extern const KeyRef configTransactionDescriptionKey; diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index 576faf6d0b..a13f5d8910 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -618,6 +618,7 @@ public: NotifiedVersion durableVersion; // At least this version will be readable from storage after a power failure Version rebootAfterDurableVersion; int8_t primaryLocality; + Version knownCommittedVersion; Deque> recoveryVersionSkips; int64_t versionLag; // An estimate for how many versions it takes for the data to move from the logs to this storage @@ -868,10 +869,10 @@ public: Histogram::Unit::microseconds)), tag(invalidTag), poppedAllAfter(std::numeric_limits::max()), cpuUsage(0.0), diskUsage(0.0), storage(this, storage), shardChangeCounter(0), lastTLogVersion(0), lastVersionWithData(0), restoredVersion(0), - rebootAfterDurableVersion(std::numeric_limits::max()), primaryLocality(tagLocalityInvalid), - versionLag(0), logProtocol(0), thisServerID(ssi.id()), tssInQuarantine(false), db(db), actors(false), - byteSampleClears(false, LiteralStringRef("\xff\xff\xff")), durableInProgress(Void()), watchBytes(0), - numWatches(0), noRecentUpdates(false), lastUpdate(now()), + knownCommittedVersion(0), rebootAfterDurableVersion(std::numeric_limits::max()), + primaryLocality(tagLocalityInvalid), versionLag(0), logProtocol(0), thisServerID(ssi.id()), + tssInQuarantine(false), db(db), actors(false), byteSampleClears(false, LiteralStringRef("\xff\xff\xff")), + durableInProgress(Void()), watchBytes(0), numWatches(0), noRecentUpdates(false), lastUpdate(now()), readQueueSizeMetric(LiteralStringRef("StorageServer.ReadQueueSize")), updateEagerReads(nullptr), fetchKeysParallelismLock(SERVER_KNOBS->FETCH_KEYS_PARALLELISM), fetchKeysBytesBudget(SERVER_KNOBS->STORAGE_FETCH_BYTES), fetchKeysBudgetUsed(false), @@ -1637,7 +1638,7 @@ MutationsAndVersionRef filterMutations(Arena& arena, MutationsAndVersionRef cons } } if (modifiedMutations.present()) { - return MutationsAndVersionRef(modifiedMutations.get(), m.version); + return MutationsAndVersionRef(modifiedMutations.get(), m.version, m.knownCommittedVersion); } return m; } @@ -1659,6 +1660,7 @@ ACTOR Future getChangeFeedMutations(StorageServer* data, Change throw unknown_change_feed(); } state Version dequeVersion = data->version.get(); + state Version dequeKnownCommit = data->knownCommittedVersion; if (req.end <= feed->second->emptyVersion + 1) { } else if (feed->second->durableVersion == invalidVersion || req.begin > feed->second->durableVersion) { for (auto& it : feed->second->mutations) { @@ -1689,11 +1691,13 @@ ACTOR Future getChangeFeedMutations(StorageServer* data, Change Version lastVersion = req.begin - 1; for (auto& kv : res) { Key id; - Version version; + Version version, knownCommittedVersion; + Standalone> mutations; std::tie(id, version) = decodeChangeFeedDurableKey(kv.key); - auto mutations = decodeChangeFeedDurableValue(kv.value); + std::tie(mutations, knownCommittedVersion) = decodeChangeFeedDurableValue(kv.value); reply.arena.dependsOn(mutations.arena()); - auto m = filterMutations(reply.arena, MutationsAndVersionRef(mutations, version), req.range); + auto m = filterMutations( + reply.arena, MutationsAndVersionRef(mutations, version, knownCommittedVersion), req.range); reply.mutations.push_back(reply.arena, m); remainingLimitBytes -= sizeof(KeyValueRef) + @@ -1745,7 +1749,8 @@ ACTOR Future getChangeFeedMutations(StorageServer* data, Change } Version finalVersion = std::min(req.end - 1, dequeVersion); if ((reply.mutations.empty() || reply.mutations.back().version < finalVersion) && remainingLimitBytes > 0) { - reply.mutations.push_back(reply.arena, MutationsAndVersionRef(finalVersion)); + reply.mutations.push_back( + reply.arena, MutationsAndVersionRef(finalVersion, finalVersion == dequeVersion ? dequeKnownCommit : 0)); } return reply; } @@ -2960,7 +2965,7 @@ void applyMutation(StorageServer* self, if (!fromFetch) { for (auto& it : self->keyChangeFeed[m.param1]) { if (it->mutations.empty() || it->mutations.back().version != version) { - it->mutations.push_back(MutationsAndVersionRef(version)); + it->mutations.push_back(MutationsAndVersionRef(version, self->knownCommittedVersion)); } it->mutations.back().mutations.push_back_deep(it->mutations.back().arena(), m); self->currentChangeFeeds.insert(it->id); @@ -2978,7 +2983,7 @@ void applyMutation(StorageServer* self, for (auto& r : ranges) { for (auto& it : r.value()) { if (it->mutations.empty() || it->mutations.back().version != version) { - it->mutations.push_back(MutationsAndVersionRef(version)); + it->mutations.push_back(MutationsAndVersionRef(version, self->knownCommittedVersion)); } it->mutations.back().mutations.push_back_deep(it->mutations.back().arena(), m); self->currentChangeFeeds.insert(it->id); @@ -3262,8 +3267,9 @@ ACTOR Future fetchChangeFeed(StorageServer* data, Key rangeId, KeyRange ra Standalone> res = waitNext(feedResults.getFuture()); for (auto& it : res) { if (it.mutations.size()) { - data->storage.writeKeyValue(KeyValueRef(changeFeedDurableKey(rangeId, it.version), - changeFeedDurableValue(it.mutations))); + data->storage.writeKeyValue( + KeyValueRef(changeFeedDurableKey(rangeId, it.version), + changeFeedDurableValue(it.mutations, it.knownCommittedVersion))); } } wait(yield()); @@ -3291,7 +3297,8 @@ ACTOR Future fetchChangeFeed(StorageServer* data, Key rangeId, KeyRange ra if (remoteResult[remoteLoc].mutations.size()) { data->storage.writeKeyValue( KeyValueRef(changeFeedDurableKey(rangeId, remoteResult[remoteLoc].version), - changeFeedDurableValue(remoteResult[remoteLoc].mutations))); + changeFeedDurableValue(remoteResult[remoteLoc].mutations, + remoteResult[remoteLoc].knownCommittedVersion))); } remoteLoc++; } else if (remoteResult[remoteLoc].version == localResult.version) { @@ -3301,7 +3308,8 @@ ACTOR Future fetchChangeFeed(StorageServer* data, Key rangeId, KeyRange ra remoteResult.arena(), localResult.mutations.begin(), localResult.mutations.size()); data->storage.writeKeyValue( KeyValueRef(changeFeedDurableKey(rangeId, remoteResult[remoteLoc].version), - changeFeedDurableValue(remoteResult[remoteLoc].mutations))); + changeFeedDurableValue(remoteResult[remoteLoc].mutations, + remoteResult[remoteLoc].knownCommittedVersion))); } remoteLoc++; Standalone _localResult = waitNext(localResults.getFuture()); @@ -3748,7 +3756,7 @@ void AddingShard::addMutation(Version version, bool fromFetch, MutationRef const if (mutation.type == MutationRef::SetValue) { for (auto& it : server->keyChangeFeed[mutation.param1]) { if (it->mutations.empty() || it->mutations.back().version != version) { - it->mutations.push_back(MutationsAndVersionRef(version)); + it->mutations.push_back(MutationsAndVersionRef(version, server->knownCommittedVersion)); } it->mutations.back().mutations.push_back_deep(it->mutations.back().arena(), mutation); server->currentChangeFeeds.insert(it->id); @@ -3758,7 +3766,7 @@ void AddingShard::addMutation(Version version, bool fromFetch, MutationRef const for (auto& r : ranges) { for (auto& it : r.value()) { if (it->mutations.empty() || it->mutations.back().version != version) { - it->mutations.push_back(MutationsAndVersionRef(version)); + it->mutations.push_back(MutationsAndVersionRef(version, server->knownCommittedVersion)); } it->mutations.back().mutations.push_back_deep(it->mutations.back().arena(), mutation); server->currentChangeFeeds.insert(it->id); @@ -4062,7 +4070,7 @@ private: rollback(data, rollbackVersion, currentVersion); } for (auto& it : data->uidChangeFeed) { - it.second->mutations.push_back(MutationsAndVersionRef(currentVersion)); + it.second->mutations.push_back(MutationsAndVersionRef(currentVersion, rollbackVersion)); it.second->mutations.back().mutations.push_back_deep(it.second->mutations.back().arena(), m); data->currentChangeFeeds.insert(it.first); } @@ -4281,6 +4289,7 @@ ACTOR Future update(StorageServer* data, bool* pReceivedUpdate) { ++data->counters.updateBatches; data->lastTLogVersion = cursor->getMaxKnownVersion(); + data->knownCommittedVersion = cursor->getMinKnownCommittedVersion(); data->versionLag = std::max(0, data->lastTLogVersion - data->version.get()); ASSERT(*pReceivedUpdate == false); @@ -4677,8 +4686,9 @@ ACTOR Future updateStorage(StorageServer* data) { if (it.version > newOldestVersion) { break; } - data->storage.writeKeyValue(KeyValueRef(changeFeedDurableKey(info->second->id, it.version), - changeFeedDurableValue(it.mutations))); + data->storage.writeKeyValue( + KeyValueRef(changeFeedDurableKey(info->second->id, it.version), + changeFeedDurableValue(it.mutations, it.knownCommittedVersion))); info->second->storageVersion = it.version; } wait(yield(TaskPriority::UpdateStorage)); From 282f84c807d09eae9262d50ce9c5c93a8c98e8ef Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Tue, 12 Oct 2021 10:36:10 -0700 Subject: [PATCH 59/72] added handling for broken promise --- fdbclient/NativeAPI.actor.cpp | 3 ++- fdbserver/storageserver.actor.cpp | 8 ++++---- 2 files changed, 6 insertions(+), 5 deletions(-) diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index 1119f41e35..7a2d21f0b4 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -6950,7 +6950,8 @@ ACTOR Future getChangeFeedStreamActor(Reference db, throw; } if (e.code() == error_code_wrong_shard_server || e.code() == error_code_all_alternatives_failed || - e.code() == error_code_connection_failed || e.code() == error_code_unknown_change_feed) { + e.code() == error_code_connection_failed || e.code() == error_code_unknown_change_feed || + e.code() == error_code_broken_promise) { cx->invalidateCache(keys); wait(delay(CLIENT_KNOBS->WRONG_SHARD_SERVER_DELAY)); } else { diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index a13f5d8910..97d9be59fc 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -869,10 +869,10 @@ public: Histogram::Unit::microseconds)), tag(invalidTag), poppedAllAfter(std::numeric_limits::max()), cpuUsage(0.0), diskUsage(0.0), storage(this, storage), shardChangeCounter(0), lastTLogVersion(0), lastVersionWithData(0), restoredVersion(0), - knownCommittedVersion(0), rebootAfterDurableVersion(std::numeric_limits::max()), - primaryLocality(tagLocalityInvalid), versionLag(0), logProtocol(0), thisServerID(ssi.id()), - tssInQuarantine(false), db(db), actors(false), byteSampleClears(false, LiteralStringRef("\xff\xff\xff")), - durableInProgress(Void()), watchBytes(0), numWatches(0), noRecentUpdates(false), lastUpdate(now()), + rebootAfterDurableVersion(std::numeric_limits::max()), primaryLocality(tagLocalityInvalid), + knownCommittedVersion(0), versionLag(0), logProtocol(0), thisServerID(ssi.id()), tssInQuarantine(false), db(db), + actors(false), byteSampleClears(false, LiteralStringRef("\xff\xff\xff")), durableInProgress(Void()), + watchBytes(0), numWatches(0), noRecentUpdates(false), lastUpdate(now()), readQueueSizeMetric(LiteralStringRef("StorageServer.ReadQueueSize")), updateEagerReads(nullptr), fetchKeysParallelismLock(SERVER_KNOBS->FETCH_KEYS_PARALLELISM), fetchKeysBytesBudget(SERVER_KNOBS->STORAGE_FETCH_BYTES), fetchKeysBudgetUsed(false), From 3f7df58a772963c0f1762ddc4b6f7e061d5a22cd Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Tue, 19 Oct 2021 13:56:52 -0700 Subject: [PATCH 60/72] fixed a number of issues --- fdbcli/CMakeLists.txt | 1 + fdbcli/ChangeFeedCommand.actor.cpp | 172 +++++++++++ fdbcli/fdbcli.actor.cpp | 188 +----------- fdbcli/fdbcli.actor.h | 2 + fdbclient/ClientKnobs.cpp | 3 + fdbclient/ClientKnobs.h | 3 + fdbclient/DatabaseContext.h | 10 +- fdbclient/ManagementAPI.actor.cpp | 84 ++++++ fdbclient/ManagementAPI.actor.h | 7 + fdbclient/NativeAPI.actor.cpp | 198 +++++-------- fdbclient/NativeAPI.actor.h | 3 - fdbclient/StorageServerInterface.cpp | 23 -- fdbclient/StorageServerInterface.h | 40 +-- fdbclient/SystemData.cpp | 12 +- fdbclient/SystemData.h | 5 +- fdbrpc/fdbrpc.h | 3 + fdbserver/storageserver.actor.cpp | 331 ++++++++++++---------- fdbserver/workloads/ChangeFeeds.actor.cpp | 12 +- flow/Arena.h | 10 + flow/error_definitions.h | 2 + 20 files changed, 574 insertions(+), 535 deletions(-) create mode 100644 fdbcli/ChangeFeedCommand.actor.cpp diff --git a/fdbcli/CMakeLists.txt b/fdbcli/CMakeLists.txt index 32323fb788..628f917436 100644 --- a/fdbcli/CMakeLists.txt +++ b/fdbcli/CMakeLists.txt @@ -16,6 +16,7 @@ set(FDBCLI_SRCS IncludeCommand.actor.cpp KillCommand.actor.cpp LockCommand.actor.cpp + ChangeFeedCommand.actor.cpp MaintenanceCommand.actor.cpp ProfileCommand.actor.cpp SetClassCommand.actor.cpp diff --git a/fdbcli/ChangeFeedCommand.actor.cpp b/fdbcli/ChangeFeedCommand.actor.cpp new file mode 100644 index 0000000000..d28d96c367 --- /dev/null +++ b/fdbcli/ChangeFeedCommand.actor.cpp @@ -0,0 +1,172 @@ +/* + * ChangeFeedCommand.actor.cpp + * + * This source file is part of the FoundationDB open source project + * + * Copyright 2013-2021 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 "fdbcli/fdbcli.actor.h" + +#include "fdbclient/FDBOptions.g.h" +#include "fdbclient/IClientApi.h" +#include "fdbclient/Knobs.h" +#include "fdbclient/Schemas.h" +#include "fdbclient/ManagementAPI.actor.h" + +#include "flow/Arena.h" +#include "flow/FastRef.h" +#include "flow/ThreadHelper.actor.h" +#include "flow/actorcompiler.h" // This must be the last #include. + +namespace { + +ACTOR Future changeFeedList(Database db) { + state ReadYourWritesTransaction tr(db); + loop { + try { + tr.setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); + tr.setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); + + RangeResult result = wait(tr.getRange(changeFeedKeys, CLIENT_KNOBS->TOO_MANY)); + // shouldn't have many quarantined TSSes + ASSERT(!result.more); + printf("Found %d range feeds%s\n", result.size(), result.size() == 0 ? "." : ":"); + for (auto& it : result) { + auto range = std::get<0>(decodeChangeFeedValue(it.value)); + printf(" %s: %s - %s\n", + it.key.removePrefix(changeFeedPrefix).toString().c_str(), + range.begin.toString().c_str(), + range.end.toString().c_str()); + } + return Void(); + } catch (Error& e) { + wait(tr.onError(e)); + } + } +} + +} // namespace + +namespace fdb_cli { + +ACTOR Future changeFeedCommandActor(Database localDb, std::vector tokens, Future warn) { + if (tokens.size() == 1) { + printUsage(tokens[0]); + return false; + } + if (tokencmp(tokens[1], "list")) { + if (tokens.size() != 2) { + printUsage(tokens[0]); + return false; + } + wait(changeFeedList(localDb)); + return true; + } else if (tokencmp(tokens[1], "register")) { + if (tokens.size() != 5) { + printUsage(tokens[0]); + return false; + } + wait(updateChangeFeed( + localDb, tokens[2], ChangeFeedStatus::CHANGE_FEED_CREATE, KeyRangeRef(tokens[3], tokens[4]))); + } else if (tokencmp(tokens[1], "stop")) { + if (tokens.size() != 3) { + printUsage(tokens[0]); + return false; + } + wait(updateChangeFeed(localDb, tokens[2], ChangeFeedStatus::CHANGE_FEED_STOP)); + } else if (tokencmp(tokens[1], "destroy")) { + if (tokens.size() != 3) { + printUsage(tokens[0]); + return false; + } + wait(updateChangeFeed(localDb, tokens[2], ChangeFeedStatus::CHANGE_FEED_DESTROY)); + } else if (tokencmp(tokens[1], "stream")) { + if (tokens.size() < 3 || tokens.size() > 5) { + printUsage(tokens[0]); + return false; + } + Version begin = 0; + Version end = std::numeric_limits::max(); + if (tokens.size() > 3) { + int n = 0; + if (sscanf(tokens[3].toString().c_str(), "%ld%n", &begin, &n) != 1 || n != tokens[3].size()) { + printUsage(tokens[0]); + return false; + } + } + if (tokens.size() > 4) { + int n = 0; + if (sscanf(tokens[4].toString().c_str(), "%ld%n", &end, &n) != 1 || n != tokens[4].size()) { + printUsage(tokens[0]); + return false; + } + } + if (warn.isValid()) { + warn.cancel(); + } + state PromiseStream>> feedResults; + state Future feed = localDb->getChangeFeedStream(feedResults, tokens[2], begin, end); + printf("\n"); + try { + state Future feedInterrupt = LineNoise::onKeyboardInterrupt(); + loop { + choose { + when(Standalone> res = waitNext(feedResults.getFuture())) { + for (auto& it : res) { + for (auto& it2 : it.mutations) { + printf("%lld %s\n", it.version, it2.toString().c_str()); + } + } + } + when(wait(feedInterrupt)) { + feedInterrupt = Future(); + feed.cancel(); + feedResults = PromiseStream>>(); + break; + } + } + } + return true; + } catch (Error& e) { + if (e.code() == error_code_end_of_stream) { + return true; + } + throw; + } + } else if (tokencmp(tokens[1], "pop")) { + if (tokens.size() != 4) { + printUsage(tokens[0]); + return false; + } + Version v; + int n = 0; + if (sscanf(tokens[3].toString().c_str(), "%ld%n", &v, &n) != 1 || n != tokens[3].size()) { + printUsage(tokens[0]); + return false; + } else { + wait(localDb->popChangeFeedMutations(tokens[2], v)); + } + } else { + printUsage(tokens[0]); + return false; + } + return true; +} + +CommandFactory changeFeedFactory( + "changefeed", + CommandHelp("changefeed ", "", "")); +} // namespace fdb_cli diff --git a/fdbcli/fdbcli.actor.cpp b/fdbcli/fdbcli.actor.cpp index 7f839c7b4f..8056150fb0 100644 --- a/fdbcli/fdbcli.actor.cpp +++ b/fdbcli/fdbcli.actor.cpp @@ -522,8 +522,6 @@ void initHelp() { helpMap["writemode"] = CommandHelp("writemode ", "enables or disables sets and clears", "Setting or clearing keys from the CLI is not recommended."); - helpMap["changefeed"] = - CommandHelp("changefeed ", "", ""); } void printVersion() { @@ -638,31 +636,6 @@ ACTOR Future commitTransaction(Reference tr) { return Void(); } -ACTOR Future changeFeedList(Database db) { - state ReadYourWritesTransaction tr(db); - loop { - try { - tr.setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); - tr.setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); - - RangeResult result = wait(tr.getRange(changeFeedKeys, CLIENT_KNOBS->TOO_MANY)); - // shouldn't have many quarantined TSSes - ASSERT(!result.more); - printf("Found %d range feeds%s\n", result.size(), result.size() == 0 ? "." : ":"); - for (auto& it : result) { - auto range = std::get<0>(decodeChangeFeedValue(it.value)); - printf(" %s: %s - %s\n", - it.key.removePrefix(changeFeedPrefix).toString().c_str(), - range.begin.toString().c_str(), - range.end.toString().c_str()); - } - return Void(); - } catch (Error& e) { - wait(tr.onError(e)); - } - } -} - // FIXME: Factor address parsing from coordinators, include, exclude ACTOR Future coordinators(Database db, std::vector tokens, bool isClusterTLS) { state StringRef setName; @@ -1840,160 +1813,6 @@ ACTOR Future cli(CLIOptions opt, LineNoise* plinenoise) { continue; } - if (tokencmp(tokens[0], "changefeed")) { - if (tokens.size() == 1) { - printUsage(tokens[0]); - is_error = true; - continue; - } - if (tokencmp(tokens[1], "list")) { - if (tokens.size() != 2) { - printUsage(tokens[0]); - is_error = true; - continue; - } - wait(changeFeedList(localDb)); - continue; - } else if (tokencmp(tokens[1], "register")) { - if (tokens.size() != 5) { - printUsage(tokens[0]); - is_error = true; - continue; - } - trx = Transaction(localDb); - loop { - try { - wait(trx.registerChangeFeed(tokens[2], KeyRangeRef(tokens[3], tokens[4]))); - wait(trx.commit()); - break; - } catch (Error& e) { - wait(trx.onError(e)); - } - } - } else if (tokencmp(tokens[1], "destroy")) { - if (tokens.size() != 3) { - printUsage(tokens[0]); - is_error = true; - continue; - } - trx = Transaction(localDb); - loop { - try { - trx.destroyChangeFeed(tokens[2]); - wait(trx.commit()); - break; - } catch (Error& e) { - wait(trx.onError(e)); - } - } - } else if (tokencmp(tokens[1], "get")) { - if (tokens.size() < 3 || tokens.size() > 5) { - printUsage(tokens[0]); - is_error = true; - continue; - } - Version begin = 0; - Version end = std::numeric_limits::max(); - if (tokens.size() > 3) { - int n = 0; - if (sscanf(tokens[3].toString().c_str(), "%ld%n", &begin, &n) != 1 || - n != tokens[3].size()) { - printUsage(tokens[0]); - is_error = true; - continue; - } - } - if (tokens.size() > 4) { - int n = 0; - if (sscanf(tokens[4].toString().c_str(), "%ld%n", &end, &n) != 1 || n != tokens[4].size()) { - printUsage(tokens[0]); - is_error = true; - continue; - } - } - Standalone> res = - wait(localDb->getChangeFeedMutations(tokens[2], begin, end)); - printf("\n"); - for (auto& it : res) { - for (auto& it2 : it.mutations) { - printf("%lld %s\n", it.version, it2.toString().c_str()); - } - } - } else if (tokencmp(tokens[1], "stream")) { - if (tokens.size() < 3 || tokens.size() > 5) { - printUsage(tokens[0]); - is_error = true; - continue; - } - Version begin = 0; - Version end = std::numeric_limits::max(); - if (tokens.size() > 3) { - int n = 0; - if (sscanf(tokens[3].toString().c_str(), "%ld%n", &begin, &n) != 1 || - n != tokens[3].size()) { - printUsage(tokens[0]); - is_error = true; - continue; - } - } - if (tokens.size() > 4) { - int n = 0; - if (sscanf(tokens[4].toString().c_str(), "%ld%n", &end, &n) != 1 || n != tokens[4].size()) { - printUsage(tokens[0]); - is_error = true; - continue; - } - } - if (warn.isValid()) { - warn.cancel(); - } - state PromiseStream>> feedResults; - state Future feed = localDb->getChangeFeedStream(feedResults, tokens[2], begin, end); - printf("\n"); - try { - state Future feedInterrupt = LineNoise::onKeyboardInterrupt(); - loop { - choose { - when(Standalone> res = - waitNext(feedResults.getFuture())) { - for (auto& it : res) { - for (auto& it2 : it.mutations) { - printf("%lld %s\n", it.version, it2.toString().c_str()); - } - } - } - when(wait(feedInterrupt)) { - feedInterrupt = Future(); - feed.cancel(); - feedResults = PromiseStream>>(); - break; - } - } - } - continue; - } catch (Error& e) { - if (e.code() == error_code_end_of_stream) { - continue; - } - throw; - } - } else if (tokencmp(tokens[1], "pop")) { - if (tokens.size() != 4) { - printUsage(tokens[0]); - is_error = true; - continue; - } - Version v; - int n = 0; - if (sscanf(tokens[3].toString().c_str(), "%ld%n", &v, &n) != 1 || n != tokens[3].size()) { - printUsage(tokens[0]); - is_error = true; - } else { - wait(localDb->popChangeFeedMutations(tokens[2], v)); - } - } - continue; - } if (tokencmp(tokens[0], "tssq")) { bool _result = wait(makeInterruptable(tssqCommandActor(db, tokens))); if (!_result) @@ -2061,6 +1880,13 @@ ACTOR Future cli(CLIOptions opt, LineNoise* plinenoise) { continue; } + if (tokencmp(tokens[0], "changefeed")) { + bool _result = wait(makeInterruptable(changeFeedCommandActor(localDb, tokens, warn))); + if (!_result) + is_error = true; + continue; + } + if (tokencmp(tokens[0], "unlock")) { if ((tokens.size() != 2) || (tokens[1].size() != 32) || !std::all_of(tokens[1].begin(), tokens[1].end(), &isxdigit)) { diff --git a/fdbcli/fdbcli.actor.h b/fdbcli/fdbcli.actor.h index 7340b3b682..7645f9beb1 100644 --- a/fdbcli/fdbcli.actor.h +++ b/fdbcli/fdbcli.actor.h @@ -165,6 +165,8 @@ ACTOR Future killCommandActor(Reference db, // lock/unlock command ACTOR Future lockCommandActor(Reference db, std::vector tokens); ACTOR Future unlockDatabaseActor(Reference db, UID uid); +// changefeed command +ACTOR Future changeFeedCommandActor(Database localDb, std::vector tokens, Future warn); // maintenance command ACTOR Future setHealthyZone(Reference db, StringRef zoneId, double seconds, bool printWarning = false); ACTOR Future clearHealthyZone(Reference db, diff --git a/fdbclient/ClientKnobs.cpp b/fdbclient/ClientKnobs.cpp index 0b42148de8..98ccf2ea33 100644 --- a/fdbclient/ClientKnobs.cpp +++ b/fdbclient/ClientKnobs.cpp @@ -75,6 +75,9 @@ void ClientKnobs::initialize(Randomize randomize) { init( VALUE_SIZE_LIMIT, 1e5 ); init( SPLIT_KEY_SIZE_LIMIT, KEY_SIZE_LIMIT/2 ); if( randomize && BUGGIFY ) SPLIT_KEY_SIZE_LIMIT = KEY_SIZE_LIMIT - 31;//serverKeysPrefixFor(UID()).size() - 1; init( METADATA_VERSION_CACHE_SIZE, 1000 ); + init( CHANGE_FEED_LOCATION_LIMIT, 10000 ); + init( CHANGE_FEED_CACHE_SIZE, 100000 ); if( randomize && BUGGIFY ) CHANGE_FEED_CACHE_SIZE = 1; + init( CHANGE_FEED_POP_TIMEOUT, 5.0 ); init( MAX_BATCH_SIZE, 1000 ); if( randomize && BUGGIFY ) MAX_BATCH_SIZE = 1; init( GRV_BATCH_TIMEOUT, 0.005 ); if( randomize && BUGGIFY ) GRV_BATCH_TIMEOUT = 0.1; diff --git a/fdbclient/ClientKnobs.h b/fdbclient/ClientKnobs.h index dd52bd98c5..6f947d3f07 100644 --- a/fdbclient/ClientKnobs.h +++ b/fdbclient/ClientKnobs.h @@ -74,6 +74,9 @@ public: int64_t VALUE_SIZE_LIMIT; int64_t SPLIT_KEY_SIZE_LIMIT; int METADATA_VERSION_CACHE_SIZE; + int64_t CHANGE_FEED_LOCATION_LIMIT; + int64_t CHANGE_FEED_CACHE_SIZE; + double CHANGE_FEED_POP_TIMEOUT; int MAX_BATCH_SIZE; double GRV_BATCH_TIMEOUT; diff --git a/fdbclient/DatabaseContext.h b/fdbclient/DatabaseContext.h index 4466001e42..79b1e4e883 100644 --- a/fdbclient/DatabaseContext.h +++ b/fdbclient/DatabaseContext.h @@ -252,14 +252,8 @@ public: // Management API, create snapshot Future createSnapshot(StringRef uid, StringRef snapshot_command); - Future>> getChangeFeedMutations( - StringRef rangeID, - Version begin = 0, - Version end = std::numeric_limits::max(), - KeyRange range = allKeys); - Future getChangeFeedStream(const PromiseStream>>& results, - StringRef rangeID, + Key rangeID, Version begin = 0, Version end = std::numeric_limits::max(), KeyRange range = allKeys); @@ -342,6 +336,8 @@ public: std::unordered_map tssMapping; // map from tssid -> metrics for that tss pair std::unordered_map> tssMetrics; + // map from changeFeedId -> changeFeedRange + std::unordered_map changeFeedCache; UID dbId; IsInternal internal; // Only contexts created through the C client and fdbcli are non-internal diff --git a/fdbclient/ManagementAPI.actor.cpp b/fdbclient/ManagementAPI.actor.cpp index cd6ed05dab..fa9525154e 100644 --- a/fdbclient/ManagementAPI.actor.cpp +++ b/fdbclient/ManagementAPI.actor.cpp @@ -2073,6 +2073,90 @@ ACTOR Future checkDatabaseLock(Reference tr, UI return Void(); } +ACTOR Future updateChangeFeed(Transaction* tr, Key rangeID, ChangeFeedStatus status, KeyRange range) { + state Key rangeIDKey = rangeID.withPrefix(changeFeedPrefix); + tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); + + Optional val = wait(tr->get(rangeIDKey)); + if (status == ChangeFeedStatus::CHANGE_FEED_CREATE) { + if (!val.present()) { + tr->set(rangeIDKey, changeFeedValue(range, invalidVersion, status)); + } else if (std::get<0>(decodeChangeFeedValue(val.get())) != range) { + throw unsupported_operation(); + } + } else if (status == ChangeFeedStatus::CHANGE_FEED_STOP) { + if (val.present()) { + tr->set(rangeIDKey, + changeFeedValue(std::get<0>(decodeChangeFeedValue(val.get())), + std::get<1>(decodeChangeFeedValue(val.get())), + status)); + } else { + throw unsupported_operation(); + } + } else if (status == ChangeFeedStatus::CHANGE_FEED_DESTROY) { + if (val.present()) { + tr->set(rangeIDKey, + changeFeedValue(std::get<0>(decodeChangeFeedValue(val.get())), + std::get<1>(decodeChangeFeedValue(val.get())), + status)); + tr->clear(rangeIDKey); + } else { + throw unsupported_operation(); + } + } + return Void(); +} + +ACTOR Future updateChangeFeed(Reference tr, + Key rangeID, + ChangeFeedStatus status, + KeyRange range) { + state Key rangeIDKey = rangeID.withPrefix(changeFeedPrefix); + tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); + + Optional val = wait(tr->get(rangeIDKey)); + if (status == ChangeFeedStatus::CHANGE_FEED_CREATE) { + if (!val.present()) { + tr->set(rangeIDKey, changeFeedValue(range, invalidVersion, status)); + } else if (std::get<0>(decodeChangeFeedValue(val.get())) != range) { + throw unsupported_operation(); + } + } else if (status == ChangeFeedStatus::CHANGE_FEED_STOP) { + if (val.present()) { + tr->set(rangeIDKey, + changeFeedValue(std::get<0>(decodeChangeFeedValue(val.get())), + std::get<1>(decodeChangeFeedValue(val.get())), + status)); + } else { + throw unsupported_operation(); + } + } else if (status == ChangeFeedStatus::CHANGE_FEED_DESTROY) { + if (val.present()) { + tr->set(rangeIDKey, + changeFeedValue(std::get<0>(decodeChangeFeedValue(val.get())), + std::get<1>(decodeChangeFeedValue(val.get())), + status)); + tr->clear(rangeIDKey); + } else { + throw unsupported_operation(); + } + } + return Void(); +} + +ACTOR Future updateChangeFeed(Database cx, Key rangeID, ChangeFeedStatus status, KeyRange range) { + state Transaction tr(cx); + loop { + try { + wait(updateChangeFeed(&tr, rangeID, status, range)); + wait(tr.commit()); + return Void(); + } catch (Error& e) { + wait(tr.onError(e)); + } + } +} + ACTOR Future advanceVersion(Database cx, Version v) { state Transaction tr(cx); loop { diff --git a/fdbclient/ManagementAPI.actor.h b/fdbclient/ManagementAPI.actor.h index 021dfde2bf..74dbefce37 100644 --- a/fdbclient/ManagementAPI.actor.h +++ b/fdbclient/ManagementAPI.actor.h @@ -207,6 +207,13 @@ ACTOR Future unlockDatabase(Database cx, UID id); ACTOR Future checkDatabaseLock(Transaction* tr, UID id); ACTOR Future checkDatabaseLock(Reference tr, UID id); +ACTOR Future updateChangeFeed(Transaction* tr, Key rangeID, ChangeFeedStatus status, KeyRange range = KeyRange()); +ACTOR Future updateChangeFeed(Reference tr, + Key rangeID, + ChangeFeedStatus status, + KeyRange range = KeyRange()); +ACTOR Future updateChangeFeed(Database cx, Key rangeID, ChangeFeedStatus status, KeyRange range = KeyRange()); + ACTOR Future advanceVersion(Database cx, Version v); ACTOR Future setDDMode(Database cx, int mode); diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index 7a2d21f0b4..212ca3d74d 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -4428,25 +4428,6 @@ Future>> Transaction::getAddressesForKey(const return getAddressesForKeyActor(key, ver, cx, info, options); } -ACTOR Future registerChangeFeedActor(Transaction* tr, Key rangeID, KeyRange range) { - state Key rangeIDKey = rangeID.withPrefix(changeFeedPrefix); - Optional val = wait(tr->get(rangeIDKey)); - if (!val.present()) { - tr->set(rangeIDKey, changeFeedValue(range, invalidVersion, false)); - } else if (std::get<0>(decodeChangeFeedValue(val.get())) != range) { - throw unsupported_operation(); - } - return Void(); -} - -Future Transaction::registerChangeFeed(const Key& rangeID, const KeyRange& range) { - return registerChangeFeedActor(this, rangeID, range); -} - -void Transaction::destroyChangeFeed(const Key& rangeID) { - clear(rangeID.withPrefix(changeFeedPrefix)); -} - ACTOR Future getKeyAndConflictRange(Database cx, KeySelector k, Future version, @@ -6672,56 +6653,6 @@ Future DatabaseContext::createSnapshot(StringRef uid, StringRef snapshot_c return createSnapshotActor(this, UID::fromString(uid_str), snapshot_command); } -ACTOR Future>> getChangeFeedMutationsActor(Reference db, - StringRef rangeID, - Version begin, - Version end, - KeyRange range) { - // FIXME: this function is out of date! - state Database cx(db); - state Transaction tr(cx); - state Key rangeIDKey = rangeID.withPrefix(changeFeedPrefix); - state Span span("NAPI:GetChangeFeedMutations"_loc); - Optional val = wait(tr.get(rangeIDKey)); - if (!val.present()) { - throw unsupported_operation(); - } - state KeyRange keys = std::get<0>(decodeChangeFeedValue(val.get())) & range; - state std::vector>> locations = - wait(getKeyRangeLocations(cx, - keys, - 100, - Reverse::False, - &StorageServerInterface::changeFeed, - TransactionInfo(TaskPriority::DefaultEndpoint, span.context))); - - if (locations.size() > 1) { - throw unsupported_operation(); - } - - state ChangeFeedRequest req; - req.rangeID = rangeID; - req.begin = begin; - req.end = end; - req.range = keys; - - ChangeFeedReply rep = wait(loadBalance(cx.getPtr(), - locations[0].second, - &StorageServerInterface::changeFeed, - req, - TaskPriority::DefaultPromiseEndpoint, - AtMostOnce::False, - cx->enableLocalityLoadBalance ? &cx->queueModel : nullptr)); - return Standalone>(rep.mutations, rep.arena); -} - -Future>> DatabaseContext::getChangeFeedMutations(StringRef rangeID, - Version begin, - Version end, - KeyRange range) { - return getChangeFeedMutationsActor(Reference::addRef(this), rangeID, begin, end, range); -} - ACTOR Future singleChangeFeedStream(StorageServerInterface interf, PromiseStream> results, Key rangeID, @@ -6827,50 +6758,63 @@ ACTOR Future mergeChangeFeedStream(std::vector getChangeFeedRange(Reference db, Database cx, Key rangeID, Version begin = 0) { + state Transaction tr(cx); + state Key rangeIDKey = rangeID.withPrefix(changeFeedPrefix); + + auto cacheLoc = db->changeFeedCache.find(rangeID); + if (cacheLoc != db->changeFeedCache.end()) { + return cacheLoc->second; + } + + loop { + try { + Version readVer = wait(tr.getReadVersion()); + if (readVer < begin) { + wait(delay(FLOW_KNOBS->PREVENT_FAST_SPIN_DELAY)); + } else { + Optional val = wait(tr.get(rangeIDKey)); + if (!val.present()) { + throw change_feed_not_registered(); + } + if (db->changeFeedCache.size() > CLIENT_KNOBS->CHANGE_FEED_CACHE_SIZE) { + db->changeFeedCache.clear(); + } + KeyRange range = std::get<0>(decodeChangeFeedValue(val.get())); + db->changeFeedCache[rangeID] = range; + return range; + } + } catch (Error& e) { + wait(tr.onError(e)); + } + } +} + ACTOR Future getChangeFeedStreamActor(Reference db, PromiseStream>> results, - StringRef rangeID, + Key rangeID, Version begin, Version end, KeyRange range) { state Database cx(db); - state Key rangeIDKey = rangeID.withPrefix(changeFeedPrefix); state Span span("NAPI:GetChangeFeedStream"_loc); - state KeyRange keys; loop { - state Transaction tr(cx); - loop { - try { - Version readVer = wait(tr.getReadVersion()); - if (readVer < begin) { - wait(delay(FLOW_KNOBS->PREVENT_FAST_SPIN_DELAY)); - } else { - Optional val = wait(tr.get(rangeIDKey)); - if (!val.present()) { - results.sendError(unsupported_operation()); - return Void(); - } - keys = std::get<0>(decodeChangeFeedValue(val.get())) & range; - break; - } - } catch (Error& e) { - wait(tr.onError(e)); - } - } - + state KeyRange keys; try { + KeyRange fullRange = wait(getChangeFeedRange(db, cx, rangeID, begin)); + keys = fullRange & range; state std::vector>> locations = wait(getKeyRangeLocations(cx, keys, - 1000, + CLIENT_KNOBS->CHANGE_FEED_LOCATION_LIMIT, Reverse::False, - &StorageServerInterface::changeFeed, + &StorageServerInterface::changeFeedStream, TransactionInfo(TaskPriority::DefaultEndpoint, span.context))); - if (locations.size() >= 1000) { - ASSERT(false); - results.sendError(unsupported_operation()); + if (locations.size() >= CLIENT_KNOBS->CHANGE_FEED_LOCATION_LIMIT) { + ASSERT_WE_THINK(false); + results.sendError(change_feed_too_large()); return Void(); } @@ -6952,6 +6896,7 @@ ACTOR Future getChangeFeedStreamActor(Reference db, if (e.code() == error_code_wrong_shard_server || e.code() == error_code_all_alternatives_failed || e.code() == error_code_connection_failed || e.code() == error_code_unknown_change_feed || e.code() == error_code_broken_promise) { + db->changeFeedCache.erase(rangeID); cx->invalidateCache(keys); wait(delay(CLIENT_KNOBS->WRONG_SHARD_SERVER_DELAY)); } else { @@ -6964,7 +6909,7 @@ ACTOR Future getChangeFeedStreamActor(Reference db, Future DatabaseContext::getChangeFeedStream( const PromiseStream>>& results, - StringRef rangeID, + Key rangeID, Version begin, Version end, KeyRange range) { @@ -7006,13 +6951,17 @@ ACTOR Future>> getOverlappingChangeFeedsAct state std::vector>> locations = wait(getKeyRangeLocations(cx, range, - 1000, + CLIENT_KNOBS->CHANGE_FEED_LOCATION_LIMIT, Reverse::False, &StorageServerInterface::overlappingChangeFeeds, TransactionInfo(TaskPriority::DefaultEndpoint, span.context))); - if (locations.size() >= 1000) { - throw unsupported_operation(); + if (locations.size() >= CLIENT_KNOBS->CHANGE_FEED_LOCATION_LIMIT) { + TraceEvent(SevError, "OverlappingRangeTooLarge") + .detail("Range", range) + .detail("Limit", CLIENT_KNOBS->CHANGE_FEED_LOCATION_LIMIT); + wait(delay(1.0)); + throw all_alternatives_failed(); } state std::vector>>> allOverlappingRequests; @@ -7054,13 +7003,13 @@ ACTOR static Future popChangeFeedBackup(Database cx, StringRef rangeID, Ve if (val.present()) { KeyRange range; Version popVersion; - bool stopped; - std::tie(range, popVersion, stopped) = decodeChangeFeedValue(val.get()); + ChangeFeedStatus status; + std::tie(range, popVersion, status) = decodeChangeFeedValue(val.get()); if (version > popVersion) { - tr.set(rangeIDKey, changeFeedValue(range, invalidVersion, stopped)); + tr.set(rangeIDKey, changeFeedValue(range, version, status)); } } else { - throw unsupported_operation(); + throw change_feed_not_registered(); } wait(tr.commit()); return Void(); @@ -7075,26 +7024,14 @@ ACTOR Future popChangeFeedMutationsActor(Reference db, St state Key rangeIDKey = rangeID.withPrefix(changeFeedPrefix); state Span span("NAPI:PopChangeFeedMutations"_loc); - state Transaction tr(cx); - state KeyRange keys; - loop { - try { - Optional val = wait(tr.get(rangeIDKey)); - if (!val.present()) { - throw unsupported_operation(); - } - keys = std::get<0>(decodeChangeFeedValue(val.get())); - break; - } catch (Error& e) { - wait(tr.onError(e)); - } - } + state KeyRange keys = wait(getChangeFeedRange(db, cx, rangeID)); + state std::vector>> locations = wait(getKeyRangeLocations(cx, keys, 3, Reverse::False, - &StorageServerInterface::changeFeed, + &StorageServerInterface::changeFeedPop, TransactionInfo(TaskPriority::DefaultEndpoint, span.context))); if (locations.size() > 2) { @@ -7102,6 +7039,22 @@ ACTOR Future popChangeFeedMutationsActor(Reference db, St return Void(); } + bool foundFailed = false; + for (int i = 0; i < locations.size() && !foundFailed; i++) { + for (int j = 0; j < locations[i].second->size() && !foundFailed; j++) { + if (IFailureMonitor::failureMonitor() + .getState(locations[i].second->get(j, &StorageServerInterface::changeFeedPop).getEndpoint()) + .isFailed()) { + foundFailed = true; + } + } + } + + if (foundFailed) { + wait(popChangeFeedBackup(cx, rangeID, version)); + return Void(); + } + // FIXME: lookup both the src and dest shards as of the pop version to ensure all locations are popped state std::vector> popRequests; for (int i = 0; i < locations.size(); i++) { @@ -7114,13 +7067,16 @@ ACTOR Future popChangeFeedMutationsActor(Reference db, St try { choose { when(wait(waitForAll(popRequests))) {} - when(wait(delay(5.0))) { wait(popChangeFeedBackup(cx, rangeID, version)); } + when(wait(delay(CLIENT_KNOBS->CHANGE_FEED_POP_TIMEOUT))) { + wait(popChangeFeedBackup(cx, rangeID, version)); + } } } catch (Error& e) { if (e.code() != error_code_unknown_change_feed && e.code() != error_code_wrong_shard_server && e.code() != error_code_all_alternatives_failed) { throw; } + db->changeFeedCache.erase(rangeID); cx->invalidateCache(keys); wait(popChangeFeedBackup(cx, rangeID, version)); } diff --git a/fdbclient/NativeAPI.actor.h b/fdbclient/NativeAPI.actor.h index 011dc2e8a9..6175205481 100644 --- a/fdbclient/NativeAPI.actor.h +++ b/fdbclient/NativeAPI.actor.h @@ -330,9 +330,6 @@ public: [[nodiscard]] Future>> getAddressesForKey(const Key& key); - Future registerChangeFeed(const Key& rangeID, const KeyRange& range); - void destroyChangeFeed(const Key& rangeID); - void enableCheckWrites(); void addReadConflictRange(KeyRangeRef const& keys); void addWriteConflictRange(KeyRangeRef const& keys); diff --git a/fdbclient/StorageServerInterface.cpp b/fdbclient/StorageServerInterface.cpp index b4a22a60d6..efca15289c 100644 --- a/fdbclient/StorageServerInterface.cpp +++ b/fdbclient/StorageServerInterface.cpp @@ -271,26 +271,6 @@ void TSS_traceMismatch(TraceEvent& event, } // change feed -template <> -bool TSS_doCompare(const ChangeFeedReply& src, const ChangeFeedReply& tss) { - ASSERT(false); - return true; -} - -template <> -const char* TSS_mismatchTraceName(const ChangeFeedRequest& req) { - ASSERT(false); - return ""; -} - -template <> -void TSS_traceMismatch(TraceEvent& event, - const ChangeFeedRequest& req, - const ChangeFeedReply& src, - const ChangeFeedReply& tss) { - ASSERT(false); -} - template <> bool TSS_doCompare(const OverlappingChangeFeedsReply& src, const OverlappingChangeFeedsReply& tss) { ASSERT(false); @@ -372,9 +352,6 @@ void TSSMetrics::recordLatency(const SplitRangeRequest& req, double ssLatency, d template <> void TSSMetrics::recordLatency(const GetKeyValuesStreamRequest& req, double ssLatency, double tssLatency) {} -template <> -void TSSMetrics::recordLatency(const ChangeFeedRequest& req, double ssLatency, double tssLatency) {} - template <> void TSSMetrics::recordLatency(const OverlappingChangeFeedsRequest& req, double ssLatency, double tssLatency) {} diff --git a/fdbclient/StorageServerInterface.h b/fdbclient/StorageServerInterface.h index ed4b2a5dbd..9ec48c59c9 100644 --- a/fdbclient/StorageServerInterface.h +++ b/fdbclient/StorageServerInterface.h @@ -78,7 +78,6 @@ struct StorageServerInterface { RequestStream getReadHotRanges; RequestStream getRangeSplitPoints; RequestStream getKeyValuesStream; - RequestStream changeFeed; RequestStream changeFeedStream; RequestStream overlappingChangeFeeds; RequestStream changeFeedPop; @@ -124,13 +123,12 @@ struct StorageServerInterface { RequestStream(getValue.getEndpoint().getAdjustedEndpoint(12)); getKeyValuesStream = RequestStream(getValue.getEndpoint().getAdjustedEndpoint(13)); - changeFeed = RequestStream(getValue.getEndpoint().getAdjustedEndpoint(14)); changeFeedStream = - RequestStream(getValue.getEndpoint().getAdjustedEndpoint(15)); + RequestStream(getValue.getEndpoint().getAdjustedEndpoint(14)); overlappingChangeFeeds = - RequestStream(getValue.getEndpoint().getAdjustedEndpoint(16)); + RequestStream(getValue.getEndpoint().getAdjustedEndpoint(15)); changeFeedPop = - RequestStream(getValue.getEndpoint().getAdjustedEndpoint(17)); + RequestStream(getValue.getEndpoint().getAdjustedEndpoint(16)); } } else { ASSERT(Ar::isDeserializing); @@ -173,7 +171,6 @@ struct StorageServerInterface { streams.push_back(getReadHotRanges.getReceiver()); streams.push_back(getRangeSplitPoints.getReceiver()); streams.push_back(getKeyValuesStream.getReceiver(TaskPriority::LoadBalancedEndpoint)); - streams.push_back(changeFeed.getReceiver()); streams.push_back(changeFeedStream.getReceiver()); streams.push_back(overlappingChangeFeeds.getReceiver()); streams.push_back(changeFeedPop.getReceiver()); @@ -666,43 +663,12 @@ struct MutationsAndVersionRef { } }; -struct ChangeFeedReply { - constexpr static FileIdentifier file_identifier = 11815134; - VectorRef mutations; - bool cached; - Arena arena; - - ChangeFeedReply() : cached(false) {} - - template - void serialize(Ar& ar) { - serializer(ar, mutations, arena); - } -}; -struct ChangeFeedRequest { - constexpr static FileIdentifier file_identifier = 10726174; - Key rangeID; - Version begin = 0; - Version end = 0; - KeyRange range; - ReplyPromise reply; - - ChangeFeedRequest() {} - explicit ChangeFeedRequest(Key const& rangeID) : rangeID(rangeID) {} - - template - void serialize(Ar& ar) { - serializer(ar, rangeID, begin, end, range, reply); - } -}; - struct ChangeFeedStreamReply : public ReplyPromiseStreamReply { constexpr static FileIdentifier file_identifier = 1783066; Arena arena; VectorRef mutations; ChangeFeedStreamReply() {} - ChangeFeedStreamReply(ChangeFeedReply r) : arena(r.arena), mutations(r.mutations) {} int expectedSize() const { return sizeof(ChangeFeedStreamReply) + mutations.expectedSize(); } diff --git a/fdbclient/SystemData.cpp b/fdbclient/SystemData.cpp index 7676dea4a7..334d0b4b7f 100644 --- a/fdbclient/SystemData.cpp +++ b/fdbclient/SystemData.cpp @@ -1036,23 +1036,23 @@ const KeyRangeRef changeFeedKeys(LiteralStringRef("\xff\x02/feed/"), LiteralStri const KeyRef changeFeedPrefix = changeFeedKeys.begin; const KeyRef changeFeedPrivatePrefix = LiteralStringRef("\xff\xff\x02/feed/"); -const Value changeFeedValue(KeyRangeRef const& range, Version popVersion, bool stopped) { +const Value changeFeedValue(KeyRangeRef const& range, Version popVersion, ChangeFeedStatus status) { BinaryWriter wr(IncludeVersion(ProtocolVersion::withChangeFeed())); wr << range; wr << popVersion; - wr << stopped; + wr << status; return wr.toValue(); } -std::tuple decodeChangeFeedValue(ValueRef const& value) { +std::tuple decodeChangeFeedValue(ValueRef const& value) { KeyRange range; Version version; - bool stopped; + ChangeFeedStatus status; BinaryReader reader(value, IncludeVersion()); reader >> range; reader >> version; - reader >> stopped; - return std::make_tuple(range, version, stopped); + reader >> status; + return std::make_tuple(range, version, status); } const KeyRangeRef changeFeedDurableKeys(LiteralStringRef("\xff\xff/cf/"), LiteralStringRef("\xff\xff/cf0")); diff --git a/fdbclient/SystemData.h b/fdbclient/SystemData.h index e7039764bd..73ac1a39c9 100644 --- a/fdbclient/SystemData.h +++ b/fdbclient/SystemData.h @@ -497,8 +497,9 @@ extern const ValueRef writeRecoveryKeyTrue; extern const KeyRef snapshotEndVersionKey; extern const KeyRangeRef changeFeedKeys; -const Value changeFeedValue(KeyRangeRef const& range, Version popVersion, bool stopped); -std::tuple decodeChangeFeedValue(ValueRef const& value); +enum class ChangeFeedStatus { CHANGE_FEED_CREATE = 0, CHANGE_FEED_STOP = 1, CHANGE_FEED_DESTROY = 2 }; +const Value changeFeedValue(KeyRangeRef const& range, Version popVersion, ChangeFeedStatus status); +std::tuple decodeChangeFeedValue(ValueRef const& value); extern const KeyRef changeFeedPrefix; extern const KeyRef changeFeedPrivatePrefix; diff --git a/fdbrpc/fdbrpc.h b/fdbrpc/fdbrpc.h index bb23b9f21a..8a7fde0e9b 100644 --- a/fdbrpc/fdbrpc.h +++ b/fdbrpc/fdbrpc.h @@ -442,6 +442,9 @@ public: template void sendError(const E& exc) const { if (queue->isRemoteEndpoint() && !queue->sentError) { + if (queue->acknowledgements.failures.isError()) { + throw queue->acknowledgements.failures.getError(); + } queue->sentError = true; FlowTransport::transport().sendUnreliable( SerializeSource>>(exc), getEndpoint(), false); diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index 97d9be59fc..fa2ddd35a9 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -318,12 +318,14 @@ struct FetchInjectionInfo { struct ChangeFeedInfo : ReferenceCounted { std::deque> mutations; - Version storageVersion = invalidVersion; - Version durableVersion = invalidVersion; - Version emptyVersion = 0; + Version storageVersion = invalidVersion; // The version between the storage version and the durable version are + // currently being written to disk + Version durableVersion = invalidVersion; // All versions before the durable version are durable on disk + Version emptyVersion = 0; // The change feed does not have any mutations before emptyVersion KeyRange range; Key id; AsyncTrigger newMutations; + bool stopped = false; // A stopped change feed no longer adds new mutations, but is still queriable }; class ServerWatchMetadata : public ReferenceCounted { @@ -1547,7 +1549,7 @@ ACTOR Future watchValueSendReply(StorageServer* data, ACTOR Future changeFeedPopQ(StorageServer* self, ChangeFeedPopRequest req) { wait(delay(0)); - TraceEvent("ChangeFeedPopQuery", self->thisServerID) + TraceEvent(SevDebug, "ChangeFeedPopQuery", self->thisServerID) .detail("RangeID", req.rangeID.printable()) .detail("Version", req.version) .detail("Range", req.range.toString()); @@ -1605,11 +1607,59 @@ ACTOR Future overlappingChangeFeedsQ(StorageServer* data, OverlappingChang return Void(); } -MutationsAndVersionRef filterMutations(Arena& arena, MutationsAndVersionRef const& m, KeyRange const& range) { +MutationsAndVersionRef filterMutationsInverted(Arena& arena, MutationsAndVersionRef const& m, KeyRange const& range) { + Optional> modifiedMutations; + for (int i = 0; i < m.mutations.size(); i++) { + if (m.mutations[i].type == MutationRef::SetValue) { + if (modifiedMutations.present() && !range.contains(m.mutations[i].param1)) { + modifiedMutations.get().push_back(arena, m.mutations[i]); + } + if (!modifiedMutations.present() && range.contains(m.mutations[i].param1)) { + modifiedMutations = m.mutations.slice(0, i); + arena.dependsOn(range.arena()); + } + } else { + ASSERT(m.mutations[i].type == MutationRef::ClearRange); + if (!modifiedMutations.present() && + ((m.mutations[i].param1 < range.begin && m.mutations[i].param2 > range.begin) || + (m.mutations[i].param2 > range.end && m.mutations[i].param1 < range.end))) { + modifiedMutations = m.mutations.slice(0, i); + arena.dependsOn(range.arena()); + } + if (modifiedMutations.present()) { + if (m.mutations[i].param1 < range.begin) { + modifiedMutations.get().push_back(arena, + MutationRef(MutationRef::ClearRange, + m.mutations[i].param1, + std::min(range.begin, m.mutations[i].param2))); + } + if (m.mutations[i].param2 > range.end) { + modifiedMutations.get().push_back(arena, + MutationRef(MutationRef::ClearRange, + std::max(range.end, m.mutations[i].param1), + m.mutations[i].param2)); + } + } + } + } + if (modifiedMutations.present()) { + return MutationsAndVersionRef(modifiedMutations.get(), m.version, m.knownCommittedVersion); + } + return m; +} + +MutationsAndVersionRef filterMutations(Arena& arena, + MutationsAndVersionRef const& m, + KeyRange const& range, + bool inverted) { if (m.mutations.size() == 1 && m.mutations.back().param1 == lastEpochEndPrivateKey) { return m; } + if (inverted) { + return filterMutationsInverted(arena, m, range); + } + Optional> modifiedMutations; for (int i = 0; i < m.mutations.size(); i++) { if (m.mutations[i].type == MutationRef::SetValue) { @@ -1643,9 +1693,13 @@ MutationsAndVersionRef filterMutations(Arena& arena, MutationsAndVersionRef cons return m; } -ACTOR Future getChangeFeedMutations(StorageServer* data, ChangeFeedRequest req) { - state ChangeFeedReply reply; +ACTOR Future getChangeFeedMutations(StorageServer* data, + ChangeFeedStreamRequest req, + bool inverted) { + state ChangeFeedStreamReply reply; + state ChangeFeedStreamReply memoryReply; state int remainingLimitBytes = CLIENT_KNOBS->REPLY_BYTE_LIMIT; + state int remainingDurableBytes = CLIENT_KNOBS->REPLY_BYTE_LIMIT; wait(delay(0, TaskPriority::DefaultEndpoint)); if (data->version.get() < req.begin) { wait(data->version.whenAtLeast(req.begin)); @@ -1659,35 +1713,38 @@ ACTOR Future getChangeFeedMutations(StorageServer* data, Change if (feed == data->uidChangeFeed.end()) { throw unknown_change_feed(); } + + // We must copy the mutationDeque when fetching the durable bytes in case mutations are popped from memory while + // waiting for the results state Version dequeVersion = data->version.get(); state Version dequeKnownCommit = data->knownCommittedVersion; - if (req.end <= feed->second->emptyVersion + 1) { - } else if (feed->second->durableVersion == invalidVersion || req.begin > feed->second->durableVersion) { + + if (req.end > feed->second->emptyVersion + 1) { for (auto& it : feed->second->mutations) { if (it.version >= req.end || remainingLimitBytes <= 0) { break; } if (it.version >= req.begin) { - reply.arena.dependsOn(it.arena()); - auto m = filterMutations(reply.arena, it, req.range); - reply.mutations.push_back(reply.arena, m); + memoryReply.arena.dependsOn(it.arena()); + auto m = filterMutations(memoryReply.arena, it, req.range, inverted); + memoryReply.mutations.push_back(memoryReply.arena, m); remainingLimitBytes -= sizeof(MutationsAndVersionRef) + m.expectedSize(); } } - } else { - state std::deque> mutationsDeque = feed->second->mutations; - state Version startingDurableVersion = feed->second->durableVersion; + } + if (req.end > feed->second->emptyVersion + 1 && feed->second->durableVersion != invalidVersion && + req.begin <= feed->second->durableVersion) { RangeResult res = wait(data->storage.readRange( - KeyRangeRef(changeFeedDurableKey(req.rangeID, req.begin), changeFeedDurableKey(req.rangeID, req.end)), + KeyRangeRef(changeFeedDurableKey(req.rangeID, std::max(req.begin, feed->second->emptyVersion)), + changeFeedDurableKey(req.rangeID, req.end)), 1 << 30, - remainingLimitBytes)); + remainingDurableBytes)); if (!req.range.empty()) { data->checkChangeCounter(changeCounter, req.range); } - bool isEmpty = res.size() == 0; Version lastVersion = req.begin - 1; for (auto& kv : res) { Key id; @@ -1697,58 +1754,28 @@ ACTOR Future getChangeFeedMutations(StorageServer* data, Change std::tie(mutations, knownCommittedVersion) = decodeChangeFeedDurableValue(kv.value); reply.arena.dependsOn(mutations.arena()); auto m = filterMutations( - reply.arena, MutationsAndVersionRef(mutations, version, knownCommittedVersion), req.range); + reply.arena, MutationsAndVersionRef(mutations, version, knownCommittedVersion), req.range, inverted); reply.mutations.push_back(reply.arena, m); - remainingLimitBytes -= + remainingDurableBytes -= sizeof(KeyValueRef) + - kv.expectedSize(); // FIXME: this is currently tracking the size on disk rather than the reply size - // because we cannot add mutaitons from memory if there are potentially more on disk + kv.expectedSize(); // This is tracking the size on disk rather than the reply size + // because we cannot add mutations from memory if there are potentially more on disk lastVersion = version; } - for (auto& it : mutationsDeque) { - if (it.version >= req.end || remainingLimitBytes <= 0) { - break; - } - if (it.version > lastVersion) { - if (it.mutations.size()) { - isEmpty = false; - } - reply.arena.dependsOn(it.arena()); - auto m = filterMutations(reply.arena, it, req.range); - reply.mutations.push_back(reply.arena, m); - remainingLimitBytes -= sizeof(MutationsAndVersionRef) + m.expectedSize(); - } - } - if (isEmpty) { - auto feed = data->uidChangeFeed.find(req.rangeID); - if (feed != data->uidChangeFeed.end() && startingDurableVersion == feed->second->storageVersion && - req.end > startingDurableVersion) { - if (req.begin == 0) { - feed->second->durableVersion = req.end > data->storageVersion() ? invalidVersion : req.end; - } else { - RangeResult emp = wait(data->storage.readRange( - KeyRangeRef(changeFeedDurableKey(req.rangeID, 0), changeFeedDurableKey(req.rangeID, req.end)), - -1)); - if (!req.range.empty()) { - data->checkChangeCounter(changeCounter, req.range); - } - auto feed = data->uidChangeFeed.find(req.rangeID); - if (feed != data->uidChangeFeed.end() && startingDurableVersion == feed->second->storageVersion) { - if (emp.empty()) { - feed->second->durableVersion = req.end > data->storageVersion() ? invalidVersion : req.end; - } else { - Key id; - Version version; - std::tie(id, version) = decodeChangeFeedDurableKey(emp[0].key); - feed->second->durableVersion = version; - } - } - } + if (remainingDurableBytes > 0) { + reply.arena.dependsOn(memoryReply.arena); + auto it = memoryReply.mutations.begin(); + int totalCount = memoryReply.mutations.size(); + while (it != memoryReply.mutations.end() && it->version <= lastVersion) { + ++it; + --totalCount; } + reply.mutations.append(reply.arena, it, totalCount); } } Version finalVersion = std::min(req.end - 1, dequeVersion); - if ((reply.mutations.empty() || reply.mutations.back().version < finalVersion) && remainingLimitBytes > 0) { + if ((reply.mutations.empty() || reply.mutations.back().version < finalVersion) && remainingLimitBytes > 0 && + remainingDurableBytes > 0) { reply.mutations.push_back( reply.arena, MutationsAndVersionRef(finalVersion, finalVersion == dequeVersion ? dequeKnownCommit : 0)); } @@ -1759,14 +1786,16 @@ ACTOR Future localChangeFeedStream(StorageServer* data, PromiseStream> results, Key rangeID, Version begin, - Version end) { + Version end, + KeyRange range) { try { loop { - state ChangeFeedRequest feedRequest; + state ChangeFeedStreamRequest feedRequest; feedRequest.rangeID = rangeID; feedRequest.begin = begin; feedRequest.end = end; - state ChangeFeedReply feedReply = wait(getChangeFeedMutations(data, feedRequest)); + feedRequest.range = range; + state ChangeFeedStreamReply feedReply = wait(getChangeFeedMutations(data, feedRequest, true)); begin = feedReply.mutations.back().version + 1; state int resultLoc = 0; while (resultLoc < feedReply.mutations.size()) { @@ -1788,22 +1817,9 @@ ACTOR Future localChangeFeedStream(StorageServer* data, } } -ACTOR Future changeFeedQ(StorageServer* data, ChangeFeedRequest req) { - try { - ChangeFeedReply rep = wait(getChangeFeedMutations(data, req)); - req.reply.send(rep); - } catch (Error& e) { - if (!canReplyWith(e)) - throw; - req.reply.sendError(e); - } - return Void(); -} - ACTOR Future changeFeedStreamQ(StorageServer* data, ChangeFeedStreamRequest req) { state Span span("SS:getChangeFeedStream"_loc, { req.spanContext }); - state Version begin = req.begin; req.reply.setByteLimit(SERVER_KNOBS->RANGESTREAM_LIMIT_BYTES); wait(delay(0, TaskPriority::DefaultEndpoint)); @@ -1811,15 +1827,11 @@ ACTOR Future changeFeedStreamQ(StorageServer* data, ChangeFeedStreamReques try { loop { wait(req.reply.onReady()); - state ChangeFeedRequest feedRequest; - feedRequest.rangeID = req.rangeID; - feedRequest.begin = begin; - feedRequest.end = req.end; - feedRequest.range = req.range; - ChangeFeedReply feedReply = wait(getChangeFeedMutations(data, feedRequest)); + ChangeFeedStreamReply _feedReply = wait(getChangeFeedMutations(data, req, false)); + ChangeFeedStreamReply feedReply = _feedReply; - begin = feedReply.mutations.back().version + 1; - req.reply.send(ChangeFeedStreamReply(feedReply)); + req.begin = feedReply.mutations.back().version + 1; + req.reply.send(feedReply); if (feedReply.mutations.back().version == req.end - 1) { req.reply.sendError(end_of_stream()); return Void(); @@ -1827,7 +1839,8 @@ ACTOR Future changeFeedStreamQ(StorageServer* data, ChangeFeedStreamReques if (feedReply.mutations.back().mutations.empty()) { auto feed = data->uidChangeFeed.find(req.rangeID); if (feed == data->uidChangeFeed.end()) { - throw unknown_change_feed(); + req.reply.sendError(unknown_change_feed()); + return Void(); } choose { when(wait(delay(5.0, TaskPriority::DefaultEndpoint))) {} @@ -2964,11 +2977,13 @@ void applyMutation(StorageServer* self, if (!fromFetch) { for (auto& it : self->keyChangeFeed[m.param1]) { - if (it->mutations.empty() || it->mutations.back().version != version) { - it->mutations.push_back(MutationsAndVersionRef(version, self->knownCommittedVersion)); + if (!it->stopped) { + if (it->mutations.empty() || it->mutations.back().version != version) { + it->mutations.push_back(MutationsAndVersionRef(version, self->knownCommittedVersion)); + } + it->mutations.back().mutations.push_back_deep(it->mutations.back().arena(), m); + self->currentChangeFeeds.insert(it->id); } - it->mutations.back().mutations.push_back_deep(it->mutations.back().arena(), m); - self->currentChangeFeeds.insert(it->id); } } } else if (m.type == MutationRef::ClearRange) { @@ -2982,11 +2997,13 @@ void applyMutation(StorageServer* self, auto ranges = self->keyChangeFeed.intersectingRanges(KeyRangeRef(m.param1, m.param2)); for (auto& r : ranges) { for (auto& it : r.value()) { - if (it->mutations.empty() || it->mutations.back().version != version) { - it->mutations.push_back(MutationsAndVersionRef(version, self->knownCommittedVersion)); + if (!it->stopped) { + if (it->mutations.empty() || it->mutations.back().version != version) { + it->mutations.push_back(MutationsAndVersionRef(version, self->knownCommittedVersion)); + } + it->mutations.back().mutations.push_back_deep(it->mutations.back().arena(), m); + self->currentChangeFeeds.insert(it->id); } - it->mutations.back().mutations.push_back_deep(it->mutations.back().arena(), m); - self->currentChangeFeeds.insert(it->id); } } } @@ -3233,7 +3250,7 @@ ACTOR Future fetchChangeFeed(StorageServer* data, Key rangeId, KeyRange ra wait(delay(0)); // allow this actor to be cancelled by removals bool existing = data->uidChangeFeed.count(rangeId); - TraceEvent("FetchChangeFeed", data->thisServerID) + TraceEvent(SevDebug, "FetchChangeFeed", data->thisServerID) .detail("RangeID", rangeId.printable()) .detail("Range", range.toString()) .detail("Existing", existing); @@ -3249,10 +3266,11 @@ ACTOR Future fetchChangeFeed(StorageServer* data, Key rangeId, KeyRange ra } data->keyChangeFeed.coalesce(range.contents()); auto& mLV = data->addVersionToMutationLog(data->data().getLatestVersion()); - data->addMutationToMutationLog(mLV, - MutationRef(MutationRef::SetValue, - persistChangeFeedKeys.begin.toString() + rangeId.toString(), - changeFeedValue(range, invalidVersion, false))); + data->addMutationToMutationLog( + mLV, + MutationRef(MutationRef::SetValue, + persistChangeFeedKeys.begin.toString() + rangeId.toString(), + changeFeedValue(range, invalidVersion, ChangeFeedStatus::CHANGE_FEED_CREATE))); } else { changeFeedInfo = data->uidChangeFeed[rangeId]; } @@ -3283,7 +3301,10 @@ ACTOR Future fetchChangeFeed(StorageServer* data, Key rangeId, KeyRange ra } state PromiseStream> localResults; - state Future localStream = localChangeFeedStream(data, localResults, rangeId, 0, fetchVersion + 2); + + // Add 2 to fetch version to make sure the local stream will have more versions in the stream than the remote stream + // to avoid edge cases in the merge logic + state Future localStream = localChangeFeedStream(data, localResults, rangeId, 0, fetchVersion + 2, range); state Standalone localResult; Standalone _localResult = waitNext(localResults.getFuture()); @@ -3755,16 +3776,7 @@ void AddingShard::addMutation(Version version, bool fromFetch, MutationRef const if (!fromFetch) { if (mutation.type == MutationRef::SetValue) { for (auto& it : server->keyChangeFeed[mutation.param1]) { - if (it->mutations.empty() || it->mutations.back().version != version) { - it->mutations.push_back(MutationsAndVersionRef(version, server->knownCommittedVersion)); - } - it->mutations.back().mutations.push_back_deep(it->mutations.back().arena(), mutation); - server->currentChangeFeeds.insert(it->id); - } - } else if (mutation.type == MutationRef::ClearRange) { - auto ranges = server->keyChangeFeed.intersectingRanges(KeyRangeRef(mutation.param1, mutation.param2)); - for (auto& r : ranges) { - for (auto& it : r.value()) { + if (!it->stopped) { if (it->mutations.empty() || it->mutations.back().version != version) { it->mutations.push_back(MutationsAndVersionRef(version, server->knownCommittedVersion)); } @@ -3772,6 +3784,19 @@ void AddingShard::addMutation(Version version, bool fromFetch, MutationRef const server->currentChangeFeeds.insert(it->id); } } + } else if (mutation.type == MutationRef::ClearRange) { + auto ranges = server->keyChangeFeed.intersectingRanges(KeyRangeRef(mutation.param1, mutation.param2)); + for (auto& r : ranges) { + for (auto& it : r.value()) { + if (!it->stopped) { + if (it->mutations.empty() || it->mutations.back().version != version) { + it->mutations.push_back(MutationsAndVersionRef(version, server->knownCommittedVersion)); + } + it->mutations.back().mutations.push_back_deep(it->mutations.back().arena(), mutation); + server->currentChangeFeeds.insert(it->id); + } + } + } } } } else if (phase == Waiting) { @@ -4101,19 +4126,19 @@ private: data->primaryLocality = BinaryReader::fromStringRef(m.param2, Unversioned()); auto& mLV = data->addVersionToMutationLog(data->data().getLatestVersion()); data->addMutationToMutationLog(mLV, MutationRef(MutationRef::SetValue, persistPrimaryLocality, m.param2)); - } else if ((m.type == MutationRef::SetValue || m.type == MutationRef::ClearRange) && - m.param1.startsWith(changeFeedPrivatePrefix)) { - if (m.type == MutationRef::SetValue) { - Key changeFeedId = m.param1.removePrefix(changeFeedPrivatePrefix); - KeyRange changeFeedRange; - Version popVersion; - bool stopped; - std::tie(changeFeedRange, popVersion, stopped) = decodeChangeFeedValue(m.param2); - auto feed = data->uidChangeFeed.find(changeFeedId); - if (feed == data->uidChangeFeed.end()) { - TraceEvent("AddingChangeFeed", data->thisServerID) + } else if (m.type == MutationRef::SetValue && m.param1.startsWith(changeFeedPrivatePrefix)) { + Key changeFeedId = m.param1.removePrefix(changeFeedPrivatePrefix); + KeyRange changeFeedRange; + Version popVersion; + ChangeFeedStatus status; + std::tie(changeFeedRange, popVersion, status) = decodeChangeFeedValue(m.param2); + auto feed = data->uidChangeFeed.find(changeFeedId); + if (feed == data->uidChangeFeed.end()) { + if (status == ChangeFeedStatus::CHANGE_FEED_CREATE) { + TraceEvent(SevDebug, "AddingChangeFeed", data->thisServerID) .detail("RangeID", changeFeedId.printable()) - .detail("Range", changeFeedRange.toString()); + .detail("Range", changeFeedRange.toString()) + .detail("Version", currentVersion); Reference changeFeedInfo(new ChangeFeedInfo()); changeFeedInfo->range = changeFeedRange; changeFeedInfo->id = changeFeedId; @@ -4131,6 +4156,27 @@ private: MutationRef(MutationRef::SetValue, persistChangeFeedKeys.begin.toString() + changeFeedId.toString(), m.param2)); + } + } else { + if (status == ChangeFeedStatus::CHANGE_FEED_DESTROY) { + Key beginClearKey = changeFeedId.withPrefix(persistChangeFeedKeys.begin); + auto& mLV = data->addVersionToMutationLog(data->data().getLatestVersion()); + data->addMutationToMutationLog( + mLV, MutationRef(MutationRef::ClearRange, beginClearKey, keyAfter(beginClearKey))); + data->addMutationToMutationLog(mLV, + MutationRef(MutationRef::ClearRange, + changeFeedDurableKey(feed->second->id, 0), + changeFeedDurableKey(feed->second->id, currentVersion))); + auto rs = data->keyChangeFeed.modify(feed->second->range); + for (auto r = rs.begin(); r != rs.end(); ++r) { + auto& feedList = r->value(); + for (int i = 0; i < feedList.size(); i++) { + if (feedList[i] == feed->second) { + swapAndPop(&feedList, i--); + } + } + } + data->uidChangeFeed.erase(feed); } else { if (popVersion != invalidVersion && popVersion - 1 > feed->second->emptyVersion) { feed->second->emptyVersion = popVersion - 1; @@ -4147,18 +4193,14 @@ private: } } } + feed->second->stopped = (status == ChangeFeedStatus::CHANGE_FEED_STOP); + auto& mLV = data->addVersionToMutationLog(data->data().getLatestVersion()); + data->addMutationToMutationLog( + mLV, + MutationRef(MutationRef::SetValue, + persistChangeFeedKeys.begin.toString() + changeFeedId.toString(), + m.param2)); } - } else { - auto& mLV = data->addVersionToMutationLog(data->data().getLatestVersion()); - auto beginFeed = m.param1.removePrefix(changeFeedPrivatePrefix); - auto endFeed = m.param2.removePrefix(changeFeedPrivatePrefix); - data->addMutationToMutationLog( - mLV, - MutationRef(MutationRef::ClearRange, - persistChangeFeedKeys.begin.toString() + beginFeed.toString(), - persistChangeFeedKeys.begin.toString() + endFeed.toString())); - data->uidChangeFeed.erase(data->uidChangeFeed.lower_bound(beginFeed), - data->uidChangeFeed.lower_bound(endFeed)); } } else if (m.param1.substr(1).startsWith(tssMappingKeys.begin) && (m.type == MutationRef::SetValue || m.type == MutationRef::ClearRange)) { @@ -5118,15 +5160,22 @@ ACTOR Future restoreDurableState(StorageServer* data, IKeyValueStore* stor state int feedLoc; for (feedLoc = 0; feedLoc < changeFeeds.size(); feedLoc++) { Key changeFeedId = changeFeeds[feedLoc].key.removePrefix(persistChangeFeedKeys.begin); - KeyRange changeFeedRange = std::get<0>(decodeChangeFeedValue(changeFeeds[feedLoc].value)); - TraceEvent("RestoringChangeFeed", data->thisServerID) + KeyRange changeFeedRange; + Version popVersion; + ChangeFeedStatus status; + std::tie(changeFeedRange, popVersion, status) = decodeChangeFeedValue(changeFeeds[feedLoc].value); + TraceEvent(SevDebug, "RestoringChangeFeed", data->thisServerID) .detail("RangeID", changeFeedId.printable()) - .detail("Range", changeFeedRange.toString()); + .detail("Range", changeFeedRange.toString()) + .detail("Status", status) + .detail("PopVer", popVersion); Reference changeFeedInfo(new ChangeFeedInfo()); changeFeedInfo->range = changeFeedRange; changeFeedInfo->id = changeFeedId; changeFeedInfo->durableVersion = version; changeFeedInfo->storageVersion = version; + changeFeedInfo->emptyVersion = popVersion - 1; + changeFeedInfo->stopped = status == ChangeFeedStatus::CHANGE_FEED_STOP; data->uidChangeFeed[changeFeedId] = changeFeedInfo; auto rs = data->keyChangeFeed.modify(changeFeedRange); for (auto r = rs.begin(); r != rs.end(); ++r) { @@ -5649,13 +5698,6 @@ ACTOR Future serveWatchValueRequests(StorageServer* self, FutureStream serveChangeFeedRequests(StorageServer* self, FutureStream changeFeed) { - loop { - ChangeFeedRequest req = waitNext(changeFeed); - self->actors.add(self->readGuard(req, changeFeedQ)); - } -} - ACTOR Future serveChangeFeedStreamRequests(StorageServer* self, FutureStream changeFeedStream) { loop { @@ -5729,7 +5771,6 @@ ACTOR Future storageServerCore(StorageServer* self, StorageServerInterface self->actors.add(serveGetKeyValuesStreamRequests(self, ssi.getKeyValuesStream.getFuture())); self->actors.add(serveGetKeyRequests(self, ssi.getKey.getFuture())); self->actors.add(serveWatchValueRequests(self, ssi.watchValue.getFuture())); - self->actors.add(serveChangeFeedRequests(self, ssi.changeFeed.getFuture())); self->actors.add(serveChangeFeedStreamRequests(self, ssi.changeFeedStream.getFuture())); self->actors.add(serveOverlappingChangeFeedsRequests(self, ssi.overlappingChangeFeeds.getFuture())); self->actors.add(serveChangeFeedPopRequests(self, ssi.changeFeedPop.getFuture())); diff --git a/fdbserver/workloads/ChangeFeeds.actor.cpp b/fdbserver/workloads/ChangeFeeds.actor.cpp index fa382f68c9..06415d0a81 100644 --- a/fdbserver/workloads/ChangeFeeds.actor.cpp +++ b/fdbserver/workloads/ChangeFeeds.actor.cpp @@ -19,6 +19,7 @@ */ #include "fdbclient/FDBOptions.g.h" +#include "fdbclient/ManagementAPI.actor.h" #include "fdbclient/NativeAPI.actor.h" #include "fdbclient/SystemData.h" #include "fdbserver/TesterInterface.actor.h" @@ -178,16 +179,7 @@ struct ChangeFeedsWorkload : TestWorkload { ACTOR Future changeFeedClient(Database cx, ChangeFeedsWorkload* self) { // Enable change feeds for a key range state Key rangeID = StringRef(deterministicRandom()->randomUniqueID().toString()); - state Transaction tr(cx); - loop { - try { - wait(tr.registerChangeFeed(rangeID, normalKeys)); - wait(tr.commit()); - break; - } catch (Error& e) { - wait(tr.onError(e)); - } - } + wait(updateChangeFeed(cx, rangeID, ChangeFeedStatus::CHANGE_FEED_CREATE, normalKeys)); loop { wait(delay(deterministicRandom()->random01())); diff --git a/flow/Arena.h b/flow/Arena.h index d2c448f886..fab4b5e12c 100644 --- a/flow/Arena.h +++ b/flow/Arena.h @@ -640,6 +640,16 @@ struct hash { }; } // namespace std +namespace std { +template <> +struct hash> { + static constexpr std::hash hashFunc{}; + std::size_t operator()(Standalone const& tag) const { + return hashFunc(std::string_view((const char*)tag.begin(), tag.size())); + } +}; +} // namespace std + template <> struct TraceableString { static const char* begin(StringRef value) { return reinterpret_cast(value.begin()); } diff --git a/flow/error_definitions.h b/flow/error_definitions.h index 07347dc3cb..768a2f6222 100755 --- a/flow/error_definitions.h +++ b/flow/error_definitions.h @@ -79,6 +79,8 @@ ERROR( version_already_compacted, 1055, "The requested changes have been compact ERROR( local_config_changed, 1056, "Local configuration file has changed. Restart and apply these changes" ) ERROR( failed_to_reach_quorum, 1057, "Failed to reach quorum from configuration database nodes. Retry sending these requests" ) ERROR( unknown_change_feed, 1058, "Change feed not found" ) +ERROR( change_feed_not_registered, 1059, "Change feed not registered" ) +ERROR( change_feed_too_large, 1060, "Change feed overlaps too many shards" ) ERROR( broken_promise, 1100, "Broken promise" ) ERROR( operation_cancelled, 1101, "Asynchronous operation cancelled" ) From ad3dcd6a74a1e6c1b4a26e4c218e3f6565fa1cd6 Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Tue, 19 Oct 2021 21:07:36 -0700 Subject: [PATCH 61/72] fix: memory replies were not being set --- fdbserver/storageserver.actor.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index fa2ddd35a9..5690768b13 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -1772,7 +1772,10 @@ ACTOR Future getChangeFeedMutations(StorageServer* data, } reply.mutations.append(reply.arena, it, totalCount); } + } else { + reply = memoryReply; } + Version finalVersion = std::min(req.end - 1, dequeVersion); if ((reply.mutations.empty() || reply.mutations.back().version < finalVersion) && remainingLimitBytes > 0 && remainingDurableBytes > 0) { From e34d24258103522707120d77159251c2541e3d20 Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Wed, 20 Oct 2021 22:04:39 -0700 Subject: [PATCH 62/72] fix: do not throw wrong_shard_server for local fetch keys --- fdbserver/storageserver.actor.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index 5690768b13..de369ff78b 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -1705,7 +1705,7 @@ ACTOR Future getChangeFeedMutations(StorageServer* data, wait(data->version.whenAtLeast(req.begin)); } state uint64_t changeCounter = data->shardChangeCounter; - if (!req.range.empty() && !data->isReadable(req.range)) { + if (!inverted && !data->isReadable(req.range)) { throw wrong_shard_server(); } From e3cd6f8b6fa3ba06755698fecc71e72a8f70384d Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Wed, 20 Oct 2021 22:05:00 -0700 Subject: [PATCH 63/72] fix: do not throw errors from sendError --- fdbrpc/fdbrpc.h | 11 +++++------ 1 file changed, 5 insertions(+), 6 deletions(-) diff --git a/fdbrpc/fdbrpc.h b/fdbrpc/fdbrpc.h index 8a7fde0e9b..652c34b676 100644 --- a/fdbrpc/fdbrpc.h +++ b/fdbrpc/fdbrpc.h @@ -441,13 +441,12 @@ public: template void sendError(const E& exc) const { - if (queue->isRemoteEndpoint() && !queue->sentError) { - if (queue->acknowledgements.failures.isError()) { - throw queue->acknowledgements.failures.getError(); + if (queue->isRemoteEndpoint()) { + if (!queue->sentError && !queue->acknowledgements.failures.isError()) { + queue->sentError = true; + FlowTransport::transport().sendUnreliable( + SerializeSource>>(exc), getEndpoint(), false); } - queue->sentError = true; - FlowTransport::transport().sendUnreliable( - SerializeSource>>(exc), getEndpoint(), false); } else { queue->sendError(exc); if (errors && errors->canBeSet()) { From 3ebabb6edcf7936a914e4ef3b711babdb0d8a9d4 Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Wed, 20 Oct 2021 22:37:31 -0700 Subject: [PATCH 64/72] fixed incorrect use of change feed errors --- fdbclient/NativeAPI.actor.cpp | 3 +-- fdbserver/storageserver.actor.cpp | 4 ++-- flow/error_definitions.h | 1 - 3 files changed, 3 insertions(+), 5 deletions(-) diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index 212ca3d74d..642f3624e9 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -6814,8 +6814,7 @@ ACTOR Future getChangeFeedStreamActor(Reference db, if (locations.size() >= CLIENT_KNOBS->CHANGE_FEED_LOCATION_LIMIT) { ASSERT_WE_THINK(false); - results.sendError(change_feed_too_large()); - return Void(); + throw unknown_change_feed(); } state std::vector chosenLocations(locations.size()); diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index de369ff78b..b982b47fce 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -3296,7 +3296,7 @@ ACTOR Future fetchChangeFeed(StorageServer* data, Key rangeId, KeyRange ra wait(yield()); } } catch (Error& e) { - if (e.code() != error_code_end_of_stream && e.code() != error_code_unsupported_operation) { + if (e.code() != error_code_end_of_stream && e.code() != error_code_change_feed_not_registered) { throw; } return Void(); @@ -3346,7 +3346,7 @@ ACTOR Future fetchChangeFeed(StorageServer* data, Key rangeId, KeyRange ra wait(yield()); } } catch (Error& e) { - if (e.code() != error_code_end_of_stream && e.code() != error_code_unsupported_operation) { + if (e.code() != error_code_end_of_stream && e.code() != error_code_change_feed_not_registered) { throw; } } diff --git a/flow/error_definitions.h b/flow/error_definitions.h index 768a2f6222..a6b11080ed 100755 --- a/flow/error_definitions.h +++ b/flow/error_definitions.h @@ -80,7 +80,6 @@ ERROR( local_config_changed, 1056, "Local configuration file has changed. Restar ERROR( failed_to_reach_quorum, 1057, "Failed to reach quorum from configuration database nodes. Retry sending these requests" ) ERROR( unknown_change_feed, 1058, "Change feed not found" ) ERROR( change_feed_not_registered, 1059, "Change feed not registered" ) -ERROR( change_feed_too_large, 1060, "Change feed overlaps too many shards" ) ERROR( broken_promise, 1100, "Broken promise" ) ERROR( operation_cancelled, 1101, "Asynchronous operation cancelled" ) From 4e79296a9ff133ca9991dc2ab8525ecc001aca99 Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Thu, 21 Oct 2021 00:44:51 -0700 Subject: [PATCH 65/72] fixed a few bugs with fetching change feeds --- fdbclient/DatabaseContext.h | 2 +- fdbclient/NativeAPI.actor.cpp | 20 ++++++++++---------- fdbclient/StorageServerInterface.h | 24 ++++++++++++++++++++++-- fdbserver/storageserver.actor.cpp | 28 ++++++++++++++++++++-------- 4 files changed, 53 insertions(+), 21 deletions(-) diff --git a/fdbclient/DatabaseContext.h b/fdbclient/DatabaseContext.h index 79b1e4e883..4a9a873628 100644 --- a/fdbclient/DatabaseContext.h +++ b/fdbclient/DatabaseContext.h @@ -258,7 +258,7 @@ public: Version end = std::numeric_limits::max(), KeyRange range = allKeys); - Future>> getOverlappingChangeFeeds(KeyRangeRef ranges, Version minVersion); + Future> getOverlappingChangeFeeds(KeyRangeRef ranges, Version minVersion); Future popChangeFeedMutations(StringRef rangeID, Version version); // private: diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index 642f3624e9..008b3c5081 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -6915,7 +6915,7 @@ Future DatabaseContext::getChangeFeedStream( return getChangeFeedStreamActor(Reference::addRef(this), results, rangeID, begin, end, range); } -ACTOR Future>> singleLocationOverlappingChangeFeeds( +ACTOR Future> singleLocationOverlappingChangeFeeds( Database cx, Reference location, KeyRangeRef range, @@ -6934,13 +6934,13 @@ ACTOR Future>> singleLocationOverlappingCha return rep.rangeIds; } -bool compareChangeFeedResult(const std::pair& i, const std::pair& j) { - return i.first < j.first; +bool compareChangeFeedResult(const OverlappingChangeFeedEntry& i, const OverlappingChangeFeedEntry& j) { + return i.rangeId < j.rangeId; } -ACTOR Future>> getOverlappingChangeFeedsActor(Reference db, - KeyRangeRef range, - Version minVersion) { +ACTOR Future> getOverlappingChangeFeedsActor(Reference db, + KeyRangeRef range, + Version minVersion) { state Database cx(db); state Transaction tr(cx); state Span span("NAPI:GetOverlappingChangeFeeds"_loc); @@ -6963,14 +6963,14 @@ ACTOR Future>> getOverlappingChangeFeedsAct throw all_alternatives_failed(); } - state std::vector>>> allOverlappingRequests; + state std::vector>> allOverlappingRequests; for (auto& it : locations) { allOverlappingRequests.push_back( singleLocationOverlappingChangeFeeds(cx, it.second, it.first & range, minVersion)); } wait(waitForAll(allOverlappingRequests)); - std::vector> result; + std::vector result; for (auto& it : allOverlappingRequests) { result.insert(result.end(), it.get().begin(), it.get().end()); } @@ -6988,8 +6988,8 @@ ACTOR Future>> getOverlappingChangeFeedsAct } } -Future>> DatabaseContext::getOverlappingChangeFeeds(KeyRangeRef range, - Version minVersion) { +Future> DatabaseContext::getOverlappingChangeFeeds(KeyRangeRef range, + Version minVersion) { return getOverlappingChangeFeedsActor(Reference::addRef(this), range, minVersion); } diff --git a/fdbclient/StorageServerInterface.h b/fdbclient/StorageServerInterface.h index 9ec48c59c9..c47f985a0b 100644 --- a/fdbclient/StorageServerInterface.h +++ b/fdbclient/StorageServerInterface.h @@ -712,14 +712,33 @@ struct ChangeFeedPopRequest { } }; +struct OverlappingChangeFeedEntry { + Key rangeId; + KeyRange range; + bool stopped = false; + + bool operator==(const OverlappingChangeFeedEntry& r) const { + return rangeId == r.rangeId && range == r.range && stopped == r.stopped; + } + + OverlappingChangeFeedEntry() {} + OverlappingChangeFeedEntry(Key const& rangeId, KeyRange const& range, bool stopped) + : rangeId(rangeId), range(range), stopped(stopped) {} + + template + void serialize(Ar& ar) { + serializer(ar, rangeId, range, stopped); + } +}; + struct OverlappingChangeFeedsReply { constexpr static FileIdentifier file_identifier = 11815134; - std::vector> rangeIds; + std::vector rangeIds; bool cached; Arena arena; OverlappingChangeFeedsReply() : cached(false) {} - explicit OverlappingChangeFeedsReply(std::vector> const& rangeIds) + explicit OverlappingChangeFeedsReply(std::vector const& rangeIds) : rangeIds(rangeIds), cached(false) {} template @@ -727,6 +746,7 @@ struct OverlappingChangeFeedsReply { serializer(ar, rangeIds, arena); } }; + struct OverlappingChangeFeedsRequest { constexpr static FileIdentifier file_identifier = 10726174; KeyRange range; diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index b982b47fce..037070c8d3 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -1592,17 +1592,16 @@ ACTOR Future overlappingChangeFeedsQ(StorageServer* data, OverlappingChang } auto ranges = data->keyChangeFeed.intersectingRanges(req.range); - std::map rangeIds; + std::map> rangeIds; for (auto r : ranges) { for (auto& it : r.value()) { - rangeIds[it->id] = it->range; + rangeIds[it->id] = std::make_pair(it->range, it->stopped); } } - std::vector> result; + OverlappingChangeFeedsReply reply; for (auto& it : rangeIds) { - result.push_back(std::make_pair(it.first, it.second)); + reply.rangeIds.push_back(OverlappingChangeFeedEntry(it.first, it.second.first, it.second.second)); } - OverlappingChangeFeedsReply reply(std::vector>(rangeIds.begin(), rangeIds.end())); req.reply.send(reply); return Void(); } @@ -3248,7 +3247,11 @@ static const KeyRangeRef persistChangeFeedKeys = KeyRangeRef(LiteralStringRef(PERSIST_PREFIX "RF/"), LiteralStringRef(PERSIST_PREFIX "RF0")); // data keys are unmangled (but never start with PERSIST_PREFIX because they are always in allKeys) -ACTOR Future fetchChangeFeed(StorageServer* data, Key rangeId, KeyRange range, Version fetchVersion) { +ACTOR Future fetchChangeFeed(StorageServer* data, + Key rangeId, + KeyRange range, + bool stopped, + Version fetchVersion) { state Reference changeFeedInfo; wait(delay(0)); // allow this actor to be cancelled by removals bool existing = data->uidChangeFeed.count(rangeId); @@ -3262,6 +3265,7 @@ ACTOR Future fetchChangeFeed(StorageServer* data, Key rangeId, KeyRange ra changeFeedInfo = Reference(new ChangeFeedInfo()); changeFeedInfo->range = range; changeFeedInfo->id = rangeId; + changeFeedInfo->stopped = stopped; data->uidChangeFeed[rangeId] = changeFeedInfo; auto rs = data->keyChangeFeed.modify(range); for (auto r = rs.begin(); r != rs.end(); ++r) { @@ -3291,6 +3295,8 @@ ACTOR Future fetchChangeFeed(StorageServer* data, Key rangeId, KeyRange ra data->storage.writeKeyValue( KeyValueRef(changeFeedDurableKey(rangeId, it.version), changeFeedDurableValue(it.mutations, it.knownCommittedVersion))); + changeFeedInfo->storageVersion = std::max(changeFeedInfo->durableVersion, it.version); + changeFeedInfo->durableVersion = changeFeedInfo->storageVersion; } } wait(yield()); @@ -3323,6 +3329,9 @@ ACTOR Future fetchChangeFeed(StorageServer* data, Key rangeId, KeyRange ra KeyValueRef(changeFeedDurableKey(rangeId, remoteResult[remoteLoc].version), changeFeedDurableValue(remoteResult[remoteLoc].mutations, remoteResult[remoteLoc].knownCommittedVersion))); + changeFeedInfo->storageVersion = + std::max(changeFeedInfo->durableVersion, remoteResult[remoteLoc].version); + changeFeedInfo->durableVersion = changeFeedInfo->storageVersion; } remoteLoc++; } else if (remoteResult[remoteLoc].version == localResult.version) { @@ -3334,6 +3343,9 @@ ACTOR Future fetchChangeFeed(StorageServer* data, Key rangeId, KeyRange ra KeyValueRef(changeFeedDurableKey(rangeId, remoteResult[remoteLoc].version), changeFeedDurableValue(remoteResult[remoteLoc].mutations, remoteResult[remoteLoc].knownCommittedVersion))); + changeFeedInfo->storageVersion = + std::max(changeFeedInfo->durableVersion, remoteResult[remoteLoc].version); + changeFeedInfo->durableVersion = changeFeedInfo->storageVersion; } remoteLoc++; Standalone _localResult = waitNext(localResults.getFuture()); @@ -3359,10 +3371,10 @@ ACTOR Future dispatchChangeFeeds(StorageServer* data, UID fetchKeysID, Key state PromiseStream removals; data->changeFeedRemovals[fetchKeysID] = removals; try { - state std::vector> feeds = + state std::vector feeds = wait(data->cx->getOverlappingChangeFeeds(keys, fetchVersion + 1)); for (auto& feed : feeds) { - feedFetches[feed.first] = fetchChangeFeed(data, feed.first, feed.second, fetchVersion); + feedFetches[feed.rangeId] = fetchChangeFeed(data, feed.rangeId, feed.range, feed.stopped, fetchVersion); } loop { From d778cea9b565922f441470b5db13faae8f5c70b0 Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Thu, 21 Oct 2021 15:16:22 -0700 Subject: [PATCH 66/72] fix: prevent infinite loop --- fdbclient/NativeAPI.actor.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index cbdf86b55d..ce15e08214 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -6777,6 +6777,7 @@ ACTOR Future getChangeFeedRange(Reference db, Databas Version readVer = wait(tr.getReadVersion()); if (readVer < begin) { wait(delay(FLOW_KNOBS->PREVENT_FAST_SPIN_DELAY)); + tr.reset(); } else { Optional val = wait(tr.get(rangeIDKey)); if (!val.present()) { From f03d32f3d4dc99ccbec2aebf9a9c273791bf87cf Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Thu, 21 Oct 2021 23:04:51 -0700 Subject: [PATCH 67/72] fix: handle the case where a fetch happens at an earlier read version than the commit version of the change feed registration --- fdbserver/storageserver.actor.cpp | 93 +++++++++++++++++++------------ 1 file changed, 56 insertions(+), 37 deletions(-) diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index 066757c63e..2b8526dd1b 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -3266,41 +3266,12 @@ static const KeyRangeRef persistChangeFeedKeys = KeyRangeRef(LiteralStringRef(PERSIST_PREFIX "RF/"), LiteralStringRef(PERSIST_PREFIX "RF0")); // data keys are unmangled (but never start with PERSIST_PREFIX because they are always in allKeys) -ACTOR Future fetchChangeFeed(StorageServer* data, - Key rangeId, - KeyRange range, - bool stopped, - Version fetchVersion) { - state Reference changeFeedInfo; - wait(delay(0)); // allow this actor to be cancelled by removals - bool existing = data->uidChangeFeed.count(rangeId); - - TraceEvent(SevDebug, "FetchChangeFeed", data->thisServerID) - .detail("RangeID", rangeId.printable()) - .detail("Range", range.toString()) - .detail("Existing", existing); - - if (!existing) { - changeFeedInfo = Reference(new ChangeFeedInfo()); - changeFeedInfo->range = range; - changeFeedInfo->id = rangeId; - changeFeedInfo->stopped = stopped; - data->uidChangeFeed[rangeId] = changeFeedInfo; - auto rs = data->keyChangeFeed.modify(range); - for (auto r = rs.begin(); r != rs.end(); ++r) { - r->value().push_back(changeFeedInfo); - } - data->keyChangeFeed.coalesce(range.contents()); - auto& mLV = data->addVersionToMutationLog(data->data().getLatestVersion()); - data->addMutationToMutationLog( - mLV, - MutationRef(MutationRef::SetValue, - persistChangeFeedKeys.begin.toString() + rangeId.toString(), - changeFeedValue(range, invalidVersion, ChangeFeedStatus::CHANGE_FEED_CREATE))); - } else { - changeFeedInfo = data->uidChangeFeed[rangeId]; - } - +ACTOR Future fetchChangeFeedApplier(StorageServer* data, + Reference changeFeedInfo, + Key rangeId, + KeyRange range, + Version fetchVersion, + bool existing) { state PromiseStream>> feedResults; state Future feed = data->cx->getChangeFeedStream( feedResults, rangeId, 0, existing ? fetchVersion + 1 : data->version.get() + 1, range); @@ -3321,7 +3292,7 @@ ACTOR Future fetchChangeFeed(StorageServer* data, wait(yield()); } } catch (Error& e) { - if (e.code() != error_code_end_of_stream && e.code() != error_code_change_feed_not_registered) { + if (e.code() != error_code_end_of_stream) { throw; } return Void(); @@ -3377,13 +3348,61 @@ ACTOR Future fetchChangeFeed(StorageServer* data, wait(yield()); } } catch (Error& e) { - if (e.code() != error_code_end_of_stream && e.code() != error_code_change_feed_not_registered) { + if (e.code() != error_code_end_of_stream) { throw; } } return Void(); } +ACTOR Future fetchChangeFeed(StorageServer* data, + Key rangeId, + KeyRange range, + bool stopped, + Version fetchVersion) { + state Reference changeFeedInfo; + wait(delay(0)); // allow this actor to be cancelled by removals + state bool existing = data->uidChangeFeed.count(rangeId); + + TraceEvent(SevDebug, "FetchChangeFeed", data->thisServerID) + .detail("RangeID", rangeId.printable()) + .detail("Range", range.toString()) + .detail("Existing", existing); + + if (!existing) { + changeFeedInfo = Reference(new ChangeFeedInfo()); + changeFeedInfo->range = range; + changeFeedInfo->id = rangeId; + changeFeedInfo->stopped = stopped; + data->uidChangeFeed[rangeId] = changeFeedInfo; + auto rs = data->keyChangeFeed.modify(range); + for (auto r = rs.begin(); r != rs.end(); ++r) { + r->value().push_back(changeFeedInfo); + } + data->keyChangeFeed.coalesce(range.contents()); + auto& mLV = data->addVersionToMutationLog(data->data().getLatestVersion()); + data->addMutationToMutationLog( + mLV, + MutationRef(MutationRef::SetValue, + persistChangeFeedKeys.begin.toString() + rangeId.toString(), + changeFeedValue(range, invalidVersion, ChangeFeedStatus::CHANGE_FEED_CREATE))); + } else { + changeFeedInfo = data->uidChangeFeed[rangeId]; + } + + loop { + try { + wait(fetchChangeFeedApplier(data, changeFeedInfo, rangeId, range, fetchVersion, existing)); + return Void(); + } catch (Error& e) { + if (e.code() != error_code_change_feed_not_registered) { + throw; + } + } + wait(delay(FLOW_KNOBS->PREVENT_FAST_SPIN_DELAY)); + } +} + ACTOR Future dispatchChangeFeeds(StorageServer* data, UID fetchKeysID, KeyRange keys, Version fetchVersion) { // find overlapping range feeds state std::map> feedFetches; From 4039dbd8da953623b135364739950280d21ddf5b Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Sun, 24 Oct 2021 13:46:45 -0700 Subject: [PATCH 68/72] fix: combine mutations from the same version into a single MutationsAndVesionRef --- fdbclient/NativeAPI.actor.cpp | 7 ++++++- fdbclient/StorageServerInterface.h | 4 ++-- 2 files changed, 8 insertions(+), 3 deletions(-) diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index ce15e08214..6b9a56e844 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -6746,7 +6746,12 @@ ACTOR Future mergeChangeFeedStream(std::vector res = waitNext(nextStream.results.getFuture()); nextStream.next = res; diff --git a/fdbclient/StorageServerInterface.h b/fdbclient/StorageServerInterface.h index c47f985a0b..ac8e9c344f 100644 --- a/fdbclient/StorageServerInterface.h +++ b/fdbclient/StorageServerInterface.h @@ -637,8 +637,8 @@ struct SplitRangeRequest { struct MutationsAndVersionRef { VectorRef mutations; - Version version; - Version knownCommittedVersion; + Version version = invalidVersion; + Version knownCommittedVersion = invalidVersion; MutationsAndVersionRef() {} explicit MutationsAndVersionRef(Version version, Version knownCommittedVersion) From 0e327d3d0a2141a09cf593a90720f0d004f69d61 Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Sun, 24 Oct 2021 19:17:11 -0700 Subject: [PATCH 69/72] fix: do not duplicate lastEpochEnd from different servers --- fdbclient/NativeAPI.actor.cpp | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index 6b9a56e844..73f276fa7e 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -6747,8 +6747,11 @@ ACTOR Future mergeChangeFeedStream(std::vector Date: Sun, 24 Oct 2021 19:18:03 -0700 Subject: [PATCH 70/72] fix: disconnectTriggers cannot be yielded because we could send a reply to a replyPromiseStream while waiting on the delay --- fdbrpc/FailureMonitor.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/fdbrpc/FailureMonitor.h b/fdbrpc/FailureMonitor.h index 04154fa103..795a6e9953 100644 --- a/fdbrpc/FailureMonitor.h +++ b/fdbrpc/FailureMonitor.h @@ -157,7 +157,7 @@ public: private: std::unordered_map addressStatus; YieldedAsyncMap endpointKnownFailed; - YieldedAsyncMap disconnectTriggers; + AsyncMap disconnectTriggers; std::unordered_set failedEndpoints; friend class OnStateChangedActorActor; From 9a6384fc266eefe9f1893eaffc51778621238481 Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Sun, 24 Oct 2021 21:18:49 -0700 Subject: [PATCH 71/72] fixed merge conflicts --- fdbserver/storageserver.actor.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index 6502c75b1e..dd01146a54 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -892,8 +892,8 @@ public: fetchKeysParallelismLock(SERVER_KNOBS->FETCH_KEYS_PARALLELISM), fetchKeysBytesBudget(SERVER_KNOBS->STORAGE_FETCH_BYTES), fetchKeysBudgetUsed(false), instanceID(deterministicRandom()->randomUniqueID().first()), shuttingDown(false), behind(false), - versionBehind(false), debug_inApplyUpdate(false), debug_lastValidateTime(0), maxQueryQueue(0), - lastBytesInputEBrake(0), lastDurableVersionEBrake(0), transactionTagCounter(ssi.id()), counters(this), + versionBehind(false), debug_inApplyUpdate(false), debug_lastValidateTime(0), lastBytesInputEBrake(0), + lastDurableVersionEBrake(0), maxQueryQueue(0), transactionTagCounter(ssi.id()), counters(this), storageServerSourceTLogIDEventHolder( makeReference(ssi.id().toString() + "/StorageServerSourceTLogID")) { version.initMetric(LiteralStringRef("StorageServer.Version"), counters.cc.id); @@ -4011,7 +4011,7 @@ void changeServerKeys(StorageServer* data, // Clear the moving-in empty range, and set it available at the latestVersion. for (const auto& range : newEmptyRanges) { MutationRef clearRange(MutationRef::ClearRange, range.begin, range.end); - data->addMutation(data->data().getLatestVersion(), clearRange, range, data->updateEagerReads); + data->addMutation(data->data().getLatestVersion(), true, clearRange, range, data->updateEagerReads); data->newestAvailableVersion.insert(range, latestVersion); setAvailableStatus(data, range, true); } From 118c307b571163f3bcf4e391a6ea31c028f56082 Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Sun, 24 Oct 2021 22:26:11 -0700 Subject: [PATCH 72/72] fixed formatting --- fdbserver/storageserver.actor.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index dd01146a54..bacbbb165d 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -1832,8 +1832,7 @@ ACTOR Future localChangeFeedStream(StorageServer* data, } } -ACTOR Future changeFeedStreamQ(StorageServer* data, ChangeFeedStreamRequest req) -{ +ACTOR Future changeFeedStreamQ(StorageServer* data, ChangeFeedStreamRequest req) { state Span span("SS:getChangeFeedStream"_loc, { req.spanContext }); req.reply.setByteLimit(SERVER_KNOBS->RANGESTREAM_LIMIT_BYTES);