A few hostname improvements. (#6825)

* Add tryResolveHostnames() in connection string.

* Add missing hostname to related interfaces.

* Do not pass RequestStream into *GetReplyFromHostname() functions.

Because we are using new RequestStream for each request anyways. Also, the passed in pointer could be nullptr, which results in seg faults.

* Add dynamic hostname resolve and reconnect intervals.

* Address comments.
This commit is contained in:
Renxuan Wang 2022-04-20 13:42:46 -07:00 committed by GitHub
parent 408c0cf1c9
commit e40cc8722c
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
13 changed files with 102 additions and 76 deletions

View File

@ -34,12 +34,16 @@ void ConfigTransactionInterface::setupWellKnownEndpoints() {
} }
ConfigTransactionInterface::ConfigTransactionInterface(NetworkAddress const& remote) ConfigTransactionInterface::ConfigTransactionInterface(NetworkAddress const& remote)
: getGeneration(Endpoint::wellKnown({ remote }, WLTOKEN_CONFIGTXN_GETGENERATION)), : _id(deterministicRandom()->randomUniqueID()),
getGeneration(Endpoint::wellKnown({ remote }, WLTOKEN_CONFIGTXN_GETGENERATION)),
get(Endpoint::wellKnown({ remote }, WLTOKEN_CONFIGTXN_GET)), get(Endpoint::wellKnown({ remote }, WLTOKEN_CONFIGTXN_GET)),
getClasses(Endpoint::wellKnown({ remote }, WLTOKEN_CONFIGTXN_GETCLASSES)), getClasses(Endpoint::wellKnown({ remote }, WLTOKEN_CONFIGTXN_GETCLASSES)),
getKnobs(Endpoint::wellKnown({ remote }, WLTOKEN_CONFIGTXN_GETKNOBS)), getKnobs(Endpoint::wellKnown({ remote }, WLTOKEN_CONFIGTXN_GETKNOBS)),
commit(Endpoint::wellKnown({ remote }, WLTOKEN_CONFIGTXN_COMMIT)) {} commit(Endpoint::wellKnown({ remote }, WLTOKEN_CONFIGTXN_COMMIT)) {}
ConfigTransactionInterface::ConfigTransactionInterface(Hostname const& remote)
: _id(deterministicRandom()->randomUniqueID()), hostname(remote) {}
bool ConfigTransactionInterface::operator==(ConfigTransactionInterface const& rhs) const { bool ConfigTransactionInterface::operator==(ConfigTransactionInterface const& rhs) const {
return _id == rhs._id; return _id == rhs._id;
} }

View File

@ -200,9 +200,12 @@ public:
class RequestStream<ConfigTransactionGetKnobsRequest> getKnobs; class RequestStream<ConfigTransactionGetKnobsRequest> getKnobs;
class RequestStream<ConfigTransactionCommitRequest> commit; class RequestStream<ConfigTransactionCommitRequest> commit;
Optional<Hostname> hostname;
ConfigTransactionInterface(); ConfigTransactionInterface();
void setupWellKnownEndpoints(); void setupWellKnownEndpoints();
ConfigTransactionInterface(NetworkAddress const& remote); ConfigTransactionInterface(NetworkAddress const& remote);
ConfigTransactionInterface(Hostname const& remote);
bool operator==(ConfigTransactionInterface const& rhs) const; bool operator==(ConfigTransactionInterface const& rhs) const;
bool operator!=(ConfigTransactionInterface const& rhs) const; bool operator!=(ConfigTransactionInterface const& rhs) const;
@ -210,6 +213,6 @@ public:
template <class Ar> template <class Ar>
void serialize(Ar& ar) { void serialize(Ar& ar) {
serializer(ar, getGeneration, get, getClasses, getKnobs, commit); serializer(ar, getGeneration, get, getClasses, getKnobs, commit, hostname);
} }
}; };

View File

