Skip to content

Commit

Permalink
Merge "Remove default timeouts" from Botond
Browse files Browse the repository at this point in the history
"
Timeouts defaulted to `db::no_timeout` are dangerous. They allow any
modifications to the code to drop timeouts and introduce a source of
unbounded request queue to the system.

This series removes the last such default timeouts from the code. No
problems were found, only test code had to be updated.

tests: unit(dev)
"

* 'no-default-timeouts/v1' of https://github.com/denesb/scylla:
  database: database::query*(), database::apply*(): remove default timeouts
  database: table::query(): remove default timeout
  mutation_query: data_query(): remove default timeout
  mutation_query: mutation_query(): remove default timeout
  multishard_mutation_query: query_mutations_on_all_shards(): remove default timeout
  reader_concurrency_semaphore: wait_admission(): remove default timeout
  utils/logallog: run_when_memory_available(): remove default timeout
  • Loading branch information
avikivity committed Mar 1, 2020
2 parents 2137017 + 1073094 commit 157fe4b
Show file tree
Hide file tree
Showing 16 changed files with 77 additions and 77 deletions.
4 changes: 2 additions & 2 deletions database.cc
Original file line number Diff line number Diff line change
Expand Up @@ -1208,10 +1208,10 @@ database::query_mutations(schema_ptr s, const query::read_command& cmd, const dh
cmd.row_limit,
cmd.partition_limit,
cmd.timestamp,
timeout,
cf.get_config().max_memory_for_unlimited_query,
std::move(accounter),
std::move(trace_state),
timeout,
std::move(cache_ctx)).then_wrapped([this, s = _stats, hit_rate = cf.get_global_cache_hit_rate(), op = cf.read_in_progress()] (auto f) {
if (f.failed()) {
++s->total_reads_failed;
Expand Down Expand Up @@ -1606,7 +1606,7 @@ future<> database::apply_streaming_mutation(schema_ptr s, utils::UUID plan_id, c
auto uuid = m.column_family_id();
auto& cf = find_column_family(uuid);
cf.apply_streaming_mutation(s, plan_id, std::move(m), fragmented);
});
}, db::no_timeout);
});
}

Expand Down
10 changes: 5 additions & 5 deletions database.hh
Original file line number Diff line number Diff line change
Expand Up @@ -787,7 +787,7 @@ public:
tracing::trace_state_ptr trace_state,
query::result_memory_limiter& memory_limiter,
uint64_t max_result_size,
db::timeout_clock::time_point timeout = db::no_timeout,
db::timeout_clock::time_point timeout,
query::querier_cache_context cache_ctx = { });

