[PATCH v6 000/109] Implement partial cache

72 views
Skip to first unread message

Tomasz Grabiec

<tgrabiec@scylladb.com>
unread,
Jun 12, 2017, 3:01:53 PM6/12/17
to scylladb-dev@googlegroups.com
This series enables cache to keep partial partitions.
Reads no longer have to read whole partition from sstables
in order to cache the result.

The 10MB threshold for partition size in cache is lifted.

Known issues:

- There is no partial eviction yet, whole partitions are still evicted,
and partition snapshots held by active reads are not evictable at all
- Information about range continuity is not recorded if that
would require inserting a dummy entry, or if previous entry
doesn't belong to the latest snapshot
- Cache update after memtable flush happening concurrently with reads
may inhibit that reads' ability to populate cache (new issue)
- Cache update from flushed memtables has partition granularity,
so may cause latency problems with large partition
- Schema is still tracked per-partition, so after schema changes
reads may induce high latency due to whole partition needing
to be converted atomically
- Range tombstones are repeated in the stream for every range between
cache entries they cover (new issue)

How was this tested:

- test.py --mode release (commitlog_test has one failure, but so does origin/master)
- row_cache_stress_test
- perf_fast_forward, passes except for the test case checking range continuity population
which would require inserting a dummy entry (mentioned above)
- perf_simple_query (-c1 -m1G --duration 32):
before: 90k [ops/s] stdev: 4k [ops/s]
after: 94k [ops/s] stdev: 2k [ops/s]

Also in:

dev tgrabiec/introduce-partial-cache-v6

Based on haaawk/partial_cache/v5 from seastar-dev.git with the following changes:

General:

- fixed undefined behavior in position_in_partition_view::for_key()
- replaced position_in_partition::is_last() with is_after_all_clustered_rows()
- fixed range_tombstone_stream::get_next() to use position() rather than key()

In row_cache:

- fixed write atomicity not being respected by cache readers (added snapshot isolation)
- avoided creation/fast forwarding of sstable readers for every partition which is present in cache,
we just take a snapshot and involve underlying reader only if we hit a discontinuous range
- fixed alloc-dealloc mismatch on _last_updated_partition resulting in a crash
- optimized _last_updated_partition update to happen only on preemption
- fixed _last_updated_partition being left stale if cache update fails
- adjusted row_cache::populate() for partial cache
- put shared reader state into read_context, which replaced the need for templating cache_streamed_mutation,
reader_with_range, and cache::underlying
- dropped unnecessary handling of schema changes in single partition reader
- added on-the-fly conversion of schema of returned streamed_mutations so
that it can be always returned without deferring and thus we can avoid
making changes to some existing code paths in cache; it's also what we
should be eventually doing.
- dropped "row_cache: remove unused cache_entry constructor" - it actually is used later in the series
- extracted unrelated test changes out of the patch which introduces cache_streamed_mutation
- added row_cache_stress_test which verifies read and write consistency
- added a test for scan over partially populated partitions
- left addition of new counters out of the series

In mutation_partition:

- merged patches which add dymmy entries and continuity flag into one patch which
introduces the concept of range continuity in mutation_partition
- added more description and documentation
- fixed _static_row_cached being lost by mutation_partition copy constructors
- fixed conversion to streamed_mutation to ignore dummy entries
- fixed mutation_partition serializer to drop dummy entries
- documented semantics of continuity on mutation_partition level
- dropped assumptions that dummy entries can be only at the last position
- changed equality to ignore continuity completely, rather than
partially (it was not ignoring dummy entries, but ignoring
continuity flag)
- added printout of continuity information in mutation_partition
- fixed handling of empty entries in apply_reversibly() with regards
to continuity; we no longer can remove empty entries before
merging, since that may affect continuity of the right-hand
mutation. Added _erased flag.
- fixed mutation_partition::clustered_row() with dummy==true to not ignore the key
- fixed partition_builder to not ignore continuity
- renamed dummy_tag_t to dummy_tag. _t suffix is reserved.
- standardized all APIs on is_dummy and is_continuous bool_class:es
- replaced add_dummy_entry() with ensure_last_dummy() with safer semantics
- dropped unused remove_dummy_entry()
- simplified and inlined cache_entry::add_dummy_entry()
- fixed mutation_partition(incomplete_tag) constructor to mark all row ranges as discontinuous
- improved random mutation generator to generate random continuity
- added tests for continuity merging

In partition_version:

- split "Prepare MVCC for partial cache" into several patches
- dropped "MVCC: extract rows_entry_compare for use in other places" in favor of reusing rows_entry::compare
- extracted partition_version detaching into partition_entry::with_detached_versions()
- dropped unnecessary rows_iterator::_version field
- dropped unnecessary allocation of rows_entry and key copies in rows_iterator
- dropped row_pointer
- replaced apply_reversibly() with weaker and faster apply()
- added handling of dummy entries at any position

In cache_streamed_mutation:

- fixed maybe_add_to_cache_and_update_continuity() leaking entries if
the key already exists in the snapshot
- fixed a problem where population race could result in a read
missing some rows, because cache_streamed_mutation was advancing
the cursor, then deferring, and then checking continuity. We
should check continuity atomically with advancing.
- fixed rows_handle.maybe_refresh() being accessed outside of update
section in read_from_underlying() (undefined behavior)
- fixed a problem in start_reading_from_underlying() where we would
use incorrect start if lower_bound ended with a range tombstone
starting before a key.
- range tombstone trimming in add_to_buffer() could create a
tombstone which has too low start bound if last_rt.end was a
prefix and had inclusive end. invert_kind(end_kind) should be used
instead of unconditional inc_start.
- range tombstone trimming incorrectly assumed it is fine to trim
the tombstone from underlying to the previous fragment's end and
emit such tombstone. That would mean the stream can't emit any
fragments which start before previous tombstone's end. Solve with
range_tombstone_stream.
- split add_to_buffer() into overloads for clustering_row, and
range_tombstone. Better than wrapping into mutation_fragment
before the call and having add_to_buffer() rediscover the
information.
- changed maybe_add_to_cache_and_update_continuity() to not set
continuity to false for existing entries, it's not necessary
- moved range tombstone trimming to range_tombstone class
- moved range tombstone slicing code to range_tombstone_list and partition_snapshot
- can_populate::can_use_cache was unused, dropped
- dropped assumption that dummy entries are only at the end
- renamed maybe_add_to_cache_and_update_continuity() to maybe_add_to_cache()
- dropped no longer needed lower_bound class
- extracted row_handle to a separate patch and renamed to partition_snapshot_row_cursor, reworked its API
to fit the new way cache_streamed_mutation is implemented
- made the copy-from-cache loop preemptable
- split maybe_add_next_to_buffer_and_update_continuity(bool)
- dropped cache_populator
- replaced can_populate class with a function
- simplified lsa_manager methods to avoid moves of the functor

In cache_streamed_mutation_test:

- extracted from a larger commit
- removed coupling with how cache_streamed_mutation is created (the
code went out of sync), used more stable make_reader(). it's simpler too.
- replaced false/true literals with is_continuous/is_dummy where appropriate
- dropped tests for cache::underlying (class is gone)
- reused streamed_mutation_assertions, it has better error messages
- fixed the tests to not create tombstones with missing timestamps
- relaxed range tombstone assertions to only check information relevant for the query range
- print cache on failure for improved debugability

Piotr Jastrzebski (29):
cache: Remove support for wide partitions
row_cache: remove unused read overload
position_in_partition: Add after_all_clustered_rows() to view
Extract position_in_partition to separate header
position_in_partition: Introduce for_key()
mutation_partition: Use rows_entry::position() in comparators
partition_snapshot_reader: Reuse rows_entry comparator
mutation_partition: Add support for specifying continuity
tests: row_cache: Add missing apply() to test_mvcc test case
row_cache_test: improve test_sliced_read_row_presence
mutation_reader: Accept forwarding flag in make_reader_returning()
mutation_source: make sure we never ignore fast forwarding
partition_version: Introduce version() getter
Introduce maybe_merge_versions
Track population phase in partition_snapshot
partition_entry: Introduce apply_to_incomplete()
mutation_fragment: make mutation_fragment copyable
mutation_partition: Add rows_entry constructor which accepts full
contents
partition_version: Make operator bool() const-qualified
partition_version: Add const-qualified variant of operator->
partition_snapshot: Add getters for static row and partition tombstone
partition_snapshot: Add const-qualified overload of version()
row_cache: Introduce autoupdating_underlying_reader
row_cache: Introduce read_context
Introduce streamed_mutation_from_forwarding_streamed_mutation
row_cache: Introduce cache_streamed_mutation
Remove unused methods from MVCC
tests: row_cache: Add test_single_partition_update()
tests: Add cache_streamed_mutation_test

Tomasz Grabiec (80):
position_in_partition: Introduce after_key() in the view
position_in_partition: Add is_after_all_clustered_rows()
mutation_partition: Introduce rows_entry::position()
mutation_partition: Make rows_entry comparable with
position_in_partition
partition_snapshot_reader: Encapsulate row walking to simplify
read_next()
partition_snapshot_reader: Use rows_entry::position() for comparing
rows
partition_snapshot_reader: Be prepared for skipping some row entries
tests: row_cache: Improve test_mvcc()
tests: row_cache: Apply only fully continuous mutations to underlying
mutation source
tests: mutation: Generate mutations with continuity
tests: random_mutation_generator: Generate random continuity
tests: Add test for continuity merging rules
mutation_partition: Introduce r-value accepting deletable_row::apply()
partition_version: Introduce partition_entry::with_detached_versions()
range_tombstone_list: Introduce trim()
range_tombstone_list: Make printable
range_tombstone_stream: Make printable
range_tombstone_list: Introduce equal()
mutation: Introduce sliced()
tests: mutation_reader_assertions: Simplify
tests: mutation_assertions: Add ability to limit verification to given
clustering_row_ranges
tests: mutation_source: Relax expectations about range tombstones
partition_snapshot: Add getter for range tombstones
position_in_partition: Introduce as_start_bound_view()
position_in_partition: Introduce no_clustering_row_between()
position_in_partition: Introduce for_range_start()/for_range_end()
range_tombstone: Introduce trim_front()
range_tombstone_list: Introduce slice() working with position range
mutation_reader: Introduce concept of snapshot_source
mutation_reader: Introduce make_empty_*_source()
mutation_reader: Introduce make_combined_mutation_source()
mutation_source: Make copying cheaper
tests: Introduce memtable_snapshot_source
database: Allow obtaining snapshot_source for sstables
row_cache: Rework invalidate() implementation
database: Add missing cache invalidation after attaching sstables
row_cache: Switch to using snapshot_source
dht: Add ring_position min()/max()
row_cache: Switch readers to use per-entry snapshots
row_cache: Ensure there is always a dummy entry after all clustered
rows
row_cache: Introduce cache_entry constructor which constructs
incomplete entry
row_cache: Make logger accessible so that cache_streamed_mutation can
use it
row_cache: Return cache_entry reference from do_find_or_create_entry
row_cache: Introduce find_or_create() helper
Introduce partition_snapshot_row_cursor
streamed_mutation: Introduce make_empty_streamed_mutation()
partition_version: Make is_referenced() const-qualified
partition_version: Add versions() getter
row_cache: Make printable
row_cache: Switch invalidate_unwrapped() to use ring_position_view
ranges
row_cache: Allow comparing with ring_position views in
row_cache::compare
row_cache: Introduce cache_entry::position()
row_cache: Print position instead of key of cache_entry
dht: Make ring_position_view copyable
row_cache: Introduce partition_range_iterator
range: Introduce trim_front()
row_cache: Keep only one streamed_mutation in
scanning_and_populating_reader
row_cache: Move autoupdating_underlying_reader to read_context.hh
row_cache: Store information whether query is a range query in
read_context
row_cache: Store autoupdating_underlying_reader in read_context
row_cache: Allow specifying desired snapshot in
autoupdating_underlying_reader
row_cache: Allow reading from underlying through read_context
intrusive_set_external_comparator: Introduce insert_check()
streamed_mutation: Introduce transform()
converting_mutation_partition_applier: Expose cell upgrade logic
Introduce schema_upgrader
row_cache: Enable partial partition population
tests: row_cache: Add test_scan_with_partial_partitions
mutation_fragment: Implement equality check
tests: mutation_assertions: Allow expecting fragments
row_cache: Introduce evict()
tests: simple_schema: Add missing include
tests: simple_schema: Fix comment
tests: simple_schema: Introduce get_value(const clustered_row&) helper
tests: simple_schema: Introduce make_ckeys()
tests: simple_schema: Make new_timestamp() public
tests: simple_schema: Make add_row() accept optional timestamp
tests: simple_schema: Accept value by reference
utils: Add helpers for dealing with nonwrapping_range<int>
tests: Introduce row_cache_stress_test

configure.py | 3 +
test.py | 1 +
cache_streamed_mutation.hh | 484 ++++++++++++
clustering_bounds_comparator.hh | 2 +-
converting_mutation_partition_applier.hh | 19 +-
database.hh | 5 +-
db/config.hh | 3 -
dht/i_partitioner.hh | 22 +
hashing_partition_visitor.hh | 5 +-
intrusive_set_external_comparator.hh | 8 +-
mutation.hh | 5 +-
mutation_partition.hh | 168 +++-
mutation_partition_applier.hh | 4 +-
mutation_partition_visitor.hh | 16 +-
mutation_reader.hh | 51 +-
partition_builder.hh | 4 +-
partition_snapshot_reader.hh | 117 ++-
partition_snapshot_row_cursor.hh | 208 +++++
partition_version.hh | 83 +-
position_in_partition.hh | 459 +++++++++++
range.hh | 6 +
range_tombstone.hh | 24 +-
range_tombstone_list.hh | 10 +
read_context.hh | 208 +++++
row_cache.hh | 268 ++++---
schema_upgrader.hh | 64 ++
streamed_mutation.hh | 488 +++---------
tests/memtable_snapshot_source.hh | 129 +++
tests/mutation_assertions.hh | 49 +-
tests/mutation_reader_assertions.hh | 54 +-
tests/simple_schema.hh | 36 +-
utils/int_range.hh | 55 ++
cql3/statements/batch_statement.cc | 2 +-
database.cc | 47 +-
mutation.cc | 50 +-
mutation_partition.cc | 114 ++-
mutation_partition_serializer.cc | 2 +-
mutation_reader.cc | 36 +-
partition_version.cc | 419 ++++++++--
range_tombstone_list.cc | 45 ++
row_cache.cc | 875 +++++++++-----------
service/storage_proxy.cc | 2 +-
streamed_mutation.cc | 61 +-
tests/cache_streamed_mutation_test.cc | 1272 ++++++++++++++++++++++++++++++
tests/memory_footprint.cc | 2 +-
tests/mutation_source_test.cc | 39 +-
tests/mutation_test.cc | 72 +-
tests/perf_row_cache_update.cc | 2 +-
tests/row_cache_alloc_stress.cc | 4 +-
tests/row_cache_stress_test.cc | 372 +++++++++
tests/row_cache_test.cc | 457 ++++++-----
tests/streamed_mutation_test.cc | 22 +-
52 files changed, 5433 insertions(+), 1520 deletions(-)
create mode 100644 cache_streamed_mutation.hh
create mode 100644 partition_snapshot_row_cursor.hh
create mode 100644 position_in_partition.hh
create mode 100644 read_context.hh
create mode 100644 schema_upgrader.hh
create mode 100644 tests/memtable_snapshot_source.hh
create mode 100644 utils/int_range.hh
create mode 100644 tests/cache_streamed_mutation_test.cc
create mode 100644 tests/row_cache_stress_test.cc

--
2.7.4

Tomasz Grabiec

<tgrabiec@scylladb.com>
unread,
Jun 12, 2017, 3:01:55 PM6/12/17
to scylladb-dev@googlegroups.com, Piotr Jastrzebski
From: Piotr Jastrzebski <pi...@scylladb.com>

This will be handled by row cache now.

Signed-off-by: Piotr Jastrzebski <pi...@scylladb.com>
---
database.hh | 1 -
db/config.hh | 3 -
mutation.hh | 1 -
row_cache.hh | 34 +--------
database.cc | 3 +-
mutation.cc | 46 +-----------
row_cache.cc | 190 +++++-------------------------------------------
tests/row_cache_test.cc | 51 -------------
8 files changed, 24 insertions(+), 305 deletions(-)

diff --git a/database.hh b/database.hh
index ea55ce6..e873ab0 100644
--- a/database.hh
+++ b/database.hh
@@ -406,7 +406,6 @@ class column_family {
restricted_mutation_reader_config read_concurrency_config;
restricted_mutation_reader_config streaming_read_concurrency_config;
::cf_stats* cf_stats = nullptr;
- uint64_t max_cached_partition_size_in_bytes;
};
struct no_commitlog {};
struct stats {
diff --git a/db/config.hh b/db/config.hh
index cee8c65..f87e710 100644
--- a/db/config.hh
+++ b/db/config.hh
@@ -373,9 +373,6 @@ class config {
val(reduce_cache_sizes_at, double, .85, Invalid, \
"When Java heap usage (after a full concurrent mark sweep (CMS) garbage collection) exceeds this percentage, Cassandra reduces the cache capacity to the fraction of the current size as specified by reduce_cache_capacity_to. To disable, set the value to 1.0." \
) \
- val(max_cached_partition_size_in_kb, uint64_t, 10240uLL, Used, \
- "Partitions with size greater than this value won't be cached." \
- ) \
/* Disks settings */ \
val(stream_throughput_outbound_megabits_per_sec, uint32_t, 400, Unused, \
"Throttles all outbound streaming file transfers on a node to the specified throughput. Cassandra does mostly sequential I/O when streaming data during bootstrap or repair, which can lead to saturating the network connection and degrading client (RPC) performance." \
diff --git a/mutation.hh b/mutation.hh
index 10c29c0..0ac0e85 100644
--- a/mutation.hh
+++ b/mutation.hh
@@ -185,4 +185,3 @@ boost::iterator_range<std::vector<mutation>::const_iterator> slice(

future<mutation_opt> mutation_from_streamed_mutation(streamed_mutation_opt sm);
future<mutation> mutation_from_streamed_mutation(streamed_mutation& sm);
-future<mutation_opt> mutation_from_streamed_mutation_with_limit(streamed_mutation sm, size_t limit);
diff --git a/row_cache.hh b/row_cache.hh
index 1b7cbd6..0824b13 100644
--- a/row_cache.hh
+++ b/row_cache.hh
@@ -60,7 +60,6 @@ class cache_entry {
// True when we know that there is nothing between this entry and the next one in cache
struct {
bool _continuous : 1;
- bool _wide_partition : 1;
bool _dummy_entry : 1;
} _flags{};
lru_link_type _lru_link;
@@ -77,15 +76,6 @@ class cache_entry {
_flags._dummy_entry = true;
}

- struct wide_partition_tag{};
-
- cache_entry(schema_ptr s, const dht::decorated_key& key, wide_partition_tag)
- : _schema(std::move(s))
- , _key(key)
- {
- _flags._wide_partition = true;
- }
-
cache_entry(schema_ptr s, const dht::decorated_key& key, const mutation_partition& p)
: _schema(std::move(s))
, _key(key)
@@ -112,19 +102,10 @@ class cache_entry {
partition_entry& partition() { return _pe; }
const schema_ptr& schema() const { return _schema; }
schema_ptr& schema() { return _schema; }
- // Requires: !wide_partition()
streamed_mutation read(row_cache&, const schema_ptr&, streamed_mutation::forwarding);
- // Requires: !wide_partition()
streamed_mutation read(row_cache&, const schema_ptr&, const query::partition_slice&, streamed_mutation::forwarding);
- // May return disengaged optional if the partition is empty.
- future<streamed_mutation_opt> read_wide(row_cache&, schema_ptr, const query::partition_slice&, const io_priority_class&, streamed_mutation::forwarding);
bool continuous() const { return _flags._continuous; }
void set_continuous(bool value) { _flags._continuous = value; }
- bool wide_partition() const { return _flags._wide_partition; }
- void set_wide_partition() {
- _flags._wide_partition = true;
- _pe = {};
- }

bool is_dummy_entry() const { return _flags._dummy_entry; }

@@ -190,13 +171,10 @@ class cache_tracker final {
struct stats {
uint64_t hits;
uint64_t misses;
- uint64_t uncached_wide_partitions;
- uint64_t wide_partition_mispopulations;
uint64_t insertions;
uint64_t concurrent_misses_same_key;
uint64_t merges;
uint64_t evictions;
- uint64_t wide_partition_evictions;
uint64_t removals;
uint64_t partitions;
uint64_t modification_count;
@@ -206,7 +184,6 @@ class cache_tracker final {
seastar::metrics::metric_groups _metrics;
logalloc::region _region;
lru_type _lru;
- lru_type _wide_partition_lru;
private:
void setup_metrics();
public:
@@ -215,21 +192,17 @@ class cache_tracker final {
void clear();
void touch(cache_entry&);
void insert(cache_entry&);
- void mark_wide(cache_entry&);
void clear_continuity(cache_entry& ce);
void on_erase();
void on_merge();
void on_hit();
void on_miss();
void on_miss_already_populated();
- void on_uncached_wide_partition();
- void on_wide_partition_mispopulation();
allocation_strategy& allocator();
logalloc::region& region();
const logalloc::region& region() const;
uint64_t modification_count() const { return _stats.modification_count; }
uint64_t partitions() const { return _stats.partitions; }
- uint64_t uncached_wide_partitions() const { return _stats.uncached_wide_partitions; }
const stats& get_stats() const { return _stats; }
};

@@ -266,7 +239,6 @@ class row_cache final {
schema_ptr _schema;
partitions_type _partitions; // Cached partitions are complete.
mutation_source _underlying;
- uint64_t _max_cached_partition_size_in_bytes;

// Synchronizes populating reads with updates of underlying data source to ensure that cache
// remains consistent across flushes with the underlying data source.
@@ -289,7 +261,6 @@ class row_cache final {
streamed_mutation::forwarding);
void on_hit();
void on_miss();
- void on_uncached_wide_partition();
void upgrade_entry(cache_entry&);
void invalidate_locked(const dht::decorated_key&);
void invalidate_unwrapped(const dht::partition_range&);
@@ -324,7 +295,7 @@ class row_cache final {
}
public:
~row_cache();
- row_cache(schema_ptr, mutation_source underlying, cache_tracker&, uint64_t _max_cached_partition_size_in_bytes = 10 * 1024 * 1024);
+ row_cache(schema_ptr, mutation_source underlying, cache_tracker&);
row_cache(row_cache&&) = default;
row_cache(const row_cache&) = delete;
row_cache& operator=(row_cache&&) = default;
@@ -346,9 +317,6 @@ class row_cache final {
// information there is for its partition in the underlying data sources.
void populate(const mutation& m, const previous_entry_pointer* previous = nullptr);

- // Caches an information that a partition with a given key is wide.
- void mark_partition_as_wide(const dht::decorated_key& key, const previous_entry_pointer* previous = nullptr);
-
// Clears the cache.
// Guarantees that cache will not be populated using readers created
// before this method was invoked.
diff --git a/database.cc b/database.cc
index 65cc58e..288540b 100644
--- a/database.cc
+++ b/database.cc
@@ -142,7 +142,7 @@ column_family::column_family(schema_ptr schema, config config, db::commitlog* cl
, _streaming_memtables(_config.enable_disk_writes ? make_streaming_memtable_list() : make_memory_only_memtable_list())
, _compaction_strategy(make_compaction_strategy(_schema->compaction_strategy(), _schema->compaction_strategy_options()))
, _sstables(make_lw_shared(_compaction_strategy.make_sstable_set(_schema)))
- , _cache(_schema, sstables_as_mutation_source(), global_cache_tracker(), _config.max_cached_partition_size_in_bytes)
+ , _cache(_schema, sstables_as_mutation_source(), global_cache_tracker())
, _commitlog(cl)
, _compaction_manager(compaction_manager)
, _flush_queue(std::make_unique<memtable_flush_queue>())
@@ -2552,7 +2552,6 @@ keyspace::make_column_family_config(const schema& s, const db::config& db_config
cfg.streaming_read_concurrency_config = _config.streaming_read_concurrency_config;
cfg.cf_stats = _config.cf_stats;
cfg.enable_incremental_backups = _config.enable_incremental_backups;
- cfg.max_cached_partition_size_in_bytes = db_config.max_cached_partition_size_in_kb() * 1024;

return cfg;
}
diff --git a/mutation.cc b/mutation.cc
index 6dbb47f..ef262a8 100644
--- a/mutation.cc
+++ b/mutation.cc
@@ -206,37 +206,14 @@ mutation& mutation::operator+=(mutation&& other) {
return *this;
}

-enum class limit_mutation_size { yes, no };
-
-template <limit_mutation_size with_limit>
class mutation_rebuilder {
mutation _m;
streamed_mutation& _sm;
size_t _remaining_limit;

- template <typename T> bool check_remaining_limit(const T& e) {
- if (with_limit == limit_mutation_size::no) {
- return true;
- }
- size_t size = e.memory_usage();
- if (_remaining_limit <= size) {
- _remaining_limit = 0;
- } else {
- _remaining_limit -= size;
- }
- return _remaining_limit > 0;
- }
public:
mutation_rebuilder(streamed_mutation& sm)
: _m(sm.decorated_key(), sm.schema()), _sm(sm), _remaining_limit(0) {
- static_assert(with_limit == limit_mutation_size::no,
- "This constructor should be used only for mutation_rebuildeer with no limit");
- }
- mutation_rebuilder(streamed_mutation& sm, size_t limit)
- : _m(sm.decorated_key(), sm.schema()), _sm(sm), _remaining_limit(limit) {
- static_assert(with_limit == limit_mutation_size::yes,
- "This constructor should be used only for mutation_rebuildeer with limit");
- check_remaining_limit(_m.key());
}

stop_iteration consume(tombstone t) {
@@ -245,25 +222,16 @@ class mutation_rebuilder {
}

stop_iteration consume(range_tombstone&& rt) {
- if (!check_remaining_limit(rt)) {
- return stop_iteration::yes;
- }
_m.partition().apply_row_tombstone(*_m.schema(), std::move(rt));
return stop_iteration::no;
}

stop_iteration consume(static_row&& sr) {
- if (!check_remaining_limit(sr)) {
- return stop_iteration::yes;
- }
_m.partition().static_row().apply(*_m.schema(), column_kind::static_column, std::move(sr.cells()));
return stop_iteration::no;
}

stop_iteration consume(clustering_row&& cr) {
- if (!check_remaining_limit(cr)) {
- return stop_iteration::yes;
- }
auto& dr = _m.partition().clustered_row(*_m.schema(), std::move(cr.key()));
dr.apply(cr.tomb());
dr.apply(cr.marker());
@@ -272,29 +240,21 @@ class mutation_rebuilder {
}

mutation_opt consume_end_of_stream() {
- return with_limit == limit_mutation_size::yes && _remaining_limit == 0 ? mutation_opt()
- : mutation_opt(std::move(_m));
+ return mutation_opt(std::move(_m));
}
};

-future<mutation_opt>
-mutation_from_streamed_mutation_with_limit(streamed_mutation sm, size_t limit) {
- return do_with(std::move(sm), [limit] (auto& sm) {
- return consume(sm, mutation_rebuilder<limit_mutation_size::yes>(sm, limit));
- });
-}
-
future<mutation_opt> mutation_from_streamed_mutation(streamed_mutation_opt sm) {
if (!sm) {
return make_ready_future<mutation_opt>();
}
return do_with(std::move(*sm), [] (auto& sm) {
- return consume(sm, mutation_rebuilder<limit_mutation_size::no>(sm));
+ return consume(sm, mutation_rebuilder(sm));
});
}

future<mutation> mutation_from_streamed_mutation(streamed_mutation& sm) {
- return consume(sm, mutation_rebuilder<limit_mutation_size::no>(sm)).then([] (mutation_opt&& mo) {
+ return consume(sm, mutation_rebuilder(sm)).then([] (mutation_opt&& mo) {
return std::move(*mo);
});
}
diff --git a/row_cache.cc b/row_cache.cc
index b9c6176..c4ee213 100644
--- a/row_cache.cc
+++ b/row_cache.cc
@@ -40,23 +40,6 @@ static logging::logger clogger("cache");

thread_local seastar::thread_scheduling_group row_cache::_update_thread_scheduling_group(1ms, 0.2);

-enum class is_wide_partition { yes, no };
-
-future<is_wide_partition, mutation_opt>
-try_to_read(uint64_t max_cached_partition_size_in_bytes, streamed_mutation_opt&& sm) {
- if (!sm) {
- return make_ready_future<is_wide_partition, mutation_opt>(is_wide_partition::no, mutation_opt());
- }
- return mutation_from_streamed_mutation_with_limit(std::move(*sm), max_cached_partition_size_in_bytes).then(
- [] (mutation_opt&& omo) mutable {
- if (omo) {
- return make_ready_future<is_wide_partition, mutation_opt>(is_wide_partition::no, std::move(omo));
- } else {
- return make_ready_future<is_wide_partition, mutation_opt>(is_wide_partition::yes, mutation_opt());
- }
- });
-}
-
cache_tracker& global_cache_tracker() {
static thread_local cache_tracker instance;
return instance;
@@ -77,19 +60,10 @@ cache_tracker::cache_tracker() {
clear_continuity(*std::next(it));
lru.pop_back_and_dispose(current_deleter<cache_entry>());
};
- if (!_wide_partition_lru.empty() && (_normal_eviction_count == 0 || _lru.empty())) {
- evict_last(_wide_partition_lru);
- _normal_eviction_count = _normal_large_eviction_ratio;
- ++_stats.wide_partition_evictions;
- } else {
- if (_lru.empty()) {
- return memory::reclaiming_result::reclaimed_nothing;
- }
- evict_last(_lru);
- if (_normal_eviction_count > 0) {
- --_normal_eviction_count;
- }
+ if (_lru.empty()) {
+ return memory::reclaiming_result::reclaimed_nothing;
}
+ evict_last(_lru);
--_stats.partitions;
++_stats.evictions;
++_stats.modification_count;
@@ -117,13 +91,10 @@ cache_tracker::setup_metrics() {
sm::make_gauge("bytes_total", sm::description("total size of memory for the cache"), [this] { return _region.occupancy().total_space(); }),
sm::make_derive("total_operations_hits", sm::description("total number of operation hits"), _stats.hits),
sm::make_derive("total_operations_misses", sm::description("total number of operation misses"), _stats.misses),
- sm::make_derive("total_operations_uncached_wide_partitions", sm::description("total number of operation of uncached wide partitions"), _stats.uncached_wide_partitions),
sm::make_derive("total_operations_insertions", sm::description("total number of operation insert"), _stats.insertions),
sm::make_derive("total_operations_concurrent_misses_same_key", sm::description("total number of operation with misses same key"), _stats.concurrent_misses_same_key),
sm::make_derive("total_operations_merges", sm::description("total number of operation merged"), _stats.merges),
sm::make_derive("total_operations_evictions", sm::description("total number of operation eviction"), _stats.evictions),
- sm::make_derive("total_operations_wide_partition_evictions", sm::description("total number of operation wide partition eviction"), _stats.wide_partition_evictions),
- sm::make_derive("total_operations_wide_partition_mispopulations", sm::description("total number of operation wide partition mispopulations"), _stats.wide_partition_mispopulations),
sm::make_derive("total_operations_removals", sm::description("total number of operation removals"), _stats.removals),
sm::make_gauge("objects_partitions", sm::description("total number of partition objects"), _stats.partitions)
});
@@ -145,7 +116,6 @@ void cache_tracker::clear() {
}
};
clear(_lru);
- clear(_wide_partition_lru);
});
_stats.removals += _stats.partitions;
_stats.partitions = 0;
@@ -157,26 +127,14 @@ void cache_tracker::touch(cache_entry& e) {
lru.erase(lru.iterator_to(e));
lru.push_front(e);
};
- move_to_front(e.wide_partition() ? _wide_partition_lru : _lru, e);
+ move_to_front(_lru, e);
}

void cache_tracker::insert(cache_entry& entry) {
++_stats.insertions;
++_stats.partitions;
++_stats.modification_count;
- if (entry.wide_partition()) {
- _wide_partition_lru.push_front(entry);
- } else {
- _lru.push_front(entry);
- }
-}
-
-void cache_tracker::mark_wide(cache_entry& entry) {
- if (entry._lru_link.is_linked()) {
- entry._lru_link.unlink();
- }
- entry.set_wide_partition();
- _wide_partition_lru.push_front(entry);
+ _lru.push_front(entry);
}

void cache_tracker::on_erase() {
@@ -201,14 +159,6 @@ void cache_tracker::on_miss_already_populated() {
++_stats.concurrent_misses_same_key;
}

-void cache_tracker::on_uncached_wide_partition() {
- ++_stats.uncached_wide_partitions;
-}
-
-void cache_tracker::on_wide_partition_mispopulation() {
- ++_stats.wide_partition_mispopulations;
-}
-
allocation_strategy& cache_tracker::allocator() {
return _region.allocator();
}
@@ -229,8 +179,6 @@ class single_partition_populating_reader final : public mutation_reader::impl {
mutation_reader _delegate;
const io_priority_class _pc;
const query::partition_slice& _slice;
- dht::partition_range _large_partition_range;
- mutation_reader _large_partition_reader;
tracing::trace_state_ptr _trace_state;
streamed_mutation::forwarding _fwd;
public:
@@ -258,28 +206,16 @@ class single_partition_populating_reader final : public mutation_reader::impl {
if (!sm) {
return make_ready_future<streamed_mutation_opt>(streamed_mutation_opt());
}
- dht::decorated_key dk = sm->decorated_key();
- return try_to_read(_cache._max_cached_partition_size_in_bytes, std::move(sm)).then(
- [this, op = std::move(op), dk = std::move(dk)]
- (is_wide_partition wide_partition, mutation_opt&& mo) {
- if (wide_partition == is_wide_partition::no) {
- if (mo) {
- _cache.populate(*mo);
- mo->upgrade(_schema);
- auto ck_ranges = query::clustering_key_filter_ranges::get_ranges(*_schema, _slice, mo->key());
- auto filtered_partition = mutation_partition(std::move(mo->partition()), *(mo->schema()), std::move(ck_ranges));
- mo->partition() = std::move(filtered_partition);
- return make_ready_future<streamed_mutation_opt>(streamed_mutation_from_mutation(std::move(*mo), _fwd));
- }
- return make_ready_future<streamed_mutation_opt>(streamed_mutation_opt());
- } else {
- _cache.on_uncached_wide_partition();
- _cache._tracker.on_wide_partition_mispopulation();
- _cache.mark_partition_as_wide(dk);
- _large_partition_range = dht::partition_range::make_singular(std::move(dk));
- _large_partition_reader = _underlying(_schema, _large_partition_range, _slice, _pc, _trace_state, _fwd);
- return _large_partition_reader();
+ return mutation_from_streamed_mutation(std::move(sm)).then([this, op = std::move(op)] (mutation_opt&& mo) {
+ if (mo) {
+ _cache.populate(*mo);
+ mo->upgrade(_schema);
+ auto ck_ranges = query::clustering_key_filter_ranges::get_ranges(*_schema, _slice, mo->key());
+ auto filtered_partition = mutation_partition(std::move(mo->partition()), *(mo->schema()), std::move(ck_ranges));
+ mo->partition() = std::move(filtered_partition);
+ return make_ready_future<streamed_mutation_opt>(streamed_mutation_from_mutation(std::move(*mo), _fwd));
}
+ return make_ready_future<streamed_mutation_opt>(streamed_mutation_opt());
});
});
}
@@ -299,10 +235,6 @@ void row_cache::on_miss() {
_tracker.on_miss();
}

-void row_cache::on_uncached_wide_partition() {
- _tracker.on_uncached_wide_partition();
-}
-
class just_cache_scanning_reader final {
schema_ptr _schema;
row_cache& _cache;
@@ -377,16 +309,6 @@ class just_cache_scanning_reader final {
_cache._tracker.touch(ce);
_cache.on_hit();
cache_data cd { { }, ce.continuous() };
- if (ce.wide_partition()) {
- return ce.read_wide(_cache, _schema, _slice, _pc, _fwd).then([this, cd = std::move(cd)] (auto smopt) mutable {
- if (smopt) {
- cd.mut = std::move(*smopt);
- } else {
- cd.mut = streamed_mutation_from_mutation(mutation(*_last, _schema), _fwd);
- }
- return std::move(cd);
- });
- }
cd.mut = ce.read(_cache, _schema, _slice, _fwd);
return make_ready_future<cache_data>(std::move(cd));
});
@@ -410,8 +332,6 @@ class range_populating_reader {
mutation_reader _reader;
bool _reader_created = false;
row_cache::previous_entry_pointer _last_key;
- dht::partition_range _large_partition_range;
- mutation_reader _large_partition_reader;
streamed_mutation::forwarding _fwd;
private:
void update_reader() {
@@ -433,26 +353,6 @@ class range_populating_reader {
}
}

- future<streamed_mutation_opt> handle_large_partition(dht::decorated_key&& dk) {
- _cache.on_uncached_wide_partition();
- _cache._tracker.on_wide_partition_mispopulation();
- _cache.mark_partition_as_wide(dk, &_last_key);
- _last_key.reset(dk, _populate_phase);
-
- _large_partition_range = dht::partition_range::make_singular(dk);
- _large_partition_reader = _cache._underlying(_schema, _large_partition_range, _slice, _pc, _trace_state, _fwd);
- return _large_partition_reader().then([this, dk = std::move(dk)] (auto smopt) mutable -> streamed_mutation_opt {
- _large_partition_reader = {};
- if (!smopt) {
- // We cannot emit disengaged optional since this is a part of range
- // read and it would incorrectly interpreted as end of stream.
- // Produce empty mutation instead.
- return streamed_mutation_from_mutation(mutation(std::move(dk), _schema));
- }
- return smopt;
- });
- }
-
void handle_end_of_stream() {
if (_last_key._populate_phase != _populate_phase) {
return;
@@ -494,14 +394,8 @@ class range_populating_reader {
future<streamed_mutation_opt> operator()() {
update_reader();
return _reader().then([this, op = _cache._populate_phaser.start()] (streamed_mutation_opt smopt) mutable {
- dht::decorated_key dk = smopt ? smopt->decorated_key() : dht::decorated_key{ {}, partition_key::make_empty() };
- return try_to_read(_cache._max_cached_partition_size_in_bytes, std::move(smopt)).then(
- [this, op = std::move(op), dk = std::move(dk)] (is_wide_partition is_wide, mutation_opt&& mo) mutable {
- if (is_wide == is_wide_partition::yes) {
- _cache.on_miss();
- return handle_large_partition(std::move(dk));
- }
-
+ return mutation_from_streamed_mutation(std::move(smopt)).then(
+ [this, op = std::move(op)] (mutation_opt&& mo) mutable {
if (!mo) {
handle_end_of_stream();
return make_ready_future<streamed_mutation_opt>();
@@ -685,14 +579,8 @@ row_cache::make_reader(schema_ptr s,
_tracker.touch(e);
upgrade_entry(e);
mutation_reader reader;
- if (e.wide_partition()) {
- reader = _underlying(s, range, slice, pc, std::move(trace_state), fwd);
- _tracker.on_uncached_wide_partition();
- on_miss();
- } else {
- reader = make_reader_returning(e.read(*this, s, slice, fwd));
- on_hit();
- }
+ reader = make_reader_returning(e.read(*this, s, slice, fwd));
+ on_hit();
return reader;
} else {
auto reader = make_mutation_reader<single_partition_populating_reader>(s, *this, _underlying,
@@ -760,17 +648,6 @@ void row_cache::do_find_or_create_entry(const dht::decorated_key& key,
});
}

-void row_cache::mark_partition_as_wide(const dht::decorated_key& key, const previous_entry_pointer* previous) {
- do_find_or_create_entry(key, previous, [&] (auto i) {
- cache_entry* entry = current_allocator().construct<cache_entry>(
- _schema, key, cache_entry::wide_partition_tag{});
- _tracker.insert(*entry);
- return _partitions.insert(i, *entry);
- }, [&] (auto i) {
- _tracker.mark_wide(*i);
- });
-}
-
void row_cache::populate(const mutation& m, const previous_entry_pointer* previous) {
do_find_or_create_entry(m.decorated_key(), previous, [&] (auto i) {
cache_entry* entry = current_allocator().construct<cache_entry>(
@@ -844,13 +721,11 @@ future<> row_cache::update(memtable& m, partition_presence_checker presence_chec
// FIXME: keep a bitmap indicating which sstables we do cover, so we don't have to
// search it.
if (cache_i != partitions_end() && cache_i->key().equal(*_schema, mem_e.key())) {
- if (!cache_i->wide_partition()) {
cache_entry& entry = *cache_i;
upgrade_entry(entry);
entry.partition().apply(*_schema, std::move(mem_e.partition()), *mem_e.schema());
_tracker.touch(entry);
_tracker.on_merge();
- }
} else if (presence_checker(mem_e.key()) ==
partition_presence_checker_result::definitely_doesnt_exist) {
cache_entry* entry = current_allocator().construct<cache_entry>(
@@ -958,13 +833,11 @@ void row_cache::invalidate_unwrapped(const dht::partition_range& range) {
});
}

-row_cache::row_cache(schema_ptr s, mutation_source fallback_factory,
- cache_tracker& tracker, uint64_t max_cached_partition_size_in_bytes)
+row_cache::row_cache(schema_ptr s, mutation_source fallback_factory, cache_tracker& tracker)
: _tracker(tracker)
, _schema(std::move(s))
, _partitions(cache_entry::compare(_schema))
, _underlying(std::move(fallback_factory))
- , _max_cached_partition_size_in_bytes(max_cached_partition_size_in_bytes)
{
with_allocator(_tracker.allocator(), [this] {
cache_entry* entry = current_allocator().construct<cache_entry>(cache_entry::dummy_entry_tag());
@@ -997,32 +870,11 @@ void row_cache::set_schema(schema_ptr new_schema) noexcept {
_schema = std::move(new_schema);
}

-future<streamed_mutation_opt> cache_entry::read_wide(row_cache& rc,
- schema_ptr s, const query::partition_slice& slice, const io_priority_class& pc, streamed_mutation::forwarding fwd)
-{
- struct range_and_underlyig_reader {
- dht::partition_range _range;
- mutation_reader _reader;
- range_and_underlyig_reader(row_cache& rc, schema_ptr s, dht::partition_range pr,
- const query::partition_slice& slice, const io_priority_class& pc, streamed_mutation::forwarding fwd)
- : _range(std::move(pr))
- , _reader(rc._underlying(s, _range, slice, pc, nullptr, fwd))
- { }
- range_and_underlyig_reader(range_and_underlyig_reader&&) = delete;
- };
- rc._tracker.on_uncached_wide_partition();
- auto pr = dht::partition_range::make_singular(_key);
- auto rd_ptr = std::make_unique<range_and_underlyig_reader>(rc, s, std::move(pr), slice, pc, fwd);
- auto& r_a_ur = *rd_ptr;
- return r_a_ur._reader().finally([rd_ptr = std::move(rd_ptr)] {});
-}
-
streamed_mutation cache_entry::read(row_cache& rc, const schema_ptr& s, streamed_mutation::forwarding fwd) {
return read(rc, s, query::full_slice, fwd);
}

streamed_mutation cache_entry::read(row_cache& rc, const schema_ptr& s, const query::partition_slice& slice, streamed_mutation::forwarding fwd) {
- assert(!wide_partition());
if (_schema->version() != s->version()) {
auto ck_ranges = query::clustering_key_filter_ranges::get_ranges(*s, slice, _key.key());
auto mp = mutation_partition(_pe.squashed(_schema, s), *s, std::move(ck_ranges));
@@ -1040,10 +892,6 @@ const schema_ptr& row_cache::schema() const {

void row_cache::upgrade_entry(cache_entry& e) {
if (e._schema != _schema) {
- if (e.wide_partition()) {
- e._schema = _schema;
- return;
- }
auto& r = _tracker.region();
assert(!r.reclaiming_enabled());
with_allocator(r.allocator(), [this, &e] {
diff --git a/tests/row_cache_test.cc b/tests/row_cache_test.cc
index f91f359..12fc5f5 100644
--- a/tests/row_cache_test.cc
+++ b/tests/row_cache_test.cc
@@ -109,7 +109,6 @@ SEASTAR_TEST_CASE(test_cache_delegates_to_underlying) {
assert_that(cache.make_reader(s, query::full_partition_range))
.produces(m)
.produces_end_of_stream();
- assert(tracker.uncached_wide_partitions() == 0);
});
}

@@ -157,56 +156,6 @@ mutation_reader make_counting_reader(mutation_reader mr, int& counter) {
return make_mutation_reader<partition_counting_reader>(std::move(mr), counter);
}

-SEASTAR_TEST_CASE(test_cache_delegates_to_underlying_only_once_for_wide_partition_full_range) {
- return seastar::async([] {
- auto s = make_schema();
- auto m = make_new_mutation(s);
- int secondary_calls_count = 0;
- cache_tracker tracker;
- row_cache cache(s, mutation_source([&secondary_calls_count, &m] (schema_ptr s, const dht::partition_range& range) {
- return make_counting_reader(make_reader_returning(m), secondary_calls_count);
- }), tracker, 0);
-
- assert_that(cache.make_reader(s, query::full_partition_range))
- .produces(m)
- .produces_end_of_stream();
- // 2 from cache reader (m & eos) + 1 from large partition read
- BOOST_REQUIRE_EQUAL(secondary_calls_count, 3);
- BOOST_REQUIRE_EQUAL(tracker.uncached_wide_partitions(), 1);
- assert_that(cache.make_reader(s, query::full_partition_range))
- .produces(m)
- .produces_end_of_stream();
- // previous 3 + 1 from large partition read
- BOOST_REQUIRE_EQUAL(secondary_calls_count, 4);
- BOOST_REQUIRE_EQUAL(tracker.uncached_wide_partitions(), 2);
- });
-}
-
-SEASTAR_TEST_CASE(test_cache_delegates_to_underlying_only_once_for_wide_partition_single_partition) {
- return seastar::async([] {
- auto s = make_schema();
- auto m = make_new_mutation(s);
- int secondary_calls_count = 0;
- cache_tracker tracker;
- row_cache cache(s, mutation_source([&secondary_calls_count, &m] (schema_ptr s, const dht::partition_range& range) {
- return make_counting_reader(make_reader_returning(m), secondary_calls_count);
- }), tracker, 0);
-
- auto singular_range = dht::partition_range::make_singular(query::ring_position(m.decorated_key()));
-
- assert_that(cache.make_reader(s, singular_range))
- .produces(m)
- .produces_end_of_stream();
- BOOST_REQUIRE_EQUAL(secondary_calls_count, 3);
- BOOST_REQUIRE_EQUAL(tracker.uncached_wide_partitions(), 1);
- assert_that(cache.make_reader(s, singular_range))
- .produces(m)
- .produces_end_of_stream();
- BOOST_REQUIRE_EQUAL(secondary_calls_count, 5);
- BOOST_REQUIRE_EQUAL(tracker.uncached_wide_partitions(), 2);
- });
-}
-
SEASTAR_TEST_CASE(test_cache_delegates_to_underlying_only_once_empty_full_range) {
return seastar::async([] {
auto s = make_schema();
--
2.7.4

Tomasz Grabiec

<tgrabiec@scylladb.com>
unread,
Jun 12, 2017, 3:01:55 PM6/12/17
to scylladb-dev@googlegroups.com, Piotr Jastrzebski
From: Piotr Jastrzebski <pi...@scylladb.com>

This will simplify the following patches and unused
code should be removed anyway.

Signed-off-by: Piotr Jastrzebski <pi...@scylladb.com>
---
row_cache.hh | 1 -
row_cache.cc | 4 ----
2 files changed, 5 deletions(-)

diff --git a/row_cache.hh b/row_cache.hh
index 0824b13..94a3054 100644
--- a/row_cache.hh
+++ b/row_cache.hh
@@ -102,7 +102,6 @@ class cache_entry {
partition_entry& partition() { return _pe; }
const schema_ptr& schema() const { return _schema; }
schema_ptr& schema() { return _schema; }
- streamed_mutation read(row_cache&, const schema_ptr&, streamed_mutation::forwarding);
streamed_mutation read(row_cache&, const schema_ptr&, const query::partition_slice&, streamed_mutation::forwarding);
bool continuous() const { return _flags._continuous; }
void set_continuous(bool value) { _flags._continuous = value; }
diff --git a/row_cache.cc b/row_cache.cc
index c4ee213..ccea333 100644
--- a/row_cache.cc
+++ b/row_cache.cc
@@ -870,10 +870,6 @@ void row_cache::set_schema(schema_ptr new_schema) noexcept {
_schema = std::move(new_schema);
}

-streamed_mutation cache_entry::read(row_cache& rc, const schema_ptr& s, streamed_mutation::forwarding fwd) {
- return read(rc, s, query::full_slice, fwd);
-}
-
streamed_mutation cache_entry::read(row_cache& rc, const schema_ptr& s, const query::partition_slice& slice, streamed_mutation::forwarding fwd) {
if (_schema->version() != s->version()) {
auto ck_ranges = query::clustering_key_filter_ranges::get_ranges(*s, slice, _key.key());
--
2.7.4

Tomasz Grabiec

<tgrabiec@scylladb.com>
unread,
Jun 12, 2017, 3:01:56 PM6/12/17
to scylladb-dev@googlegroups.com, Piotr Jastrzebski
From: Piotr Jastrzebski <pi...@scylladb.com>

This is a position that's always in the end after any
other position. It will be used for dummy rows_entry.

Signed-off-by: Piotr Jastrzebski <pi...@scylladb.com>
---
streamed_mutation.hh | 3 +++
1 file changed, 3 insertions(+)

diff --git a/streamed_mutation.hh b/streamed_mutation.hh
index 55392ed..d74f94f 100644
--- a/streamed_mutation.hh
+++ b/streamed_mutation.hh
@@ -374,6 +374,9 @@ class position_in_partition_view {
static position_in_partition_view before_all_clustered_rows() {
return {range_tag_t(), bound_view::bottom()};
}
+ static position_in_partition_view after_all_clustered_rows() {
+ return {position_in_partition_view::range_tag_t(), bound_view::top()};
+ }

static position_in_partition_view for_static_row() {
return {static_row_tag_t()};
--
2.7.4

Tomasz Grabiec

<tgrabiec@scylladb.com>
unread,
Jun 12, 2017, 3:01:58 PM6/12/17
to scylladb-dev@googlegroups.com, Piotr Jastrzebski
From: Piotr Jastrzebski <pi...@scylladb.com>

[tgrabiec: Take the key by reference]
---
position_in_partition.hh | 4 ++++
1 file changed, 4 insertions(+)

diff --git a/position_in_partition.hh b/position_in_partition.hh
index b374873..0d97b06 100644
--- a/position_in_partition.hh
+++ b/position_in_partition.hh
@@ -103,6 +103,10 @@ class position_in_partition_view {
return {static_row_tag_t()};
}

+ static position_in_partition_view for_key(const clustering_key& ck) {
+ return {clustering_row_tag_t(), ck};
+ }
+
bool is_static_row() const { return !_ck; }

// Returns true if all fragments that can be seen for given schema have
--
2.7.4

Tomasz Grabiec

<tgrabiec@scylladb.com>
unread,
Jun 12, 2017, 3:01:58 PM6/12/17
to scylladb-dev@googlegroups.com, Piotr Jastrzebski
From: Piotr Jastrzebski <pi...@scylladb.com>

This will allow it's usage in mutation_partition.hh

Signed-off-by: Piotr Jastrzebski <pi...@scylladb.com>
---
position_in_partition.hh | 411 +++++++++++++++++++++++++++++++++++++++++++++++
streamed_mutation.hh | 396 +--------------------------------------------
2 files changed, 416 insertions(+), 391 deletions(-)
create mode 100644 position_in_partition.hh

diff --git a/position_in_partition.hh b/position_in_partition.hh
new file mode 100644
index 0000000..b374873
--- /dev/null
+++ b/position_in_partition.hh
@@ -0,0 +1,411 @@
+/*
+ * Copyright (C) 2017 ScyllaDB
+ */
+
+/*
+ * This file is part of Scylla.
+ *
+ * Scylla is free software: you can redistribute it and/or modify
+ * it under the terms of the GNU Affero General Public License as published by
+ * the Free Software Foundation, either version 3 of the License, or
+ * (at your option) any later version.
+ *
+ * Scylla is distributed in the hope that it will be useful,
+ * but WITHOUT ANY WARRANTY; without even the implied warranty of
+ * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
+ * GNU General Public License for more details.
+ *
+ * You should have received a copy of the GNU General Public License
+ * along with Scylla. If not, see <http://www.gnu.org/licenses/>.
+ */
+
+#pragma once
+
+#include "types.hh"
+#include "keys.hh"
+#include "clustering_bounds_comparator.hh"
+#include "query-request.hh"
+
+inline
+lexicographical_relation relation_for_lower_bound(composite_view v) {
+ switch (v.last_eoc()) {
+ case composite::eoc::start:
+ case composite::eoc::none:
+ return lexicographical_relation::before_all_prefixed;
+ case composite::eoc::end:
+ return lexicographical_relation::after_all_prefixed;
+ default:
+ assert(0);
+ }
+}
+
+inline
+lexicographical_relation relation_for_upper_bound(composite_view v) {
+ switch (v.last_eoc()) {
+ case composite::eoc::start:
+ return lexicographical_relation::before_all_prefixed;
+ case composite::eoc::none:
+ return lexicographical_relation::before_all_strictly_prefixed;
+ case composite::eoc::end:
+ return lexicographical_relation::after_all_prefixed;
+ default:
+ assert(0);
+ }
+}
+
+class position_in_partition_view {
+ friend class position_in_partition;
+
+ int _bound_weight = 0;
+ const clustering_key_prefix* _ck; // nullptr for static row
+private:
+ position_in_partition_view(int bound_weight, const clustering_key_prefix* ck)
+ : _bound_weight(bound_weight)
+ , _ck(ck)
+ { }
+ // Returns placement of this position_in_partition relative to *_ck,
+ // or lexicographical_relation::at_prefix if !_ck.
+ lexicographical_relation relation() const {
+ // FIXME: Currently position_range cannot represent a range end bound which
+ // includes just the prefix key or a range start which excludes just a prefix key.
+ // In both cases we should return lexicographical_relation::before_all_strictly_prefixed here.
+ // Refs #1446.
+ if (_bound_weight <= 0) {
+ return lexicographical_relation::before_all_prefixed;
+ } else {
+ return lexicographical_relation::after_all_prefixed;
+ }
+ }
+public:
+ struct static_row_tag_t { };
+ struct clustering_row_tag_t { };
+ struct range_tag_t { };
+ using range_tombstone_tag_t = range_tag_t;
+
+ position_in_partition_view(static_row_tag_t) : _ck(nullptr) { }
+ position_in_partition_view(clustering_row_tag_t, const clustering_key_prefix& ck)
+ : _ck(&ck) { }
+ position_in_partition_view(range_tag_t, bound_view bv)
+ : _bound_weight(weight(bv.kind)), _ck(&bv.prefix) { }
+
+ static position_in_partition_view for_range_start(const query::clustering_range&);
+ static position_in_partition_view for_range_end(const query::clustering_range&);
+
+ static position_in_partition_view before_all_clustered_rows() {
+ return {range_tag_t(), bound_view::bottom()};
+ }
+
+ static position_in_partition_view after_all_clustered_rows() {
+ return {position_in_partition_view::range_tag_t(), bound_view::top()};
+ }
+
+ static position_in_partition_view for_static_row() {
+ return {static_row_tag_t()};
+ }
+
+ bool is_static_row() const { return !_ck; }
+
+ // Returns true if all fragments that can be seen for given schema have
+ // positions >= than this.
+ bool is_before_all_fragments(const schema& s) const {
+ return !_ck || (!s.has_static_columns() && _bound_weight < 0 && _ck->is_empty(s));
+ }
+
+ friend std::ostream& operator<<(std::ostream&, position_in_partition_view);
+};
+
+inline
+position_in_partition_view position_in_partition_view::for_range_start(const query::clustering_range& r) {
+ return {position_in_partition_view::range_tag_t(), bound_view::from_range_start(r)};
+}
+
+inline
+position_in_partition_view position_in_partition_view::for_range_end(const query::clustering_range& r) {
+ return {position_in_partition_view::range_tag_t(), bound_view::from_range_end(r)};
+}
+
+class position_in_partition {
+ int _bound_weight = 0;
+ stdx::optional<clustering_key_prefix> _ck;
+public:
+ struct static_row_tag_t { };
+ struct after_static_row_tag_t { };
+ struct clustering_row_tag_t { };
+ struct after_clustering_row_tag_t { };
+ struct range_tag_t { };
+ using range_tombstone_tag_t = range_tag_t;
+
+ explicit position_in_partition(static_row_tag_t) { }
+ position_in_partition(clustering_row_tag_t, clustering_key_prefix ck)
+ : _ck(std::move(ck)) { }
+ position_in_partition(after_clustering_row_tag_t, clustering_key_prefix ck)
+ // FIXME: Use lexicographical_relation::before_strictly_prefixed here. Refs #1446
+ : _bound_weight(1), _ck(std::move(ck)) { }
+ position_in_partition(range_tag_t, bound_view bv)
+ : _bound_weight(weight(bv.kind)), _ck(bv.prefix) { }
+ position_in_partition(after_static_row_tag_t) :
+ position_in_partition(range_tag_t(), bound_view::bottom()) { }
+ explicit position_in_partition(position_in_partition_view view)
+ : _bound_weight(view._bound_weight)
+ {
+ if (view._ck) {
+ _ck = *view._ck;
+ }
+ }
+
+ static position_in_partition before_all_clustered_rows() {
+ return {position_in_partition::range_tag_t(), bound_view::bottom()};
+ }
+
+ static position_in_partition after_all_clustered_rows() {
+ return {position_in_partition::range_tag_t(), bound_view::top()};
+ }
+
+ static position_in_partition after_key(clustering_key ck) {
+ return {after_clustering_row_tag_t(), std::move(ck)};
+ }
+
+ static position_in_partition for_key(clustering_key ck) {
+ return {clustering_row_tag_t(), std::move(ck)};
+ }
+
+ bool is_static_row() const { return !_ck; }
+ bool is_clustering_row() const { return _ck && !_bound_weight; }
+
+ template<typename Hasher>
+ void feed_hash(Hasher& hasher, const schema& s) const {
+ ::feed_hash(hasher, _bound_weight);
+ if (_ck) {
+ ::feed_hash(hasher, true);
+ _ck->feed_hash(hasher, s);
+ } else {
+ ::feed_hash(hasher, false);
+ }
+ }
+
+ clustering_key_prefix& key() {
+ return *_ck;
+ }
+ const clustering_key_prefix& key() const {
+ return *_ck;
+ }
+ operator position_in_partition_view() const {
+ return { _bound_weight, _ck ? &*_ck : nullptr };
+ }
+
+ // Defines total order on the union of position_and_partition and composite objects.
+ //
+ // The ordering is compatible with position_range (r). The following is satisfied for
+ // all cells with name c included by the range:
+ //
+ // r.start() <= c < r.end()
+ //
+ // The ordering on composites given by this is compatible with but weaker than the cell name order.
+ //
+ // The ordering on position_in_partition given by this is compatible but weaker than the ordering
+ // given by position_in_partition::tri_compare.
+ //
+ class composite_tri_compare {
+ const schema& _s;
+ public:
+ composite_tri_compare(const schema& s) : _s(s) {}
+
+ int operator()(position_in_partition_view a, position_in_partition_view b) const {
+ if (a.is_static_row() || b.is_static_row()) {
+ return b.is_static_row() - a.is_static_row();
+ }
+ auto&& types = _s.clustering_key_type()->types();
+ auto cmp = [&] (const data_type& t, bytes_view c1, bytes_view c2) { return t->compare(c1, c2); };
+ return lexicographical_tri_compare(types.begin(), types.end(),
+ a._ck->begin(_s), a._ck->end(_s),
+ b._ck->begin(_s), b._ck->end(_s),
+ cmp, a.relation(), b.relation());
+ }
+
+ int operator()(position_in_partition_view a, composite_view b) const {
+ if (b.empty()) {
+ return 1; // a cannot be empty.
+ }
+ if (a.is_static_row() || b.is_static()) {
+ return b.is_static() - a.is_static_row();
+ }
+ auto&& types = _s.clustering_key_type()->types();
+ auto b_values = b.values();
+ auto cmp = [&] (const data_type& t, bytes_view c1, bytes_view c2) { return t->compare(c1, c2); };
+ return lexicographical_tri_compare(types.begin(), types.end(),
+ a._ck->begin(_s), a._ck->end(_s),
+ b_values.begin(), b_values.end(),
+ cmp, a.relation(), relation_for_lower_bound(b));
+ }
+
+ int operator()(composite_view a, position_in_partition_view b) const {
+ return -(*this)(b, a);
+ }
+
+ int operator()(composite_view a, composite_view b) const {
+ if (a.is_static() != b.is_static()) {
+ return a.is_static() ? -1 : 1;
+ }
+ auto&& types = _s.clustering_key_type()->types();
+ auto a_values = a.values();
+ auto b_values = b.values();
+ auto cmp = [&] (const data_type& t, bytes_view c1, bytes_view c2) { return t->compare(c1, c2); };
+ return lexicographical_tri_compare(types.begin(), types.end(),
+ a_values.begin(), a_values.end(),
+ b_values.begin(), b_values.end(),
+ cmp,
+ relation_for_lower_bound(a),
+ relation_for_lower_bound(b));
+ }
+ };
+
+ // Less comparator giving the same order as composite_tri_compare.
+ class composite_less_compare {
+ composite_tri_compare _cmp;
+ public:
+ composite_less_compare(const schema& s) : _cmp(s) {}
+
+ template<typename T, typename U>
+ bool operator()(const T& a, const U& b) const {
+ return _cmp(a, b) < 0;
+ }
+ };
+
+ class tri_compare {
+ bound_view::tri_compare _cmp;
+ private:
+ template<typename T, typename U>
+ int compare(const T& a, const U& b) const {
+ bool a_rt_weight = bool(a._ck);
+ bool b_rt_weight = bool(b._ck);
+ if (!a_rt_weight || !b_rt_weight) {
+ return a_rt_weight - b_rt_weight;
+ }
+ return _cmp(*a._ck, a._bound_weight, *b._ck, b._bound_weight);
+ }
+ public:
+ tri_compare(const schema& s) : _cmp(s) { }
+ int operator()(const position_in_partition& a, const position_in_partition& b) const {
+ return compare(a, b);
+ }
+ int operator()(const position_in_partition_view& a, const position_in_partition_view& b) const {
+ return compare(a, b);
+ }
+ int operator()(const position_in_partition& a, const position_in_partition_view& b) const {
+ return compare(a, b);
+ }
+ int operator()(const position_in_partition_view& a, const position_in_partition& b) const {
+ return compare(a, b);
+ }
+ };
+ class less_compare {
+ tri_compare _cmp;
+ public:
+ less_compare(const schema& s) : _cmp(s) { }
+ bool operator()(const position_in_partition& a, const position_in_partition& b) const {
+ return _cmp(a, b) < 0;
+ }
+ bool operator()(const position_in_partition_view& a, const position_in_partition_view& b) const {
+ return _cmp(a, b) < 0;
+ }
+ bool operator()(const position_in_partition& a, const position_in_partition_view& b) const {
+ return _cmp(a, b) < 0;
+ }
+ bool operator()(const position_in_partition_view& a, const position_in_partition& b) const {
+ return _cmp(a, b) < 0;
+ }
+ };
+ class equal_compare {
+ clustering_key_prefix::equality _equal;
+ template<typename T, typename U>
+ bool compare(const T& a, const U& b) const {
+ bool a_rt_weight = bool(a._ck);
+ bool b_rt_weight = bool(b._ck);
+ return a_rt_weight == b_rt_weight
+ && (!a_rt_weight || (_equal(*a._ck, *b._ck)
+ && a._bound_weight == b._bound_weight));
+ }
+ public:
+ equal_compare(const schema& s) : _equal(s) { }
+ bool operator()(const position_in_partition& a, const position_in_partition& b) const {
+ return compare(a, b);
+ }
+ bool operator()(const position_in_partition_view& a, const position_in_partition_view& b) const {
+ return compare(a, b);
+ }
+ bool operator()(const position_in_partition_view& a, const position_in_partition& b) const {
+ return compare(a, b);
+ }
+ bool operator()(const position_in_partition& a, const position_in_partition_view& b) const {
+ return compare(a, b);
+ }
+ };
+ friend std::ostream& operator<<(std::ostream&, const position_in_partition&);
+};
+
+// Includes all position_in_partition objects "p" for which: start <= p < end
+// And only those.
+class position_range {
+private:
+ position_in_partition _start;
+ position_in_partition _end;
+public:
+ static position_range from_range(const query::clustering_range&);
+
+ static position_range for_static_row() {
+ return {
+ position_in_partition(position_in_partition::static_row_tag_t()),
+ position_in_partition(position_in_partition::after_static_row_tag_t())
+ };
+ }
+
+ static position_range full() {
+ return {
+ position_in_partition(position_in_partition::static_row_tag_t()),
+ position_in_partition::after_all_clustered_rows()
+ };
+ }
+
+ static position_range all_clustered_rows() {
+ return {
+ position_in_partition::before_all_clustered_rows(),
+ position_in_partition::after_all_clustered_rows()
+ };
+ }
+
+ position_range(position_range&&) = default;
+ position_range& operator=(position_range&&) = default;
+ position_range(const position_range&) = default;
+ position_range& operator=(const position_range&) = default;
+
+ // Constructs position_range which covers the same rows as given clustering_range.
+ // position_range includes a fragment if it includes position of that fragment.
+ position_range(const query::clustering_range&);
+ position_range(query::clustering_range&&);
+
+ position_range(position_in_partition start, position_in_partition end)
+ : _start(std::move(start))
+ , _end(std::move(end))
+ { }
+
+ const position_in_partition& start() const& { return _start; }
+ position_in_partition&& start() && { return std::move(_start); }
+ const position_in_partition& end() const& { return _end; }
+ position_in_partition&& end() && { return std::move(_end); }
+ bool contains(const schema& s, position_in_partition_view pos) const;
+ bool overlaps(const schema& s, position_in_partition_view start, position_in_partition_view end) const;
+
+ friend std::ostream& operator<<(std::ostream&, const position_range&);
+};
+
+inline
+bool position_range::contains(const schema& s, position_in_partition_view pos) const {
+ position_in_partition::less_compare less(s);
+ return !less(pos, _start) && less(pos, _end);
+}
+
+inline
+bool position_range::overlaps(const schema& s, position_in_partition_view start, position_in_partition_view end) const {
+ position_in_partition::less_compare less(s);
+ return !less(end, _start) && less(start, _end);
+}
diff --git a/streamed_mutation.hh b/streamed_mutation.hh
index d74f94f..f8f9849 100644
--- a/streamed_mutation.hh
+++ b/streamed_mutation.hh
@@ -23,6 +23,7 @@

#include "mutation_partition.hh"
#include "utils/optimized_optional.hh"
+#include "position_in_partition.hh"

#include <experimental/optional>

@@ -38,8 +39,6 @@
// mutation_fragment objects. It reflects the order in which content of
// partition appears in the sstables.

-class position_in_partition_view;
-
class clustering_row {
clustering_key_prefix _ck;
row_tombstone _t;
@@ -300,395 +299,6 @@ class mutation_fragment {
friend std::ostream& operator<<(std::ostream&, const mutation_fragment& mf);
};

-std::ostream& operator<<(std::ostream&, mutation_fragment::kind);
-
-std::ostream& operator<<(std::ostream&, const mutation_fragment& mf);
-
-class position_in_partition;
-
-inline
-lexicographical_relation relation_for_lower_bound(composite_view v) {
- switch (v.last_eoc()) {
- case composite::eoc::start:
- case composite::eoc::none:
- return lexicographical_relation::before_all_prefixed;
- case composite::eoc::end:
- return lexicographical_relation::after_all_prefixed;
- default:
- assert(0);
- }
-}
-
-inline
-lexicographical_relation relation_for_upper_bound(composite_view v) {
- switch (v.last_eoc()) {
- case composite::eoc::start:
- return lexicographical_relation::before_all_prefixed;
- case composite::eoc::none:
- return lexicographical_relation::before_all_strictly_prefixed;
- case composite::eoc::end:
- return lexicographical_relation::after_all_prefixed;
- default:
- assert(0);
- }
-}
-
-class position_in_partition_view {
- friend class position_in_partition;
-
- int _bound_weight = 0;
- const clustering_key_prefix* _ck; // nullptr for static row
-private:
- position_in_partition_view(int bound_weight, const clustering_key_prefix* ck)
- : _bound_weight(bound_weight)
- , _ck(ck)
- { }
- // Returns placement of this position_in_partition relative to *_ck,
- // or lexicographical_relation::at_prefix if !_ck.
- lexicographical_relation relation() const {
- // FIXME: Currently position_range cannot represent a range end bound which
- // includes just the prefix key or a range start which excludes just a prefix key.
- // In both cases we should return lexicographical_relation::before_all_strictly_prefixed here.
- // Refs #1446.
- if (_bound_weight <= 0) {
- return lexicographical_relation::before_all_prefixed;
- } else {
- return lexicographical_relation::after_all_prefixed;
- }
- }
-public:
- struct static_row_tag_t { };
- struct clustering_row_tag_t { };
- struct range_tag_t { };
- using range_tombstone_tag_t = range_tag_t;
-
- position_in_partition_view(static_row_tag_t) : _ck(nullptr) { }
- position_in_partition_view(clustering_row_tag_t, const clustering_key_prefix& ck)
- : _ck(&ck) { }
- position_in_partition_view(range_tag_t, bound_view bv)
- : _bound_weight(weight(bv.kind)), _ck(&bv.prefix) { }
-
- static position_in_partition_view for_range_start(const query::clustering_range&);
- static position_in_partition_view for_range_end(const query::clustering_range&);
-
- static position_in_partition_view before_all_clustered_rows() {
- return {range_tag_t(), bound_view::bottom()};
- }
- static position_in_partition_view after_all_clustered_rows() {
- return {position_in_partition_view::range_tag_t(), bound_view::top()};
- }
-
- static position_in_partition_view for_static_row() {
- return {static_row_tag_t()};
- }
-
- bool is_static_row() const { return !_ck; }
-
- // Returns true if all fragments that can be seen for given schema have
- // positions >= than this.
- bool is_before_all_fragments(const schema& s) const {
- return !_ck || (!s.has_static_columns() && _bound_weight < 0 && _ck->is_empty(s));
- }
-
- friend std::ostream& operator<<(std::ostream&, position_in_partition_view);
-};
-
-inline
-position_in_partition_view position_in_partition_view::for_range_start(const query::clustering_range& r) {
- return {position_in_partition_view::range_tag_t(), bound_view::from_range_start(r)};
-}
-
-inline
-position_in_partition_view position_in_partition_view::for_range_end(const query::clustering_range& r) {
- return {position_in_partition_view::range_tag_t(), bound_view::from_range_end(r)};
-}
-
-class position_in_partition {
- int _bound_weight = 0;
- stdx::optional<clustering_key_prefix> _ck;
-public:
- struct static_row_tag_t { };
- struct after_static_row_tag_t { };
- struct clustering_row_tag_t { };
- struct after_clustering_row_tag_t { };
- struct range_tag_t { };
- using range_tombstone_tag_t = range_tag_t;
-
- explicit position_in_partition(static_row_tag_t) { }
- position_in_partition(clustering_row_tag_t, clustering_key_prefix ck)
- : _ck(std::move(ck)) { }
- position_in_partition(after_clustering_row_tag_t, clustering_key_prefix ck)
- // FIXME: Use lexicographical_relation::before_strictly_prefixed here. Refs #1446
- : _bound_weight(1), _ck(std::move(ck)) { }
- position_in_partition(range_tag_t, bound_view bv)
- : _bound_weight(weight(bv.kind)), _ck(bv.prefix) { }
- position_in_partition(after_static_row_tag_t) :
- position_in_partition(range_tag_t(), bound_view::bottom()) { }
- explicit position_in_partition(position_in_partition_view view)
- : _bound_weight(view._bound_weight)
- {
- if (view._ck) {
- _ck = *view._ck;
- }
- }
-
- static position_in_partition before_all_clustered_rows() {
- return {position_in_partition::range_tag_t(), bound_view::bottom()};
- }
-
- static position_in_partition after_all_clustered_rows() {
- return {position_in_partition::range_tag_t(), bound_view::top()};
- }
-
- static position_in_partition after_key(clustering_key ck) {
- return {after_clustering_row_tag_t(), std::move(ck)};
- }
-
- static position_in_partition for_key(clustering_key ck) {
- return {clustering_row_tag_t(), std::move(ck)};
- }
-
- bool is_static_row() const { return !_ck; }
- bool is_clustering_row() const { return _ck && !_bound_weight; }
-
- template<typename Hasher>
- void feed_hash(Hasher& hasher, const schema& s) const {
- ::feed_hash(hasher, _bound_weight);
- if (_ck) {
- ::feed_hash(hasher, true);
- _ck->feed_hash(hasher, s);
- } else {
- ::feed_hash(hasher, false);
- }
- }
-
- clustering_key_prefix& key() {
- return *_ck;
- }
- const clustering_key_prefix& key() const {
- return *_ck;
- }
- operator position_in_partition_view() const {
- return { _bound_weight, _ck ? &*_ck : nullptr };
- }
-
- // Defines total order on the union of position_and_partition and composite objects.
- //
- // The ordering is compatible with position_range (r). The following is satisfied for
- // all cells with name c included by the range:
- //
- // r.start() <= c < r.end()
- //
- // The ordering on composites given by this is compatible with but weaker than the cell name order.
- //
- // The ordering on position_in_partition given by this is compatible but weaker than the ordering
- // given by position_in_partition::tri_compare.
- //
- class composite_tri_compare {
- const schema& _s;
- public:
- composite_tri_compare(const schema& s) : _s(s) {}
-
- int operator()(position_in_partition_view a, position_in_partition_view b) const {
- if (a.is_static_row() || b.is_static_row()) {
- return b.is_static_row() - a.is_static_row();
- }
- auto&& types = _s.clustering_key_type()->types();
- auto cmp = [&] (const data_type& t, bytes_view c1, bytes_view c2) { return t->compare(c1, c2); };
- return lexicographical_tri_compare(types.begin(), types.end(),
- a._ck->begin(_s), a._ck->end(_s),
- b._ck->begin(_s), b._ck->end(_s),
- cmp, a.relation(), b.relation());
- }
-
- int operator()(position_in_partition_view a, composite_view b) const {
- if (b.empty()) {
- return 1; // a cannot be empty.
- }
- if (a.is_static_row() || b.is_static()) {
- return b.is_static() - a.is_static_row();
- }
- auto&& types = _s.clustering_key_type()->types();
- auto b_values = b.values();
- auto cmp = [&] (const data_type& t, bytes_view c1, bytes_view c2) { return t->compare(c1, c2); };
- return lexicographical_tri_compare(types.begin(), types.end(),
- a._ck->begin(_s), a._ck->end(_s),
- b_values.begin(), b_values.end(),
- cmp, a.relation(), relation_for_lower_bound(b));
- }
-
- int operator()(composite_view a, position_in_partition_view b) const {
- return -(*this)(b, a);
- }
-
- int operator()(composite_view a, composite_view b) const {
- if (a.is_static() != b.is_static()) {
- return a.is_static() ? -1 : 1;
- }
- auto&& types = _s.clustering_key_type()->types();
- auto a_values = a.values();
- auto b_values = b.values();
- auto cmp = [&] (const data_type& t, bytes_view c1, bytes_view c2) { return t->compare(c1, c2); };
- return lexicographical_tri_compare(types.begin(), types.end(),
- a_values.begin(), a_values.end(),
- b_values.begin(), b_values.end(),
- cmp,
- relation_for_lower_bound(a),
- relation_for_lower_bound(b));
- }
- };
-
- // Less comparator giving the same order as composite_tri_compare.
- class composite_less_compare {
- composite_tri_compare _cmp;
- public:
- composite_less_compare(const schema& s) : _cmp(s) {}
-
- template<typename T, typename U>
- bool operator()(const T& a, const U& b) const {
- return _cmp(a, b) < 0;
- }
- };
-
- class tri_compare {
- bound_view::tri_compare _cmp;
- private:
- template<typename T, typename U>
- int compare(const T& a, const U& b) const {
- bool a_rt_weight = bool(a._ck);
- bool b_rt_weight = bool(b._ck);
- if (!a_rt_weight || !b_rt_weight) {
- return a_rt_weight - b_rt_weight;
- }
- return _cmp(*a._ck, a._bound_weight, *b._ck, b._bound_weight);
- }
- public:
- tri_compare(const schema& s) : _cmp(s) { }
- int operator()(const position_in_partition& a, const position_in_partition& b) const {
- return compare(a, b);
- }
- int operator()(const position_in_partition_view& a, const position_in_partition_view& b) const {
- return compare(a, b);
- }
- int operator()(const position_in_partition& a, const position_in_partition_view& b) const {
- return compare(a, b);
- }
- int operator()(const position_in_partition_view& a, const position_in_partition& b) const {
- return compare(a, b);
- }
- };
- class less_compare {
- tri_compare _cmp;
- public:
- less_compare(const schema& s) : _cmp(s) { }
- bool operator()(const position_in_partition& a, const position_in_partition& b) const {
- return _cmp(a, b) < 0;
- }
- bool operator()(const position_in_partition_view& a, const position_in_partition_view& b) const {
- return _cmp(a, b) < 0;
- }
- bool operator()(const position_in_partition& a, const position_in_partition_view& b) const {
- return _cmp(a, b) < 0;
- }
- bool operator()(const position_in_partition_view& a, const position_in_partition& b) const {
- return _cmp(a, b) < 0;
- }
- };
- class equal_compare {
- clustering_key_prefix::equality _equal;
- template<typename T, typename U>
- bool compare(const T& a, const U& b) const {
- bool a_rt_weight = bool(a._ck);
- bool b_rt_weight = bool(b._ck);
- return a_rt_weight == b_rt_weight
- && (!a_rt_weight || (_equal(*a._ck, *b._ck)
- && a._bound_weight == b._bound_weight));
- }
- public:
- equal_compare(const schema& s) : _equal(s) { }
- bool operator()(const position_in_partition& a, const position_in_partition& b) const {
- return compare(a, b);
- }
- bool operator()(const position_in_partition_view& a, const position_in_partition_view& b) const {
- return compare(a, b);
- }
- bool operator()(const position_in_partition_view& a, const position_in_partition& b) const {
- return compare(a, b);
- }
- bool operator()(const position_in_partition& a, const position_in_partition_view& b) const {
- return compare(a, b);
- }
- };
- friend std::ostream& operator<<(std::ostream&, const position_in_partition&);
-};
-
-// Includes all position_in_partition objects "p" for which: start <= p < end
-// And only those.
-class position_range {
-private:
- position_in_partition _start;
- position_in_partition _end;
-public:
- static position_range from_range(const query::clustering_range&);
-
- static position_range for_static_row() {
- return {
- position_in_partition(position_in_partition::static_row_tag_t()),
- position_in_partition(position_in_partition::after_static_row_tag_t())
- };
- }
-
- static position_range full() {
- return {
- position_in_partition(position_in_partition::static_row_tag_t()),
- position_in_partition::after_all_clustered_rows()
- };
- }
-
- static position_range all_clustered_rows() {
- return {
- position_in_partition::before_all_clustered_rows(),
- position_in_partition::after_all_clustered_rows()
- };
- }
-
- position_range(position_range&&) = default;
- position_range& operator=(position_range&&) = default;
- position_range(const position_range&) = default;
- position_range& operator=(const position_range&) = default;
-
- // Constructs position_range which covers the same rows as given clustering_range.
- // position_range includes a fragment if it includes position of that fragment.
- position_range(const query::clustering_range&);
- position_range(query::clustering_range&&);
-
- position_range(position_in_partition start, position_in_partition end)
- : _start(std::move(start))
- , _end(std::move(end))
- { }
-
- const position_in_partition& start() const& { return _start; }
- position_in_partition&& start() && { return std::move(_start); }
- const position_in_partition& end() const& { return _end; }
- position_in_partition&& end() && { return std::move(_end); }
- bool contains(const schema& s, position_in_partition_view pos) const;
- bool overlaps(const schema& s, position_in_partition_view start, position_in_partition_view end) const;
-
- friend std::ostream& operator<<(std::ostream&, const position_range&);
-};
-
-inline
-bool position_range::contains(const schema& s, position_in_partition_view pos) const {
- position_in_partition::less_compare less(s);
- return !less(pos, _start) && less(pos, _end);
-}
-
-inline
-bool position_range::overlaps(const schema& s, position_in_partition_view start, position_in_partition_view end) const {
- position_in_partition::less_compare less(s);
- return !less(end, _start) && less(start, _end);
-}
-
inline position_in_partition_view static_row::position() const
{
return position_in_partition_view(position_in_partition_view::static_row_tag_t());
@@ -699,6 +309,10 @@ inline position_in_partition_view clustering_row::position() const
return position_in_partition_view(position_in_partition_view::clustering_row_tag_t(), _ck);
}

+std::ostream& operator<<(std::ostream&, mutation_fragment::kind);
+
+std::ostream& operator<<(std::ostream&, const mutation_fragment& mf);
+
template<>
struct move_constructor_disengages<mutation_fragment> {
enum { value = true };
--
2.7.4

Tomasz Grabiec

<tgrabiec@scylladb.com>
unread,
Jun 12, 2017, 3:01:59 PM6/12/17
to scylladb-dev@googlegroups.com
---
position_in_partition.hh | 4 ++++
1 file changed, 4 insertions(+)

diff --git a/position_in_partition.hh b/position_in_partition.hh
index 0d97b06..16dc5bd 100644
--- a/position_in_partition.hh
+++ b/position_in_partition.hh
@@ -107,6 +107,10 @@ class position_in_partition_view {
return {clustering_row_tag_t(), ck};
}

+ static position_in_partition_view after_key(const clustering_key& ck) {
+ return {1, &ck};
+ }
+
bool is_static_row() const { return !_ck; }

// Returns true if all fragments that can be seen for given schema have
--
2.7.4

Tomasz Grabiec

<tgrabiec@scylladb.com>
unread,
Jun 12, 2017, 3:02:00 PM6/12/17
to scylladb-dev@googlegroups.com
---
position_in_partition.hh | 8 ++++++++
1 file changed, 8 insertions(+)

diff --git a/position_in_partition.hh b/position_in_partition.hh
index 16dc5bd..61ebffb 100644
--- a/position_in_partition.hh
+++ b/position_in_partition.hh
@@ -119,6 +119,10 @@ class position_in_partition_view {
return !_ck || (!s.has_static_columns() && _bound_weight < 0 && _ck->is_empty(s));
}

+ bool is_after_all_clustered_rows(const schema& s) const {
+ return _ck && _ck->is_empty(s) && _bound_weight > 0;
+ }
+
friend std::ostream& operator<<(std::ostream&, position_in_partition_view);
};

@@ -180,6 +184,10 @@ class position_in_partition {
bool is_static_row() const { return !_ck; }
bool is_clustering_row() const { return _ck && !_bound_weight; }

+ bool is_after_all_clustered_rows(const schema& s) const {
+ return _ck && _ck->is_empty(s) && _bound_weight > 0;
+ }
+
template<typename Hasher>
void feed_hash(Hasher& hasher, const schema& s) const {
::feed_hash(hasher, _bound_weight);
--
2.7.4

Tomasz Grabiec

<tgrabiec@scylladb.com>
unread,
Jun 12, 2017, 3:02:01 PM6/12/17
to scylladb-dev@googlegroups.com
In preparation for enabling dummy entries with postion past all
clustering rows.
---
mutation_partition.hh | 2 ++
mutation_partition.cc | 5 +++++
2 files changed, 7 insertions(+)

diff --git a/mutation_partition.hh b/mutation_partition.hh
index b8ff57d..6175e03 100644
--- a/mutation_partition.hh
+++ b/mutation_partition.hh
@@ -33,6 +33,7 @@
#include "schema.hh"
#include "tombstone.hh"
#include "keys.hh"
+#include "position_in_partition.hh"
#include "atomic_cell_or_collection.hh"
#include "query-result.hh"
#include "mutation_partition_view.hh"
@@ -673,6 +674,7 @@ class rows_entry {
const deletable_row& row() const {
return _row;
}
+ position_in_partition_view position() const;
void apply(row_tombstone t) {
_row.apply(t);
}
diff --git a/mutation_partition.cc b/mutation_partition.cc
index 1d1665d..b03950b 100644
--- a/mutation_partition.cc
+++ b/mutation_partition.cc
@@ -910,6 +910,11 @@ rows_entry::equal(const schema& s, const rows_entry& other) const {
return equal(s, other, s);
}

+position_in_partition_view rows_entry::position() const {
+ return position_in_partition_view(
+ position_in_partition_view::clustering_row_tag_t(), _key);
+}
+
bool
rows_entry::equal(const schema& s, const rows_entry& other, const schema& other_schema) const {
return key().equal(s, other.key()) // Only representation-compatible changes are allowed
--
2.7.4

Tomasz Grabiec

<tgrabiec@scylladb.com>
unread,
Jun 12, 2017, 3:02:02 PM6/12/17
to scylladb-dev@googlegroups.com
---
mutation_partition.hh | 15 +++++++++++++++
1 file changed, 15 insertions(+)

diff --git a/mutation_partition.hh b/mutation_partition.hh
index d1434a5..6e492bf 100644
--- a/mutation_partition.hh
+++ b/mutation_partition.hh
@@ -701,6 +701,15 @@ class rows_entry {
int operator()(const rows_entry& e, const clustering_key& key) const {
return _c(e.position(), position_in_partition_view::for_key(key));
}
+ int operator()(const rows_entry& e, position_in_partition_view p) const {
+ return _c(e.position(), p);
+ }
+ int operator()(position_in_partition_view p, const rows_entry& e) const {
+ return _c(p, e.position());
+ }
+ int operator()(position_in_partition_view p1, position_in_partition_view p2) const {
+ return _c(p1, p2);
+ }
};
struct compare {
tri_compare _c;
@@ -720,6 +729,12 @@ class rows_entry {
bool operator()(const rows_entry& e, const clustering_key_view& key) const {
return _c(e, key) < 0;
}
+ bool operator()(const rows_entry& e, position_in_partition_view p) const {
+ return _c(e.position(), p) < 0;
+ }
+ bool operator()(position_in_partition_view p, const rows_entry& e) const {
+ return _c(p, e.position()) < 0;
+ }
};
template <typename Comparator>
struct delegating_compare {
--
2.7.4

Tomasz Grabiec

<tgrabiec@scylladb.com>
unread,
Jun 12, 2017, 3:02:03 PM6/12/17
to scylladb-dev@googlegroups.com, Piotr Jastrzebski
From: Piotr Jastrzebski <pi...@scylladb.com>

key() will not be valid for dummy entries, but position() is always
valid.

[tgrabiec: Extracted from other commits]
[tgrabiec: Added missing change to range_tombstone_stream::get_next]
---
mutation_partition.hh | 27 ++++++++++++++++++++-------
mutation_partition.cc | 3 ++-
streamed_mutation.cc | 3 +--
3 files changed, 23 insertions(+), 10 deletions(-)

diff --git a/mutation_partition.hh b/mutation_partition.hh
index 6175e03..d1434a5 100644
--- a/mutation_partition.hh
+++ b/mutation_partition.hh
@@ -689,23 +689,36 @@ class rows_entry {
bool empty() const {
return _row.empty();
}
+ struct tri_compare {
+ position_in_partition::tri_compare _c;
+ explicit tri_compare(const schema& s) : _c(s) {}
+ int operator()(const rows_entry& e1, const rows_entry& e2) const {
+ return _c(e1.position(), e2.position());
+ }
+ int operator()(const clustering_key& key, const rows_entry& e) const {
+ return _c(position_in_partition_view::for_key(key), e.position());
+ }
+ int operator()(const rows_entry& e, const clustering_key& key) const {
+ return _c(e.position(), position_in_partition_view::for_key(key));
+ }
+ };
struct compare {
- clustering_key::less_compare _c;
- compare(const schema& s) : _c(s) {}
+ tri_compare _c;
+ explicit compare(const schema& s) : _c(s) {}
bool operator()(const rows_entry& e1, const rows_entry& e2) const {
- return _c(e1._key, e2._key);
+ return _c(e1, e2) < 0;
}
bool operator()(const clustering_key& key, const rows_entry& e) const {
- return _c(key, e._key);
+ return _c(key, e) < 0;
}
bool operator()(const rows_entry& e, const clustering_key& key) const {
- return _c(e._key, key);
+ return _c(e, key) < 0;
}
bool operator()(const clustering_key_view& key, const rows_entry& e) const {
- return _c(key, e._key);
+ return _c(key, e) < 0;
}
bool operator()(const rows_entry& e, const clustering_key_view& key) const {
- return _c(e._key, key);
+ return _c(e, key) < 0;
}
};
template <typename Comparator>
diff --git a/mutation_partition.cc b/mutation_partition.cc
index b03950b..f73fa47 100644
--- a/mutation_partition.cc
+++ b/mutation_partition.cc
@@ -917,7 +917,8 @@ position_in_partition_view rows_entry::position() const {

bool
rows_entry::equal(const schema& s, const rows_entry& other, const schema& other_schema) const {
- return key().equal(s, other.key()) // Only representation-compatible changes are allowed
+ position_in_partition::equal_compare eq(s);
+ return eq(position(), other.position())
&& row().equal(column_kind::regular_column, s, other.row(), other_schema);
}

diff --git a/streamed_mutation.cc b/streamed_mutation.cc
index 5a1bf99..97da4b7 100644
--- a/streamed_mutation.cc
+++ b/streamed_mutation.cc
@@ -472,8 +472,7 @@ mutation_fragment_opt range_tombstone_stream::do_get_next()
mutation_fragment_opt range_tombstone_stream::get_next(const rows_entry& re)
{
if (!_list.empty()) {
- position_in_partition_view view(position_in_partition_view::clustering_row_tag_t(), re.key());
- return !_cmp(view, _list.begin()->position()) ? do_get_next() : mutation_fragment_opt();
+ return !_cmp(re.position(), _list.begin()->position()) ? do_get_next() : mutation_fragment_opt();
}
return { };
}
--
2.7.4

Tomasz Grabiec

<tgrabiec@scylladb.com>
unread,
Jun 12, 2017, 3:02:04 PM6/12/17
to scylladb-dev@googlegroups.com
---
partition_snapshot_reader.hh | 30 +++++++++++++++++++-----------
1 file changed, 19 insertions(+), 11 deletions(-)

diff --git a/partition_snapshot_reader.hh b/partition_snapshot_reader.hh
index 3173d20..0781e2b 100644
--- a/partition_snapshot_reader.hh
+++ b/partition_snapshot_reader.hh
@@ -117,14 +117,27 @@ class partition_snapshot_reader : public streamed_mutation::impl, public MemoryA
boost::range::make_heap(_clustering_rows, _heap_cmp);
}

- void pop_clustering_row() {
+ // Valid if has_more_rows()
+ const rows_entry& pop_clustering_row() {
+ boost::range::pop_heap(_clustering_rows, _heap_cmp);
auto& current = _clustering_rows.back();
+ const rows_entry& e = *current._position;
current._position = std::next(current._position);
if (current._position == current._end) {
_clustering_rows.pop_back();
} else {
boost::range::push_heap(_clustering_rows, _heap_cmp);
}
+ return e;
+ }
+
+ // Valid if has_more_rows()
+ const rows_entry& peek_row() const {
+ return *_clustering_rows.front()._position;
+ }
+
+ bool has_more_rows() const {
+ return !_clustering_rows.empty();
}

mutation_fragment_opt read_static_row() {
@@ -143,20 +156,15 @@ class partition_snapshot_reader : public streamed_mutation::impl, public MemoryA
}

mutation_fragment_opt read_next() {
- if (!_clustering_rows.empty()) {
- auto mf = _range_tombstones.get_next(*_clustering_rows.front()._position);
+ if (has_more_rows()) {
+ auto mf = _range_tombstones.get_next(peek_row());
if (mf) {
return mf;
}

- boost::range::pop_heap(_clustering_rows, _heap_cmp);
- clustering_row result = *_clustering_rows.back()._position;
- pop_clustering_row();
- while (!_clustering_rows.empty() && _eq(_clustering_rows.front()._position->key(), result.key())) {
- boost::range::pop_heap(_clustering_rows, _heap_cmp);
- auto& current = _clustering_rows.back();
- result.apply(*_schema, *current._position);
- pop_clustering_row();
+ clustering_row result = pop_clustering_row();
+ while (has_more_rows() && _eq(peek_row().key(), result.key())) {
+ result.apply(*_schema, pop_clustering_row());
}
_last_entry = position_in_partition(result.position());
return mutation_fragment(std::move(result));
--
2.7.4

Tomasz Grabiec

<tgrabiec@scylladb.com>
unread,
Jun 12, 2017, 3:02:05 PM6/12/17
to scylladb-dev@googlegroups.com, Piotr Jastrzebski
From: Piotr Jastrzebski <pi...@scylladb.com>

---
partition_snapshot_reader.hh | 17 +----------------
1 file changed, 1 insertion(+), 16 deletions(-)

diff --git a/partition_snapshot_reader.hh b/partition_snapshot_reader.hh
index 0781e2b..afba6aa 100644
--- a/partition_snapshot_reader.hh
+++ b/partition_snapshot_reader.hh
@@ -45,21 +45,6 @@ class partition_snapshot_reader : public streamed_mutation::impl, public MemoryA
return _cmp(*b._position, *a._position);
}
};
- class rows_entry_compare {
- position_in_partition::less_compare _cmp;
- public:
- explicit rows_entry_compare(const schema& s) : _cmp(s) { }
- bool operator()(const rows_entry& a, const position_in_partition& b) const {
- position_in_partition_view a_view(position_in_partition_view::clustering_row_tag_t(),
- a.key());
- return _cmp(a_view, b);
- }
- bool operator()(const position_in_partition& a, const rows_entry& b) const {
- position_in_partition_view b_view(position_in_partition_view::clustering_row_tag_t(),
- b.key());
- return _cmp(a, b_view);
- }
- };
private:
// Keeps shared pointer to the container we read mutation from to make sure
// that its lifetime is appropriately extended.
@@ -70,7 +55,7 @@ class partition_snapshot_reader : public streamed_mutation::impl, public MemoryA
query::clustering_row_ranges::const_iterator _ck_range_end;
bool _in_ck_range = false;

- rows_entry_compare _cmp;
+ rows_entry::compare _cmp;
clustering_key_prefix::equality _eq;
heap_compare _heap_cmp;

--
2.7.4

Tomasz Grabiec

<tgrabiec@scylladb.com>
unread,
Jun 12, 2017, 3:02:06 PM6/12/17
to scylladb-dev@googlegroups.com
If some row entries may have to be skipped by the reader then it could
be that _clustering_rows is not empty, but read_next() will return a
disengaged optional because there are no more rows in the current
range. The code assumed that it's never the case, and if read_next()
returns a disengaged optional then we exhousted all ranges. Before
introducing dummy entries this needs to be refactored.
---
partition_snapshot_reader.hh | 27 +++++++++++----------------
1 file changed, 11 insertions(+), 16 deletions(-)

diff --git a/partition_snapshot_reader.hh b/partition_snapshot_reader.hh
index 83afa38..cd6a09b 100644
--- a/partition_snapshot_reader.hh
+++ b/partition_snapshot_reader.hh
@@ -79,8 +79,14 @@ class partition_snapshot_reader : public streamed_mutation::impl, public MemoryA
void refresh_iterators() {
_clustering_rows.clear();

- if (!_in_ck_range && _current_ck_range == _ck_range_end) {
- return;
+ if (!_in_ck_range) {
+ if (_current_ck_range == _ck_range_end) {
+ _end_of_stream = true;
+ return;
+ }
+ for (auto&& v : _snapshot->versions()) {
+ _range_tombstones.apply(v.partition().row_tombstones(), *_current_ck_range);
+ }
}

for (auto&& v : _snapshot->versions()) {
@@ -177,18 +183,13 @@ class partition_snapshot_reader : public streamed_mutation::impl, public MemoryA
}

while (!is_end_of_stream() && !is_buffer_full()) {
- if (_in_ck_range && _clustering_rows.empty()) {
- _in_ck_range = false;
- _current_ck_range = std::next(_current_ck_range);
- refresh_iterators();
- continue;
- }
-
auto mfopt = read_next();
if (mfopt) {
emplace_mutation_fragment(std::move(*mfopt));
} else {
- _end_of_stream = true;
+ _in_ck_range = false;
+ _current_ck_range = std::next(_current_ck_range);
+ refresh_iterators();
}
}
}
@@ -219,12 +220,6 @@ class partition_snapshot_reader : public streamed_mutation::impl, public MemoryA
, _range_tombstones(*s)
, _lsa_region(region)
, _read_section(read_section) {
- for (auto&& v : _snapshot->versions()) {
- auto&& rt_list = v.partition().row_tombstones();
- for (auto&& range : _ck_ranges.ranges()) {
- _range_tombstones.apply(rt_list, range);
- }
- }
do_fill_buffer();
}

--
2.7.4

Tomasz Grabiec

<tgrabiec@scylladb.com>
unread,
Jun 12, 2017, 3:02:06 PM6/12/17
to scylladb-dev@googlegroups.com
key() will not be valid for dummy entries.
---
partition_snapshot_reader.hh | 4 ++--
1 file changed, 2 insertions(+), 2 deletions(-)

diff --git a/partition_snapshot_reader.hh b/partition_snapshot_reader.hh
index afba6aa..83afa38 100644
--- a/partition_snapshot_reader.hh
+++ b/partition_snapshot_reader.hh
@@ -56,7 +56,7 @@ class partition_snapshot_reader : public streamed_mutation::impl, public MemoryA
bool _in_ck_range = false;

rows_entry::compare _cmp;
- clustering_key_prefix::equality _eq;
+ position_in_partition::equal_compare _eq;
heap_compare _heap_cmp;

lw_shared_ptr<partition_snapshot> _snapshot;
@@ -148,7 +148,7 @@ class partition_snapshot_reader : public streamed_mutation::impl, public MemoryA
}

clustering_row result = pop_clustering_row();
- while (has_more_rows() && _eq(peek_row().key(), result.key())) {
+ while (has_more_rows() && _eq(peek_row().position(), result.position())) {
result.apply(*_schema, pop_clustering_row());
}
_last_entry = position_in_partition(result.position());
--
2.7.4

Tomasz Grabiec

<tgrabiec@scylladb.com>
unread,
Jun 12, 2017, 3:02:08 PM6/12/17
to scylladb-dev@googlegroups.com, Piotr Jastrzebski
From: Piotr Jastrzebski <pi...@scylladb.com>

This will allow expressing lack of information about certain range of
rows (including the static row), which will be used in cache to
determine if information in cache is complete or not.

Range continuity is encoded with rows_entry::continuous() flag, which
specifies if the range between that entry and the previous one (both
ends exclusive) is continuous or not. The range after the last
rows_entry is assumed to be continuous. When rows_entry::dummy() is
true, the row itself is discontinuous.

[tgrabiec:
- based on the following commits:
4a5bf75 - Piotr Jastrzebski : mutation_partition: introduce dummy rows_entry
773070e - Piotr Jastrzebski : mutation_partition: add continuity flag to rows_entry
- documented continuity semantics on mutation_partition
- fixed _static_row_cached being lost by mutation_partition copy constructors
- fixed conversion to streamed_mutation to ignore dummy entries
- fixed mutation_partition serializer to drop dummy entries
- documented semantics of continuity on mutation_partition level
- dropped assumptions that dummy entries can be only at the last position
- changed equality to ignore continuity completely, rather than
partially (it was not ignoring dummy entries, but ignoring
continuity flag)
- added printout of continuity information in mutation_partition
- fixed handling of empty entries in apply_reversibly() with regards
to continuity; we no longer can remove empty entries before
merging, since that may affect continuity of the right-hand
mutation. Added _erased flag.
- fixed mutation_partition::clustered_row() with dummy==true to not ignore the key
- fixed partition_builder to not ignore continuity
- renamed dummy_tag_t to dummy_tag. _t suffix is reserved.
- standardized all APIs on is_dummy and is_continuous bool_class:es
- replaced add_dummy_entry() with ensure_last_dummy() with safer semantics
- dropped unused remove_dummy_entry()
- simplified and inlined cache_entry::add_dummy_entry()
- fixed mutation_partition(incomplete_tag) constructor to mark all row ranges as discontinuous
]
---
clustering_bounds_comparator.hh | 2 +-
converting_mutation_partition_applier.hh | 5 +-
hashing_partition_visitor.hh | 5 +-
mutation_partition.hh | 113 +++++++++++++++++++++++++++++--
mutation_partition_applier.hh | 4 +-
mutation_partition_visitor.hh | 16 ++++-
partition_builder.hh | 4 +-
partition_snapshot_reader.hh | 9 ++-
partition_version.hh | 3 +
cql3/statements/batch_statement.cc | 2 +-
mutation_partition.cc | 91 ++++++++++++++++++-------
mutation_partition_serializer.cc | 2 +-
service/storage_proxy.cc | 2 +-
streamed_mutation.cc | 12 +++-
tests/streamed_mutation_test.cc | 2 +-
15 files changed, 221 insertions(+), 51 deletions(-)

diff --git a/clustering_bounds_comparator.hh b/clustering_bounds_comparator.hh
index f55d6a2..61445e9 100644
--- a/clustering_bounds_comparator.hh
+++ b/clustering_bounds_comparator.hh
@@ -54,8 +54,8 @@ static inline bound_kind flip_bound_kind(bound_kind bk)
}

class bound_view {
- const static thread_local clustering_key empty_prefix;
public:
+ const static thread_local clustering_key empty_prefix;
const clustering_key_prefix& prefix;
bound_kind kind;
bound_view(const clustering_key_prefix& prefix, bound_kind kind)
diff --git a/converting_mutation_partition_applier.hh b/converting_mutation_partition_applier.hh
index d06228c..d6dce15 100644
--- a/converting_mutation_partition_applier.hh
+++ b/converting_mutation_partition_applier.hh
@@ -22,6 +22,7 @@
#pragma once

#include "mutation_partition_view.hh"
+#include "mutation_partition.hh"
#include "schema.hh"

// Mutation partition visitor which applies visited data into
@@ -94,8 +95,8 @@ class converting_mutation_partition_applier : public mutation_partition_visitor
_p.apply_row_tombstone(_p_schema, rt);
}

- virtual void accept_row(clustering_key_view key, const row_tombstone& deleted_at, const row_marker& rm) override {
- deletable_row& r = _p.clustered_row(_p_schema, key);
+ virtual void accept_row(clustering_key_view key, const row_tombstone& deleted_at, const row_marker& rm, is_dummy dummy, is_continuous continuous) override {
+ deletable_row& r = _p.clustered_row(_p_schema, key, dummy, continuous);
r.apply(rm);
r.apply(deleted_at);
_current_row = &r;
diff --git a/hashing_partition_visitor.hh b/hashing_partition_visitor.hh
index 53e284d..06a003c 100644
--- a/hashing_partition_visitor.hh
+++ b/hashing_partition_visitor.hh
@@ -63,7 +63,10 @@ class hashing_partition_visitor : public mutation_partition_visitor {
rt.feed_hash(_h, _s);
}

- virtual void accept_row(clustering_key_view key, const row_tombstone& deleted_at, const row_marker& rm) {
+ virtual void accept_row(clustering_key_view key, const row_tombstone& deleted_at, const row_marker& rm, is_dummy dummy, is_continuous) {
+ if (dummy) {
+ return;
+ }
key.feed_hash(_h, _s);
feed_hash(_h, deleted_at);
feed_hash(_h, rm);
diff --git a/mutation_partition.hh b/mutation_partition.hh
index 6e492bf..690d875 100644
--- a/mutation_partition.hh
+++ b/mutation_partition.hh
@@ -643,14 +643,45 @@ class rows_entry {
intrusive_set_external_comparator_member_hook _link;
clustering_key _key;
deletable_row _row;
+ struct flags {
+ // This flag is used only in cache.
+ // If true then there's no uncached data (clustering row) between the clustering row represented by this rows_entry
+ // and the previous rows_entry in the mutation_partition this rows_entry belongs to.
+ bool _continuous : 1;
+ bool _dummy : 1;
+ bool _last : 1;
+ bool _erased : 1; // Used only temporarily during apply_reversibly(). Refs #2012.
+ flags() : _continuous(true), _dummy(false), _last(false), _erased(false) { }
+ } _flags{};
friend class mutation_partition;
public:
- explicit rows_entry(clustering_key&& key)
+ // Creates dummy rows_entry at position which is after all clustered rows
+ rows_entry(dummy_tag, is_continuous continuous)
+ : _key(bound_view::empty_prefix)
+ {
+ _flags._dummy = true;
+ _flags._last = true;
+ _flags._continuous = bool(continuous);
+ }
+ struct erased_tag {};
+ rows_entry(erased_tag, const rows_entry& e)
+ : _key(e._key)
+ {
+ _flags._erased = true;
+ _flags._last = e._flags._last;
+ }
+ explicit rows_entry(clustering_key&& key, is_dummy dummy = is_dummy::no, is_continuous continuous = is_continuous::no)
: _key(std::move(key))
- { }
- explicit rows_entry(const clustering_key& key)
+ {
+ _flags._dummy = bool(dummy);
+ _flags._continuous = bool(continuous);
+ }
+ explicit rows_entry(const clustering_key& key, is_dummy dummy = is_dummy::no, is_continuous continuous = is_continuous::no)
: _key(key)
- { }
+ {
+ _flags._dummy = bool(dummy);
+ _flags._continuous = bool(continuous);
+ }
rows_entry(const clustering_key& key, deletable_row&& row)
: _key(key), _row(std::move(row))
{ }
@@ -661,10 +692,13 @@ class rows_entry {
rows_entry(const rows_entry& e)
: _key(e._key)
, _row(e._row)
+ , _flags(e._flags)
{ }
+ // Valid only if !dummy()
clustering_key& key() {
return _key;
}
+ // Valid only if !dummy()
const clustering_key& key() const {
return _key;
}
@@ -675,6 +709,10 @@ class rows_entry {
return _row;
}
position_in_partition_view position() const;
+ is_continuous continuous() const { return is_continuous(_flags._continuous); }
+ void set_continuous(bool value) { _flags._continuous = value; }
+ void set_continuous(is_continuous value) { set_continuous(bool(value)); }
+ is_dummy dummy() const { return is_dummy(_flags._dummy); }
void apply(row_tombstone t) {
_row.apply(t);
}
@@ -689,6 +727,9 @@ class rows_entry {
bool empty() const {
return _row.empty();
}
+ bool erased() const {
+ return _flags._erased;
+ }
struct tri_compare {
position_in_partition::tri_compare _c;
explicit tri_compare(const schema& s) : _c(s) {}
@@ -742,10 +783,16 @@ class rows_entry {
delegating_compare(Comparator&& c) : _c(std::move(c)) {}
template <typename Comparable>
bool operator()(const Comparable& v, const rows_entry& e) const {
+ if (e._flags._last) {
+ return true;
+ }
return _c(v, e._key);
}
template <typename Comparable>
bool operator()(const rows_entry& e, const Comparable& v) const {
+ if (e._flags._last) {
+ return false;
+ }
return _c(e._key, v);
}
};
@@ -758,6 +805,39 @@ class rows_entry {
bool equal(const schema& s, const rows_entry& other, const schema& other_schema) const;
};

+/**
+ * Represents a set of writes made to a single partition.
+ *
+ * The object is schema-dependent. Each instance is governed by some
+ * specific schema version. Accessors require a reference to a schema object
+ * of that version.
+ *
+ * There is an operation of addition defined on mutation_partition objects (also called "apply"),
+ * which gives as a result an object representing the sum of writes contained in the addends.
+ * For instances governed by the same schema, addition is commutative and associative.
+ *
+ * Supports marking static row and ranges of clustering keys as continuous or discontinuous.
+ * This can be used to represent lack of information about certain range of
+ * rows. By default everything is continuous.
+ *
+ * Continuity information is ignored by instance equality. It's also transient,
+ * not preserved by serialization.
+ *
+ * Range continuity is encoded with rows_entry::continuous() flag, which specifies if
+ * the range between that entry and the previous one (both ends exclusive) is continuous
+ * or not. The range after the last rows_entry is assumed to be continuous.
+ * When rows_entry::dummy() is true, the row itself is discontinuous.
+ *
+ * Adding two fully-continuous instances gives a fully-continuous instance.
+ *
+ * Addition of continuity is not commutative. Continuity flags on objects representing
+ * the same thing (e.g. rows_entry with the same key) are merged such that the information
+ * stored in the left-hand operand wins. Flags on objects which are present only in one of
+ * the operands are transferred as-is. Such merging rules are useful for layering
+ * information in MVCC, where newer versions specify continuity with respect to the
+ * combined set of rows in all prior versions, not just in their versions.
+ *
+ */
class mutation_partition final {
public:
using rows_type = intrusive_set_external_comparator<rows_entry, &rows_entry::_link>;
@@ -766,6 +846,7 @@ class mutation_partition final {
private:
tombstone _tombstone;
row _static_row;
+ bool _static_row_cached = true;
rows_type _rows;
// Contains only strict prefixes so that we don't have to lookup full keys
// in both _row_tombstones and _rows.
@@ -775,6 +856,12 @@ class mutation_partition final {
friend class converting_mutation_partition_applier;
public:
struct copy_comparators_only {};
+ struct incomplete_tag {};
+ // Constructs an empty instance which is fully discontinuous.
+ mutation_partition(incomplete_tag, const schema& s);
+ static mutation_partition make_incomplete(const schema& s) {
+ return mutation_partition(incomplete_tag(), s);
+ }
mutation_partition(schema_ptr s)
: _rows()
, _row_tombstones(*s)
@@ -792,6 +879,7 @@ class mutation_partition final {
mutation_partition& operator=(mutation_partition&& x) noexcept;
bool equal(const schema&, const mutation_partition&) const;
bool equal(const schema& this_schema, const mutation_partition& p, const schema& p_schema) const;
+ bool equal_continuity(const schema&, const mutation_partition&) const;
// Consistent with equal()
template<typename Hasher>
void feed_hash(Hasher& h, const schema& s) const {
@@ -800,6 +888,12 @@ class mutation_partition final {
}
friend std::ostream& operator<<(std::ostream& os, const mutation_partition& mp);
public:
+ // Makes sure there is a dummy entry after all clustered rows. Doesn't affect continuity.
+ // Doesn't invalidate iterators.
+ void ensure_last_dummy(const schema&);
+ bool is_static_row_cached() const { return _static_row_cached; }
+ void set_static_row_cached(bool value) { _static_row_cached = value; }
+ bool is_fully_continuous() const;
void apply(tombstone t) { _tombstone.apply(t); }
void apply_delete(const schema& schema, const clustering_key_prefix& prefix, tombstone t);
void apply_delete(const schema& schema, range_tombstone rt);
@@ -894,9 +988,9 @@ class mutation_partition final {
// Returns true if there is no live data or tombstones.
bool empty() const;
public:
- deletable_row& clustered_row(const schema& s, const clustering_key& key);
- deletable_row& clustered_row(const schema& s, clustering_key&& key);
- deletable_row& clustered_row(const schema& s, const clustering_key_view& key);
+ deletable_row& clustered_row(const schema& s, const clustering_key& key, is_dummy dummy = is_dummy::no, is_continuous continuous = is_continuous::yes);
+ deletable_row& clustered_row(const schema& s, clustering_key&& key, is_dummy dummy = is_dummy::no, is_continuous continuous = is_continuous::yes);
+ deletable_row& clustered_row(const schema& s, clustering_key_view key, is_dummy dummy = is_dummy::no, is_continuous continuous = is_continuous::yes);
public:
tombstone partition_tombstone() const { return _tombstone; }
row& static_row() { return _static_row; }
@@ -916,6 +1010,11 @@ class mutation_partition final {
rows_type::iterator lower_bound(const schema& schema, const query::clustering_range& r);
rows_type::iterator upper_bound(const schema& schema, const query::clustering_range& r);
boost::iterator_range<rows_type::iterator> range(const schema& schema, const query::clustering_range& r);
+ // Returns an iterator range of rows_entry, with only non-dummy entries.
+ auto non_dummy_rows() const {
+ return boost::make_iterator_range(_rows.begin(), _rows.end())
+ | boost::adaptors::filtered([] (const rows_entry& e) { return bool(!e.dummy()); });
+ }
// Writes this partition using supplied query result writer.
// The partition should be first compacted with compact_for_query(), otherwise
// results may include data which is deleted/expired.
diff --git a/mutation_partition_applier.hh b/mutation_partition_applier.hh
index 23a2600..4bcf54e 100644
--- a/mutation_partition_applier.hh
+++ b/mutation_partition_applier.hh
@@ -50,8 +50,8 @@ class mutation_partition_applier : public mutation_partition_visitor {
_p.apply_row_tombstone(_schema, rt);
}

- virtual void accept_row(clustering_key_view key, const row_tombstone& deleted_at, const row_marker& rm) override {
- deletable_row& r = _p.clustered_row(_schema, key);
+ virtual void accept_row(clustering_key_view key, const row_tombstone& deleted_at, const row_marker& rm, is_dummy dummy, is_continuous continuous) override {
+ deletable_row& r = _p.clustered_row(_schema, key, dummy, continuous);
r.apply(rm);
r.apply(deleted_at);
_current_row = &r;
diff --git a/mutation_partition_visitor.hh b/mutation_partition_visitor.hh
index 3e3f014..86b337b 100644
--- a/mutation_partition_visitor.hh
+++ b/mutation_partition_visitor.hh
@@ -29,6 +29,19 @@
class row_marker;
class row_tombstone;

+// When used on an entry, marks the range between this entry and the previous
+// one as continuous or discontinuous, excluding the keys of both entries.
+// This information doesn't apply to continuity of the entries themselves,
+// that is specified by is_dummy flag.
+// See class doc of mutation_partition.
+using is_continuous = bool_class<class continuous_tag>;
+
+// Dummy entry is an entry which is incomplete.
+// Typically used for marking bounds of continuity range.
+// See class doc of mutation_partition.
+class dummy_tag {};
+using is_dummy = bool_class<dummy_tag>;
+
// Guarantees:
//
// - any tombstones which affect cell's liveness are visited before that cell
@@ -56,7 +69,8 @@ class mutation_partition_visitor {

virtual void accept_row_tombstone(const range_tombstone&) = 0;

- virtual void accept_row(clustering_key_view key, const row_tombstone& deleted_at, const row_marker& rm) = 0;
+ virtual void accept_row(clustering_key_view key, const row_tombstone& deleted_at, const row_marker& rm,
+ is_dummy = is_dummy::no, is_continuous = is_continuous::yes) = 0;

virtual void accept_row_cell(column_id id, atomic_cell_view) = 0;

diff --git a/partition_builder.hh b/partition_builder.hh
index f26a9d2..8069b23 100644
--- a/partition_builder.hh
+++ b/partition_builder.hh
@@ -56,8 +56,8 @@ class partition_builder : public mutation_partition_visitor {
_partition.apply_row_tombstone(_schema, rt);
}

- virtual void accept_row(clustering_key_view key, const row_tombstone& deleted_at, const row_marker& rm) override {
- deletable_row& r = _partition.clustered_row(_schema, key);
+ virtual void accept_row(clustering_key_view key, const row_tombstone& deleted_at, const row_marker& rm, is_dummy dummy, is_continuous continuous) override {
+ deletable_row& r = _partition.clustered_row(_schema, key, dummy, continuous);
r.apply(rm);
r.apply(deleted_at);
_current_row = &r;
diff --git a/partition_snapshot_reader.hh b/partition_snapshot_reader.hh
index cd6a09b..dcbdca3 100644
--- a/partition_snapshot_reader.hh
+++ b/partition_snapshot_reader.hh
@@ -147,13 +147,16 @@ class partition_snapshot_reader : public streamed_mutation::impl, public MemoryA
}

mutation_fragment_opt read_next() {
- if (has_more_rows()) {
+ while (has_more_rows()) {
auto mf = _range_tombstones.get_next(peek_row());
if (mf) {
return mf;
}
-
- clustering_row result = pop_clustering_row();
+ const rows_entry& e = pop_clustering_row();
+ if (e.dummy()) {
+ continue;
+ }
+ clustering_row result = e;
while (has_more_rows() && _eq(peek_row().position(), result.position())) {
result.apply(*_schema, pop_clustering_row());
}
diff --git a/partition_version.hh b/partition_version.hh
index e78a14a..66b74a4 100644
--- a/partition_version.hh
+++ b/partition_version.hh
@@ -117,6 +117,8 @@ class partition_version : public anchorless_list_base_hook<partition_version> {

friend class partition_version_ref;
public:
+ explicit partition_version(schema_ptr s) noexcept
+ : _partition(std::move(s)) { }
explicit partition_version(mutation_partition mp) noexcept
: _partition(std::move(mp)) { }
partition_version(partition_version&& pv) noexcept;
@@ -213,6 +215,7 @@ class partition_entry {
partition_version_ref _version;

friend class partition_snapshot;
+ friend class cache_entry;
private:
void set_version(partition_version*);

diff --git a/cql3/statements/batch_statement.cc b/cql3/statements/batch_statement.cc
index fac623d..095f521 100644
--- a/cql3/statements/batch_statement.cc
+++ b/cql3/statements/batch_statement.cc
@@ -233,7 +233,7 @@ void batch_statement::verify_batch_size(const std::vector<mutation>& mutations)
size += v.data.size();
}
void accept_row_tombstone(const range_tombstone&) override {}
- void accept_row(clustering_key_view, const row_tombstone&, const row_marker&) override {}
+ void accept_row(clustering_key_view, const row_tombstone&, const row_marker&, is_dummy, is_continuous) override {}
void accept_row_cell(column_id, atomic_cell_view v) override {
size += v.value().size();
}
diff --git a/mutation_partition.cc b/mutation_partition.cc
index f73fa47..ffbc545 100644
--- a/mutation_partition.cc
+++ b/mutation_partition.cc
@@ -175,7 +175,7 @@ void revert_intrusive_set_range(const schema& s, mutation_partition::rows_type&
assert(i != dst.end());
rows_entry& dst_e = *i;

- if (e.empty()) {
+ if (e.erased()) {
dst.erase(i);
start = src.erase_and_dispose(start, deleter);
start = src.insert_before(start, dst_e);
@@ -203,18 +203,10 @@ auto apply_reversibly_intrusive_set(const schema& s, mutation_partition::rows_ty
while (src_i != src.end()) {
rows_entry& src_e = *src_i;

- // neutral entries will be given special meaning for the purpose of revert, so
- // get rid of empty rows from the input as if they were not there. This doesn't change
- // the value of src.
- if (src_e.empty()) {
- src_i = src.erase_and_dispose(src_i, current_deleter<rows_entry>());
- continue;
- }
-
auto i = dst.lower_bound(src_e, cmp);
if (i == dst.end() || cmp(src_e, *i)) {
- // Construct neutral entry which will represent missing dst entry for revert.
- rows_entry* empty_e = current_allocator().construct<rows_entry>(src_e.key());
+ // Construct erased entry which will represent missing dst entry for revert.
+ rows_entry* empty_e = current_allocator().construct<rows_entry>(rows_entry::erased_tag{}, src_e);
[&] () noexcept {
src_i = src.erase(src_i);
src_i = src.insert_before(src_i, *empty_e);
@@ -235,6 +227,7 @@ auto apply_reversibly_intrusive_set(const schema& s, mutation_partition::rows_ty
mutation_partition::mutation_partition(const mutation_partition& x)
: _tombstone(x._tombstone)
, _static_row(x._static_row)
+ , _static_row_cached(x._static_row_cached)
, _rows()
, _row_tombstones(x._row_tombstones) {
auto cloner = [] (const auto& x) {
@@ -247,6 +240,7 @@ mutation_partition::mutation_partition(const mutation_partition& x, const schema
query::clustering_key_filter_ranges ck_ranges)
: _tombstone(x._tombstone)
, _static_row(x._static_row)
+ , _static_row_cached(x._static_row_cached)
, _rows()
, _row_tombstones(x._row_tombstones, range_tombstone_list::copy_comparator_only()) {
try {
@@ -271,6 +265,7 @@ mutation_partition::mutation_partition(mutation_partition&& x, const schema& sch
query::clustering_key_filter_ranges ck_ranges)
: _tombstone(x._tombstone)
, _static_row(std::move(x._static_row))
+ , _static_row_cached(x._static_row_cached)
, _rows(std::move(x._rows))
, _row_tombstones(std::move(x._row_tombstones))
{
@@ -319,6 +314,13 @@ mutation_partition::operator=(mutation_partition&& x) noexcept {
return *this;
}

+void mutation_partition::ensure_last_dummy(const schema& s) {
+ if (_rows.empty() || !_rows.rbegin()->position().is_after_all_clustered_rows(s)) {
+ _rows.insert_before(_rows.end(),
+ *current_allocator().construct<rows_entry>(dummy_tag{}, is_continuous::yes));
+ }
+}
+
void
mutation_partition::apply(const schema& s, const mutation_partition& p, const schema& p_schema) {
if (s.version() != p_schema.version()) {
@@ -485,10 +487,10 @@ mutation_partition::find_row(const schema& s, const clustering_key& key) const {
}

deletable_row&
-mutation_partition::clustered_row(const schema& s, clustering_key&& key) {
+mutation_partition::clustered_row(const schema& s, clustering_key&& key, is_dummy dummy, is_continuous continuous) {
auto i = _rows.find(key, rows_entry::compare(s));
if (i == _rows.end()) {
- auto e = current_allocator().construct<rows_entry>(std::move(key));
+ auto e = current_allocator().construct<rows_entry>(std::move(key), dummy, continuous);
_rows.insert(i, *e, rows_entry::compare(s));
return e->row();
}
@@ -496,10 +498,10 @@ mutation_partition::clustered_row(const schema& s, clustering_key&& key) {
}

deletable_row&
-mutation_partition::clustered_row(const schema& s, const clustering_key& key) {
+mutation_partition::clustered_row(const schema& s, const clustering_key& key, is_dummy dummy, is_continuous continuous) {
auto i = _rows.find(key, rows_entry::compare(s));
if (i == _rows.end()) {
- auto e = current_allocator().construct<rows_entry>(key);
+ auto e = current_allocator().construct<rows_entry>(key, dummy, continuous);
_rows.insert(i, *e, rows_entry::compare(s));
return e->row();
}
@@ -507,10 +509,10 @@ mutation_partition::clustered_row(const schema& s, const clustering_key& key) {
}

deletable_row&
-mutation_partition::clustered_row(const schema& s, const clustering_key_view& key) {
+mutation_partition::clustered_row(const schema& s, clustering_key_view key, is_dummy dummy, is_continuous continuous) {
auto i = _rows.find(key, rows_entry::compare(s));
if (i == _rows.end()) {
- auto e = current_allocator().construct<rows_entry>(key);
+ auto e = current_allocator().construct<rows_entry>(key, dummy, continuous);
_rows.insert(i, *e, rows_entry::compare(s));
return e->row();
}
@@ -843,13 +845,13 @@ operator<<(std::ostream& os, const deletable_row& dr) {

std::ostream&
operator<<(std::ostream& os, const rows_entry& re) {
- return fprint(os, "{rows_entry: %s %s}", re._key, re._row);
+ return fprint(os, "{rows_entry: cont=%d dummy=%d %s %s}", re.continuous(), re.dummy(), re._key, re._row);
}

std::ostream&
operator<<(std::ostream& os, const mutation_partition& mp) {
- return fprint(os, "{mutation_partition: %s (%s) static %s clustered %s}",
- mp._tombstone, ::join(", ", mp._row_tombstones), mp._static_row,
+ return fprint(os, "{mutation_partition: %s (%s) static cont=%d %s clustered %s}",
+ mp._tombstone, ::join(", ", mp._row_tombstones), mp._static_row_cached, mp._static_row,
::join(", ", mp._rows));
}

@@ -911,8 +913,12 @@ rows_entry::equal(const schema& s, const rows_entry& other) const {
}

position_in_partition_view rows_entry::position() const {
- return position_in_partition_view(
- position_in_partition_view::clustering_row_tag_t(), _key);
+ if (_flags._last) {
+ return position_in_partition_view::after_all_clustered_rows();
+ } else {
+ return position_in_partition_view(
+ position_in_partition_view::clustering_row_tag_t(), _key);
+ }
}

bool
@@ -931,7 +937,7 @@ bool mutation_partition::equal(const schema& this_schema, const mutation_partiti
return false;
}

- if (!std::equal(_rows.begin(), _rows.end(), p._rows.begin(), p._rows.end(),
+ if (!boost::equal(non_dummy_rows(), p.non_dummy_rows(),
[&] (const rows_entry& e1, const rows_entry& e2) {
return e1.equal(this_schema, e2, p_schema);
}
@@ -949,6 +955,16 @@ bool mutation_partition::equal(const schema& this_schema, const mutation_partiti
return _static_row.equal(column_kind::static_column, this_schema, p._static_row, p_schema);
}

+bool mutation_partition::equal_continuity(const schema& s, const mutation_partition& p) const {
+ return _static_row_cached == p._static_row_cached
+ && boost::equal(_rows, p._rows, [&] (const rows_entry& e1, const rows_entry& e2) {
+ position_in_partition::equal_compare eq(s);
+ return eq(e1.position(), e2.position())
+ && e1.continuous() == e2.continuous()
+ && e1.dummy() == e2.dummy();
+ });
+}
+
void
apply_reversibly(const column_definition& def, atomic_cell_or_collection& dst, atomic_cell_or_collection& src) {
// Must be run via with_linearized_managed_bytes() context, but assume it is
@@ -1339,6 +1355,7 @@ rows_entry::rows_entry(rows_entry&& o) noexcept
: _link(std::move(o._link))
, _key(std::move(o._key))
, _row(std::move(o._row))
+ , _flags(std::move(o._flags))
{ }

row::row(const row& o)
@@ -1647,7 +1664,10 @@ mutation_partition mutation_partition::difference(schema_ptr s, const mutation_p
auto it_r = other._rows.begin();
rows_entry::compare cmp_r(*s);
for (auto&& r : _rows) {
- while (it_r != other._rows.end() && cmp_r(*it_r, r)) {
+ if (r.dummy()) {
+ continue;
+ }
+ while (it_r != other._rows.end() && (it_r->dummy() || cmp_r(*it_r, r))) {
++it_r;
}
if (it_r == other._rows.end() || !it_r->key().equal(*s, r.key())) {
@@ -1677,7 +1697,7 @@ void mutation_partition::accept(const schema& s, mutation_partition_visitor& v)
}
for (const rows_entry& e : _rows) {
const deletable_row& dr = e.row();
- v.accept_row(e.key(), dr.deleted_at(), dr.marker());
+ v.accept_row(e.key(), dr.deleted_at(), dr.marker(), e.dummy(), e.continuous());
dr.cells().for_each_cell([&] (column_id id, const atomic_cell_or_collection& cell) {
const column_definition& def = s.regular_column_at(id);
if (def.is_atomic()) {
@@ -2075,6 +2095,27 @@ class counter_write_query_result_builder {
}
};

+mutation_partition::mutation_partition(mutation_partition::incomplete_tag, const schema& s)
+ : _static_row_cached(false)
+ , _rows()
+ , _row_tombstones(s)
+{
+ _rows.insert_before(_rows.end(),
+ *current_allocator().construct<rows_entry>(dummy_tag{}, is_continuous::no));
+}
+
+bool mutation_partition::is_fully_continuous() const {
+ if (!_static_row_cached) {
+ return false;
+ }
+ for (auto&& row : _rows) {
+ if (!row.continuous()) {
+ return false;
+ }
+ }
+ return true;
+}
+
future<mutation_opt> counter_write_query(schema_ptr s, const mutation_source& source,
const dht::decorated_key& dk,
const query::partition_slice& slice,
diff --git a/mutation_partition_serializer.cc b/mutation_partition_serializer.cc
index bc621a0..179b3b6 100644
--- a/mutation_partition_serializer.cc
+++ b/mutation_partition_serializer.cc
@@ -209,7 +209,7 @@ void mutation_partition_serializer::write_serialized(Writer&& writer, const sche
auto row_tombstones = write_row_cells(std::move(srow_writer), mp.static_row(), s, column_kind::static_column).end_static_row().start_range_tombstones();
write_tombstones(s, row_tombstones, mp.row_tombstones());
auto clustering_rows = std::move(row_tombstones).end_range_tombstones().start_rows();
- for (auto&& cr : mp.clustered_rows()) {
+ for (auto&& cr : mp.non_dummy_rows()) {
write_row(clustering_rows.add(), s, cr.key(), cr.row().cells(), cr.row().marker(), cr.row().deleted_at());
}
std::move(clustering_rows).end_rows().end_mutation_partition();
diff --git a/service/storage_proxy.cc b/service/storage_proxy.cc
index b755c69..907b43e 100644
--- a/service/storage_proxy.cc
+++ b/service/storage_proxy.cc
@@ -2040,7 +2040,7 @@ class data_read_resolver : public abstract_read_resolver {
virtual void accept_static_cell(column_id, atomic_cell_view) override { }
virtual void accept_static_cell(column_id, collection_mutation_view) override { }
virtual void accept_row_tombstone(const range_tombstone&) override { }
- virtual void accept_row(clustering_key_view key, const row_tombstone&, const row_marker&) override {
+ virtual void accept_row(clustering_key_view key, const row_tombstone&, const row_marker&, is_dummy, is_continuous) override {
if (!_is_reversed || !_last_ck) {
_last_ck = clustering_key(key);
}
diff --git a/streamed_mutation.cc b/streamed_mutation.cc
index 97da4b7..9ffab21 100644
--- a/streamed_mutation.cc
+++ b/streamed_mutation.cc
@@ -165,10 +165,16 @@ streamed_mutation streamed_mutation_from_mutation(mutation m, streamed_mutation:
private:
void prepare_next_clustering_row() {
auto& crs = _mutation.partition().clustered_rows();
- auto re = crs.unlink_leftmost_without_rebalance();
- if (re) {
+ while (true) {
+ auto re = crs.unlink_leftmost_without_rebalance();
+ if (!re) {
+ break;
+ }
auto re_deleter = defer([re] { current_deleter<rows_entry>()(re); });
- _cr = mutation_fragment(std::move(*re));
+ if (!re->dummy()) {
+ _cr = mutation_fragment(std::move(*re));
+ break;
+ }
}
}
void prepare_next_range_tombstone() {
diff --git a/tests/streamed_mutation_test.cc b/tests/streamed_mutation_test.cc
index a6172dd..bd651ac 100644
--- a/tests/streamed_mutation_test.cc
+++ b/tests/streamed_mutation_test.cc
@@ -240,7 +240,7 @@ SEASTAR_TEST_CASE(test_fragmenting_and_freezing_streamed_mutations) {
return make_ready_future<>();
}, 1).get0();

- auto expected_fragments = m.partition().clustered_rows().calculate_size()
+ auto expected_fragments = boost::size(m.partition().non_dummy_rows())
+ m.partition().row_tombstones().size()
+ !m.partition().static_row().empty();
BOOST_REQUIRE_EQUAL(fms.size(), std::max(expected_fragments, size_t(1)));
--
2.7.4

Tomasz Grabiec

<tgrabiec@scylladb.com>
unread,
Jun 12, 2017, 3:02:09 PM6/12/17
to scylladb-dev@googlegroups.com
assert_that().is_equal_to() gives better error message.

Also, there is code which can be replaces with
assert_that_stream().has_monotonic_positions()
---
tests/row_cache_test.cc | 32 ++++++++------------------------
1 file changed, 8 insertions(+), 24 deletions(-)

diff --git a/tests/row_cache_test.cc b/tests/row_cache_test.cc
index 12fc5f5..ce02cb6 100644
--- a/tests/row_cache_test.cc
+++ b/tests/row_cache_test.cc
@@ -1116,12 +1116,7 @@ SEASTAR_TEST_CASE(test_cache_population_and_clear_race) {

SEASTAR_TEST_CASE(test_mvcc) {
return seastar::async([] {
- auto no_difference = [] (auto& m1, auto& m2) {
- return m1.partition().difference(m1.schema(), m2.partition()).empty()
- && m2.partition().difference(m1.schema(), m1.partition()).empty();
- };
-
- auto test = [&no_difference] (const mutation& m1, const mutation& m2, bool with_active_memtable_reader) {
+ auto test = [&] (const mutation& m1, const mutation& m2, bool with_active_memtable_reader) {
auto s = m1.schema();

auto mt = make_lw_shared<memtable>(s);
@@ -1144,8 +1139,7 @@ SEASTAR_TEST_CASE(test_mvcc) {
auto mt1 = make_lw_shared<memtable>(s);
mt1->apply(m2);

- auto m12 = m1;
- m12.apply(m2);
+ auto m12 = m1 + m2;

stdx::optional<mutation_reader> mt1_reader_opt;
stdx::optional<streamed_mutation_opt> mt1_reader_sm_opt;
@@ -1168,38 +1162,28 @@ SEASTAR_TEST_CASE(test_mvcc) {
BOOST_REQUIRE(sm5);
BOOST_REQUIRE(eq(sm5->key(), pk));

- stdx::optional<position_in_partition> previous;
- position_in_partition::less_compare cmp(*sm3->schema());
- auto mf = (*sm3)().get0();
- while (mf) {
- if (previous) {
- BOOST_REQUIRE(cmp(*previous, mf->position()));
- }
- previous = position_in_partition(mf->position());
- mf = (*sm3)().get0();
- }
- sm3 = { };
+ assert_that_stream(std::move(*sm3)).has_monotonic_positions();

if (with_active_memtable_reader) {
assert(mt1_reader_sm_opt);
auto mt1_reader_mutation = mutation_from_streamed_mutation(std::move(*mt1_reader_sm_opt)).get0();
BOOST_REQUIRE(mt1_reader_mutation);
- BOOST_REQUIRE(no_difference(m2, *mt1_reader_mutation));
+ assert_that(*mt1_reader_mutation).is_equal_to(m2);
}

auto m_4 = mutation_from_streamed_mutation(std::move(sm4)).get0();
- BOOST_REQUIRE(no_difference(m12, *m_4));
+ assert_that(*m_4).is_equal_to(m12);

auto m_1 = mutation_from_streamed_mutation(std::move(sm1)).get0();
- BOOST_REQUIRE(no_difference(m1, *m_1));
+ assert_that(*m_1).is_equal_to(m1);

cache.clear().get0();

auto m_2 = mutation_from_streamed_mutation(std::move(sm2)).get0();
- BOOST_REQUIRE(no_difference(m1, *m_2));
+ assert_that(*m_2).is_equal_to(m1);

auto m_5 = mutation_from_streamed_mutation(std::move(sm5)).get0();
- BOOST_REQUIRE(no_difference(m12, *m_5));
+ assert_that(*m_5).is_equal_to(m12);
};

for_each_mutation_pair([&] (const mutation& m1, const mutation& m2_, are_equal) {
--
2.7.4

Tomasz Grabiec

<tgrabiec@scylladb.com>
unread,
Jun 12, 2017, 3:02:10 PM6/12/17
to scylladb-dev@googlegroups.com, Piotr Jastrzebski
From: Piotr Jastrzebski <pi...@scylladb.com>

[tgrabiec: Extracted from "row_cache: Introduce cache_streamed_mutation"]
---
tests/row_cache_test.cc | 1 +
1 file changed, 1 insertion(+)

diff --git a/tests/row_cache_test.cc b/tests/row_cache_test.cc
index ce02cb6..e5d9db6 100644
--- a/tests/row_cache_test.cc
+++ b/tests/row_cache_test.cc
@@ -1140,6 +1140,7 @@ SEASTAR_TEST_CASE(test_mvcc) {
mt1->apply(m2);

auto m12 = m1 + m2;
+ mt->apply(m2);

stdx::optional<mutation_reader> mt1_reader_opt;
stdx::optional<streamed_mutation_opt> mt1_reader_sm_opt;
--
2.7.4

Tomasz Grabiec

<tgrabiec@scylladb.com>
unread,
Jun 12, 2017, 3:02:10 PM6/12/17
to scylladb-dev@googlegroups.com
Cache currently assumes that mutations coming from outside are fully
continuous.
---
mutation_partition.hh | 1 +
mutation_partition.cc | 13 +++++++++++++
tests/row_cache_test.cc | 12 ++++++++----
3 files changed, 22 insertions(+), 4 deletions(-)

diff --git a/mutation_partition.hh b/mutation_partition.hh
index 690d875..568f0ba 100644
--- a/mutation_partition.hh
+++ b/mutation_partition.hh
@@ -894,6 +894,7 @@ class mutation_partition final {
bool is_static_row_cached() const { return _static_row_cached; }
void set_static_row_cached(bool value) { _static_row_cached = value; }
bool is_fully_continuous() const;
+ void make_fully_continuous();
void apply(tombstone t) { _tombstone.apply(t); }
void apply_delete(const schema& schema, const clustering_key_prefix& prefix, tombstone t);
void apply_delete(const schema& schema, range_tombstone rt);
diff --git a/mutation_partition.cc b/mutation_partition.cc
index ffbc545..2c3c268 100644
--- a/mutation_partition.cc
+++ b/mutation_partition.cc
@@ -2116,6 +2116,19 @@ bool mutation_partition::is_fully_continuous() const {
return true;
}

+void mutation_partition::make_fully_continuous() {
+ _static_row_cached = true;
+ auto i = _rows.begin();
+ while (i != _rows.end()) {
+ if (i->dummy()) {
+ i = _rows.erase_and_dispose(i, alloc_strategy_deleter<rows_entry>());
+ } else {
+ i->set_continuous(true);
+ ++i;
+ }
+ }
+}
+
future<mutation_opt> counter_write_query(schema_ptr s, const mutation_source& source,
const dht::decorated_key& dk,
const query::partition_slice& slice,
diff --git a/tests/row_cache_test.cc b/tests/row_cache_test.cc
index e5d9db6..a94d05c 100644
--- a/tests/row_cache_test.cc
+++ b/tests/row_cache_test.cc
@@ -1187,17 +1187,21 @@ SEASTAR_TEST_CASE(test_mvcc) {
assert_that(*m_5).is_equal_to(m12);
};

- for_each_mutation_pair([&] (const mutation& m1, const mutation& m2_, are_equal) {
- if (m1.schema() != m2_.schema()) {
+ for_each_mutation_pair([&] (const mutation& m1_, const mutation& m2_, are_equal) {
+ if (m1_.schema() != m2_.schema()) {
return;
}
- if (m1.partition().empty() || m2_.partition().empty()) {
+ if (m1_.partition().empty() || m2_.partition().empty()) {
return;
}
- auto s = m1.schema();
+ auto s = m1_.schema();
+
+ auto m1 = m1_;
+ m1.partition().make_fully_continuous();

auto m2 = mutation(m1.decorated_key(), m1.schema());
m2.partition().apply(*s, m2_.partition(), *s);
+ m2.partition().make_fully_continuous();

test(m1, m2, false);
test(m1, m2, true);
--
2.7.4

Tomasz Grabiec

<tgrabiec@scylladb.com>
unread,
Jun 12, 2017, 3:02:12 PM6/12/17
to scylladb-dev@googlegroups.com
---
tests/mutation_source_test.cc | 18 +++++++++++++++---
1 file changed, 15 insertions(+), 3 deletions(-)

diff --git a/tests/mutation_source_test.cc b/tests/mutation_source_test.cc
index 16eabde..4b0f734 100644
--- a/tests/mutation_source_test.cc
+++ b/tests/mutation_source_test.cc
@@ -941,6 +941,7 @@ class random_mutation_generator::impl {
std::vector<bytes> _blobs;
std::uniform_int_distribution<size_t> _ck_index_dist{0, n_blobs - 1};
std::uniform_int_distribution<int> _bool_dist{0, 1};
+ std::uniform_int_distribution<int> _not_dummy_dist{0, 19};

template <typename Generator>
static gc_clock::time_point expiry_dist(Generator& gen) {
@@ -1171,9 +1172,14 @@ class random_mutation_generator::impl {
size_t row_count = row_count_dist(_gen);
for (size_t i = 0; i < row_count; ++i) {
auto ckey = make_random_key();
- deletable_row& row = m.partition().clustered_row(*_schema, ckey);
- set_random_cells(row.cells(), column_kind::regular_column);
- row.marker() = random_row_marker();
+ is_continuous continuous = is_continuous(_bool_dist(_gen));
+ if (_not_dummy_dist(_gen)) {
+ deletable_row& row = m.partition().clustered_row(*_schema, ckey, is_dummy::no, continuous);
+ set_random_cells(row.cells(), column_kind::regular_column);
+ row.marker() = random_row_marker();
+ } else {
+ m.partition().clustered_row(*_schema, ckey, is_dummy::yes, continuous);
+ }
}

size_t range_tombstone_count = row_count_dist(_gen);
@@ -1187,6 +1193,12 @@ class random_mutation_generator::impl {
m.partition().apply_row_tombstone(*_schema,
range_tombstone(std::move(start), std::move(end), random_tombstone()));
}
+
+ if (_bool_dist(_gen)) {
+ m.partition().ensure_last_dummy(*_schema);
+ m.partition().clustered_rows().rbegin()->set_continuous(is_continuous(_bool_dist(_gen)));
+ }
+
return m;
}
};
--
2.7.4

Tomasz Grabiec

<tgrabiec@scylladb.com>
unread,
Jun 12, 2017, 3:02:12 PM6/12/17
to scylladb-dev@googlegroups.com
---
tests/mutation_source_test.cc | 9 ++++++++-
1 file changed, 8 insertions(+), 1 deletion(-)

diff --git a/tests/mutation_source_test.cc b/tests/mutation_source_test.cc
index a523fe1..16eabde 100644
--- a/tests/mutation_source_test.cc
+++ b/tests/mutation_source_test.cc
@@ -761,7 +761,6 @@ static mutation_sets generate_mutation_sets() {

auto m1 = mutation(partition_key::from_single_value(*s1, to_bytes("key1")), s1);
auto m2 = mutation(partition_key::from_single_value(*s2, to_bytes("key1")), s2);
-
result.equal.emplace_back(mutations{m1, m2});

clustering_key ck1 = clustering_key::from_deeply_exploded(*s1, {data_value(bytes("ck1_0")), data_value(bytes("ck1_1"))});
@@ -842,6 +841,14 @@ static mutation_sets generate_mutation_sets() {
}

{
+ m1.partition().ensure_last_dummy(*m1.schema());
+ result.equal.emplace_back(mutations{m1, m2});
+
+ m2.partition().ensure_last_dummy(*m2.schema());
+ result.equal.emplace_back(mutations{m1, m2});
+ }
+
+ {
auto ts = new_timestamp();
m1.set_clustered_cell(ck2, "regular_col_1_s1", data_value(bytes("x")), ts);
result.unequal.emplace_back(mutations{m1, m2});
--
2.7.4

Tomasz Grabiec

<tgrabiec@scylladb.com>
unread,
Jun 12, 2017, 3:02:13 PM6/12/17
to scylladb-dev@googlegroups.com
---
tests/mutation_assertions.hh | 7 +++++
tests/mutation_test.cc | 72 ++++++++++++++++++++++++++++++++++++++++++--
2 files changed, 77 insertions(+), 2 deletions(-)

diff --git a/tests/mutation_assertions.hh b/tests/mutation_assertions.hh
index 48cbbb4..41fd7d6 100644
--- a/tests/mutation_assertions.hh
+++ b/tests/mutation_assertions.hh
@@ -54,6 +54,13 @@ class mutation_assertion {
return *this;
}

+ mutation_assertion& has_same_continuity(const mutation& other) {
+ if (!_m.partition().equal_continuity(*_m.schema(), other.partition())) {
+ BOOST_FAIL(sprint("Continuity doesn't match: %s\n ...and: %s", other, _m));
+ }
+ return *this;
+ }
+
// Verifies that mutation data remains unchanged when upgraded to the new schema
void is_upgrade_equivalent(schema_ptr new_schema) {
mutation m2 = _m;
diff --git a/tests/mutation_test.cc b/tests/mutation_test.cc
index e20014c..9e70559 100644
--- a/tests/mutation_test.cc
+++ b/tests/mutation_test.cc
@@ -48,6 +48,7 @@
#include "cell_locking.hh"

#include "disk-error-handler.hh"
+#include "simple_schema.hh"

thread_local disk_error_signal_type commit_error;
thread_local disk_error_signal_type general_disk_error;
@@ -830,7 +831,8 @@ SEASTAR_TEST_CASE(test_apply_is_atomic_in_case_of_allocation_failures) {
break; // we exhausted all allocation points
} catch (const std::bad_alloc&) {
BOOST_TEST_MESSAGE("Checking that apply was reverted");
- assert_that(m).is_equal_to(target);
+ assert_that(m).is_equal_to(target)
+ .has_same_continuity(target);
}
}
}
@@ -851,7 +853,8 @@ SEASTAR_TEST_CASE(test_apply_is_atomic_in_case_of_allocation_failures) {
assert_that(m).is_equal_to(target);
// they should still commute
m.apply(copy_of_second);
- assert_that(m).is_equal_to(expected_apply_result);
+ assert_that(m).is_equal_to(expected_apply_result)
+ .has_same_continuity(expected_apply_result);
}
}
}
@@ -1513,3 +1516,68 @@ SEASTAR_TEST_CASE(test_mutation_diff_with_random_generator) {
});
});
}
+
+SEASTAR_TEST_CASE(test_continuity_merging) {
+ return seastar::async([] {
+ simple_schema table;
+ auto&& s = *table.schema();
+
+ auto new_mutation = [&] {
+ return mutation(table.make_pkey(0), table.schema());
+ };
+
+ {
+ auto left = new_mutation();
+ auto right = new_mutation();
+ auto result = new_mutation();
+
+ left.partition().clustered_row(s, table.make_ckey(0), is_dummy::no, is_continuous::yes);
+ right.partition().clustered_row(s, table.make_ckey(0), is_dummy::no, is_continuous::no);
+ result.partition().clustered_row(s, table.make_ckey(0), is_dummy::no, is_continuous::yes);
+
+ left.partition().clustered_row(s, table.make_ckey(1), is_dummy::yes, is_continuous::yes);
+ right.partition().clustered_row(s, table.make_ckey(2), is_dummy::yes, is_continuous::no);
+ result.partition().clustered_row(s, table.make_ckey(1), is_dummy::yes, is_continuous::yes);
+ result.partition().clustered_row(s, table.make_ckey(2), is_dummy::yes, is_continuous::no);
+
+ left.partition().clustered_row(s, table.make_ckey(3), is_dummy::yes, is_continuous::yes);
+ right.partition().clustered_row(s, table.make_ckey(3), is_dummy::no, is_continuous::no);
+ result.partition().clustered_row(s, table.make_ckey(3), is_dummy::yes, is_continuous::yes);
+
+ left.partition().clustered_row(s, table.make_ckey(4), is_dummy::no, is_continuous::no);
+ right.partition().clustered_row(s, table.make_ckey(4), is_dummy::no, is_continuous::yes);
+ result.partition().clustered_row(s, table.make_ckey(4), is_dummy::no, is_continuous::no);
+
+ left.partition().clustered_row(s, table.make_ckey(5), is_dummy::no, is_continuous::no);
+ right.partition().clustered_row(s, table.make_ckey(5), is_dummy::yes, is_continuous::yes);
+ result.partition().clustered_row(s, table.make_ckey(5), is_dummy::no, is_continuous::no);
+
+ left.partition().clustered_row(s, table.make_ckey(6), is_dummy::no, is_continuous::yes);
+ right.partition().clustered_row(s, table.make_ckey(6), is_dummy::yes, is_continuous::no);
+ result.partition().clustered_row(s, table.make_ckey(6), is_dummy::no, is_continuous::yes);
+
+ left.partition().clustered_row(s, table.make_ckey(7), is_dummy::yes, is_continuous::yes);
+ right.partition().clustered_row(s, table.make_ckey(7), is_dummy::yes, is_continuous::no);
+ result.partition().clustered_row(s, table.make_ckey(7), is_dummy::yes, is_continuous::yes);
+
+ left.partition().clustered_row(s, table.make_ckey(8), is_dummy::yes, is_continuous::no);
+ right.partition().clustered_row(s, table.make_ckey(8), is_dummy::yes, is_continuous::yes);
+ result.partition().clustered_row(s, table.make_ckey(8), is_dummy::yes, is_continuous::no);
+
+ assert_that(left + right).has_same_continuity(result);
+ }
+
+ // static row continuity
+ {
+ auto complete = mutation(table.make_pkey(0), table.schema());
+ auto incomplete = mutation(table.make_pkey(0), table.schema());
+ incomplete.partition().set_static_row_cached(false);
+
+ assert_that(complete + complete).has_same_continuity(complete);
+ assert_that(complete + incomplete).has_same_continuity(complete);
+ assert_that(incomplete + complete).has_same_continuity(incomplete);
+ assert_that(incomplete + incomplete).has_same_continuity(incomplete);
+ }
+ });
+
+}
--
2.7.4

Tomasz Grabiec

<tgrabiec@scylladb.com>
unread,
Jun 12, 2017, 3:02:14 PM6/12/17
to scylladb-dev@googlegroups.com, Piotr Jastrzebski
From: Piotr Jastrzebski <pi...@scylladb.com>

Remove unused parameter and add checks to make sure
all expected rows have been received.

Signed-off-by: Piotr Jastrzebski <pi...@scylladb.com>
---
tests/row_cache_test.cc | 13 +++++++------
1 file changed, 7 insertions(+), 6 deletions(-)

diff --git a/tests/row_cache_test.cc b/tests/row_cache_test.cc
index a94d05c..03a35f3 100644
--- a/tests/row_cache_test.cc
+++ b/tests/row_cache_test.cc
@@ -1209,7 +1209,7 @@ SEASTAR_TEST_CASE(test_mvcc) {
});
}

-void test_sliced_read_row_presence(mutation_reader reader, schema_ptr s, const query::partition_slice& ps, std::deque<int> expected)
+void test_sliced_read_row_presence(mutation_reader reader, schema_ptr s, std::deque<int> expected)
{
clustering_key::equality ck_eq(*s);

@@ -1218,13 +1218,14 @@ void test_sliced_read_row_presence(mutation_reader reader, schema_ptr s, const q
auto mfopt = (*smopt)().get0();
while (mfopt) {
if (mfopt->is_clustering_row()) {
+ BOOST_REQUIRE(!expected.empty());
auto& cr = mfopt->as_clustering_row();
BOOST_REQUIRE(ck_eq(cr.key(), clustering_key_prefix::from_single_value(*s, int32_type->decompose(expected.front()))));
expected.pop_front();
}
mfopt = (*smopt)().get0();
}
-
+ BOOST_REQUIRE(expected.empty());
BOOST_REQUIRE(!reader().get0());
}

@@ -1254,21 +1255,21 @@ SEASTAR_TEST_CASE(test_slicing_mutation_reader) {
cache.clear().get0();

auto reader = cache.make_reader(s, query::full_partition_range, ps);
- test_sliced_read_row_presence(std::move(reader), s, ps, expected);
+ test_sliced_read_row_presence(std::move(reader), s, expected);

reader = cache.make_reader(s, query::full_partition_range, ps);
- test_sliced_read_row_presence(std::move(reader), s, ps, expected);
+ test_sliced_read_row_presence(std::move(reader), s, expected);

auto dk = dht::global_partitioner().decorate_key(*s, pk);
auto singular_range = dht::partition_range::make_singular(dk);

reader = cache.make_reader(s, singular_range, ps);
- test_sliced_read_row_presence(std::move(reader), s, ps, expected);
+ test_sliced_read_row_presence(std::move(reader), s, expected);

cache.clear().get0();

reader = cache.make_reader(s, singular_range, ps);
- test_sliced_read_row_presence(std::move(reader), s, ps, expected);
+ test_sliced_read_row_presence(std::move(reader), s, expected);
};

{
--
2.7.4

Tomasz Grabiec

<tgrabiec@scylladb.com>
unread,
Jun 12, 2017, 3:02:15 PM6/12/17
to scylladb-dev@googlegroups.com, Piotr Jastrzebski
From: Piotr Jastrzebski <pi...@scylladb.com>

By default make_reader_returning creates a reader that does not
support fast forwarding but the second parameter can be used to
make it support fast forwarding.

[tgrabiec: Improve title]

Signed-off-by: Piotr Jastrzebski <pi...@scylladb.com>
---
mutation_reader.hh | 2 +-
mutation_reader.cc | 4 ++--
2 files changed, 3 insertions(+), 3 deletions(-)

diff --git a/mutation_reader.hh b/mutation_reader.hh
index 46ee21e..1cdd956 100644
--- a/mutation_reader.hh
+++ b/mutation_reader.hh
@@ -145,7 +145,7 @@ class combined_mutation_reader : public mutation_reader::impl {
mutation_reader make_combined_reader(std::vector<mutation_reader>);
mutation_reader make_combined_reader(mutation_reader&& a, mutation_reader&& b);
// reads from the input readers, in order
-mutation_reader make_reader_returning(mutation);
+mutation_reader make_reader_returning(mutation, streamed_mutation::forwarding fwd = streamed_mutation::forwarding::no);
mutation_reader make_reader_returning(streamed_mutation);
mutation_reader make_reader_returning_many(std::vector<mutation>,
const query::partition_slice& slice = query::full_slice,
diff --git a/mutation_reader.cc b/mutation_reader.cc
index 584c3f5..33c3070 100644
--- a/mutation_reader.cc
+++ b/mutation_reader.cc
@@ -153,8 +153,8 @@ class reader_returning final : public mutation_reader::impl {
}
};

-mutation_reader make_reader_returning(mutation m) {
- return make_mutation_reader<reader_returning>(streamed_mutation_from_mutation(std::move(m)));
+mutation_reader make_reader_returning(mutation m, streamed_mutation::forwarding fwd) {
+ return make_mutation_reader<reader_returning>(streamed_mutation_from_mutation(std::move(m), std::move(fwd)));
}

mutation_reader make_reader_returning(streamed_mutation m) {
--
2.7.4

Tomasz Grabiec

<tgrabiec@scylladb.com>
unread,
Jun 12, 2017, 3:02:16 PM6/12/17
to scylladb-dev@googlegroups.com, Piotr Jastrzebski
From: Piotr Jastrzebski <pi...@scylladb.com>

mutation source sometimes ignore fast forwarding parameter so
this change adds assertion to check that this parameter
can be safely ignored.

Signed-off-by: Piotr Jastrzebski <pi...@scylladb.com>
---
mutation_reader.hh | 9 ++++++---
tests/row_cache_test.cc | 44 +++++++++++++++++++++++++-------------------
2 files changed, 31 insertions(+), 22 deletions(-)

diff --git a/mutation_reader.hh b/mutation_reader.hh
index 1cdd956..9d4e26c 100644
--- a/mutation_reader.hh
+++ b/mutation_reader.hh
@@ -272,15 +272,18 @@ class mutation_source {
public:
mutation_source(func_type fn) : _fn(std::make_unique<func_type>(std::move(fn))) {}
mutation_source(std::function<mutation_reader(schema_ptr, partition_range, const query::partition_slice&, io_priority)> fn)
- : _fn(std::make_unique<func_type>([fn = std::move(fn)] (schema_ptr s, partition_range range, const query::partition_slice& slice, io_priority pc, tracing::trace_state_ptr, streamed_mutation::forwarding) {
+ : _fn(std::make_unique<func_type>([fn = std::move(fn)] (schema_ptr s, partition_range range, const query::partition_slice& slice, io_priority pc, tracing::trace_state_ptr, streamed_mutation::forwarding fwd) {
+ assert(!fwd);
return fn(s, range, slice, pc);
})) {}
mutation_source(std::function<mutation_reader(schema_ptr, partition_range, const query::partition_slice&)> fn)
- : _fn(std::make_unique<func_type>([fn = std::move(fn)] (schema_ptr s, partition_range range, const query::partition_slice& slice, io_priority, tracing::trace_state_ptr, streamed_mutation::forwarding) {
+ : _fn(std::make_unique<func_type>([fn = std::move(fn)] (schema_ptr s, partition_range range, const query::partition_slice& slice, io_priority, tracing::trace_state_ptr, streamed_mutation::forwarding fwd) {
+ assert(!fwd);
return fn(s, range, slice);
})) {}
mutation_source(std::function<mutation_reader(schema_ptr, partition_range range)> fn)
- : _fn(std::make_unique<func_type>([fn = std::move(fn)] (schema_ptr s, partition_range range, const query::partition_slice&, io_priority, tracing::trace_state_ptr, streamed_mutation::forwarding) {
+ : _fn(std::make_unique<func_type>([fn = std::move(fn)] (schema_ptr s, partition_range range, const query::partition_slice&, io_priority, tracing::trace_state_ptr, streamed_mutation::forwarding fwd) {
+ assert(!fwd);
return fn(s, range);
})) {}

diff --git a/tests/row_cache_test.cc b/tests/row_cache_test.cc
index 03a35f3..e1d708f 100644
--- a/tests/row_cache_test.cc
+++ b/tests/row_cache_test.cc
@@ -101,9 +101,9 @@ SEASTAR_TEST_CASE(test_cache_delegates_to_underlying) {
auto m = make_new_mutation(s);

cache_tracker tracker;
- row_cache cache(s, mutation_source([m] (schema_ptr s, const dht::partition_range&) {
+ row_cache cache(s, mutation_source([m] (schema_ptr s, const dht::partition_range&, const query::partition_slice&, const io_priority_class&, tracing::trace_state_ptr, streamed_mutation::forwarding fwd) {
assert(m.schema() == s);
- return make_reader_returning(m);
+ return make_reader_returning(m, std::move(fwd));
}), tracker);

assert_that(cache.make_reader(s, query::full_partition_range))
@@ -118,9 +118,9 @@ SEASTAR_TEST_CASE(test_cache_works_after_clearing) {
auto m = make_new_mutation(s);

cache_tracker tracker;
- row_cache cache(s, mutation_source([m] (schema_ptr s, const dht::partition_range&) {
+ row_cache cache(s, mutation_source([m] (schema_ptr s, const dht::partition_range&, const query::partition_slice&, const io_priority_class&, tracing::trace_state_ptr, streamed_mutation::forwarding fwd) {
assert(m.schema() == s);
- return make_reader_returning(m);
+ return make_reader_returning(m, std::move(fwd));
}), tracker);

assert_that(cache.make_reader(s, query::full_partition_range))
@@ -161,7 +161,7 @@ SEASTAR_TEST_CASE(test_cache_delegates_to_underlying_only_once_empty_full_range)
auto s = make_schema();
int secondary_calls_count = 0;
cache_tracker tracker;
- row_cache cache(s, mutation_source([&secondary_calls_count] (schema_ptr s, const dht::partition_range& range) {
+ row_cache cache(s, mutation_source([&secondary_calls_count] (schema_ptr s, const dht::partition_range& range, const query::partition_slice&, const io_priority_class&, tracing::trace_state_ptr, streamed_mutation::forwarding fwd) {
return make_counting_reader(make_empty_reader(), secondary_calls_count);
}), tracker);

@@ -179,10 +179,10 @@ void test_cache_delegates_to_underlying_only_once_with_single_partition(schema_p
const dht::partition_range& range) {
int secondary_calls_count = 0;
cache_tracker tracker;
- row_cache cache(s, mutation_source([m, &secondary_calls_count] (schema_ptr s, const dht::partition_range& range) {
+ row_cache cache(s, mutation_source([m, &secondary_calls_count] (schema_ptr s, const dht::partition_range& range, const query::partition_slice&, const io_priority_class&, tracing::trace_state_ptr, streamed_mutation::forwarding fwd) {
assert(m.schema() == s);
if (range.contains(dht::ring_position(m.decorated_key()), dht::ring_position_comparator(*s))) {
- return make_counting_reader(make_reader_returning(m), secondary_calls_count);
+ return make_counting_reader(make_reader_returning(m, std::move(fwd)), secondary_calls_count);
} else {
return make_counting_reader(make_empty_reader(), secondary_calls_count);
}
@@ -275,8 +275,9 @@ SEASTAR_TEST_CASE(test_cache_delegates_to_underlying_only_once_multiple_mutation
}

auto make_cache = [&tracker, &mt](schema_ptr s, int& secondary_calls_count) -> lw_shared_ptr<row_cache> {
- auto secondary = mutation_source([&mt, &secondary_calls_count] (schema_ptr s, const dht::partition_range& range) {
- return make_counting_reader(mt->as_data_source()(s, range), secondary_calls_count);
+ auto secondary = mutation_source([&mt, &secondary_calls_count] (schema_ptr s, const dht::partition_range& range,
+ const query::partition_slice& slice, const io_priority_class& pc, tracing::trace_state_ptr trace, streamed_mutation::forwarding fwd) {
+ return make_counting_reader(mt->as_data_source()(s, range, slice, pc, std::move(trace), std::move(fwd)), secondary_calls_count);
});

return make_lw_shared<row_cache>(s, secondary, tracker);
@@ -284,8 +285,9 @@ SEASTAR_TEST_CASE(test_cache_delegates_to_underlying_only_once_multiple_mutation

auto make_ds = [&make_cache](schema_ptr s, int& secondary_calls_count) -> mutation_source {
auto cache = make_cache(s, secondary_calls_count);
- return mutation_source([cache] (schema_ptr s, const dht::partition_range& range) {
- return cache->make_reader(s, range);
+ return mutation_source([cache] (schema_ptr s, const dht::partition_range& range,
+ const query::partition_slice& slice, const io_priority_class& pc, tracing::trace_state_ptr trace, streamed_mutation::forwarding fwd) {
+ return cache->make_reader(s, range, slice, pc, std::move(trace), std::move(fwd));
});
};

@@ -408,8 +410,9 @@ SEASTAR_TEST_CASE(test_cache_delegates_to_underlying_only_once_multiple_mutation
};

auto cache = make_cache(s, secondary_calls_count);
- auto ds = mutation_source([cache] (schema_ptr s, const dht::partition_range& range) {
- return cache->make_reader(s, range);
+ auto ds = mutation_source([cache] (schema_ptr s, const dht::partition_range& range,
+ const query::partition_slice& slice, const io_priority_class& pc, tracing::trace_state_ptr trace, streamed_mutation::forwarding fwd) {
+ return cache->make_reader(s, range, slice, pc, std::move(trace), std::move(fwd));
});

test(ds, query::full_partition_range, partitions.size() + 1);
@@ -820,8 +823,9 @@ class throttled_mutation_source {
: _underlying(std::move(underlying))
{ }

- mutation_reader make_reader(schema_ptr s, const dht::partition_range& pr) {
- return make_mutation_reader<reader>(_throttle, _underlying(s, pr));
+ mutation_reader make_reader(schema_ptr s, const dht::partition_range& pr,
+ const query::partition_slice& slice, const io_priority_class& pc, tracing::trace_state_ptr trace, streamed_mutation::forwarding fwd) {
+ return make_mutation_reader<reader>(_throttle, _underlying(s, pr, slice, pc, std::move(trace), std::move(fwd)));
}

::throttle& throttle() { return _throttle; }
@@ -841,8 +845,9 @@ class throttled_mutation_source {
}

operator mutation_source() const {
- return mutation_source([this] (schema_ptr s, const dht::partition_range& pr) {
- return _impl->make_reader(std::move(s), pr);
+ return mutation_source([this] (schema_ptr s, const dht::partition_range& pr,
+ const query::partition_slice& slice, const io_priority_class& pc, tracing::trace_state_ptr trace, streamed_mutation::forwarding fwd) {
+ return _impl->make_reader(std::move(s), pr, slice, pc, std::move(trace), std::move(fwd));
});
}
};
@@ -856,10 +861,11 @@ static std::vector<mutation> updated_ring(std::vector<mutation>& mutations) {
}

static mutation_source make_mutation_source(std::vector<lw_shared_ptr<memtable>>& memtables) {
- return mutation_source([&memtables] (schema_ptr s, const dht::partition_range& pr) {
+ return mutation_source([&memtables] (schema_ptr s, const dht::partition_range& pr,
+ const query::partition_slice& slice, const io_priority_class& pc, tracing::trace_state_ptr trace, streamed_mutation::forwarding fwd) {
std::vector<mutation_reader> readers;
for (auto&& mt : memtables) {
- readers.emplace_back(mt->make_reader(s, pr));
+ readers.emplace_back(mt->make_reader(s, pr, slice, pc, trace, fwd));
}
return make_combined_reader(std::move(readers));
});
--
2.7.4

Tomasz Grabiec

<tgrabiec@scylladb.com>
unread,
Jun 12, 2017, 3:02:17 PM6/12/17
to scylladb-dev@googlegroups.com, Piotr Jastrzebski
From: Piotr Jastrzebski <pi...@scylladb.com>

---
partition_version.hh | 4 ++++
1 file changed, 4 insertions(+)

diff --git a/partition_version.hh b/partition_version.hh
index 66b74a4..0a6b830 100644
--- a/partition_version.hh
+++ b/partition_version.hh
@@ -241,6 +241,10 @@ class partition_entry {
return *this;
}

+ partition_version_ref& version() {
+ return _version;
+ }
+
// Strong exception guarantees.
void apply(const schema& s, const mutation_partition& mp, const schema& mp_schema);

--
2.7.4

Tomasz Grabiec

<tgrabiec@scylladb.com>
unread,
Jun 12, 2017, 3:02:19 PM6/12/17
to scylladb-dev@googlegroups.com, Piotr Jastrzebski
From: Piotr Jastrzebski <pi...@scylladb.com>

This will be used in the following patches by partial cache.

Signed-off-by: Piotr Jastrzebski <pi...@scylladb.com>
---
partition_snapshot_reader.hh | 36 +++++++++++++++++++++---------------
1 file changed, 21 insertions(+), 15 deletions(-)

diff --git a/partition_snapshot_reader.hh b/partition_snapshot_reader.hh
index dcbdca3..a144538 100644
--- a/partition_snapshot_reader.hh
+++ b/partition_snapshot_reader.hh
@@ -30,6 +30,26 @@ struct partition_snapshot_reader_dummy_accounter {
};
extern partition_snapshot_reader_dummy_accounter no_accounter;

+inline void maybe_merge_versions(lw_shared_ptr<partition_snapshot>& snp,
+ logalloc::region& lsa_region,
+ logalloc::allocating_section& read_section) {
+ if (!snp.owned()) {
+ return;
+ }
+ // If no one else is using this particular snapshot try to merge partition
+ // versions.
+ with_allocator(lsa_region.allocator(), [&snp, &lsa_region, &read_section] {
+ return with_linearized_managed_bytes([&snp, &lsa_region, &read_section] {
+ try {
+ read_section(lsa_region, [&snp] {
+ snp->merge_partition_versions();
+ });
+ } catch (...) { }
+ snp = {};
+ });
+ });
+}
+
template <typename MemoryAccounter = partition_snapshot_reader_dummy_accounter>
class partition_snapshot_reader : public streamed_mutation::impl, public MemoryAccounter {
struct rows_position {
@@ -227,21 +247,7 @@ class partition_snapshot_reader : public streamed_mutation::impl, public MemoryA
}

~partition_snapshot_reader() {
- if (!_snapshot.owned()) {
- return;
- }
- // If no one else is using this particular snapshot try to merge partition
- // versions.
- with_allocator(_lsa_region.allocator(), [this] {
- return with_linearized_managed_bytes([this] {
- try {
- _read_section(_lsa_region, [this] {
- _snapshot->merge_partition_versions();
- });
- } catch (...) { }
- _snapshot = {};
- });
- });
+ maybe_merge_versions(_snapshot, _lsa_region, _read_section);
}

virtual future<> fill_buffer() override {
--
2.7.4

Tomasz Grabiec

<tgrabiec@scylladb.com>
unread,
Jun 12, 2017, 3:02:20 PM6/12/17
to scylladb-dev@googlegroups.com, Piotr Jastrzebski
From: Piotr Jastrzebski <pi...@scylladb.com>

This will be used by partial cache in later patches.

[tgrabiec: Chenged title,
documented meaning of the variable,
renamed the variable]

Signed-off-by: Piotr Jastrzebski <pi...@scylladb.com>
---
partition_version.hh | 16 +++++++++++++---
partition_version.cc | 18 ++++++++++++------
2 files changed, 25 insertions(+), 9 deletions(-)

diff --git a/partition_version.hh b/partition_version.hh
index 0a6b830..1d9beec 100644
--- a/partition_version.hh
+++ b/partition_version.hh
@@ -180,15 +180,23 @@ class partition_version_ref {
class partition_entry;

class partition_snapshot : public enable_lw_shared_from_this<partition_snapshot> {
+public:
+ // Only snapshots created with the same value of phase can point to the same version.
+ using phase_type = uint64_t;
+ static constexpr phase_type default_phase = 0;
+private:
schema_ptr _schema;
// Either _version or _entry is non-null.
partition_version_ref _version;
partition_entry* _entry;
+ phase_type _phase;

friend class partition_entry;
public:
- explicit partition_snapshot(schema_ptr s, partition_entry* entry)
- : _schema(std::move(s)), _entry(entry) { }
+ explicit partition_snapshot(schema_ptr s,
+ partition_entry* entry,
+ phase_type phase = default_phase)
+ : _schema(std::move(s)), _entry(entry), _phase(phase) { }
partition_snapshot(const partition_snapshot&) = delete;
partition_snapshot(partition_snapshot&&) = delete;
partition_snapshot& operator=(const partition_snapshot&) = delete;
@@ -266,7 +274,9 @@ class partition_entry {
// needs to be called with reclaiming disabled
void upgrade(schema_ptr from, schema_ptr to);

- lw_shared_ptr<partition_snapshot> read(schema_ptr entry_schema);
+ // Snapshots with different values of phase will point to different partition_version objects.
+ lw_shared_ptr<partition_snapshot> read(schema_ptr entry_schema,
+ partition_snapshot::phase_type phase = partition_snapshot::default_phase);
};

inline partition_version_ref& partition_snapshot::version()
diff --git a/partition_version.cc b/partition_version.cc
index 6cbb0eb..48fd4ec 100644
--- a/partition_version.cc
+++ b/partition_version.cc
@@ -282,13 +282,19 @@ void partition_entry::upgrade(schema_ptr from, schema_ptr to)
remove_or_mark_as_unique_owner(old_version);
}

-lw_shared_ptr<partition_snapshot> partition_entry::read(schema_ptr entry_schema)
+lw_shared_ptr<partition_snapshot>
+partition_entry::read(schema_ptr entry_schema, partition_snapshot::phase_type phase)
{
if (_snapshot) {
- return _snapshot->shared_from_this();
- } else {
- auto snp = make_lw_shared<partition_snapshot>(entry_schema, this);
- _snapshot = snp.get();
- return snp;
+ if (_snapshot->_phase == phase) {
+ return _snapshot->shared_from_this();
+ } else {
+ auto new_version = current_allocator().construct<partition_version>(mutation_partition(entry_schema));
+ new_version->insert_before(*_version);
+ set_version(new_version);
+ }
}
+ auto snp = make_lw_shared<partition_snapshot>(entry_schema, this, phase);
+ _snapshot = snp.get();
+ return snp;
}
--
2.7.4

Tomasz Grabiec

<tgrabiec@scylladb.com>
unread,
Jun 12, 2017, 3:02:21 PM6/12/17
to scylladb-dev@googlegroups.com
---
mutation_partition.hh | 4 ++++
mutation_partition.cc | 6 ++++++
2 files changed, 10 insertions(+)

diff --git a/mutation_partition.hh b/mutation_partition.hh
index 568f0ba..c04915a 100644
--- a/mutation_partition.hh
+++ b/mutation_partition.hh
@@ -625,6 +625,10 @@ class deletable_row final {
void apply_reversibly(const schema& s, deletable_row& src);
// See reversibly_mergeable.hh
void revert(const schema& s, deletable_row& src);
+
+ // Weak exception guarantees. After exception, both src and this will commute to the same value as
+ // they would should the exception not happen.
+ void apply(const schema& s, deletable_row&& src);
public:
row_tombstone deleted_at() const { return _deleted_at; }
api::timestamp_type created_at() const { return _marker.timestamp(); }
diff --git a/mutation_partition.cc b/mutation_partition.cc
index 2c3c268..8bd80aa 100644
--- a/mutation_partition.cc
+++ b/mutation_partition.cc
@@ -907,6 +907,12 @@ void deletable_row::revert(const schema& s, deletable_row& src) {
_marker.revert(src._marker);
}

+void deletable_row::apply(const schema& s, deletable_row&& src) {
+ _cells.apply(s, column_kind::regular_column, std::move(src._cells));
+ _marker.apply(src._marker);
+ _deleted_at.apply(src._deleted_at, _marker);
+}
+
bool
rows_entry::equal(const schema& s, const rows_entry& other) const {
return equal(s, other, s);
--
2.7.4

Tomasz Grabiec

<tgrabiec@scylladb.com>
unread,
Jun 12, 2017, 3:02:22 PM6/12/17
to scylladb-dev@googlegroups.com
---
partition_version.hh | 5 +++++
partition_version.cc | 26 ++++++++++++++++++++++++++
2 files changed, 31 insertions(+)

diff --git a/partition_version.hh b/partition_version.hh
index 1d9beec..740d755 100644
--- a/partition_version.hh
+++ b/partition_version.hh
@@ -225,6 +225,11 @@ class partition_entry {
friend class partition_snapshot;
friend class cache_entry;
private:
+ // Detaches all versions temporarily around execution of the function.
+ // The function receives partition_version* pointing to the latest version.
+ template<typename Func>
+ void with_detached_versions(Func&&);
+
void set_version(partition_version*);

void apply(const schema& s, partition_version* pv, const schema& pv_schema);
diff --git a/partition_version.cc b/partition_version.cc
index 48fd4ec..1cbb993 100644
--- a/partition_version.cc
+++ b/partition_version.cc
@@ -20,6 +20,7 @@
*/

#include <boost/range/algorithm/heap_algorithm.hpp>
+#include <seastar/util/defer.hh>

#include "partition_version.hh"

@@ -256,6 +257,31 @@ void partition_entry::apply(const schema& s, partition_entry&& pe, const schema&
}
}

+
+template<typename Func>
+void partition_entry::with_detached_versions(Func&& func) {
+ partition_version* current = &*_version;
+ auto snapshot = _snapshot;
+ if (snapshot) {
+ snapshot->_version = std::move(_version);
+ snapshot->_entry = nullptr;
+ _snapshot = nullptr;
+ }
+ _version = { };
+
+ auto revert = defer([&] {
+ if (snapshot) {
+ _snapshot = snapshot;
+ snapshot->_entry = this;
+ _version = std::move(snapshot->_version);
+ } else {
+ _version = partition_version_ref(*current);
+ }
+ });
+
+ func(current);
+}
+
mutation_partition partition_entry::squashed(schema_ptr from, schema_ptr to)
{
mutation_partition mp(to);
--
2.7.4

Tomasz Grabiec

<tgrabiec@scylladb.com>
unread,
Jun 12, 2017, 3:02:22 PM6/12/17
to scylladb-dev@googlegroups.com, Piotr Jastrzebski
From: Piotr Jastrzebski <pi...@scylladb.com>

Signed-off-by: Piotr Jastrzebski <pi...@scylladb.com>

[tgrabiec:
- extracted from a larger commit
- extracted partition_version detaching into partition_entry::with_detached_versions()
- dropped unnecessary rows_iterator::_version field
- dropped unnecessary allocation of rows_entry and key copies in rows_iterator
- dropped row_pointer
- replaced apply_reversibly() with weaker and faster apply()
- added handling of dummy entries at any position
]
---
partition_version.hh | 25 ++++++
partition_version.cc | 238 +++++++++++++++++++++++++++++++++++++++++++++++++++
2 files changed, 263 insertions(+)

diff --git a/partition_version.hh b/partition_version.hh
index 740d755..5f86693 100644
--- a/partition_version.hh
+++ b/partition_version.hh
@@ -216,9 +216,14 @@ class partition_snapshot : public enable_lw_shared_from_this<partition_snapshot>
}

unsigned version_count();
+
+ bool at_latest_version() const {
+ return _entry != nullptr;
+ }
};

class partition_entry {
+ class rows_iterator;
partition_snapshot* _snapshot = nullptr;
partition_version_ref _version;

@@ -233,6 +238,7 @@ class partition_entry {
void set_version(partition_version*);

void apply(const schema& s, partition_version* pv, const schema& pv_schema);
+ void apply_to_incomplete(const schema& s, partition_version* other);
public:
partition_entry() = default;
explicit partition_entry(mutation_partition mp);
@@ -268,6 +274,25 @@ class partition_entry {
// Strong exception guarantees.
void apply(const schema& s, mutation_partition_view mpv, const schema& mp_schema);

+ // Adds mutation_partition represented by "other" to the one represented
+ // by this entry.
+ //
+ // The argument must be fully-continuous.
+ //
+ // The rules of addition differ from that used by regular
+ // mutation_partition addition with regards to continuity. The continuity
+ // of the result is the same as in this instance. Information from "other"
+ // which is incomplete in this instance is dropped. In other words, this
+ // performs set intersection on continuity information, and applies
+ // regular merging rules for elements which are continuous in both
+ // addends.
+ //
+ // Weak exception guarantees.
+ // If an exception is thrown this and pe will be left in some valid states
+ // such that if the operation is retried (possibly many times) and eventually
+ // succeeds the result will be as if the first attempt didn't fail.
+ void apply_to_incomplete(const schema& s, partition_entry&& pe, const schema& pe_schema);
+
// Weak exception guarantees.
// If an exception is thrown this and pe will be left in some valid states
// such that if the operation is retried (possibly many times) and eventually
diff --git a/partition_version.cc b/partition_version.cc
index 1cbb993..a8758bd 100644
--- a/partition_version.cc
+++ b/partition_version.cc
@@ -257,6 +257,183 @@ void partition_entry::apply(const schema& s, partition_entry&& pe, const schema&
}
}

+namespace {
+
+// When applying partition_entry to an incomplete partition_entry this class is used to represent
+// the target incomplete partition_entry. It encapsulates the logic needed for handling multiple versions.
+class apply_incomplete_target final {
+ struct version {
+ mutation_partition::rows_type::iterator current_row;
+ mutation_partition::rows_type* rows;
+ size_t version_no;
+
+ struct compare {
+ const rows_entry::tri_compare& _cmp;
+ public:
+ explicit compare(const rows_entry::tri_compare& cmp) : _cmp(cmp) { }
+ bool operator()(const version& a, const version& b) const {
+ auto res = _cmp(*a.current_row, *b.current_row);
+ return res > 0 || (res == 0 && a.version_no < b.version_no);
+ }
+ };
+ };
+ const schema& _schema;
+ partition_entry& _pe;
+ rows_entry::tri_compare _rows_cmp;
+ rows_entry::compare _rows_less_cmp;
+ version::compare _version_cmp;
+ std::vector<version> _heap;
+ mutation_partition::rows_type::iterator _next_in_latest_version;
+public:
+ apply_incomplete_target(partition_entry& pe, const schema& schema)
+ : _schema(schema)
+ , _pe(pe)
+ , _rows_cmp(schema)
+ , _rows_less_cmp(schema)
+ , _version_cmp(_rows_cmp)
+ {
+ size_t version_no = 0;
+ _next_in_latest_version = pe.version()->partition().clustered_rows().begin();
+ for (auto&& v : pe.version()->elements_from_this()) {
+ if (!v.partition().clustered_rows().empty()) {
+ _heap.push_back({v.partition().clustered_rows().begin(), &v.partition().clustered_rows(), version_no});
+ }
+ ++version_no;
+ }
+ boost::range::make_heap(_heap, _version_cmp);
+ }
+ // Prepares this iterator for applying a row with given key.
+ // Returns true iff the key falls into continuous range.
+ // Must be called with monotonic keys.
+ bool advance_to(const clustering_key& key) {
+ while (!_heap.empty() && _rows_less_cmp(*_heap[0].current_row, key)) {
+ boost::range::pop_heap(_heap, _version_cmp);
+ auto& curr = _heap.back();
+ curr.current_row = curr.rows->lower_bound(key, _rows_less_cmp);
+ if (curr.version_no == 0) {
+ _next_in_latest_version = curr.current_row;
+ }
+ if (curr.current_row == curr.rows->end()) {
+ _heap.pop_back();
+ } else {
+ boost::range::push_heap(_heap, _version_cmp);
+ }
+ }
+
+ if (_heap.empty()) {
+ return true;
+ }
+ rows_entry& next_row = *_heap[0].current_row;
+ if (_rows_cmp(key, next_row) == 0) {
+ return bool(!next_row.dummy());
+ } else {
+ return bool(next_row.continuous());
+ }
+ }
+
+ // Applies the row.
+ // advance_to() must be called prior to this with the same key.
+ void apply(const clustering_key& key, deletable_row&& row) {
+ mutation_partition::rows_type& rows = _pe.version()->partition().clustered_rows();
+ if (_next_in_latest_version != rows.end() && _rows_cmp(key, *_next_in_latest_version) == 0) {
+ _next_in_latest_version->row().apply(_schema, std::move(row));
+ } else {
+ auto e = current_allocator().construct<rows_entry>(key, std::move(row));
+ e->set_continuous(_heap.empty() ? is_continuous::yes : _heap[0].current_row->continuous());
+ rows.insert_before(_next_in_latest_version, *e);
+ }
+ }
+};
+
+} // namespace
+
+// Iterates over all rows in mutation represented by partition_entry.
+// It abstracts away the fact that rows may be spread across multiple versions.
+class partition_entry::rows_iterator final {
+ struct version {
+ mutation_partition::rows_type::iterator current_row;
+ mutation_partition::rows_type* rows;
+ bool can_move;
+ struct compare {
+ const rows_entry::tri_compare& _cmp;
+ public:
+ explicit compare(const rows_entry::tri_compare& cmp) : _cmp(cmp) { }
+ bool operator()(const version& a, const version& b) const {
+ return _cmp(*a.current_row, *b.current_row) > 0;
+ }
+ };
+ };
+ const schema& _schema;
+ rows_entry::tri_compare _rows_cmp;
+ rows_entry::compare _rows_less_cmp;
+ version::compare _version_cmp;
+ std::vector<version> _heap;
+ std::vector<version> _current_row;
+public:
+ rows_iterator(partition_version* version, const schema& schema)
+ : _schema(schema)
+ , _rows_cmp(schema)
+ , _rows_less_cmp(schema)
+ , _version_cmp(_rows_cmp)
+ {
+ bool can_move = true;
+ while (version) {
+ can_move &= !version->is_referenced();
+ auto& rows = version->partition().clustered_rows();
+ if (!rows.empty()) {
+ _heap.push_back({rows.begin(), &rows, can_move});
+ }
+ version = version->next();
+ }
+ boost::range::make_heap(_heap, _version_cmp);
+ move_to_next_row();
+ }
+ bool done() const {
+ return _current_row.empty();
+ }
+ // Return clustering key of the current row in source.
+ const clustering_key& key() const {
+ return _current_row[0].current_row->key();
+ }
+ deletable_row extract_current_row() {
+ assert(!_current_row.empty());
+ deletable_row result;
+ // versions in _current_row are not ordered but it is not a problem
+ // due to the fact that all rows are continuous.
+ for (version& v : _current_row) {
+ if (!v.can_move) {
+ result.apply(_schema, deletable_row(v.current_row->row()));
+ } else {
+ result.apply(_schema, std::move(v.current_row->row()));
+ }
+ }
+ return result;
+ }
+ void remove_current_row_when_possible() {
+ assert(!_current_row.empty());
+ auto deleter = current_deleter<rows_entry>();
+ for (version& v : _current_row) {
+ if (v.can_move) {
+ v.rows->erase_and_dispose(v.current_row, deleter);
+ }
+ }
+ }
+ void move_to_next_row() {
+ _current_row.clear();
+ while (!_heap.empty() &&
+ (_current_row.empty() || _rows_cmp(*_current_row[0].current_row, *_heap[0].current_row) == 0)) {
+ boost::range::pop_heap(_heap, _version_cmp);
+ auto& curr = _heap.back();
+ _current_row.push_back({curr.current_row, curr.rows, curr.can_move});
+ ++curr.current_row;
+ if (curr.current_row == curr.rows->end()) {
+ _heap.pop_back();
+ } else {
+ boost::range::push_heap(_heap, _version_cmp);
+ }
+ }
+ }
+};

template<typename Func>
void partition_entry::with_detached_versions(Func&& func) {
@@ -282,6 +459,67 @@ void partition_entry::with_detached_versions(Func&& func) {
func(current);
}

+void partition_entry::apply_to_incomplete(const schema& s, partition_entry&& pe, const schema& pe_schema)
+{
+ if (s.version() != pe_schema.version()) {
+ partition_entry entry(pe.squashed(pe_schema.shared_from_this(), s.shared_from_this()));
+ entry.with_detached_versions([&] (partition_version* v) {
+ apply_to_incomplete(s, v);
+ });
+ } else {
+ pe.with_detached_versions([&](partition_version* v) {
+ apply_to_incomplete(s, v);
+ });
+ }
+}
+
+void partition_entry::apply_to_incomplete(const schema& s, partition_version* version) {
+ // Make sure information about static row being cached stays unchanged.
+ bool is_static_row_cached = _version->partition().is_static_row_cached();
+ auto setup_static_row_cached = defer([this, is_static_row_cached] {
+ _version->partition().set_static_row_cached(is_static_row_cached);
+ });
+
+ if (_snapshot) {
+ auto new_version = current_allocator().construct<partition_version>(s.shared_from_this());
+ new_version->insert_before(*_version);
+ set_version(new_version);
+ }
+
+ bool can_move = true;
+ auto current = version;
+ while (current) {
+ can_move &= !current->is_referenced();
+ _version->partition().apply(current->partition().partition_tombstone());
+ if (is_static_row_cached) {
+ row& static_row = _version->partition().static_row();
+ if (can_move) {
+ static_row.apply(s, column_kind::static_column, std::move(current->partition().static_row()));
+ } else {
+ static_row.apply(s, column_kind::static_column, current->partition().static_row());
+ }
+ }
+ range_tombstone_list& tombstones = _version->partition().row_tombstones();
+ if (can_move) {
+ tombstones.apply_reversibly(s, current->partition().row_tombstones()).cancel();
+ } else {
+ tombstones.apply(s, current->partition().row_tombstones());
+ }
+ current = current->next();
+ }
+
+ partition_entry::rows_iterator source(version, s);
+ apply_incomplete_target target(*this, s);
+
+ while (!source.done()) {
+ if (target.advance_to(source.key())) {
+ target.apply(source.key(), source.extract_current_row());
+ }
+ source.remove_current_row_when_possible();
+ source.move_to_next_row();
+ }

Tomasz Grabiec

<tgrabiec@scylladb.com>
unread,
Jun 12, 2017, 3:02:24 PM6/12/17
to scylladb-dev@googlegroups.com, Piotr Jastrzebski
From: Piotr Jastrzebski <pi...@scylladb.com>

This will be needed by implementation of cache_streamed_mutation

Signed-off-by: Piotr Jastrzebski <pi...@scylladb.com>
---
streamed_mutation.hh | 24 ++++++++++++++++++++++--
1 file changed, 22 insertions(+), 2 deletions(-)

diff --git a/streamed_mutation.hh b/streamed_mutation.hh
index f8f9849..35e4950 100644
--- a/streamed_mutation.hh
+++ b/streamed_mutation.hh
@@ -184,9 +184,29 @@ class mutation_fragment {
mutation_fragment(clustering_row&& r);
mutation_fragment(range_tombstone&& r);

- mutation_fragment(const mutation_fragment&) = delete;
+ mutation_fragment(const mutation_fragment& o)
+ : _kind(o._kind), _data(std::make_unique<data>()) {
+ switch(_kind) {
+ case kind::static_row:
+ new (&_data->_static_row) static_row(o._data->_static_row);
+ break;
+ case kind::clustering_row:
+ new (&_data->_clustering_row) clustering_row(o._data->_clustering_row);
+ break;
+ case kind::range_tombstone:
+ new (&_data->_range_tombstone) range_tombstone(o._data->_range_tombstone);
+ break;
+ }
+ }
mutation_fragment(mutation_fragment&& other) = default;
- mutation_fragment& operator=(const mutation_fragment&) = delete;
+ mutation_fragment& operator=(const mutation_fragment& other) {
+ if (this != &other) {
+ mutation_fragment copy(other);
+ this->~mutation_fragment();
+ new (this) mutation_fragment(std::move(copy));
+ }
+ return *this;
+ }
mutation_fragment& operator=(mutation_fragment&& other) noexcept {
if (this != &other) {
this->~mutation_fragment();
--
2.7.4

Tomasz Grabiec

<tgrabiec@scylladb.com>
unread,
Jun 12, 2017, 3:02:25 PM6/12/17
to scylladb-dev@googlegroups.com
---
range_tombstone_list.hh | 3 +++
range_tombstone_list.cc | 16 ++++++++++++++++
2 files changed, 19 insertions(+)

diff --git a/range_tombstone_list.hh b/range_tombstone_list.hh
index 8d27fab..0bc9202 100644
--- a/range_tombstone_list.hh
+++ b/range_tombstone_list.hh
@@ -140,6 +140,9 @@ class range_tombstone_list final {
// Returns range of tombstones which overlap with given range
boost::iterator_range<const_iterator> slice(const schema& s, const query::clustering_range&) const;
iterator erase(const_iterator, const_iterator);
+ // Ensures that every range tombstone is strictly contained within given clustering ranges.
+ // Preserves all information which may be relevant for rows from that ranges.
+ void trim(const schema& s, const query::clustering_row_ranges&);
range_tombstone_list difference(const schema& s, const range_tombstone_list& rt_list) const;
// Erases the range tombstones for which filter returns true.
template <typename Pred>
diff --git a/range_tombstone_list.cc b/range_tombstone_list.cc
index 5a950ae..b3ffda8 100644
--- a/range_tombstone_list.cc
+++ b/range_tombstone_list.cc
@@ -312,6 +312,22 @@ range_tombstone_list::erase(const_iterator a, const_iterator b) {
return _tombstones.erase_and_dispose(a, b, current_deleter<range_tombstone>());
}

+void range_tombstone_list::trim(const schema& s, const query::clustering_row_ranges& ranges) {
+ range_tombstone_list list(s);
+ bound_view::compare less(s);
+ for (auto&& range : ranges) {
+ auto start = bound_view::from_range_start(range);
+ auto end = bound_view::from_range_end(range);
+ for (const range_tombstone& rt : slice(s, range)) {
+ list.apply(s, range_tombstone(
+ std::max(rt.start_bound(), start, less),
+ std::min(rt.end_bound(), end, less),
+ rt.tomb));
+ }
+ }
+ *this = std::move(list);
+}
+
range_tombstone_list::range_tombstones_type::iterator
range_tombstone_list::reverter::insert(range_tombstones_type::iterator it, range_tombstone& new_rt) {
_ops.emplace_back(insert_undo_op(new_rt));
--
2.7.4

Tomasz Grabiec

<tgrabiec@scylladb.com>
unread,
Jun 12, 2017, 3:02:26 PM6/12/17
to scylladb-dev@googlegroups.com
---
range_tombstone_list.hh | 3 +++
range_tombstone_list.cc | 4 ++++
2 files changed, 7 insertions(+)

diff --git a/range_tombstone_list.hh b/range_tombstone_list.hh
index 0bc9202..d9e8963 100644
--- a/range_tombstone_list.hh
+++ b/range_tombstone_list.hh
@@ -23,6 +23,7 @@

#include "range_tombstone.hh"
#include "query-request.hh"
+#include <iosfwd>

class range_tombstone_list final {
using range_tombstones_type = range_tombstone::container_type;
@@ -164,6 +165,8 @@ class range_tombstone_list final {
void apply(const schema& s, const range_tombstone_list& rt_list);
// See reversibly_mergeable.hh
reverter apply_reversibly(const schema& s, range_tombstone_list& rt_list);
+
+ friend std::ostream& operator<<(std::ostream& out, const range_tombstone_list&);
private:
void apply_reversibly(const schema& s, clustering_key_prefix start, bound_kind start_kind,
clustering_key_prefix end, bound_kind end_kind, tombstone tomb, reverter& rev);
diff --git a/range_tombstone_list.cc b/range_tombstone_list.cc
index b3ffda8..8a97b16 100644
--- a/range_tombstone_list.cc
+++ b/range_tombstone_list.cc
@@ -384,3 +384,7 @@ void range_tombstone_list::update_undo_op::undo(const schema& s, range_tombstone
assert (it != rt_list.end());
*it = std::move(_old_rt);
}
+
+std::ostream& operator<<(std::ostream& out, const range_tombstone_list& list) {
+ return out << "{" << ::join(", ", list) << "}";
+}
--
2.7.4

Tomasz Grabiec

<tgrabiec@scylladb.com>
unread,
Jun 12, 2017, 3:02:27 PM6/12/17
to scylladb-dev@googlegroups.com
---
streamed_mutation.hh | 1 +
streamed_mutation.cc | 4 ++++
2 files changed, 5 insertions(+)

diff --git a/streamed_mutation.hh b/streamed_mutation.hh
index 35e4950..6217fe9 100644
--- a/streamed_mutation.hh
+++ b/streamed_mutation.hh
@@ -572,6 +572,7 @@ class range_tombstone_stream {
}
void apply(const range_tombstone_list&, const query::clustering_range&);
void reset();
+ friend std::ostream& operator<<(std::ostream& out, const range_tombstone_stream&);
};

// mutation_hasher is an equivalent of hashing_partition_visitor for
diff --git a/streamed_mutation.cc b/streamed_mutation.cc
index 9ffab21..e45aaa8 100644
--- a/streamed_mutation.cc
+++ b/streamed_mutation.cc
@@ -637,3 +637,7 @@ bool mutation_fragment::relevant_for_range_assuming_after(const schema& s, posit
// Range tombstones overlapping with the new range are let in
return is_range_tombstone() && cmp(pos, as_range_tombstone().end_position());
}
+
+std::ostream& operator<<(std::ostream& out, const range_tombstone_stream& rtl) {
+ return out << rtl._list;
+}
--
2.7.4

Tomasz Grabiec

<tgrabiec@scylladb.com>
unread,
Jun 12, 2017, 3:02:28 PM6/12/17
to scylladb-dev@googlegroups.com
---
range_tombstone_list.hh | 1 +
range_tombstone_list.cc | 6 ++++++
2 files changed, 7 insertions(+)

diff --git a/range_tombstone_list.hh b/range_tombstone_list.hh
index d9e8963..f1f85a6 100644
--- a/range_tombstone_list.hh
+++ b/range_tombstone_list.hh
@@ -167,6 +167,7 @@ class range_tombstone_list final {
reverter apply_reversibly(const schema& s, range_tombstone_list& rt_list);

friend std::ostream& operator<<(std::ostream& out, const range_tombstone_list&);
+ bool equal(const schema&, const range_tombstone_list&) const;
private:
void apply_reversibly(const schema& s, clustering_key_prefix start, bound_kind start_kind,
clustering_key_prefix end, bound_kind end_kind, tombstone tomb, reverter& rev);
diff --git a/range_tombstone_list.cc b/range_tombstone_list.cc
index 8a97b16..c8d7d8e 100644
--- a/range_tombstone_list.cc
+++ b/range_tombstone_list.cc
@@ -388,3 +388,9 @@ void range_tombstone_list::update_undo_op::undo(const schema& s, range_tombstone
std::ostream& operator<<(std::ostream& out, const range_tombstone_list& list) {
return out << "{" << ::join(", ", list) << "}";
}
+
+bool range_tombstone_list::equal(const schema& s, const range_tombstone_list& other) const {
+ return boost::equal(_tombstones, other._tombstones, [&s] (auto&& rt1, auto&& rt2) {
+ return rt1.equal(s, rt2);
+ });
+}
--
2.7.4

Tomasz Grabiec

<tgrabiec@scylladb.com>
unread,
Jun 12, 2017, 3:02:30 PM6/12/17
to scylladb-dev@googlegroups.com
---
mutation.hh | 4 ++++
mutation.cc | 6 ++++++
2 files changed, 10 insertions(+)

diff --git a/mutation.hh b/mutation.hh
index 0ac0e85..f3f6345 100644
--- a/mutation.hh
+++ b/mutation.hh
@@ -133,6 +133,10 @@ class mutation final {
mutation operator+(const mutation& other) const;
mutation& operator+=(const mutation& other);
mutation& operator+=(mutation&& other);
+
+ // Returns a subset of this mutation holding only information relevant for given clustering ranges.
+ // Range tombstones will be trimmed to the boundaries of the clustering ranges.
+ mutation sliced(const query::clustering_row_ranges&) const;
private:
friend std::ostream& operator<<(std::ostream& os, const mutation& m);
};
diff --git a/mutation.cc b/mutation.cc
index ef262a8..e801c2a 100644
--- a/mutation.cc
+++ b/mutation.cc
@@ -206,6 +206,12 @@ mutation& mutation::operator+=(mutation&& other) {
return *this;
}

+mutation mutation::sliced(const query::clustering_row_ranges& ranges) const {
+ auto m = mutation(schema(), decorated_key(), mutation_partition(partition(), *schema(), ranges));
+ m.partition().row_tombstones().trim(*schema(), ranges);
+ return m;
+}
+
class mutation_rebuilder {
mutation _m;
streamed_mutation& _sm;
--
2.7.4

Tomasz Grabiec

<tgrabiec@scylladb.com>
unread,
Jun 12, 2017, 3:02:30 PM6/12/17
to scylladb-dev@googlegroups.com
---
tests/mutation_reader_assertions.hh | 52 ++++++++++++++++---------------------
1 file changed, 23 insertions(+), 29 deletions(-)

diff --git a/tests/mutation_reader_assertions.hh b/tests/mutation_reader_assertions.hh
index 20f81f8..83f6615 100644
--- a/tests/mutation_reader_assertions.hh
+++ b/tests/mutation_reader_assertions.hh
@@ -29,6 +29,11 @@
class reader_assertions {
mutation_reader _reader;
dht::partition_range _pr;
+private:
+ mutation_opt read_next() {
+ auto smo = _reader().get0();
+ return mutation_from_streamed_mutation(std::move(smo)).get0();
+ }
public:
reader_assertions(mutation_reader reader)
: _reader(std::move(reader))
@@ -36,35 +41,28 @@ class reader_assertions {

reader_assertions& produces(const dht::decorated_key& dk) {
BOOST_TEST_MESSAGE(sprint("Expecting key %s", dk));
- _reader().then([&] (auto sm) {
- if (!sm) {
- BOOST_FAIL(sprint("Expected: %s, got end of stream", dk));
- }
- if (!sm->decorated_key().equal(*sm->schema(), dk)) {
- BOOST_FAIL(sprint("Expected: %s, got: %s", dk, sm->decorated_key()));
- }
- }).get0();
+ auto mo = read_next();
+ if (!mo) {
+ BOOST_FAIL(sprint("Expected: %s, got end of stream", dk));
+ }
+ if (!mo->decorated_key().equal(*mo->schema(), dk)) {
+ BOOST_FAIL(sprint("Expected: %s, got: %s", dk, mo->decorated_key()));
+ }
return *this;
}

reader_assertions& produces(mutation m) {
BOOST_TEST_MESSAGE(sprint("Expecting %s", m));
- _reader().then([] (auto sm) {
- return mutation_from_streamed_mutation(std::move(sm));
- }).then([this, m = std::move(m)] (mutation_opt&& mo) mutable {
- BOOST_REQUIRE(bool(mo));
- assert_that(*mo).is_equal_to(m);
- }).get0();
+ auto mo = read_next();
+ BOOST_REQUIRE(bool(mo));
+ assert_that(*mo).is_equal_to(m);
return *this;
}

mutation_assertion next_mutation() {
- return _reader().then([] (auto sm) {
- return mutation_from_streamed_mutation(std::move(sm));
- }).then([] (mutation_opt&& mo) mutable {
- BOOST_REQUIRE(bool(mo));
- return mutation_assertion(std::move(*mo));
- }).get0();
+ auto mo = read_next();
+ BOOST_REQUIRE(bool(mo));
+ return mutation_assertion(std::move(*mo));
}

template<typename RangeOfMutations>
@@ -77,20 +75,16 @@ class reader_assertions {

reader_assertions& produces_end_of_stream() {
BOOST_TEST_MESSAGE("Expecting end of stream");
- _reader().then([] (auto sm) {
- return mutation_from_streamed_mutation(std::move(sm));
- }).then([this] (mutation_opt&& mo) mutable {
- if (bool(mo)) {
- BOOST_FAIL(sprint("Expected end of stream, got %s", *mo));
- }
- }).get0();
+ auto mo = read_next();
+ if (bool(mo)) {
+ BOOST_FAIL(sprint("Expected end of stream, got %s", *mo));
+ }
return *this;
}

reader_assertions& produces_eos_or_empty_mutation() {
BOOST_TEST_MESSAGE("Expecting eos or empty mutation");
- auto sm = _reader().get0();
- mutation_opt mo = mutation_from_streamed_mutation(std::move(sm)).get0();
+ auto mo = read_next();
if (mo) {
if (!mo->partition().empty()) {
BOOST_FAIL(sprint("Mutation is not empty: %s", *mo));
--
2.7.4

Tomasz Grabiec

<tgrabiec@scylladb.com>
unread,
Jun 12, 2017, 3:02:31 PM6/12/17
to scylladb-dev@googlegroups.com
Currently mutation sources are free to return range tombstones
covering range which is larger than the query range. The cache
mutation source will soon become more eager about trimming such
tombstones. To cover up for such differences, allow telling the
restrictions to only care about differences relevant for given
clustering ranges.
---
tests/mutation_assertions.hh | 23 ++++++++++++++++++++---
tests/mutation_reader_assertions.hh | 4 ++--
2 files changed, 22 insertions(+), 5 deletions(-)

diff --git a/tests/mutation_assertions.hh b/tests/mutation_assertions.hh
index 41fd7d6..af30dbe 100644
--- a/tests/mutation_assertions.hh
+++ b/tests/mutation_assertions.hh
@@ -30,7 +30,12 @@ class mutation_assertion {
: _m(std::move(m))
{ }

- mutation_assertion& is_equal_to(const mutation& other) {
+ // If ck_ranges is passed, verifies only that information relevant for ck_ranges matches.
+ mutation_assertion& is_equal_to(const mutation& other, const query::clustering_row_ranges& ck_ranges = {}) {
+ if (!ck_ranges.empty()) {
+ mutation_assertion(_m.sliced(ck_ranges)).is_equal_to(other.sliced(ck_ranges));
+ return *this;
+ }
if (_m != other) {
BOOST_FAIL(sprint("Mutations differ, expected %s\n ...but got: %s", other, _m));
}
@@ -171,7 +176,8 @@ class streamed_mutation_assertions {
return *this;
}

- streamed_mutation_assertions& produces_range_tombstone(const range_tombstone& rt) {
+ // If ck_ranges is passed, verifies only that information relevant for ck_ranges matches.
+ streamed_mutation_assertions& produces_range_tombstone(const range_tombstone& rt, const query::clustering_row_ranges& ck_ranges = {}) {
BOOST_TEST_MESSAGE(sprint("Expect %s", rt));
auto mfo = _sm().get0();
if (!mfo) {
@@ -181,7 +187,18 @@ class streamed_mutation_assertions {
BOOST_FAIL(sprint("Expected range tombstone %s, but got %s", rt, *mfo));
}
auto& actual = mfo->as_range_tombstone();
- if (!actual.equal(*_sm.schema(), rt)) {
+ const schema& s = *_sm.schema();
+ if (!ck_ranges.empty()) {
+ range_tombstone_list actual_list(s);
+ range_tombstone_list expected_list(s);
+ actual_list.apply(s, actual);
+ expected_list.apply(s, rt);
+ actual_list.trim(s, ck_ranges);
+ expected_list.trim(s, ck_ranges);
+ if (!actual_list.equal(s, expected_list)) {
+ BOOST_FAIL(sprint("Expected %s, but got %s", expected_list, actual_list));
+ }
+ } else if (!actual.equal(s, rt)) {
BOOST_FAIL(sprint("Expected range tombstone %s, but got %s", rt, actual));
}
return *this;
diff --git a/tests/mutation_reader_assertions.hh b/tests/mutation_reader_assertions.hh
index 83f6615..4c24a89 100644
--- a/tests/mutation_reader_assertions.hh
+++ b/tests/mutation_reader_assertions.hh
@@ -51,11 +51,11 @@ class reader_assertions {
return *this;
}

- reader_assertions& produces(mutation m) {
+ reader_assertions& produces(mutation m, const query::clustering_row_ranges& ck_ranges = {}) {
BOOST_TEST_MESSAGE(sprint("Expecting %s", m));
auto mo = read_next();
BOOST_REQUIRE(bool(mo));
- assert_that(*mo).is_equal_to(m);
+ assert_that(*mo).is_equal_to(m, ck_ranges);
return *this;
}

--
2.7.4

Tomasz Grabiec

<tgrabiec@scylladb.com>
unread,
Jun 12, 2017, 3:02:32 PM6/12/17
to scylladb-dev@googlegroups.com
In preparation for having partial cache which trims range tombstones
to the lower bound of the query.
---
tests/mutation_source_test.cc | 12 ++++++------
1 file changed, 6 insertions(+), 6 deletions(-)

diff --git a/tests/mutation_source_test.cc b/tests/mutation_source_test.cc
index 4b0f734..476e1fe 100644
--- a/tests/mutation_source_test.cc
+++ b/tests/mutation_source_test.cc
@@ -90,7 +90,7 @@ static void test_streamed_mutation_forwarding_is_consistent_with_slicing(populat
}

mutation sliced_m = mutation_from_streamed_mutation(sliced_sm).get0();
- assert_that(sliced_m).is_equal_to(fwd_m);
+ assert_that(sliced_m).is_equal_to(fwd_m, slice_with_ranges.row_ranges(*m.schema(), m.key()));
}
}

@@ -295,9 +295,9 @@ static void test_streamed_mutation_slicing_returns_only_relevant_tombstones(popu
auto sm = assert_that_stream(std::move(*smo));

sm.produces_row_with_key(keys[2]);
- sm.produces_range_tombstone(rt3);
+ sm.produces_range_tombstone(rt3, slice.row_ranges(*s, m.key()));
sm.produces_row_with_key(keys[8]);
- sm.produces_range_tombstone(rt4);
+ sm.produces_range_tombstone(rt4, slice.row_ranges(*s, m.key()));
sm.produces_end_of_stream();
}

@@ -314,9 +314,9 @@ static void test_streamed_mutation_slicing_returns_only_relevant_tombstones(popu
streamed_mutation_opt smo = rd().get0();
BOOST_REQUIRE(bool(smo));
assert_that_stream(std::move(*smo))
- .produces_range_tombstone(rt3)
+ .produces_range_tombstone(rt3, slice.row_ranges(*s, m.key()))
.produces_row_with_key(keys[8])
- .produces_range_tombstone(rt4)
+ .produces_range_tombstone(rt4, slice.row_ranges(*s, m.key()))
.produces_end_of_stream();
}
}
@@ -676,7 +676,7 @@ static void test_clustering_slices(populate_fn populate) {
.with_range(query::clustering_range::make_singular(make_ck(2)))
.build();
assert_that(ds(s, pr, slice))
- .produces(row6 + row7 + del_1 + del_2)
+ .produces(row6 + row7 + del_1 + del_2, slice.row_ranges(*s, pk.key()))
.produces_end_of_stream();
}

--
2.7.4

Tomasz Grabiec

<tgrabiec@scylladb.com>
unread,
Jun 12, 2017, 3:02:33 PM6/12/17
to scylladb-dev@googlegroups.com, Piotr Jastrzebski
From: Piotr Jastrzebski <pi...@scylladb.com>

[tgrabiec: Extracted from different patch]
---
mutation_partition.hh | 6 ++++++
1 file changed, 6 insertions(+)

diff --git a/mutation_partition.hh b/mutation_partition.hh
index c04915a..ae87456 100644
--- a/mutation_partition.hh
+++ b/mutation_partition.hh
@@ -599,6 +599,9 @@ class deletable_row final {
public:
deletable_row() {}
explicit deletable_row(clustering_row&&);
+ deletable_row(row_tombstone tomb, const row_marker& marker, const row& cells)
+ : _deleted_at(tomb), _marker(marker), _cells(cells)
+ {}

void apply(tombstone deleted_at) {
_deleted_at.apply(deleted_at);
@@ -692,6 +695,9 @@ class rows_entry {
rows_entry(const clustering_key& key, const deletable_row& row)
: _key(key), _row(row)
{ }
+ rows_entry(const clustering_key& key, row_tombstone tomb, const row_marker& marker, const row& row)
+ : _key(key), _row(tomb, marker, row)
+ { }
rows_entry(rows_entry&& o) noexcept;
rows_entry(const rows_entry& e)
: _key(e._key)
--
2.7.4

Tomasz Grabiec

<tgrabiec@scylladb.com>
unread,
Jun 12, 2017, 3:02:34 PM6/12/17
to scylladb-dev@googlegroups.com, Piotr Jastrzebski
From: Piotr Jastrzebski <pi...@scylladb.com>

[tgrabiec: Extracted from a different patch]
---
partition_version.hh | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/partition_version.hh b/partition_version.hh
index 5f86693..c6a64dc 100644
--- a/partition_version.hh
+++ b/partition_version.hh
@@ -162,7 +162,7 @@ class partition_version_ref {
return *this;
}

- explicit operator bool() { return _version; }
+ explicit operator bool() const { return _version; }

partition_version& operator*() {
assert(_version);
--
2.7.4

Tomasz Grabiec

<tgrabiec@scylladb.com>
unread,
Jun 12, 2017, 3:02:35 PM6/12/17
to scylladb-dev@googlegroups.com, Piotr Jastrzebski
From: Piotr Jastrzebski <pi...@scylladb.com>

[tgrabiec: Extracted from a different patch]
---
partition_version.hh | 4 ++++
1 file changed, 4 insertions(+)

diff --git a/partition_version.hh b/partition_version.hh
index c6a64dc..d2a0792 100644
--- a/partition_version.hh
+++ b/partition_version.hh
@@ -172,6 +172,10 @@ class partition_version_ref {
assert(_version);
return _version;
}
+ const partition_version* operator->() const {
+ assert(_version);
+ return _version;
+ }

bool is_unique_owner() const { return _unique_owner; }
void mark_as_unique_owner() { _unique_owner = true; }
--
2.7.4

Tomasz Grabiec

<tgrabiec@scylladb.com>
unread,
Jun 12, 2017, 3:02:36 PM6/12/17
to scylladb-dev@googlegroups.com, Piotr Jastrzebski
From: Piotr Jastrzebski <pi...@scylladb.com>

[tgrabiec: Extracted from a different patch]
---
partition_version.hh | 2 ++
partition_version.cc | 47 +++++++++++++++++++++++++++++++++++++++++++++++
2 files changed, 49 insertions(+)

diff --git a/partition_version.hh b/partition_version.hh
index d2a0792..5ff3af7 100644
--- a/partition_version.hh
+++ b/partition_version.hh
@@ -224,6 +224,8 @@ class partition_snapshot : public enable_lw_shared_from_this<partition_snapshot>
bool at_latest_version() const {
return _entry != nullptr;
}
+ tombstone partition_tombstone() const;
+ row static_row() const;
};

class partition_entry {
diff --git a/partition_version.cc b/partition_version.cc
index a8758bd..de9297e 100644
--- a/partition_version.cc
+++ b/partition_version.cc
@@ -63,6 +63,53 @@ partition_version::~partition_version()
}
}

+namespace {
+
+/*
+template<typename U, typename T>
+concept bool FetchConcept() {
+ return requires(U obj, T result, mutation_partition& mp) {
+ { obj(mp) } -> T;
+ };
+}
+template<typename U, typename T>
+concept bool CombineConcept() {
+ return requires(U obj, T& a, T& b) {
+ { obj(a, b) } -> void;
+ };
+}
+*/
+
+
+
+// Aggregates particular part of the mutation_partition from all versions starting from v.
+// |fetch| extracts data from mutation_partition.
+// |combine| combines data from two versions.
+template <typename Result, typename Fetch, typename Combine>
+inline Result aggregate(const partition_version_ref& v, Fetch&& fetch, Combine&& combine) {
+ Result r = fetch(v->partition());
+ auto it = v->next();
+ while (it) {
+ combine(r, fetch(it->partition()));
+ it = it->next();
+ }
+ return r;
+}
+
+}
+
+row partition_snapshot::static_row() const {
+ return aggregate<row>(version(),
+ [] (const mutation_partition& mp) -> const row& { return mp.static_row(); },
+ [this] (row& a, const row& b) { a.apply(*_schema, column_kind::static_column, b); });
+}
+
+tombstone partition_snapshot::partition_tombstone() const {
+ return aggregate<tombstone>(version(),
+ [] (const mutation_partition& mp) { return mp.partition_tombstone(); },
+ [] (tombstone& a, tombstone b) { a.apply(b); });
+}
+
partition_snapshot::~partition_snapshot() {
if (_version && _version.is_unique_owner()) {
auto v = &*_version;
--
2.7.4

Tomasz Grabiec

<tgrabiec@scylladb.com>
unread,
Jun 12, 2017, 3:02:37 PM6/12/17
to scylladb-dev@googlegroups.com
---
partition_version.hh | 2 ++
partition_version.cc | 12 ++++++++++++
2 files changed, 14 insertions(+)

diff --git a/partition_version.hh b/partition_version.hh
index 5ff3af7..903c6f5 100644
--- a/partition_version.hh
+++ b/partition_version.hh
@@ -226,6 +226,8 @@ class partition_snapshot : public enable_lw_shared_from_this<partition_snapshot>
}
tombstone partition_tombstone() const;
row static_row() const;
+ // Returns range tombstones overlapping with [start, end)
+ std::vector<range_tombstone> range_tombstones(const schema& s, position_in_partition_view start, position_in_partition_view end);
};

class partition_entry {
diff --git a/partition_version.cc b/partition_version.cc
index de9297e..37cce5a 100644
--- a/partition_version.cc
+++ b/partition_version.cc
@@ -609,3 +609,15 @@ partition_entry::read(schema_ptr entry_schema, partition_snapshot::phase_type ph
_snapshot = snp.get();
return snp;
}
+
+std::vector<range_tombstone>
+partition_snapshot::range_tombstones(const schema& s, position_in_partition_view start, position_in_partition_view end)
+{
+ range_tombstone_list list(s);
+ for (auto&& v : versions()) {
+ for (auto&& rt : v.partition().row_tombstones().slice(s, start, end)) {
+ list.apply(s, rt);
+ }
+ }
+ return boost::copy_range<std::vector<range_tombstone>>(list);
+}
--
2.7.4

Tomasz Grabiec

<tgrabiec@scylladb.com>
unread,
Jun 12, 2017, 3:02:38 PM6/12/17
to scylladb-dev@googlegroups.com, Piotr Jastrzebski
From: Piotr Jastrzebski <pi...@scylladb.com>

[tgrabiec: Extracted from a different patch]
---
partition_version.hh | 11 +++++++++++
1 file changed, 11 insertions(+)

diff --git a/partition_version.hh b/partition_version.hh
index 903c6f5..a48ae76 100644
--- a/partition_version.hh
+++ b/partition_version.hh
@@ -215,6 +215,8 @@ class partition_snapshot : public enable_lw_shared_from_this<partition_snapshot>

partition_version_ref& version();

+ const partition_version_ref& version() const;
+
auto versions() {
return version()->elements_from_this();
}
@@ -325,3 +327,12 @@ inline partition_version_ref& partition_snapshot::version()
return _entry->_version;
}
}
+
+inline const partition_version_ref& partition_snapshot::version() const
+{
+ if (_version) {
+ return _version;
+ } else {
+ return _entry->_version;
+ }
+}
--
2.7.4

Tomasz Grabiec

<tgrabiec@scylladb.com>
unread,
Jun 12, 2017, 3:02:40 PM6/12/17
to scylladb-dev@googlegroups.com
---
position_in_partition.hh | 13 +++++++++++++
1 file changed, 13 insertions(+)

diff --git a/position_in_partition.hh b/position_in_partition.hh
index 767128c1..47f539c 100644
--- a/position_in_partition.hh
+++ b/position_in_partition.hh
@@ -130,6 +130,7 @@ class position_in_partition_view {
}

friend std::ostream& operator<<(std::ostream&, position_in_partition_view);
+ friend bool no_clustering_row_between(const schema&, position_in_partition_view, position_in_partition_view);
};

inline
@@ -365,6 +366,18 @@ class position_in_partition {
friend std::ostream& operator<<(std::ostream&, const position_in_partition&);
};

+// Returns true if and only if there can't be any clustering_row with position > a and < b.
+// It is assumed that a <= b.
+inline
+bool no_clustering_row_between(const schema& s, position_in_partition_view a, position_in_partition_view b) {
+ clustering_key_prefix::equality eq(s);
+ if (a._ck && b._ck) {
+ return eq(*a._ck, *b._ck) && (a._bound_weight >= 0 || b._bound_weight <= 0);
+ } else {
+ return !a._ck && !b._ck;
+ }
+}
+
// Includes all position_in_partition objects "p" for which: start <= p < end
// And only those.
class position_range {
--
2.7.4

Tomasz Grabiec

<tgrabiec@scylladb.com>
unread,
Jun 12, 2017, 3:02:40 PM6/12/17
to scylladb-dev@googlegroups.com
---
position_in_partition.hh | 6 ++++++
1 file changed, 6 insertions(+)

diff --git a/position_in_partition.hh b/position_in_partition.hh
index 61ebffb..767128c1 100644
--- a/position_in_partition.hh
+++ b/position_in_partition.hh
@@ -123,6 +123,12 @@ class position_in_partition_view {
return _ck && _ck->is_empty(s) && _bound_weight > 0;
}

+ // Can be called only when !is_static_row && !is_clustering_row().
+ bound_view as_start_bound_view() const {
+ assert(_bound_weight != 0);
+ return bound_view(*_ck, _bound_weight < 0 ? bound_kind::incl_start : bound_kind::excl_start);
+ }
+
friend std::ostream& operator<<(std::ostream&, position_in_partition_view);
};

--
2.7.4

Tomasz Grabiec

<tgrabiec@scylladb.com>
unread,
Jun 12, 2017, 3:02:41 PM6/12/17
to scylladb-dev@googlegroups.com
---
position_in_partition.hh | 13 +++++++++++++
1 file changed, 13 insertions(+)

diff --git a/position_in_partition.hh b/position_in_partition.hh
index 47f539c..eb6823a 100644
--- a/position_in_partition.hh
+++ b/position_in_partition.hh
@@ -188,6 +188,9 @@ class position_in_partition {
return {clustering_row_tag_t(), std::move(ck)};
}

+ static position_in_partition for_range_start(const query::clustering_range&);
+ static position_in_partition for_range_end(const query::clustering_range&);
+
bool is_static_row() const { return !_ck; }
bool is_clustering_row() const { return _ck && !_bound_weight; }

@@ -366,6 +369,16 @@ class position_in_partition {
friend std::ostream& operator<<(std::ostream&, const position_in_partition&);
};

+inline
+position_in_partition position_in_partition::for_range_start(const query::clustering_range& r) {
+ return {position_in_partition::range_tag_t(), bound_view::from_range_start(r)};
+}
+
+inline
+position_in_partition position_in_partition::for_range_end(const query::clustering_range& r) {
+ return {position_in_partition::range_tag_t(), bound_view::from_range_end(r)};
+}
+
// Returns true if and only if there can't be any clustering_row with position > a and < b.
// It is assumed that a <= b.
inline
--
2.7.4

Tomasz Grabiec

<tgrabiec@scylladb.com>
unread,
Jun 12, 2017, 3:02:42 PM6/12/17
to scylladb-dev@googlegroups.com
---
range_tombstone.hh | 24 ++++++++++++++++++++++--
1 file changed, 22 insertions(+), 2 deletions(-)

diff --git a/range_tombstone.hh b/range_tombstone.hh
index 79fe526..9ab3338 100644
--- a/range_tombstone.hh
+++ b/range_tombstone.hh
@@ -29,11 +29,10 @@
#include "tombstone.hh"
#include "clustering_bounds_comparator.hh"
#include "stdx.hh"
+#include "position_in_partition.hh"

namespace bi = boost::intrusive;

-class position_in_partition_view;
-
/**
* Represents a ranged deletion operation. Can be empty.
*/
@@ -149,6 +148,27 @@ class range_tombstone final {
// is larger than the end bound of this.
stdx::optional<range_tombstone> apply(const schema& s, range_tombstone&& src);

+ // Intersects the range of this tombstone with [pos, +inf) and replaces
+ // the range of the tombstone if there is an overlap.
+ // Returns true if there is an overlap. When returns false, the tombstone
+ // is not modified.
+ //
+ // pos must satisfy:
+ // 1) before_all_clustered_rows() <= pos
+ // 2) !pos.is_clustering_row() - because range_tombstone bounds can't represent such positions
+ bool trim_front(const schema& s, position_in_partition_view pos) {
+ position_in_partition::less_compare less(s);
+ if (!less(pos, end_position())) {
+ return false;
+ }
+ if (less(position(), pos)) {
+ bound_view new_start = pos.as_start_bound_view();
+ start = new_start.prefix;
+ start_kind = new_start.kind;
+ }
+ return true;
+ }
+
size_t external_memory_usage() const {
return start.external_memory_usage() + end.external_memory_usage();
}
--
2.7.4

Tomasz Grabiec

<tgrabiec@scylladb.com>
unread,
Jun 12, 2017, 3:02:42 PM6/12/17
to scylladb-dev@googlegroups.com
---
range_tombstone_list.hh | 3 +++
range_tombstone_list.cc | 19 +++++++++++++++++++
2 files changed, 22 insertions(+)

diff --git a/range_tombstone_list.hh b/range_tombstone_list.hh
index f1f85a6..646e30e 100644
--- a/range_tombstone_list.hh
+++ b/range_tombstone_list.hh
@@ -23,6 +23,7 @@

#include "range_tombstone.hh"
#include "query-request.hh"
+#include "position_in_partition.hh"
#include <iosfwd>

class range_tombstone_list final {
@@ -140,6 +141,8 @@ class range_tombstone_list final {
tombstone search_tombstone_covering(const schema& s, const clustering_key_prefix& key) const;
// Returns range of tombstones which overlap with given range
boost::iterator_range<const_iterator> slice(const schema& s, const query::clustering_range&) const;
+ // Returns range tombstones which overlap with [start, end)
+ boost::iterator_range<const_iterator> slice(const schema& s, position_in_partition_view start, position_in_partition_view end) const;
iterator erase(const_iterator, const_iterator);
// Ensures that every range tombstone is strictly contained within given clustering ranges.
// Preserves all information which may be relevant for rows from that ranges.
diff --git a/range_tombstone_list.cc b/range_tombstone_list.cc
index c8d7d8e..78eb115 100644
--- a/range_tombstone_list.cc
+++ b/range_tombstone_list.cc
@@ -307,6 +307,25 @@ range_tombstone_list::slice(const schema& s, const query::clustering_range& r) c
_tombstones.upper_bound(bv_range.second, order_by_start{s}));
}

+boost::iterator_range<range_tombstone_list::const_iterator>
+range_tombstone_list::slice(const schema& s, position_in_partition_view start, position_in_partition_view end) const {
+ struct order_by_end {
+ position_in_partition::less_compare less;
+ order_by_end(const schema& s) : less(s) {}
+ bool operator()(position_in_partition_view v, const range_tombstone& rt) const { return less(v, rt.end_position()); }
+ bool operator()(const range_tombstone& rt, position_in_partition_view v) const { return less(rt.end_position(), v); }
+ };
+ struct order_by_start {
+ position_in_partition::less_compare less;
+ order_by_start(const schema& s) : less(s) {}
+ bool operator()(position_in_partition_view v, const range_tombstone& rt) const { return less(v, rt.position()); }
+ bool operator()(const range_tombstone& rt, position_in_partition_view v) const { return less(rt.position(), v); }
+ };
+ return boost::make_iterator_range(
+ _tombstones.upper_bound(start, order_by_end{s}), // end_position() is exclusive, hence upper_bound()
+ _tombstones.lower_bound(end, order_by_start{s}));
+}
+
range_tombstone_list::iterator
range_tombstone_list::erase(const_iterator a, const_iterator b) {
return _tombstones.erase_and_dispose(a, b, current_deleter<range_tombstone>());
--
2.7.4

Tomasz Grabiec

<tgrabiec@scylladb.com>
unread,
Jun 12, 2017, 3:02:44 PM6/12/17
to scylladb-dev@googlegroups.com, Piotr Jastrzebski
From: Piotr Jastrzebski <pi...@scylladb.com>

This is an abstraction that represents a reader
to the underlying source and auto updates itself
to make sure the reader reflects the latest state
of the underlying source.

Signed-off-by: Piotr Jastrzebski <pi...@scylladb.com>

[tgrabiec: Add range getter to avoid friendships]
---
row_cache.hh | 1 +
row_cache.cc | 155 ++++++++++++++++++++++++++++++++++++-----------------------
2 files changed, 95 insertions(+), 61 deletions(-)

diff --git a/row_cache.hh b/row_cache.hh
index 94a3054..c033bde 100644
--- a/row_cache.hh
+++ b/row_cache.hh
@@ -225,6 +225,7 @@ class row_cache final {
bi::member_hook<cache_entry, cache_entry::cache_link_type, &cache_entry::_cache_link>,
bi::constant_time_size<false>, // we need this to have bi::auto_unlink on hooks
bi::compare<cache_entry::compare>>;
+ friend class autoupdating_underlying_reader;
friend class single_partition_populating_reader;
friend class cache_entry;
public:
diff --git a/row_cache.cc b/row_cache.cc
index ccea333..e4fd35c 100644
--- a/row_cache.cc
+++ b/row_cache.cc
@@ -171,6 +171,74 @@ const logalloc::region& cache_tracker::region() const {
return _region;
}

+/*
+ * Represent a reader to the underlying source.
+ * This reader automatically makes sure that it's up to date with all cache updates
+ */
+class autoupdating_underlying_reader final {
+ row_cache& _cache;
+ stdx::optional<mutation_reader> _reader;
+ utils::phased_barrier::phase_type _reader_creation_phase;
+ const query::partition_slice& _slice;
+ const io_priority_class& _pc;
+ tracing::trace_state_ptr _trace_state;
+ dht::partition_range _range = { };
+ stdx::optional<dht::decorated_key> _last_key;
+
+public:
+ autoupdating_underlying_reader(row_cache& cache,
+ const query::partition_slice& slice,
+ const io_priority_class& pc,
+ tracing::trace_state_ptr trace_state)
+ : _cache(cache)
+ , _slice(slice)
+ , _pc(pc)
+ , _trace_state(std::move(trace_state))
+ { }
+ future<streamed_mutation_opt> operator()() {
+ auto phase = _cache._populate_phaser.phase();
+ if (!_reader || _reader_creation_phase != phase) {
+ if (_last_key) {
+ auto cmp = dht::ring_position_comparator(*_cache._schema);
+ auto&& new_range = _range.split_after(*_last_key, cmp);
+ if (!new_range) {
+ return make_ready_future<streamed_mutation_opt>(streamed_mutation_opt());
+ }
+ _range = std::move(*new_range);
+ _last_key = { };
+ }
+ _reader = _cache._underlying(_cache._schema, _range, _slice, _pc, _trace_state,
+ streamed_mutation::forwarding::no);
+ _reader_creation_phase = phase;
+ }
+ return (*_reader)().then([this] (auto&& smopt) {
+ if (smopt) {
+ _last_key = smopt->decorated_key();
+ }
+ return std::move(smopt);
+ });
+ }
+ future<> fast_forward_to(dht::partition_range&& range) {
+ _range = std::move(range);
+ _last_key = { };
+ auto phase = _cache._populate_phaser.phase();
+ if (_reader && _reader_creation_phase == phase) {
+ return _reader->fast_forward_to(_range);
+ }
+ _reader = _cache._underlying(_cache._schema, _range, _slice, _pc, _trace_state,
+ streamed_mutation::forwarding::no);
+ _reader_creation_phase = phase;
+ return make_ready_future<>();
+ }
+ utils::phased_barrier::phase_type creation_phase() const {
+ assert(_reader);
+ return _reader_creation_phase;
+ }
+ const dht::partition_range& range() const {
+ return _range;
+ }
+};
+
// Reader which populates the cache using data from the delegate.
class single_partition_populating_reader final : public mutation_reader::impl {
schema_ptr _schema;
@@ -324,43 +392,19 @@ class just_cache_scanning_reader final {
class range_populating_reader {
row_cache& _cache;
schema_ptr _schema;
- dht::partition_range _range;
+ autoupdating_underlying_reader _reader;
const query::partition_slice& _slice;
- utils::phased_barrier::phase_type _populate_phase;
- const io_priority_class& _pc;
- tracing::trace_state_ptr _trace_state;
- mutation_reader _reader;
- bool _reader_created = false;
row_cache::previous_entry_pointer _last_key;
streamed_mutation::forwarding _fwd;
private:
- void update_reader() {
- // TODO: allow updating sstables without fully recreating the reader
- if (_populate_phase != _cache._populate_phaser.phase()) {
- _populate_phase = _cache._populate_phaser.phase();
- if (_last_key._key) {
- auto cmp = dht::ring_position_comparator(*_schema);
- auto&& new_range = _range.split_after(*_last_key._key, cmp);
- if (new_range) {
- _range = std::move(new_range).value();
- } else {
- _reader = make_empty_reader();
- _reader_created = false;
- return;
- }
- }
- _reader = _cache._underlying(_cache._schema, _range, query::full_slice, _pc, _trace_state);
- }
- }
-
void handle_end_of_stream() {
- if (_last_key._populate_phase != _populate_phase) {
+ if (_last_key._populate_phase != _reader.creation_phase()) {
return;
}
- if (!_range.end() || !_range.end()->is_inclusive()) {
+ if (!_reader.range().end() || !_reader.range().end()->is_inclusive()) {
cache_entry::compare cmp(_cache._schema);
- auto it = _range.end() ? _cache._partitions.find(_range.end()->value(), cmp)
- : std::prev(_cache._partitions.end());
+ auto it = _reader.range().end() ? _cache._partitions.find(_reader.range().end()->value(), cmp)
+ : std::prev(_cache._partitions.end());
if (it != _cache._partitions.end()) {
if (it == _cache._partitions.begin()) {
if (!_last_key._key) {
@@ -377,22 +421,20 @@ class range_populating_reader {
}
public:
range_populating_reader(
- row_cache& cache,
- schema_ptr schema,
- const query::partition_slice& slice,
- const io_priority_class& pc,
- tracing::trace_state_ptr trace_state,
- streamed_mutation::forwarding fwd)
+ row_cache& cache,
+ schema_ptr schema,
+ const query::partition_slice& slice,
+ const io_priority_class& pc,
+ tracing::trace_state_ptr trace_state,
+ streamed_mutation::forwarding fwd)
: _cache(cache)
, _schema(std::move(schema))
+ , _reader(cache, slice, pc, std::move(trace_state))
, _slice(slice)
- , _pc(pc)
- , _trace_state(std::move(trace_state))
, _fwd(fwd)
- {}
+ { }

future<streamed_mutation_opt> operator()() {
- update_reader();
return _reader().then([this, op = _cache._populate_phaser.start()] (streamed_mutation_opt smopt) mutable {
return mutation_from_streamed_mutation(std::move(smopt)).then(
[this, op = std::move(op)] (mutation_opt&& mo) mutable {
@@ -403,7 +445,7 @@ class range_populating_reader {

_cache.on_miss();
_cache.populate(*mo, &_last_key);
- _last_key.reset(mo->decorated_key(), _populate_phase);
+ _last_key.reset(mo->decorated_key(), _reader.creation_phase());

mo->upgrade(_schema);
auto ck_ranges = query::clustering_key_filter_ranges::get_ranges(*_schema, _slice, mo->key());
@@ -414,34 +456,24 @@ class range_populating_reader {
});
}

- future<> fast_forward_to(const dht::partition_range& pr) {
- _range = pr;
-
+ future<> fast_forward_to(dht::partition_range&& pr) {
auto phase = _cache._populate_phaser.phase();
- if (!_range.start()) {
+ if (!pr.start()) {
_last_key.reset({ }, phase);
- } else if (!_range.start()->is_inclusive() && _range.start()->value().has_key()) {
- _last_key.reset(_range.start()->value().as_decorated_key(), phase);
+ } else if (!pr.start()->is_inclusive() && pr.start()->value().has_key()) {
+ _last_key.reset(pr.start()->value().as_decorated_key(), phase);
} else {
// Inclusive start bound, cannot set continuity flag.
_last_key.reset(stdx::nullopt, phase - 1);
}

- if (!_reader_created || phase != _populate_phase) {
- _populate_phase = _cache._populate_phaser.phase();
- _reader = _cache._underlying(_cache._schema, _range, query::full_slice, _pc, _trace_state);
- _reader_created = true;
- return make_ready_future();
- }
- return _reader.fast_forward_to(_range);
+ return _reader.fast_forward_to(std::move(pr));
}
};

class scanning_and_populating_reader final : public mutation_reader::impl {
const dht::partition_range* _pr;
schema_ptr _schema;
- dht::partition_range _secondary_range;
-
just_cache_scanning_reader _primary_reader;
range_populating_reader _secondary_reader;
streamed_mutation::forwarding _fwd;
@@ -473,31 +505,32 @@ class scanning_and_populating_reader final : public mutation_reader::impl {
} else {
_next_primary = std::move(smopt);

+ dht::partition_range secondary_range = { };
if (!_next_primary) {
if (!_last_key) {
- _secondary_range = *_pr;
+ secondary_range = *_pr;
} else {
dht::ring_position_comparator cmp(*_schema);
auto&& new_range = _pr->split_after(*_last_key, cmp);
if (!new_range) {
return make_ready_future<streamed_mutation_opt>();
}
- _secondary_range = std::move(*new_range);
+ secondary_range = std::move(*new_range);
}
} else {
if (_last_key) {
- _secondary_range = dht::partition_range::make({ *_last_key, false }, { _next_primary->decorated_key(), false });
+ secondary_range = dht::partition_range::make({ *_last_key, false }, { _next_primary->decorated_key(), false });
} else {
if (!_pr->start()) {
- _secondary_range = dht::partition_range::make_ending_with({ _next_primary->decorated_key(), false });
+ secondary_range = dht::partition_range::make_ending_with({ _next_primary->decorated_key(), false });
} else {
- _secondary_range = dht::partition_range::make(*_pr->start(), { _next_primary->decorated_key(), false });
+ secondary_range = dht::partition_range::make(*_pr->start(), { _next_primary->decorated_key(), false });
}
}
}

_secondary_in_progress = true;
- return _secondary_reader.fast_forward_to(_secondary_range).then([this] {
+ return _secondary_reader.fast_forward_to(std::move(secondary_range)).then([this] {
return read_from_secondary();
});
}
--
2.7.4

Tomasz Grabiec

<tgrabiec@scylladb.com>
unread,
Jun 12, 2017, 3:02:45 PM6/12/17
to scylladb-dev@googlegroups.com, Piotr Jastrzebski
From: Piotr Jastrzebski <pi...@scylladb.com>

This object stores all read relevant context required all
over the place. This leads to a cleaner code.

Signed-off-by: Piotr Jastrzebski <pi...@scylladb.com>

[tgrabiec:
- made read_context shareable to allow storing shared
mutable state later
- added range and cache getters
]
---
read_context.hh | 67 +++++++++++++++++++++++++
row_cache.hh | 17 ++++---
row_cache.cc | 153 ++++++++++++++++++++++----------------------------------
3 files changed, 137 insertions(+), 100 deletions(-)
create mode 100644 read_context.hh

diff --git a/read_context.hh b/read_context.hh
new file mode 100644
index 0000000..33477d9
--- /dev/null
+++ b/read_context.hh
@@ -0,0 +1,67 @@
+/*
+ * Copyright (C) 2017 ScyllaDB
+ */
+
+/*
+ * This file is part of Scylla.
+ *
+ * Scylla is free software: you can redistribute it and/or modify
+ * it under the terms of the GNU Affero General Public License as published by
+ * the Free Software Foundation, either version 3 of the License, or
+ * (at your option) any later version.
+ *
+ * Scylla is distributed in the hope that it will be useful,
+ * but WITHOUT ANY WARRANTY; without even the implied warranty of
+ * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
+ * GNU General Public License for more details.
+ *
+ * You should have received a copy of the GNU General Public License
+ * along with Scylla. If not, see <http://www.gnu.org/licenses/>.
+ */
+
+#pragma once
+
+#include "schema.hh"
+#include "query-request.hh"
+#include "streamed_mutation.hh"
+#include "partition_version.hh"
+#include "tracing/tracing.hh"
+#include "row_cache.hh"
+
+namespace cache {
+
+class read_context final : public enable_lw_shared_from_this<read_context> {
+ row_cache& _cache;
+ schema_ptr _schema;
+ const dht::partition_range& _range;
+ const query::partition_slice& _slice;
+ const io_priority_class& _pc;
+ tracing::trace_state_ptr _trace_state;
+ streamed_mutation::forwarding _fwd;
+public:
+ read_context(row_cache& cache,
+ schema_ptr schema,
+ const dht::partition_range& range,
+ const query::partition_slice& slice,
+ const io_priority_class& pc,
+ tracing::trace_state_ptr trace_state,
+ streamed_mutation::forwarding fwd)
+ : _cache(cache)
+ , _schema(std::move(schema))
+ , _range(range)
+ , _slice(slice)
+ , _pc(pc)
+ , _trace_state(std::move(trace_state))
+ , _fwd(fwd)
+ { }
+ read_context(const read_context&) = delete;
+ row_cache& cache() { return _cache; }
+ const schema_ptr& schema() const { return _schema; }
+ const dht::partition_range& range() const { return _range; }
+ const query::partition_slice& slice() const { return _slice; }
+ const io_priority_class& pc() const { return _pc; }
+ tracing::trace_state_ptr trace_state() const { return _trace_state; }
+ streamed_mutation::forwarding fwd() const { return _fwd; }
+};
+
+}
diff --git a/row_cache.hh b/row_cache.hh
index c033bde..9e58114 100644
--- a/row_cache.hh
+++ b/row_cache.hh
@@ -41,6 +41,12 @@ namespace bi = boost::intrusive;

class row_cache;

+namespace cache {
+
+class read_context;
+
+}
+
// Intrusive set entry which holds partition data.
//
// TODO: Make memtables use this format too.
@@ -102,7 +108,7 @@ class cache_entry {
partition_entry& partition() { return _pe; }
const schema_ptr& schema() const { return _schema; }
schema_ptr& schema() { return _schema; }
- streamed_mutation read(row_cache&, const schema_ptr&, const query::partition_slice&, streamed_mutation::forwarding);
+ streamed_mutation read(row_cache&, cache::read_context& reader);
bool continuous() const { return _flags._continuous; }
void set_continuous(bool value) { _flags._continuous = value; }

@@ -228,6 +234,7 @@ class row_cache final {
friend class autoupdating_underlying_reader;
friend class single_partition_populating_reader;
friend class cache_entry;
+ friend class cache::read_context;
public:
struct stats {
utils::timed_rate_moving_average hits;
@@ -253,12 +260,8 @@ class row_cache final {
logalloc::allocating_section _update_section;
logalloc::allocating_section _populate_section;
logalloc::allocating_section _read_section;
- mutation_reader make_scanning_reader(schema_ptr,
- const dht::partition_range&,
- const io_priority_class& pc,
- const query::partition_slice& slice,
- tracing::trace_state_ptr trace_state,
- streamed_mutation::forwarding);
+ mutation_reader create_underlying_reader(cache::read_context&, const dht::partition_range&);
+ mutation_reader make_scanning_reader(const dht::partition_range&, lw_shared_ptr<cache::read_context>);
void on_hit();
void on_miss();
void upgrade_entry(cache_entry&);
diff --git a/row_cache.cc b/row_cache.cc
index e4fd35c..eb6a359 100644
--- a/row_cache.cc
+++ b/row_cache.cc
@@ -32,14 +32,21 @@
#include <boost/version.hpp>
#include <sys/sdt.h>
#include "stdx.hh"
+#include "read_context.hh"

using namespace std::chrono_literals;
+using namespace cache;


static logging::logger clogger("cache");

thread_local seastar::thread_scheduling_group row_cache::_update_thread_scheduling_group(1ms, 0.2);

+mutation_reader
+row_cache::create_underlying_reader(read_context& ctx, const dht::partition_range& pr) {
+ return _underlying(_schema, pr, query::full_slice, ctx.pc(), ctx.trace_state(), streamed_mutation::forwarding::no);
+}
+
cache_tracker& global_cache_tracker() {
static thread_local cache_tracker instance;
return instance;
@@ -177,23 +184,16 @@ const logalloc::region& cache_tracker::region() const {
*/
class autoupdating_underlying_reader final {
row_cache& _cache;
+ read_context& _read_context;
stdx::optional<mutation_reader> _reader;
utils::phased_barrier::phase_type _reader_creation_phase;
- const query::partition_slice& _slice;
- const io_priority_class& _pc;
- tracing::trace_state_ptr _trace_state;
dht::partition_range _range = { };
stdx::optional<dht::decorated_key> _last_key;

public:
- autoupdating_underlying_reader(row_cache& cache,
- const query::partition_slice& slice,
- const io_priority_class& pc,
- tracing::trace_state_ptr trace_state)
+ autoupdating_underlying_reader(row_cache& cache, read_context& context)
: _cache(cache)
- , _slice(slice)
- , _pc(pc)
- , _trace_state(std::move(trace_state))
+ , _read_context(context)
{ }
future<streamed_mutation_opt> operator()() {
auto phase = _cache._populate_phaser.phase();
@@ -207,8 +207,7 @@ class autoupdating_underlying_reader final {
_range = std::move(*new_range);
_last_key = { };
}
- _reader = _cache._underlying(_cache._schema, _range, _slice, _pc, _trace_state,
- streamed_mutation::forwarding::no);
+ _reader = _cache.create_underlying_reader(_read_context, _range);
_reader_creation_phase = phase;
}
return (*_reader)().then([this] (auto&& smopt) {
@@ -225,8 +224,7 @@ class autoupdating_underlying_reader final {
if (_reader && _reader_creation_phase == phase) {
return _reader->fast_forward_to(_range);
}
- _reader = _cache._underlying(_cache._schema, _range, _slice, _pc, _trace_state,
- streamed_mutation::forwarding::no);
+ _reader = _cache.create_underlying_reader(_read_context, _range);
_reader_creation_phase = phase;
return make_ready_future<>();
}
@@ -241,31 +239,19 @@ class autoupdating_underlying_reader final {

// Reader which populates the cache using data from the delegate.
class single_partition_populating_reader final : public mutation_reader::impl {
- schema_ptr _schema;
row_cache& _cache;
mutation_source& _underlying;
mutation_reader _delegate;
- const io_priority_class _pc;
- const query::partition_slice& _slice;
- tracing::trace_state_ptr _trace_state;
- streamed_mutation::forwarding _fwd;
+ lw_shared_ptr<read_context> _read_context;
public:
- single_partition_populating_reader(schema_ptr s,
- row_cache& cache,
+ single_partition_populating_reader(row_cache& cache,
mutation_source& underlying,
mutation_reader delegate,
- const io_priority_class pc,
- const query::partition_slice& slice,
- tracing::trace_state_ptr trace_state,
- streamed_mutation::forwarding fwd)
- : _schema(std::move(s))
- , _cache(cache)
+ lw_shared_ptr<read_context> context)
+ : _cache(cache)
, _underlying(underlying)
, _delegate(std::move(delegate))
- , _pc(pc)
- , _slice(slice)
- , _trace_state(std::move(trace_state))
- , _fwd(fwd)
+ , _read_context(std::move(context))
{ }

virtual future<streamed_mutation_opt> operator()() override {
@@ -277,11 +263,11 @@ class single_partition_populating_reader final : public mutation_reader::impl {
return mutation_from_streamed_mutation(std::move(sm)).then([this, op = std::move(op)] (mutation_opt&& mo) {
if (mo) {
_cache.populate(*mo);
- mo->upgrade(_schema);
- auto ck_ranges = query::clustering_key_filter_ranges::get_ranges(*_schema, _slice, mo->key());
+ mo->upgrade(_read_context->schema());
+ auto ck_ranges = query::clustering_key_filter_ranges::get_ranges(*_read_context->schema(), _read_context->slice(), mo->key());
auto filtered_partition = mutation_partition(std::move(mo->partition()), *(mo->schema()), std::move(ck_ranges));
mo->partition() = std::move(filtered_partition);
- return make_ready_future<streamed_mutation_opt>(streamed_mutation_from_mutation(std::move(*mo), _fwd));
+ return make_ready_future<streamed_mutation_opt>(streamed_mutation_from_mutation(std::move(*mo), _read_context->fwd()));
}
return make_ready_future<streamed_mutation_opt>(streamed_mutation_opt());
});
@@ -304,7 +290,6 @@ void row_cache::on_miss() {
}

class just_cache_scanning_reader final {
- schema_ptr _schema;
row_cache& _cache;
row_cache::partitions_type::iterator _it;
row_cache::partitions_type::iterator _end;
@@ -312,9 +297,7 @@ class just_cache_scanning_reader final {
stdx::optional<dht::decorated_key> _last;
uint64_t _last_reclaim_count;
size_t _last_modification_count;
- const query::partition_slice& _slice;
- const io_priority_class _pc;
- streamed_mutation::forwarding _fwd;
+ read_context& _read_context;
private:
void update_iterators() {
auto cmp = cache_entry::compare(_cache._schema);
@@ -355,13 +338,12 @@ class just_cache_scanning_reader final {
streamed_mutation_opt mut;
bool continuous;
};
- just_cache_scanning_reader(schema_ptr s,
- row_cache& cache,
- const dht::partition_range& range,
- const query::partition_slice& slice,
- const io_priority_class& pc,
- streamed_mutation::forwarding fwd)
- : _schema(std::move(s)), _cache(cache), _range(&range), _slice(slice), _pc(pc), _fwd(fwd)
+ just_cache_scanning_reader(row_cache& cache,
+ const dht::partition_range& range,
+ read_context& ctx)
+ : _cache(cache)
+ , _range(&range)
+ , _read_context(ctx)
{ }
future<cache_data> operator()() {
return _cache._read_section(_cache._tracker.region(), [this] {
@@ -377,7 +359,7 @@ class just_cache_scanning_reader final {
_cache._tracker.touch(ce);
_cache.on_hit();
cache_data cd { { }, ce.continuous() };
- cd.mut = ce.read(_cache, _schema, _slice, _fwd);
+ cd.mut = ce.read(_cache, _read_context);
return make_ready_future<cache_data>(std::move(cd));
});
});
@@ -391,11 +373,9 @@ class just_cache_scanning_reader final {

class range_populating_reader {
row_cache& _cache;
- schema_ptr _schema;
autoupdating_underlying_reader _reader;
- const query::partition_slice& _slice;
row_cache::previous_entry_pointer _last_key;
- streamed_mutation::forwarding _fwd;
+ read_context& _read_context;
private:
void handle_end_of_stream() {
if (_last_key._populate_phase != _reader.creation_phase()) {
@@ -420,19 +400,11 @@ class range_populating_reader {
}
}
public:
- range_populating_reader(
- row_cache& cache,
- schema_ptr schema,
- const query::partition_slice& slice,
- const io_priority_class& pc,
- tracing::trace_state_ptr trace_state,
- streamed_mutation::forwarding fwd)
+ range_populating_reader(row_cache& cache, read_context& ctx)
: _cache(cache)
- , _schema(std::move(schema))
- , _reader(cache, slice, pc, std::move(trace_state))
- , _slice(slice)
- , _fwd(fwd)
- { }
+ , _reader(cache, ctx)
+ , _read_context(ctx)
+ {}

future<streamed_mutation_opt> operator()() {
return _reader().then([this, op = _cache._populate_phaser.start()] (streamed_mutation_opt smopt) mutable {
@@ -447,11 +419,11 @@ class range_populating_reader {
_cache.populate(*mo, &_last_key);
_last_key.reset(mo->decorated_key(), _reader.creation_phase());

- mo->upgrade(_schema);
- auto ck_ranges = query::clustering_key_filter_ranges::get_ranges(*_schema, _slice, mo->key());
+ mo->upgrade(_read_context.schema());
+ auto ck_ranges = query::clustering_key_filter_ranges::get_ranges(*_read_context.schema(), _read_context.slice(), mo->key());
auto filtered_partition = mutation_partition(std::move(mo->partition()), *mo->schema(), std::move(ck_ranges));
mo->partition() = std::move(filtered_partition);
- return make_ready_future<streamed_mutation_opt>(streamed_mutation_from_mutation(std::move(*mo), _fwd));
+ return make_ready_future<streamed_mutation_opt>(streamed_mutation_from_mutation(std::move(*mo), _read_context.fwd()));
});
});
}
@@ -473,10 +445,9 @@ class range_populating_reader {

class scanning_and_populating_reader final : public mutation_reader::impl {
const dht::partition_range* _pr;
- schema_ptr _schema;
+ lw_shared_ptr<read_context> _read_context;
just_cache_scanning_reader _primary_reader;
range_populating_reader _secondary_reader;
- streamed_mutation::forwarding _fwd;
streamed_mutation_opt _next_primary;
bool _secondary_in_progress = false;
bool _first_element = true;
@@ -492,7 +463,7 @@ class scanning_and_populating_reader final : public mutation_reader::impl {
if (!_first_element) {
return false;
}
- return _pr->start() && _pr->start()->is_inclusive() && _pr->start()->value().equal(*_schema, dk);
+ return _pr->start() && _pr->start()->is_inclusive() && _pr->start()->value().equal(*_read_context->schema(), dk);
}

future<streamed_mutation_opt> read_from_primary() {
@@ -510,7 +481,7 @@ class scanning_and_populating_reader final : public mutation_reader::impl {
if (!_last_key) {
secondary_range = *_pr;
} else {
- dht::ring_position_comparator cmp(*_schema);
+ dht::ring_position_comparator cmp(*_read_context->schema());
auto&& new_range = _pr->split_after(*_last_key, cmp);
if (!new_range) {
return make_ready_future<streamed_mutation_opt>();
@@ -549,18 +520,13 @@ class scanning_and_populating_reader final : public mutation_reader::impl {
});
}
public:
- scanning_and_populating_reader(schema_ptr s,
- row_cache& cache,
- const dht::partition_range& range,
- const query::partition_slice& slice,
- const io_priority_class& pc,
- tracing::trace_state_ptr trace_state,
- streamed_mutation::forwarding fwd)
+ scanning_and_populating_reader(row_cache& cache,
+ const dht::partition_range& range,
+ lw_shared_ptr<read_context> context)
: _pr(&range)
- , _schema(s)
- , _primary_reader(s, cache, range, slice, pc, fwd)
- , _secondary_reader(cache, s, slice, pc, trace_state, fwd)
- , _fwd(fwd)
+ , _read_context(std::move(context))
+ , _primary_reader(cache, range, *_read_context)
+ , _secondary_reader(cache, *_read_context)
{ }

future<streamed_mutation_opt> operator()() {
@@ -580,13 +546,8 @@ class scanning_and_populating_reader final : public mutation_reader::impl {
};

mutation_reader
-row_cache::make_scanning_reader(schema_ptr s,
- const dht::partition_range& range,
- const io_priority_class& pc,
- const query::partition_slice& slice,
- tracing::trace_state_ptr trace_state,
- streamed_mutation::forwarding fwd) {
- return make_mutation_reader<scanning_and_populating_reader>(std::move(s), *this, range, slice, pc, std::move(trace_state), fwd);
+row_cache::make_scanning_reader(const dht::partition_range& range, lw_shared_ptr<read_context> context) {
+ return make_mutation_reader<scanning_and_populating_reader>(*this, range, std::move(context));
}

mutation_reader
@@ -595,12 +556,15 @@ row_cache::make_reader(schema_ptr s,
const query::partition_slice& slice,
const io_priority_class& pc,
tracing::trace_state_ptr trace_state,
- streamed_mutation::forwarding fwd) {
+ streamed_mutation::forwarding fwd)
+{
+ auto ctx = make_lw_shared<read_context>(*this, std::move(s), range, slice, pc, trace_state, fwd);
+
if (range.is_singular()) {
const query::ring_position& pos = range.start()->value();

if (!pos.has_key()) {
- return make_scanning_reader(std::move(s), range, pc, slice, std::move(trace_state), fwd);
+ return make_scanning_reader(range, std::move(ctx));
}

return _read_section(_tracker.region(), [&] {
@@ -612,12 +576,12 @@ row_cache::make_reader(schema_ptr s,
_tracker.touch(e);
upgrade_entry(e);
mutation_reader reader;
- reader = make_reader_returning(e.read(*this, s, slice, fwd));
+ reader = make_reader_returning(e.read(*this, *ctx));
on_hit();
return reader;
} else {
- auto reader = make_mutation_reader<single_partition_populating_reader>(s, *this, _underlying,
- _underlying(_schema, range, query::full_slice, pc, trace_state), pc, slice, trace_state, fwd);
+ auto reader = make_mutation_reader<single_partition_populating_reader>(*this, _underlying,
+ _underlying(_schema, range, query::full_slice, pc, trace_state), std::move(ctx));
on_miss();
return reader;
}
@@ -625,7 +589,7 @@ row_cache::make_reader(schema_ptr s,
});
}

- return make_scanning_reader(std::move(s), range, pc, slice, std::move(trace_state), fwd);
+ return make_scanning_reader(range, std::move(ctx));
}

row_cache::~row_cache() {
@@ -903,7 +867,10 @@ void row_cache::set_schema(schema_ptr new_schema) noexcept {
_schema = std::move(new_schema);
}

-streamed_mutation cache_entry::read(row_cache& rc, const schema_ptr& s, const query::partition_slice& slice, streamed_mutation::forwarding fwd) {
+streamed_mutation cache_entry::read(row_cache& rc, read_context& ctx) {
+ auto s = ctx.schema();
+ auto& slice = ctx.slice();
+ auto fwd = ctx.fwd();
if (_schema->version() != s->version()) {
auto ck_ranges = query::clustering_key_filter_ranges::get_ranges(*s, slice, _key.key());
auto mp = mutation_partition(_pe.squashed(_schema, s), *s, std::move(ck_ranges));
--
2.7.4

Tomasz Grabiec

<tgrabiec@scylladb.com>
unread,
Jun 12, 2017, 3:02:46 PM6/12/17
to scylladb-dev@googlegroups.com
---
mutation_reader.hh | 17 +++++++++++++++++
1 file changed, 17 insertions(+)

diff --git a/mutation_reader.hh b/mutation_reader.hh
index 9d4e26c..d458389 100644
--- a/mutation_reader.hh
+++ b/mutation_reader.hh
@@ -313,6 +313,23 @@ class mutation_source {
}
};

+// Represent mutation_source which can be snapshotted.
+class snapshot_source {
+private:
+ std::function<mutation_source()> _func;
+public:
+ snapshot_source(std::function<mutation_source()> func)
+ : _func(std::move(func))
+ { }
+
+ // Creates a new snapshot.
+ // The returned mutation_source represents all earlier writes and only those.
+ // Note though that the mutations in the snapshot may get compacted over time.
+ mutation_source operator()() {
+ return _func();
+ }
+};
+
template<>
struct move_constructor_disengages<mutation_source> {
enum { value = true };
--
2.7.4

Tomasz Grabiec

<tgrabiec@scylladb.com>
unread,
Jun 12, 2017, 3:02:46 PM6/12/17
to scylladb-dev@googlegroups.com
---
mutation_reader.hh | 3 +++
mutation_reader.cc | 17 +++++++++++++++++
2 files changed, 20 insertions(+)

diff --git a/mutation_reader.hh b/mutation_reader.hh
index d458389..f2fb787 100644
--- a/mutation_reader.hh
+++ b/mutation_reader.hh
@@ -330,6 +330,9 @@ class snapshot_source {
}
};

+mutation_source make_empty_mutation_source();
+snapshot_source make_empty_snapshot_source();
+
template<>
struct move_constructor_disengages<mutation_source> {
enum { value = true };
diff --git a/mutation_reader.cc b/mutation_reader.cc
index 33c3070..9704cce 100644
--- a/mutation_reader.cc
+++ b/mutation_reader.cc
@@ -321,3 +321,20 @@ make_multi_range_reader(schema_ptr s, mutation_source source, const dht::partiti
return make_mutation_reader<multi_range_mutation_reader>(std::move(s), std::move(source), ranges,
slice, pc, std::move(trace_state), fwd);
}
+
+snapshot_source make_empty_snapshot_source() {
+ return snapshot_source([] {
+ return make_empty_mutation_source();
+ });
+}
+
+mutation_source make_empty_mutation_source() {
+ return mutation_source([] (schema_ptr s,
+ const dht::partition_range& pr,
+ const query::partition_slice& slice,
+ const io_priority_class& pc,
+ tracing::trace_state_ptr tr,
+ streamed_mutation::forwarding fwd) {
+ return make_empty_reader();

Tomasz Grabiec

<tgrabiec@scylladb.com>
unread,
Jun 12, 2017, 3:02:47 PM6/12/17
to scylladb-dev@googlegroups.com
---
mutation_reader.hh | 6 ++++++
mutation_reader.cc | 17 ++++++++++++++++-
2 files changed, 22 insertions(+), 1 deletion(-)

diff --git a/mutation_reader.hh b/mutation_reader.hh
index f2fb787..1c4ce51 100644
--- a/mutation_reader.hh
+++ b/mutation_reader.hh
@@ -313,6 +313,12 @@ class mutation_source {
}
};

+// Returns a mutation_source which is the sum of given mutation_sources.
+//
+// Adding two mutation sources gives a mutation source which contains
+// the sum of writes contained in the addends.
+mutation_source make_combined_mutation_source(std::vector<mutation_source>);
+
// Represent mutation_source which can be snapshotted.
class snapshot_source {
private:
diff --git a/mutation_reader.cc b/mutation_reader.cc
index 9704cce..3319fc8 100644
--- a/mutation_reader.cc
+++ b/mutation_reader.cc
@@ -329,7 +329,7 @@ snapshot_source make_empty_snapshot_source() {
}

mutation_source make_empty_mutation_source() {
- return mutation_source([] (schema_ptr s,
+ return mutation_source([](schema_ptr s,
const dht::partition_range& pr,
const query::partition_slice& slice,
const io_priority_class& pc,
@@ -338,3 +338,18 @@ mutation_source make_empty_mutation_source() {
return make_empty_reader();
});
}
+
+mutation_source make_combined_mutation_source(std::vector<mutation_source> addends) {
+ return mutation_source([addends = std::move(addends)] (schema_ptr s,
+ const dht::partition_range& pr,
+ const query::partition_slice& slice,
+ const io_priority_class& pc,
+ tracing::trace_state_ptr tr,
+ streamed_mutation::forwarding fwd) {
+ std::vector<mutation_reader> rd;
+ for (auto&& ms : addends) {
+ rd.emplace_back(ms(s, pr, slice, pc, tr, fwd));
+ }
+ return make_combined_reader(std::move(rd));

Tomasz Grabiec

<tgrabiec@scylladb.com>
unread,
Jun 12, 2017, 3:02:48 PM6/12/17
to scylladb-dev@googlegroups.com
Cache readers will need to take snapshots by copying the
mutation_source. That's going to happen quite often, so make copying
cheaper.
---
mutation_reader.hh | 20 +++++++-------------
1 file changed, 7 insertions(+), 13 deletions(-)

diff --git a/mutation_reader.hh b/mutation_reader.hh
index 1c4ce51..a59b86a 100644
--- a/mutation_reader.hh
+++ b/mutation_reader.hh
@@ -264,37 +264,31 @@ class mutation_source {
// We could have our own version of std::function<> that is nothrow
// move constructible and save some indirection and allocation.
// Probably not worth the effort though.
- std::unique_ptr<func_type> _fn;
+ lw_shared_ptr<func_type> _fn;
private:
mutation_source() = default;
explicit operator bool() const { return bool(_fn); }
friend class optimized_optional<mutation_source>;
public:
- mutation_source(func_type fn) : _fn(std::make_unique<func_type>(std::move(fn))) {}
+ mutation_source(func_type fn) : _fn(make_lw_shared<func_type>(std::move(fn))) {}
mutation_source(std::function<mutation_reader(schema_ptr, partition_range, const query::partition_slice&, io_priority)> fn)
- : _fn(std::make_unique<func_type>([fn = std::move(fn)] (schema_ptr s, partition_range range, const query::partition_slice& slice, io_priority pc, tracing::trace_state_ptr, streamed_mutation::forwarding fwd) {
+ : _fn(make_lw_shared<func_type>([fn = std::move(fn)] (schema_ptr s, partition_range range, const query::partition_slice& slice, io_priority pc, tracing::trace_state_ptr, streamed_mutation::forwarding fwd) {
assert(!fwd);
return fn(s, range, slice, pc);
})) {}
mutation_source(std::function<mutation_reader(schema_ptr, partition_range, const query::partition_slice&)> fn)
- : _fn(std::make_unique<func_type>([fn = std::move(fn)] (schema_ptr s, partition_range range, const query::partition_slice& slice, io_priority, tracing::trace_state_ptr, streamed_mutation::forwarding fwd) {
+ : _fn(make_lw_shared<func_type>([fn = std::move(fn)] (schema_ptr s, partition_range range, const query::partition_slice& slice, io_priority, tracing::trace_state_ptr, streamed_mutation::forwarding fwd) {
assert(!fwd);
return fn(s, range, slice);
})) {}
mutation_source(std::function<mutation_reader(schema_ptr, partition_range range)> fn)
- : _fn(std::make_unique<func_type>([fn = std::move(fn)] (schema_ptr s, partition_range range, const query::partition_slice&, io_priority, tracing::trace_state_ptr, streamed_mutation::forwarding fwd) {
+ : _fn(make_lw_shared<func_type>([fn = std::move(fn)] (schema_ptr s, partition_range range, const query::partition_slice&, io_priority, tracing::trace_state_ptr, streamed_mutation::forwarding fwd) {
assert(!fwd);
return fn(s, range);
})) {}

- mutation_source(const mutation_source& other)
- : _fn(std::make_unique<func_type>(*other._fn)) { }
-
- mutation_source& operator=(const mutation_source& other) {
- _fn = std::make_unique<func_type>(*other._fn);
- return *this;
- }
-
+ mutation_source(const mutation_source& other) = default;
+ mutation_source& operator=(const mutation_source& other) = default;
mutation_source(mutation_source&&) = default;
mutation_source& operator=(mutation_source&&) = default;

--
2.7.4

Tomasz Grabiec

<tgrabiec@scylladb.com>
unread,
Jun 12, 2017, 3:02:49 PM6/12/17
to scylladb-dev@googlegroups.com
Snapshottable in-memory mutation source for use in row_cache tests.
---
tests/memtable_snapshot_source.hh | 129 ++++++++++++++++++++++++++++++++++++++
1 file changed, 129 insertions(+)
create mode 100644 tests/memtable_snapshot_source.hh

diff --git a/tests/memtable_snapshot_source.hh b/tests/memtable_snapshot_source.hh
new file mode 100644
index 0000000..9647701
--- /dev/null
+++ b/tests/memtable_snapshot_source.hh
@@ -0,0 +1,129 @@
+/*
+ * Copyright (C) 2017 ScyllaDB
+ */
+
+/*
+ * This file is part of Scylla.
+ *
+ * Scylla is free software: you can redistribute it and/or modify
+ * it under the terms of the GNU Affero General Public License as published by
+ * the Free Software Foundation, either version 3 of the License, or
+ * (at your option) any later version.
+ *
+ * Scylla is distributed in the hope that it will be useful,
+ * but WITHOUT ANY WARRANTY; without even the implied warranty of
+ * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
+ * GNU General Public License for more details.
+ *
+ * You should have received a copy of the GNU General Public License
+ * along with Scylla. If not, see <http://www.gnu.org/licenses/>.
+ */
+
+#pragma once
+
+#include "mutation_reader.hh"
+#include "memtable.hh"
+#include "utils/phased_barrier.hh"
+#include <seastar/core/circular_buffer.hh>
+#include <seastar/core/thread.hh>
+#include <seastar/core/condition-variable.hh>
+
+// in-memory snapshottable mutation source.
+// Must be destroyed in a seastar thread.
+class memtable_snapshot_source {
+ schema_ptr _s;
+ circular_buffer<lw_shared_ptr<memtable>> _memtables;
+ utils::phased_barrier _apply;
+ bool _closed = false;
+ seastar::condition_variable _should_compact;
+ future<> _compactor;
+private:
+ bool should_compact() const {
+ return !_closed && _memtables.size() >= 3;
+ }
+ lw_shared_ptr<memtable> new_memtable() {
+ return make_lw_shared<memtable>(_s);
+ }
+ lw_shared_ptr<memtable> pending() {
+ if (_memtables.empty()) {
+ _memtables.push_back(new_memtable());
+ on_new_memtable();
+ }
+ return _memtables.back();
+ }
+ void on_new_memtable() {
+ if (should_compact()) {
+ _should_compact.signal();
+ }
+ }
+ void compact() {
+ if (_memtables.empty()) {
+ return;
+ }
+ auto count = _memtables.size();
+ auto op = _apply.start();
+ auto new_mt = make_lw_shared<memtable>(_memtables.back()->schema());
+ std::vector<mutation_reader> readers;
+ for (auto&& mt : _memtables) {
+ readers.push_back(mt->make_reader(new_mt->schema()));
+ }
+ auto&& rd = make_combined_reader(std::move(readers));
+ consume(rd, [&] (mutation&& m) {
+ new_mt->apply(std::move(m));
+ return stop_iteration::no;
+ }).get();
+ _memtables.erase(_memtables.begin(), _memtables.begin() + count);
+ _memtables.push_back(new_mt);
+ }
+public:
+ memtable_snapshot_source(schema_ptr s)
+ : _s(s)
+ , _compactor(seastar::async([this] {
+ while (!_closed) {
+ _should_compact.wait().get();
+ while (should_compact()) {
+ compact();
+ }
+ }
+ }))
+ { }
+ memtable_snapshot_source(memtable_snapshot_source&&) = delete; // 'this' captured.
+ ~memtable_snapshot_source() {
+ _closed = true;
+ _should_compact.broadcast();
+ _compactor.get();
+ }
+ // Must run in a seastar thread
+ void clear() {
+ _memtables.erase(_memtables.begin(), _memtables.end());
+ _apply.advance_and_await().get();
+ _memtables.erase(_memtables.begin(), _memtables.end());
+ }
+ // Must run in a seastar thread
+ void apply(const mutation& mt) {
+ pending()->apply(mt);
+ }
+ // Must run in a seastar thread
+ void apply(memtable& mt) {
+ auto op = _apply.start();
+ auto new_mt = new_memtable();
+ new_mt->apply(mt).get();
+ _memtables.push_back(new_mt);
+ }
+ // Must run in a seastar thread
+ // mt must not change from now on.
+ void apply(lw_shared_ptr<memtable> mt) {
+ auto op = _apply.start();
+ _memtables.push_back(std::move(mt));
+ on_new_memtable();
+ }
+ mutation_source operator()() {
+ std::vector<mutation_source> src;
+ for (auto&& mt : _memtables) {
+ src.push_back(mt->as_data_source());
+ }
+ _memtables.push_back(new_memtable()); // so that src won't change any more.
+ on_new_memtable();
+ return make_combined_mutation_source(std::move(src));
+ }
+};
--
2.7.4

Tomasz Grabiec

<tgrabiec@scylladb.com>
unread,
Jun 12, 2017, 3:02:50 PM6/12/17
to scylladb-dev@googlegroups.com
---
database.hh | 4 +++-
database.cc | 29 +++++++++++++++++++++++------
2 files changed, 26 insertions(+), 7 deletions(-)

diff --git a/database.hh b/database.hh
index e873ab0..7b3ebfd 100644
--- a/database.hh
+++ b/database.hh
@@ -582,11 +582,12 @@ class column_family {
void remove_ancestors_needed_rewrite(std::unordered_set<uint64_t> ancestors);
private:
mutation_source_opt _virtual_reader;
- // Creates a mutation reader which covers sstables.
+ // Creates a mutation reader which covers given sstables.
// Caller needs to ensure that column_family remains live (FIXME: relax this).
// The 'range' parameter must be live as long as the reader is used.
// Mutations returned by the reader will all have given schema.
mutation_reader make_sstable_reader(schema_ptr schema,
+ lw_shared_ptr<sstables::sstable_set> sstables,
const dht::partition_range& range,
const query::partition_slice& slice,
const io_priority_class& pc,
@@ -594,6 +595,7 @@ class column_family {
streamed_mutation::forwarding fwd) const;

mutation_source sstables_as_mutation_source();
+ snapshot_source sstables_as_snapshot_source();
partition_presence_checker make_partition_presence_checker(lw_shared_ptr<sstables::sstable_set>);
std::chrono::steady_clock::time_point _sstable_writes_disabled_at;
void do_trigger_compaction();
diff --git a/database.cc b/database.cc
index 288540b..cd4d0ca 100644
--- a/database.cc
+++ b/database.cc
@@ -180,7 +180,23 @@ column_family::sstables_as_mutation_source() {
const io_priority_class& pc,
tracing::trace_state_ptr trace_state,
streamed_mutation::forwarding fwd) {
- return make_sstable_reader(std::move(s), r, slice, pc, std::move(trace_state), fwd);
+ return make_sstable_reader(std::move(s), _sstables, r, slice, pc, std::move(trace_state), fwd);
+ });
+}
+
+snapshot_source
+column_family::sstables_as_snapshot_source() {
+ return snapshot_source([this] () {
+ // FIXME: Will keep sstables on disk until next memtable flush. Make compaction force cache refresh.
+ auto sst_set = _sstables;
+ return mutation_source([this, sst_set = std::move(sst_set)] (schema_ptr s,
+ const dht::partition_range& r,
+ const query::partition_slice& slice,
+ const io_priority_class& pc,
+ tracing::trace_state_ptr trace_state,
+ streamed_mutation::forwarding fwd) {
+ return make_sstable_reader(std::move(s), sst_set, r, slice, pc, std::move(trace_state), fwd);
+ });
});
}

@@ -523,6 +539,7 @@ class single_key_sstable_reader final : public mutation_reader::impl {

mutation_reader
column_family::make_sstable_reader(schema_ptr s,
+ lw_shared_ptr<sstables::sstable_set> sstables,
const dht::partition_range& pr,
const query::partition_slice& slice,
const io_priority_class& pc,
@@ -548,11 +565,11 @@ column_family::make_sstable_reader(schema_ptr s,
if (dht::shard_of(pos.token()) != engine().cpu_id()) {
return make_empty_reader(); // range doesn't belong to this shard
}
- return restrict_reader(make_mutation_reader<single_key_sstable_reader>(const_cast<column_family*>(this), std::move(s), _sstables,
+ return restrict_reader(make_mutation_reader<single_key_sstable_reader>(const_cast<column_family*>(this), std::move(s), std::move(sstables),
_stats.estimated_sstable_per_read, pr, slice, pc, std::move(trace_state), fwd));
} else {
// range_sstable_reader is not movable so we need to wrap it
- return restrict_reader(make_mutation_reader<range_sstable_reader>(std::move(s), _sstables, pr, slice, pc, std::move(trace_state), fwd));
+ return restrict_reader(make_mutation_reader<range_sstable_reader>(std::move(s), std::move(sstables), pr, slice, pc, std::move(trace_state), fwd));
}
}

@@ -635,7 +652,7 @@ column_family::make_reader(schema_ptr s,
if (_config.enable_cache) {
readers.emplace_back(_cache.make_reader(s, range, slice, pc, std::move(trace_state), fwd));
} else {
- readers.emplace_back(make_sstable_reader(s, range, slice, pc, std::move(trace_state), fwd));
+ readers.emplace_back(make_sstable_reader(s, _sstables, range, slice, pc, std::move(trace_state), fwd));
}

return make_combined_reader(std::move(readers));
@@ -654,7 +671,7 @@ column_family::make_streaming_reader(schema_ptr s,
readers.emplace_back(mt->make_reader(s, range, slice, pc, nullptr, streamed_mutation::forwarding::no));
}

- readers.emplace_back(make_sstable_reader(s, range, slice, pc, nullptr, streamed_mutation::forwarding::no));
+ readers.emplace_back(make_sstable_reader(s, _sstables, range, slice, pc, nullptr, streamed_mutation::forwarding::no));

return make_combined_reader(std::move(readers));
}
@@ -672,7 +689,7 @@ column_family::make_streaming_reader(schema_ptr s,
for (auto&& mt : *_memtables) {
readers.emplace_back(mt->make_reader(s, range, slice, pc, trace_state, fwd));
}
- readers.emplace_back(make_sstable_reader(s, range, slice, pc, std::move(trace_state), fwd));
+ readers.emplace_back(make_sstable_reader(s, _sstables, range, slice, pc, std::move(trace_state), fwd));
return make_combined_reader(std::move(readers));
});

--
2.7.4

Tomasz Grabiec

<tgrabiec@scylladb.com>
unread,
Jun 12, 2017, 3:02:51 PM6/12/17
to scylladb-dev@googlegroups.com
1) Reduce duplication by delegating to more general overloads

2) Improve documentation to not mention effects in terms of
population (detail) but rather write visibiliy

3) Rename clear() to invalidate() and merge with the range variant,
it has the same semantics
---
row_cache.hh | 30 ++++++++++--------------------
database.cc | 14 +++++---------
row_cache.cc | 24 +++++++++---------------
tests/row_cache_test.cc | 10 +++++-----
4 files changed, 29 insertions(+), 49 deletions(-)

diff --git a/row_cache.hh b/row_cache.hh
index 9e58114..58d3850 100644
--- a/row_cache.hh
+++ b/row_cache.hh
@@ -320,11 +320,6 @@ class row_cache final {
// information there is for its partition in the underlying data sources.
void populate(const mutation& m, const previous_entry_pointer* previous = nullptr);

- // Clears the cache.
- // Guarantees that cache will not be populated using readers created
- // before this method was invoked.
- future<> clear();
-
// Synchronizes cache with the underlying data source from a memtable which
// has just been flushed to the underlying data source.
// The memtable can be queried during the process, but must not be written.
@@ -334,22 +329,17 @@ class row_cache final {
// Moves given partition to the front of LRU if present in cache.
void touch(const dht::decorated_key&);

- // Removes given partition from cache.
- //
- // Guarantees that cache will not be populated with given key
- // using readers created before this method was invoked.
- //
- // The key must be kept alive until method resolves.
- future<> invalidate(const dht::decorated_key& key);
-
- // Removes given range of partitions from cache.
- // The range can be a wrap around.
- //
- // Guarantees that cache will not be populated with partitions from that range
- // using readers created before this method was invoked.
+ // Synchronizes cache with the underlying mutation source
+ // by invalidating ranges which were modified. This will force
+ // them to be re-read from the underlying mutation source
+ // during next read overlapping with the invalidated ranges.
//
- // The range must be kept alive until method resolves.
- future<> invalidate(const dht::partition_range&);
+ // Guarantees that readers created after invalidate()
+ // completes will see all writes from the underlying
+ // mutation source made prior to the call to invalidate().
+ future<> invalidate(const dht::decorated_key&);
+ future<> invalidate(const dht::partition_range& = query::full_partition_range);
+ future<> invalidate(dht::partition_range_vector&&);

auto num_entries() const {
return _partitions.size();
diff --git a/database.cc b/database.cc
index cd4d0ca..8dd61b5 100644
--- a/database.cc
+++ b/database.cc
@@ -1794,7 +1794,7 @@ future<> distributed_loader::load_new_sstables(distributed<database>& db, sstrin
cf.trigger_compaction();
// Drop entire cache for this column family because it may be populated
// with stale data.
- return cf.get_row_cache().clear();
+ return cf.get_row_cache().invalidate();
});
}).then([&db, ks, cf] () mutable {
return smp::submit_to(0, [&db, ks = std::move(ks), cf = std::move(cf)] () mutable {
@@ -3790,15 +3790,11 @@ future<> column_family::flush_streaming_mutations(utils::UUID plan_id, dht::part
return _streaming_memtables->seal_active_memtable(memtable_list::flush_behavior::delayed);
}).finally([this] {
return _streaming_flush_phaser.advance_and_await();
- }).finally([this, ranges = std::move(ranges)] {
+ }).finally([this, ranges = std::move(ranges)] () mutable {
if (!_config.enable_cache) {
return make_ready_future<>();
}
- return do_with(std::move(ranges), [this] (auto& ranges) {
- return parallel_for_each(ranges, [this](auto&& range) {
- return _cache.invalidate(range);
- });
- });
+ return _cache.invalidate(std::move(ranges));
});
});
}
@@ -3850,7 +3846,7 @@ future<> column_family::clear() {
_streaming_memtables->clear();
_streaming_memtables->add_memtable();
_streaming_memtables_big.clear();
- return _cache.clear();
+ return _cache.invalidate();
}

// NOTE: does not need to be futurized, but might eventually, depending on
@@ -3876,7 +3872,7 @@ future<db::replay_position> column_family::discard_sstables(db_clock::time_point

_sstables = std::move(pruned);
dblog.debug("cleaning out row cache");
- return _cache.clear().then([rp, remove = std::move(remove)] () mutable {
+ return _cache.invalidate().then([rp, remove = std::move(remove)] () mutable {
return parallel_for_each(remove, [](sstables::shared_sstable s) {
return sstables::delete_atomically({s});
}).then([rp] {
diff --git a/row_cache.cc b/row_cache.cc
index eb6a359..35c5c1b 100644
--- a/row_cache.cc
+++ b/row_cache.cc
@@ -660,10 +660,6 @@ void row_cache::populate(const mutation& m, const previous_entry_pointer* previo
});
}

-future<> row_cache::clear() {
- return invalidate(query::full_partition_range);
-}
-
future<> row_cache::update(memtable& m, partition_presence_checker presence_checker) {
m.on_detach_from_region_group();
_tracker.region().merge(m); // Now all data in memtable belongs to cache
@@ -781,21 +777,19 @@ void row_cache::invalidate_locked(const dht::decorated_key& dk) {
}

future<> row_cache::invalidate(const dht::decorated_key& dk) {
-return _populate_phaser.advance_and_await().then([this, &dk] {
- _read_section(_tracker.region(), [&] {
- with_allocator(_tracker.allocator(), [this, &dk] {
- with_linearized_managed_bytes([&] {
- invalidate_locked(dk);
- });
- });
- });
-});
+ return invalidate(dht::partition_range::make_singular(dk));
}

future<> row_cache::invalidate(const dht::partition_range& range) {
- return _populate_phaser.advance_and_await().then([this, &range] {
+ return invalidate(dht::partition_range_vector({range}));
+}
+
+future<> row_cache::invalidate(dht::partition_range_vector&& ranges) {
+ return _populate_phaser.advance_and_await().then([this, ranges = std::move(ranges)] () mutable {
with_linearized_managed_bytes([&] {
- invalidate_unwrapped(range);
+ for (auto&& range : ranges) {
+ this->invalidate_unwrapped(range);
+ }
});
});
}
diff --git a/tests/row_cache_test.cc b/tests/row_cache_test.cc
index e1d708f..427ae43 100644
--- a/tests/row_cache_test.cc
+++ b/tests/row_cache_test.cc
@@ -911,7 +911,7 @@ SEASTAR_TEST_CASE(test_continuity_flag_and_invalidate_race) {
.produces(ring[2]);

// Invalidate whole cache.
- cache.clear().get();
+ cache.invalidate();

rd.produces(ring[3])
.produces_end_of_stream();
@@ -1086,7 +1086,7 @@ SEASTAR_TEST_CASE(test_cache_population_and_clear_race) {
memtables.push_back(mt2);

// This update should miss on all partitions
- auto cache_cleared = cache.clear();
+ auto cache_cleared = cache.invalidate();

auto rd2 = cache.make_reader(s);

@@ -1184,7 +1184,7 @@ SEASTAR_TEST_CASE(test_mvcc) {
auto m_1 = mutation_from_streamed_mutation(std::move(sm1)).get0();
assert_that(*m_1).is_equal_to(m1);

- cache.clear().get0();
+ cache.invalidate().get0();

auto m_2 = mutation_from_streamed_mutation(std::move(sm2)).get0();
assert_that(*m_2).is_equal_to(m1);
@@ -1258,7 +1258,7 @@ SEASTAR_TEST_CASE(test_slicing_mutation_reader) {
row_cache cache(s, mt->as_data_source(), tracker);

auto run_tests = [&] (auto& ps, std::deque<int> expected) {
- cache.clear().get0();
+ cache.invalidate().get0();

auto reader = cache.make_reader(s, query::full_partition_range, ps);
test_sliced_read_row_presence(std::move(reader), s, expected);
@@ -1272,7 +1272,7 @@ SEASTAR_TEST_CASE(test_slicing_mutation_reader) {
reader = cache.make_reader(s, singular_range, ps);
test_sliced_read_row_presence(std::move(reader), s, expected);

- cache.clear().get0();
+ cache.invalidate().get0();

reader = cache.make_reader(s, singular_range, ps);
test_sliced_read_row_presence(std::move(reader), s, expected);
--
2.7.4

Tomasz Grabiec

<tgrabiec@scylladb.com>
unread,
Jun 12, 2017, 3:02:52 PM6/12/17
to scylladb-dev@googlegroups.com
This violation of the contract is currently benign, because there are
no reads from those tables before they are populated. If there were,
the cache would mark the whole (empty) range as continuous and the
table would appear empty.

It will cause similar problem once cache starts using snapshots of the
underlying mutation source. Then this lack of invalidate() will also
result in cache thinking that the table is still empty.
---
database.cc | 1 +
1 file changed, 1 insertion(+)

diff --git a/database.cc b/database.cc
index 8dd61b5..345cc9c 100644
--- a/database.cc
+++ b/database.cc
@@ -1827,6 +1827,7 @@ future<sstables::entry_descriptor> distributed_loader::probe_file(distributed<da
return cf.open_sstable(std::move(info), sstdir, comps.generation, comps.version, comps.format).then([&cf] (sstables::shared_sstable sst) mutable {
if (sst) {
cf.load_sstable(sst);
+ return cf.get_row_cache().invalidate();
}
return make_ready_future<>();
});
--
2.7.4

Tomasz Grabiec

<tgrabiec@scylladb.com>
unread,
Jun 12, 2017, 3:02:53 PM6/12/17
to scylladb-dev@googlegroups.com
Currently every time cache needs to create reader for missing data it
obtains a reader which is most up to date. That reader includes writes
from later populate phases, for which update() was not yet
called. This will be problematic once we allow partitions to be
partially populated, because different parts of the partition could be
partially populated using readers using different sets of writes, and break
write atomicity.

The solution will be to always populate given partition using the same
set of writes, using reader created from the current snapshot. The
snapshot changes only on update(), with update() gradually converting
each partition to the new snapshot.
---
row_cache.hh | 6 +-
database.cc | 2 +-
row_cache.cc | 12 +++-
tests/memory_footprint.cc | 2 +-
tests/perf_row_cache_update.cc | 2 +-
tests/row_cache_alloc_stress.cc | 4 +-
tests/row_cache_test.cc | 134 ++++++++++++++++++++++------------------
7 files changed, 92 insertions(+), 70 deletions(-)

diff --git a/row_cache.hh b/row_cache.hh
index 58d3850..fb0fc9f 100644
--- a/row_cache.hh
+++ b/row_cache.hh
@@ -245,7 +245,11 @@ class row_cache final {
stats _stats{};
schema_ptr _schema;
partitions_type _partitions; // Cached partitions are complete.
+
+ // Represents all mutations behind the cache.
+ // The set of mutations doesn't change within a single populate phase (it's a snapshot).
mutation_source _underlying;
+ snapshot_source _snapshot_source;

// Synchronizes populating reads with updates of underlying data source to ensure that cache
// remains consistent across flushes with the underlying data source.
@@ -298,7 +302,7 @@ class row_cache final {
}
public:
~row_cache();
- row_cache(schema_ptr, mutation_source underlying, cache_tracker&);
+ row_cache(schema_ptr, snapshot_source, cache_tracker&);
row_cache(row_cache&&) = default;
row_cache(const row_cache&) = delete;
row_cache& operator=(row_cache&&) = default;
diff --git a/database.cc b/database.cc
index 345cc9c..2332541 100644
--- a/database.cc
+++ b/database.cc
@@ -142,7 +142,7 @@ column_family::column_family(schema_ptr schema, config config, db::commitlog* cl
, _streaming_memtables(_config.enable_disk_writes ? make_streaming_memtable_list() : make_memory_only_memtable_list())
, _compaction_strategy(make_compaction_strategy(_schema->compaction_strategy(), _schema->compaction_strategy_options()))
, _sstables(make_lw_shared(_compaction_strategy.make_sstable_set(_schema)))
- , _cache(_schema, sstables_as_mutation_source(), global_cache_tracker())
+ , _cache(_schema, sstables_as_snapshot_source(), global_cache_tracker())
, _commitlog(cl)
, _compaction_manager(compaction_manager)
, _flush_queue(std::make_unique<memtable_flush_queue>())
diff --git a/row_cache.cc b/row_cache.cc
index 35c5c1b..c79e03d 100644
--- a/row_cache.cc
+++ b/row_cache.cc
@@ -690,6 +690,7 @@ future<> row_cache::update(memtable& m, partition_presence_checker presence_chec
}
});
});
+ _underlying = _snapshot_source();
_populate_phaser.advance_and_await().get();
while (!m.partitions.empty()) {
with_allocator(_tracker.allocator(), [this, &m, &presence_checker] () {
@@ -785,12 +786,16 @@ future<> row_cache::invalidate(const dht::partition_range& range) {
}

future<> row_cache::invalidate(dht::partition_range_vector&& ranges) {
- return _populate_phaser.advance_and_await().then([this, ranges = std::move(ranges)] () mutable {
+ auto f = _populate_phaser.advance_and_await();
+ _underlying = _snapshot_source();
+ return f.then([this, ranges = std::move(ranges)] () mutable {
+ auto on_failure = defer([this] { this->clear_now(); });
with_linearized_managed_bytes([&] {
for (auto&& range : ranges) {
this->invalidate_unwrapped(range);
}
});
+ on_failure.cancel();
});
}

@@ -824,11 +829,12 @@ void row_cache::invalidate_unwrapped(const dht::partition_range& range) {
});
}

-row_cache::row_cache(schema_ptr s, mutation_source fallback_factory, cache_tracker& tracker)
+row_cache::row_cache(schema_ptr s, snapshot_source src, cache_tracker& tracker)
: _tracker(tracker)
, _schema(std::move(s))
, _partitions(cache_entry::compare(_schema))
- , _underlying(std::move(fallback_factory))
+ , _underlying(src())
+ , _snapshot_source(std::move(src))
{
with_allocator(_tracker.allocator(), [this] {
cache_entry* entry = current_allocator().construct<cache_entry>(cache_entry::dummy_entry_tag());
diff --git a/tests/memory_footprint.cc b/tests/memory_footprint.cc
index 433ebe5..3a094de 100644
--- a/tests/memory_footprint.cc
+++ b/tests/memory_footprint.cc
@@ -175,7 +175,7 @@ static sizes calculate_sizes(const mutation& m) {
auto s = m.schema();
auto mt = make_lw_shared<memtable>(s);
cache_tracker tracker;
- row_cache cache(s, mt->as_data_source(), tracker);
+ row_cache cache(s, make_empty_snapshot_source(), tracker);

auto cache_initial_occupancy = tracker.region().occupancy().used_space();

diff --git a/tests/perf_row_cache_update.cc b/tests/perf_row_cache_update.cc
index 96ee48e..797c76b 100644
--- a/tests/perf_row_cache_update.cc
+++ b/tests/perf_row_cache_update.cc
@@ -73,7 +73,7 @@ int main(int argc, char** argv) {
.build();

cache_tracker tracker;
- row_cache cache(s, mutation_source([] (schema_ptr, auto&&) { return make_empty_reader(); }), tracker);
+ row_cache cache(s, make_empty_snapshot_source(), tracker);

size_t partitions = app.configuration()["partitions"].as<unsigned>();
size_t cell_size = app.configuration()["cell-size"].as<unsigned>();
diff --git a/tests/row_cache_alloc_stress.cc b/tests/row_cache_alloc_stress.cc
index 47e54f4..e3b15dd 100644
--- a/tests/row_cache_alloc_stress.cc
+++ b/tests/row_cache_alloc_stress.cc
@@ -70,10 +70,8 @@ int main(int argc, char** argv) {
.with_column("v", bytes_type, column_kind::regular_column)
.build();

- auto mt0 = make_lw_shared<memtable>(s);
-
cache_tracker tracker;
- row_cache cache(s, mt0->as_data_source(), tracker);
+ row_cache cache(s, make_empty_snapshot_source(), tracker);

auto mt = make_lw_shared<memtable>(s);
std::vector<dht::decorated_key> keys;
diff --git a/tests/row_cache_test.cc b/tests/row_cache_test.cc
index 427ae43..b7f2e6f 100644
--- a/tests/row_cache_test.cc
+++ b/tests/row_cache_test.cc
@@ -33,6 +33,7 @@
#include "core/thread.hh"
#include "memtable.hh"
#include "partition_slice_builder.hh"
+#include "tests/memtable_snapshot_source.hh"

#include "disk-error-handler.hh"

@@ -95,16 +96,33 @@ mutation make_new_mutation(schema_ptr s, int key) {
return make_new_mutation(s, partition_key::from_single_value(*s, to_bytes(sprint("key%d", key))));
}

+snapshot_source make_decorated_snapshot_source(snapshot_source src, std::function<mutation_source(mutation_source)> decorator) {
+ return snapshot_source([src = std::move(src), decorator = std::move(decorator)] () mutable {
+ return decorator(src());
+ });
+}
+
+mutation_source make_source_with(mutation m) {
+ return mutation_source([m] (schema_ptr s, const dht::partition_range&, const query::partition_slice&, const io_priority_class&, tracing::trace_state_ptr, streamed_mutation::forwarding fwd) {
+ assert(m.schema() == s);
+ return make_reader_returning(m, std::move(fwd));
+ });
+}
+
+// It is assumed that src won't change.
+snapshot_source snapshot_source_from_snapshot(mutation_source src) {
+ return snapshot_source([src = std::move(src)] {
+ return src;
+ });
+}
+
SEASTAR_TEST_CASE(test_cache_delegates_to_underlying) {
return seastar::async([] {
auto s = make_schema();
auto m = make_new_mutation(s);

cache_tracker tracker;
- row_cache cache(s, mutation_source([m] (schema_ptr s, const dht::partition_range&, const query::partition_slice&, const io_priority_class&, tracing::trace_state_ptr, streamed_mutation::forwarding fwd) {
- assert(m.schema() == s);
- return make_reader_returning(m, std::move(fwd));
- }), tracker);
+ row_cache cache(s, snapshot_source_from_snapshot(make_source_with(m)), tracker);

assert_that(cache.make_reader(s, query::full_partition_range))
.produces(m)
@@ -118,10 +136,7 @@ SEASTAR_TEST_CASE(test_cache_works_after_clearing) {
auto m = make_new_mutation(s);

cache_tracker tracker;
- row_cache cache(s, mutation_source([m] (schema_ptr s, const dht::partition_range&, const query::partition_slice&, const io_priority_class&, tracing::trace_state_ptr, streamed_mutation::forwarding fwd) {
- assert(m.schema() == s);
- return make_reader_returning(m, std::move(fwd));
- }), tracker);
+ row_cache cache(s, snapshot_source_from_snapshot(make_source_with(m)), tracker);

assert_that(cache.make_reader(s, query::full_partition_range))
.produces(m)
@@ -161,9 +176,9 @@ SEASTAR_TEST_CASE(test_cache_delegates_to_underlying_only_once_empty_full_range)
auto s = make_schema();
int secondary_calls_count = 0;
cache_tracker tracker;
- row_cache cache(s, mutation_source([&secondary_calls_count] (schema_ptr s, const dht::partition_range& range, const query::partition_slice&, const io_priority_class&, tracing::trace_state_ptr, streamed_mutation::forwarding fwd) {
+ row_cache cache(s, snapshot_source_from_snapshot(mutation_source([&secondary_calls_count] (schema_ptr s, const dht::partition_range& range, const query::partition_slice&, const io_priority_class&, tracing::trace_state_ptr, streamed_mutation::forwarding fwd) {
return make_counting_reader(make_empty_reader(), secondary_calls_count);
- }), tracker);
+ })), tracker);

assert_that(cache.make_reader(s, query::full_partition_range))
.produces_end_of_stream();
@@ -179,14 +194,14 @@ void test_cache_delegates_to_underlying_only_once_with_single_partition(schema_p
const dht::partition_range& range) {
int secondary_calls_count = 0;
cache_tracker tracker;
- row_cache cache(s, mutation_source([m, &secondary_calls_count] (schema_ptr s, const dht::partition_range& range, const query::partition_slice&, const io_priority_class&, tracing::trace_state_ptr, streamed_mutation::forwarding fwd) {
+ row_cache cache(s, snapshot_source_from_snapshot(mutation_source([m, &secondary_calls_count] (schema_ptr s, const dht::partition_range& range, const query::partition_slice&, const io_priority_class&, tracing::trace_state_ptr, streamed_mutation::forwarding fwd) {
assert(m.schema() == s);
if (range.contains(dht::ring_position(m.decorated_key()), dht::ring_position_comparator(*s))) {
return make_counting_reader(make_reader_returning(m, std::move(fwd)), secondary_calls_count);
} else {
return make_counting_reader(make_empty_reader(), secondary_calls_count);
}
- }), tracker);
+ })), tracker);

assert_that(cache.make_reader(s, range))
.produces(m)
@@ -280,7 +295,7 @@ SEASTAR_TEST_CASE(test_cache_delegates_to_underlying_only_once_multiple_mutation
return make_counting_reader(mt->as_data_source()(s, range, slice, pc, std::move(trace), std::move(fwd)), secondary_calls_count);
});

- return make_lw_shared<row_cache>(s, secondary, tracker);
+ return make_lw_shared<row_cache>(s, snapshot_source_from_snapshot(secondary), tracker);
};

auto make_ds = [&make_cache](schema_ptr s, int& secondary_calls_count) -> mutation_source {
@@ -449,7 +464,7 @@ SEASTAR_TEST_CASE(test_query_of_incomplete_range_goes_to_underlying) {
}

cache_tracker tracker;
- row_cache cache(s, mt->as_data_source(), tracker);
+ row_cache cache(s, snapshot_source_from_snapshot(mt->as_data_source()), tracker);

auto get_partition_range = [] (const mutation& m) {
return dht::partition_range::make_singular(query::ring_position(m.decorated_key()));
@@ -499,7 +514,7 @@ SEASTAR_TEST_CASE(test_single_key_queries_after_population_in_reverse_order) {
}

cache_tracker tracker;
- row_cache cache(s, mt->as_data_source(), tracker);
+ row_cache cache(s, snapshot_source_from_snapshot(mt->as_data_source()), tracker);

auto get_partition_range = [] (const mutation& m) {
return dht::partition_range::make_singular(query::ring_position(m.decorated_key()));
@@ -536,7 +551,7 @@ SEASTAR_TEST_CASE(test_row_cache_conforms_to_mutation_source) {
mt->apply(m);
}

- auto cache = make_lw_shared<row_cache>(s, mt->as_data_source(), tracker);
+ auto cache = make_lw_shared<row_cache>(s, snapshot_source_from_snapshot(mt->as_data_source()), tracker);
return mutation_source([cache] (schema_ptr s,
const dht::partition_range& range,
const query::partition_slice& slice,
@@ -555,7 +570,7 @@ SEASTAR_TEST_CASE(test_eviction) {
auto mt = make_lw_shared<memtable>(s);

cache_tracker tracker;
- row_cache cache(s, mt->as_data_source(), tracker);
+ row_cache cache(s, snapshot_source_from_snapshot(mt->as_data_source()), tracker);

std::vector<dht::decorated_key> keys;
for (int i = 0; i < 100000; i++) {
@@ -603,7 +618,7 @@ SEASTAR_TEST_CASE(test_update) {
auto cache_mt = make_lw_shared<memtable>(s);

cache_tracker tracker;
- row_cache cache(s, cache_mt->as_data_source(), tracker);
+ row_cache cache(s, snapshot_source_from_snapshot(cache_mt->as_data_source()), tracker);

BOOST_TEST_MESSAGE("Check cache miss with populate");

@@ -688,7 +703,7 @@ SEASTAR_TEST_CASE(test_update_failure) {
auto cache_mt = make_lw_shared<memtable>(s);

cache_tracker tracker;
- row_cache cache(s, cache_mt->as_data_source(), tracker);
+ row_cache cache(s, snapshot_source_from_snapshot(cache_mt->as_data_source()), tracker);

int partition_count = 1000;

@@ -797,7 +812,7 @@ class throttled_mutation_source {
private:
class impl : public enable_lw_shared_from_this<impl> {
mutation_source _underlying;
- ::throttle _throttle;
+ ::throttle& _throttle;
private:
class reader : public mutation_reader::impl {
throttle& _throttle;
@@ -819,35 +834,26 @@ class throttled_mutation_source {
}
};
public:
- impl(mutation_source underlying)
+ impl(::throttle& t, mutation_source underlying)
: _underlying(std::move(underlying))
+ , _throttle(t)
{ }

mutation_reader make_reader(schema_ptr s, const dht::partition_range& pr,
const query::partition_slice& slice, const io_priority_class& pc, tracing::trace_state_ptr trace, streamed_mutation::forwarding fwd) {
return make_mutation_reader<reader>(_throttle, _underlying(s, pr, slice, pc, std::move(trace), std::move(fwd)));
}
-
- ::throttle& throttle() { return _throttle; }
};
lw_shared_ptr<impl> _impl;
public:
- throttled_mutation_source(mutation_source underlying)
- : _impl(make_lw_shared<impl>(std::move(underlying)))
+ throttled_mutation_source(throttle& t, mutation_source underlying)
+ : _impl(make_lw_shared<impl>(t, std::move(underlying)))
{ }

- void block() {
- _impl->throttle().block();
- }
-
- void unblock() {
- _impl->throttle().unblock();
- }
-
operator mutation_source() const {
- return mutation_source([this] (schema_ptr s, const dht::partition_range& pr,
+ return mutation_source([impl = _impl] (schema_ptr s, const dht::partition_range& pr,
const query::partition_slice& slice, const io_priority_class& pc, tracing::trace_state_ptr trace, streamed_mutation::forwarding fwd) {
- return _impl->make_reader(std::move(s), pr, slice, pc, std::move(trace), std::move(fwd));
+ return impl->make_reader(std::move(s), pr, slice, pc, std::move(trace), std::move(fwd));
});
}
};
@@ -876,14 +882,14 @@ SEASTAR_TEST_CASE(test_continuity_flag_and_invalidate_race) {
auto s = make_schema();
lw_shared_ptr<memtable> mt = make_lw_shared<memtable>(s);

- cache_tracker tracker;
- row_cache cache(s, mt->as_data_source(), tracker);
-
auto ring = make_ring(s, 4);
for (auto&& m : ring) {
mt->apply(m);
}

+ cache_tracker tracker;
+ row_cache cache(s, snapshot_source_from_snapshot(mt->as_data_source()), tracker);
+
// Bring ring[2]and ring[3] to cache.
auto range = dht::partition_range::make_starting_with({ ring[2].ring_position(), true });
assert_that(cache.make_reader(s, range))
@@ -929,17 +935,21 @@ SEASTAR_TEST_CASE(test_continuity_flag_and_invalidate_race) {
SEASTAR_TEST_CASE(test_cache_population_and_update_race) {
return seastar::async([] {
auto s = make_schema();
- std::vector<lw_shared_ptr<memtable>> memtables;
- throttled_mutation_source cache_source(make_mutation_source(memtables));
+ memtable_snapshot_source memtables(s);
+ throttle thr;
+ auto cache_source = make_decorated_snapshot_source(snapshot_source([&] { return memtables(); }), [&] (mutation_source src) {
+ return throttled_mutation_source(thr, std::move(src));
+ });
cache_tracker tracker;
- row_cache cache(s, cache_source, tracker);

auto mt1 = make_lw_shared<memtable>(s);
- memtables.push_back(mt1);
auto ring = make_ring(s, 3);
for (auto&& m : ring) {
mt1->apply(m);
}
+ memtables.apply(*mt1);
+
+ row_cache cache(s, cache_source, tracker);

auto mt2 = make_lw_shared<memtable>(s);
auto ring2 = updated_ring(ring);
@@ -947,7 +957,7 @@ SEASTAR_TEST_CASE(test_cache_population_and_update_race) {
mt2->apply(m);
}

- cache_source.block();
+ thr.block();

auto m0_range = dht::partition_range::make_singular(ring[0].ring_position());
auto rd1 = cache.make_reader(s, m0_range);
@@ -957,9 +967,8 @@ SEASTAR_TEST_CASE(test_cache_population_and_update_race) {
auto rd2_result = rd2();

sleep(10ms).get();
- auto mt2_flushed = make_lw_shared<memtable>(s);
- mt2_flushed->apply(*mt2).get();
- memtables.push_back(mt2_flushed);
+
+ memtables.apply(*mt2);

// This update should miss on all partitions
auto update_future = cache.update(*mt2, make_default_partition_presence_checker());
@@ -967,7 +976,7 @@ SEASTAR_TEST_CASE(test_cache_population_and_update_race) {
auto rd3 = cache.make_reader(s);

// rd2, which is in progress, should not prevent forward progress of update()
- cache_source.unblock();
+ thr.unblock();
update_future.get();

// Reads started before memtable flush should return previous value, otherwise this test
@@ -1002,7 +1011,7 @@ SEASTAR_TEST_CASE(test_invalidate) {
auto mt = make_lw_shared<memtable>(s);

cache_tracker tracker;
- row_cache cache(s, mt->as_data_source(), tracker);
+ row_cache cache(s, snapshot_source_from_snapshot(mt->as_data_source()), tracker);

int partition_count = 1000;

@@ -1057,17 +1066,21 @@ SEASTAR_TEST_CASE(test_invalidate) {
SEASTAR_TEST_CASE(test_cache_population_and_clear_race) {
return seastar::async([] {
auto s = make_schema();
- std::vector<lw_shared_ptr<memtable>> memtables;
- throttled_mutation_source cache_source(make_mutation_source(memtables));
+ memtable_snapshot_source memtables(s);
+ throttle thr;
+ auto cache_source = make_decorated_snapshot_source(snapshot_source([&] { return memtables(); }), [&] (mutation_source src) {
+ return throttled_mutation_source(thr, std::move(src));
+ });
cache_tracker tracker;
- row_cache cache(s, cache_source, tracker);

auto mt1 = make_lw_shared<memtable>(s);
- memtables.push_back(mt1);
auto ring = make_ring(s, 3);
for (auto&& m : ring) {
mt1->apply(m);
}
+ memtables.apply(*mt1);
+
+ row_cache cache(s, std::move(cache_source), tracker);

auto mt2 = make_lw_shared<memtable>(s);
auto ring2 = updated_ring(ring);
@@ -1075,7 +1088,7 @@ SEASTAR_TEST_CASE(test_cache_population_and_clear_race) {
mt2->apply(m);
}

- cache_source.block();
+ thr.block();

auto rd1 = cache.make_reader(s);
auto rd1_result = rd1();
@@ -1083,7 +1096,7 @@ SEASTAR_TEST_CASE(test_cache_population_and_clear_race) {
sleep(10ms).get();

memtables.clear();
- memtables.push_back(mt2);
+ memtables.apply(*mt2);

// This update should miss on all partitions
auto cache_cleared = cache.invalidate();
@@ -1091,7 +1104,7 @@ SEASTAR_TEST_CASE(test_cache_population_and_clear_race) {
auto rd2 = cache.make_reader(s);

// rd1, which is in progress, should not prevent forward progress of clear()
- cache_source.unblock();
+ thr.unblock();
cache_cleared.get();

// Reads started before memtable flush should return previous value, otherwise this test
@@ -1125,11 +1138,11 @@ SEASTAR_TEST_CASE(test_mvcc) {
auto test = [&] (const mutation& m1, const mutation& m2, bool with_active_memtable_reader) {
auto s = m1.schema();

- auto mt = make_lw_shared<memtable>(s);
+ memtable_snapshot_source underlying(s);
partition_key::equality eq(*s);

cache_tracker tracker;
- row_cache cache(s, mt->as_data_source(), tracker);
+ row_cache cache(s, snapshot_source([&] { return underlying(); }), tracker);

auto pk = m1.key();
cache.populate(m1);
@@ -1146,7 +1159,6 @@ SEASTAR_TEST_CASE(test_mvcc) {
mt1->apply(m2);

auto m12 = m1 + m2;
- mt->apply(m2);

stdx::optional<mutation_reader> mt1_reader_opt;
stdx::optional<streamed_mutation_opt> mt1_reader_sm_opt;
@@ -1156,7 +1168,9 @@ SEASTAR_TEST_CASE(test_mvcc) {
BOOST_REQUIRE(*mt1_reader_sm_opt);
}

+ underlying.apply(*mt1);
cache.update(*mt1, make_default_partition_presence_checker()).get();
+
auto sm3 = cache.make_reader(s)().get0();
BOOST_REQUIRE(sm3);
BOOST_REQUIRE(eq(sm3->key(), pk));
@@ -1255,7 +1269,7 @@ SEASTAR_TEST_CASE(test_slicing_mutation_reader) {
mt->apply(m);

cache_tracker tracker;
- row_cache cache(s, mt->as_data_source(), tracker);
+ row_cache cache(s, snapshot_source_from_snapshot(mt->as_data_source()), tracker);

auto run_tests = [&] (auto& ps, std::deque<int> expected) {
cache.invalidate().get0();
@@ -1330,7 +1344,7 @@ SEASTAR_TEST_CASE(test_lru) {
auto cache_mt = make_lw_shared<memtable>(s);

cache_tracker tracker;
- row_cache cache(s, cache_mt->as_data_source(), tracker);
+ row_cache cache(s, snapshot_source_from_snapshot(cache_mt->as_data_source()), tracker);

int partition_count = 10;

--
2.7.4

Tomasz Grabiec

<tgrabiec@scylladb.com>
unread,
Jun 12, 2017, 3:02:54 PM6/12/17
to scylladb-dev@googlegroups.com
---
dht/i_partitioner.hh | 8 ++++++++
1 file changed, 8 insertions(+)

diff --git a/dht/i_partitioner.hh b/dht/i_partitioner.hh
index a568b90..e6fcc69 100644
--- a/dht/i_partitioner.hh
+++ b/dht/i_partitioner.hh
@@ -374,6 +374,14 @@ class ring_position {
token_bound _token_bound; // valid when !_key
std::experimental::optional<partition_key> _key;
public:
+ static ring_position min() {
+ return { minimum_token(), token_bound::start };
+ }
+
+ static ring_position max() {
+ return { maximum_token(), token_bound::end };
+ }
+
static ring_position starting_at(dht::token token) {
return { std::move(token), token_bound::start };
}
--
2.7.4

Tomasz Grabiec

<tgrabiec@scylladb.com>
unread,
Jun 12, 2017, 3:02:55 PM6/12/17
to scylladb-dev@googlegroups.com
Currently readers are always using the latest snapshot. This is fine
for respecting write atomicity if partitions are fully continuous in
cache (now), but will break write atomicity once partial population is
allowed.

Consider the following case:

flush write(ck=1), write(ck=2) -> snapshot_1
cache reader 1 reads and inserts ck=1 @snapshot_1
flush write(ck=1), write(ck=2) -> snapshot_2
cache reader 2 reads and inserts ck=2 @snapshot_2

Because cache update is not atomic, it can happen that reader 2 will
complete while the partition hasn't been updated yet for snapshot_2.
In such case, after read 2 the partition would contain ck=1 from
snapshot_1 and ck=2 from snapshot_2. It will match neither of the
snapshots, and this could violate write atomicity.

To solve this problem we conceptually assign each partition key in the
ring to its current snapshot which it reflects. The update process
gradually converts entries in ring order to the new snapshot. Reads
will not be using the latest snapshot, but rather the current snapshot
for the position in the ring they are at.

There is a race between the update process and populating reads. Since
after the update all entries must reflect the new snapshot, reads
using the old snapshot cannot be allowed to insert data which can no
longer be reached by the update process. Before this patch this race
was prevented by the use of a phased_barrier, where readers would keep
phased_barrier::operation alive between starting a read of a partition
and inserting it into cache. Cache update was waiting for all prior
operations before starting the update. Any later read which was not
waited for would use the latest snapshot for reads, so the update
process didn't have to fix anything up for such reads.

After this change, later reads cannot always use the latest snapshot,
they have to use the snapshot corresponding to given entry. So it's
not enough for update() to wait for prior reads in order to prevent
stale populations. The (simple) solution implemented in this patch is
to detect the conflict and abandon population of given sub-range. In
general, reads are allowed to populate given range only if it belongs
to a single snapshot.

Note that the range here is not the whole query range. For population
of continuity, it is the range starting after the previous key and
ending after the key being inserted. When populating a partition
entry, the range is a singular range containing only the partition
key. Readers switch to new snapshots automatically as they move across
the ring. It's possible that the insertion of the partition doesn't
conflict, but continuity does. In such case the entry will be inserted
but continuity will not be set.
---
row_cache.hh | 87 ++++++++++++++++++++-------
row_cache.cc | 157 +++++++++++++++++++++++++++++++++---------------
tests/row_cache_test.cc | 13 ++--
3 files changed, 183 insertions(+), 74 deletions(-)

diff --git a/row_cache.hh b/row_cache.hh
index fb0fc9f..f0a6c17 100644
--- a/row_cache.hh
+++ b/row_cache.hh
@@ -183,6 +183,7 @@ class cache_tracker final {
uint64_t removals;
uint64_t partitions;
uint64_t modification_count;
+ uint64_t mispopulations;
};
private:
stats _stats{};
@@ -203,6 +204,7 @@ class cache_tracker final {
void on_hit();
void on_miss();
void on_miss_already_populated();
+ void on_mispopulate();
allocation_strategy& allocator();
logalloc::region& region();
const logalloc::region& region() const;
@@ -218,15 +220,15 @@ cache_tracker& global_cache_tracker();
// A data source which wraps another data source such that data obtained from the underlying data source
// is cached in-memory in order to serve queries faster.
//
-// To query the underlying data source through cache, use make_reader().
-//
// Cache populates itself automatically during misses.
//
-// Cache needs to be maintained externally so that it remains consistent with the underlying data source.
-// Any incremental change to the underlying data source should result in update() being called on cache.
+// Cache represents a snapshot of the underlying mutation source. When the
+// underlying mutation source changes, cache needs to be explicitly synchronized
+// to the latest snapshot. This is done by calling update() or invalidate().
//
class row_cache final {
public:
+ using phase_type = utils::phased_barrier::phase_type;
using partitions_type = bi::set<cache_entry,
bi::member_hook<cache_entry, cache_entry::cache_link_type, &cache_entry::_cache_link>,
bi::constant_time_size<false>, // we need this to have bi::auto_unlink on hooks
@@ -246,25 +248,45 @@ class row_cache final {
schema_ptr _schema;
partitions_type _partitions; // Cached partitions are complete.

- // Represents all mutations behind the cache.
- // The set of mutations doesn't change within a single populate phase (it's a snapshot).
+ // The snapshots used by cache are versioned. The version number of a snapshot is
+ // called the "population phase", or simply "phase". Between updates, cache
+ // represents the same snapshot.
+ //
+ // Update doesn't happen atomically. Before it completes, some entries reflect
+ // the old snapshot, while others reflect the new snapshot. After update
+ // completes, all entries must reflect the new snapshot. There is a race between the
+ // update process and populating reads. Since after the update all entries must
+ // reflect the new snapshot, reads using the old snapshot cannot be allowed to
+ // insert data which will no longer be reached by the update process. The whole
+ // range can be therefore divided into two sub-ranges, one which was already
+ // processed by the update and one which hasn't. Each key can be assigned a
+ // population phase which determines to which range it belongs, as well as which
+ // snapshot it reflects. The methods snapshot_of() and phase_of() can
+ // be used to determine this.
+ //
+ // In general, reads are allowed to populate given range only if the phase of the
+ // snapshot they use matches the phase of all keys in that range when the population
+ // is committed. This guarantees that the range will be reached by the update process
+ // or already has been in its entirety. Since the update process is a scan, it's sufficient
+ // to check if the start of the range has the same phase at the start and at the end of population.
+ // In case of phase conflict, current solution is to give up on population.
+
mutation_source _underlying;
+ phase_type _underlying_phase = 0;
+ mutation_source_opt _prev_snapshot;
+
+ // Positions >= than this are using _prev_snapshot, the rest is using _underlying.
+ stdx::optional<dht::ring_position> _prev_snapshot_pos;
+
snapshot_source _snapshot_source;

- // Synchronizes populating reads with updates of underlying data source to ensure that cache
- // remains consistent across flushes with the underlying data source.
- // Readers obtained from the underlying data source in earlier than
- // current phases must not be used to populate the cache, unless they hold
- // phaser::operation created in the reader's phase of origin. Readers
- // should hold to a phase only briefly because this inhibits progress of
- // updates. Phase changes occur in update()/clear(), which can be assumed to
- // be asynchronous wrt invoking of the underlying data source.
- utils::phased_barrier _populate_phaser;
+ // There can be at most one update in progress.
+ seastar::semaphore _update_sem = {1};

logalloc::allocating_section _update_section;
logalloc::allocating_section _populate_section;
logalloc::allocating_section _read_section;
- mutation_reader create_underlying_reader(cache::read_context&, const dht::partition_range&);
+ mutation_reader create_underlying_reader(cache::read_context&, mutation_source&, const dht::partition_range&);
mutation_reader make_scanning_reader(const dht::partition_range&, lw_shared_ptr<cache::read_context>);
void on_hit();
void on_miss();
@@ -275,13 +297,10 @@ class row_cache final {
static thread_local seastar::thread_scheduling_group _update_thread_scheduling_group;

struct previous_entry_pointer {
- utils::phased_barrier::phase_type _populate_phase;
stdx::optional<dht::decorated_key> _key;

- void reset(stdx::optional<dht::decorated_key> key, utils::phased_barrier::phase_type populate_phase) {
- _populate_phase = populate_phase;
- _key = std::move(key);
- }
+ previous_entry_pointer() = default; // Represents dht::ring_position_view::min()
+ previous_entry_pointer(dht::decorated_key key) : _key(std::move(key)) {};

// TODO: Currently inserting an entry to the cache increases
// modification counter. That doesn't seem to be necessary and if we
@@ -300,6 +319,27 @@ class row_cache final {
partitions_type::iterator partitions_end() {
return std::prev(_partitions.end());
}
+
+ // Only active phases are accepted.
+ // Reference valid only until next deferring point.
+ mutation_source& snapshot_for_phase(phase_type);
+
+ // Returns population phase for given position in the ring.
+ // snapshot_for_phase() can be called to obtain mutation_source for given phase, but
+ // only until the next deferring point.
+ // Should be only called outside update().
+ phase_type phase_of(dht::ring_position_view);
+
+ struct snapshot_and_phase {
+ mutation_source& snapshot;
+ phase_type phase;
+ };
+
+ // Optimized version of:
+ //
+ // { snapshot_for_phase(phase_of(pos)), phase_of(pos) };
+ //
+ snapshot_and_phase snapshot_of(dht::ring_position_view pos);
public:
~row_cache();
row_cache(schema_ptr, snapshot_source, cache_tracker&);
@@ -338,6 +378,11 @@ class row_cache final {
// them to be re-read from the underlying mutation source
// during next read overlapping with the invalidated ranges.
//
+ // The ranges passed to invalidate() must include all
+ // data which changed since last synchronization. Failure
+ // to do so may result in reads seeing partial writes,
+ // which would violate write atomicity.
+ //
// Guarantees that readers created after invalidate()
// completes will see all writes from the underlying
// mutation source made prior to the call to invalidate().
diff --git a/row_cache.cc b/row_cache.cc
index c79e03d..cda64d4 100644
--- a/row_cache.cc
+++ b/row_cache.cc
@@ -43,8 +43,8 @@ static logging::logger clogger("cache");
thread_local seastar::thread_scheduling_group row_cache::_update_thread_scheduling_group(1ms, 0.2);

mutation_reader
-row_cache::create_underlying_reader(read_context& ctx, const dht::partition_range& pr) {
- return _underlying(_schema, pr, query::full_slice, ctx.pc(), ctx.trace_state(), streamed_mutation::forwarding::no);
+row_cache::create_underlying_reader(read_context& ctx, mutation_source& src, const dht::partition_range& pr) {
+ return src(_schema, pr, query::full_slice, ctx.pc(), ctx.trace_state(), streamed_mutation::forwarding::no);
}

cache_tracker& global_cache_tracker() {
@@ -103,6 +103,7 @@ cache_tracker::setup_metrics() {
sm::make_derive("total_operations_merges", sm::description("total number of operation merged"), _stats.merges),
sm::make_derive("total_operations_evictions", sm::description("total number of operation eviction"), _stats.evictions),
sm::make_derive("total_operations_removals", sm::description("total number of operation removals"), _stats.removals),
+ sm::make_derive("total_operations_mispopulations", sm::description("number of entries not inserted by reads"), _stats.mispopulations),
sm::make_gauge("objects_partitions", sm::description("total number of partition objects"), _stats.partitions)
});
}
@@ -162,6 +163,10 @@ void cache_tracker::on_miss() {
++_stats.misses;
}

+void cache_tracker::on_mispopulate() {
+ ++_stats.mispopulations;
+}
+
void cache_tracker::on_miss_already_populated() {
++_stats.concurrent_misses_same_key;
}
@@ -189,14 +194,16 @@ class autoupdating_underlying_reader final {
utils::phased_barrier::phase_type _reader_creation_phase;
dht::partition_range _range = { };
stdx::optional<dht::decorated_key> _last_key;
-
+ stdx::optional<dht::decorated_key> _new_last_key;
public:
autoupdating_underlying_reader(row_cache& cache, read_context& context)
: _cache(cache)
, _read_context(context)
{ }
future<streamed_mutation_opt> operator()() {
- auto phase = _cache._populate_phaser.phase();
+ _last_key = std::move(_new_last_key);
+ auto start = population_range_start();
+ auto phase = _cache.phase_of(start);
if (!_reader || _reader_creation_phase != phase) {
if (_last_key) {
auto cmp = dht::ring_position_comparator(*_cache._schema);
@@ -205,14 +212,15 @@ class autoupdating_underlying_reader final {
return make_ready_future<streamed_mutation_opt>(streamed_mutation_opt());
}
_range = std::move(*new_range);
- _last_key = { };
+ _last_key = {};
}
- _reader = _cache.create_underlying_reader(_read_context, _range);
+ auto& snap = _cache.snapshot_for_phase(phase);
+ _reader = _cache.create_underlying_reader(_read_context, snap, _range);
_reader_creation_phase = phase;
}
return (*_reader)().then([this] (auto&& smopt) {
if (smopt) {
- _last_key = smopt->decorated_key();
+ _new_last_key = smopt->decorated_key();
}
return std::move(smopt);
});
@@ -220,11 +228,12 @@ class autoupdating_underlying_reader final {
future<> fast_forward_to(dht::partition_range&& range) {
_range = std::move(range);
_last_key = { };
- auto phase = _cache._populate_phaser.phase();
+ _new_last_key = { };
+ auto phase = _cache.phase_of(dht::ring_position_view::for_range_start(_range));
if (_reader && _reader_creation_phase == phase) {
return _reader->fast_forward_to(_range);
}
- _reader = _cache.create_underlying_reader(_read_context, _range);
+ _reader = _cache.create_underlying_reader(_read_context, _cache.snapshot_for_phase(phase), _range);
_reader_creation_phase = phase;
return make_ready_future<>();
}
@@ -235,34 +244,44 @@ class autoupdating_underlying_reader final {
const dht::partition_range& range() const {
return _range;
}
+ dht::ring_position_view population_range_start() const {
+ return _last_key ? dht::ring_position_view::for_after_key(*_last_key)
+ : dht::ring_position_view::for_range_start(_range);
+ }
};

// Reader which populates the cache using data from the delegate.
class single_partition_populating_reader final : public mutation_reader::impl {
row_cache& _cache;
- mutation_source& _underlying;
mutation_reader _delegate;
lw_shared_ptr<read_context> _read_context;
+ bool done = false;
public:
single_partition_populating_reader(row_cache& cache,
- mutation_source& underlying,
- mutation_reader delegate,
lw_shared_ptr<read_context> context)
: _cache(cache)
- , _underlying(underlying)
- , _delegate(std::move(delegate))
, _read_context(std::move(context))
{ }

virtual future<streamed_mutation_opt> operator()() override {
- auto op = _cache._populate_phaser.start();
- return _delegate().then([this, op = std::move(op)] (auto sm) mutable {
+ if (done) {
+ return make_ready_future<streamed_mutation_opt>(streamed_mutation_opt());
+ }
+ done = true;
+ auto src_and_phase = _cache.snapshot_of(_read_context->range().start()->value());
+ auto phase = src_and_phase.phase;
+ _delegate = _cache.create_underlying_reader(*_read_context, src_and_phase.snapshot, _read_context->range());
+ return _delegate().then([this, phase] (auto sm) mutable {
if (!sm) {
return make_ready_future<streamed_mutation_opt>(streamed_mutation_opt());
}
- return mutation_from_streamed_mutation(std::move(sm)).then([this, op = std::move(op)] (mutation_opt&& mo) {
+ return mutation_from_streamed_mutation(std::move(sm)).then([this, phase] (mutation_opt&& mo) {
if (mo) {
- _cache.populate(*mo);
+ if (phase == _cache.phase_of(_read_context->range().start()->value())) {
+ _cache.populate(*mo);
+ } else {
+ _cache._tracker.on_mispopulate();
+ }
mo->upgrade(_read_context->schema());
auto ck_ranges = query::clustering_key_filter_ranges::get_ranges(*_read_context->schema(), _read_context->slice(), mo->key());
auto filtered_partition = mutation_partition(std::move(mo->partition()), *(mo->schema()), std::move(ck_ranges));
@@ -374,11 +393,14 @@ class just_cache_scanning_reader final {
class range_populating_reader {
row_cache& _cache;
autoupdating_underlying_reader _reader;
- row_cache::previous_entry_pointer _last_key;
+ stdx::optional<row_cache::previous_entry_pointer> _last_key;
read_context& _read_context;
private:
+ bool can_set_continuity() const {
+ return _last_key && _reader.creation_phase() == _cache.phase_of(_reader.population_range_start());
+ }
void handle_end_of_stream() {
- if (_last_key._populate_phase != _reader.creation_phase()) {
+ if (!can_set_continuity()) {
return;
}
if (!_reader.range().end() || !_reader.range().end()->is_inclusive()) {
@@ -387,12 +409,12 @@ class range_populating_reader {
: std::prev(_cache._partitions.end());
if (it != _cache._partitions.end()) {
if (it == _cache._partitions.begin()) {
- if (!_last_key._key) {
+ if (!_last_key->_key) {
it->set_continuous(true);
}
} else {
auto prev = std::prev(it);
- if (prev->key().equal(*_cache._schema, *_last_key._key)) {
+ if (prev->key().equal(*_cache._schema, *_last_key->_key)) {
it->set_continuous(true);
}
}
@@ -407,17 +429,21 @@ class range_populating_reader {
{}

future<streamed_mutation_opt> operator()() {
- return _reader().then([this, op = _cache._populate_phaser.start()] (streamed_mutation_opt smopt) mutable {
+ return _reader().then([this] (streamed_mutation_opt smopt) mutable {
return mutation_from_streamed_mutation(std::move(smopt)).then(
- [this, op = std::move(op)] (mutation_opt&& mo) mutable {
+ [this] (mutation_opt&& mo) mutable {
if (!mo) {
handle_end_of_stream();
return make_ready_future<streamed_mutation_opt>();
}

_cache.on_miss();
- _cache.populate(*mo, &_last_key);
- _last_key.reset(mo->decorated_key(), _reader.creation_phase());
+ if (_reader.creation_phase() == _cache.phase_of(mo->decorated_key())) {
+ _cache.populate(*mo, can_set_continuity() ? &*_last_key : nullptr);
+ } else {
+ _cache._tracker.on_mispopulate();
+ }
+ _last_key = mo->decorated_key();

mo->upgrade(_read_context.schema());
auto ck_ranges = query::clustering_key_filter_ranges::get_ranges(*_read_context.schema(), _read_context.slice(), mo->key());
@@ -429,14 +455,13 @@ class range_populating_reader {
}

future<> fast_forward_to(dht::partition_range&& pr) {
- auto phase = _cache._populate_phaser.phase();
if (!pr.start()) {
- _last_key.reset({ }, phase);
+ _last_key = row_cache::previous_entry_pointer();
} else if (!pr.start()->is_inclusive() && pr.start()->value().has_key()) {
- _last_key.reset(pr.start()->value().as_decorated_key(), phase);
+ _last_key = pr.start()->value().as_decorated_key();
} else {
// Inclusive start bound, cannot set continuity flag.
- _last_key.reset(stdx::nullopt, phase - 1);
+ _last_key = {};
}

return _reader.fast_forward_to(std::move(pr));
@@ -580,10 +605,8 @@ row_cache::make_reader(schema_ptr s,
on_hit();
return reader;
} else {
- auto reader = make_mutation_reader<single_partition_populating_reader>(*this, _underlying,
- _underlying(_schema, range, query::full_slice, pc, trace_state), std::move(ctx));
on_miss();
- return reader;
+ return make_mutation_reader<single_partition_populating_reader>(*this, std::move(ctx));
}
});
});
@@ -631,7 +654,7 @@ void row_cache::do_find_or_create_entry(const dht::decorated_key& key,
visit_entry(i);
}

- if (!previous || previous->_populate_phase != _populate_phaser.phase()) {
+ if (!previous) {
return;
}

@@ -660,6 +683,33 @@ void row_cache::populate(const mutation& m, const previous_entry_pointer* previo
});
}

+mutation_source& row_cache::snapshot_for_phase(phase_type phase) {
+ if (phase == _underlying_phase) {
+ return _underlying;
+ } else {
+ if (phase + 1 < _underlying_phase) {
+ throw std::runtime_error(sprint("attempted to read from retired phase {} (current={})", phase, _underlying_phase));
+ }
+ return *_prev_snapshot;
+ }
+}
+
+row_cache::snapshot_and_phase row_cache::snapshot_of(dht::ring_position_view pos) {
+ dht::ring_position_less_comparator less(*_schema);
+ if (!_prev_snapshot_pos || less(pos, *_prev_snapshot_pos)) {
+ return {_underlying, _underlying_phase};
+ }
+ return {*_prev_snapshot, _underlying_phase - 1};
+}
+
+row_cache::phase_type row_cache::phase_of(dht::ring_position_view pos) {
+ dht::ring_position_less_comparator less(*_schema);
+ if (!_prev_snapshot_pos || less(pos, *_prev_snapshot_pos)) {
+ return _underlying_phase;
+ }
+ return _underlying_phase - 1;
+}
+
future<> row_cache::update(memtable& m, partition_presence_checker presence_checker) {
m.on_detach_from_region_group();
_tracker.region().merge(m); // Now all data in memtable belongs to cache
@@ -690,8 +740,14 @@ future<> row_cache::update(memtable& m, partition_presence_checker presence_chec
}
});
});
- _underlying = _snapshot_source();
- _populate_phaser.advance_and_await().get();
+ auto permit = get_units(_update_sem, 1).get0();
+ ++_underlying_phase;
+ _prev_snapshot = std::exchange(_underlying, _snapshot_source());
+ _prev_snapshot_pos = dht::ring_position::min();
+ auto cleanup_prev_snapshot = defer([this] {
+ _prev_snapshot_pos = {};
+ _prev_snapshot = {};
+ });
while (!m.partitions.empty()) {
with_allocator(_tracker.allocator(), [this, &m, &presence_checker] () {
unsigned quota = 30;
@@ -736,6 +792,13 @@ future<> row_cache::update(memtable& m, partition_presence_checker presence_chec
});
STAP_PROBE(scylla, row_cache_update_partition_end);
} while (!m.partitions.empty() && quota && !need_preempt());
+ with_allocator(standard_allocator(), [&] {
+ if (m.partitions.empty()) {
+ _prev_snapshot_pos = {};
+ } else {
+ _prev_snapshot_pos = m.partitions.begin()->key();
+ }
+ });
STAP_PROBE1(scylla, row_cache_update_one_batch_end, quota_before - quota);
});
if (quota == 0 && seastar::thread::should_yield()) {
@@ -786,17 +849,17 @@ future<> row_cache::invalidate(const dht::partition_range& range) {
}

future<> row_cache::invalidate(dht::partition_range_vector&& ranges) {
- auto f = _populate_phaser.advance_and_await();
- _underlying = _snapshot_source();
- return f.then([this, ranges = std::move(ranges)] () mutable {
- auto on_failure = defer([this] { this->clear_now(); });
- with_linearized_managed_bytes([&] {
- for (auto&& range : ranges) {
- this->invalidate_unwrapped(range);
- }
- });
- on_failure.cancel();
- });
+ return get_units(_update_sem, 1).then([this, ranges = std::move(ranges)] (auto permit) mutable {
+ _underlying = _snapshot_source();
+ ++_underlying_phase;
+ auto on_failure = defer([this] { this->clear_now(); });
+ with_linearized_managed_bytes([&] {
+ for (auto&& range : ranges) {
+ this->invalidate_unwrapped(range);
+ }
+ });
+ on_failure.cancel();
+ });
}

void row_cache::invalidate_unwrapped(const dht::partition_range& range) {
diff --git a/tests/row_cache_test.cc b/tests/row_cache_test.cc
index b7f2e6f..de07ce2 100644
--- a/tests/row_cache_test.cc
+++ b/tests/row_cache_test.cc
@@ -191,7 +191,8 @@ SEASTAR_TEST_CASE(test_cache_delegates_to_underlying_only_once_empty_full_range)

void test_cache_delegates_to_underlying_only_once_with_single_partition(schema_ptr s,
const mutation& m,
- const dht::partition_range& range) {
+ const dht::partition_range& range,
+ int calls_to_secondary) {
int secondary_calls_count = 0;
cache_tracker tracker;
row_cache cache(s, snapshot_source_from_snapshot(mutation_source([m, &secondary_calls_count] (schema_ptr s, const dht::partition_range& range, const query::partition_slice&, const io_priority_class&, tracing::trace_state_ptr, streamed_mutation::forwarding fwd) {
@@ -206,11 +207,11 @@ void test_cache_delegates_to_underlying_only_once_with_single_partition(schema_p
assert_that(cache.make_reader(s, range))
.produces(m)
.produces_end_of_stream();
- BOOST_REQUIRE_EQUAL(secondary_calls_count, 2);
+ BOOST_REQUIRE_EQUAL(secondary_calls_count, calls_to_secondary);
assert_that(cache.make_reader(s, range))
.produces(m)
.produces_end_of_stream();
- BOOST_REQUIRE_EQUAL(secondary_calls_count, 2);
+ BOOST_REQUIRE_EQUAL(secondary_calls_count, calls_to_secondary);
}

SEASTAR_TEST_CASE(test_cache_delegates_to_underlying_only_once_single_key_range) {
@@ -218,7 +219,7 @@ SEASTAR_TEST_CASE(test_cache_delegates_to_underlying_only_once_single_key_range)
auto s = make_schema();
auto m = make_new_mutation(s);
test_cache_delegates_to_underlying_only_once_with_single_partition(s, m,
- dht::partition_range::make_singular(query::ring_position(m.decorated_key())));
+ dht::partition_range::make_singular(query::ring_position(m.decorated_key())), 1);
});
}

@@ -226,7 +227,7 @@ SEASTAR_TEST_CASE(test_cache_delegates_to_underlying_only_once_full_range) {
return seastar::async([] {
auto s = make_schema();
auto m = make_new_mutation(s);
- test_cache_delegates_to_underlying_only_once_with_single_partition(s, m, query::full_partition_range);
+ test_cache_delegates_to_underlying_only_once_with_single_partition(s, m, query::full_partition_range, 2);
});
}

@@ -236,7 +237,7 @@ SEASTAR_TEST_CASE(test_cache_delegates_to_underlying_only_once_range_open) {
auto m = make_new_mutation(s);
dht::partition_range::bound end = {dht::ring_position(m.decorated_key()), true};
dht::partition_range range = dht::partition_range::make_ending_with(end);
- test_cache_delegates_to_underlying_only_once_with_single_partition(s, m, range);
+ test_cache_delegates_to_underlying_only_once_with_single_partition(s, m, range, 2);
});
}

--
2.7.4

Tomasz Grabiec

<tgrabiec@scylladb.com>
unread,
Jun 12, 2017, 3:02:56 PM6/12/17
to scylladb-dev@googlegroups.com
---
row_cache.hh | 6 ++++++
1 file changed, 6 insertions(+)

diff --git a/row_cache.hh b/row_cache.hh
index cd88139..f8708ca 100644
--- a/row_cache.hh
+++ b/row_cache.hh
@@ -77,12 +77,18 @@ class cache_entry {
friend class cache_tracker;

struct dummy_entry_tag{};
+ struct incomplete_tag{};
+
cache_entry(dummy_entry_tag)
: _key{dht::token(), partition_key::make_empty()}
{
_flags._dummy_entry = true;
}

+ cache_entry(incomplete_tag, schema_ptr s, const dht::decorated_key& key)
+ : cache_entry(s, key, mutation_partition::make_incomplete(*s))
+ { }
+
cache_entry(schema_ptr s, const dht::decorated_key& key, const mutation_partition& p)
: _schema(std::move(s))
, _key(key)
--
2.7.4

Tomasz Grabiec

<tgrabiec@scylladb.com>
unread,
Jun 12, 2017, 3:02:56 PM6/12/17
to scylladb-dev@googlegroups.com
Algorithms will assume that.
---
row_cache.hh | 17 ++++++++++++++---
1 file changed, 14 insertions(+), 3 deletions(-)

diff --git a/row_cache.hh b/row_cache.hh
index f0a6c17..cd88139 100644
--- a/row_cache.hh
+++ b/row_cache.hh
@@ -71,6 +71,7 @@ class cache_entry {
lru_link_type _lru_link;
cache_link_type _cache_link;
friend class size_calculator;
+
public:
friend class row_cache;
friend class cache_tracker;
@@ -86,19 +87,29 @@ class cache_entry {
: _schema(std::move(s))
, _key(key)
, _pe(p)
- { }
+ {
+ _pe.version()->partition().ensure_last_dummy(*_schema);
+ }

cache_entry(schema_ptr s, dht::decorated_key&& key, mutation_partition&& p) noexcept
: _schema(std::move(s))
, _key(std::move(key))
, _pe(std::move(p))
- { }
+ {
+ _pe.version()->partition().ensure_last_dummy(*_schema);
+ }

+ // It is assumed that pe is fully continuous
cache_entry(schema_ptr s, dht::decorated_key&& key, partition_entry&& pe) noexcept
: _schema(std::move(s))
, _key(std::move(key))
, _pe(std::move(pe))
- { }
+ {
+ // If we can assume that _pe is fully continuous, we don't need to check all versions
+ // to determine what the continuity is.
+ // This doesn't change value and doesn't invalidate iterators, so can be called even with a snapshot.
+ _pe.version()->partition().ensure_last_dummy(*_schema);
+ }

cache_entry(cache_entry&&) noexcept;

--
2.7.4

Tomasz Grabiec

<tgrabiec@scylladb.com>
unread,
Jun 12, 2017, 3:02:57 PM6/12/17
to scylladb-dev@googlegroups.com
---
row_cache.hh | 2 ++
row_cache.cc | 2 +-
2 files changed, 3 insertions(+), 1 deletion(-)

diff --git a/row_cache.hh b/row_cache.hh
index f8708ca..ec40209 100644
--- a/row_cache.hh
+++ b/row_cache.hh
@@ -47,6 +47,8 @@ class read_context;

}

+extern logging::logger clogger;
+
// Intrusive set entry which holds partition data.
//
// TODO: Make memtables use this format too.
diff --git a/row_cache.cc b/row_cache.cc
index cda64d4..8a106ad 100644
--- a/row_cache.cc
+++ b/row_cache.cc
@@ -38,7 +38,7 @@ using namespace std::chrono_literals;
using namespace cache;


-static logging::logger clogger("cache");
+logging::logger clogger("cache");

thread_local seastar::thread_scheduling_group row_cache::_update_thread_scheduling_group(1ms, 0.2);

--
2.7.4

Tomasz Grabiec

<tgrabiec@scylladb.com>
unread,
Jun 12, 2017, 3:02:58 PM6/12/17
to scylladb-dev@googlegroups.com
Will be useful when additional action needs to be done on the entry
after it was created or constructed.
---
row_cache.hh | 4 +++-
row_cache.cc | 14 ++++++++------
2 files changed, 11 insertions(+), 7 deletions(-)

diff --git a/row_cache.hh b/row_cache.hh
index ec40209..4c45045 100644
--- a/row_cache.hh
+++ b/row_cache.hh
@@ -332,7 +332,9 @@ class row_cache final {
// { create(it) } -> partitions_type::iterator;
// { visit(it) } -> void;
// }
- void do_find_or_create_entry(const dht::decorated_key& key, const previous_entry_pointer* previous,
+ //
+ // Must be run under reclaim lock
+ cache_entry& do_find_or_create_entry(const dht::decorated_key& key, const previous_entry_pointer* previous,
CreateEntry&& create_entry, VisitEntry&& visit_entry);

partitions_type::iterator partitions_end() {
diff --git a/row_cache.cc b/row_cache.cc
index 8a106ad..e3fc30a 100644
--- a/row_cache.cc
+++ b/row_cache.cc
@@ -641,12 +641,11 @@ template<typename CreateEntry, typename VisitEntry>
// { create(it) } -> row_cache::partitions_type::iterator;
// { visit(it) } -> void;
// }
-void row_cache::do_find_or_create_entry(const dht::decorated_key& key,
+cache_entry& row_cache::do_find_or_create_entry(const dht::decorated_key& key,
const previous_entry_pointer* previous, CreateEntry&& create_entry, VisitEntry&& visit_entry)
{
- with_allocator(_tracker.allocator(), [&] {
- _populate_section(_tracker.region(), [&] {
- with_linearized_managed_bytes([&] {
+ return with_allocator(_tracker.allocator(), [&] () -> cache_entry& {
+ return with_linearized_managed_bytes([&] () -> cache_entry& {
auto i = _partitions.lower_bound(key, cache_entry::compare(_schema));
if (i == _partitions.end() || !i->key().equal(*_schema, key)) {
i = create_entry(i);
@@ -655,7 +654,7 @@ void row_cache::do_find_or_create_entry(const dht::decorated_key& key,
}

if (!previous) {
- return;
+ return *i;
}

if ((!previous->_key && i == _partitions.begin())
@@ -663,12 +662,14 @@ void row_cache::do_find_or_create_entry(const dht::decorated_key& key,
&& std::prev(i)->key().equal(*_schema, *previous->_key))) {
i->set_continuous(true);
}
+
+ return *i;
});
- });
});
}

void row_cache::populate(const mutation& m, const previous_entry_pointer* previous) {
+ _populate_section(_tracker.region(), [&] {
do_find_or_create_entry(m.decorated_key(), previous, [&] (auto i) {
cache_entry* entry = current_allocator().construct<cache_entry>(
m.schema(), m.decorated_key(), m.partition());
@@ -681,6 +682,7 @@ void row_cache::populate(const mutation& m, const previous_entry_pointer* previo
// it must be complete, so do nothing.
_tracker.on_miss_already_populated(); // #1534
});
+ });
}

mutation_source& row_cache::snapshot_for_phase(phase_type phase) {
--
2.7.4

Tomasz Grabiec

<tgrabiec@scylladb.com>
unread,
Jun 12, 2017, 3:02:59 PM6/12/17
to scylladb-dev@googlegroups.com
---
row_cache.hh | 4 ++++
row_cache.cc | 11 +++++++++++
2 files changed, 15 insertions(+)

diff --git a/row_cache.hh b/row_cache.hh
index 4c45045..23d1bcd 100644
--- a/row_cache.hh
+++ b/row_cache.hh
@@ -337,6 +337,10 @@ class row_cache final {
cache_entry& do_find_or_create_entry(const dht::decorated_key& key, const previous_entry_pointer* previous,
CreateEntry&& create_entry, VisitEntry&& visit_entry);

+ // Must be run under reclaim lock
+ // Ensures that partition entry for given key exists in cache and returns a reference to it.
+ cache_entry& find_or_create(const dht::decorated_key& key, const previous_entry_pointer* previous = nullptr);
+
partitions_type::iterator partitions_end() {
return std::prev(_partitions.end());
}
diff --git a/row_cache.cc b/row_cache.cc
index e3fc30a..e721535 100644
--- a/row_cache.cc
+++ b/row_cache.cc
@@ -668,6 +668,17 @@ cache_entry& row_cache::do_find_or_create_entry(const dht::decorated_key& key,
});
}

+cache_entry& row_cache::find_or_create(const dht::decorated_key& key, const previous_entry_pointer* previous) {
+ return do_find_or_create_entry(key, previous, [&] (auto i) { // create
+ auto entry = current_allocator().construct<cache_entry>(cache_entry::incomplete_tag{}, _schema, key);
+ _tracker.insert(*entry);
+ return _partitions.insert(i, *entry);
+ }, [&] (auto i) { // visit
+ _tracker.touch(*i);
+ upgrade_entry(*i);
+ });
+}
+
void row_cache::populate(const mutation& m, const previous_entry_pointer* previous) {
_populate_section(_tracker.region(), [&] {
do_find_or_create_entry(m.decorated_key(), previous, [&] (auto i) {
--
2.7.4

Tomasz Grabiec

<tgrabiec@scylladb.com>
unread,
Jun 12, 2017, 3:03:00 PM6/12/17
to scylladb-dev@googlegroups.com
---
partition_snapshot_row_cursor.hh | 208 +++++++++++++++++++++++++++++++++++++++
1 file changed, 208 insertions(+)
create mode 100644 partition_snapshot_row_cursor.hh

diff --git a/partition_snapshot_row_cursor.hh b/partition_snapshot_row_cursor.hh
new file mode 100644
index 0000000..2a296db
--- /dev/null
+++ b/partition_snapshot_row_cursor.hh
@@ -0,0 +1,208 @@
+/*
+ * Copyright (C) 2017 ScyllaDB
+ */
+
+/*
+ * This file is part of Scylla.
+ *
+ * Scylla is free software: you can redistribute it and/or modify
+ * it under the terms of the GNU Affero General Public License as published by
+ * the Free Software Foundation, either version 3 of the License, or
+ * (at your option) any later version.
+ *
+ * Scylla is distributed in the hope that it will be useful,
+ * but WITHOUT ANY WARRANTY; without even the implied warranty of
+ * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
+ * GNU General Public License for more details.
+ *
+ * You should have received a copy of the GNU General Public License
+ * along with Scylla. If not, see <http://www.gnu.org/licenses/>.
+ */
+
+#pragma once
+
+#include "partition_version.hh"
+
+// Allows iterating over rows of mutation_partition represented by given partition_snapshot.
+//
+// The cursor initially has a position before all rows and is not pointing at any row.
+// To position the cursor, use advance_to().
+//
+// All methods should be called with the region of the snapshot locked. The cursor is invalidated
+// when that lock section is left, or if the snapshot is modified.
+//
+// When the cursor is invalidated, it still maintains its previous position. It can be brought
+// back to validity by calling maybe_refresh(), or advance_to().
+//
+class partition_snapshot_row_cursor final {
+ struct position_in_version {
+ mutation_partition::rows_type::iterator it;
+ mutation_partition::rows_type::iterator end;
+ int version_no;
+
+ struct less_compare {
+ rows_entry::tri_compare _cmp;
+ public:
+ explicit less_compare(const schema& s) : _cmp(s) { }
+ bool operator()(const position_in_version& a, const position_in_version& b) {
+ auto res = _cmp(*a.it, *b.it);
+ return res > 0 || (res == 0 && a.version_no > b.version_no);
+ }
+ };
+ };
+
+ const schema& _schema;
+ logalloc::region& _region;
+ partition_snapshot& _snp;
+ std::vector<position_in_version> _heap;
+ std::vector<position_in_version> _current_row;
+ position_in_partition _position;
+ uint64_t _last_reclaim_count = 0;
+ size_t _last_versions_count = 0;
+
+ // Removes the next row from _heap and puts it into _current_row
+ void recreate_current_row() {
+ position_in_version::less_compare heap_less(_schema);
+ rows_entry::tri_compare _cmp(_schema);
+ do {
+ boost::range::pop_heap(_heap, heap_less);
+ _current_row.push_back(_heap.back());
+ _heap.pop_back();
+ } while (!_heap.empty() && _cmp(*_current_row[0].it, *_heap[0].it) == 0);
+ _position = position_in_partition(_current_row[0].it->position());
+ }
+public:
+ partition_snapshot_row_cursor(const schema& s, logalloc::region& region, partition_snapshot& snp)
+ : _schema(s)
+ , _region(region)
+ , _snp(snp)
+ , _position(position_in_partition::static_row_tag_t{})
+ { }
+ bool has_up_to_date_row_from_latest_version() const {
+ return up_to_date() && _current_row[0].version_no == 0;
+ }
+ mutation_partition::rows_type::iterator get_iterator_in_latest_version() const {
+ return _current_row[0].it;
+ }
+ bool up_to_date() const {
+ return _region.reclaim_counter() == _last_reclaim_count && _last_versions_count == _snp.version_count();
+ }
+
+ // Brings back the cursor to validity.
+ // Can be only called when cursor is pointing at a row.
+ //
+ // Semantically equivalent to:
+ //
+ // advance_to(position());
+ //
+ // but avoids work if not necessary.
+ bool maybe_refresh() {
+ if (!up_to_date()) {
+ return advance_to(_position);
+ }
+ return true;
+ }
+
+ // Moves the cursor to the first entry with position >= pos.
+ //
+ // The caller must ensure that such entry exists.
+ //
+ // Returns true iff there can't be any clustering row entries
+ // between lower_bound (inclusive) and the entry to which the cursor
+ // was advanced.
+ //
+ // May be called when cursor is not valid.
+ // The cursor is valid after the call.
+ // Must be called under reclaim lock.
+ bool advance_to(position_in_partition_view lower_bound) {
+ rows_entry::compare less(_schema);
+ position_in_version::less_compare heap_less(_schema);
+ _heap.clear();
+ _current_row.clear();
+ int version_no = 0;
+ for (auto&& v : _snp.versions()) {
+ auto& rows = v.partition().clustered_rows();
+ auto pos = rows.lower_bound(lower_bound, less);
+ auto end = rows.end();
+ if (pos != end) {
+ _heap.push_back({pos, end, version_no});
+ }
+ ++version_no;
+ }
+ boost::range::make_heap(_heap, heap_less);
+ _last_reclaim_count = _region.reclaim_counter();
+ _last_versions_count = _snp.version_count();
+ bool found = no_clustering_row_between(_schema, lower_bound, _heap[0].it->position());
+ recreate_current_row();
+ return found;
+ }
+
+ // Advances the cursor to the next row.
+ // If there is no next row, returns false and the cursor is no longer pointing at a row.
+ // Can be only called on a valid cursor pointing at a row.
+ bool next() {
+ position_in_version::less_compare heap_less(_schema);
+ assert(up_to_date());
+ for (auto&& curr : _current_row) {
+ ++curr.it;
+ if (curr.it != curr.end) {
+ _heap.push_back(curr);
+ boost::range::push_heap(_heap, heap_less);
+ }
+ }
+ _current_row.clear();
+ if (_heap.empty()) {
+ return false;
+ }
+ recreate_current_row();
+ return true;
+ }
+
+ // Can be called only when cursor is valid and pointing at a row.
+ bool continuous() const { return bool(_current_row[0].it->continuous()); }
+
+ // Can be called only when cursor is valid and pointing at a row.
+ bool dummy() const { return bool(_current_row[0].it->dummy()); }
+
+ // Can be called only when cursor is valid and pointing at a row, and !dummy().
+ const clustering_key& key() const { return _current_row[0].it->key(); }
+
+ // Can be called only when cursor is valid and pointing at a row.
+ clustering_row row() const {
+ clustering_row result(key());
+ for (auto&& v : _current_row) {
+ result.apply(_schema, *v.it);
+ }
+ return result;
+ }
+
+ // Can be called when cursor is pointing at a row, even when invalid.
+ const position_in_partition& position() const {
+ return _position;
+ }
+
+ bool is_in_latest_version() const;
+ bool previous_row_in_latest_version_has_key(const clustering_key_prefix& key) const;
+ void set_continuous(bool val);
+};
+
+inline
+bool partition_snapshot_row_cursor::is_in_latest_version() const {
+ return _current_row[0].version_no == 0;
+}
+
+inline
+bool partition_snapshot_row_cursor::previous_row_in_latest_version_has_key(const clustering_key_prefix& key) const {
+ if (_current_row[0].it == _snp.version()->partition().clustered_rows().begin()) {
+ return false;
+ }
+ auto prev_it = _current_row[0].it;
+ --prev_it;
+ clustering_key_prefix::tri_compare tri_comp(_schema);
+ return tri_comp(prev_it->key(), key) == 0;
+}
+
+inline
+void partition_snapshot_row_cursor::set_continuous(bool val) {
+ _current_row[0].it->set_continuous(val);
+}
--
2.7.4

Tomasz Grabiec

<tgrabiec@scylladb.com>
unread,
Jun 12, 2017, 3:03:01 PM6/12/17
to scylladb-dev@googlegroups.com
---
streamed_mutation.hh | 2 ++
streamed_mutation.cc | 4 ++++
2 files changed, 6 insertions(+)

diff --git a/streamed_mutation.hh b/streamed_mutation.hh
index 6217fe9..c342281 100644
--- a/streamed_mutation.hh
+++ b/streamed_mutation.hh
@@ -531,6 +531,8 @@ streamed_mutation streamed_mutation_returning(schema_ptr, dht::decorated_key, st
streamed_mutation merge_mutations(std::vector<streamed_mutation>);
streamed_mutation reverse_streamed_mutation(streamed_mutation);

+streamed_mutation make_empty_streamed_mutation(schema_ptr, dht::decorated_key, streamed_mutation::forwarding fwd = streamed_mutation::forwarding::no);
+
// range_tombstone_stream is a helper object that simplifies producing a stream
// of range tombstones and merging it with a stream of clustering rows.
// Tombstones are added using apply() and retrieved using get_next().
diff --git a/streamed_mutation.cc b/streamed_mutation.cc
index e45aaa8..cd908ce 100644
--- a/streamed_mutation.cc
+++ b/streamed_mutation.cc
@@ -154,6 +154,10 @@ std::ostream& operator<<(std::ostream& os, const mutation_fragment& mf) {
return os;
}

+streamed_mutation make_empty_streamed_mutation(schema_ptr s, dht::decorated_key key, streamed_mutation::forwarding fwd) {
+ return streamed_mutation_from_mutation(mutation(std::move(key), std::move(s)), fwd);
+}
+
streamed_mutation streamed_mutation_from_mutation(mutation m, streamed_mutation::forwarding fwd)
{
class reader final : public streamed_mutation::impl {
--
2.7.4

Tomasz Grabiec

<tgrabiec@scylladb.com>
unread,
Jun 12, 2017, 3:03:02 PM6/12/17
to scylladb-dev@googlegroups.com, Piotr Jastrzebski
From: Piotr Jastrzebski <pi...@scylladb.com>

This will allow conversion from streamed_mutation that
supports fast forwarding to streamed_mutation that does not.

Signed-off-by: Piotr Jastrzebski <pi...@scylladb.com>
---
streamed_mutation.hh | 1 +
streamed_mutation.cc | 38 ++++++++++++++++++++++++++++++++++++++
2 files changed, 39 insertions(+)

diff --git a/streamed_mutation.hh b/streamed_mutation.hh
index c342281..710338a 100644
--- a/streamed_mutation.hh
+++ b/streamed_mutation.hh
@@ -526,6 +526,7 @@ class mutation;

streamed_mutation streamed_mutation_from_mutation(mutation, streamed_mutation::forwarding fwd = streamed_mutation::forwarding::no);
streamed_mutation streamed_mutation_returning(schema_ptr, dht::decorated_key, std::vector<mutation_fragment>, tombstone t = {});
+streamed_mutation streamed_mutation_from_forwarding_streamed_mutation(streamed_mutation&&);

//Requires all streamed_mutations to have the same schema.
streamed_mutation merge_mutations(std::vector<streamed_mutation>);
diff --git a/streamed_mutation.cc b/streamed_mutation.cc
index cd908ce..31a1686 100644
--- a/streamed_mutation.cc
+++ b/streamed_mutation.cc
@@ -272,6 +272,44 @@ streamed_mutation streamed_mutation_from_mutation(mutation m, streamed_mutation:
return std::move(sm);
}

+streamed_mutation streamed_mutation_from_forwarding_streamed_mutation(streamed_mutation&& sm)
+{
+ class reader final : public streamed_mutation::impl {
+ streamed_mutation _sm;
+ bool _static_row_done = false;
+ public:
+ explicit reader(streamed_mutation&& sm)
+ : streamed_mutation::impl(sm.schema(), sm.decorated_key(), sm.partition_tombstone())
+ , _sm(std::move(sm))
+ { }
+
+ virtual future<> fill_buffer() override {
+ if (!_static_row_done) {
+ _static_row_done = true;
+ return _sm().then([this] (auto&& mf) {
+ if (mf) {
+ this->push_mutation_fragment(std::move(*mf));
+ }
+ return _sm.fast_forward_to(query::clustering_range{}).then([this] {
+ return this->fill_buffer();
+ });
+ });
+ }
+ return do_until([this] { return is_end_of_stream() || is_buffer_full(); }, [this] {
+ return _sm().then([this] (auto&& mf) {
+ if (mf) {
+ this->push_mutation_fragment(std::move(*mf));
+ } else {
+ _end_of_stream = true;
+ }
+ });
+ });
+ }
+ };
+
+ return make_streamed_mutation<reader>(std::move(sm));
+}
+
streamed_mutation make_forwardable(streamed_mutation m) {
class reader : public streamed_mutation::impl {
streamed_mutation _sm;
--
2.7.4

Tomasz Grabiec

<tgrabiec@scylladb.com>
unread,
Jun 12, 2017, 3:03:03 PM6/12/17
to scylladb-dev@googlegroups.com
---
partition_version.hh | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/partition_version.hh b/partition_version.hh
index a48ae76..954a73c 100644
--- a/partition_version.hh
+++ b/partition_version.hh
@@ -128,7 +128,7 @@ class partition_version : public anchorless_list_base_hook<partition_version> {
mutation_partition& partition() { return _partition; }
const mutation_partition& partition() const { return _partition; }

- bool is_referenced() { return _backref; }
+ bool is_referenced() const { return _backref; }
partition_version_ref& back_reference() { return *_backref; }
};

--
2.7.4

Tomasz Grabiec

<tgrabiec@scylladb.com>
unread,
Jun 12, 2017, 3:03:04 PM6/12/17
to scylladb-dev@googlegroups.com
---
partition_version.hh | 4 ++++
1 file changed, 4 insertions(+)

diff --git a/partition_version.hh b/partition_version.hh
index 954a73c..236ff75 100644
--- a/partition_version.hh
+++ b/partition_version.hh
@@ -274,6 +274,10 @@ class partition_entry {
return _version;
}

+ auto versions() {
+ return _version->elements_from_this();
+ }
+
// Strong exception guarantees.
void apply(const schema& s, const mutation_partition& mp, const schema& mp_schema);

--
2.7.4

Tomasz Grabiec

<tgrabiec@scylladb.com>
unread,
Jun 12, 2017, 3:03:04 PM6/12/17
to scylladb-dev@googlegroups.com
---
partition_version.hh | 2 ++
row_cache.hh | 4 ++++
partition_version.cc | 19 +++++++++++++++++++
row_cache.cc | 15 +++++++++++++++
4 files changed, 40 insertions(+)

diff --git a/partition_version.hh b/partition_version.hh
index 236ff75..9a8e874 100644
--- a/partition_version.hh
+++ b/partition_version.hh
@@ -321,6 +321,8 @@ class partition_entry {
// Snapshots with different values of phase will point to different partition_version objects.
lw_shared_ptr<partition_snapshot> read(schema_ptr entry_schema,
partition_snapshot::phase_type phase = partition_snapshot::default_phase);
+
+ friend std::ostream& operator<<(std::ostream& out, partition_entry& e);
};

inline partition_version_ref& partition_snapshot::version()
diff --git a/row_cache.hh b/row_cache.hh
index 23d1bcd..b13ebd1 100644
--- a/row_cache.hh
+++ b/row_cache.hh
@@ -178,6 +178,8 @@ class cache_entry {
return _c(k1._key, k2);
}
};
+
+ friend std::ostream& operator<<(std::ostream&, cache_entry&);
};

// Tracks accesses and performs eviction of cache entries.
@@ -425,6 +427,8 @@ class row_cache final {
void set_schema(schema_ptr) noexcept;
const schema_ptr& schema() const;

+ friend std::ostream& operator<<(std::ostream&, row_cache&);
+
friend class just_cache_scanning_reader;
friend class scanning_and_populating_reader;
friend class range_populating_reader;
diff --git a/partition_version.cc b/partition_version.cc
index 37cce5a..f94842c 100644
--- a/partition_version.cc
+++ b/partition_version.cc
@@ -621,3 +621,22 @@ partition_snapshot::range_tombstones(const schema& s, position_in_partition_view
}
return boost::copy_range<std::vector<range_tombstone>>(list);
}
+
+std::ostream& operator<<(std::ostream& out, partition_entry& e) {
+ out << "{";
+ bool first = true;
+ if (e._version) {
+ for (const partition_version& v : e.versions()) {
+ if (!first) {
+ out << ", ";
+ }
+ if (v.is_referenced()) {
+ out << "(*) ";
+ }
+ out << v.partition();
+ first = false;
+ }
+ }
+ out << "}";
+ return out;
+}
diff --git a/row_cache.cc b/row_cache.cc
index e721535..3de5e0f 100644
--- a/row_cache.cc
+++ b/row_cache.cc
@@ -974,3 +974,18 @@ void row_cache::upgrade_entry(cache_entry& e) {
});
}
}
+
+std::ostream& operator<<(std::ostream& out, row_cache& rc) {
+ rc._read_section(rc._tracker.region(), [&] {
+ out << "{row_cache: " << ::join(", ", rc._partitions.begin(), rc._partitions.end()) << "}";
+ });
+ return out;
+}
+
+std::ostream& operator<<(std::ostream& out, cache_entry& e) {
+ return out << "{cache_entry: " << e.key()
+ << ", cont=" << e.continuous()
+ << ", dummy=" << e.is_dummy_entry()
+ << ", " << e.partition()
+ << "}";
+}
--
2.7.4

Tomasz Grabiec

<tgrabiec@scylladb.com>
unread,
Jun 12, 2017, 3:03:05 PM6/12/17
to scylladb-dev@googlegroups.com
It's needed before switching cache_entry ordering to rely solely on
cache_entry::position() so that invalidate_unwrapped() never removes
the dummy entry at the end. Currently if the range has upper bound
like this:

{ ring_position::max(), inclusive=true }

The code which selects entries for removal would include the dummy row
at the end. It uses upper_bound() to get the end iterator, and the
dummy entry has a position which is equal to the position in the
bound.

ring_position_view ranges are end-exclusive, so it's impossible to
create a partition range which would include a dummy entry.

The code is also simpler.
---
row_cache.cc | 18 ++----------------
1 file changed, 2 insertions(+), 16 deletions(-)

diff --git a/row_cache.cc b/row_cache.cc
index 3de5e0f..18d97f9 100644
--- a/row_cache.cc
+++ b/row_cache.cc
@@ -879,22 +879,8 @@ void row_cache::invalidate_unwrapped(const dht::partition_range& range) {
logalloc::reclaim_lock _(_tracker.region());

auto cmp = cache_entry::compare(_schema);
- auto begin = _partitions.begin();
- if (range.start()) {
- if (range.start()->is_inclusive()) {
- begin = _partitions.lower_bound(range.start()->value(), cmp);
- } else {
- begin = _partitions.upper_bound(range.start()->value(), cmp);
- }
- }
- auto end = partitions_end();
- if (range.end()) {
- if (range.end()->is_inclusive()) {
- end = _partitions.upper_bound(range.end()->value(), cmp);
- } else {
- end = _partitions.lower_bound(range.end()->value(), cmp);
- }
- }
+ auto begin = _partitions.lower_bound(dht::ring_position_view::for_range_start(range), cmp);
+ auto end = _partitions.lower_bound(dht::ring_position_view::for_range_end(range), cmp);
with_allocator(_tracker.allocator(), [this, begin, end] {
auto it = _partitions.erase_and_dispose(begin, end, [this, deleter = current_deleter<cache_entry>()] (auto&& p) mutable {
_tracker.on_erase();
--
2.7.4

Tomasz Grabiec

<tgrabiec@scylladb.com>
unread,
Jun 12, 2017, 3:03:06 PM6/12/17
to scylladb-dev@googlegroups.com
---
row_cache.hh | 12 ++++++++----
1 file changed, 8 insertions(+), 4 deletions(-)

diff --git a/row_cache.hh b/row_cache.hh
index b13ebd1..70dffbe 100644
--- a/row_cache.hh
+++ b/row_cache.hh
@@ -134,10 +134,10 @@ class cache_entry {
bool is_dummy_entry() const { return _flags._dummy_entry; }

struct compare {
- dht::decorated_key::less_comparator _c;
+ dht::ring_position_less_comparator _c;

compare(schema_ptr s)
- : _c(std::move(s))
+ : _c(*s)
{}

bool operator()(const dht::decorated_key& k1, const cache_entry& k2) const {
@@ -147,7 +147,7 @@ class cache_entry {
return _c(k1, k2._key);
}

- bool operator()(const dht::ring_position& k1, const cache_entry& k2) const {
+ bool operator()(dht::ring_position_view k1, const cache_entry& k2) const {
if (k2.is_dummy_entry()) {
return true;
}
@@ -171,12 +171,16 @@ class cache_entry {
return _c(k1._key, k2);
}

- bool operator()(const cache_entry& k1, const dht::ring_position& k2) const {
+ bool operator()(const cache_entry& k1, dht::ring_position_view k2) const {
if (k1.is_dummy_entry()) {
return false;
}
return _c(k1._key, k2);
}
+
+ bool operator()(dht::ring_position_view k1, dht::ring_position_view k2) const {
+ return _c(k1, k2);
+ }
};

friend std::ostream& operator<<(std::ostream&, cache_entry&);
--
2.7.4

Tomasz Grabiec

<tgrabiec@scylladb.com>
unread,
Jun 12, 2017, 3:03:07 PM6/12/17
to scylladb-dev@googlegroups.com
---
row_cache.hh | 34 +++++++++++-----------------------
1 file changed, 11 insertions(+), 23 deletions(-)

diff --git a/row_cache.hh b/row_cache.hh
index 70dffbe..958da04 100644
--- a/row_cache.hh
+++ b/row_cache.hh
@@ -123,6 +123,12 @@ class cache_entry {

bool is_evictable() { return _lru_link.is_linked(); }
const dht::decorated_key& key() const { return _key; }
+ dht::ring_position_view position() const {
+ if (is_dummy_entry()) {
+ return dht::ring_position_view::max();
+ }
+ return _key;
+ }
const partition_entry& partition() const { return _pe; }
partition_entry& partition() { return _pe; }
const schema_ptr& schema() const { return _schema; }
@@ -141,41 +147,23 @@ class cache_entry {
{}

bool operator()(const dht::decorated_key& k1, const cache_entry& k2) const {
- if (k2.is_dummy_entry()) {
- return true;
- }
- return _c(k1, k2._key);
+ return _c(k1, k2.position());
}

bool operator()(dht::ring_position_view k1, const cache_entry& k2) const {
- if (k2.is_dummy_entry()) {
- return true;
- }
- return _c(k1, k2._key);
+ return _c(k1, k2.position());
}

bool operator()(const cache_entry& k1, const cache_entry& k2) const {
- if (k1.is_dummy_entry()) {
- return false;
- }
- if (k2.is_dummy_entry()) {
- return true;
- }
- return _c(k1._key, k2._key);
+ return _c(k1.position(), k2.position());
}

bool operator()(const cache_entry& k1, const dht::decorated_key& k2) const {
- if (k1.is_dummy_entry()) {
- return false;
- }
- return _c(k1._key, k2);
+ return _c(k1.position(), k2);
}

bool operator()(const cache_entry& k1, dht::ring_position_view k2) const {
- if (k1.is_dummy_entry()) {
- return false;
- }
- return _c(k1._key, k2);
+ return _c(k1.position(), k2);
}

bool operator()(dht::ring_position_view k1, dht::ring_position_view k2) const {
--
2.7.4

Tomasz Grabiec

<tgrabiec@scylladb.com>
unread,
Jun 12, 2017, 3:03:08 PM6/12/17
to scylladb-dev@googlegroups.com
---
dht/i_partitioner.hh | 14 ++++++++++++++
1 file changed, 14 insertions(+)

diff --git a/dht/i_partitioner.hh b/dht/i_partitioner.hh
index e6fcc69..d23681a 100644
--- a/dht/i_partitioner.hh
+++ b/dht/i_partitioner.hh
@@ -516,6 +516,20 @@ class ring_position_view {
, _weight(pos.has_key() ? bool(after) : pos.relation_to_keys())
{ }

+ ring_position_view(const ring_position_view& pos) noexcept
+ : _token(pos._token)
+ , _key(pos._key)
+ , _weight(pos._weight)
+ { }
+
+ ring_position_view& operator=(const ring_position_view& other) noexcept {
+ if (this != &other) {
+ this->~ring_position_view();
+ new (this) ring_position_view(other);
+ }
+ return *this;
+ }
+
ring_position_view(after_key_tag, const ring_position_view& v)
: _token(v._token)
, _key(v._key)
--
2.7.4

Tomasz Grabiec

<tgrabiec@scylladb.com>
unread,
Jun 12, 2017, 3:03:08 PM6/12/17
to scylladb-dev@googlegroups.com
---
row_cache.cc | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/row_cache.cc b/row_cache.cc
index 18d97f9..1299223 100644
--- a/row_cache.cc
+++ b/row_cache.cc
@@ -969,7 +969,7 @@ std::ostream& operator<<(std::ostream& out, row_cache& rc) {
}

std::ostream& operator<<(std::ostream& out, cache_entry& e) {
- return out << "{cache_entry: " << e.key()
+ return out << "{cache_entry: " << e.position()
<< ", cont=" << e.continuous()
<< ", dummy=" << e.is_dummy_entry()
<< ", " << e.partition()
--
2.7.4

Tomasz Grabiec

<tgrabiec@scylladb.com>
unread,
Jun 12, 2017, 3:03:09 PM6/12/17
to scylladb-dev@googlegroups.com
---
row_cache.hh | 1 +
row_cache.cc | 93 ++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++
2 files changed, 94 insertions(+)

diff --git a/row_cache.hh b/row_cache.hh
index 958da04..bf50d85 100644
--- a/row_cache.hh
+++ b/row_cache.hh
@@ -250,6 +250,7 @@ class row_cache final {
friend class single_partition_populating_reader;
friend class cache_entry;
friend class cache::read_context;
+ friend class partition_range_iterator;
public:
struct stats {
utils::timed_rate_moving_average hits;
diff --git a/row_cache.cc b/row_cache.cc
index 1299223..798deed 100644
--- a/row_cache.cc
+++ b/row_cache.cc
@@ -183,6 +183,99 @@ const logalloc::region& cache_tracker::region() const {
return _region;
}

+// Stable cursor over partition entries from given range.
+//
+// Must be accessed with reclaim lock held on the cache region.
+// The position of the cursor is always valid, but cache entry reference
+// is not always valid. It remains valid as long as the iterators
+// into _cache._partitions remain valid. Cache entry reference can be
+// brought back to validity by calling refresh().
+//
+class partition_range_iterator final {
+ std::reference_wrapper<row_cache> _cache;
+ row_cache::partitions_type::iterator _it;
+ row_cache::partitions_type::iterator _end;
+ dht::ring_position_view _start_pos;
+ dht::ring_position_view _end_pos;
+ stdx::optional<dht::decorated_key> _last;
+ uint64_t _last_reclaim_count;
+ size_t _last_modification_count;
+private:
+ void set_position(cache_entry& e) {
+ // FIXME: make ring_position_view convertible to ring_position, so we can use e.position()
+ if (e.is_dummy_entry()) {
+ _last = {};
+ _start_pos = dht::ring_position_view::max();
+ } else {
+ _last = e.key();
+ _start_pos = dht::ring_position_view(*_last);
+ }
+ }
+public:
+ // Creates an iterator positioned at the lower bound of the range.
+ // The cache entry reference is not valid.
+ // The range reference must remain live as long as this instance is used.
+ partition_range_iterator(row_cache& cache, const dht::partition_range& range)
+ : _cache(cache)
+ , _start_pos(dht::ring_position_view::for_range_start(range))
+ , _end_pos(dht::ring_position_view::for_range_end(range))
+ , _last_reclaim_count(std::numeric_limits<uint64_t>::max())
+ , _last_modification_count(std::numeric_limits<size_t>::max())
+ { }
+
+ // Ensures that cache entry reference is valid.
+ // The cursor will point to the first entry with position >= current position.
+ // Returns true if and only if the position of the cursor changed.
+ // Strong exception guarantees.
+ bool refresh() {
+ auto reclaim_count = _cache.get().get_cache_tracker().region().reclaim_counter();
+ auto modification_count = _cache.get().get_cache_tracker().modification_count();
+
+ if (reclaim_count == _last_reclaim_count && modification_count == _last_modification_count) {
+ return true;
+ }
+
+ auto cmp = cache_entry::compare(_cache.get()._schema);
+ if (cmp(_end_pos, _start_pos)) { // next() may have moved _start_pos past the _end_pos.
+ _end_pos = _start_pos;
+ }
+ _end = _cache.get()._partitions.lower_bound(_end_pos, cmp);
+ _it = _cache.get()._partitions.lower_bound(_start_pos, cmp);
+ auto same = !cmp(_start_pos, _it->position());
+ set_position(*_it);
+ _last_reclaim_count = reclaim_count;
+ _last_modification_count = modification_count;
+ return same;
+ }
+
+ // Positions the cursor at the next entry.
+ // May advance past the requested range. Use in_range() after the call to determine that.
+ // Call only when in_range() and cache entry reference is valid.
+ // Strong exception guarantees.
+ void next() {
+ auto next = std::next(_it);
+ set_position(*next);
+ _it = std::move(next);
+ }
+
+ // Valid only after refresh() and before _cache._partitions iterators are invalidated.
+ // Points inside the requested range if in_range().
+ cache_entry& entry() {
+ return *_it;
+ }
+
+ // Call only when cache entry reference is valid.
+ bool in_range() {
+ return _it != _end;
+ }
+
+ // Returns current position of the cursor.
+ // Result valid as long as this instance is valid and not advanced.
+ dht::ring_position_view position() const {
+ return _start_pos;
+ }
+};
+
/*
* Represent a reader to the underlying source.
* This reader automatically makes sure that it's up to date with all cache updates
--
2.7.4

Tomasz Grabiec

<tgrabiec@scylladb.com>
unread,
Jun 12, 2017, 3:03:10 PM6/12/17
to scylladb-dev@googlegroups.com
---
range.hh | 6 ++++++
1 file changed, 6 insertions(+)

diff --git a/range.hh b/range.hh
index a08017e..63be891 100644
--- a/range.hh
+++ b/range.hh
@@ -548,6 +548,12 @@ class nonwrapping_range {
return nonwrapping_range(range_bound<T>(split_point, false), end());
}
}
+ // Creates a new sub-range which is the intersection of this range and a range starting with "start".
+ // If there is no overlap, returns stdx::nullopt.
+ template<typename Comparator>
+ stdx::optional<nonwrapping_range> trim_front(stdx::optional<bound>&& start, Comparator&& cmp) const {
+ return intersection(nonwrapping_range(std::move(start), {}), cmp);
+ }
// Transforms this range into a new range of a different value type
// Supplied transformer should transform value of type T (the old type) into value of type U (the new type).
template<typename Transformer, typename U = typename std::result_of<Transformer(T)>::type>
--
2.7.4

Tomasz Grabiec

<tgrabiec@scylladb.com>
unread,
Jun 12, 2017, 3:03:11 PM6/12/17
to scylladb-dev@googlegroups.com
Currently scanning_and_populating_reader asks
just_cache_scanning_reader for the next partition from cache, together
with information if the range is continuous. If it's not, it saves the
partition it got from it and moves on to reading from the underlying
reader up to that partition. When that's done, it emits the stored
partition.

This approach won't work well with upcoming changes for storing
partial partitions. We won't have whole partitions any more, so
streamed_mutation returned for the entry needs to be prepared for
reading from the underlying mutation source. We want to reuse the same
underlying reader as much as possible, so all streamed_mutations for
given read (read_context) will share the state of the underlying
reader. Construction of a streamed_mutation will depend on the fact
that the shared state is set up for it, so we cannot have two
streamed_mutations prepared at the same time (one for entry from
primary, and one for the earlier entry being populated). This change
defers the creation of a streamed_mutation for the entry present in
cache until the whole reader reaches it to avoid this problem.

This will also have antoher potentially beneficial effect. Since we
defer the decision about which snapshot to use until we reach the
entry, there is a higher chance that the current snapshot of the entry
will match the one used last by the populating read, and that we will
be able to reuse the reader.

It's implemented by utilizing a stable partition cursor which tracks
its current position so that it's possible to revisit the cache entry
(if it's still there) after population ends. The functionality of
just_cache_scanning_reader was inlined into
scanning_and_populating_reader.
---
row_cache.cc | 194 +++++++++++++++++++----------------------------------------
1 file changed, 61 insertions(+), 133 deletions(-)

diff --git a/row_cache.cc b/row_cache.cc
index 798deed..c8c250b 100644
--- a/row_cache.cc
+++ b/row_cache.cc
@@ -401,88 +401,6 @@ void row_cache::on_miss() {
_tracker.on_miss();
}

-class just_cache_scanning_reader final {
- row_cache& _cache;
- row_cache::partitions_type::iterator _it;
- row_cache::partitions_type::iterator _end;
- const dht::partition_range* _range;
- stdx::optional<dht::decorated_key> _last;
- uint64_t _last_reclaim_count;
- size_t _last_modification_count;
- read_context& _read_context;
-private:
- void update_iterators() {
- auto cmp = cache_entry::compare(_cache._schema);
- auto update_end = [&] {
- if (_range->end()) {
- if (_range->end()->is_inclusive()) {
- _end = _cache._partitions.upper_bound(_range->end()->value(), cmp);
- } else {
- _end = _cache._partitions.lower_bound(_range->end()->value(), cmp);
- }
- } else {
- _end = _cache.partitions_end();
- }
- };
-
- auto reclaim_count = _cache.get_cache_tracker().region().reclaim_counter();
- auto modification_count = _cache.get_cache_tracker().modification_count();
- if (!_last) {
- if (_range->start()) {
- if (_range->start()->is_inclusive()) {
- _it = _cache._partitions.lower_bound(_range->start()->value(), cmp);
- } else {
- _it = _cache._partitions.upper_bound(_range->start()->value(), cmp);
- }
- } else {
- _it = _cache._partitions.begin();
- }
- update_end();
- } else if (reclaim_count != _last_reclaim_count || modification_count != _last_modification_count) {
- _it = _cache._partitions.upper_bound(*_last, cmp);
- update_end();
- }
- _last_reclaim_count = reclaim_count;
- _last_modification_count = modification_count;
- }
-public:
- struct cache_data {
- streamed_mutation_opt mut;
- bool continuous;
- };
- just_cache_scanning_reader(row_cache& cache,
- const dht::partition_range& range,
- read_context& ctx)
- : _cache(cache)
- , _range(&range)
- , _read_context(ctx)
- { }
- future<cache_data> operator()() {
- return _cache._read_section(_cache._tracker.region(), [this] {
- return with_linearized_managed_bytes([&] {
- update_iterators();
- if (_it == _end) {
- return make_ready_future<cache_data>(cache_data { {}, _it->continuous() });
- }
- cache_entry& ce = *_it;
- ++_it;
- _last = ce.key();
- _cache.upgrade_entry(ce);
- _cache._tracker.touch(ce);
- _cache.on_hit();
- cache_data cd { { }, ce.continuous() };
- cd.mut = ce.read(_cache, _read_context);
- return make_ready_future<cache_data>(std::move(cd));
- });
- });
- }
- future<> fast_forward_to(const dht::partition_range& pr) {
- _last = {};
- _range = &pr;
- return make_ready_future<>();
- }
-};
-
class range_populating_reader {
row_cache& _cache;
autoupdating_underlying_reader _reader;
@@ -563,77 +481,84 @@ class range_populating_reader {

class scanning_and_populating_reader final : public mutation_reader::impl {
const dht::partition_range* _pr;
+ row_cache& _cache;
lw_shared_ptr<read_context> _read_context;
- just_cache_scanning_reader _primary_reader;
+ partition_range_iterator _primary;
range_populating_reader _secondary_reader;
- streamed_mutation_opt _next_primary;
bool _secondary_in_progress = false;
- bool _first_element = true;
- stdx::optional<dht::decorated_key> _last_key;
+ bool _advance_primary = false;
+ stdx::optional<dht::partition_range::bound> _lower_bound;
+ dht::partition_range _secondary_range;
private:
- void update_last_key(const streamed_mutation_opt& smopt) {
- if (smopt) {
- _last_key = smopt->decorated_key();
- }
+ streamed_mutation read_from_entry(cache_entry& ce) {
+ _cache.upgrade_entry(ce);
+ _cache._tracker.touch(ce);
+ _cache.on_hit();
+ return ce.read(_cache, *_read_context);
}

- bool is_inclusive_start_bound(const dht::decorated_key& dk) {
- if (!_first_element) {
- return false;
- }
- return _pr->start() && _pr->start()->is_inclusive() && _pr->start()->value().equal(*_read_context->schema(), dk);
- }
+ streamed_mutation_opt do_read_from_primary() {
+ return _cache._read_section(_cache._tracker.region(), [this] {
+ return with_linearized_managed_bytes([&] () -> streamed_mutation_opt {
+ auto not_moved = _primary.refresh();

- future<streamed_mutation_opt> read_from_primary() {
- return _primary_reader().then([this] (just_cache_scanning_reader::cache_data cd) {
- auto& smopt = cd.mut;
- if (cd.continuous || (smopt && is_inclusive_start_bound(smopt->decorated_key()))) {
- _first_element = false;
- update_last_key(smopt);
- return make_ready_future<streamed_mutation_opt>(std::move(smopt));
- } else {
- _next_primary = std::move(smopt);
+ if (_advance_primary && not_moved) {
+ _primary.next();
+ not_moved = false;
+ }
+ _advance_primary = false;

- dht::partition_range secondary_range = { };
- if (!_next_primary) {
- if (!_last_key) {
- secondary_range = *_pr;
- } else {
- dht::ring_position_comparator cmp(*_read_context->schema());
- auto&& new_range = _pr->split_after(*_last_key, cmp);
- if (!new_range) {
- return make_ready_future<streamed_mutation_opt>();
- }
- secondary_range = std::move(*new_range);
+ if (not_moved || _primary.entry().continuous()) {
+ if (!_primary.in_range()) {
+ return stdx::nullopt;
}
+ cache_entry& e = _primary.entry();
+ auto sm = read_from_entry(e);
+ _lower_bound = {e.key(), false};
+ // Delay the call to next() so that we don't see stale continuity on next invocation.
+ _advance_primary = true;
+ return streamed_mutation_opt(std::move(sm));
} else {
- if (_last_key) {
- secondary_range = dht::partition_range::make({ *_last_key, false }, { _next_primary->decorated_key(), false });
+ if (_primary.in_range()) {
+ cache_entry& e = _primary.entry();
+ _secondary_range = dht::partition_range(_lower_bound ? std::move(_lower_bound) : _pr->start(),
+ dht::partition_range::bound{e.key(), false});
+ _lower_bound = {e.key(), true};
+ _secondary_in_progress = true;
+ return stdx::nullopt;
} else {
- if (!_pr->start()) {
- secondary_range = dht::partition_range::make_ending_with({ _next_primary->decorated_key(), false });
- } else {
- secondary_range = dht::partition_range::make(*_pr->start(), { _next_primary->decorated_key(), false });
+ dht::ring_position_comparator cmp(*_read_context->schema());
+ auto range = _pr->trim_front(std::move(_lower_bound), cmp);
+ if (!range) {
+ return stdx::nullopt;
}
+ _lower_bound = {dht::ring_position::max()};
+ _secondary_range = std::move(*range);
+ _secondary_in_progress = true;
+ return stdx::nullopt;
}
}
+ });
+ });
+ }

- _secondary_in_progress = true;
- return _secondary_reader.fast_forward_to(std::move(secondary_range)).then([this] {
- return read_from_secondary();
- });
- }
+ future<streamed_mutation_opt> read_from_primary() {
+ auto smo = do_read_from_primary();
+ if (!_secondary_in_progress) {
+ return make_ready_future<streamed_mutation_opt>(std::move(smo));
+ }
+ return _secondary_reader.fast_forward_to(std::move(_secondary_range)).then([this] {
+ return read_from_secondary();
});
}

future<streamed_mutation_opt> read_from_secondary() {
return _secondary_reader().then([this] (streamed_mutation_opt smopt) {
if (smopt) {
- return smopt;
+ return make_ready_future<streamed_mutation_opt>(std::move(smopt));
} else {
_secondary_in_progress = false;
- update_last_key(_next_primary);
- return std::move(_next_primary);
+ return read_from_primary();
}
});
}
@@ -642,8 +567,9 @@ class scanning_and_populating_reader final : public mutation_reader::impl {
const dht::partition_range& range,
lw_shared_ptr<read_context> context)
: _pr(&range)
+ , _cache(cache)
, _read_context(std::move(context))
- , _primary_reader(cache, range, *_read_context)
+ , _primary(cache, range)
, _secondary_reader(cache, *_read_context)
{ }

@@ -657,9 +583,11 @@ class scanning_and_populating_reader final : public mutation_reader::impl {

future<> fast_forward_to(const dht::partition_range& pr) {
_secondary_in_progress = false;
- _first_element = true;
+ _advance_primary = false;
_pr = &pr;
- return _primary_reader.fast_forward_to(pr);
+ _primary = partition_range_iterator{_cache, pr};
+ _lower_bound = {};
+ return make_ready_future<>();
}
};

--
2.7.4

Tomasz Grabiec

<tgrabiec@scylladb.com>
unread,
Jun 12, 2017, 3:03:12 PM6/12/17
to scylladb-dev@googlegroups.com
---
read_context.hh | 67 +++++++++++++++++++++++++++++++++++++++++++++++++++++++++
row_cache.hh | 3 ++-
row_cache.cc | 67 ---------------------------------------------------------
3 files changed, 69 insertions(+), 68 deletions(-)

diff --git a/read_context.hh b/read_context.hh
index 33477d9..f7c63d8 100644
--- a/read_context.hh
+++ b/read_context.hh
@@ -30,6 +30,73 @@

namespace cache {

+/*
+ * Represent a reader to the underlying source.
+ * This reader automatically makes sure that it's up to date with all cache updates
+ */
+class autoupdating_underlying_reader final {
+ row_cache& _cache;
+ read_context& _read_context;
+ stdx::optional<mutation_reader> _reader;
+ utils::phased_barrier::phase_type _reader_creation_phase;
+ dht::partition_range _range = { };
+ stdx::optional<dht::decorated_key> _last_key;
+ stdx::optional<dht::decorated_key> _new_last_key;
+public:
+ autoupdating_underlying_reader(row_cache& cache, read_context& context)
+ : _cache(cache)
+ , _read_context(context)
+ { }
+ future<streamed_mutation_opt> operator()() {
+ _last_key = std::move(_new_last_key);
+ auto start = population_range_start();
+ auto phase = _cache.phase_of(start);
+ if (!_reader || _reader_creation_phase != phase) {
+ if (_last_key) {
+ auto cmp = dht::ring_position_comparator(*_cache._schema);
+ auto&& new_range = _range.split_after(*_last_key, cmp);
+ if (!new_range) {
+ return make_ready_future<streamed_mutation_opt>(streamed_mutation_opt());
+ }
+ _range = std::move(*new_range);
+ _last_key = {};
+ }
+ auto& snap = _cache.snapshot_for_phase(phase);
+ _reader = _cache.create_underlying_reader(_read_context, snap, _range);
+ _reader_creation_phase = phase;
+ }
+ return (*_reader)().then([this] (auto&& smopt) {
+ if (smopt) {
+ _new_last_key = smopt->decorated_key();
+ }
+ return std::move(smopt);
+ });
+ }
+ future<> fast_forward_to(dht::partition_range&& range) {
+ _range = std::move(range);
+ _last_key = { };
+ _new_last_key = { };
+ auto phase = _cache.phase_of(dht::ring_position_view::for_range_start(_range));
+ if (_reader && _reader_creation_phase == phase) {
+ return _reader->fast_forward_to(_range);
+ }
+ _reader = _cache.create_underlying_reader(_read_context, _cache.snapshot_for_phase(phase), _range);
+ _reader_creation_phase = phase;
+ return make_ready_future<>();
+ }
+ utils::phased_barrier::phase_type creation_phase() const {
+ assert(_reader);
+ return _reader_creation_phase;
+ }
+ const dht::partition_range& range() const {
+ return _range;
+ }
+ dht::ring_position_view population_range_start() const {
+ return _last_key ? dht::ring_position_view::for_after_key(*_last_key)
+ : dht::ring_position_view::for_range_start(_range);
+ }
+};
+
class read_context final : public enable_lw_shared_from_this<read_context> {
row_cache& _cache;
schema_ptr _schema;
diff --git a/row_cache.hh b/row_cache.hh
index bf50d85..9b7b5bd 100644
--- a/row_cache.hh
+++ b/row_cache.hh
@@ -43,6 +43,7 @@ class row_cache;

namespace cache {

+class autoupdating_underlying_reader;
class read_context;

}
@@ -246,7 +247,7 @@ class row_cache final {
bi::member_hook<cache_entry, cache_entry::cache_link_type, &cache_entry::_cache_link>,
bi::constant_time_size<false>, // we need this to have bi::auto_unlink on hooks
bi::compare<cache_entry::compare>>;
- friend class autoupdating_underlying_reader;
+ friend class cache::autoupdating_underlying_reader;
friend class single_partition_populating_reader;
friend class cache_entry;
friend class cache::read_context;
diff --git a/row_cache.cc b/row_cache.cc
index c8c250b..652a782 100644
--- a/row_cache.cc
+++ b/row_cache.cc
@@ -276,73 +276,6 @@ class partition_range_iterator final {
}
};

-/*
- * Represent a reader to the underlying source.
- * This reader automatically makes sure that it's up to date with all cache updates
- */
-class autoupdating_underlying_reader final {
- row_cache& _cache;
- read_context& _read_context;
- stdx::optional<mutation_reader> _reader;
- utils::phased_barrier::phase_type _reader_creation_phase;
- dht::partition_range _range = { };
- stdx::optional<dht::decorated_key> _last_key;
- stdx::optional<dht::decorated_key> _new_last_key;
-public:
- autoupdating_underlying_reader(row_cache& cache, read_context& context)
- : _cache(cache)
- , _read_context(context)
- { }
- future<streamed_mutation_opt> operator()() {
- _last_key = std::move(_new_last_key);
- auto start = population_range_start();
- auto phase = _cache.phase_of(start);
- if (!_reader || _reader_creation_phase != phase) {
- if (_last_key) {
- auto cmp = dht::ring_position_comparator(*_cache._schema);
- auto&& new_range = _range.split_after(*_last_key, cmp);
- if (!new_range) {
- return make_ready_future<streamed_mutation_opt>(streamed_mutation_opt());
- }
- _range = std::move(*new_range);
- _last_key = {};
- }
- auto& snap = _cache.snapshot_for_phase(phase);
- _reader = _cache.create_underlying_reader(_read_context, snap, _range);
- _reader_creation_phase = phase;
- }
- return (*_reader)().then([this] (auto&& smopt) {
- if (smopt) {
- _new_last_key = smopt->decorated_key();
- }
- return std::move(smopt);
- });
- }
- future<> fast_forward_to(dht::partition_range&& range) {
- _range = std::move(range);
- _last_key = { };
- _new_last_key = { };
- auto phase = _cache.phase_of(dht::ring_position_view::for_range_start(_range));
- if (_reader && _reader_creation_phase == phase) {
- return _reader->fast_forward_to(_range);
- }
- _reader = _cache.create_underlying_reader(_read_context, _cache.snapshot_for_phase(phase), _range);
- _reader_creation_phase = phase;
- return make_ready_future<>();
- }
- utils::phased_barrier::phase_type creation_phase() const {
- assert(_reader);
- return _reader_creation_phase;
- }
- const dht::partition_range& range() const {
- return _range;
- }
- dht::ring_position_view population_range_start() const {
- return _last_key ? dht::ring_position_view::for_after_key(*_last_key)
- : dht::ring_position_view::for_range_start(_range);
- }
-};
-
// Reader which populates the cache using data from the delegate.
class single_partition_populating_reader final : public mutation_reader::impl {
row_cache& _cache;
--
2.7.4

Tomasz Grabiec

<tgrabiec@scylladb.com>
unread,
Jun 12, 2017, 3:03:13 PM6/12/17
to scylladb-dev@googlegroups.com
We will need to use this information later in yet another place, when
creating a reader for incomplete cache entry. This refactors the code
so that there is a single place which determines this fact.
---
read_context.hh | 3 +++
row_cache.cc | 11 ++---------
2 files changed, 5 insertions(+), 9 deletions(-)

diff --git a/read_context.hh b/read_context.hh
index f7c63d8..270cf66 100644
--- a/read_context.hh
+++ b/read_context.hh
@@ -105,6 +105,7 @@ class read_context final : public enable_lw_shared_from_this<read_context> {
const io_priority_class& _pc;
tracing::trace_state_ptr _trace_state;
streamed_mutation::forwarding _fwd;
+ bool _range_query;
public:
read_context(row_cache& cache,
schema_ptr schema,
@@ -120,6 +121,7 @@ class read_context final : public enable_lw_shared_from_this<read_context> {
, _pc(pc)
, _trace_state(std::move(trace_state))
, _fwd(fwd)
+ , _range_query(!range.is_singular() || !range.start()->value().has_key())
{ }
read_context(const read_context&) = delete;
row_cache& cache() { return _cache; }
@@ -129,6 +131,7 @@ class read_context final : public enable_lw_shared_from_this<read_context> {
const io_priority_class& pc() const { return _pc; }
tracing::trace_state_ptr trace_state() const { return _trace_state; }
streamed_mutation::forwarding fwd() const { return _fwd; }
+ bool is_range_query() const { return _range_query; }
};

}
diff --git a/row_cache.cc b/row_cache.cc
index 652a782..301592b 100644
--- a/row_cache.cc
+++ b/row_cache.cc
@@ -539,17 +539,10 @@ row_cache::make_reader(schema_ptr s,
{
auto ctx = make_lw_shared<read_context>(*this, std::move(s), range, slice, pc, trace_state, fwd);

- if (range.is_singular()) {
- const query::ring_position& pos = range.start()->value();
-
- if (!pos.has_key()) {
- return make_scanning_reader(range, std::move(ctx));
- }
-
+ if (!ctx->is_range_query()) {
return _read_section(_tracker.region(), [&] {
return with_linearized_managed_bytes([&] {
- const dht::decorated_key& dk = pos.as_decorated_key();
- auto i = _partitions.find(dk, cache_entry::compare(_schema));
+ auto i = _partitions.find(ctx->range().start()->value(), cache_entry::compare(_schema));
if (i != _partitions.end()) {
cache_entry& e = *i;
_tracker.touch(e);
--
2.7.4

Tomasz Grabiec

<tgrabiec@scylladb.com>
unread,
Jun 12, 2017, 3:03:14 PM6/12/17
to scylladb-dev@googlegroups.com
When reading from incomplete partition entry, we may discover we need
to read something from the underlying mutation source. In such case we
will fast forward this reader to that partition. But we must do it
using a specific snapshot, the one we obtained when entering the
partition, not the latest one.
---
read_context.hh | 17 +++++++++++++++--
1 file changed, 15 insertions(+), 2 deletions(-)

diff --git a/read_context.hh b/read_context.hh
index 08e4562..b87a64f 100644
--- a/read_context.hh
+++ b/read_context.hh
@@ -47,6 +47,16 @@ class autoupdating_underlying_reader final {
: _cache(cache)
, _read_context(context)
{ }
+ // Reads next partition without changing mutation source snapshot.
+ future<streamed_mutation_opt> read_next_same_phase() {
+ _last_key = std::move(_new_last_key);
+ return (*_reader)().then([this] (auto&& smopt) {
+ if (smopt) {
+ _new_last_key = smopt->decorated_key();
+ }
+ return std::move(smopt);
+ });
+ }
future<streamed_mutation_opt> operator()() {
_last_key = std::move(_new_last_key);
auto start = population_range_start();
@@ -73,14 +83,17 @@ class autoupdating_underlying_reader final {
});
}
future<> fast_forward_to(dht::partition_range&& range) {
+ auto snapshot_and_phase = _cache.snapshot_of(dht::ring_position_view::for_range_start(_range));
+ return fast_forward_to(std::move(range), snapshot_and_phase.snapshot, snapshot_and_phase.phase);
+ }
+ future<> fast_forward_to(dht::partition_range&& range, mutation_source& snapshot, row_cache::phase_type phase) {
_range = std::move(range);
_last_key = { };
_new_last_key = { };
- auto phase = _cache.phase_of(dht::ring_position_view::for_range_start(_range));
if (_reader && _reader_creation_phase == phase) {
return _reader->fast_forward_to(_range);
}
- _reader = _cache.create_underlying_reader(_read_context, _cache.snapshot_for_phase(phase), _range);
+ _reader = _cache.create_underlying_reader(_read_context, snapshot, _range);
_reader_creation_phase = phase;

Tomasz Grabiec

<tgrabiec@scylladb.com>
unread,
Jun 12, 2017, 3:03:14 PM6/12/17
to scylladb-dev@googlegroups.com
Will be reused for reading of incomplete partition entries.
---
read_context.hh | 3 +++
row_cache.cc | 4 ++--
2 files changed, 5 insertions(+), 2 deletions(-)

diff --git a/read_context.hh b/read_context.hh
index 270cf66..08e4562 100644
--- a/read_context.hh
+++ b/read_context.hh
@@ -106,6 +106,7 @@ class read_context final : public enable_lw_shared_from_this<read_context> {
tracing::trace_state_ptr _trace_state;
streamed_mutation::forwarding _fwd;
bool _range_query;
+ autoupdating_underlying_reader _underlying;
public:
read_context(row_cache& cache,
schema_ptr schema,
@@ -122,6 +123,7 @@ class read_context final : public enable_lw_shared_from_this<read_context> {
, _trace_state(std::move(trace_state))
, _fwd(fwd)
, _range_query(!range.is_singular() || !range.start()->value().has_key())
+ , _underlying(_cache, *this)
{ }
read_context(const read_context&) = delete;
row_cache& cache() { return _cache; }
@@ -132,6 +134,7 @@ class read_context final : public enable_lw_shared_from_this<read_context> {
tracing::trace_state_ptr trace_state() const { return _trace_state; }
streamed_mutation::forwarding fwd() const { return _fwd; }
bool is_range_query() const { return _range_query; }
+ autoupdating_underlying_reader& underlying() { return _underlying; }
};

}
diff --git a/row_cache.cc b/row_cache.cc
index 301592b..b0e08df 100644
--- a/row_cache.cc
+++ b/row_cache.cc
@@ -336,7 +336,7 @@ void row_cache::on_miss() {

class range_populating_reader {
row_cache& _cache;
- autoupdating_underlying_reader _reader;
+ autoupdating_underlying_reader& _reader;
stdx::optional<row_cache::previous_entry_pointer> _last_key;
read_context& _read_context;
private:
@@ -368,7 +368,7 @@ class range_populating_reader {
public:
range_populating_reader(row_cache& cache, read_context& ctx)
: _cache(cache)
- , _reader(cache, ctx)
+ , _reader(ctx.underlying())
, _read_context(ctx)
{}

--
2.7.4

Tomasz Grabiec

<tgrabiec@scylladb.com>
unread,
Jun 12, 2017, 3:03:16 PM6/12/17
to scylladb-dev@googlegroups.com
The interaction will be as follows:

- Before creating cache_streamed_mutation for given partition, cache
mutation reader sets up read_context for current partition (in one
of two ways) so that the matching underlying streamed_mutation can
be accessed at any time by cached_stream_mutation.

- cache_streamed_mutation assumes that read_context is set up for
current partition and invokes fast_forward_to() and
get_next_fragment() to access the underlying
streamed_mutation.
---
read_context.hh | 55 +++++++++++++++++++++++++++++++++++++++++++++++++++++++
row_cache.cc | 19 +++++++++++++++++++
2 files changed, 74 insertions(+)

diff --git a/read_context.hh b/read_context.hh
index b87a64f..e85adfb 100644
--- a/read_context.hh
+++ b/read_context.hh
@@ -120,6 +120,21 @@ class read_context final : public enable_lw_shared_from_this<read_context> {
streamed_mutation::forwarding _fwd;
bool _range_query;
autoupdating_underlying_reader _underlying;
+
+ // When reader enters a partition, it must be set up for reading that
+ // partition from the underlying mutation source (_sm) in one of two ways:
+ //
+ // 1) either _underlying is already in that partition, then _sm is set to the
+ // stream obtained from it.
+ //
+ // 2) _underlying is before the partition, then _underlying_snapshot and _key
+ // are set so that _sm can be created on demand.
+ //
+ streamed_mutation_opt _sm;
+ mutation_source_opt _underlying_snapshot;
+ dht::partition_range _sm_range;
+ stdx::optional<dht::decorated_key> _key;
+ row_cache::phase_type _phase;
public:
read_context(row_cache& cache,
schema_ptr schema,
@@ -148,6 +163,46 @@ class read_context final : public enable_lw_shared_from_this<read_context> {
streamed_mutation::forwarding fwd() const { return _fwd; }
bool is_range_query() const { return _range_query; }
autoupdating_underlying_reader& underlying() { return _underlying; }
+ row_cache::phase_type phase() const { return _phase; }
+ const dht::decorated_key& key() const { return _sm->decorated_key(); }
+private:
+ future<> create_sm();
+ future<> ensure_sm_created() {
+ if (_sm) {
+ return make_ready_future<>();
+ }
+ return create_sm();
+ }
+public:
+ // Prepares the underlying streamed_mutation to represent dk in given snapshot.
+ // Partitions must be entered with strictly monotonic keys.
+ // The key must be after the current range of the underlying() reader.
+ // The phase argument must match the snapshot's phase.
+ void enter_partition(const dht::decorated_key& dk, mutation_source& snapshot, row_cache::phase_type phase) {
+ _phase = phase;
+ _sm = {};
+ _underlying_snapshot = snapshot;
+ _key = dk;
+ }
+ // Prepares the underlying streamed_mutation to be sm.
+ // The phase argument must match the phase of the snapshot used to obtain sm.
+ void enter_partition(streamed_mutation&& sm, row_cache::phase_type phase) {
+ _phase = phase;
+ _sm = std::move(sm);
+ _underlying_snapshot = {};
+ }
+ // Fast forwards the underlying streamed_mutation to given range.
+ future<> fast_forward_to(position_range range) {
+ return ensure_sm_created().then([this, range = std::move(range)] () mutable {
+ return _sm->fast_forward_to(std::move(range));
+ });
+ }
+ // Gets the next fragment from the underlying streamed_mutation
+ future<mutation_fragment_opt> get_next_fragment() {
+ return ensure_sm_created().then([this] {
+ return (*_sm)();
+ });
+ }
};

}
diff --git a/row_cache.cc b/row_cache.cc
index b0e08df..7320359 100644
--- a/row_cache.cc
+++ b/row_cache.cc
@@ -276,6 +276,25 @@ class partition_range_iterator final {
}
};

+future<> read_context::create_sm() {
+ if (_range_query) {
+ // FIXME: Singular-range mutation reader don't support fast_forward_to(), so need to use a wide range
+ // here in case the same reader will need to be fast forwarded later.
+ _sm_range = dht::partition_range({dht::ring_position(*_key)}, {dht::ring_position(*_key)});
+ } else {
+ _sm_range = dht::partition_range::make_singular({dht::ring_position(*_key)});
+ }
+ return _underlying.fast_forward_to(std::move(_sm_range), *_underlying_snapshot, _phase).then([this] {
+ return _underlying.read_next_same_phase().then([this] (auto&& smo) {
+ if (!smo) {
+ _sm = make_empty_streamed_mutation(_cache.schema(), *_key, streamed_mutation::forwarding::yes);
+ } else {
+ _sm = std::move(*smo);
+ }
+ });
+ });
+}
+

Tomasz Grabiec

<tgrabiec@scylladb.com>
unread,
Jun 12, 2017, 3:03:16 PM6/12/17
to scylladb-dev@googlegroups.com
---
intrusive_set_external_comparator.hh | 8 ++++++--
1 file changed, 6 insertions(+), 2 deletions(-)

diff --git a/intrusive_set_external_comparator.hh b/intrusive_set_external_comparator.hh
index 6e2c513..b1a0f2f 100644
--- a/intrusive_set_external_comparator.hh
+++ b/intrusive_set_external_comparator.hh
@@ -208,6 +208,10 @@ class intrusive_set_external_comparator final {
}
template<class ElemCompare>
iterator insert(const_iterator hint, Elem& value, ElemCompare cmp) {
+ return insert_check(hint, value, std::move(cmp)).first;
+ }
+ template<class ElemCompare>
+ std::pair<iterator, bool> insert_check(const_iterator hint, Elem& value, ElemCompare cmp) {
algo::insert_commit_data commit_data;
std::pair<node_ptr, bool> ret =
algo::insert_unique_check(_header.this_ptr(),
@@ -215,8 +219,8 @@ class intrusive_set_external_comparator final {
key_of_value()(value),
key_node_comp(cmp),
commit_data);
- return ret.second ? insert_unique_commit(value, commit_data)
- : iterator(ret.first, priv_value_traits_ptr());
+ return ret.second ? std::make_pair(insert_unique_commit(value, commit_data), true)
+ : std::make_pair(iterator(ret.first, priv_value_traits_ptr()), false);
}
};

--
2.7.4

Tomasz Grabiec

<tgrabiec@scylladb.com>
unread,
Jun 12, 2017, 3:03:18 PM6/12/17
to scylladb-dev@googlegroups.com
---
streamed_mutation.hh | 47 +++++++++++++++++++++++++++++++++++++++++++++++
1 file changed, 47 insertions(+)

diff --git a/streamed_mutation.hh b/streamed_mutation.hh
index 710338a..348c3fb 100644
--- a/streamed_mutation.hh
+++ b/streamed_mutation.hh
@@ -690,3 +690,50 @@ class mutation_hasher {
consume_range_tombstones_until_end();
}
};
+
+
+GCC6_CONCEPT(
+ // F gets a stream element as an argument and returns the new value which replaces that element
+ // in the transformed stream.
+ template<typename F, typename Domain>
+ concept bool StreamedMutationTranformer() {
+ return requires(F f, mutation_fragment mf, schema_ptr s) {
+ { f(std::move(mf)) } -> mutation_fragment
+ { f(s) } -> schema_ptr
+ };
+ }
+)
+
+// Creates a stream which is like sm but with transformation applied to the elements.
+template<typename T>
+GCC6_CONCEPT(
+ requires StreamedMutationTranformer<T>
+)
+streamed_mutation transform(streamed_mutation sm, T t) {
+ class reader : public streamed_mutation::impl {
+ streamed_mutation _sm;
+ T _t;
+ public:
+ explicit reader(streamed_mutation sm, T&& t)
+ : impl(t(sm.schema()), sm.decorated_key(), sm.partition_tombstone())
+ , _sm(std::move(sm))
+ , _t(std::move(t))
+ { }
+
+ virtual future<> fill_buffer() override {
+ return _sm.fill_buffer().then([this] {
+ while (!_sm.is_buffer_empty()) {
+ push_mutation_fragment(_t(_sm.pop_mutation_fragment()));
+ }
+ _end_of_stream = _sm.is_end_of_stream();
+ });
+ }
+
+ virtual future<> fast_forward_to(position_range pr) override {
+ _end_of_stream = false;
+ forward_buffer_to(pr.start());
+ return _sm.fast_forward_to(std::move(pr));
+ }
+ };
+ return make_streamed_mutation<reader>(std::move(sm), std::move(t));
+}
--
2.7.4

Tomasz Grabiec

<tgrabiec@scylladb.com>
unread,
Jun 12, 2017, 3:03:18 PM6/12/17
to scylladb-dev@googlegroups.com, Piotr Jastrzebski
From: Piotr Jastrzebski <pi...@scylladb.com>

This streamed mutation populates cache with
the rows requested by the read. It takes whatever
it can find in the cache and fetches the remainings
from underlying source.

Signed-off-by: Piotr Jastrzebski <pi...@scylladb.com>

[tgrabiec:

- fixed maybe_add_to_cache_and_update_continuity() leaking entries if
the key already exists in the snapshot

- fixed a problem where population race could result in a read
missing some rows, because cache_streamed_mutation was advancing
the cursor, then deferring, and then checking continuity. We
should check continuity atomically with advancing.

- fixed rows_handle.maybe_refresh() being accessed outside of update
section in read_from_underlying() (undefined behavior)

- fixed a problem in start_reading_from_underlying() where we would
use incorrect start if lower_bound ended with a range tombstone
starting before a key.

- range tombstone trimming in add_to_buffer() could create a
tombstone which has too low start bound if last_rt.end was a
prefix and had inclusive end. invert_kind(end_kind) should be used
instead of unconditional inc_start.

- range tombstone trimming incorrectly assumed it is fine to trim
the tombstone from underlying to the previous fragment's end and
emit such tombstone. That would mean the stream can't emit any
fragments which start before previous tombstone's end. Solve with
range_tombstone_stream.

- split add_to_buffer() into overloads for clustering_row, and
range_tombstone. Better than wrapping into mutation_fragment
before the call and having add_to_buffer() rediscover the
information.

- changed maybe_add_to_cache_and_update_continuity() to not set
continuity to false for existing entries, it's not necessary

- moved range tombstone trimming to range_tombstone class
- moved range tombstone slicing code to range_tombstone_list and partition_snapshot
- can_populate::can_use_cache was unused, dropped
- dropped assumption that dummy entries are only at the end
- renamed maybe_add_to_cache_and_update_continuity() to maybe_add_to_cache()
- dropped no longer needed lower_bound class
- extracted row_handle to a seaparate patch
- made the copy-from-cache loop preemptable
- split maybe_add_next_to_buffer_and_update_continuity(bool)
- dropped cache_populator
- replaced "underlying" class with use of read_context
- replaced can_populate class with a function
- simplified lsa_manager methods to avoid moves
]
---
cache_streamed_mutation.hh | 484 +++++++++++++++++++++++++++++++++++++++++++++
row_cache.hh | 4 +
2 files changed, 488 insertions(+)
create mode 100644 cache_streamed_mutation.hh

diff --git a/cache_streamed_mutation.hh b/cache_streamed_mutation.hh
new file mode 100644
index 0000000..c9a351b
--- /dev/null
+++ b/cache_streamed_mutation.hh
@@ -0,0 +1,484 @@
+/*
+ * Copyright (C) 2017 ScyllaDB
+ */
+
+/*
+ * This file is part of Scylla.
+ *
+ * Scylla is free software: you can redistribute it and/or modify
+ * it under the terms of the GNU Affero General Public License as published by
+ * the Free Software Foundation, either version 3 of the License, or
+ * (at your option) any later version.
+ *
+ * Scylla is distributed in the hope that it will be useful,
+ * but WITHOUT ANY WARRANTY; without even the implied warranty of
+ * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
+ * GNU General Public License for more details.
+ *
+ * You should have received a copy of the GNU General Public License
+ * along with Scylla. If not, see <http://www.gnu.org/licenses/>.
+ */
+
+#pragma once
+
+#include <vector>
+#include "row_cache.hh"
+#include "mutation_reader.hh"
+#include "streamed_mutation.hh"
+#include "partition_version.hh"
+#include "utils/logalloc.hh"
+#include "query-request.hh"
+#include "partition_snapshot_reader.hh"
+#include "partition_snapshot_row_cursor.hh"
+#include "read_context.hh"
+
+namespace cache {
+
+class lsa_manager {
+ row_cache& _cache;
+public:
+ lsa_manager(row_cache& cache) : _cache(cache) { }
+ template<typename Func>
+ decltype(auto) run_in_read_section(const Func& func) {
+ return _cache._read_section(_cache._tracker.region(), [&func] () {
+ return with_linearized_managed_bytes([&func] () {
+ return func();
+ });
+ });
+ }
+ template<typename Func>
+ decltype(auto) run_in_update_section(const Func& func) {
+ return _cache._update_section(_cache._tracker.region(), [&func] () {
+ return with_linearized_managed_bytes([&func] () {
+ return func();
+ });
+ });
+ }
+ template<typename Func>
+ void run_in_update_section_with_allocator(Func&& func) {
+ return _cache._update_section(_cache._tracker.region(), [this, &func] () {
+ return with_linearized_managed_bytes([this, &func] () {
+ return with_allocator(_cache._tracker.region().allocator(), [this, &func] () mutable {
+ return func();
+ });
+ });
+ });
+ }
+ logalloc::region& region() { return _cache._tracker.region(); }
+ logalloc::allocating_section& read_section() { return _cache._read_section; }
+};
+
+class cache_streamed_mutation final : public streamed_mutation::impl {
+ lw_shared_ptr<partition_snapshot> _snp;
+ position_in_partition::tri_compare _position_cmp;
+
+ query::clustering_key_filter_ranges _ck_ranges;
+ query::clustering_row_ranges::const_iterator _ck_ranges_curr;
+ query::clustering_row_ranges::const_iterator _ck_ranges_end;
+
+ lsa_manager _lsa_manager;
+
+ stdx::optional<clustering_key> _last_row_key;
+
+ // We need to be prepared that we may get overlapping and out of order
+ // range tombstones. We must emit fragments with strictly monotonic positions,
+ // so we can't just trim such tombstones to the position of the last fragment.
+ // To solve that, range tombstones are accumulated first in a range_tombstone_stream
+ // and emitted once we have a fragment with a larger position.
+ range_tombstone_stream _tombstones;
+
+ // Holds the lower bound of a position range which hasn't been processed yet.
+ // Only fragments with positions < _lower_bound have been emitted.
+ position_in_partition _lower_bound;
+ position_in_partition_view _upper_bound;
+
+ bool _static_row_done = false;
+ bool _reading_underlying = false;
+ lw_shared_ptr<read_context> _read_context;
+ partition_snapshot_row_cursor _next_row;
+ bool _next_row_in_range = false;
+
+ future<> do_fill_buffer();
+ future<> copy_from_cache_to_buffer();
+ future<> process_static_row();
+ void move_to_end();
+ future<> move_to_next_range();
+ future<> move_to_current_range();
+ future<> move_to_next_entry();
+ // Emits all delayed range tombstones with positions smaller than upper_bound.
+ void drain_tombstones(position_in_partition_view upper_bound);
+ // Emits all delayed range tombstones.
+ void drain_tombstones();
+ void add_to_buffer(const partition_snapshot_row_cursor&);
+ void add_to_buffer(clustering_row&&);
+ void add_to_buffer(range_tombstone&&);
+ void add_to_buffer(mutation_fragment&&);
+ future<> read_from_underlying();
+ future<> start_reading_from_underlying();
+ bool after_current_range(position_in_partition_view position);
+ bool can_populate() const;
+ void maybe_update_continuity();
+ void maybe_add_to_cache(const mutation_fragment& mf);
+ void maybe_add_to_cache(const clustering_row& cr);
+ void maybe_add_to_cache(const range_tombstone& rt);
+ void maybe_add_to_cache(const static_row& sr);
+ void maybe_set_static_row_cached();
+public:
+ cache_streamed_mutation(schema_ptr s,
+ dht::decorated_key dk,
+ query::clustering_key_filter_ranges&& crr,
+ lw_shared_ptr<read_context> ctx,
+ lw_shared_ptr<partition_snapshot> snp,
+ row_cache& cache)
+ : streamed_mutation::impl(std::move(s), dk, snp->partition_tombstone())
+ , _snp(std::move(snp))
+ , _position_cmp(*_schema)
+ , _ck_ranges(std::move(crr))
+ , _ck_ranges_curr(_ck_ranges.begin())
+ , _ck_ranges_end(_ck_ranges.end())
+ , _lsa_manager(cache)
+ , _tombstones(*_schema)
+ , _lower_bound(position_in_partition::before_all_clustered_rows())
+ , _upper_bound(position_in_partition_view::before_all_clustered_rows())
+ , _read_context(std::move(ctx))
+ , _next_row(*_schema, cache._tracker.region(), *_snp)
+ { }
+ cache_streamed_mutation(const cache_streamed_mutation&) = delete;
+ cache_streamed_mutation(cache_streamed_mutation&&) = delete;
+ virtual future<> fill_buffer() override;
+ virtual ~cache_streamed_mutation() {
+ maybe_merge_versions(_snp, _lsa_manager.region(), _lsa_manager.read_section());
+ }
+};
+
+inline
+future<> cache_streamed_mutation::process_static_row() {
+ if (_snp->version()->partition().is_static_row_cached()) {
+ row sr = _snp->static_row();
+ if (!sr.empty()) {
+ push_mutation_fragment(mutation_fragment(static_row(std::move(sr))));
+ }
+ return make_ready_future<>();
+ } else {
+ return _read_context->get_next_fragment().then([this] (mutation_fragment_opt&& sr) {
+ if (sr) {
+ assert(sr->is_static_row());
+ maybe_add_to_cache(sr->as_static_row());
+ push_mutation_fragment(std::move(*sr));
+ }
+ maybe_set_static_row_cached();
+ });
+ }
+}
+
+inline
+future<> cache_streamed_mutation::fill_buffer() {
+ if (!_static_row_done) {
+ _static_row_done = true;
+ return process_static_row().then([this] {
+ return _lsa_manager.run_in_read_section([this] {
+ return move_to_current_range();
+ }).then([this] {
+ return fill_buffer();
+ });
+ });
+ }
+ return do_until([this] { return _end_of_stream || is_buffer_full(); }, [this] {
+ return do_fill_buffer();
+ });
+}
+
+inline
+future<> cache_streamed_mutation::do_fill_buffer() {
+ if (_reading_underlying) {
+ return read_from_underlying();
+ }
+ return _lsa_manager.run_in_read_section([this] {
+ auto same_pos = _next_row.maybe_refresh();
+ // FIXME: If continuity changed anywhere between _lower_bound and _next_row.position()
+ // we need to redo the lookup with _lower_bound. There is no eviction yet, so not yet a problem.
+ assert(same_pos);
+ while (!is_buffer_full() && !_end_of_stream && !_reading_underlying) {
+ future<> f = copy_from_cache_to_buffer();
+ if (!f.available() || need_preempt()) {
+ return f;
+ }
+ }
+ return make_ready_future<>();
+ });
+}
+
+inline
+future<> cache_streamed_mutation::read_from_underlying() {
+ return do_until([this] { return !_reading_underlying || is_buffer_full(); }, [this] {
+ return _read_context->get_next_fragment().then([this] (auto&& mfopt) {
+ if (!mfopt) {
+ _reading_underlying = false;
+ return _lsa_manager.run_in_update_section([this] {
+ auto same_pos = _next_row.maybe_refresh();
+ assert(same_pos); // FIXME: handle eviction
+ if (_next_row_in_range) {
+ this->maybe_update_continuity();
+ this->add_to_buffer(_next_row);
+ return this->move_to_next_entry();
+ } else {
+ if (no_clustering_row_between(*_schema, _upper_bound, _next_row.position())) {
+ this->maybe_update_continuity();
+ } else {
+ // FIXME: Insert dummy entry at _upper_bound.
+ }
+ return this->move_to_next_range();
+ }
+ });
+ } else {
+ this->maybe_add_to_cache(*mfopt);
+ this->add_to_buffer(std::move(*mfopt));
+ return make_ready_future<>();
+ }
+ });
+ });
+}
+
+inline
+void cache_streamed_mutation::maybe_update_continuity() {
+ if (can_populate() && _next_row.is_in_latest_version()) {
+ if (_last_row_key) {
+ if (_next_row.previous_row_in_latest_version_has_key(*_last_row_key)) {
+ _next_row.set_continuous(true);
+ }
+ } else if (!_ck_ranges_curr->start()) {
+ _next_row.set_continuous(true);
+ }
+ }
+}
+
+inline
+void cache_streamed_mutation::maybe_add_to_cache(const mutation_fragment& mf) {
+ if (mf.is_range_tombstone()) {
+ maybe_add_to_cache(mf.as_range_tombstone());
+ } else {
+ assert(mf.is_clustering_row());
+ const clustering_row& cr = mf.as_clustering_row();
+ maybe_add_to_cache(cr);
+ }
+}
+
+inline
+void cache_streamed_mutation::maybe_add_to_cache(const clustering_row& cr) {
+ if (!can_populate()) {
+ return;
+ }
+ _lsa_manager.run_in_update_section_with_allocator([this, &cr] {
+ mutation_partition& mp = _snp->version()->partition();
+ rows_entry::compare less(*_schema);
+
+ // FIXME: If _next_row is up to date, but latest version doesn't have iterator in
+ // current row (could be far away, so we'd do this often), then this will do
+ // the lookup in mp. This is not necessary, because _next_row has iterators for
+ // next rows in each version, even if they're not part of the current row.
+ // They're currently buried in the heap, but you could keep a vector of
+ // iterators per each version in addition to the heap.
+ auto new_entry = alloc_strategy_unique_ptr<rows_entry>(
+ current_allocator().construct<rows_entry>(cr.key(), cr.tomb(), cr.marker(), cr.cells()));
+ new_entry->set_continuous(false);
+ auto it = _next_row.has_up_to_date_row_from_latest_version()
+ ? _next_row.get_iterator_in_latest_version() : mp.clustered_rows().lower_bound(cr.key(), less);
+ auto insert_result = mp.clustered_rows().insert_check(it, *new_entry, less);
+ if (insert_result.second) {
+ new_entry.release();
+ }
+ it = insert_result.first;
+
+ rows_entry& e = *it;
+ if (_last_row_key) {
+ if (it == mp.clustered_rows().begin()) {
+ // FIXME: check whether entry for _last_row_key is in older versions and if so set
+ // continuity to true.
+ } else {
+ auto prev_it = it;
+ --prev_it;
+ clustering_key_prefix::tri_compare tri_comp(*_schema);
+ if (tri_comp(*_last_row_key, prev_it->key()) == 0) {
+ e.set_continuous(true);
+ }
+ }
+ } else if (!_ck_ranges_curr->start()) {
+ e.set_continuous(true);
+ } else {
+ // FIXME: Insert dummy entry at _ck_ranges_curr->start()
+ }
+ });
+}
+
+inline
+bool cache_streamed_mutation::after_current_range(position_in_partition_view p) {
+ return _position_cmp(p, _upper_bound) >= 0;
+}
+
+inline
+future<> cache_streamed_mutation::start_reading_from_underlying() {
+ _reading_underlying = true;
+ auto end = _next_row_in_range ? position_in_partition(_next_row.position())
+ : position_in_partition(_upper_bound);
+ return _read_context->fast_forward_to(position_range{_lower_bound, std::move(end)});
+}
+
+inline
+future<> cache_streamed_mutation::copy_from_cache_to_buffer() {
+ position_in_partition_view next_lower_bound = _next_row.dummy() ? _next_row.position() : position_in_partition_view::after_key(_next_row.key());
+ for (auto&& rts : _snp->range_tombstones(*_schema, _lower_bound, _next_row_in_range ? next_lower_bound : _upper_bound)) {
+ add_to_buffer(std::move(rts));
+ if (is_buffer_full()) {
+ return make_ready_future<>();
+ }
+ }
+ if (_next_row_in_range) {
+ add_to_buffer(_next_row);
+ return move_to_next_entry();
+ } else {
+ return move_to_next_range();
+ }
+}
+
+inline
+void cache_streamed_mutation::move_to_end() {
+ drain_tombstones();
+ _end_of_stream = true;
+}
+
+inline
+future<> cache_streamed_mutation::move_to_next_range() {
+ ++_ck_ranges_curr;
+ if (_ck_ranges_curr == _ck_ranges_end) {
+ move_to_end();
+ return make_ready_future<>();
+ } else {
+ return move_to_current_range();
+ }
+}
+
+inline
+future<> cache_streamed_mutation::move_to_current_range() {
+ _last_row_key = std::experimental::nullopt;
+ _lower_bound = position_in_partition::for_range_start(*_ck_ranges_curr);
+ _upper_bound = position_in_partition_view::for_range_end(*_ck_ranges_curr);
+ auto complete_until_next = _next_row.advance_to(_lower_bound) || _next_row.continuous();
+ _next_row_in_range = !after_current_range(_next_row.position());
+ if (!complete_until_next) {
+ return start_reading_from_underlying();
+ }
+ return make_ready_future<>();
+}
+
+// _next_row must be inside the range.
+inline
+future<> cache_streamed_mutation::move_to_next_entry() {
+ if (no_clustering_row_between(*_schema, _next_row.position(), _upper_bound)) {
+ return move_to_next_range();
+ } else {
+ if (!_next_row.next()) {
+ move_to_end();
+ return make_ready_future<>();
+ }
+ _next_row_in_range = !after_current_range(_next_row.position());
+ if (!_next_row.continuous()) {
+ return start_reading_from_underlying();
+ }
+ return make_ready_future<>();
+ }
+}
+
+inline
+void cache_streamed_mutation::drain_tombstones(position_in_partition_view pos) {
+ while (auto mfo = _tombstones.get_next(pos)) {
+ push_mutation_fragment(std::move(*mfo));
+ }
+}
+
+inline
+void cache_streamed_mutation::drain_tombstones() {
+ while (auto mfo = _tombstones.get_next()) {
+ push_mutation_fragment(std::move(*mfo));
+ }
+}
+
+inline
+void cache_streamed_mutation::add_to_buffer(mutation_fragment&& mf) {
+ if (mf.is_clustering_row()) {
+ add_to_buffer(std::move(std::move(mf).as_clustering_row()));
+ } else {
+ assert(mf.is_range_tombstone());
+ add_to_buffer(std::move(mf).as_range_tombstone());
+ }
+}
+
+inline
+void cache_streamed_mutation::add_to_buffer(const partition_snapshot_row_cursor& row) {
+ if (!row.dummy()) {
+ add_to_buffer(row.row());
+ } else {
+ _lower_bound = row.position();
+ }
+}
+
+inline
+void cache_streamed_mutation::add_to_buffer(clustering_row&& row) {
+ drain_tombstones(row.position());
+ _last_row_key = row.key();
+ _lower_bound = position_in_partition::after_key(row.key());
+ push_mutation_fragment(std::move(row));
+}
+
+inline
+void cache_streamed_mutation::add_to_buffer(range_tombstone&& rt) {
+ // This guarantees that rt starts after any emitted clustering_row
+ if (!rt.trim_front(*_schema, _lower_bound)) {
+ return;
+ }
+ _lower_bound = position_in_partition(rt.position());
+ _tombstones.apply(std::move(rt));
+ drain_tombstones(_lower_bound);
+}
+
+inline
+void cache_streamed_mutation::maybe_add_to_cache(const range_tombstone& rt) {
+ if (can_populate()) {
+ _lsa_manager.run_in_update_section_with_allocator([&] {
+ _snp->version()->partition().apply_row_tombstone(*_schema, rt);
+ });
+ }
+}
+
+inline
+void cache_streamed_mutation::maybe_add_to_cache(const static_row& sr) {
+ if (can_populate()) {
+ _lsa_manager.run_in_update_section_with_allocator([&] {
+ _snp->version()->partition().static_row().apply(*_schema, column_kind::static_column, sr.cells());
+ });
+ }
+}
+
+inline
+void cache_streamed_mutation::maybe_set_static_row_cached() {
+ if (can_populate()) {
+ _snp->version()->partition().set_static_row_cached(true);
+ }
+}
+
+inline
+bool cache_streamed_mutation::can_populate() const {
+ return _snp->at_latest_version() && _read_context->cache().phase_of(_read_context->key()) == _read_context->phase();
+}
+
+} // namespace cache
+
+inline streamed_mutation make_cache_streamed_mutation(schema_ptr s,
+ dht::decorated_key dk,
+ query::clustering_key_filter_ranges crr,
+ row_cache& cache,
+ lw_shared_ptr<cache::read_context> ctx,
+ lw_shared_ptr<partition_snapshot> snp)
+{
+ return make_streamed_mutation<cache::cache_streamed_mutation>(
+ std::move(s), std::move(dk), std::move(crr), std::move(ctx), std::move(snp), cache);
+}
diff --git a/row_cache.hh b/row_cache.hh
index 9b7b5bd..c3485a1 100644
--- a/row_cache.hh
+++ b/row_cache.hh
@@ -44,7 +44,9 @@ class row_cache;
namespace cache {

class autoupdating_underlying_reader;
+class cache_streamed_mutation;
class read_context;
+class lsa_manager;

}

@@ -250,6 +252,8 @@ class row_cache final {
friend class cache::autoupdating_underlying_reader;
friend class single_partition_populating_reader;
friend class cache_entry;
+ friend class cache::cache_streamed_mutation;
+ friend class cache::lsa_manager;
friend class cache::read_context;
friend class partition_range_iterator;
public:
--
2.7.4

Tomasz Grabiec

<tgrabiec@scylladb.com>
unread,
Jun 12, 2017, 3:03:19 PM6/12/17
to scylladb-dev@googlegroups.com
---
converting_mutation_partition_applier.hh | 14 ++++++++++++--
1 file changed, 12 insertions(+), 2 deletions(-)

diff --git a/converting_mutation_partition_applier.hh b/converting_mutation_partition_applier.hh
index d6dce15..bd5f1d2 100644
--- a/converting_mutation_partition_applier.hh
+++ b/converting_mutation_partition_applier.hh
@@ -38,12 +38,12 @@ class converting_mutation_partition_applier : public mutation_partition_visitor
static bool is_compatible(const column_definition& new_def, const data_type& old_type, column_kind kind) {
return ::is_compatible(new_def.kind, kind) && new_def.type->is_value_compatible_with(*old_type);
}
- void accept_cell(row& dst, column_kind kind, const column_definition& new_def, const data_type& old_type, atomic_cell_view cell) {
+ static void accept_cell(row& dst, column_kind kind, const column_definition& new_def, const data_type& old_type, atomic_cell_view cell) {
if (is_compatible(new_def, old_type, kind) && cell.timestamp() > new_def.dropped_at()) {
dst.apply(new_def, atomic_cell_or_collection(cell));
}
}
- void accept_cell(row& dst, column_kind kind, const column_definition& new_def, const data_type& old_type, collection_mutation_view cell) {
+ static void accept_cell(row& dst, column_kind kind, const column_definition& new_def, const data_type& old_type, collection_mutation_view cell) {
if (!is_compatible(new_def, old_type, kind)) {
return;
}
@@ -117,4 +117,14 @@ class converting_mutation_partition_applier : public mutation_partition_visitor
accept_cell(_current_row->cells(), column_kind::regular_column, *def, col.type(), collection);
}
}
+
+ // Appends the cell to dst upgrading it to the new schema.
+ // Cells must have monotonic names.
+ static void append_cell(row& dst, column_kind kind, const column_definition& new_def, const data_type& old_type, const atomic_cell_or_collection& cell) {
+ if (new_def.is_atomic()) {
+ accept_cell(dst, kind, new_def, old_type, cell.as_atomic_cell());
+ } else {
+ accept_cell(dst, kind, new_def, old_type, cell.as_collection_mutation());
+ }
+ }
};
--
2.7.4

Tomasz Grabiec

<tgrabiec@scylladb.com>
unread,
Jun 12, 2017, 3:03:20 PM6/12/17
to scylladb-dev@googlegroups.com
---
schema_upgrader.hh | 64 +++++++++++++++++++++++++++++++++++++++++
tests/streamed_mutation_test.cc | 20 +++++++++++++
2 files changed, 84 insertions(+)
create mode 100644 schema_upgrader.hh

diff --git a/schema_upgrader.hh b/schema_upgrader.hh
new file mode 100644
index 0000000..a8a52b0
--- /dev/null
+++ b/schema_upgrader.hh
@@ -0,0 +1,64 @@
+/*
+ * Copyright (C) 2017 ScyllaDB
+ */
+
+/*
+ * This file is part of Scylla.
+ *
+ * Scylla is free software: you can redistribute it and/or modify
+ * it under the terms of the GNU Affero General Public License as published by
+ * the Free Software Foundation, either version 3 of the License, or
+ * (at your option) any later version.
+ *
+ * Scylla is distributed in the hope that it will be useful,
+ * but WITHOUT ANY WARRANTY; without even the implied warranty of
+ * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
+ * GNU General Public License for more details.
+ *
+ * You should have received a copy of the GNU General Public License
+ * along with Scylla. If not, see <http://www.gnu.org/licenses/>.
+ */
+
+#pragma once
+
+#include "streamed_mutation.hh"
+#include "converting_mutation_partition_applier.hh"
+
+// A StreamedMutationTransformer which transforms the stream to a different schema
+class schema_upgrader {
+ schema_ptr _prev;
+ schema_ptr _new;
+private:
+ row transform(row&& r, column_kind kind) {
+ row new_row;
+ r.for_each_cell([&] (column_id id, atomic_cell_or_collection& cell) {
+ const column_definition& col = _prev->column_at(kind, id);
+ const column_definition* new_col = _new->get_column_definition(col.name());
+ if (new_col) {
+ converting_mutation_partition_applier::append_cell(new_row, kind, *new_col, col.type, std::move(cell));
+ }
+ });
+ return new_row;
+ }
+public:
+ schema_upgrader(schema_ptr s)
+ : _new(std::move(s))
+ { }
+ schema_ptr operator()(schema_ptr old) {
+ _prev = std::move(old);
+ return _new;
+ }
+ mutation_fragment consume(static_row&& row) {
+ return mutation_fragment(static_row(transform(std::move(row.cells()), column_kind::static_column)));
+ }
+ mutation_fragment consume(clustering_row&& row) {
+ return mutation_fragment(clustering_row(row.key(), row.tomb(), row.marker(),
+ transform(std::move(row.cells()), column_kind::regular_column)));
+ }
+ mutation_fragment consume(range_tombstone&& rt) {
+ return std::move(rt);
+ }
+ mutation_fragment operator()(mutation_fragment&& mf) {
+ return std::move(mf).consume(*this);
+ }
+};
diff --git a/tests/streamed_mutation_test.cc b/tests/streamed_mutation_test.cc
index bd651ac..89e677c 100644
--- a/tests/streamed_mutation_test.cc
+++ b/tests/streamed_mutation_test.cc
@@ -29,8 +29,10 @@
#include "tests/test_services.hh"
#include "schema_builder.hh"
#include "total_order_check.hh"
+#include "schema_upgrader.hh"

#include "disk-error-handler.hh"
+#include "mutation_assertions.hh"

thread_local disk_error_signal_type commit_error;
thread_local disk_error_signal_type general_disk_error;
@@ -537,3 +539,21 @@ SEASTAR_TEST_CASE(test_ordering_of_position_in_partition_and_composite_view_in_a
.check();
});
}
+
+SEASTAR_TEST_CASE(test_schema_upgrader_is_equivalent_with_mutation_upgrade) {
+ return seastar::async([] {
+ for_each_mutation_pair([](const mutation& m1, const mutation& m2, are_equal eq) {
+ if (m1.schema()->version() != m2.schema()->version()) {
+ // upgrade m1 to m2's schema
+
+ auto from_upgrader = mutation_from_streamed_mutation(
+ transform(streamed_mutation_from_mutation(m1), schema_upgrader(m2.schema()))).get0();
+
+ auto regular = m1;
+ regular.upgrade(m2.schema());
+
+ assert_that(from_upgrader).has_mutation().is_equal_to(regular);
+ }
+ });
+ });
+}
--
2.7.4

Tomasz Grabiec

<tgrabiec@scylladb.com>
unread,
Jun 12, 2017, 3:03:21 PM6/12/17
to scylladb-dev@googlegroups.com
---
row_cache.hh | 7 +++-
row_cache.cc | 118 +++++++++++++++++++++++++++++------------------------------
2 files changed, 63 insertions(+), 62 deletions(-)

diff --git a/row_cache.hh b/row_cache.hh
index c3485a1..b6e8c5b 100644
--- a/row_cache.hh
+++ b/row_cache.hh
@@ -77,6 +77,7 @@ class cache_entry {
cache_link_type _cache_link;
friend class size_calculator;

+ streamed_mutation do_read(row_cache&, cache::read_context& reader);
public:
friend class row_cache;
friend class cache_tracker;
@@ -137,6 +138,7 @@ class cache_entry {
const schema_ptr& schema() const { return _schema; }
schema_ptr& schema() { return _schema; }
streamed_mutation read(row_cache&, cache::read_context& reader);
+ streamed_mutation read(row_cache&, cache::read_context& reader, streamed_mutation&& underlying, utils::phased_barrier::phase_type);
bool continuous() const { return _flags._continuous; }
void set_continuous(bool value) { _flags._continuous = value; }

@@ -385,8 +387,9 @@ class row_cache final {

const stats& stats() const { return _stats; }
public:
- // Populate cache from given mutation. The mutation must contain all
- // information there is for its partition in the underlying data sources.
+ // Populate cache from given mutation, which must be fully continuous.
+ // Intended to be used only in tests.
+ // Can only be called prior to any reads.
void populate(const mutation& m, const previous_entry_pointer* previous = nullptr);

// Synchronizes cache with the underlying data source from a memtable which
diff --git a/row_cache.cc b/row_cache.cc
index 7320359..ee4105c 100644
--- a/row_cache.cc
+++ b/row_cache.cc
@@ -32,7 +32,9 @@
#include <boost/version.hpp>
#include <sys/sdt.h>
#include "stdx.hh"
+#include "cache_streamed_mutation.hh"
#include "read_context.hh"
+#include "schema_upgrader.hh"

using namespace std::chrono_literals;
using namespace cache;
@@ -44,7 +46,7 @@ thread_local seastar::thread_scheduling_group row_cache::_update_thread_scheduli

mutation_reader
row_cache::create_underlying_reader(read_context& ctx, mutation_source& src, const dht::partition_range& pr) {
- return src(_schema, pr, query::full_slice, ctx.pc(), ctx.trace_state(), streamed_mutation::forwarding::no);
+ return src(_schema, pr, ctx.slice(), ctx.pc(), ctx.trace_state(), streamed_mutation::forwarding::yes);
}

cache_tracker& global_cache_tracker() {
@@ -300,7 +302,6 @@ class single_partition_populating_reader final : public mutation_reader::impl {
row_cache& _cache;
mutation_reader _delegate;
lw_shared_ptr<read_context> _read_context;
- bool done = false;
public:
single_partition_populating_reader(row_cache& cache,
lw_shared_ptr<read_context> context)
@@ -309,32 +310,21 @@ class single_partition_populating_reader final : public mutation_reader::impl {
{ }

virtual future<streamed_mutation_opt> operator()() override {
- if (done) {
+ if (!_read_context) {
return make_ready_future<streamed_mutation_opt>(streamed_mutation_opt());
}
- done = true;
auto src_and_phase = _cache.snapshot_of(_read_context->range().start()->value());
auto phase = src_and_phase.phase;
_delegate = _cache.create_underlying_reader(*_read_context, src_and_phase.snapshot, _read_context->range());
- return _delegate().then([this, phase] (auto sm) mutable {
+ return _delegate().then([this, phase] (auto sm) mutable -> streamed_mutation_opt {
+ auto ctx = std::move(_read_context);
if (!sm) {
- return make_ready_future<streamed_mutation_opt>(streamed_mutation_opt());
+ return std::move(sm);
}
- return mutation_from_streamed_mutation(std::move(sm)).then([this, phase] (mutation_opt&& mo) {
- if (mo) {
- if (phase == _cache.phase_of(_read_context->range().start()->value())) {
- _cache.populate(*mo);
- } else {
- _cache._tracker.on_mispopulate();
- }
- mo->upgrade(_read_context->schema());
- auto ck_ranges = query::clustering_key_filter_ranges::get_ranges(*_read_context->schema(), _read_context->slice(), mo->key());
- auto filtered_partition = mutation_partition(std::move(mo->partition()), *(mo->schema()), std::move(ck_ranges));
- mo->partition() = std::move(filtered_partition);
- return make_ready_future<streamed_mutation_opt>(streamed_mutation_from_mutation(std::move(*mo), _read_context->fwd()));
- }
- return make_ready_future<streamed_mutation_opt>(streamed_mutation_opt());
- });
+ return _cache._read_section(_cache._tracker.region(), [&] {
+ cache_entry& e = _cache.find_or_create(sm->decorated_key());
+ return e.read(_cache, *ctx, std::move(*sm), phase);
+ });
});
}
};
@@ -392,28 +382,19 @@ class range_populating_reader {
{}

future<streamed_mutation_opt> operator()() {
- return _reader().then([this] (streamed_mutation_opt smopt) mutable {
- return mutation_from_streamed_mutation(std::move(smopt)).then(
- [this] (mutation_opt&& mo) mutable {
- if (!mo) {
+ return _reader().then([this] (streamed_mutation_opt smopt) mutable -> streamed_mutation_opt {
+ {
+ if (!smopt) {
handle_end_of_stream();
- return make_ready_future<streamed_mutation_opt>();
+ return std::move(smopt);
}
-
_cache.on_miss();
- if (_reader.creation_phase() == _cache.phase_of(mo->decorated_key())) {
- _cache.populate(*mo, can_set_continuity() ? &*_last_key : nullptr);
- } else {
- _cache._tracker.on_mispopulate();
- }
- _last_key = mo->decorated_key();
-
- mo->upgrade(_read_context.schema());
- auto ck_ranges = query::clustering_key_filter_ranges::get_ranges(*_read_context.schema(), _read_context.slice(), mo->key());
- auto filtered_partition = mutation_partition(std::move(mo->partition()), *mo->schema(), std::move(ck_ranges));
- mo->partition() = std::move(filtered_partition);
- return make_ready_future<streamed_mutation_opt>(streamed_mutation_from_mutation(std::move(*mo), _read_context.fwd()));
- });
+ return _cache._read_section(_cache._tracker.region(), [&] {
+ cache_entry& e = _cache.find_or_create(smopt->decorated_key(), can_set_continuity() ? &*_last_key : nullptr);
+ _last_key = smopt->decorated_key();
+ return e.read(_cache, _read_context, std::move(*smopt), _reader.creation_phase());
+ });
+ }
});
}

@@ -566,10 +547,8 @@ row_cache::make_reader(schema_ptr s,
cache_entry& e = *i;
_tracker.touch(e);
upgrade_entry(e);
- mutation_reader reader;
- reader = make_reader_returning(e.read(*this, *ctx));
on_hit();
- return reader;
+ return make_reader_returning(e.read(*this, *ctx));
} else {
on_miss();
return make_mutation_reader<single_partition_populating_reader>(*this, std::move(ctx));
@@ -654,10 +633,7 @@ void row_cache::populate(const mutation& m, const previous_entry_pointer* previo
_tracker.insert(*entry);
return _partitions.insert(i, *entry);
}, [&] (auto i) {
- _tracker.touch(*i);
- // We cache whole partitions right now, so if cache already has this partition,
- // it must be complete, so do nothing.
- _tracker.on_miss_already_populated(); // #1534
+ throw std::runtime_error(sprint("cache already contains entry for {}", m.key()));
});
});
}
@@ -752,7 +728,7 @@ future<> row_cache::update(memtable& m, partition_presence_checker presence_chec
if (cache_i != partitions_end() && cache_i->key().equal(*_schema, mem_e.key())) {
cache_entry& entry = *cache_i;
upgrade_entry(entry);
- entry.partition().apply(*_schema, std::move(mem_e.partition()), *mem_e.schema());
+ entry.partition().apply_to_incomplete(*_schema, std::move(mem_e.partition()), *mem_e.schema());
_tracker.touch(entry);
_tracker.on_merge();
} else if (presence_checker(mem_e.key()) ==
@@ -895,19 +871,41 @@ void row_cache::set_schema(schema_ptr new_schema) noexcept {
_schema = std::move(new_schema);
}

-streamed_mutation cache_entry::read(row_cache& rc, read_context& ctx) {
- auto s = ctx.schema();
- auto& slice = ctx.slice();
- auto fwd = ctx.fwd();
- if (_schema->version() != s->version()) {
- auto ck_ranges = query::clustering_key_filter_ranges::get_ranges(*s, slice, _key.key());
- auto mp = mutation_partition(_pe.squashed(_schema, s), *s, std::move(ck_ranges));
- auto m = mutation(s, _key, std::move(mp));
- return streamed_mutation_from_mutation(std::move(m), fwd);
+streamed_mutation cache_entry::read(row_cache& rc, read_context& reader) {
+ auto source_and_phase = rc.snapshot_of(_key);
+ reader.enter_partition(_key, source_and_phase.snapshot, source_and_phase.phase);
+ return do_read(rc, reader);
+}
+
+streamed_mutation cache_entry::read(row_cache& rc, read_context& reader,
+ streamed_mutation&& sm, row_cache::phase_type phase) {
+ auto entry_phase = rc.phase_of(_key);
+ if (entry_phase != phase) {
+ rc._tracker.on_mispopulate();
+ if (reader.schema()->version() != _schema->version()) {
+ sm = transform(std::move(sm), schema_upgrader(reader.schema()));
+ }
+ if (reader.fwd() == streamed_mutation::forwarding::no) {
+ sm = streamed_mutation_from_forwarding_streamed_mutation(std::move(sm));
+ }
+ return std::move(sm);
+ }
+ reader.enter_partition(std::move(sm), phase);
+ return do_read(rc, reader);
+}
+
+// Assumes reader is in the corresponding partition
+streamed_mutation cache_entry::do_read(row_cache& rc, read_context& reader) {
+ auto snp = _pe.read(_schema, reader.phase());
+ auto ckr = query::clustering_key_filter_ranges::get_ranges(*_schema, reader.slice(), _key.key());
+ auto sm = make_cache_streamed_mutation(_schema, _key, std::move(ckr), rc, reader.shared_from_this(), std::move(snp));
+ if (reader.schema()->version() != _schema->version()) {
+ sm = transform(std::move(sm), schema_upgrader(reader.schema()));
+ }
+ if (reader.fwd() == streamed_mutation::forwarding::yes) {
+ sm = make_forwardable(std::move(sm));
}
- auto ckr = query::clustering_key_filter_ranges::get_ranges(*s, slice, _key.key());
- auto snp = _pe.read(_schema);
- return make_partition_snapshot_reader(_schema, _key, std::move(ckr), snp, rc._tracker.region(), rc._read_section, { }, fwd);
+ return std::move(sm);
}

const schema_ptr& row_cache::schema() const {
--
2.7.4

Tomasz Grabiec

<tgrabiec@scylladb.com>
unread,
Jun 12, 2017, 3:03:22 PM6/12/17
to scylladb-dev@googlegroups.com, Piotr Jastrzebski
From: Piotr Jastrzebski <pi...@scylladb.com>

Some apply methods where replaced by apply_to_incomplete.ZZ

Signed-off-by: Piotr Jastrzebski <pi...@scylladb.com>
---
partition_version.hh | 11 -------
partition_version.cc | 87 ----------------------------------------------------
2 files changed, 98 deletions(-)

diff --git a/partition_version.hh b/partition_version.hh
index 9a8e874..b49145c 100644
--- a/partition_version.hh
+++ b/partition_version.hh
@@ -247,7 +247,6 @@ class partition_entry {

void set_version(partition_version*);

- void apply(const schema& s, partition_version* pv, const schema& pv_schema);
void apply_to_incomplete(const schema& s, partition_version* other);
public:
partition_entry() = default;
@@ -281,10 +280,6 @@ class partition_entry {
// Strong exception guarantees.
void apply(const schema& s, const mutation_partition& mp, const schema& mp_schema);

- // Same exception guarantees as:
- // mutation_partition::apply(const schema&, mutation_partition&&, const schema&)
- void apply(const schema& s, mutation_partition&& mp, const schema& mp_schema);
-
// Strong exception guarantees.
void apply(const schema& s, mutation_partition_view mpv, const schema& mp_schema);

@@ -307,12 +302,6 @@ class partition_entry {
// succeeds the result will be as if the first attempt didn't fail.
void apply_to_incomplete(const schema& s, partition_entry&& pe, const schema& pe_schema);

- // Weak exception guarantees.
- // If an exception is thrown this and pe will be left in some valid states
- // such that if the operation is retried (possibly many times) and eventually
- // succeeds the result will be as if the first attempt didn't fail.
- void apply(const schema& s, partition_entry&& pe, const schema& pe_schema);
-
mutation_partition squashed(schema_ptr from, schema_ptr to);

// needs to be called with reclaiming disabled
diff --git a/partition_version.cc b/partition_version.cc
index f94842c..c9d3d85 100644
--- a/partition_version.cc
+++ b/partition_version.cc
@@ -187,20 +187,6 @@ void partition_entry::set_version(partition_version* new_version)
_version = partition_version_ref(*new_version);
}

-void partition_entry::apply(const schema& s, partition_version* pv, const schema& pv_schema)
-{
- if (!_snapshot) {
- _version->partition().apply(s, std::move(pv->partition()), pv_schema);
- current_allocator().destroy(pv);
- } else {
- if (s.version() != pv_schema.version()) {
- pv->partition().upgrade(pv_schema, s);
- }
- pv->insert_before(*_version);
- set_version(pv);
- }
-}
-
void partition_entry::apply(const schema& s, const mutation_partition& mp, const schema& mp_schema)
{
if (!_snapshot) {
@@ -217,22 +203,6 @@ void partition_entry::apply(const schema& s, const mutation_partition& mp, const
}
}

-void partition_entry::apply(const schema& s, mutation_partition&& mp, const schema& mp_schema)
-{
- if (!_snapshot) {
- _version->partition().apply(s, std::move(mp), mp_schema);
- } else {
- if (s.version() != mp_schema.version()) {
- apply(s, mp, mp_schema);
- } else {
- auto new_version = current_allocator().construct<partition_version>(std::move(mp));
- new_version->insert_before(*_version);
-
- set_version(new_version);
- }
- }
-}
-
void partition_entry::apply(const schema& s, mutation_partition_view mpv, const schema& mp_schema)
{
if (!_snapshot) {
@@ -247,63 +217,6 @@ void partition_entry::apply(const schema& s, mutation_partition_view mpv, const
}
}

-void partition_entry::apply(const schema& s, partition_entry&& pe, const schema& mp_schema)
-{
- auto begin = &*pe._version;
- auto snapshot = pe._snapshot;
- if (pe._snapshot) {
- pe._snapshot->_version = std::move(pe._version);
- pe._snapshot->_entry = nullptr;
- pe._snapshot = nullptr;
- }
- pe._version = { };
-
- auto current = begin;
- if (!current->next() && !current->is_referenced()) {
- try {
- apply(s, current, mp_schema);
- } catch (...) {
- pe._version = partition_version_ref(*current);
- throw;
- }
- return;
- }
-
- try {
- while (current && !current->is_referenced()) {
- auto next = current->next();
- apply(s, std::move(current->partition()), mp_schema);
- // Leave current->partition() valid (albeit empty) in case we throw later.
- current->partition() = mutation_partition(mp_schema.shared_from_this());
- current = next;
- }
- while (current) {
- auto next = current->next();
- apply(s, current->partition(), mp_schema);
- current = next;
- }
- } catch (...) {
- if (snapshot) {
- pe._snapshot = snapshot;
- snapshot->_entry = &pe;
- pe._version = std::move(snapshot->_version);
- } else {
- pe._version = partition_version_ref(*begin);
- }
- throw;
- }
-
- current = begin;
- while (current && !current->is_referenced()) {
- auto next = current->next();
- current_allocator().destroy(current);
- current = next;
- }
- if (current) {
- current->back_reference().mark_as_unique_owner();
- }
-}
-
namespace {

// When applying partition_entry to an incomplete partition_entry this class is used to represent
--
2.7.4

Tomasz Grabiec

<tgrabiec@scylladb.com>
unread,
Jun 12, 2017, 3:03:23 PM6/12/17
to scylladb-dev@googlegroups.com
---
tests/row_cache_test.cc | 68 +++++++++++++++++++++++++++++++++++++++++++++++++
1 file changed, 68 insertions(+)

diff --git a/tests/row_cache_test.cc b/tests/row_cache_test.cc
index de07ce2..6b05da2 100644
--- a/tests/row_cache_test.cc
+++ b/tests/row_cache_test.cc
@@ -29,6 +29,7 @@
#include "tests/mutation_source_test.hh"

#include "schema_builder.hh"
+#include "simple_schema.hh"
#include "row_cache.hh"
#include "core/thread.hh"
#include "memtable.hh"
@@ -1396,3 +1397,70 @@ SEASTAR_TEST_CASE(test_lru) {
.produces_end_of_stream();
});
}
+
+SEASTAR_TEST_CASE(test_scan_with_partial_partitions) {
+ return seastar::async([] {
+ simple_schema s;
+ auto cache_mt = make_lw_shared<memtable>(s.schema());
+
+ auto pkeys = s.make_pkeys(3);
+
+ mutation m1(pkeys[0], s.schema());
+ s.add_row(m1, s.make_ckey(0), "v1");
+ s.add_row(m1, s.make_ckey(1), "v2");
+ s.add_row(m1, s.make_ckey(2), "v3");
+ s.add_row(m1, s.make_ckey(3), "v4");
+ cache_mt->apply(m1);
+
+ mutation m2(pkeys[1], s.schema());
+ s.add_row(m2, s.make_ckey(0), "v5");
+ s.add_row(m2, s.make_ckey(1), "v6");
+ s.add_row(m2, s.make_ckey(2), "v7");
+ cache_mt->apply(m2);
+
+ mutation m3(pkeys[2], s.schema());
+ s.add_row(m3, s.make_ckey(0), "v8");
+ s.add_row(m3, s.make_ckey(1), "v9");
+ s.add_row(m3, s.make_ckey(2), "v10");
+ cache_mt->apply(m3);
+
+ cache_tracker tracker;
+ row_cache cache(s.schema(), snapshot_source_from_snapshot(cache_mt->as_data_source()), tracker);
+
+ // partially populate all up to middle of m1
+ {
+ auto slice = partition_slice_builder(*s.schema())
+ .with_range(query::clustering_range::make_ending_with(s.make_ckey(1)))
+ .build();
+ auto prange = dht::partition_range::make_ending_with(dht::ring_position(m1.decorated_key()));
+ assert_that(cache.make_reader(s.schema(), prange, slice))
+ .produces(m1, slice.row_ranges(*s.schema(), m1.key()))
+ .produces_end_of_stream();
+ }
+
+ // partially populate m3
+ {
+ auto slice = partition_slice_builder(*s.schema())
+ .with_range(query::clustering_range::make_ending_with(s.make_ckey(1)))
+ .build();
+ auto prange = dht::partition_range::make_singular(m3.decorated_key());
+ assert_that(cache.make_reader(s.schema(), prange, slice))
+ .produces(m3, slice.row_ranges(*s.schema(), m3.key()))
+ .produces_end_of_stream();
+ }
+
+ // full scan
+ assert_that(cache.make_reader(s.schema()))
+ .produces(m1)
+ .produces(m2)
+ .produces(m3)
+ .produces_end_of_stream();
+
+ // full scan after full scan
+ assert_that(cache.make_reader(s.schema()))
+ .produces(m1)
+ .produces(m2)
+ .produces(m3)
+ .produces_end_of_stream();

Tomasz Grabiec

<tgrabiec@scylladb.com>
unread,
Jun 12, 2017, 3:03:23 PM6/12/17
to scylladb-dev@googlegroups.com, Piotr Jastrzebski
From: Piotr Jastrzebski <pi...@scylladb.com>

[tgrabiec: Extracted from "row_cache: Introduce cache_streamed_mutation"]
---
tests/row_cache_test.cc | 103 ++++++++++++++++++++++++++++++++++++++----------
1 file changed, 83 insertions(+), 20 deletions(-)

diff --git a/tests/row_cache_test.cc b/tests/row_cache_test.cc
index 6b05da2..891258f 100644
--- a/tests/row_cache_test.cc
+++ b/tests/row_cache_test.cc
@@ -614,6 +614,89 @@ void verify_has(row_cache& cache, const mutation& m) {
assert_that(reader().get0()).has_mutation().is_equal_to(m);
}

+void test_sliced_read_row_presence(mutation_reader reader, schema_ptr s, std::deque<int> expected)
+{
+ clustering_key::equality ck_eq(*s);
+
+ auto smopt = reader().get0();
+ BOOST_REQUIRE(smopt);
+ auto mfopt = (*smopt)().get0();
+ while (mfopt) {
+ if (mfopt->is_clustering_row()) {
+ BOOST_REQUIRE(!expected.empty());
+ auto expected_ck = expected.front();
+ auto ck = clustering_key_prefix::from_single_value(*s, int32_type->decompose(expected_ck));
+ expected.pop_front();
+ auto& cr = mfopt->as_clustering_row();
+ if (!ck_eq(cr.key(), ck)) {
+ BOOST_FAIL(sprint("Expected %s, but got %s", ck, cr.key()));
+ }
+ }
+ mfopt = (*smopt)().get0();
+ }
+ BOOST_REQUIRE(expected.empty());
+ BOOST_REQUIRE(!reader().get0());
+}
+
+SEASTAR_TEST_CASE(test_single_partition_update) {
+ return seastar::async([] {
+ auto s = schema_builder("ks", "cf")
+ .with_column("pk", int32_type, column_kind::partition_key)
+ .with_column("ck", int32_type, column_kind::clustering_key)
+ .with_column("v", int32_type)
+ .build();
+ auto pk = partition_key::from_exploded(*s, { int32_type->decompose(100) });
+ auto dk = dht::global_partitioner().decorate_key(*s, pk);
+ auto range = dht::partition_range::make_singular(dk);
+ auto make_ck = [&s] (int v) {
+ return clustering_key_prefix::from_single_value(*s, int32_type->decompose(v));
+ };
+ auto ck1 = make_ck(1);
+ auto ck2 = make_ck(2);
+ auto ck3 = make_ck(3);
+ auto ck4 = make_ck(4);
+ auto ck7 = make_ck(7);
+ memtable_snapshot_source cache_mt(s);
+ {
+ mutation m(pk, s);
+ m.set_clustered_cell(ck1, "v", data_value(101), 1);
+ m.set_clustered_cell(ck2, "v", data_value(101), 1);
+ m.set_clustered_cell(ck4, "v", data_value(101), 1);
+ m.set_clustered_cell(ck7, "v", data_value(101), 1);
+ cache_mt.apply(m);
+ }
+
+ cache_tracker tracker;
+ row_cache cache(s, snapshot_source([&] { return cache_mt(); }), tracker);
+
+ {
+ auto slice = partition_slice_builder(*s)
+ .with_range(query::clustering_range::make_ending_with(ck1))
+ .with_range(query::clustering_range::make_starting_with(ck4))
+ .build();
+ auto reader = cache.make_reader(s, range, slice);
+ test_sliced_read_row_presence(std::move(reader), s, {1, 4, 7});
+ }
+
+ auto mt = make_lw_shared<memtable>(s);
+ {
+ mutation m(pk, s);
+ m.set_clustered_cell(ck3, "v", data_value(101), 1);
+ mt->apply(m);
+ cache_mt.apply(m);
+ }
+ cache.update(*mt, [] (auto&& key) {
+ return partition_presence_checker_result::maybe_exists;
+ }).get();
+
+ {
+ auto reader = cache.make_reader(s, range);
+ test_sliced_read_row_presence(std::move(reader), s, {1, 2, 3, 4, 7});
+ }
+
+ });
+}
+
SEASTAR_TEST_CASE(test_update) {
return seastar::async([] {
auto s = make_schema();
@@ -1231,26 +1314,6 @@ SEASTAR_TEST_CASE(test_mvcc) {
});
}

-void test_sliced_read_row_presence(mutation_reader reader, schema_ptr s, std::deque<int> expected)
-{
- clustering_key::equality ck_eq(*s);
-
- auto smopt = reader().get0();
- BOOST_REQUIRE(smopt);
- auto mfopt = (*smopt)().get0();
- while (mfopt) {
- if (mfopt->is_clustering_row()) {
- BOOST_REQUIRE(!expected.empty());
- auto& cr = mfopt->as_clustering_row();
- BOOST_REQUIRE(ck_eq(cr.key(), clustering_key_prefix::from_single_value(*s, int32_type->decompose(expected.front()))));
- expected.pop_front();
- }
- mfopt = (*smopt)().get0();
- }
- BOOST_REQUIRE(expected.empty());
- BOOST_REQUIRE(!reader().get0());
-}
-
SEASTAR_TEST_CASE(test_slicing_mutation_reader) {
return seastar::async([] {
auto s = schema_builder("ks", "cf")
--
2.7.4

Tomasz Grabiec

<tgrabiec@scylladb.com>
unread,
Jun 12, 2017, 3:03:24 PM6/12/17
to scylladb-dev@googlegroups.com
---
streamed_mutation.hh | 26 ++++++++++++++++++++++++++
1 file changed, 26 insertions(+)

diff --git a/streamed_mutation.hh b/streamed_mutation.hh
index 348c3fb..79411d3 100644
--- a/streamed_mutation.hh
+++ b/streamed_mutation.hh
@@ -110,6 +110,13 @@ class clustering_row {
return sizeof(clustering_row) + external_memory_usage();
}

+ bool equal(const schema& s, const clustering_row& other) const {
+ return _ck.equal(s, other._ck)
+ && _t == other._t
+ && _marker == other._marker
+ && _cells.equal(column_kind::static_column, s, other._cells, s);
+ }
+
friend std::ostream& operator<<(std::ostream& os, const clustering_row& row);
};

@@ -147,6 +154,10 @@ class static_row {
return sizeof(static_row) + external_memory_usage();
}

+ bool equal(const schema& s, const static_row& other) const {
+ return _cells.equal(column_kind::static_column, s, other._cells, s);
+ }
+
friend std::ostream& operator<<(std::ostream& is, const static_row& row);
};

@@ -316,6 +327,21 @@ class mutation_fragment {
return *_data->_size_in_bytes;
}

+ bool equal(const schema& s, const mutation_fragment& other) const {
+ if (other._kind != _kind) {
+ return false;
+ }
+ switch(_kind) {
+ case kind::static_row:
+ return as_static_row().equal(s, other.as_static_row());
+ case kind::clustering_row:
+ return as_clustering_row().equal(s, other.as_clustering_row());
+ case kind::range_tombstone:
+ return as_range_tombstone().equal(s, other.as_range_tombstone());
+ }
+ abort();
+ }
+
friend std::ostream& operator<<(std::ostream&, const mutation_fragment& mf);
};

--
2.7.4

Tomasz Grabiec

<tgrabiec@scylladb.com>
unread,
Jun 12, 2017, 3:03:25 PM6/12/17
to scylladb-dev@googlegroups.com
---
tests/mutation_assertions.hh | 19 +++++++++++++++++++
1 file changed, 19 insertions(+)

diff --git a/tests/mutation_assertions.hh b/tests/mutation_assertions.hh
index af30dbe..bcb0d8c 100644
--- a/tests/mutation_assertions.hh
+++ b/tests/mutation_assertions.hh
@@ -160,6 +160,25 @@ class streamed_mutation_assertions {
return *this;
}

+ streamed_mutation_assertions& produces(mutation_fragment mf) {
+ auto mfopt = _sm().get0();
+ if (!mfopt) {
+ BOOST_FAIL(sprint("Expected mutation fragment %s, got end of stream", mf));
+ }
+ if (!mfopt->equal(*_sm.schema(), mf)) {
+ BOOST_FAIL(sprint("Expected %s, but got %s", mf, *mfopt));
+ }
+ return *this;
+ }
+
+ streamed_mutation_assertions& produces_only(const std::deque<mutation_fragment>& fragments) {
+ for (auto&& f : fragments) {
+ produces(f);
+ }
+ produces_end_of_stream();
+ return *this;
+ }
+
streamed_mutation_assertions& produces_row_with_key(const clustering_key& ck) {
BOOST_TEST_MESSAGE(sprint("Expect %s", ck));
auto mfo = _sm().get0();
--
2.7.4

Tomasz Grabiec

<tgrabiec@scylladb.com>
unread,
Jun 12, 2017, 3:03:27 PM6/12/17
to scylladb-dev@googlegroups.com, Piotr Jastrzebski
From: Piotr Jastrzebski <pi...@scylladb.com>

[tgrabiec:
- extracted from a larger commit
- removed coupling with how cache_streamed_mutation is created (the
code went out of sync), used more stable make_reader(). it's simpler too.
- replaced false/true literals with is_continuous/is_dummy where appropraite
- dropped tests for cache::underlying (class is gone)
- reused streamed_mutation_assertions, it has better error messages
- fixed the tests to not create tombstones with missing timestamps
- relaxed range tombstone assertions to only check information relevant for the query range
- print cache on failure for improved debuggability
]
---
configure.py | 1 +
test.py | 1 +
row_cache.hh | 1 +
tests/cache_streamed_mutation_test.cc | 1272 +++++++++++++++++++++++++++++++++
4 files changed, 1275 insertions(+)
create mode 100644 tests/cache_streamed_mutation_test.cc

diff --git a/configure.py b/configure.py
index 77c3c4c..a90c43d 100755
--- a/configure.py
+++ b/configure.py
@@ -184,6 +184,7 @@ scylla_tests = [
'tests/perf/perf_cql_parser',
'tests/perf/perf_simple_query',
'tests/perf/perf_fast_forward',
+ 'tests/cache_streamed_mutation_test',
'tests/memory_footprint',
'tests/perf/perf_sstable',
'tests/cql_query_test',
diff --git a/test.py b/test.py
index e783ed0..5c70753 100755
--- a/test.py
+++ b/test.py
@@ -53,6 +53,7 @@ boost_tests = [
'canonical_mutation_test',
'gossiping_property_file_snitch_test',
'row_cache_test',
+ 'cache_streamed_mutation_test',
'network_topology_strategy_test',
'query_processor_test',
'batchlog_manager_test',
diff --git a/row_cache.hh b/row_cache.hh
index b6e8c5b..c6e822e 100644
--- a/row_cache.hh
+++ b/row_cache.hh
@@ -258,6 +258,7 @@ class row_cache final {
friend class cache::lsa_manager;
friend class cache::read_context;
friend class partition_range_iterator;
+ friend class cache_tester;
public:
struct stats {
utils::timed_rate_moving_average hits;
diff --git a/tests/cache_streamed_mutation_test.cc b/tests/cache_streamed_mutation_test.cc
new file mode 100644
index 0000000..c0c4492
--- /dev/null
+++ b/tests/cache_streamed_mutation_test.cc
@@ -0,0 +1,1272 @@
+
+/*
+ * Copyright (C) 2017 ScyllaDB
+ */
+
+/*
+ * This file is part of Scylla.
+ *
+ * Scylla is free software: you can redistribute it and/or modify
+ * it under the terms of the GNU Affero General Public License as published by
+ * the Free Software Foundation, either version 3 of the License, or
+ * (at your option) any later version.
+ *
+ * Scylla is distributed in the hope that it will be useful,
+ * but WITHOUT ANY WARRANTY; without even the implied warranty of
+ * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
+ * GNU General Public License for more details.
+ *
+ * You should have received a copy of the GNU General Public License
+ * along with Scylla. If not, see <http://www.gnu.org/licenses/>.
+ */
+
+
+#include <boost/test/unit_test.hpp>
+
+#include "tests/test-utils.hh"
+#include "core/thread.hh"
+#include "schema_builder.hh"
+#include "keys.hh"
+#include "mutation_partition.hh"
+#include "partition_version.hh"
+#include "mutation.hh"
+#include "memtable.hh"
+#include "cache_streamed_mutation.hh"
+#include "row_cache.hh"
+
+#include "disk-error-handler.hh"
+#include "memtable_snapshot_source.hh"
+#include "mutation_assertions.hh"
+
+thread_local disk_error_signal_type commit_error;
+thread_local disk_error_signal_type general_disk_error;
+
+/*
+ * ===================
+ * ====== Utils ======
+ * ===================
+ */
+
+static schema_ptr make_schema() {
+ return schema_builder("ks", "cf")
+ .with_column("pk", int32_type, column_kind::partition_key)
+ .with_column("ck", int32_type, column_kind::clustering_key)
+ .with_column("v", int32_type)
+ .build();
+}
+
+static const thread_local schema_ptr SCHEMA = make_schema();
+
+static partition_key make_pk(int value) {
+ return partition_key::from_exploded(*SCHEMA, { int32_type->decompose(value) });
+}
+
+static const thread_local partition_key PK = make_pk(0);
+static const thread_local dht::decorated_key DK =
+ dht::global_partitioner().decorate_key(*SCHEMA, PK);
+
+static clustering_key make_ck(int value) {
+ return clustering_key_prefix::from_single_value(*SCHEMA, int32_type->decompose(value));
+}
+
+static void add_row(mutation& m, int ck, int value) {
+ m.set_clustered_cell(make_ck(ck), "v", data_value(value), 1);
+}
+
+static void add_tombstone(mutation& m, range_tombstone rt) {
+ m.partition().apply_row_tombstone(*SCHEMA, rt);
+}
+
+static void set_row_continuous(mutation_partition& mp, int ck, is_continuous value) {
+ auto it = mp.clustered_rows().find(make_ck(ck), rows_entry::compare(*SCHEMA));
+ assert(it != mp.clustered_rows().end());
+ it->set_continuous(value);
+}
+
+static query::partition_slice make_slice(std::vector<query::clustering_range> ranges) {
+ return query::partition_slice(std::move(ranges), { }, { }, { });
+}
+
+struct expected_fragment {
+ boost::variant<int, range_tombstone> f;
+
+ expected_fragment(int row_key) : f(row_key) { }
+ expected_fragment(range_tombstone rt) : f(rt) { }
+
+ void check(streamed_mutation_assertions& sm, const query::clustering_row_ranges& ranges) {
+ if (f.which() == 0) {
+ sm.produces_row_with_key(make_ck(boost::get<int>(f)));
+ } else {
+ sm.produces_range_tombstone(boost::get<range_tombstone>(f), ranges);
+ }
+ }
+};
+
+static
+mutation make_incomplete_mutation() {
+ return mutation(SCHEMA, DK, mutation_partition::make_incomplete(*SCHEMA));
+}
+
+static void assert_single_version(lw_shared_ptr<partition_snapshot> snp) {
+ BOOST_REQUIRE(snp->at_latest_version());
+ BOOST_REQUIRE_EQUAL(snp->version_count(), 1);
+}
+
+struct expected_row {
+ int ck;
+ is_continuous continuous;
+ is_dummy dummy;
+
+ struct dummy_tag_t { };
+
+ expected_row(int k, is_continuous cont)
+ : ck(k), continuous(cont), dummy(false) { }
+ expected_row(dummy_tag_t, is_continuous cont)
+ : continuous(cont), dummy(true) { }
+
+ void check(const rows_entry& r) const {
+ clustering_key::equality ck_eq(*SCHEMA);
+ BOOST_REQUIRE_EQUAL(r.continuous(), continuous);
+ BOOST_REQUIRE_EQUAL(r.dummy(), dummy);
+ if (!r.dummy()) {
+ BOOST_REQUIRE(ck_eq(r.key(), make_ck(ck)));
+ }
+ }
+
+ friend std::ostream& operator<<(std::ostream& out, const expected_row& e) {
+ return out << "{ck=" << e.ck << ", cont=" << bool(e.continuous) << ", dummy=" << bool(e.dummy) << "}";
+ }
+};
+
+static void assert_cached_rows(lw_shared_ptr<partition_snapshot> snp, std::deque<expected_row> expected) {
+ auto&& rows = snp->version()->partition().clustered_rows();
+ for (auto&& r : rows) {
+ BOOST_REQUIRE(!expected.empty());
+ expected.front().check(r);
+ expected.pop_front();
+ }
+ if (!expected.empty()) {
+ BOOST_FAIL(sprint("Expected %s next, but no more rows", expected.front()));
+ }
+}
+
+struct expected_tombstone {
+ int start;
+ bool start_inclusive;
+ int end;
+ bool end_inclusive;
+
+ expected_tombstone(int s, bool s_i, int e, bool e_i)
+ : start(s)
+ , start_inclusive(s_i)
+ , end(e)
+ , end_inclusive(e_i)
+ { }
+ void check(const range_tombstone& rt) const {
+ clustering_key::equality ck_eq(*SCHEMA);
+ BOOST_REQUIRE(ck_eq(rt.start, make_ck(start)));
+ BOOST_REQUIRE_EQUAL(rt.start_kind, start_inclusive ? bound_kind::incl_start : bound_kind::excl_start);
+ BOOST_REQUIRE(ck_eq(rt.end, make_ck(end)));
+ BOOST_REQUIRE_EQUAL(rt.end_kind, end_inclusive ? bound_kind::incl_end : bound_kind::excl_end);
+ }
+};
+
+static void assert_cached_tombstones(lw_shared_ptr<partition_snapshot> snp, std::deque<range_tombstone> expected) {
+ const range_tombstone_list& rts = snp->version()->partition().row_tombstones();
+ for (auto&& rt : rts) {
+ BOOST_REQUIRE(!expected.empty());
+ if (!expected.front().equal(*SCHEMA, rt)) {
+ BOOST_FAIL(sprint("Expected %s, but found %s", expected.front(), rt));
+ }
+ expected.pop_front();
+ }
+ BOOST_REQUIRE(expected.empty());
+}
+
+class cache_tester {
+public:
+ static lw_shared_ptr<partition_snapshot> snapshot_for_key(row_cache& rc, const dht::decorated_key& dk) {
+ return rc._read_section(rc._tracker.region(), [&] {
+ return with_linearized_managed_bytes([&] {
+ cache_entry& e = rc.find_or_create(dk);
+ return e.partition().read(e.schema());
+ });
+ });
+ }
+};
+
+static void check_produces_only(streamed_mutation sm, std::deque<expected_fragment> expected, const query::clustering_row_ranges& ranges) {
+ auto sa = assert_that_stream(std::move(sm));
+ for (auto&& e : expected) {
+ e.check(sa, ranges);
+ }
+ sa.produces_end_of_stream();
+}
+
+void test_slice_single_version(mutation& underlying,
+ mutation& cache_mutation,
+ const query::partition_slice& slice,
+ std::deque<expected_fragment> expected_sm_fragments,
+ std::deque<expected_row> expected_cache_rows,
+ std::deque<range_tombstone> expected_cache_tombstones) {
+ // Set up underlying
+ memtable_snapshot_source source_mt(SCHEMA);
+ source_mt.apply(underlying);
+ cache_tracker tracker;
+ row_cache cache(SCHEMA, snapshot_source([&] { return source_mt(); }), tracker);
+
+ cache.populate(cache_mutation);
+
+ try {
+ auto range = dht::partition_range::make_singular(DK);
+ auto reader = cache.make_reader(SCHEMA, range, slice);
+ auto smo = reader().get0();
+ BOOST_REQUIRE(bool(smo));
+
+ check_produces_only(std::move(*smo), expected_sm_fragments, slice.row_ranges(*SCHEMA, DK.key()));
+
+ auto snp = cache_tester::snapshot_for_key(cache, DK);
+ assert_single_version(snp);
+ assert_cached_rows(snp, expected_cache_rows);
+ assert_cached_tombstones(snp, expected_cache_tombstones);
+ } catch (...) {
+ std::cerr << "cache: " << cache << "\n";
+ throw;
+ }
+}
+
+/*
+ * ========================================================
+ * ====== Tests for single row with a single version ======
+ * ========================================================
+ */
+void test_single_row(int ck,
+ bool cached,
+ is_continuous continuous,
+ const query::partition_slice& slice,
+ std::deque<int> expected_sm_rows,
+ std::deque<expected_row> expected_cache_rows) {
+ const int value = 12;
+
+ mutation underlying(PK, SCHEMA);
+ add_row(underlying, ck, value);
+
+ auto m = make_incomplete_mutation();
+ if (cached) {
+ add_row(m, ck, value);
+ set_row_continuous(m.partition(), ck, continuous);
+ }
+
+ std::deque<expected_fragment> expected_sm_fragments;
+ for (int r : expected_sm_rows) {
+ expected_sm_fragments.push_back(expected_fragment(r));
+ }
+ test_slice_single_version(underlying, m, slice, expected_sm_fragments, expected_cache_rows, { });
+}
+
+SEASTAR_TEST_CASE(test_single_row_not_cached_full_range) {
+ return seastar::async([] {
+ test_single_row(1, false, is_continuous::yes, query::full_slice, { 1 }, {
+ expected_row(1, is_continuous::yes),
+ expected_row(expected_row::dummy_tag_t{}, is_continuous::yes)
+ });
+ });
+}
+
+SEASTAR_TEST_CASE(test_single_row_not_cached_single_row_range) {
+ return seastar::async([] {
+ test_single_row(1, false, is_continuous::yes, make_slice({ query::clustering_range::make_singular(make_ck(1)) }), { 1 }, {
+ expected_row(1, is_continuous::no),
+ expected_row(expected_row::dummy_tag_t{}, is_continuous::no)
+ });
+ });
+}
+
+SEASTAR_TEST_CASE(test_single_row_not_cached_range_from_start_to_row) {
+ return seastar::async([] {
+ test_single_row(1, false, is_continuous::yes, make_slice({ query::clustering_range::make_ending_with(make_ck(1)) }), { 1 }, {
+ expected_row(1, is_continuous::yes),
+ expected_row(expected_row::dummy_tag_t{}, is_continuous::no)
+ });
+ });
+}
+
+SEASTAR_TEST_CASE(test_single_row_not_cached_range_from_row_to_end) {
+ return seastar::async([] {
+ test_single_row(1, false, is_continuous::yes, make_slice({ query::clustering_range::make_starting_with(make_ck(1)) }), { 1 }, {
+ expected_row(1, is_continuous::no),
+ expected_row(expected_row::dummy_tag_t{}, is_continuous::yes)
+ });
+ });
+}
+
+SEASTAR_TEST_CASE(test_single_row_not_cached_exclusive_range_on_the_left) {
+ return seastar::async([] {
+ test_single_row(1, false, is_continuous::yes, make_slice({ query::clustering_range::make_ending_with({make_ck(1), false}) }),
+ { }, { expected_row(expected_row::dummy_tag_t{}, is_continuous::no) });
+ });
+}
+
+SEASTAR_TEST_CASE(test_single_row_not_cached_exclusive_range_on_the_right) {
+ return seastar::async([] {
+ test_single_row(1, false, is_continuous::yes, make_slice({ query::clustering_range::make_starting_with({make_ck(1), false}) }),
+ { }, { expected_row(expected_row::dummy_tag_t{}, is_continuous::no) });
+ });
+}
+
+SEASTAR_TEST_CASE(test_single_row_not_cached_small_range) {
+ return seastar::async([] {
+ test_single_row(1, false, is_continuous::yes, make_slice({ query::clustering_range::make(make_ck(0), make_ck(2)) }), { 1 }, {
+ expected_row(1, is_continuous::no),
+ expected_row(expected_row::dummy_tag_t{}, is_continuous::no)
+ });
+ });
+}
+
+SEASTAR_TEST_CASE(test_single_row_not_cached_small_range_on_left) {
+ return seastar::async([] {
+ test_single_row(1, false, is_continuous::yes, make_slice({ query::clustering_range::make(make_ck(0), make_ck(1)) }), { 1 }, {
+ expected_row(1, is_continuous::no),
+ expected_row(expected_row::dummy_tag_t{}, is_continuous::no)
+ });
+ });
+}
+
+SEASTAR_TEST_CASE(test_single_row_not_cached_small_range_on_right) {
+ return seastar::async([] {
+ test_single_row(1, false, is_continuous::yes, make_slice({ query::clustering_range::make(make_ck(1), make_ck(2)) }), { 1 }, {
+ expected_row(1, is_continuous::no),
+ expected_row(expected_row::dummy_tag_t{}, is_continuous::no)
+ });
+ });
+}
+
+SEASTAR_TEST_CASE(test_single_row_cached_as_continuous_full_range) {
+ return seastar::async([] {
+ test_single_row(1, true, is_continuous::yes, query::full_slice, { 1 }, {
+ expected_row(1, is_continuous::yes),
+ expected_row(expected_row::dummy_tag_t{}, is_continuous::yes)
+ });
+ });
+}
+
+SEASTAR_TEST_CASE(test_single_row_cached_as_continuous_single_row_range) {
+ return seastar::async([] {
+ test_single_row(1, true, is_continuous::yes, make_slice({ query::clustering_range::make_singular(make_ck(1)) }), { 1 }, {
+ expected_row(1, is_continuous::yes),
+ expected_row(expected_row::dummy_tag_t{}, is_continuous::no)
+ });
+ });
+}
+
+SEASTAR_TEST_CASE(test_single_row_cached_as_continuous_range_from_start_to_row) {
+ return seastar::async([] {
+ test_single_row(1, true, is_continuous::yes, make_slice({ query::clustering_range::make_ending_with(make_ck(1)) }), { 1 }, {
+ expected_row(1, is_continuous::yes),
+ expected_row(expected_row::dummy_tag_t{}, is_continuous::no)
+ });
+ });
+}
+
+SEASTAR_TEST_CASE(test_single_row_cached_as_continuous_range_from_row_to_end) {
+ return seastar::async([] {
+ test_single_row(1, true, is_continuous::yes, make_slice({ query::clustering_range::make_starting_with(make_ck(1)) }), { 1 }, {
+ expected_row(1, is_continuous::yes),
+ expected_row(expected_row::dummy_tag_t{}, is_continuous::yes)
+ });
+ });
+}
+
+SEASTAR_TEST_CASE(test_single_row_cached_as_continuous_exclusive_range_on_the_left) {
+ return seastar::async([] {
+ test_single_row(1, true, is_continuous::yes, make_slice({ query::clustering_range::make_ending_with({make_ck(1), false}) }), { }, {
+ expected_row(1, is_continuous::yes),
+ expected_row(expected_row::dummy_tag_t{}, is_continuous::no)
+ });
+ });
+}
+
+SEASTAR_TEST_CASE(test_single_row_cached_as_continuous_exclusive_range_on_the_right) {
+ return seastar::async([] {
+ test_single_row(1, true, is_continuous::yes, make_slice({ query::clustering_range::make_starting_with({make_ck(1), false}) }), { }, {
+ expected_row(1, is_continuous::yes),
+ expected_row(expected_row::dummy_tag_t{}, is_continuous::no)
+ });
+ });
+}
+
+SEASTAR_TEST_CASE(test_single_row_cached_as_continuous_small_range) {
+ return seastar::async([] {
+ test_single_row(1, true, is_continuous::yes, make_slice({ query::clustering_range::make(make_ck(0), make_ck(2)) }), { 1 }, {
+ expected_row(1, is_continuous::yes),
+ expected_row(expected_row::dummy_tag_t{}, is_continuous::no)
+ });
+ });
+}
+
+SEASTAR_TEST_CASE(test_single_row_cached_as_continuous_small_range_on_left) {
+ return seastar::async([] {
+ test_single_row(1, true, is_continuous::yes, make_slice({ query::clustering_range::make(make_ck(0), make_ck(1)) }), { 1 }, {
+ expected_row(1, is_continuous::yes),
+ expected_row(expected_row::dummy_tag_t{}, is_continuous::no)
+ });
+ });
+}
+
+SEASTAR_TEST_CASE(test_single_row_cached_as_continuous_small_range_on_right) {
+ return seastar::async([] {
+ test_single_row(1, true, is_continuous::yes, make_slice({ query::clustering_range::make(make_ck(1), make_ck(2)) }), { 1 }, {
+ expected_row(1, is_continuous::yes),
+ expected_row(expected_row::dummy_tag_t{}, is_continuous::no)
+ });
+ });
+}
+
+SEASTAR_TEST_CASE(test_single_row_cached_as_noncontinuous_full_range) {
+ return seastar::async([] {
+ test_single_row(1, true, is_continuous::no, query::full_slice, { 1 }, {
+ expected_row(1, is_continuous::yes),
+ expected_row(expected_row::dummy_tag_t{}, is_continuous::yes)
+ });
+ });
+}
+
+SEASTAR_TEST_CASE(test_single_row_cached_as_noncontinuous_single_row_range) {
+ return seastar::async([] {
+ test_single_row(1, true, is_continuous::no, make_slice({ query::clustering_range::make_singular(make_ck(1)) }), { 1 }, {
+ expected_row(1, is_continuous::no),
+ expected_row(expected_row::dummy_tag_t{}, is_continuous::no)
+ });
+ });
+}
+
+SEASTAR_TEST_CASE(test_single_row_cached_as_noncontinuous_range_from_start_to_row) {
+ return seastar::async([] {
+ test_single_row(1, true, is_continuous::no, make_slice({ query::clustering_range::make_ending_with(make_ck(1)) }), { 1 }, {
+ expected_row(1, is_continuous::yes),
+ expected_row(expected_row::dummy_tag_t{}, is_continuous::no)
+ });
+ });
+}
+
+SEASTAR_TEST_CASE(test_single_row_cached_as_noncontinuous_range_from_row_to_end) {
+ return seastar::async([] {
+ test_single_row(1, true, is_continuous::no, make_slice({ query::clustering_range::make_starting_with(make_ck(1)) }), { 1 }, {
+ expected_row(1, is_continuous::no),
+ expected_row(expected_row::dummy_tag_t{}, is_continuous::yes)
+ });
+ });
+}
+
+SEASTAR_TEST_CASE(test_single_row_cached_as_noncontinuous_exclusive_range_on_the_left) {
+ return seastar::async([] {
+ test_single_row(1, true, is_continuous::no, make_slice({ query::clustering_range::make_ending_with({make_ck(1), false}) }), { }, {
+ expected_row(1, is_continuous::yes),
+ expected_row(expected_row::dummy_tag_t{}, is_continuous::no)
+ });
+ });
+}
+
+SEASTAR_TEST_CASE(test_single_row_cached_as_noncontinuous_exclusive_range_on_the_right) {
+ return seastar::async([] {
+ test_single_row(1, true, is_continuous::no, make_slice({ query::clustering_range::make_starting_with({make_ck(1), false}) }), { }, {
+ expected_row(1, is_continuous::no),
+ expected_row(expected_row::dummy_tag_t{}, is_continuous::no)
+ });
+ });
+}
+
+SEASTAR_TEST_CASE(test_single_row_cached_as_noncontinuous_small_range) {
+ return seastar::async([] {
+ test_single_row(1, true, is_continuous::no, make_slice({ query::clustering_range::make(make_ck(0), make_ck(2)) }), { 1 }, {
+ expected_row(1, is_continuous::no),
+ expected_row(expected_row::dummy_tag_t{}, is_continuous::no)
+ });
+ });
+}
+
+SEASTAR_TEST_CASE(test_single_row_cached_as_noncontinuous_small_range_on_left) {
+ return seastar::async([] {
+ test_single_row(1, true, is_continuous::no, make_slice({ query::clustering_range::make(make_ck(0), make_ck(1)) }), { 1 }, {
+ expected_row(1, is_continuous::no),
+ expected_row(expected_row::dummy_tag_t{}, is_continuous::no)
+ });
+ });
+}
+
+SEASTAR_TEST_CASE(test_single_row_cached_as_noncontinuous_small_range_on_right) {
+ return seastar::async([] {
+ test_single_row(1, true, is_continuous::no, make_slice({ query::clustering_range::make(make_ck(1), make_ck(2)) }), { 1 }, {
+ expected_row(1, is_continuous::no),
+ expected_row(expected_row::dummy_tag_t{}, is_continuous::no)
+ });
+ });
+}
+
+/*
+ * ======================================================
+ * ====== Tests for two rows with a single version ======
+ * ======================================================
+ */
+
+void test_two_rows(int ck1,
+ bool cached1,
+ is_continuous continuous1,
+ int ck2,
+ bool cached2,
+ is_continuous continuous2,
+ const query::partition_slice& slice,
+ std::deque<int> expected_sm_rows,
+ std::deque<expected_row> expected_cache_rows) {
+ const int value1 = 12;
+ const int value2 = 34;
+
+ mutation underlying(PK, SCHEMA);
+ add_row(underlying, ck1, value1);
+ add_row(underlying, ck2, value2);
+
+ auto cache = make_incomplete_mutation();
+ if (cached1) {
+ add_row(cache, ck1, value1);
+ set_row_continuous(cache.partition(), ck1, continuous1);
+ }
+ if (cached2) {
+ add_row(cache, ck2, value2);
+ set_row_continuous(cache.partition(), ck2, continuous2);
+ }
+ std::deque<expected_fragment> expected_sm_fragments;
+ for (int r : expected_sm_rows) {
+ expected_sm_fragments.push_back(expected_fragment(r));
+ }
+ test_slice_single_version(underlying, cache, slice, expected_sm_fragments, expected_cache_rows, { });
+}
+
+SEASTAR_TEST_CASE(test_two_rows_not_cached_full_range) {
+ return seastar::async([] {
+ test_two_rows(1, false, is_continuous::no, 3, false, is_continuous::no, query::full_slice, { 1, 3 }, {
+ expected_row(1, is_continuous::yes),
+ expected_row(3, is_continuous::yes),
+ expected_row(expected_row::dummy_tag_t{}, is_continuous::yes)
+ });
+ });
+}
+
+SEASTAR_TEST_CASE(test_two_rows_not_cached_single_row_range1) {
+ return seastar::async([] {
+ test_two_rows(1, false, is_continuous::no, 3, false, is_continuous::no, make_slice({ query::clustering_range::make_singular(make_ck(1)) }), { 1 }, {
+ expected_row(1, is_continuous::no),
+ expected_row(expected_row::dummy_tag_t{}, is_continuous::no)
+ });
+ });
+}
+
+SEASTAR_TEST_CASE(test_two_rows_not_cached_single_row_range2) {
+ return seastar::async([] {
+ test_two_rows(1, false, is_continuous::no, 3, false, is_continuous::no, make_slice({ query::clustering_range::make_singular(make_ck(3)) }), { 3 }, {
+ expected_row(3, is_continuous::no),
+ expected_row(expected_row::dummy_tag_t{}, is_continuous::no)
+ });
+ });
+}
+
+SEASTAR_TEST_CASE(test_two_rows_not_cached_range_from_start_to_row1) {
+ return seastar::async([] {
+ test_two_rows(1, false, is_continuous::no, 3, false, is_continuous::no, make_slice({ query::clustering_range::make_ending_with(make_ck(1)) }), { 1 }, {
+ expected_row(1, is_continuous::yes),
+ expected_row(expected_row::dummy_tag_t{}, is_continuous::no)
+ });
+ });
+}
+
+SEASTAR_TEST_CASE(test_two_rows_not_cached_range_from_start_to_row2) {
+ return seastar::async([] {
+ test_two_rows(1, false, is_continuous::no, 3, false, is_continuous::no, make_slice({ query::clustering_range::make_ending_with(make_ck(3)) }), { 1, 3 }, {
+ expected_row(1, is_continuous::yes),
+ expected_row(3, is_continuous::yes),
+ expected_row(expected_row::dummy_tag_t{}, is_continuous::no)
+ });
+ });
+}
+
+SEASTAR_TEST_CASE(test_two_rows_not_cached_range_from_row1_to_end) {
+ return seastar::async([] {
+ test_two_rows(1, false, is_continuous::no, 3, false, is_continuous::no, make_slice({ query::clustering_range::make_starting_with(make_ck(1)) }), { 1, 3 }, {
+ expected_row(1, is_continuous::no),
+ expected_row(3, is_continuous::yes),
+ expected_row(expected_row::dummy_tag_t{}, is_continuous::yes)
+ });
+ });
+}
+
+SEASTAR_TEST_CASE(test_two_rows_not_cached_range_from_row2_to_end) {
+ return seastar::async([] {
+ test_two_rows(1, false, is_continuous::no, 3, false, is_continuous::no, make_slice({ query::clustering_range::make_starting_with(make_ck(3)) }), { 3 }, {
+ expected_row(3, is_continuous::no),
+ expected_row(expected_row::dummy_tag_t{}, is_continuous::yes)
+ });
+ });
+}
+
+SEASTAR_TEST_CASE(test_two_rows_not_cached_exclusive_range_on_the_left) {
+ return seastar::async([] {
+ test_two_rows(1, false, is_continuous::no, 3, false, is_continuous::no, make_slice({ query::clustering_range::make_ending_with({make_ck(3), false}) }), { 1 }, {
+ expected_row(1, is_continuous::yes),
+ expected_row(expected_row::dummy_tag_t{}, is_continuous::no)
+ });
+ });
+}
+
+SEASTAR_TEST_CASE(test_two_rows_not_cached_exclusive_range_on_the_right) {
+ return seastar::async([] {
+ test_two_rows(1, false, is_continuous::no, 3, false, is_continuous::no, make_slice({ query::clustering_range::make_starting_with({make_ck(1), false}) }), { 3 }, {
+ expected_row(3, is_continuous::no), // TODO: this should be possible to mark continuous here
+ expected_row(expected_row::dummy_tag_t{}, is_continuous::yes)
+ });
+ });
+}
+
+SEASTAR_TEST_CASE(test_two_rows_not_cached_exclusive_range_between_rows1) {
+ return seastar::async([] {
+ test_two_rows(1, false, is_continuous::no, 3, false, is_continuous::no, make_slice({ query::clustering_range::make({make_ck(1), false}, {make_ck(3), false}) }),
+ { }, { expected_row(expected_row::dummy_tag_t{}, is_continuous::no) });
+ });
+}
+
+SEASTAR_TEST_CASE(test_two_rows_not_cached_exclusive_range_between_rows2) {
+ return seastar::async([] {
+ test_two_rows(1, false, is_continuous::no, 3, false, is_continuous::no, make_slice({ query::clustering_range::make({make_ck(1), false}, make_ck(3)) }), { 3 }, {
+ expected_row(3, is_continuous::no), // TODO: this should be possible to mark continuous here
+ expected_row(expected_row::dummy_tag_t{}, is_continuous::no)
+ });
+ });
+}
+
+SEASTAR_TEST_CASE(test_two_rows_not_cached_exclusive_range_between_rows3) {
+ return seastar::async([] {
+ test_two_rows(1, false, is_continuous::no, 3, false, is_continuous::no, make_slice({ query::clustering_range::make(make_ck(1), {make_ck(3), false}) }), { 1 }, {
+ expected_row(1, is_continuous::no),
+ expected_row(expected_row::dummy_tag_t{}, is_continuous::no)
+ });
+ });
+}
+
+SEASTAR_TEST_CASE(test_two_rows_not_cached_small_range) {
+ return seastar::async([] {
+ test_two_rows(1, false, is_continuous::no, 3, false, is_continuous::no, make_slice({ query::clustering_range::make(make_ck(0), make_ck(4)) }), { 1, 3 }, {
+ expected_row(1, is_continuous::no),
+ expected_row(3, is_continuous::yes),
+ expected_row(expected_row::dummy_tag_t{}, is_continuous::no)
+ });
+ });
+}
+
+SEASTAR_TEST_CASE(test_two_rows_not_cached_small_range_row1) {
+ return seastar::async([] {
+ test_two_rows(1, false, is_continuous::no, 3, false, is_continuous::no, make_slice({ query::clustering_range::make(make_ck(0), make_ck(2)) }), { 1 }, {
+ expected_row(1, is_continuous::no),
+ expected_row(expected_row::dummy_tag_t{}, is_continuous::no)
+ });
+ });
+}
+
+SEASTAR_TEST_CASE(test_two_rows_not_cached_small_range_row2) {
+ return seastar::async([] {
+ test_two_rows(1, false, is_continuous::no, 3, false, is_continuous::no, make_slice({ query::clustering_range::make(make_ck(2), make_ck(4)) }), { 3 }, {
+ expected_row(3, is_continuous::no),
+ expected_row(expected_row::dummy_tag_t{}, is_continuous::no)
+ });
+ });
+}
+
+SEASTAR_TEST_CASE(test_two_rows_cached_continuous_full_range) {
+ return seastar::async([] {
+ test_two_rows(1, true, is_continuous::yes, 3, true, is_continuous::yes, query::full_slice, { 1, 3 }, {
+ expected_row(1, is_continuous::yes),
+ expected_row(3, is_continuous::yes),
+ expected_row(expected_row::dummy_tag_t{}, is_continuous::yes)
+ });
+ });
+}
+
+SEASTAR_TEST_CASE(test_two_rows_cached_continuous_single_row_range1) {
+ return seastar::async([] {
+ test_two_rows(1, true, is_continuous::yes, 3, true, is_continuous::yes, make_slice({ query::clustering_range::make_singular(make_ck(1)) }), { 1 }, {
+ expected_row(1, is_continuous::yes),
+ expected_row(3, is_continuous::yes),
+ expected_row(expected_row::dummy_tag_t{}, is_continuous::no)
+ });
+ });
+}
+
+SEASTAR_TEST_CASE(test_two_rows_cached_continuous_single_row_range2) {
+ return seastar::async([] {
+ test_two_rows(1, true, is_continuous::yes, 3, true, is_continuous::yes, make_slice({ query::clustering_range::make_singular(make_ck(3)) }), { 3 }, {
+ expected_row(1, is_continuous::yes),
+ expected_row(3, is_continuous::yes),
+ expected_row(expected_row::dummy_tag_t{}, is_continuous::no)
+ });
+ });
+}
+
+SEASTAR_TEST_CASE(test_two_rows_cached_continuous_range_from_start_to_row1) {
+ return seastar::async([] {
+ test_two_rows(1, true, is_continuous::yes, 3, true, is_continuous::yes, make_slice({ query::clustering_range::make_ending_with(make_ck(1)) }), { 1 }, {
+ expected_row(1, is_continuous::yes),
+ expected_row(3, is_continuous::yes),
+ expected_row(expected_row::dummy_tag_t{}, is_continuous::no)
+ });
+ });
+}
+
+SEASTAR_TEST_CASE(test_two_rows_cached_continuous_range_from_start_to_row2) {
+ return seastar::async([] {
+ test_two_rows(1, true, is_continuous::yes, 3, true, is_continuous::yes, make_slice({ query::clustering_range::make_ending_with(make_ck(3)) }), { 1, 3 }, {
+ expected_row(1, is_continuous::yes),
+ expected_row(3, is_continuous::yes),
+ expected_row(expected_row::dummy_tag_t{}, is_continuous::no)
+ });
+ });
+}
+
+SEASTAR_TEST_CASE(test_two_rows_cached_continuous_range_from_row1_to_end) {
+ return seastar::async([] {
+ test_two_rows(1, true, is_continuous::yes, 3, true, is_continuous::yes, make_slice({ query::clustering_range::make_starting_with(make_ck(1)) }), { 1, 3 }, {
+ expected_row(1, is_continuous::yes),
+ expected_row(3, is_continuous::yes),
+ expected_row(expected_row::dummy_tag_t{}, is_continuous::yes)
+ });
+ });
+}
+
+SEASTAR_TEST_CASE(test_two_rows_cached_continuous_range_from_row2_to_end) {
+ return seastar::async([] {
+ test_two_rows(1, true, is_continuous::yes, 3, true, is_continuous::yes, make_slice({ query::clustering_range::make_starting_with(make_ck(3)) }), { 3 }, {
+ expected_row(1, is_continuous::yes),
+ expected_row(3, is_continuous::yes),
+ expected_row(expected_row::dummy_tag_t{}, is_continuous::yes)
+ });
+ });
+}
+
+SEASTAR_TEST_CASE(test_two_rows_cached_continuous_exclusive_range_on_the_left) {
+ return seastar::async([] {
+ test_two_rows(1, true, is_continuous::yes, 3, true, is_continuous::yes, make_slice({ query::clustering_range::make_ending_with({make_ck(3), false}) }), { 1 }, {
+ expected_row(1, is_continuous::yes),
+ expected_row(3, is_continuous::yes),
+ expected_row(expected_row::dummy_tag_t{}, is_continuous::no)
+ });
+ });
+}
+
+SEASTAR_TEST_CASE(test_two_rows_cached_continuous_exclusive_range_on_the_right) {
+ return seastar::async([] {
+ test_two_rows(1, true, is_continuous::yes, 3, true, is_continuous::yes, make_slice({ query::clustering_range::make_starting_with({make_ck(1), false}) }), { 3 }, {
+ expected_row(1, is_continuous::yes),
+ expected_row(3, is_continuous::yes),
+ expected_row(expected_row::dummy_tag_t{}, is_continuous::yes)
+ });
+ });
+}
+
+SEASTAR_TEST_CASE(test_two_rows_cached_continuous_exclusive_range_between_rows1) {
+ return seastar::async([] {
+ test_two_rows(1, true, is_continuous::yes, 3, true, is_continuous::yes, make_slice({ query::clustering_range::make({make_ck(1), false}, {make_ck(3), false}) }), { }, {
+ expected_row(1, is_continuous::yes),
+ expected_row(3, is_continuous::yes),
+ expected_row(expected_row::dummy_tag_t{}, is_continuous::no)
+ });
+ });
+}
+
+SEASTAR_TEST_CASE(test_two_rows_cached_continuous_exclusive_range_between_rows2) {
+ return seastar::async([] {
+ test_two_rows(1, true, is_continuous::yes, 3, true, is_continuous::yes, make_slice({ query::clustering_range::make({make_ck(1), false}, make_ck(3)) }), { 3 }, {
+ expected_row(1, is_continuous::yes),
+ expected_row(3, is_continuous::yes),
+ expected_row(expected_row::dummy_tag_t{}, is_continuous::no)
+ });
+ });
+}
+
+SEASTAR_TEST_CASE(test_two_rows_cached_continuous_exclusive_range_between_rows3) {
+ return seastar::async([] {
+ test_two_rows(1, true, is_continuous::yes, 3, true, is_continuous::yes, make_slice({ query::clustering_range::make(make_ck(1), {make_ck(3), false}) }), { 1 }, {
+ expected_row(1, is_continuous::yes),
+ expected_row(3, is_continuous::yes),
+ expected_row(expected_row::dummy_tag_t{}, is_continuous::no)
+ });
+ });
+}
+
+SEASTAR_TEST_CASE(test_two_rows_cached_continuous_small_range) {
+ return seastar::async([] {
+ test_two_rows(1, true, is_continuous::yes, 3, true, is_continuous::yes, make_slice({ query::clustering_range::make(make_ck(0), make_ck(4)) }), { 1, 3 }, {
+ expected_row(1, is_continuous::yes),
+ expected_row(3, is_continuous::yes),
+ expected_row(expected_row::dummy_tag_t{}, is_continuous::no)
+ });
+ });
+}
+
+SEASTAR_TEST_CASE(test_two_rows_cached_continuous_small_range_row1) {
+ return seastar::async([] {
+ test_two_rows(1, true, is_continuous::yes, 3, true, is_continuous::yes, make_slice({ query::clustering_range::make(make_ck(0), make_ck(2)) }), { 1 }, {
+ expected_row(1, is_continuous::yes),
+ expected_row(3, is_continuous::yes),
+ expected_row(expected_row::dummy_tag_t{}, is_continuous::no)
+ });
+ });
+}
+
+SEASTAR_TEST_CASE(test_two_rows_cached_continuous_small_range_row2) {
+ return seastar::async([] {
+ test_two_rows(1, true, is_continuous::yes, 3, true, is_continuous::yes, make_slice({ query::clustering_range::make(make_ck(2), make_ck(4)) }), { 3 }, {
+ expected_row(1, is_continuous::yes),
+ expected_row(3, is_continuous::yes),
+ expected_row(expected_row::dummy_tag_t{}, is_continuous::no)
+ });
+ });
+}
+
+SEASTAR_TEST_CASE(test_two_rows_cached_non_continuous_full_range) {
+ return seastar::async([] {
+ test_two_rows(1, true, is_continuous::no, 3, true, is_continuous::no, query::full_slice, { 1, 3 }, {
+ expected_row(1, is_continuous::yes),
+ expected_row(3, is_continuous::yes),
+ expected_row(expected_row::dummy_tag_t{}, is_continuous::yes)
+ });
+ });
+}
+
+SEASTAR_TEST_CASE(test_two_rows_cached_non_continuous_single_row_range1) {
+ return seastar::async([] {
+ test_two_rows(1, true, is_continuous::no, 3, true, is_continuous::no, make_slice({ query::clustering_range::make_singular(make_ck(1)) }), { 1 }, {
+ expected_row(1, is_continuous::no),
+ expected_row(3, is_continuous::no),
+ expected_row(expected_row::dummy_tag_t{}, is_continuous::no)
+ });
+ });
+}
+
+SEASTAR_TEST_CASE(test_two_rows_cached_non_continuous_single_row_range2) {
+ return seastar::async([] {
+ test_two_rows(1, true, is_continuous::no, 3, true, is_continuous::no, make_slice({ query::clustering_range::make_singular(make_ck(3)) }), { 3 }, {
+ expected_row(1, is_continuous::no),
+ expected_row(3, is_continuous::no),
+ expected_row(expected_row::dummy_tag_t{}, is_continuous::no)
+ });
+ });
+}
+
+SEASTAR_TEST_CASE(test_two_rows_cached_non_continuous_range_from_start_to_row1) {
+ return seastar::async([] {
+ test_two_rows(1, true, is_continuous::no, 3, true, is_continuous::no, make_slice({ query::clustering_range::make_ending_with(make_ck(1)) }), { 1 }, {
+ expected_row(1, is_continuous::yes),
+ expected_row(3, is_continuous::no),
+ expected_row(expected_row::dummy_tag_t{}, is_continuous::no)
+ });
+ });
+}
+
+SEASTAR_TEST_CASE(test_two_rows_cached_non_continuous_range_from_start_to_row2) {
+ return seastar::async([] {
+ test_two_rows(1, true, is_continuous::no, 3, true, is_continuous::no, make_slice({ query::clustering_range::make_ending_with(make_ck(3)) }), { 1, 3 }, {
+ expected_row(1, is_continuous::yes),
+ expected_row(3, is_continuous::yes),
+ expected_row(expected_row::dummy_tag_t{}, is_continuous::no)
+ });
+ });
+}
+
+SEASTAR_TEST_CASE(test_two_rows_cached_non_continuous_range_from_row1_to_end) {
+ return seastar::async([] {
+ test_two_rows(1, true, is_continuous::no, 3, true, is_continuous::no, make_slice({ query::clustering_range::make_starting_with(make_ck(1)) }), { 1, 3 }, {
+ expected_row(1, is_continuous::no),
+ expected_row(3, is_continuous::yes),
+ expected_row(expected_row::dummy_tag_t{}, is_continuous::yes)
+ });
+ });
+}
+
+SEASTAR_TEST_CASE(test_two_rows_cached_non_continuous_range_from_row2_to_end) {
+ return seastar::async([] {
+ test_two_rows(1, true, is_continuous::no, 3, true, is_continuous::no, make_slice({ query::clustering_range::make_starting_with(make_ck(3)) }), { 3 }, {
+ expected_row(1, is_continuous::no),
+ expected_row(3, is_continuous::no),
+ expected_row(expected_row::dummy_tag_t{}, is_continuous::yes)
+ });
+ });
+}
+
+SEASTAR_TEST_CASE(test_two_rows_cached_non_continuous_exclusive_range_on_the_left) {
+ return seastar::async([] {
+ test_two_rows(1, true, is_continuous::no, 3, true, is_continuous::no, make_slice({ query::clustering_range::make_ending_with({make_ck(3), false}) }), { 1 }, {
+ expected_row(1, is_continuous::yes),
+ expected_row(3, is_continuous::yes),
+ expected_row(expected_row::dummy_tag_t{}, is_continuous::no)
+ });
+ });
+}
+
+SEASTAR_TEST_CASE(test_two_rows_cached_non_continuous_exclusive_range_on_the_right) {
+ return seastar::async([] {
+ test_two_rows(1, true, is_continuous::no, 3, true, is_continuous::no, make_slice({ query::clustering_range::make_starting_with({make_ck(1), false}) }), { 3 }, {
+ expected_row(1, is_continuous::no),
+ expected_row(3, is_continuous::no),
+ expected_row(expected_row::dummy_tag_t{}, is_continuous::yes)
+ });
+ });
+}
+
+SEASTAR_TEST_CASE(test_two_rows_cached_non_continuous_exclusive_range_between_rows1) {
+ return seastar::async([] {
+ test_two_rows(1, true, is_continuous::no, 3, true, is_continuous::no, make_slice({ query::clustering_range::make({make_ck(1), false}, {make_ck(3), false}) }), { }, {
+ expected_row(1, is_continuous::no),
+ expected_row(3, is_continuous::no),
+ expected_row(expected_row::dummy_tag_t{}, is_continuous::no)
+ });
+ });
+}
+
+SEASTAR_TEST_CASE(test_two_rows_cached_non_continuous_exclusive_range_between_rows2) {
+ return seastar::async([] {
+ test_two_rows(1, true, is_continuous::no, 3, true, is_continuous::no, make_slice({ query::clustering_range::make({make_ck(1), false}, make_ck(3)) }), { 3 }, {
+ expected_row(1, is_continuous::no),
+ expected_row(3, is_continuous::no),
+ expected_row(expected_row::dummy_tag_t{}, is_continuous::no)
+ });
+ });
+}
+
+SEASTAR_TEST_CASE(test_two_rows_cached_non_continuous_exclusive_range_between_rows3) {
+ return seastar::async([] {
+ test_two_rows(1, true, is_continuous::no, 3, true, is_continuous::no, make_slice({ query::clustering_range::make(make_ck(1), {make_ck(3), false}) }), { 1 }, {
+ expected_row(1, is_continuous::no),
+ expected_row(3, is_continuous::yes),
+ expected_row(expected_row::dummy_tag_t{}, is_continuous::no)
+ });
+ });
+}
+
+SEASTAR_TEST_CASE(test_two_rows_cached_non_continuous_small_range) {
+ return seastar::async([] {
+ test_two_rows(1, true, is_continuous::no, 3, true, is_continuous::no, make_slice({ query::clustering_range::make(make_ck(0), make_ck(4)) }), { 1, 3 }, {
+ expected_row(1, is_continuous::no),
+ expected_row(3, is_continuous::yes),
+ expected_row(expected_row::dummy_tag_t{}, is_continuous::no)
+ });
+ });
+}
+
+SEASTAR_TEST_CASE(test_two_rows_cached_non_continuous_small_range_row1) {
+ return seastar::async([] {
+ test_two_rows(1, true, is_continuous::no, 3, true, is_continuous::no, make_slice({ query::clustering_range::make(make_ck(0), make_ck(2)) }), { 1 }, {
+ expected_row(1, is_continuous::no),
+ expected_row(3, is_continuous::no),
+ expected_row(expected_row::dummy_tag_t{}, is_continuous::no)
+ });
+ });
+}
+
+SEASTAR_TEST_CASE(test_two_rows_cached_non_continuous_small_range_row2) {
+ return seastar::async([] {
+ test_two_rows(1, true, is_continuous::no, 3, true, is_continuous::no, make_slice({ query::clustering_range::make(make_ck(2), make_ck(4)) }), { 3 }, {
+ expected_row(1, is_continuous::no),
+ expected_row(3, is_continuous::no),
+ expected_row(expected_row::dummy_tag_t{}, is_continuous::no)
+ });
+ });
+}
+
+SEASTAR_TEST_CASE(test_two_rows_first_not_cached_second_cached_non_continuous1) {
+ return seastar::async([] {
+ test_two_rows(1, false, is_continuous::no, 3, true, is_continuous::no, make_slice({ query::clustering_range::make(make_ck(2), make_ck(4)) }), { 3 }, {
+ expected_row(3, is_continuous::no),
+ expected_row(expected_row::dummy_tag_t{}, is_continuous::no)
+ });
+ });
+}
+
+SEASTAR_TEST_CASE(test_two_rows_first_not_cached_second_cached_non_continuous2) {
+ return seastar::async([] {
+ test_two_rows(1, false, is_continuous::no, 3, true, is_continuous::no, make_slice({ query::clustering_range::make(make_ck(0), make_ck(4)) }), { 1, 3 }, {
+ expected_row(1, is_continuous::no),
+ expected_row(3, is_continuous::yes),
+ expected_row(expected_row::dummy_tag_t{}, is_continuous::no)
+ });
+ });
+}
+
+SEASTAR_TEST_CASE(test_two_rows_first_cached_non_continuous_second_not_cached1) {
+ return seastar::async([] {
+ test_two_rows(1, true, is_continuous::no, 3, false, is_continuous::no, make_slice({ query::clustering_range::make(make_ck(2), make_ck(4)) }), { 3 }, {
+ expected_row(1, is_continuous::no),
+ expected_row(3, is_continuous::no),
+ expected_row(expected_row::dummy_tag_t{}, is_continuous::no)
+ });
+ });
+}
+
+SEASTAR_TEST_CASE(test_two_rows_first_cached_non_continuous_second_not_cached2) {
+ return seastar::async([] {
+ test_two_rows(1, true, is_continuous::no, 3, false, is_continuous::no, make_slice({ query::clustering_range::make(make_ck(0), make_ck(4)) }), { 1, 3 }, {
+ expected_row(1, is_continuous::no),
+ expected_row(3, is_continuous::yes),
+ expected_row(expected_row::dummy_tag_t{}, is_continuous::no)
+ });
+ });
+}
+
+SEASTAR_TEST_CASE(test_two_rows_first_cached_continuous_second_not_cached1) {
+ return seastar::async([] {
+ test_two_rows(1, true, is_continuous::yes, 3, false, is_continuous::no, make_slice({ query::clustering_range::make(make_ck(2), make_ck(4)) }), { 3 }, {
+ expected_row(1, is_continuous::yes),
+ expected_row(3, is_continuous::no),
+ expected_row(expected_row::dummy_tag_t{}, is_continuous::no)
+ });
+ });
+}
+
+SEASTAR_TEST_CASE(test_two_rows_first_cached_continuous_second_not_cached2) {
+ return seastar::async([] {
+ test_two_rows(1, true, is_continuous::yes, 3, false, is_continuous::no, make_slice({ query::clustering_range::make(make_ck(0), make_ck(4)) }), { 1, 3 }, {
+ expected_row(1, is_continuous::yes),
+ expected_row(3, is_continuous::yes),
+ expected_row(expected_row::dummy_tag_t{}, is_continuous::no)
+ });
+ });
+}
+
+/*
+ * ========================================================
+ * ====== Tests for three rows with a single version ======
+ * ========================================================
+ */
+
+void test_three_rows(int ck1,
+ bool cached1,
+ is_continuous continuous1,
+ int ck2,
+ bool cached2,
+ is_continuous continuous2,
+ int ck3,
+ bool cached3,
+ is_continuous continuous3,
+ const query::partition_slice& slice,
+ std::deque<int> expected_sm_rows,
+ std::deque<expected_row> expected_cache_rows) {
+ const int value1 = 12;
+ const int value2 = 34;
+ const int value3 = 56;
+
+ mutation underlying(PK, SCHEMA);
+ add_row(underlying, ck1, value1);
+ add_row(underlying, ck2, value2);
+ add_row(underlying, ck3, value3);
+
+ auto cache = make_incomplete_mutation();
+ if (cached1) {
+ add_row(cache, ck1, value1);
+ set_row_continuous(cache.partition(), ck1, continuous1);
+ }
+ if (cached2) {
+ add_row(cache, ck2, value2);
+ set_row_continuous(cache.partition(), ck2, continuous2);
+ }
+ if (cached3) {
+ add_row(cache, ck3, value3);
+ set_row_continuous(cache.partition(), ck3, continuous3);
+ }
+ std::deque<expected_fragment> expected_sm_fragments;
+ for (int r : expected_sm_rows) {
+ expected_sm_fragments.push_back(expected_fragment(r));
+ }
+ test_slice_single_version(underlying, cache, slice, expected_sm_fragments, expected_cache_rows, { });
+}
+
+SEASTAR_TEST_CASE(test_three_rows_first_continuous1) {
+ return seastar::async([] {
+ test_three_rows(1, true, is_continuous::yes, 3, false, is_continuous::no, 5, true, is_continuous::no, make_slice({ query::clustering_range::make(make_ck(0), make_ck(6)) }), { 1, 3, 5 }, {
+ expected_row(1, is_continuous::yes),
+ expected_row(3, is_continuous::yes),
+ expected_row(5, is_continuous::yes),
+ expected_row(expected_row::dummy_tag_t{}, is_continuous::no)
+ });
+ });
+}
+
+SEASTAR_TEST_CASE(test_three_rows_first_continuous2) {
+ return seastar::async([] {
+ test_three_rows(1, true, is_continuous::yes, 3, false, is_continuous::no, 5, true, is_continuous::no, make_slice({ query::clustering_range::make(make_ck(2), make_ck(6)) }), { 3, 5 }, {
+ expected_row(1, is_continuous::yes),
+ expected_row(3, is_continuous::no),
+ expected_row(5, is_continuous::yes),
+ expected_row(expected_row::dummy_tag_t{}, is_continuous::no)
+ });
+ });
+}
+
+SEASTAR_TEST_CASE(test_three_rows_first_continuous3) {
+ return seastar::async([] {
+ test_three_rows(1, true, is_continuous::yes, 3, false, is_continuous::no, 5, true, is_continuous::no, make_slice({ query::clustering_range::make(make_ck(0), make_ck(4)) }), { 1, 3 }, {
+ expected_row(1, is_continuous::yes),
+ expected_row(3, is_continuous::yes),
+ expected_row(5, is_continuous::no),
+ expected_row(expected_row::dummy_tag_t{}, is_continuous::no)
+ });
+ });
+}
+
+SEASTAR_TEST_CASE(test_three_rows_first_continuous4) {
+ return seastar::async([] {
+ test_three_rows(1, true, is_continuous::yes, 3, false, is_continuous::no, 5, true, is_continuous::no, make_slice({ query::clustering_range::make(make_ck(2), make_ck(4)) }), { 3 }, {
+ expected_row(1, is_continuous::yes),
+ expected_row(3, is_continuous::no),
+ expected_row(5, is_continuous::no),
+ expected_row(expected_row::dummy_tag_t{}, is_continuous::no)
+ });
+ });
+}
+
+SEASTAR_TEST_CASE(test_three_rows_first_noncontinuous1) {
+ return seastar::async([] {
+ test_three_rows(1, true, is_continuous::no, 3, false, is_continuous::no, 5, true, is_continuous::no, make_slice({ query::clustering_range::make(make_ck(0), make_ck(6)) }), { 1, 3, 5 }, {
+ expected_row(1, is_continuous::no),
+ expected_row(3, is_continuous::yes),
+ expected_row(5, is_continuous::yes),
+ expected_row(expected_row::dummy_tag_t{}, is_continuous::no)
+ });
+ });
+}
+
+SEASTAR_TEST_CASE(test_three_rows_first_noncontinuous2) {
+ return seastar::async([] {
+ test_three_rows(1, true, is_continuous::no, 3, false, is_continuous::no, 5, true, is_continuous::no, make_slice({ query::clustering_range::make(make_ck(2), make_ck(6)) }), { 3, 5 }, {
+ expected_row(1, is_continuous::no),
+ expected_row(3, is_continuous::no),
+ expected_row(5, is_continuous::yes),
+ expected_row(expected_row::dummy_tag_t{}, is_continuous::no)
+ });
+ });
+}
+
+SEASTAR_TEST_CASE(test_three_rows_first_nonecontinuous3) {
+ return seastar::async([] {
+ test_three_rows(1, true, is_continuous::no, 3, false, is_continuous::no, 5, true, is_continuous::no, make_slice({ query::clustering_range::make(make_ck(0), make_ck(4)) }), { 1, 3 }, {
+ expected_row(1, is_continuous::no),
+ expected_row(3, is_continuous::yes),
+ expected_row(5, is_continuous::no),
+ expected_row(expected_row::dummy_tag_t{}, is_continuous::no)
+ });
+ });
+}
+
+SEASTAR_TEST_CASE(test_three_rows_first_nonecontinuous4) {
+ return seastar::async([] {
+ test_three_rows(1, true, is_continuous::no, 3, false, is_continuous::no, 5, true, is_continuous::no, make_slice({ query::clustering_range::make(make_ck(2), make_ck(4)) }), { 3 }, {
+ expected_row(1, is_continuous::no),
+ expected_row(3, is_continuous::no),
+ expected_row(5, is_continuous::no),
+ expected_row(expected_row::dummy_tag_t{}, is_continuous::no)
+ });
+ });
+}
+
+/*
+ * ================================================================================================
+ * ====== Tests for single rows and range tombstone with single version and single row range ======
+ * ================================================================================================
+ */
+
+static tombstone new_tombstone() {
+ return tombstone(api::new_timestamp(), gc_clock::now());
+}
+
+SEASTAR_TEST_CASE(test_single_row_and_tombstone_not_cached_single_row_range1) {
+ return seastar::async([] {
+ const int ck1 = 1;
+ const int value1 = 12;
+ range_tombstone rt(make_ck(0), bound_kind::incl_start, make_ck(2), bound_kind::incl_end, new_tombstone());
+
+ mutation underlying(PK, SCHEMA);
+ add_row(underlying, ck1, value1);
+ add_tombstone(underlying, rt);
+
+ auto cache = make_incomplete_mutation();
+ auto slice = make_slice({ query::clustering_range::make_singular(make_ck(ck1)) });
+
+ test_slice_single_version(underlying, cache, slice, {
+ expected_fragment(rt),
+ expected_fragment(1)
+ }, {
+ expected_row(1, is_continuous::no),
+ expected_row(expected_row::dummy_tag_t{}, is_continuous::no)
+ }, { rt });
+ });
+}
+
+SEASTAR_TEST_CASE(test_single_row_and_tombstone_not_cached_single_row_range2) {
+ return seastar::async([] {
+ const int ck1 = 1;
+ const int value1 = 12;
+ range_tombstone rt(make_ck(0), bound_kind::incl_start, make_ck(2), bound_kind::incl_end, new_tombstone());
+
+ mutation underlying(PK, SCHEMA);
+ add_row(underlying, ck1, value1);
+ add_tombstone(underlying, rt);
+
+ auto cache = make_incomplete_mutation();
+ auto slice = make_slice({ query::clustering_range::make(make_ck(0), {make_ck(1), false}) });
+
+ test_slice_single_version(underlying, cache, slice, {
+ expected_fragment(rt),
+ }, {
+ expected_row(expected_row::dummy_tag_t{}, is_continuous::no)
+ }, { rt });
+ });
+}
+
+SEASTAR_TEST_CASE(test_single_row_and_tombstone_not_cached_single_row_range3) {
+ return seastar::async([] {
+ const int ck1 = 4;
+ const int value1 = 12;
+ range_tombstone rt(make_ck(0), bound_kind::incl_start, make_ck(2), bound_kind::incl_end, new_tombstone());
+
+ mutation underlying(PK, SCHEMA);
+ add_row(underlying, ck1, value1);
+ add_tombstone(underlying, rt);
+
+ auto cache = make_incomplete_mutation();
+ auto slice = make_slice({ query::clustering_range::make(make_ck(0), make_ck(5)) });
+
+ test_slice_single_version(underlying, cache, slice, {
+ expected_fragment(rt),
+ expected_fragment(4)
+ }, {
+ expected_row(4, is_continuous::no),
+ expected_row(expected_row::dummy_tag_t{}, is_continuous::no)
+ }, { rt });
+ });
+}
+
+SEASTAR_TEST_CASE(test_single_row_and_tombstone_not_cached_single_row_range4) {
+ return seastar::async([] {
+ const int ck1 = 4;
+ const int value1 = 12;
+ range_tombstone rt(make_ck(0), bound_kind::incl_start, make_ck(2), bound_kind::incl_end, new_tombstone());
+
+ mutation underlying(PK, SCHEMA);
+ add_row(underlying, ck1, value1);
+ add_tombstone(underlying, rt);
+
+ auto cache = make_incomplete_mutation();
+ auto slice = make_slice({ query::clustering_range::make(make_ck(3), make_ck(5)) });
+
+ test_slice_single_version(underlying, cache, slice, {
+ expected_fragment(4)
+ }, {
+ expected_row(4, is_continuous::no),
+ expected_row(expected_row::dummy_tag_t{}, is_continuous::no)
+ }, {});
It is loading more messages.
0 new messages