[backoff] fix bug and improve API (#37595)

Fixes a bug in the backoff implementation whereby we were incorrectly failing to apply jitter to the initial backoff.

Also change the API to return `Duration` instead of `Timestamp`.  The only caller that actually wants to count the backoff from the start of the previous attempt instead of the end of the previous attempt is the subchannel code, and it handles that on its end.

Closes #37595

COPYBARA_INTEGRATE_REVIEW=https://github.com/grpc/grpc/pull/37595 from markdroth:backoff_fixes_and_api_improvement 39d083c0f4
PiperOrigin-RevId: 669112557
pull/37598/head
Mark D. Roth 6 months ago committed by Copybara-Service
parent b69291a18e
commit 81d7bdc4c5
  1. 2
      src/core/client_channel/retry_filter_legacy_call_data.cc
  2. 5
      src/core/client_channel/subchannel.cc
  3. 2
      src/core/client_channel/subchannel_stream_client.cc
  4. 17
      src/core/lib/backoff/backoff.cc
  5. 10
      src/core/lib/backoff/backoff.h
  6. 8
      src/core/lib/event_engine/thread_pool/work_stealing_thread_pool.cc
  7. 8
      src/core/lib/security/credentials/token_fetcher/token_fetcher_credentials.cc
  8. 8
      src/core/load_balancing/grpclb/grpclb.cc
  9. 16
      src/core/load_balancing/rls/rls.cc
  10. 21
      src/core/resolver/polling_resolver.cc
  11. 2
      src/core/resolver/polling_resolver.h
  12. 8
      src/core/xds/xds_client/xds_client.cc
  13. 1
      test/core/backoff/BUILD
  14. 191
      test/core/backoff/backoff_test.cc
  15. 2
      test/core/end2end/tests/retry_cancel_during_delay.cc
  16. 8
      test/cpp/end2end/client_lb_end2end_test.cc

@ -1890,7 +1890,7 @@ void RetryFilter::LegacyCallData::StartRetryTimer(
next_attempt_timeout = *server_pushback;
retry_backoff_.Reset();
} else {
next_attempt_timeout = retry_backoff_.NextAttemptTime() - Timestamp::Now();
next_attempt_timeout = retry_backoff_.NextAttemptDelay();
}
GRPC_TRACE_LOG(retry, INFO)
<< "chand=" << chand_ << " calld=" << this << ": retrying failed call in "

@ -764,8 +764,9 @@ void Subchannel::OnRetryTimerLocked() {
void Subchannel::StartConnectingLocked() {
// Set next attempt time.
const Timestamp min_deadline = min_connect_timeout_ + Timestamp::Now();
next_attempt_time_ = backoff_.NextAttemptTime();
const Timestamp now = Timestamp::Now();
const Timestamp min_deadline = now + min_connect_timeout_;
next_attempt_time_ = now + backoff_.NextAttemptDelay();
// Report CONNECTING.
SetConnectivityStateLocked(GRPC_CHANNEL_CONNECTING, absl::OkStatus());
// Start connection attempt.

@ -130,7 +130,7 @@ void SubchannelStreamClient::StartRetryTimerLocked() {
if (event_handler_ != nullptr) {
event_handler_->OnRetryTimerStartLocked(this);
}
const Duration timeout = retry_backoff_.NextAttemptTime() - Timestamp::Now();
const Duration timeout = retry_backoff_.NextAttemptDelay();
if (GPR_UNLIKELY(tracer_ != nullptr)) {
LOG(INFO) << tracer_ << " " << this
<< ": SubchannelStreamClient health check call lost...";

@ -20,23 +20,20 @@
#include <algorithm>
#include <grpc/support/port_platform.h>
namespace grpc_core {
BackOff::BackOff(const Options& options) : options_(options) { Reset(); }
Timestamp BackOff::NextAttemptTime() {
Duration BackOff::NextAttemptDelay() {
if (initial_) {
initial_ = false;
return current_backoff_ + Timestamp::Now();
} else {
current_backoff_ = std::min(current_backoff_ * options_.multiplier(),
options_.max_backoff());
}
current_backoff_ = std::min(current_backoff_ * options_.multiplier(),
options_.max_backoff());
const Duration jitter = Duration::FromSecondsAsDouble(
absl::Uniform(rand_gen_, -options_.jitter() * current_backoff_.seconds(),
options_.jitter() * current_backoff_.seconds()));
return Timestamp::Now() + current_backoff_ + jitter;
const double jitter =
absl::Uniform(rand_gen_, 1 - options_.jitter(), 1 + options_.jitter());
return current_backoff_ * jitter;
}
void BackOff::Reset() {

@ -21,8 +21,6 @@
#include "absl/random/random.h"
#include <grpc/support/port_platform.h>
#include "src/core/lib/gprpp/time.h"
namespace grpc_core {
@ -36,11 +34,11 @@ class BackOff {
/// Initialize backoff machinery - does not need to be destroyed
explicit BackOff(const Options& options);
/// Returns the time at which the next attempt should start.
Timestamp NextAttemptTime();
/// Returns the delay before the next attempt should start.
Duration NextAttemptDelay();
/// Reset the backoff, so the next value returned by NextAttemptTime()
/// will be the time of the second attempt (rather than the Nth).
/// Reset the backoff, so the next value returned by NextAttemptDelay()
/// will be the delay for the second attempt (rather than the Nth).
void Reset();
class Options {

@ -400,9 +400,7 @@ void WorkStealingThreadPool::WorkStealingThreadPoolImpl::Lifeguard::
if (pool_->IsQuiesced()) break;
} else {
lifeguard_should_shut_down_->WaitForNotificationWithTimeout(
absl::Milliseconds(
(backoff_.NextAttemptTime() - grpc_core::Timestamp::Now())
.millis()));
absl::Milliseconds(backoff_.NextAttemptDelay().millis()));
}
MaybeStartNewThread();
}
@ -556,8 +554,8 @@ bool WorkStealingThreadPool::ThreadState::Step() {
// No closures were retrieved from anywhere.
// Quit the thread if the pool has been shut down.
if (pool_->IsShutdown()) break;
bool timed_out = pool_->work_signal()->WaitWithTimeout(
backoff_.NextAttemptTime() - grpc_core::Timestamp::Now());
bool timed_out =
pool_->work_signal()->WaitWithTimeout(backoff_.NextAttemptDelay());
if (pool_->IsForking() || pool_->IsShutdown()) break;
// Quit a thread if the pool has more than it requires, and this thread
// has been idle long enough.

@ -60,15 +60,13 @@ void TokenFetcherCredentials::Token::AddTokenToClientInitialMetadata(
TokenFetcherCredentials::FetchState::BackoffTimer::BackoffTimer(
RefCountedPtr<FetchState> fetch_state)
: fetch_state_(std::move(fetch_state)) {
const Timestamp next_attempt_time = fetch_state_->backoff_.NextAttemptTime();
const Duration duration = next_attempt_time - Timestamp::Now();
const Duration delay = fetch_state_->backoff_.NextAttemptDelay();
GRPC_TRACE_LOG(token_fetcher_credentials, INFO)
<< "[TokenFetcherCredentials " << fetch_state_->creds_.get()
<< "]: fetch_state=" << fetch_state_.get() << " backoff_timer=" << this
<< ": starting backoff timer for " << next_attempt_time << " ("
<< duration << " from now)";
<< ": starting backoff timer for " << delay;
timer_handle_ = fetch_state_->creds_->event_engine().RunAfter(
duration, [self = Ref()]() mutable {
delay, [self = Ref()]() mutable {
ApplicationCallbackExecCtx callback_exec_ctx;
ExecCtx exec_ctx;
self->OnTimer();

@ -1655,12 +1655,12 @@ void GrpcLb::StartBalancerCallLocked() {
}
void GrpcLb::StartBalancerCallRetryTimerLocked() {
Duration timeout = lb_call_backoff_.NextAttemptTime() - Timestamp::Now();
Duration delay = lb_call_backoff_.NextAttemptDelay();
if (GRPC_TRACE_FLAG_ENABLED(glb)) {
LOG(INFO) << "[grpclb " << this << "] Connection to LB server lost...";
if (timeout > Duration::Zero()) {
if (delay > Duration::Zero()) {
LOG(INFO) << "[grpclb " << this << "] ... retry_timer_active in "
<< timeout.millis() << "ms.";
<< delay.millis() << "ms.";
} else {
LOG(INFO) << "[grpclb " << this
<< "] ... retry_timer_active immediately.";
@ -1668,7 +1668,7 @@ void GrpcLb::StartBalancerCallRetryTimerLocked() {
}
lb_call_retry_timer_handle_ =
channel_control_helper()->GetEventEngine()->RunAfter(
timeout,
delay,
[self = RefAsSubclass<GrpcLb>(
DEBUG_LOCATION, "on_balancer_call_retry_timer")]() mutable {
ApplicationCallbackExecCtx callback_exec_ctx;

@ -526,7 +526,7 @@ class RlsLb final : public LoadBalancingPolicy {
private:
class BackoffTimer final : public InternallyRefCounted<BackoffTimer> {
public:
BackoffTimer(RefCountedPtr<Entry> entry, Timestamp backoff_time);
BackoffTimer(RefCountedPtr<Entry> entry, Duration delay);
// Note: We are forced to disable lock analysis here because
// Orphan() is called by OrphanablePtr<>, which cannot have lock
@ -1138,12 +1138,11 @@ LoadBalancingPolicy::PickResult RlsLb::Picker::PickFromDefaultTargetOrFail(
//
RlsLb::Cache::Entry::BackoffTimer::BackoffTimer(RefCountedPtr<Entry> entry,
Timestamp backoff_time)
Duration delay)
: entry_(std::move(entry)) {
backoff_timer_task_handle_ =
entry_->lb_policy_->channel_control_helper()->GetEventEngine()->RunAfter(
backoff_time - Timestamp::Now(),
[self = Ref(DEBUG_LOCATION, "BackoffTimer")]() mutable {
delay, [self = Ref(DEBUG_LOCATION, "BackoffTimer")]() mutable {
ApplicationCallbackExecCtx callback_exec_ctx;
ExecCtx exec_ctx;
auto self_ptr = self.get();
@ -1311,11 +1310,12 @@ RlsLb::Cache::Entry::OnRlsResponseLocked(
} else {
backoff_state_ = MakeCacheEntryBackoff();
}
backoff_time_ = backoff_state_->NextAttemptTime();
Timestamp now = Timestamp::Now();
backoff_expiration_time_ = now + (backoff_time_ - now) * 2;
const Duration delay = backoff_state_->NextAttemptDelay();
const Timestamp now = Timestamp::Now();
backoff_time_ = now + delay;
backoff_expiration_time_ = now + delay * 2;
backoff_timer_ = MakeOrphanable<BackoffTimer>(
Ref(DEBUG_LOCATION, "BackoffTimer"), backoff_time_);
Ref(DEBUG_LOCATION, "BackoffTimer"), delay);
lb_policy_->UpdatePickerAsync();
return {};
}

@ -104,10 +104,10 @@ void PollingResolver::ShutdownLocked() {
request_.reset();
}
void PollingResolver::ScheduleNextResolutionTimer(const Duration& timeout) {
void PollingResolver::ScheduleNextResolutionTimer(Duration delay) {
next_resolution_timer_handle_ =
channel_args_.GetObject<EventEngine>()->RunAfter(
timeout, [self = RefAsSubclass<PollingResolver>()]() mutable {
delay, [self = RefAsSubclass<PollingResolver>()]() mutable {
ApplicationCallbackExecCtx callback_exec_ctx;
ExecCtx exec_ctx;
auto* self_ptr = self.get();
@ -198,22 +198,13 @@ void PollingResolver::GetResultStatus(absl::Status status) {
}
} else {
// Set up for retry.
// InvalidateNow to avoid getting stuck re-initializing this timer
// in a loop while draining the currently-held WorkSerializer.
// Also see https://github.com/grpc/grpc/issues/26079.
ExecCtx::Get()->InvalidateNow();
const Timestamp next_try = backoff_.NextAttemptTime();
const Duration timeout = next_try - Timestamp::Now();
const Duration delay = backoff_.NextAttemptDelay();
CHECK(!next_resolution_timer_handle_.has_value());
if (GPR_UNLIKELY(tracer_ != nullptr && tracer_->enabled())) {
if (timeout > Duration::Zero()) {
LOG(INFO) << "[polling resolver " << this << "] retrying in "
<< timeout.millis() << " ms";
} else {
LOG(INFO) << "[polling resolver " << this << "] retrying immediately";
}
LOG(INFO) << "[polling resolver " << this << "] retrying in "
<< delay.millis() << " ms";
}
ScheduleNextResolutionTimer(timeout);
ScheduleNextResolutionTimer(delay);
// Reset result_status_state_. Note that even if re-resolution was
// requested while the result-health callback was pending, we can
// ignore it here, because we are in backoff to re-resolve anyway.

@ -77,7 +77,7 @@ class PollingResolver : public Resolver {
void OnRequestCompleteLocked(Result result);
void GetResultStatus(absl::Status status);
void ScheduleNextResolutionTimer(const Duration& timeout);
void ScheduleNextResolutionTimer(Duration delay);
void OnNextResolutionLocked();
void MaybeCancelNextResolutionTimer();

@ -715,16 +715,14 @@ void XdsClient::XdsChannel::RetryableCall<T>::StartNewCallLocked() {
template <typename T>
void XdsClient::XdsChannel::RetryableCall<T>::StartRetryTimerLocked() {
if (shutting_down_) return;
const Timestamp next_attempt_time = backoff_.NextAttemptTime();
const Duration timeout =
std::max(next_attempt_time - Timestamp::Now(), Duration::Zero());
const Duration delay = backoff_.NextAttemptDelay();
GRPC_TRACE_LOG(xds_client, INFO)
<< "[xds_client " << xds_channel()->xds_client() << "] xds server "
<< xds_channel()->server_.server_uri()
<< ": call attempt failed; retry timer will fire in " << timeout.millis()
<< ": call attempt failed; retry timer will fire in " << delay.millis()
<< "ms.";
timer_handle_ = xds_channel()->xds_client()->engine()->RunAfter(
timeout,
delay,
[self = this->Ref(DEBUG_LOCATION, "RetryableCall+retry_timer_start")]() {
ApplicationCallbackExecCtx callback_exec_ctx;
ExecCtx exec_ctx;

@ -32,7 +32,6 @@ grpc_cc_test(
uses_polling = False,
deps = [
"//:backoff",
"//:exec_ctx",
"//:grpc",
"//src/core:time",
"//test/core/test_util:grpc_test_util",

@ -21,162 +21,97 @@
#include <algorithm>
#include <memory>
#include "gmock/gmock.h"
#include "gtest/gtest.h"
#include <grpc/grpc.h>
#include "src/core/lib/gprpp/time.h"
#include "src/core/lib/iomgr/exec_ctx.h"
#include "test/core/test_util/test_config.h"
namespace grpc {
namespace grpc_core {
namespace testing {
namespace {
using grpc_core::BackOff;
TEST(BackOffTest, ConstantBackOff) {
const auto initial_backoff = grpc_core::Duration::Milliseconds(200);
const double multiplier = 1.0;
const double jitter = 0.0;
const auto max_backoff = grpc_core::Duration::Seconds(1);
grpc_core::ExecCtx exec_ctx;
const auto kInitialBackoff = Duration::Milliseconds(200);
const double kMultiplier = 1.0;
const double kJitter = 0.0;
const auto kMaxBackoff = Duration::Seconds(1);
BackOff::Options options;
options.set_initial_backoff(initial_backoff)
.set_multiplier(multiplier)
.set_jitter(jitter)
.set_max_backoff(max_backoff);
options.set_initial_backoff(kInitialBackoff)
.set_multiplier(kMultiplier)
.set_jitter(kJitter)
.set_max_backoff(kMaxBackoff);
BackOff backoff(options);
grpc_core::Timestamp next_attempt_start_time = backoff.NextAttemptTime();
EXPECT_EQ(next_attempt_start_time - grpc_core::Timestamp::Now(),
initial_backoff);
for (int i = 0; i < 10000; i++) {
next_attempt_start_time = backoff.NextAttemptTime();
EXPECT_EQ(next_attempt_start_time - grpc_core::Timestamp::Now(),
initial_backoff);
}
EXPECT_EQ(backoff.NextAttemptDelay(), kInitialBackoff);
EXPECT_EQ(backoff.NextAttemptDelay(), kInitialBackoff);
EXPECT_EQ(backoff.NextAttemptDelay(), kInitialBackoff);
EXPECT_EQ(backoff.NextAttemptDelay(), kInitialBackoff);
EXPECT_EQ(backoff.NextAttemptDelay(), kInitialBackoff);
}
TEST(BackOffTest, MinConnect) {
const auto initial_backoff = grpc_core::Duration::Milliseconds(100);
const double multiplier = 1.0;
const double jitter = 0.0;
const auto max_backoff = grpc_core::Duration::Seconds(1);
grpc_core::ExecCtx exec_ctx;
TEST(BackOffTest, NoJitterBackOff) {
const auto kInitialBackoff = Duration::Milliseconds(2);
const double kMultiplier = 2.0;
const double kJitter = 0.0;
const auto kMaxBackoff = Duration::Milliseconds(32);
BackOff::Options options;
options.set_initial_backoff(initial_backoff)
.set_multiplier(multiplier)
.set_jitter(jitter)
.set_max_backoff(max_backoff);
options.set_initial_backoff(kInitialBackoff)
.set_multiplier(kMultiplier)
.set_jitter(kJitter)
.set_max_backoff(kMaxBackoff);
BackOff backoff(options);
grpc_core::Timestamp next = backoff.NextAttemptTime();
EXPECT_EQ(next - grpc_core::Timestamp::Now(), initial_backoff);
EXPECT_EQ(backoff.NextAttemptDelay(), Duration::Milliseconds(2));
EXPECT_EQ(backoff.NextAttemptDelay(), Duration::Milliseconds(4));
EXPECT_EQ(backoff.NextAttemptDelay(), Duration::Milliseconds(8));
EXPECT_EQ(backoff.NextAttemptDelay(), Duration::Milliseconds(16));
EXPECT_EQ(backoff.NextAttemptDelay(), Duration::Milliseconds(32));
// No more increases after kMaxBackoff.
EXPECT_EQ(backoff.NextAttemptDelay(), Duration::Milliseconds(32));
EXPECT_EQ(backoff.NextAttemptDelay(), Duration::Milliseconds(32));
}
TEST(BackOffTest, NoJitterBackOff) {
const auto initial_backoff = grpc_core::Duration::Milliseconds(2);
const double multiplier = 2.0;
const double jitter = 0.0;
const auto max_backoff = grpc_core::Duration::Milliseconds(513);
BackOff::Options options;
options.set_initial_backoff(initial_backoff)
.set_multiplier(multiplier)
.set_jitter(jitter)
.set_max_backoff(max_backoff);
BackOff backoff(options);
// x_1 = 2
// x_n = 2**i + x_{i-1} ( = 2**(n+1) - 2 )
grpc_core::ExecCtx exec_ctx;
grpc_core::ExecCtx::Get()->TestOnlySetNow(
grpc_core::Timestamp::FromMillisecondsAfterProcessEpoch(0));
grpc_core::Timestamp next = backoff.NextAttemptTime();
EXPECT_EQ(next, grpc_core::Timestamp::FromMillisecondsAfterProcessEpoch(2));
grpc_core::ExecCtx::Get()->TestOnlySetNow(next);
next = backoff.NextAttemptTime();
EXPECT_EQ(next, grpc_core::Timestamp::FromMillisecondsAfterProcessEpoch(6));
grpc_core::ExecCtx::Get()->TestOnlySetNow(next);
next = backoff.NextAttemptTime();
EXPECT_EQ(next, grpc_core::Timestamp::FromMillisecondsAfterProcessEpoch(14));
grpc_core::ExecCtx::Get()->TestOnlySetNow(next);
next = backoff.NextAttemptTime();
EXPECT_EQ(next, grpc_core::Timestamp::FromMillisecondsAfterProcessEpoch(30));
grpc_core::ExecCtx::Get()->TestOnlySetNow(next);
next = backoff.NextAttemptTime();
EXPECT_EQ(next, grpc_core::Timestamp::FromMillisecondsAfterProcessEpoch(62));
grpc_core::ExecCtx::Get()->TestOnlySetNow(next);
next = backoff.NextAttemptTime();
EXPECT_EQ(next, grpc_core::Timestamp::FromMillisecondsAfterProcessEpoch(126));
grpc_core::ExecCtx::Get()->TestOnlySetNow(next);
next = backoff.NextAttemptTime();
EXPECT_EQ(next, grpc_core::Timestamp::FromMillisecondsAfterProcessEpoch(254));
grpc_core::ExecCtx::Get()->TestOnlySetNow(next);
next = backoff.NextAttemptTime();
EXPECT_EQ(next, grpc_core::Timestamp::FromMillisecondsAfterProcessEpoch(510));
grpc_core::ExecCtx::Get()->TestOnlySetNow(next);
next = backoff.NextAttemptTime();
EXPECT_EQ(next,
grpc_core::Timestamp::FromMillisecondsAfterProcessEpoch(1022));
grpc_core::ExecCtx::Get()->TestOnlySetNow(next);
next = backoff.NextAttemptTime();
// Hit the maximum timeout. From this point onwards, retries will increase
// only by max timeout.
EXPECT_EQ(next,
grpc_core::Timestamp::FromMillisecondsAfterProcessEpoch(1535));
grpc_core::ExecCtx::Get()->TestOnlySetNow(next);
next = backoff.NextAttemptTime();
EXPECT_EQ(next,
grpc_core::Timestamp::FromMillisecondsAfterProcessEpoch(2048));
grpc_core::ExecCtx::Get()->TestOnlySetNow(next);
next = backoff.NextAttemptTime();
EXPECT_EQ(next,
grpc_core::Timestamp::FromMillisecondsAfterProcessEpoch(2561));
MATCHER_P2(InJitterRange, value, jitter, "") {
bool ok = true;
ok &= ::testing::ExplainMatchResult(::testing::Ge(arg * (1 - jitter)), arg,
result_listener);
ok &= ::testing::ExplainMatchResult(::testing::Le(arg * (1 + jitter)), arg,
result_listener);
return ok;
}
TEST(BackOffTest, JitterBackOff) {
const auto initial_backoff = grpc_core::Duration::Milliseconds(500);
auto current_backoff = initial_backoff;
const auto max_backoff = grpc_core::Duration::Seconds(1);
const double multiplier = 1.0;
const double jitter = 0.1;
const auto kInitialBackoff = Duration::Milliseconds(2);
const double kMultiplier = 2.0;
const double kJitter = 0.1;
const auto kMaxBackoff = Duration::Milliseconds(32);
BackOff::Options options;
options.set_initial_backoff(initial_backoff)
.set_multiplier(multiplier)
.set_jitter(jitter)
.set_max_backoff(max_backoff);
options.set_initial_backoff(kInitialBackoff)
.set_multiplier(kMultiplier)
.set_jitter(kJitter)
.set_max_backoff(kMaxBackoff);
BackOff backoff(options);
grpc_core::ExecCtx exec_ctx;
grpc_core::Timestamp next = backoff.NextAttemptTime();
EXPECT_EQ(next - grpc_core::Timestamp::Now(), initial_backoff);
auto expected_next_lower_bound = grpc_core::Duration::Milliseconds(
static_cast<double>(current_backoff.millis()) * (1 - jitter));
auto expected_next_upper_bound = grpc_core::Duration::Milliseconds(
static_cast<double>(current_backoff.millis()) * (1 + jitter));
for (int i = 0; i < 10000; i++) {
next = backoff.NextAttemptTime();
// next-now must be within (jitter*100)% of the current backoff (which
// increases by * multiplier up to max_backoff).
const grpc_core::Duration timeout_millis =
next - grpc_core::Timestamp::Now();
EXPECT_GE(timeout_millis, expected_next_lower_bound);
EXPECT_LE(timeout_millis, expected_next_upper_bound);
current_backoff = std::min(
grpc_core::Duration::Milliseconds(
static_cast<double>(current_backoff.millis()) * multiplier),
max_backoff);
expected_next_lower_bound = grpc_core::Duration::Milliseconds(
static_cast<double>(current_backoff.millis()) * (1 - jitter));
expected_next_upper_bound = grpc_core::Duration::Milliseconds(
static_cast<double>(current_backoff.millis()) * (1 + jitter));
}
EXPECT_THAT(backoff.NextAttemptDelay(),
InJitterRange(Duration::Milliseconds(2), kJitter));
EXPECT_THAT(backoff.NextAttemptDelay(),
InJitterRange(Duration::Milliseconds(4), kJitter));
EXPECT_THAT(backoff.NextAttemptDelay(),
InJitterRange(Duration::Milliseconds(8), kJitter));
EXPECT_THAT(backoff.NextAttemptDelay(),
InJitterRange(Duration::Milliseconds(16), kJitter));
EXPECT_THAT(backoff.NextAttemptDelay(),
InJitterRange(Duration::Milliseconds(32), kJitter));
// No more increases after kMaxBackoff.
EXPECT_THAT(backoff.NextAttemptDelay(),
InJitterRange(Duration::Milliseconds(32), kJitter));
EXPECT_THAT(backoff.NextAttemptDelay(),
InJitterRange(Duration::Milliseconds(32), kJitter));
}
} // namespace
} // namespace testing
} // namespace grpc
} // namespace grpc_core
int main(int argc, char** argv) {
::testing::InitGoogleTest(&argc, argv);

@ -56,7 +56,7 @@ void TestRetryCancelDuringDelay(
35 * grpc_test_slowdown_factor())));
auto expect_finish_before = test.TimestampAfterDuration(Duration::Minutes(2));
auto c = test.NewClientCall("/service/method")
.Timeout(Duration::Seconds(30))
.Timeout(Duration::Seconds(20))
.Create();
EXPECT_NE(c.GetPeer(), absl::nullopt);
// Client starts a batch with all 6 ops.

@ -982,11 +982,11 @@ TEST_F(PickFirstTest, BackOffInitialReconnect) {
// Check how long it took.
const grpc_core::Duration waited = grpc_core::Timestamp::Now() - t0;
VLOG(2) << "Waited " << waited.millis() << " milliseconds";
// We should have waited at least kInitialBackOffMs. We substract one to
// account for test and precision accuracy drift.
// We should have waited at least kInitialBackOffMs, plus or minus
// jitter. We give a little more leeway on the high end to account
// for things taking a little longer than expected in other threads.
EXPECT_GE(waited.millis(),
(kInitialBackOffMs * grpc_test_slowdown_factor()) - 1);
// But not much more.
(kInitialBackOffMs * grpc_test_slowdown_factor()) * 0.8);
EXPECT_LE(waited.millis(),
(kInitialBackOffMs * grpc_test_slowdown_factor()) * 1.3);
}

Loading…
Cancel
Save