Skip to content

Commit

Permalink
flat_mutation_reader: Split readers by file and remove unnecessary in…
Browse files Browse the repository at this point in the history
…cludes.

The flat_mutation_reader files were conflated and contained multiple
readers, which were not strictly necessary. Splitting optimizes both
iterative compilation times, as touching rarely used readers doesn't
recompile large chunks of codebase. Total compilation times are also
improved, as the size of flat_mutation_reader.hh and
flat_mutation_reader_v2.hh have been reduced and those files are
included by many file in the codebase.

With changes

real	29m14.051s
user	168m39.071s
sys	5m13.443s

Without changes

real	30m36.203s
user	175m43.354s
sys	5m26.376s

Closes scylladb#10194
  • Loading branch information
Mikołaj Sielużycki authored and denesb committed Mar 14, 2022
1 parent 26b1be0 commit 1d84a25
Show file tree
Hide file tree
Showing 83 changed files with 2,178 additions and 1,681 deletions.
3 changes: 2 additions & 1 deletion cache_flat_mutation_reader.hh
Original file line number Diff line number Diff line change
Expand Up @@ -15,7 +15,8 @@
#include "query-request.hh"
#include "partition_snapshot_row_cursor.hh"
#include "read_context.hh"
#include "flat_mutation_reader.hh"
#include "readers/flat_mutation_reader.hh"
#include "readers/delegating.hh"
#include "clustering_key_filter.hh"