void start();
Expand Down Expand Up @@ -1474,14 +1474,14 @@ public:
unsigned shard_of(const frozen_mutation& m);
future<lw_shared_ptr<query::result>, cache_temperature> query(schema_ptr, const query::read_command& cmd, query::result_options opts,
const dht::partition_range_vector& ranges, tracing::trace_state_ptr trace_state,
uint64_t max_result_size, db::timeout_clock::time_point timeout = db::no_timeout);
uint64_t max_result_size, db::timeout_clock::time_point timeout);
future<reconcilable_result, cache_temperature> query_mutations(schema_ptr, const query::read_command& cmd, const dht::partition_range& range,
query::result_memory_accounter&& accounter, tracing::trace_state_ptr trace_state,
db::timeout_clock::time_point timeout = db::no_timeout);
db::timeout_clock::time_point timeout);
// Apply the mutation atomically.
// Throws timed_out_error when timeout is reached.
future<> apply(schema_ptr, const frozen_mutation&, db::commitlog::force_sync sync, db::timeout_clock::time_point timeout = db::no_timeout);
future<> apply_hint(schema_ptr, const frozen_mutation&, db::timeout_clock::time_point timeout = db::no_timeout);
future<> apply(schema_ptr, const frozen_mutation&, db::commitlog::force_sync sync, db::timeout_clock::time_point timeout);
future<> apply_hint(schema_ptr, const frozen_mutation&, db::timeout_clock::time_point timeout);
future<> apply_streaming_mutation(schema_ptr, utils::UUID plan_id, const frozen_mutation&, bool fragmented);
future<mutation> apply_counter_update(schema_ptr, const frozen_mutation& m, db::timeout_clock::time_point timeout, tracing::trace_state_ptr trace_state);
keyspace::config make_keyspace_config(const keyspace_metadata& ksm);
Expand Down
2 changes: 1 addition & 1 deletion multishard_mutation_query.hh
Original file line number Diff line number Diff line change
Expand Up @@ -68,4 +68,4 @@ future<std::tuple<foreign_ptr<lw_shared_ptr<reconcilable_result>>, cache_tempera
const dht::partition_range_vector& ranges,
tracing::trace_state_ptr trace_state,
uint64_t max_size,
db::timeout_clock::time_point timeout = db::no_timeout);
db::timeout_clock::time_point timeout);
8 changes: 4 additions & 4 deletions mutation_partition.cc
Original file line number Diff line number Diff line change
Expand Up @@ -2168,9 +2168,9 @@ future<> data_query(
uint32_t partition_limit,
gc_clock::time_point query_time,
query::result::builder& builder,
db::timeout_clock::time_point timeout,
uint64_t max_memory_reverse_query,
tracing::trace_state_ptr trace_ptr,
db::timeout_clock::time_point timeout,
query::querier_cache_context cache_ctx)
{
if (row_limit == 0 || slice.partition_row_limit() == 0 || partition_limit == 0) {
Expand Down Expand Up @@ -2263,10 +2263,10 @@ static do_mutation_query(schema_ptr s,
uint32_t row_limit,
uint32_t partition_limit,
gc_clock::time_point query_time,
db::timeout_clock::time_point timeout,
uint64_t max_memory_reverse_query,
query::result_memory_accounter&& accounter,
tracing::trace_state_ptr trace_ptr,
db::timeout_clock::time_point timeout,
query::querier_cache_context cache_ctx)
{
if (row_limit == 0 || slice.partition_row_limit() == 0 || partition_limit == 0) {
Expand Down Expand Up @@ -2303,14 +2303,14 @@ mutation_query(schema_ptr s,
uint32_t row_limit,
uint32_t partition_limit,
gc_clock::time_point query_time,
db::timeout_clock::time_point timeout,
uint64_t max_memory_reverse_query,
query::result_memory_accounter&& accounter,
tracing::trace_state_ptr trace_ptr,
db::timeout_clock::time_point timeout,
query::querier_cache_context cache_ctx)
{
return do_mutation_query(std::move(s), std::move(source), seastar::cref(range), seastar::cref(slice),
row_limit, partition_limit, query_time, max_memory_reverse_query, std::move(accounter), std::move(trace_ptr), timeout, std::move(cache_ctx));
row_limit, partition_limit, query_time, timeout, max_memory_reverse_query, std::move(accounter), std::move(trace_ptr), std::move(cache_ctx));
}

deletable_row::deletable_row(clustering_row&& cr)
Expand Down
6 changes: 3 additions & 3 deletions mutation_query.hh
Original file line number Diff line number Diff line change
Expand Up @@ -161,10 +161,10 @@ future<reconcilable_result> mutation_query(
uint32_t row_limit,
uint32_t partition_limit,
gc_clock::time_point query_time,
db::timeout_clock::time_point timeout,
uint64_t max_memory_reverse_query,
query::result_memory_accounter&& accounter = { },
tracing::trace_state_ptr trace_ptr = nullptr,
db::timeout_clock::time_point timeout = db::no_timeout,
query::querier_cache_context cache_ctx = { });

future<> data_query(
Expand All @@ -176,9 +176,9 @@ future<> data_query(
uint32_t partition_limit,
gc_clock::time_point query_time,
query::result::builder& builder,
db::timeout_clock::time_point timeout,
uint64_t max_memory_reverse_query,
tracing::trace_state_ptr trace_ptr = nullptr,
db::timeout_clock::time_point timeout = db::no_timeout,
query::querier_cache_context cache_ctx = { });


Expand All @@ -191,10 +191,10 @@ class mutation_query_stage {
uint32_t,
uint32_t,
gc_clock::time_point,
db::timeout_clock::time_point,
uint64_t,
query::result_memory_accounter&&,
tracing::trace_state_ptr,
db::timeout_clock::time_point,
query::querier_cache_context> _execution_stage;
public:
explicit mutation_query_stage();
Expand Down
2 changes: 1 addition & 1 deletion reader_concurrency_semaphore.hh
Original file line number Diff line number Diff line change
Expand Up @@ -199,7 +199,7 @@ public:
return _inactive_read_stats;
}

future<reader_permit> wait_admission(size_t memory, db::timeout_clock::time_point timeout = db::no_timeout);
future<reader_permit> wait_admission(size_t memory, db::timeout_clock::time_point timeout);

/// Consume the specific amount of resources without waiting.
reader_permit consume_resources(resources r);
Expand Down
2 changes: 1 addition & 1 deletion table.cc
Original file line number Diff line number Diff line change
Expand Up @@ -2397,7 +2397,7 @@ table::query(schema_ptr s,
return do_until(std::bind(&query_state::done, &qs), [this, &qs, trace_state = std::move(trace_state), timeout, cache_ctx = std::move(cache_ctx)] {
auto&& range = *qs.current_partition_range++;
return data_query(qs.schema, as_mutation_source(), range, qs.cmd.slice, qs.remaining_rows(),
qs.remaining_partitions(), qs.cmd.timestamp, qs.builder, _config.max_memory_for_unlimited_query, trace_state, timeout, cache_ctx);
qs.remaining_partitions(), qs.cmd.timestamp, qs.builder, timeout, _config.max_memory_for_unlimited_query, trace_state, cache_ctx);
}).then([qs_ptr = std::move(qs_ptr), &qs] {
return make_ready_future<lw_shared_ptr<query::result>>(
make_lw_shared<query::result>(qs.builder.build()));
Expand Down
16 changes: 8 additions & 8 deletions test/boost/database_test.cc
Original file line number Diff line number Diff line change
Expand Up @@ -56,13 +56,13 @@ SEASTAR_TEST_CASE(test_safety_after_truncate) {
mutation m(s, pkey);
m.set_clustered_cell(clustering_key_prefix::make_empty(), "v", int32_t(42), {});
pranges.emplace_back(dht::partition_range::make_singular(dht::decorate_key(*s, std::move(pkey))));
db.apply(s, freeze(m), db::commitlog::force_sync::no).get();
db.apply(s, freeze(m), db::commitlog::force_sync::no, db::no_timeout).get();
}

auto assert_query_result = [&] (size_t expected_size) {
auto max_size = std::numeric_limits<size_t>::max();
auto cmd = query::read_command(s->id(), s->version(), partition_slice_builder(*s).build(), 1000);
auto result = db.query(s, cmd, query::result_options::only_result(), pranges, nullptr, max_size).get0();
auto result = db.query(s, cmd, query::result_options::only_result(), pranges, nullptr, max_size, db::no_timeout).get0();
assert_that(query::result_set::from_raw_result(s, cmd.slice, *result)).has_size(expected_size);
};
assert_query_result(1000);
Expand Down Expand Up @@ -92,34 +92,34 @@ SEASTAR_TEST_CASE(test_querying_with_limits) {
auto pkey = partition_key::from_single_value(*s, to_bytes(format("key{:d}", i)));
mutation m(s, pkey);
m.partition().apply(tombstone(api::timestamp_type(1), gc_clock::now()));
db.apply(s, freeze(m), db::commitlog::force_sync::no).get();
db.apply(s, freeze(m), db::commitlog::force_sync::no, db::no_timeout).get();
}
for (uint32_t i = 3; i <= 8; ++i) {
auto pkey = partition_key::from_single_value(*s, to_bytes(format("key{:d}", i)));
mutation m(s, pkey);
m.set_clustered_cell(clustering_key_prefix::make_empty(), "v", int32_t(42), 1);
db.apply(s, freeze(m), db::commitlog::force_sync::no).get();
db.apply(s, freeze(m), db::commitlog::force_sync::no, db::no_timeout).get();
pranges.emplace_back(dht::partition_range::make_singular(dht::decorate_key(*s, std::move(pkey))));
}

auto max_size = std::numeric_limits<size_t>::max();
{
auto cmd = query::read_command(s->id(), s->version(), partition_slice_builder(*s).build(), 3);
auto result = db.query(s, cmd, query::result_options::only_result(), pranges, nullptr, max_size).get0();
auto result = db.query(s, cmd, query::result_options::only_result(), pranges, nullptr, max_size, db::no_timeout).get0();
assert_that(query::result_set::from_raw_result(s, cmd.slice, *result)).has_size(3);
}

{
auto cmd = query::read_command(s->id(), s->version(), partition_slice_builder(*s).build(),
query::max_rows, gc_clock::now(), std::nullopt, 5);
auto result = db.query(s, cmd, query::result_options::only_result(), pranges, nullptr, max_size).get0();
auto result = db.query(s, cmd, query::result_options::only_result(), pranges, nullptr, max_size, db::no_timeout).get0();
assert_that(query::result_set::from_raw_result(s, cmd.slice, *result)).has_size(5);
}

{
auto cmd = query::read_command(s->id(), s->version(), partition_slice_builder(*s).build(),
query::max_rows, gc_clock::now(), std::nullopt, 3);
auto result = db.query(s, cmd, query::result_options::only_result(), pranges, nullptr, max_size).get0();
auto result = db.query(s, cmd, query::result_options::only_result(), pranges, nullptr, max_size, db::no_timeout).get0();
assert_that(query::result_set::from_raw_result(s, cmd.slice, *result)).has_size(3);
}
});
Expand All @@ -138,7 +138,7 @@ SEASTAR_THREAD_TEST_CASE(test_database_with_data_in_sstables_is_a_mutation_sourc
service::get_local_migration_manager().announce_new_column_family(s, true).get();
column_family& cf = e.local_db().find_column_family(s);
for (auto&& m : partitions) {
e.local_db().apply(cf.schema(), freeze(m), db::commitlog::force_sync::no).get();
e.local_db().apply(cf.schema(), freeze(m), db::commitlog::force_sync::no, db::no_timeout).get();
}
cf.flush().get();
cf.get_row_cache().invalidate([] {}).get();
Expand Down
34 changes: 17 additions & 17 deletions test/boost/logalloc_test.cc
Original file line number Diff line number Diff line change
Expand Up @@ -649,11 +649,11 @@ SEASTAR_TEST_CASE(test_region_groups_basic_throttling) {
//
// The allocation of the first element won't change the memory usage inside
// the group and we'll be okay to do that a second time.
auto fut = simple.run_when_memory_available([&simple_region] { simple_region->alloc_small(); });
auto fut = simple.run_when_memory_available([&simple_region] { simple_region->alloc_small(); }, db::no_timeout);
BOOST_REQUIRE_EQUAL(fut.available(), true);
BOOST_REQUIRE_EQUAL(simple.memory_used(), logalloc::segment_size);

fut = simple.run_when_memory_available([&simple_region] { simple_region->alloc_small(); });
fut = simple.run_when_memory_available([&simple_region] { simple_region->alloc_small(); }, db::no_timeout);
BOOST_REQUIRE_EQUAL(fut.available(), true);
BOOST_REQUIRE_EQUAL(simple.memory_used(), logalloc::segment_size);

Expand All @@ -663,7 +663,7 @@ SEASTAR_TEST_CASE(test_region_groups_basic_throttling) {

// We should not be permitted to go forward with a new allocation now...
BOOST_TEST_MESSAGE(format("now = {}", lowres_clock::now().time_since_epoch().count()));
fut = simple.run_when_memory_available([&simple_region] { simple_region->alloc_small(); });
fut = simple.run_when_memory_available([&simple_region] { simple_region->alloc_small(); }, db::no_timeout);
BOOST_REQUIRE_EQUAL(fut.available(), false);
BOOST_REQUIRE_GT(simple.memory_used(), logalloc::segment_size);

Expand Down Expand Up @@ -705,7 +705,7 @@ SEASTAR_TEST_CASE(test_region_groups_linear_hierarchy_throttling_child_alloc) {
child_region->alloc();
BOOST_REQUIRE_GE(parent.memory_used(), logalloc::segment_size);

auto fut = parent.run_when_memory_available([&parent_region] { parent_region->alloc_small(); });
auto fut = parent.run_when_memory_available([&parent_region] { parent_region->alloc_small(); }, db::no_timeout);
BOOST_REQUIRE_EQUAL(fut.available(), true);
BOOST_REQUIRE_GE(parent.memory_used(), 2 * logalloc::segment_size);

Expand All @@ -714,7 +714,7 @@ SEASTAR_TEST_CASE(test_region_groups_linear_hierarchy_throttling_child_alloc) {
child_region->alloc();
BOOST_REQUIRE_GE(child.memory_used(), 2 * logalloc::segment_size);

fut = parent.run_when_memory_available([&parent_region] { parent_region->alloc_small(); });
fut = parent.run_when_memory_available([&parent_region] { parent_region->alloc_small(); }, db::no_timeout);
BOOST_REQUIRE_EQUAL(fut.available(), false);
BOOST_REQUIRE_GE(parent.memory_used(), 2 * logalloc::segment_size);

Expand All @@ -735,7 +735,7 @@ SEASTAR_TEST_CASE(test_region_groups_linear_hierarchy_throttling_parent_alloc) {
parent_region->alloc();
BOOST_REQUIRE_GE(parent.memory_used(), logalloc::segment_size);

auto fut = child.run_when_memory_available([] {});
auto fut = child.run_when_memory_available([] {}, db::no_timeout);
BOOST_REQUIRE_EQUAL(fut.available(), false);

parent_region.reset();
Expand All @@ -762,7 +762,7 @@ SEASTAR_TEST_CASE(test_region_groups_fifo_order) {
for (auto index = 0; index < 100; ++index) {
auto fut = rg.run_when_memory_available([exec_cnt, index] {
BOOST_REQUIRE_EQUAL(index, (*exec_cnt)++);
});
}, db::no_timeout);
BOOST_REQUIRE_EQUAL(fut.available(), false);
executions.push_back(std::move(fut));
}
Expand Down Expand Up @@ -794,7 +794,7 @@ SEASTAR_TEST_CASE(test_region_groups_linear_hierarchy_throttling_moving_restrict
});
BOOST_REQUIRE_GE(inner.memory_used(), logalloc::segment_size);

auto fut = child.run_when_memory_available([] {});
auto fut = child.run_when_memory_available([] {}, db::no_timeout);
BOOST_REQUIRE_EQUAL(fut.available(), false);

// Now fill the root...
Expand Down Expand Up @@ -846,7 +846,7 @@ SEASTAR_TEST_CASE(test_region_groups_tree_hierarchy_throttling_leaf_alloc) {
future<> try_alloc(size_t size) {
return _rg.run_when_memory_available([this, size] {
alloc(size);
});
}, db::no_timeout);
}
void reset() {
_region.reset(new test_region(_rg));
Expand Down Expand Up @@ -981,7 +981,7 @@ SEASTAR_TEST_CASE(test_region_groups_basic_throttling_simple_active_reclaim) {
(void)simple.unleash(make_ready_future<>());

// Can't run this function until we have reclaimed something
auto fut = simple.rg().run_when_memory_available([] {});
auto fut = simple.rg().run_when_memory_available([] {}, db::no_timeout);

// Initially not available
BOOST_REQUIRE_EQUAL(fut.available(), false);
Expand Down Expand Up @@ -1009,7 +1009,7 @@ SEASTAR_TEST_CASE(test_region_groups_basic_throttling_active_reclaim_worst_offen
// Can't run this function until we have reclaimed
auto fut = simple.rg().run_when_memory_available([&simple] {
BOOST_REQUIRE_EQUAL(simple.reclaim_sizes().size(), 3);
});
}, db::no_timeout);

// Initially not available
BOOST_REQUIRE_EQUAL(fut.available(), false);
Expand Down Expand Up @@ -1043,7 +1043,7 @@ SEASTAR_TEST_CASE(test_region_groups_basic_throttling_active_reclaim_leaf_offend
// that the leaves are forced correctly.
auto fut = root.rg().run_when_memory_available([&root] {
BOOST_REQUIRE_EQUAL(root.reclaim_sizes().size(), 3);
});
}, db::no_timeout);

// Initially not available
BOOST_REQUIRE_EQUAL(fut.available(), false);
Expand Down Expand Up @@ -1072,7 +1072,7 @@ SEASTAR_TEST_CASE(test_region_groups_basic_throttling_active_reclaim_ancestor_bl
// that the root reclaims
auto fut = leaf.rg().run_when_memory_available([&root] {
BOOST_REQUIRE_EQUAL(root.reclaim_sizes().size(), 1);
});
}, db::no_timeout);

// Initially not available
BOOST_REQUIRE_EQUAL(fut.available(), false);
Expand All @@ -1098,7 +1098,7 @@ SEASTAR_TEST_CASE(test_region_groups_basic_throttling_active_reclaim_big_region_

auto fut = root.rg().run_when_memory_available([&root] {
BOOST_REQUIRE_EQUAL(root.reclaim_sizes().size(), 3);
});
}, db::no_timeout);

// Initially not available
BOOST_REQUIRE_EQUAL(fut.available(), false);
Expand Down Expand Up @@ -1127,11 +1127,11 @@ SEASTAR_TEST_CASE(test_region_groups_basic_throttling_active_reclaim_no_double_r

auto fut_root = root.rg().run_when_memory_available([&root] {
BOOST_REQUIRE_EQUAL(root.reclaim_sizes().size(), 1);
});
}, db::no_timeout);

auto fut_leaf = leaf.rg().run_when_memory_available([&root] {
BOOST_REQUIRE_EQUAL(root.reclaim_sizes().size(), 1);
});
}, db::no_timeout);

// Initially not available
BOOST_REQUIRE_EQUAL(fut_root.available(), false);
Expand Down Expand Up @@ -1185,7 +1185,7 @@ SEASTAR_TEST_CASE(test_no_crash_when_a_lot_of_requests_released_which_change_reg
// Trigger group size change (Refs issue #2021)
gr.update(-10);
gr.update(+10);
});
}, db::no_timeout);
BOOST_REQUIRE(!f.available());
}

Expand Down
Loading

0 comments on commit 157fe4b

Please sign in to comment.