diff --git a/ent/src/yb/cdc/cdc_rpc.cc b/ent/src/yb/cdc/cdc_rpc.cc index 21e08a844b23..e7fb80896db9 100644 --- a/ent/src/yb/cdc/cdc_rpc.cc +++ b/ent/src/yb/cdc/cdc_rpc.cc @@ -90,7 +90,7 @@ class CDCWriteRpc : public rpc::Rpc, public client::internal::TabletRpc { private: void SendRpcToTserver(int attempt_num) override { InvokeAsync(invoker_.proxy().get(), - PrepareController(invoker_.client().default_rpc_timeout()), + PrepareController(), std::bind(&CDCWriteRpc::Finished, this, Status::OK())); } @@ -219,7 +219,7 @@ class CDCReadRpc : public rpc::Rpc, public client::internal::TabletRpc { &invoker_.client().proxy_cache(), invoker_.ProxyEndpoint()); InvokeAsync(cdc_proxy_.get(), - PrepareController(invoker_.client().default_rpc_timeout()), + PrepareController(), std::bind(&CDCReadRpc::Finished, this, Status::OK())); } diff --git a/java/yb-pgsql/src/test/java/org/yb/pgsql/BasePgSQLTest.java b/java/yb-pgsql/src/test/java/org/yb/pgsql/BasePgSQLTest.java index 8bf53b07488c..dcd3cc8c43b3 100644 --- a/java/yb-pgsql/src/test/java/org/yb/pgsql/BasePgSQLTest.java +++ b/java/yb-pgsql/src/test/java/org/yb/pgsql/BasePgSQLTest.java @@ -189,11 +189,7 @@ protected static int getPerfMaxRuntime(int releaseRuntime, } protected Map getMasterAndTServerFlags() { - Map flagMap = new TreeMap<>(); - flagMap.put( - "retryable_rpc_single_call_timeout_ms", - String.valueOf(getRetryableRpcSingleCallTimeoutMs())); - return flagMap; + return new TreeMap<>(); } protected Integer getYsqlPrefetchLimit() { diff --git a/src/yb/client/async_rpc.cc b/src/yb/client/async_rpc.cc index 3498d732de6b..a3b269ee30bb 100644 --- a/src/yb/client/async_rpc.cc +++ b/src/yb/client/async_rpc.cc @@ -105,7 +105,7 @@ AsyncRpcMetrics::AsyncRpcMetrics(const scoped_refptr& entity) time_to_send(METRIC_handler_latency_yb_client_time_to_send.Instantiate(entity)) { } -AsyncRpc::AsyncRpc(AsyncRpcData* data, YBConsistencyLevel yb_consistency_level, MonoDelta timeout) +AsyncRpc::AsyncRpc(AsyncRpcData* data, YBConsistencyLevel yb_consistency_level) : Rpc(data->batcher->deadline(), data->batcher->messenger(), &data->batcher->proxy_cache()), batcher_(data->batcher), trace_(new Trace), @@ -119,8 +119,7 @@ AsyncRpc::AsyncRpc(AsyncRpcData* data, YBConsistencyLevel yb_consistency_level, trace_.get()), ops_(std::move(data->ops)), start_(MonoTime::Now()), - async_rpc_metrics_(data->batcher->async_rpc_metrics()), - timeout_(timeout) { + async_rpc_metrics_(data->batcher->async_rpc_metrics()) { mutable_retrier()->mutable_controller()->set_allow_local_calls_in_curr_thread( data->allow_local_calls_in_curr_thread); @@ -271,9 +270,8 @@ void AsyncRpc::SendRpcToTserver(int attempt_num) { template AsyncRpcBase::AsyncRpcBase(AsyncRpcData* data, - YBConsistencyLevel consistency_level, - MonoDelta timeout) - : AsyncRpc(data, consistency_level, timeout) { + YBConsistencyLevel consistency_level) + : AsyncRpc(data, consistency_level) { req_.set_tablet_id(tablet_invoker_.tablet()->tablet_id()); req_.set_include_trace(IsTracingEnabled()); @@ -350,8 +348,8 @@ void AsyncRpcBase::SendRpcToTserver(int attempt_num) { AsyncRpc::SendRpcToTserver(attempt_num); } -WriteRpc::WriteRpc(AsyncRpcData* data, MonoDelta timeout) - : AsyncRpcBase(data, YBConsistencyLevel::STRONG, timeout) { +WriteRpc::WriteRpc(AsyncRpcData* data) + : AsyncRpcBase(data, YBConsistencyLevel::STRONG) { TRACE_TO(trace_, "WriteRpc initiated to $0", data->tablet->tablet_id()); @@ -444,7 +442,7 @@ void WriteRpc::CallRemoteMethod() { ADOPT_TRACE(trace.get()); tablet_invoker_.proxy()->WriteAsync( - req_, &resp_, PrepareController(timeout_), + req_, &resp_, PrepareController(), std::bind(&WriteRpc::Finished, this, Status::OK())); TRACE_TO(trace, "RpcDispatched Asynchronously"); } @@ -581,8 +579,8 @@ void WriteRpc::ProcessResponseFromTserver(const Status& status) { SwapRequestsAndResponses(false); } -ReadRpc::ReadRpc(AsyncRpcData* data, YBConsistencyLevel yb_consistency_level, MonoDelta timeout) - : AsyncRpcBase(data, yb_consistency_level, timeout) { +ReadRpc::ReadRpc(AsyncRpcData* data, YBConsistencyLevel yb_consistency_level) + : AsyncRpcBase(data, yb_consistency_level) { TRACE_TO(trace_, "ReadRpc initiated to $0", data->tablet->tablet_id()); req_.set_consistency_level(yb_consistency_level); @@ -658,7 +656,7 @@ void ReadRpc::CallRemoteMethod() { ADOPT_TRACE(trace.get()); tablet_invoker_.proxy()->ReadAsync( - req_, &resp_, PrepareController(timeout_), + req_, &resp_, PrepareController(), std::bind(&ReadRpc::Finished, this, Status::OK())); TRACE_TO(trace, "RpcDispatched Asynchronously"); } diff --git a/src/yb/client/async_rpc.h b/src/yb/client/async_rpc.h index cf0acf05718e..ad3ed89c2c90 100644 --- a/src/yb/client/async_rpc.h +++ b/src/yb/client/async_rpc.h @@ -73,7 +73,7 @@ struct FlushExtraResult { // This class deletes itself after Rpc returns and is processed. class AsyncRpc : public rpc::Rpc, public TabletRpc { public: - explicit AsyncRpc(AsyncRpcData* data, YBConsistencyLevel consistency_level, MonoDelta timeout); + explicit AsyncRpc(AsyncRpcData* data, YBConsistencyLevel consistency_level); virtual ~AsyncRpc(); @@ -119,16 +119,12 @@ class AsyncRpc : public rpc::Rpc, public TabletRpc { MonoTime start_; std::shared_ptr async_rpc_metrics_; rpc::RpcCommandPtr retained_self_; - - // Amount of time to wait for a given op, from start to finish. - MonoDelta timeout_; }; template class AsyncRpcBase : public AsyncRpc { public: - explicit AsyncRpcBase(AsyncRpcData* data, YBConsistencyLevel consistency_level, - MonoDelta timeout); + explicit AsyncRpcBase(AsyncRpcData* data, YBConsistencyLevel consistency_level); const Resp& resp() const { return resp_; } Resp& resp() { return resp_; } @@ -155,7 +151,7 @@ class AsyncRpcBase : public AsyncRpc { class WriteRpc : public AsyncRpcBase { public: - explicit WriteRpc(AsyncRpcData* data, MonoDelta timeout = MonoDelta::kZero); + explicit WriteRpc(AsyncRpcData* data); virtual ~WriteRpc(); @@ -168,8 +164,7 @@ class WriteRpc : public AsyncRpcBase { public: explicit ReadRpc( - AsyncRpcData* data, YBConsistencyLevel yb_consistency_level = YBConsistencyLevel::STRONG, - MonoDelta timeout = MonoDelta::kZero); + AsyncRpcData* data, YBConsistencyLevel yb_consistency_level = YBConsistencyLevel::STRONG); virtual ~ReadRpc(); diff --git a/src/yb/client/batcher.cc b/src/yb/client/batcher.cc index d407f028a9fb..e0929083a550 100644 --- a/src/yb/client/batcher.cc +++ b/src/yb/client/batcher.cc @@ -165,12 +165,6 @@ void Batcher::SetTimeout(MonoDelta timeout) { timeout_ = timeout; } -void Batcher::SetSingleRpcTimeout(MonoDelta timeout) { - CHECK_GE(timeout, MonoDelta::kZero); - std::lock_guard lock(mutex_); - single_rpc_timeout_ = timeout; -} - bool Batcher::HasPendingOperations() const { std::lock_guard lock(mutex_); return !ops_.empty(); @@ -636,13 +630,11 @@ std::shared_ptr Batcher::CreateRpc( hybrid_time_for_write_, std::move(ops)}; switch (op_group) { case OpGroup::kWrite: - return std::make_shared(&data, single_rpc_timeout_); + return std::make_shared(&data); case OpGroup::kLeaderRead: - return std::make_shared(&data, YBConsistencyLevel::STRONG, single_rpc_timeout_); + return std::make_shared(&data, YBConsistencyLevel::STRONG); case OpGroup::kConsistentPrefixRead: - return std::make_shared(&data, - YBConsistencyLevel::CONSISTENT_PREFIX, - single_rpc_timeout_); + return std::make_shared(&data, YBConsistencyLevel::CONSISTENT_PREFIX); } FATAL_INVALID_ENUM_VALUE(OpGroup, op_group); } diff --git a/src/yb/client/batcher.h b/src/yb/client/batcher.h index 2235dc57e9d5..51f5da375176 100644 --- a/src/yb/client/batcher.h +++ b/src/yb/client/batcher.h @@ -123,8 +123,6 @@ class Batcher : public RefCountedThreadSafe { // may time out before even sending an op). TODO: implement that void SetTimeout(MonoDelta timeout); - void SetSingleRpcTimeout(MonoDelta timeout); - // Add a new operation to the batch. Requires that the batch has not yet been flushed. // TODO: in other flush modes, this may not be the case -- need to // update this when they're implemented. @@ -299,9 +297,6 @@ class Batcher : public RefCountedThreadSafe { // Set by SetTimeout. MonoDelta timeout_; - // Timeout for the rpc. - MonoDelta single_rpc_timeout_; - // After flushing, the absolute deadline for all in-flight ops. CoarseTimePoint deadline_; diff --git a/src/yb/client/meta_cache.cc b/src/yb/client/meta_cache.cc index 23ae1fd65103..40c035027951 100644 --- a/src/yb/client/meta_cache.cc +++ b/src/yb/client/meta_cache.cc @@ -600,7 +600,7 @@ void LookupRpc::SendRpc() { Finished(STATUS(TimedOut, "timed out after deadline expired")); return; } - mutable_retrier()->PrepareController(MonoDelta()); + mutable_retrier()->PrepareController(); DoSendRpc(); } diff --git a/src/yb/client/ql-tablet-test.cc b/src/yb/client/ql-tablet-test.cc index 8483cec0ec17..79d54d6b5db3 100644 --- a/src/yb/client/ql-tablet-test.cc +++ b/src/yb/client/ql-tablet-test.cc @@ -60,7 +60,6 @@ DECLARE_int32(leader_lease_duration_ms); DECLARE_int64(db_write_buffer_size); DECLARE_string(time_source); DECLARE_int32(TEST_delay_execute_async_ms); -DECLARE_int64(retryable_rpc_single_call_timeout_ms); DECLARE_int32(retryable_request_timeout_secs); DECLARE_bool(enable_lease_revocation); DECLARE_bool(rocksdb_disable_compactions); @@ -757,7 +756,6 @@ TEST_F(QLTabletTest, LeaderChange) { req->mutable_if_expr()->mutable_condition(), kValueColumn, QL_OP_EQUAL, kValue1); req->mutable_column_refs()->add_ids(table.ColumnId(kValueColumn)); ASSERT_OK(session->Apply(write_op)); - FLAGS_retryable_rpc_single_call_timeout_ms = 60000; SetAtomicFlag(30000, &FLAGS_TEST_delay_execute_async_ms); auto flush_future = session->FlushFuture(); diff --git a/src/yb/client/ql-transaction-test.cc b/src/yb/client/ql-transaction-test.cc index e02164f4f7b2..e2f57a947a59 100644 --- a/src/yb/client/ql-transaction-test.cc +++ b/src/yb/client/ql-transaction-test.cc @@ -56,7 +56,7 @@ DECLARE_bool(flush_rocksdb_on_shutdown); DECLARE_bool(transaction_disable_proactive_cleanup_in_tests); DECLARE_uint64(aborted_intent_cleanup_ms); DECLARE_int32(remote_bootstrap_max_chunk_size); -DECLARE_int32(TEST_master_inject_latency_on_transactional_tablet_lookups_ms); +DECLARE_bool(TEST_master_fail_transactional_tablet_lookups); DECLARE_int64(transaction_rpc_timeout_ms); DECLARE_bool(rocksdb_disable_compactions); DECLARE_int32(TEST_delay_init_tablet_peer_ms); @@ -113,13 +113,12 @@ TEST_F(QLTransactionTest, Simple) { } TEST_F(QLTransactionTest, LookupTabletFailure) { - FLAGS_TEST_master_inject_latency_on_transactional_tablet_lookups_ms = - TransactionRpcTimeout().ToMilliseconds() + 500; + FLAGS_TEST_master_fail_transactional_tablet_lookups = true; auto txn = CreateTransaction(); auto result = WriteRow(CreateSession(txn), 0 /* key */, 1 /* value */); - ASSERT_TRUE(!result.ok() && result.status().IsTimedOut()) << "Result: " << result; + ASSERT_TRUE(!result.ok() && result.status().IsTimedOut()) << "Result: " << AsString(result); } TEST_F(QLTransactionTest, ReadWithTimeInFuture) { diff --git a/src/yb/client/session.cc b/src/yb/client/session.cc index b4992a9bf4fa..ad5d7ad018b7 100644 --- a/src/yb/client/session.cc +++ b/src/yb/client/session.cc @@ -111,14 +111,6 @@ void YBSession::SetTimeout(MonoDelta timeout) { } } -void YBSession::SetSingleRpcTimeout(MonoDelta timeout) { - CHECK_GE(timeout, MonoDelta::kZero); - single_rpc_timeout_ = timeout; - if (batcher_) { - batcher_->SetSingleRpcTimeout(timeout); - } -} - Status YBSession::Flush() { Synchronizer s; FlushAsync(s.AsStatusFunctor()); @@ -209,9 +201,6 @@ internal::Batcher& YBSession::Batcher() { if (timeout_.Initialized()) { batcher_->SetTimeout(timeout_); } - if (single_rpc_timeout_.Initialized()) { - batcher_->SetSingleRpcTimeout(single_rpc_timeout_); - } batcher_->SetRejectionScoreSource(rejection_score_source_); if (hybrid_time_for_write_.is_valid()) { batcher_->SetHybridTimeForWrite(hybrid_time_for_write_); diff --git a/src/yb/client/session.h b/src/yb/client/session.h index 40e7c5cdafe9..9b8721bc5cfd 100644 --- a/src/yb/client/session.h +++ b/src/yb/client/session.h @@ -117,8 +117,6 @@ class YBSession : public std::enable_shared_from_this { // Set the timeout for writes made in this session. void SetTimeout(MonoDelta timeout); - void SetSingleRpcTimeout(MonoDelta timeout); - MonoDelta timeout() const { return timeout_; } @@ -287,9 +285,6 @@ class YBSession : public std::enable_shared_from_this { // Timeout for the next batch. MonoDelta timeout_; - // Timeout for the rpcs. Initialized only on SetRpcTimeout. - MonoDelta single_rpc_timeout_; - // HybridTime for Write. Used for Index Backfill. HybridTime hybrid_time_for_write_; diff --git a/src/yb/docdb/pgsql_operation.cc b/src/yb/docdb/pgsql_operation.cc index a87e924e373a..c90e3520c015 100644 --- a/src/yb/docdb/pgsql_operation.cc +++ b/src/yb/docdb/pgsql_operation.cc @@ -32,7 +32,6 @@ DECLARE_bool(trace_docdb_calls); DECLARE_bool(ysql_disable_index_backfill); -DECLARE_int64(retryable_rpc_single_call_timeout_ms); DEFINE_double(ysql_scan_timeout_multiplier, 0.5, "YSQL read scan timeout multipler of retryable_rpc_single_call_timeout_ms."); @@ -504,9 +503,6 @@ Result PgsqlReadOperation::ExecuteScalar(const common::YQLStorageIf& ql_ // Set scan start time. bool scan_time_exceeded = false; - const int64 scan_time_limit = - FLAGS_retryable_rpc_single_call_timeout_ms * FLAGS_ysql_scan_timeout_multiplier; - const MonoTime start_time = MonoTime::Now(); // Fetching data. int match_count = 0; @@ -552,8 +548,7 @@ Result PgsqlReadOperation::ExecuteScalar(const common::YQLStorageIf& ql_ // Check every row_count_limit matches whether we've exceeded our scan time. if (match_count % row_count_limit == 0) { - const MonoDelta elapsed_time = MonoTime::Now().GetDeltaSince(start_time); - scan_time_exceeded = elapsed_time.ToMilliseconds() > scan_time_limit; + scan_time_exceeded = CoarseMonoClock::now() >= deadline; } } diff --git a/src/yb/master/master_service.cc b/src/yb/master/master_service.cc index 9b13bfe36c8d..3c3258562656 100644 --- a/src/yb/master/master_service.cc +++ b/src/yb/master/master_service.cc @@ -58,9 +58,8 @@ DEFINE_int32(master_inject_latency_on_tablet_lookups_ms, 0, TAG_FLAG(master_inject_latency_on_tablet_lookups_ms, unsafe); TAG_FLAG(master_inject_latency_on_tablet_lookups_ms, hidden); -DEFINE_test_flag(int32, master_inject_latency_on_transactional_tablet_lookups_ms, 0, - "Number of milliseconds that the master will sleep before responding to " - "requests for transactional tablet locations."); +DEFINE_test_flag(bool, master_fail_transactional_tablet_lookups, false, + "Whether to fail all lookup requests to transactional table."); DEFINE_double(master_slow_get_registration_probability, 0, "Probability of injecting delay in GetMasterRegistration."); @@ -221,7 +220,7 @@ void MasterServiceImpl::GetTabletLocations(const GetTabletLocationsRequestPB* re if (PREDICT_FALSE(FLAGS_master_inject_latency_on_tablet_lookups_ms > 0)) { SleepFor(MonoDelta::FromMilliseconds(FLAGS_master_inject_latency_on_tablet_lookups_ms)); } - if (PREDICT_FALSE(FLAGS_TEST_master_inject_latency_on_transactional_tablet_lookups_ms > 0)) { + if (PREDICT_FALSE(FLAGS_TEST_master_fail_transactional_tablet_lookups)) { std::vector> tables; server_->catalog_manager()->GetAllTables(&tables); const auto& tablet_id = req->tablet_ids(0); @@ -230,10 +229,14 @@ void MasterServiceImpl::GetTabletLocations(const GetTabletLocationsRequestPB* re table->GetAllTablets(&tablets); for (const auto& tablet : tablets) { if (tablet->tablet_id() == tablet_id) { - auto lock = table->LockForRead(); - if (table->metadata().state().table_type() == TableType::TRANSACTION_STATUS_TABLE_TYPE) { - SleepFor(MonoDelta::FromMilliseconds( - FLAGS_TEST_master_inject_latency_on_transactional_tablet_lookups_ms)); + TableType table_type; + { + auto lock = table->LockForRead(); + table_type = table->metadata().state().table_type(); + } + if (table_type == TableType::TRANSACTION_STATUS_TABLE_TYPE) { + rpc.RespondFailure(STATUS(InvalidCommand, "TEST: Artificial failure")); + return; } break; } diff --git a/src/yb/rpc/rpc.cc b/src/yb/rpc/rpc.cc index 08539ed067b6..dc0a51e179ea 100644 --- a/src/yb/rpc/rpc.cc +++ b/src/yb/rpc/rpc.cc @@ -53,8 +53,6 @@ DEFINE_int64(rpcs_shutdown_timeout_ms, 15000 * yb::kTimeMultiplier, "Timeout for a batch of multiple RPCs invoked in parallel to shutdown."); DEFINE_int64(rpcs_shutdown_extra_delay_ms, 5000 * yb::kTimeMultiplier, "Extra allowed time for a single RPC command to complete after its deadline."); -DEFINE_int64(retryable_rpc_single_call_timeout_ms, 2500 * yb::kTimeMultiplier, - "Timeout of single RPC call in retryable RPC command."); DEFINE_int32(min_backoff_ms_exponent, 7, "Min amount of backoff delay if the server responds with TOO BUSY (default: 128ms). " "Set this to some amount, during which the server might have recovered."); @@ -268,12 +266,8 @@ std::string RpcRetrier::ToString() const { deadline_); } -RpcController* RpcRetrier::PrepareController(MonoDelta single_call_timeout) { - if (!single_call_timeout) { - single_call_timeout = MonoDelta::FromMilliseconds(FLAGS_retryable_rpc_single_call_timeout_ms); - } - controller_.set_timeout(std::min( - deadline_ - CoarseMonoClock::now(), single_call_timeout)); +RpcController* RpcRetrier::PrepareController() { + controller_.set_timeout(deadline_ - CoarseMonoClock::now()); return &controller_; } diff --git a/src/yb/rpc/rpc.h b/src/yb/rpc/rpc.h index 621c55b70562..954df975d9d3 100644 --- a/src/yb/rpc/rpc.h +++ b/src/yb/rpc/rpc.h @@ -124,7 +124,7 @@ class RpcRetrier { // Sets up deadline and returns controller. // Do not forget that setting deadline in RpcController is NOT thread safe. - RpcController* PrepareController(MonoDelta single_call_timeout); + RpcController* PrepareController(); CoarseTimePoint deadline() const { return deadline_; } @@ -208,8 +208,8 @@ class Rpc : public RpcCommand { protected: const RpcRetrier& retrier() const { return retrier_; } RpcRetrier* mutable_retrier() { return &retrier_; } - RpcController* PrepareController(MonoDelta single_call_timeout = MonoDelta()) { - return retrier_.PrepareController(single_call_timeout); + RpcController* PrepareController() { + return retrier_.PrepareController(); } private: diff --git a/src/yb/yql/pggate/pg_session.cc b/src/yb/yql/pggate/pg_session.cc index c632ce56f584..f3469205f409 100644 --- a/src/yb/yql/pggate/pg_session.cc +++ b/src/yb/yql/pggate/pg_session.cc @@ -409,7 +409,6 @@ PgSession::PgSession( // Sets the timeout for each rpc as well as the whole operation to // 'FLAGS_pg_yb_session_timeout_ms'. session_->SetTimeout(MonoDelta::FromMilliseconds(FLAGS_pg_yb_session_timeout_ms)); - session_->SetSingleRpcTimeout(MonoDelta::FromMilliseconds(FLAGS_pg_yb_session_timeout_ms)); session_->SetForceConsistentRead(client::ForceConsistentRead::kTrue); } @@ -944,7 +943,6 @@ Status PgSession::TabletServerCount(int *tserver_count, bool primary_only, bool void PgSession::SetTimeout(const int timeout_ms) { session_->SetTimeout(MonoDelta::FromMilliseconds(timeout_ms)); - session_->SetSingleRpcTimeout(MonoDelta::FromMilliseconds(timeout_ms)); } Result PgSession::WaitUntilIndexPermissionsAtLeast( diff --git a/src/yb/yql/pgwrapper/pg_libpq-test.cc b/src/yb/yql/pgwrapper/pg_libpq-test.cc index ece8255fafa5..61a211970102 100644 --- a/src/yb/yql/pgwrapper/pg_libpq-test.cc +++ b/src/yb/yql/pgwrapper/pg_libpq-test.cc @@ -26,7 +26,6 @@ using namespace std::literals; DECLARE_int64(external_mini_cluster_max_log_bytes); -DECLARE_int64(retryable_rpc_single_call_timeout_ms); METRIC_DECLARE_entity(tablet); METRIC_DECLARE_counter(transaction_not_found); diff --git a/src/yb/yql/pgwrapper/pg_mini-test.cc b/src/yb/yql/pgwrapper/pg_mini-test.cc index b3e8e133d47f..2b159cadcd9d 100644 --- a/src/yb/yql/pgwrapper/pg_mini-test.cc +++ b/src/yb/yql/pgwrapper/pg_mini-test.cc @@ -48,7 +48,6 @@ DECLARE_int32(history_cutoff_propagation_interval_ms); DECLARE_int32(pggate_rpc_timeout_secs); DECLARE_int32(timestamp_history_retention_interval_sec); DECLARE_int32(ysql_num_shards_per_tserver); -DECLARE_int64(retryable_rpc_single_call_timeout_ms); DECLARE_uint64(max_clock_skew_usec); DECLARE_int64(db_write_buffer_size); DECLARE_bool(ysql_enable_manual_sys_table_txn_ctl); @@ -72,7 +71,6 @@ class PgMiniTest : public YBMiniClusterTestBase { FLAGS_enable_ysql = true; FLAGS_hide_pg_catalog_table_creation_logs = true; FLAGS_master_auto_run_initdb = true; - FLAGS_retryable_rpc_single_call_timeout_ms = 30000; FLAGS_pggate_rpc_timeout_secs = 120; FLAGS_ysql_num_shards_per_tserver = 1; diff --git a/src/yb/yql/pgwrapper/pg_on_conflict-test.cc b/src/yb/yql/pgwrapper/pg_on_conflict-test.cc index 2c3eae957814..8d843b22af6d 100644 --- a/src/yb/yql/pgwrapper/pg_on_conflict-test.cc +++ b/src/yb/yql/pgwrapper/pg_on_conflict-test.cc @@ -22,7 +22,6 @@ using namespace std::literals; DECLARE_int64(external_mini_cluster_max_log_bytes); -DECLARE_int64(retryable_rpc_single_call_timeout_ms); METRIC_DECLARE_entity(tablet); METRIC_DECLARE_counter(transaction_not_found); diff --git a/src/yb/yql/pgwrapper/pg_wrapper_test_base.cc b/src/yb/yql/pgwrapper/pg_wrapper_test_base.cc index eab0a482bdd8..2aa1b270e79a 100644 --- a/src/yb/yql/pgwrapper/pg_wrapper_test_base.cc +++ b/src/yb/yql/pgwrapper/pg_wrapper_test_base.cc @@ -33,15 +33,6 @@ void PgWrapperTestBase::SetUp() { ExternalMiniClusterOptions opts; opts.enable_ysql = true; - // TODO Increase the rpc timeout (from 2500) to not time out for long master queries (i.e. for - // Postgres system tables). Should be removed once the long lock issue is fixed. - const int kSingleCallTimeoutMs = NonTsanVsTsan(10000, 30000); - const string rpc_flag_str = - "--retryable_rpc_single_call_timeout_ms=" + std::to_string(kSingleCallTimeoutMs); - opts.extra_master_flags.emplace_back(rpc_flag_str); - - opts.extra_tserver_flags.emplace_back(rpc_flag_str); - // With ysql_num_shards_per_tserver=1 and 3 tservers we'll be creating 3 tablets per table, which // is enough for most tests. opts.extra_tserver_flags.emplace_back("--ysql_num_shards_per_tserver=1");