namespace cache {
Expand Down
2 changes: 1 addition & 1 deletion compaction/compaction_strategy.hh
Original file line number Diff line number Diff line change
Expand Up @@ -16,7 +16,7 @@
#include "sstables/shared_sstable.hh"
#include "exceptions/exceptions.hh"
#include "compaction_strategy_type.hh"
#include "flat_mutation_reader.hh"
#include "readers/flat_mutation_reader.hh"
#include "table_state.hh"
#include "strategy_control.hh"

Expand Down
3 changes: 2 additions & 1 deletion configure.py
Original file line number Diff line number Diff line change
Expand Up @@ -698,7 +698,8 @@ def find_headers(repodir, excluded_dirs):
'mutation_partition_serializer.cc',
'converting_mutation_partition_applier.cc',
'mutation_reader.cc',
'flat_mutation_reader.cc',
'readers/mutation_reader.cc',
'readers/mutation_readers.cc',
'mutation_query.cc',
'keys.cc',
'counters.cc',
Expand Down
2 changes: 1 addition & 1 deletion db/chained_delegating_reader.hh
Original file line number Diff line number Diff line change
Expand Up @@ -10,7 +10,7 @@

#include <seastar/core/shared_future.hh>

#include "flat_mutation_reader_v2.hh"
#include "readers/flat_mutation_reader_v2.hh"

// A reader which allows to insert a deferring operation before reading.
// All calls will first wait for a future to resolve, then forward to a given underlying reader.
Expand Down
2 changes: 1 addition & 1 deletion db/data_listeners.hh
Original file line number Diff line number Diff line change
Expand Up @@ -15,7 +15,7 @@

#include "utils/hash.hh"
#include "schema_fwd.hh"
#include "flat_mutation_reader_v2.hh"
#include "readers/flat_mutation_reader_v2.hh"
#include "mutation_reader.hh"
#include "utils/top_k.hh"
#include "schema_registry.hh"
Expand Down
1 change: 1 addition & 0 deletions db/size_estimates_virtual_reader.cc
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@
#include "replica/database.hh"

#include "db/size_estimates_virtual_reader.hh"
#include "readers/from_mutations.hh"

namespace db {

Expand Down
3 changes: 2 additions & 1 deletion db/size_estimates_virtual_reader.hh
Original file line number Diff line number Diff line change
Expand Up @@ -8,7 +8,8 @@
* SPDX-License-Identifier: AGPL-3.0-or-later
*/

#include "flat_mutation_reader_v2.hh"
#include "readers/flat_mutation_reader_v2.hh"
#include "readers/flat_mutation_reader.hh"
#include "db/system_keyspace.hh"

namespace replica {
Expand Down
2 changes: 1 addition & 1 deletion db/view/build_progress_virtual_reader.hh
Original file line number Diff line number Diff line change
Expand Up @@ -10,7 +10,7 @@
#include "db/system_keyspace.hh"
#include "db/timeout_clock.hh"
#include "dht/i_partitioner.hh"
#include "flat_mutation_reader_v2.hh"
#include "readers/flat_mutation_reader_v2.hh"
#include "mutation_fragment.hh"
#include "mutation_reader.hh"
#include "query-request.hh"
Expand Down
1 change: 1 addition & 0 deletions db/view/view.cc
Original file line number Diff line number Diff line change
Expand Up @@ -56,6 +56,7 @@
#include "utils/exponential_backoff_retry.hh"
#include "utils/fb_utilities.hh"
#include "query-result-writer.hh"
#include "readers/from_fragments.hh"

using namespace std::chrono_literals;

Expand Down
2 changes: 1 addition & 1 deletion db/view/view.hh
Original file line number Diff line number Diff line change
Expand Up @@ -12,7 +12,7 @@
#include "gc_clock.hh"
#include "query-request.hh"
#include "schema_fwd.hh"
#include "flat_mutation_reader.hh"
#include "readers/flat_mutation_reader.hh"
#include "frozen_mutation.hh"

class frozen_mutation_and_schema;
Expand Down
1 change: 1 addition & 0 deletions db/view/view_updating_consumer.hh
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@
#include "mutation_rebuilder.hh"

class evictable_reader_handle;
class evictable_reader_handle_v2;

namespace db::view {

Expand Down
2 changes: 2 additions & 0 deletions db/virtual_table.cc
Original file line number Diff line number Diff line change
Expand Up @@ -12,6 +12,8 @@

#include "db/virtual_table.hh"
#include "db/chained_delegating_reader.hh"
#include "readers/reversing.hh"
#include "readers/forwardable.hh"

namespace db {

Expand Down
36 changes: 36 additions & 0 deletions dht/i_partitioner_fwd.hh
Original file line number Diff line number Diff line change
@@ -0,0 +1,36 @@
/*
* Modified by ScyllaDB
* Copyright (C) 2022-present ScyllaDB
*/

/*
* SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0)
*/

#pragma once
#include <vector>
#include "range.hh"

namespace sstables {

class key_view;
class decorated_key_view;

}

namespace dht {

class decorated_key;
class ring_position;
class token;

using partition_range = nonwrapping_range<ring_position>;
using token_range = nonwrapping_range<token>;

using partition_range_vector = std::vector<partition_range>;
using token_range_vector = std::vector<token_range>;

class decorated_key;

using decorated_key_opt = std::optional<decorated_key>;
}
2 changes: 1 addition & 1 deletion frozen_mutation.cc
Original file line number Diff line number Diff line change
Expand Up @@ -26,7 +26,7 @@
#include "idl/uuid.dist.impl.hh"
#include "idl/keys.dist.impl.hh"
#include "idl/mutation.dist.impl.hh"
#include "flat_mutation_reader.hh"
#include "readers/flat_mutation_reader.hh"
#include "converting_mutation_partition_applier.hh"
#include "mutation_partition_view.hh"

Expand Down
2 changes: 1 addition & 1 deletion index/built_indexes_virtual_reader.hh
Original file line number Diff line number Diff line change
Expand Up @@ -8,7 +8,7 @@

#include "replica/database.hh"
#include "db/system_keyspace.hh"
#include "flat_mutation_reader_v2.hh"
#include "readers/flat_mutation_reader_v2.hh"
#include "mutation_fragment_v2.hh"
#include "mutation_reader.hh"
#include "query-request.hh"
Expand Down
2 changes: 1 addition & 1 deletion message/messaging_service.cc
Original file line number Diff line number Diff line change
Expand Up @@ -88,7 +88,7 @@
#include <boost/range/adaptor/filtered.hpp>
#include <boost/range/adaptor/indirected.hpp>
#include "frozen_mutation.hh"
#include "flat_mutation_reader.hh"
#include "readers/flat_mutation_reader.hh"
#include "streaming/stream_manager.hh"
#include "streaming/stream_mutation_fragments_cmd.hh"
#include "locator/snitch_base.hh"
Expand Down
2 changes: 1 addition & 1 deletion mutation.cc
Original file line number Diff line number Diff line change
Expand Up @@ -8,7 +8,7 @@

#include "mutation.hh"
#include "query-result-writer.hh"
#include "flat_mutation_reader.hh"
#include "readers/flat_mutation_reader.hh"
#include "mutation_rebuilder.hh"

mutation::data::data(dht::decorated_key&& key, schema_ptr&& schema)
Expand Down
27 changes: 1 addition & 26 deletions mutation_fragment.hh
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@

#include "db/timeout_clock.hh"
#include "reader_permit.hh"
#include "mutation_fragment_fwd.hh"

// mutation_fragments are the objects that streamed_mutation are going to
// stream. They can represent:
Expand Down Expand Up @@ -525,32 +526,6 @@ inline position_in_partition_view partition_end::position() const
std::ostream& operator<<(std::ostream&, partition_region);
std::ostream& operator<<(std::ostream&, mutation_fragment::kind);

using mutation_fragment_opt = optimized_optional<mutation_fragment>;

namespace streamed_mutation {
// Determines whether streamed_mutation is in forwarding mode or not.
//
// In forwarding mode the stream does not return all fragments right away,
// but only those belonging to the current clustering range. Initially
// current range only covers the static row. The stream can be forwarded
// (even before end-of- stream) to a later range with fast_forward_to().
// Forwarding doesn't change initial restrictions of the stream, it can
// only be used to skip over data.
//
// Monotonicity of positions is preserved by forwarding. That is fragments
// emitted after forwarding will have greater positions than any fragments
// emitted before forwarding.
//
// For any range, all range tombstones relevant for that range which are
// present in the original stream will be emitted. Range tombstones
// emitted before forwarding which overlap with the new range are not
// necessarily re-emitted.
//
// When streamed_mutation is not in forwarding mode, fast_forward_to()
// cannot be used.
class forwarding_tag;
using forwarding = bool_class<forwarding_tag>;
}

// range_tombstone_stream is a helper object that simplifies producing a stream
// of range tombstones and merging it with a stream of clustering rows.
Expand Down
20 changes: 20 additions & 0 deletions mutation_fragment_fwd.hh
Original file line number Diff line number Diff line change
@@ -0,0 +1,20 @@
/*
* Copyright (C) 2016-present ScyllaDB
*/

/*
* SPDX-License-Identifier: AGPL-3.0-or-later
*/

#pragma once
#include <seastar/util/bool_class.hh>
#include <seastar/util/optimized_optional.hh>

using namespace seastar;

class mutation_fragment;
class mutation_fragment_v2;

using mutation_fragment_opt = optimized_optional<mutation_fragment>;
using mutation_fragment_v2_opt = optimized_optional<mutation_fragment_v2>;

3 changes: 0 additions & 3 deletions mutation_fragment_v2.hh
Original file line number Diff line number Diff line change
Expand Up @@ -354,9 +354,6 @@ private:

std::ostream& operator<<(std::ostream&, mutation_fragment_v2::kind);

using mutation_fragment_v2_opt = optimized_optional<mutation_fragment_v2>;


// F gets a stream element as an argument and returns the new value which replaces that element
// in the transformed stream.
template<typename F>
Expand Down
4 changes: 3 additions & 1 deletion mutation_reader.cc
Original file line number Diff line number Diff line change
Expand Up @@ -16,10 +16,12 @@
#include <seastar/util/closeable.hh>

#include "mutation_reader.hh"
#include "flat_mutation_reader.hh"
#include "readers/flat_mutation_reader.hh"
#include "readers/empty.hh"
#include "schema_registry.hh"
#include "mutation_compactor.hh"
#include "dht/sharder.hh"
#include "readers/empty_v2.hh"

logging::logger mrlog("mutation_reader");

Expand Down
5 changes: 3 additions & 2 deletions mutation_reader.hh
Original file line number Diff line number Diff line change
Expand Up @@ -14,9 +14,10 @@
#include <seastar/core/future-util.hh>
#include <seastar/core/do_with.hh>
#include "tracing/trace_state.hh"
#include "flat_mutation_reader.hh"
#include "flat_mutation_reader_v2.hh"
#include "readers/flat_mutation_reader.hh"
#include "readers/flat_mutation_reader_v2.hh"
#include "reader_concurrency_semaphore.hh"
#include <seastar/core/io_priority_class.hh>

class reader_selector {
protected:
Expand Down
2 changes: 1 addition & 1 deletion mutation_writer/feed_writers.hh
Original file line number Diff line number Diff line change
Expand Up @@ -8,7 +8,7 @@

#pragma once

#include "flat_mutation_reader.hh"
#include "readers/flat_mutation_reader.hh"
#include "mutation_reader.hh"
#include "seastar/core/coroutine.hh"

Expand Down
2 changes: 1 addition & 1 deletion mutation_writer/multishard_writer.hh
Original file line number Diff line number Diff line change
Expand Up @@ -9,7 +9,7 @@
#pragma once

#include "schema_fwd.hh"
#include "flat_mutation_reader.hh"
#include "readers/flat_mutation_reader.hh"
#include "dht/i_partitioner.hh"
#include "utils/phased_barrier.hh"

Expand Down
4 changes: 2 additions & 2 deletions partition_snapshot_reader.hh
Original file line number Diff line number Diff line change
Expand Up @@ -9,8 +9,8 @@
#pragma once

#include "partition_version.hh"
#include "flat_mutation_reader.hh"
#include "flat_mutation_reader_v2.hh"
#include "readers/flat_mutation_reader.hh"
#include "readers/flat_mutation_reader_v2.hh"
#include "clustering_key_filter.hh"
#include "query-request.hh"
#include <boost/range/algorithm/heap_algorithm.hpp>
Expand Down
1 change: 1 addition & 0 deletions reader_concurrency_semaphore.cc
Original file line number Diff line number Diff line change
Expand Up @@ -8,6 +8,7 @@

#include <seastar/core/seastar.hh>
#include <seastar/core/print.hh>
#include <seastar/core/file.hh>
#include <seastar/util/lazy.hh>
#include <seastar/util/log.hh>
#include <seastar/core/coroutine.hh>
Expand Down
2 changes: 1 addition & 1 deletion reader_concurrency_semaphore.hh
Original file line number Diff line number Diff line change
Expand Up @@ -14,7 +14,7 @@
#include <seastar/core/queue.hh>
#include <seastar/core/expiring_fifo.hh>
#include "reader_permit.hh"
#include "flat_mutation_reader_v2.hh"
#include "readers/flat_mutation_reader_v2.hh"

namespace bi = boost::intrusive;

Expand Down
18 changes: 18 additions & 0 deletions readers/conversion.hh
Original file line number Diff line number Diff line change
@@ -0,0 +1,18 @@
/*
* Copyright (C) 2022-present ScyllaDB
*/

/*
* SPDX-License-Identifier: AGPL-3.0-or-later
*/

#pragma once

class flat_mutation_reader;
class flat_mutation_reader_v2;

// Adapts a v2 reader to v1 reader
flat_mutation_reader downgrade_to_v1(flat_mutation_reader_v2);

// Adapts a v1 reader to v2 reader
flat_mutation_reader_v2 upgrade_to_v2(flat_mutation_reader);
42 changes: 42 additions & 0 deletions readers/delegating.hh
Original file line number Diff line number Diff line change
@@ -0,0 +1,42 @@
/*
* Copyright (C) 2022-present ScyllaDB
*/

/*
* SPDX-License-Identifier: AGPL-3.0-or-later
*/

#pragma once
#include "readers/flat_mutation_reader.hh"

class flat_mutation_reader;

flat_mutation_reader make_delegating_reader(flat_mutation_reader&);

class delegating_reader : public flat_mutation_reader::impl {
flat_mutation_reader_opt _underlying_holder;
flat_mutation_reader* _underlying;
public:
// when passed a lvalue reference to the reader
// we don't own it and the caller is responsible
// for evenetually closing the reader.
delegating_reader(flat_mutation_reader& r)
: impl(r.schema(), r.permit())
, _underlying_holder()
, _underlying(&r)
{ }
// when passed a rvalue reference to the reader
// we assume ownership of it and will close it
// in close().
delegating_reader(flat_mutation_reader&& r)
: impl(r.schema(), r.permit())
, _underlying_holder(std::move(r))
, _underlying(&*_underlying_holder)
{ }

virtual future<> fill_buffer() override;
virtual future<> fast_forward_to(position_range pr) override;
virtual future<> next_partition() override;
virtual future<> fast_forward_to(const dht::partition_range& pr) override;
virtual future<> close() noexcept override;
};
Loading

0 comments on commit 1d84a25

Please sign in to comment.