@ -104,6 +104,10 @@ public:
ConnectionStringStatus status = RESOLVED; ConnectionStringStatus status = RESOLVED;
AsyncTrigger resolveFinish; AsyncTrigger resolveFinish;
// This function tries to resolve all hostnames once, and return them with coords.
// Best effort, does not guarantee that the resolves succeed.
Future<std::vector<NetworkAddress>> tryResolveHostnames();
std::vector<NetworkAddress> coords; std::vector<NetworkAddress> coords;
std::vector<Hostname> hostnames; std::vector<Hostname> hostnames;
std::unordered_map<NetworkAddress, Hostname> networkAddressToHostname; std::unordered_map<NetworkAddress, Hostname> networkAddressToHostname;

View File

@ -364,6 +364,29 @@ TEST_CASE("/fdbclient/MonitorLeader/ConnectionString") {
return Void(); return Void();
} }
ACTOR Future<std::vector<NetworkAddress>> tryResolveHostnamesImpl(ClusterConnectionString* self) {
state std::set<NetworkAddress> allCoordinatorsSet;
std::vector<Future<Void>> fs;
for (auto& hostname : self->hostnames) {
fs.push_back(map(hostname.resolve(), [&](Optional<NetworkAddress> const& addr) -> Void {
if (addr.present()) {
allCoordinatorsSet.insert(addr.get());
}
return Void();
}));
}
wait(waitForAll(fs));
for (const auto& coord : self->coords) {
allCoordinatorsSet.insert(coord);
}
std::vector<NetworkAddress> allCoordinators(allCoordinatorsSet.begin(), allCoordinatorsSet.end());
return allCoordinators;
}
Future<std::vector<NetworkAddress>> ClusterConnectionString::tryResolveHostnames() {
return tryResolveHostnamesImpl(this);
}
TEST_CASE("/fdbclient/MonitorLeader/PartialResolve") { TEST_CASE("/fdbclient/MonitorLeader/PartialResolve") {
std::string connectionString = "TestCluster:0@host.name:1234,host-name:5678"; std::string connectionString = "TestCluster:0@host.name:1234,host-name:5678";
std::string hn = "host-name", port = "5678"; std::string hn = "host-name", port = "5678";
@ -373,19 +396,9 @@ TEST_CASE("/fdbclient/MonitorLeader/PartialResolve") {
INetworkConnections::net()->addMockTCPEndpoint(hn, port, { address }); INetworkConnections::net()->addMockTCPEndpoint(hn, port, { address });
state ClusterConnectionString cs(connectionString); state ClusterConnectionString cs(connectionString);
state std::vector<NetworkAddress> allCoordinators = wait(cs.tryResolveHostnames());
state std::unordered_set<NetworkAddress> coordinatorAddresses; ASSERT(allCoordinators.size() == 1 &&
std::vector<Future<Void>> fs; std::find(allCoordinators.begin(), allCoordinators.end(), address) != allCoordinators.end());
for (auto& hostname : cs.hostnames) {
fs.push_back(map(hostname.resolve(), [&](Optional<NetworkAddress> const& addr) -> Void {
if (addr.present()) {
coordinatorAddresses.insert(addr.get());
}
return Void();
}));
}
wait(waitForAll(fs));
ASSERT(coordinatorAddresses.size() == 1 && coordinatorAddresses.count(address) == 1);
return Void(); return Void();
} }
@ -585,7 +598,7 @@ ACTOR Future<Void> monitorNominee(Key key,
.detail("OldAddr", coord.getLeader.getEndpoint().getPrimaryAddress().toString()); .detail("OldAddr", coord.getLeader.getEndpoint().getPrimaryAddress().toString());
if (rep.getError().code() == error_code_request_maybe_delivered) { if (rep.getError().code() == error_code_request_maybe_delivered) {
// Delay to prevent tight resolving loop due to outdated DNS cache // Delay to prevent tight resolving loop due to outdated DNS cache
wait(delay(FLOW_KNOBS->HOSTNAME_RESOLVE_DELAY)); wait(delay(FLOW_KNOBS->HOSTNAME_RECONNECT_INIT_INTERVAL));
throw coordinators_changed(); throw coordinators_changed();
} else { } else {
throw rep.getError(); throw rep.getError();

View File

@ -73,10 +73,7 @@ Future<REPLY_TYPE(Req)> retryBrokenPromise(RequestStream<Req, P> to, Req request
} }
ACTOR template <class Req> ACTOR template <class Req>
Future<ErrorOr<REPLY_TYPE(Req)>> tryGetReplyFromHostname(RequestStream<Req>* to, Future<ErrorOr<REPLY_TYPE(Req)>> tryGetReplyFromHostname(Req request, Hostname hostname, WellKnownEndpoints token) {
Req request,
Hostname hostname,
WellKnownEndpoints token) {
// A wrapper of tryGetReply(request), except that the request is sent to an address resolved from a hostname. // A wrapper of tryGetReply(request), except that the request is sent to an address resolved from a hostname.
// If resolving fails, return lookup_failed(). // If resolving fails, return lookup_failed().
// Otherwise, return tryGetReply(request). // Otherwise, return tryGetReply(request).
@ -84,8 +81,8 @@ Future<ErrorOr<REPLY_TYPE(Req)>> tryGetReplyFromHostname(RequestStream<Req>* to,
if (!address.present()) { if (!address.present()) {
return ErrorOr<REPLY_TYPE(Req)>(lookup_failed()); return ErrorOr<REPLY_TYPE(Req)>(lookup_failed());
} }
*to = RequestStream<Req>(Endpoint::wellKnown({ address.get() }, token)); RequestStream<Req> to(Endpoint::wellKnown({ address.get() }, token));
ErrorOr<REPLY_TYPE(Req)> reply = wait(to->tryGetReply(request)); state ErrorOr<REPLY_TYPE(Req)> reply = wait(to.tryGetReply(request));
if (reply.isError()) { if (reply.isError()) {
resetReply(request); resetReply(request);
if (reply.getError().code() == error_code_request_maybe_delivered) { if (reply.getError().code() == error_code_request_maybe_delivered) {
@ -98,8 +95,7 @@ Future<ErrorOr<REPLY_TYPE(Req)>> tryGetReplyFromHostname(RequestStream<Req>* to,
} }
ACTOR template <class Req> ACTOR template <class Req>
Future<ErrorOr<REPLY_TYPE(Req)>> tryGetReplyFromHostname(RequestStream<Req>* to, Future<ErrorOr<REPLY_TYPE(Req)>> tryGetReplyFromHostname(Req request,
Req request,
Hostname hostname, Hostname hostname,
WellKnownEndpoints token, WellKnownEndpoints token,
TaskPriority taskID) { TaskPriority taskID) {
@ -110,8 +106,8 @@ Future<ErrorOr<REPLY_TYPE(Req)>> tryGetReplyFromHostname(RequestStream<Req>* to,
if (!address.present()) { if (!address.present()) {
return ErrorOr<REPLY_TYPE(Req)>(lookup_failed()); return ErrorOr<REPLY_TYPE(Req)>(lookup_failed());
} }
*to = RequestStream<Req>(Endpoint::wellKnown({ address.get() }, token)); RequestStream<Req> to(Endpoint::wellKnown({ address.get() }, token));
ErrorOr<REPLY_TYPE(Req)> reply = wait(to->tryGetReply(request, taskID)); state ErrorOr<REPLY_TYPE(Req)> reply = wait(to.tryGetReply(request, taskID));
if (reply.isError()) { if (reply.isError()) {
resetReply(request); resetReply(request);
if (reply.getError().code() == error_code_request_maybe_delivered) { if (reply.getError().code() == error_code_request_maybe_delivered) {
@ -124,21 +120,21 @@ Future<ErrorOr<REPLY_TYPE(Req)>> tryGetReplyFromHostname(RequestStream<Req>* to,
} }
ACTOR template <class Req> ACTOR template <class Req>
Future<REPLY_TYPE(Req)> retryGetReplyFromHostname(RequestStream<Req>* to, Future<REPLY_TYPE(Req)> retryGetReplyFromHostname(Req request, Hostname hostname, WellKnownEndpoints token) {
Req request,
Hostname hostname,
WellKnownEndpoints token) {
// Like tryGetReplyFromHostname, except that request_maybe_delivered results in re-resolving the hostname. // Like tryGetReplyFromHostname, except that request_maybe_delivered results in re-resolving the hostname.
// Suitable for use with hostname, where RequestStream is NOT initialized yet. // Suitable for use with hostname, where RequestStream is NOT initialized yet.
// Not normally useful for endpoints initialized with NetworkAddress. // Not normally useful for endpoints initialized with NetworkAddress.
state double reconnetInterval = FLOW_KNOBS->HOSTNAME_RECONNECT_INIT_INTERVAL;
loop { loop {
NetworkAddress address = wait(hostname.resolveWithRetry()); NetworkAddress address = wait(hostname.resolveWithRetry());
*to = RequestStream<Req>(Endpoint::wellKnown({ address }, token)); RequestStream<Req> to(Endpoint::wellKnown({ address }, token));
ErrorOr<REPLY_TYPE(Req)> reply = wait(to->tryGetReply(request)); state ErrorOr<REPLY_TYPE(Req)> reply = wait(to.tryGetReply(request));
if (reply.isError()) { if (reply.isError()) {
resetReply(request); resetReply(request);
if (reply.getError().code() == error_code_request_maybe_delivered) { if (reply.getError().code() == error_code_request_maybe_delivered) {
// Connection failure. // Connection failure.
wait(delay(reconnetInterval));
reconnetInterval = std::min(2 * reconnetInterval, FLOW_KNOBS->HOSTNAME_RECONNECT_MAX_INTERVAL);
hostname.resetToUnresolved(); hostname.resetToUnresolved();
INetworkConnections::net()->removeCachedDNS(hostname.host, hostname.service); INetworkConnections::net()->removeCachedDNS(hostname.host, hostname.service);
} else { } else {
@ -151,22 +147,24 @@ Future<REPLY_TYPE(Req)> retryGetReplyFromHostname(RequestStream<Req>* to,
} }
ACTOR template <class Req> ACTOR template <class Req>
Future<REPLY_TYPE(Req)> retryGetReplyFromHostname(RequestStream<Req>* to, Future<REPLY_TYPE(Req)> retryGetReplyFromHostname(Req request,
Req request,
Hostname hostname, Hostname hostname,
WellKnownEndpoints token, WellKnownEndpoints token,
TaskPriority taskID) { TaskPriority taskID) {
// Like tryGetReplyFromHostname, except that request_maybe_delivered results in re-resolving the hostname. // Like tryGetReplyFromHostname, except that request_maybe_delivered results in re-resolving the hostname.
// Suitable for use with hostname, where RequestStream is NOT initialized yet. // Suitable for use with hostname, where RequestStream is NOT initialized yet.
// Not normally useful for endpoints initialized with NetworkAddress. // Not normally useful for endpoints initialized with NetworkAddress.
state double reconnetInterval = FLOW_KNOBS->HOSTNAME_RECONNECT_INIT_INTERVAL;
loop { loop {
NetworkAddress address = wait(hostname.resolveWithRetry()); NetworkAddress address = wait(hostname.resolveWithRetry());
*to = RequestStream<Req>(Endpoint::wellKnown({ address }, token)); RequestStream<Req> to(Endpoint::wellKnown({ address }, token));
ErrorOr<REPLY_TYPE(Req)> reply = wait(to->tryGetReply(request, taskID)); state ErrorOr<REPLY_TYPE(Req)> reply = wait(to.tryGetReply(request, taskID));
if (reply.isError()) { if (reply.isError()) {
resetReply(request); resetReply(request);
if (reply.getError().code() == error_code_request_maybe_delivered) { if (reply.getError().code() == error_code_request_maybe_delivered) {
// Connection failure. // Connection failure.
wait(delay(reconnetInterval));
reconnetInterval = std::min(2 * reconnetInterval, FLOW_KNOBS->HOSTNAME_RECONNECT_MAX_INTERVAL);
hostname.resetToUnresolved(); hostname.resetToUnresolved();
INetworkConnections::net()->removeCachedDNS(hostname.host, hostname.service); INetworkConnections::net()->removeCachedDNS(hostname.host, hostname.service);
} else { } else {

View File

@ -1097,18 +1097,24 @@ void haltRegisteringOrCurrentSingleton(ClusterControllerData* self,
} }
} }
void registerWorker(RegisterWorkerRequest req, ACTOR Future<Void> registerWorker(RegisterWorkerRequest req,
ClusterControllerData* self, ClusterControllerData* self,
std::unordered_set<NetworkAddress> coordinatorAddresses, ClusterConnectionString cs,
ConfigBroadcaster* configBroadcaster) { ConfigBroadcaster* configBroadcaster) {
std::vector<NetworkAddress> coordinatorAddresses = wait(cs.tryResolveHostnames());
const WorkerInterface& w = req.wi; const WorkerInterface& w = req.wi;
ProcessClass newProcessClass = req.processClass; ProcessClass newProcessClass = req.processClass;
auto info = self->id_worker.find(w.locality.processId()); auto info = self->id_worker.find(w.locality.processId());
ClusterControllerPriorityInfo newPriorityInfo = req.priorityInfo; ClusterControllerPriorityInfo newPriorityInfo = req.priorityInfo;
newPriorityInfo.processClassFitness = newProcessClass.machineClassFitness(ProcessClass::ClusterController); newPriorityInfo.processClassFitness = newProcessClass.machineClassFitness(ProcessClass::ClusterController);
bool isCoordinator = bool isCoordinator =
(coordinatorAddresses.count(req.wi.address()) > 0) || (std::find(coordinatorAddresses.begin(), coordinatorAddresses.end(), req.wi.address()) !=
(req.wi.secondaryAddress().present() && coordinatorAddresses.count(req.wi.secondaryAddress().get()) > 0); coordinatorAddresses.end()) ||
(req.wi.secondaryAddress().present() &&
std::find(coordinatorAddresses.begin(), coordinatorAddresses.end(), req.wi.secondaryAddress().get()) !=
coordinatorAddresses.end());
for (auto it : req.incompatiblePeers) { for (auto it : req.incompatiblePeers) {
self->db.incompatibleConnections[it] = now() + SERVER_KNOBS->INCOMPATIBLE_PEERS_LOGGING_INTERVAL; self->db.incompatibleConnections[it] = now() + SERVER_KNOBS->INCOMPATIBLE_PEERS_LOGGING_INTERVAL;
@ -1271,6 +1277,8 @@ void registerWorker(RegisterWorkerRequest req,
if (!req.reply.isSet() && newPriorityInfo != req.priorityInfo) { if (!req.reply.isSet() && newPriorityInfo != req.priorityInfo) {
req.reply.send(RegisterWorkerReply(newProcessClass, newPriorityInfo)); req.reply.send(RegisterWorkerReply(newProcessClass, newPriorityInfo));
} }
return Void();
} }
#define TIME_KEEPER_VERSION LiteralStringRef("1") #define TIME_KEEPER_VERSION LiteralStringRef("1")
@ -2543,30 +2551,12 @@ ACTOR Future<Void> clusterControllerCore(Reference<IClusterConnectionRecord> con
when(RecruitBlobWorkerRequest req = waitNext(interf.recruitBlobWorker.getFuture())) { when(RecruitBlobWorkerRequest req = waitNext(interf.recruitBlobWorker.getFuture())) {
clusterRecruitBlobWorker(&self, req); clusterRecruitBlobWorker(&self, req);
} }
when(state RegisterWorkerRequest req = waitNext(interf.registerWorker.getFuture())) { when(RegisterWorkerRequest req = waitNext(interf.registerWorker.getFuture())) {
++self.registerWorkerRequests; ++self.registerWorkerRequests;
state ClusterConnectionString ccs = coordinators.ccr->getConnectionString(); self.addActor.send(registerWorker(req,
&self,
state std::unordered_set<NetworkAddress> coordinatorAddresses; coordinators.ccr->getConnectionString(),
std::vector<Future<Void>> fs; (configDBType == ConfigDBType::DISABLED) ? nullptr : &configBroadcaster));
for (auto& hostname : ccs.hostnames) {
fs.push_back(map(hostname.resolve(), [&](Optional<NetworkAddress> const& addr) -> Void {
if (addr.present()) {
coordinatorAddresses.insert(addr.get());
}
return Void();
}));
}
wait(waitForAll(fs));
for (const auto& coord : ccs.coordinators()) {
coordinatorAddresses.insert(coord);
}
registerWorker(req,
&self,
coordinatorAddresses,
(configDBType == ConfigDBType::DISABLED) ? nullptr : &configBroadcaster);
} }
when(GetWorkersRequest req = waitNext(interf.getWorkers.getFuture())) { when(GetWorkersRequest req = waitNext(interf.getWorkers.getFuture())) {
++self.getWorkersRequests; ++self.getWorkersRequests;

View File

@ -75,7 +75,7 @@ struct GenerationRegVal {
} }
}; };
GenerationRegInterface::GenerationRegInterface(NetworkAddress remote) GenerationRegInterface::GenerationRegInterface(NetworkAddress const& remote)
: read(Endpoint::wellKnown({ remote }, WLTOKEN_GENERATIONREG_READ)), : read(Endpoint::wellKnown({ remote }, WLTOKEN_GENERATIONREG_READ)),
write(Endpoint::wellKnown({ remote }, WLTOKEN_GENERATIONREG_WRITE)) {} write(Endpoint::wellKnown({ remote }, WLTOKEN_GENERATIONREG_WRITE)) {}
@ -84,7 +84,7 @@ GenerationRegInterface::GenerationRegInterface(INetwork* local) {
write.makeWellKnownEndpoint(WLTOKEN_GENERATIONREG_WRITE, TaskPriority::Coordination); write.makeWellKnownEndpoint(WLTOKEN_GENERATIONREG_WRITE, TaskPriority::Coordination);
} }
LeaderElectionRegInterface::LeaderElectionRegInterface(NetworkAddress remote) LeaderElectionRegInterface::LeaderElectionRegInterface(NetworkAddress const& remote)
: ClientLeaderRegInterface(remote), candidacy(Endpoint::wellKnown({ remote }, WLTOKEN_LEADERELECTIONREG_CANDIDACY)), : ClientLeaderRegInterface(remote), candidacy(Endpoint::wellKnown({ remote }, WLTOKEN_LEADERELECTIONREG_CANDIDACY)),
electionResult(Endpoint::wellKnown({ remote }, WLTOKEN_LEADERELECTIONREG_ELECTIONRESULT)), electionResult(Endpoint::wellKnown({ remote }, WLTOKEN_LEADERELECTIONREG_ELECTIONRESULT)),
leaderHeartbeat(Endpoint::wellKnown({ remote }, WLTOKEN_LEADERELECTIONREG_LEADERHEARTBEAT)), leaderHeartbeat(Endpoint::wellKnown({ remote }, WLTOKEN_LEADERELECTIONREG_LEADERHEARTBEAT)),

View File

@ -53,9 +53,9 @@ struct GenerationRegInterface {
// the v2 of the previous generation is the v1 of the next. // the v2 of the previous generation is the v1 of the next.
GenerationRegInterface() {} GenerationRegInterface() {}
GenerationRegInterface(NetworkAddress remote); GenerationRegInterface(NetworkAddress const& remote);
GenerationRegInterface(INetwork* local); GenerationRegInterface(INetwork* local);
GenerationRegInterface(Hostname hostname) : hostname(hostname){}; GenerationRegInterface(Hostname const& hostname) : hostname(hostname){};
}; };
struct UniqueGeneration { struct UniqueGeneration {
@ -128,9 +128,9 @@ struct LeaderElectionRegInterface : ClientLeaderRegInterface {
RequestStream<struct ForwardRequest> forward; RequestStream<struct ForwardRequest> forward;
LeaderElectionRegInterface() {} LeaderElectionRegInterface() {}
LeaderElectionRegInterface(NetworkAddress remote); LeaderElectionRegInterface(NetworkAddress const& remote);
LeaderElectionRegInterface(INetwork* local); LeaderElectionRegInterface(INetwork* local);
LeaderElectionRegInterface(Hostname hostname) : ClientLeaderRegInterface(hostname) {} LeaderElectionRegInterface(Hostname const& hostname) : ClientLeaderRegInterface(hostname) {}
}; };
struct CandidacyRequest { struct CandidacyRequest {
@ -220,7 +220,7 @@ class ConfigNode;
class ServerCoordinators : public ClientCoordinators { class ServerCoordinators : public ClientCoordinators {
public: public:
explicit ServerCoordinators(Reference<IClusterConnectionRecord>); explicit ServerCoordinators(Reference<IClusterConnectionRecord> ccr);
std::vector<LeaderElectionRegInterface> leaderElectionServers; std::vector<LeaderElectionRegInterface> leaderElectionServers;
std::vector<GenerationRegInterface> stateServers; std::vector<GenerationRegInterface> stateServers;

View File

@ -51,7 +51,7 @@ ACTOR Future<Void> submitCandidacy(Key key,
.detail("OldAddr", coord.candidacy.getEndpoint().getPrimaryAddress().toString()); .detail("OldAddr", coord.candidacy.getEndpoint().getPrimaryAddress().toString());
if (rep.getError().code() == error_code_request_maybe_delivered) { if (rep.getError().code() == error_code_request_maybe_delivered) {
// Delay to prevent tight resolving loop due to outdated DNS cache // Delay to prevent tight resolving loop due to outdated DNS cache
wait(delay(FLOW_KNOBS->HOSTNAME_RESOLVE_DELAY)); wait(delay(FLOW_KNOBS->HOSTNAME_RECONNECT_INIT_INTERVAL));
throw coordinators_changed(); throw coordinators_changed();
} else { } else {
throw rep.getError(); throw rep.getError();

View File

@ -84,13 +84,15 @@ ACTOR Future<Optional<NetworkAddress>> resolveImpl(Hostname* self) {
} }
ACTOR Future<NetworkAddress> resolveWithRetryImpl(Hostname* self) { ACTOR Future<NetworkAddress> resolveWithRetryImpl(Hostname* self) {
state double resolveInterval = FLOW_KNOBS->HOSTNAME_RESOLVE_INIT_INTERVAL;
loop { loop {
try { try {
Optional<NetworkAddress> address = wait(resolveImpl(self)); Optional<NetworkAddress> address = wait(resolveImpl(self));
if (address.present()) { if (address.present()) {
return address.get(); return address.get();
} }
wait(delay(FLOW_KNOBS->HOSTNAME_RESOLVE_DELAY)); wait(delay(resolveInterval));
resolveInterval = std::min(2 * resolveInterval, FLOW_KNOBS->HOSTNAME_RESOLVE_MAX_INTERVAL);
} catch (Error& e) { } catch (Error& e) {
ASSERT(e.code() == error_code_actor_cancelled); ASSERT(e.code() == error_code_actor_cancelled);
throw; throw;

View File

@ -74,6 +74,7 @@ struct Hostname {
Optional<NetworkAddress> resolvedAddress; Optional<NetworkAddress> resolvedAddress;
enum HostnameStatus { UNRESOLVED, RESOLVING, RESOLVED }; enum HostnameStatus { UNRESOLVED, RESOLVING, RESOLVED };
// The resolve functions below use DNS cache.
Future<Optional<NetworkAddress>> resolve(); Future<Optional<NetworkAddress>> resolve();
Future<NetworkAddress> resolveWithRetry(); Future<NetworkAddress> resolveWithRetry();
Optional<NetworkAddress> resolveBlocking(); // This one should only be used when resolving asynchronously is Optional<NetworkAddress> resolveBlocking(); // This one should only be used when resolving asynchronously is
@ -81,6 +82,11 @@ struct Hostname {
void resetToUnresolved(); void resetToUnresolved();
HostnameStatus status = UNRESOLVED; HostnameStatus status = UNRESOLVED;
AsyncTrigger resolveFinish; AsyncTrigger resolveFinish;
template <class Ar>
void serialize(Ar& ar) {
serializer(ar, host, service, isTLS, resolvedAddress, status);
}
}; };
#endif #endif

View File

@ -40,7 +40,10 @@ FlowKnobs const* FLOW_KNOBS = &bootstrapGlobalFlowKnobs;
void FlowKnobs::initialize(Randomize randomize, IsSimulated isSimulated) { void FlowKnobs::initialize(Randomize randomize, IsSimulated isSimulated) {
init( AUTOMATIC_TRACE_DUMP, 1 ); init( AUTOMATIC_TRACE_DUMP, 1 );
init( PREVENT_FAST_SPIN_DELAY, .01 ); init( PREVENT_FAST_SPIN_DELAY, .01 );
init( HOSTNAME_RESOLVE_DELAY, .05 ); init( HOSTNAME_RESOLVE_INIT_INTERVAL, .05 );
init( HOSTNAME_RESOLVE_MAX_INTERVAL, 1.0 );
init( HOSTNAME_RECONNECT_INIT_INTERVAL, .05 );
init( HOSTNAME_RECONNECT_MAX_INTERVAL, 1.0 );
init( CACHE_REFRESH_INTERVAL_WHEN_ALL_ALTERNATIVES_FAILED, 1.0 ); init( CACHE_REFRESH_INTERVAL_WHEN_ALL_ALTERNATIVES_FAILED, 1.0 );
init( DELAY_JITTER_OFFSET, 0.9 ); init( DELAY_JITTER_OFFSET, 0.9 );

View File

@ -113,7 +113,10 @@ class FlowKnobs : public KnobsImpl<FlowKnobs> {
public: public:
int AUTOMATIC_TRACE_DUMP; int AUTOMATIC_TRACE_DUMP;
double PREVENT_FAST_SPIN_DELAY; double PREVENT_FAST_SPIN_DELAY;
double HOSTNAME_RESOLVE_DELAY; double HOSTNAME_RESOLVE_INIT_INTERVAL;
double HOSTNAME_RESOLVE_MAX_INTERVAL;
double HOSTNAME_RECONNECT_INIT_INTERVAL;
double HOSTNAME_RECONNECT_MAX_INTERVAL;
double CACHE_REFRESH_INTERVAL_WHEN_ALL_ALTERNATIVES_FAILED; double CACHE_REFRESH_INTERVAL_WHEN_ALL_ALTERNATIVES_FAILED;
double DELAY_JITTER_OFFSET; double DELAY_JITTER_OFFSET;