When we, for some reason, fail to compact an SSTable, we do not log the file
name leaving us with cryptic messages that tell us what happened, but not where
it happened.
This patch adds logging in compaction so that we'll know what's going on.
Please note that readers are more of a concern, because the SSTable being
written technically do not exist yet. Still, better safe than sorry: if
open_data fails, or we leave an unfinished SSTable, it is still good to know
which one was the culprit.
Some argument can be made about whether we should log this at the lower SSTable
level, or at the compaction level.
The reason I am logging this at the compaction level, is that we don't really
know which exception will trigger, and where: it may be the case that we're
seeing exceptions that are not SSTable specific, and may not have the chance to
log it properly.
In particular, if the exception happens inside the reader: read_rows() and
friends only return a mutation reader, which doesn't really do anything until
we call read(). But at that time, we don't hold any pointers to the SSTable
anymore.
In Summary, logging at the compaction level guarantees that we always do it no
matter what. Exceptions that are part of the main SSTable path can log the file
name as well if they want: if that's the case, we'll be left with the name
appearing twice. That's totally harmless, and better than none.
Fixes#1123
Signed-off-by: Glauber Costa <glauber@scylladb.com>
Message-Id: <c5c969fb6aeb788a037bd7a4ea69979c1042cb34.1459263847.git.glauber@scylladb.com>
When scylla stopped an ongoing compaction, the event was reported
as an error. This patch introduces a specialized exception for
compaction stop so that the event can be handled appropriately.
Before:
ERROR [shard 0] compaction_manager - compaction failed: read exception:
std::runtime_error (Compaction for keyspace1/standard1 was deliberately
stopped.)
After:
INFO [shard 0] compaction_manager - compaction info: Compaction for
keyspace1/standard1 was stopped due to shutdown.
Signed-off-by: Raphael S. Carvalho <raphaelsc@scylladb.com>
Message-Id: <1f85d4e5c24d23a1b4e7e0370a2cffc97cbc6d44.1455034236.git.raphaelsc@scylladb.com>
storage_service::get_local_ranges returns sorted ranges, which are
not overlapping nor wrap-around. As a result, there is no need for
the consumer to do anything.
Signed-off-by: Raphael S. Carvalho <raphaelsc@scylladb.com>
After this patch, our I/O operations will be tagged into a specific priority class.
The available classes are 5, and were defined in the previous patch:
1) memtable flush
2) commitlog writes
3) streaming mutation
4) SSTable compaction
5) CQL query
Signed-off-by: Glauber Costa <glauber@scylladb.com>
"This series moves the "backup" logic into the sstable::write_components()
methods, adds a support for enabling backup for sstables flushed in the
compaction flow (in addition to a regular flushing flow which had this support
already) and enables the "incremental_backups" configuration option."
I fixed up a merge conflict with commit 5e953b5 ("Merge "Add support to
stop ongoing compaction" from Raphael").
Enable incremental backup when sstables are flushed if
incremental backup has been requested.
It has been enabled in the regular flushing flow before but
wasn't in the compaction flow.
This patch enables it in both places and does it using a
backup capability of sstable::write_components() method(s).
Signed-off-by: Vlad Zolotarov <vladz@cloudius-systems.com>
That's needed for nodetool stop, which is called to stop all ongoing
compaction. The implementation is about informing an ongoing compaction
that it was asked to stop, so the compaction itself will trigger an
exception. Compaction manager will catch this exception and re-schedule
the compaction.
Signed-off-by: Raphael S. Carvalho <raphaelsc@scylladb.com>
compaction_info makes more sense because this structure doesn't
only store stats about ongoing compaction. Soon, we will add
information to it about whether or not an user asked to stop the
respective ongoing compaction.
Signed-off-by: Raphael S. Carvalho <raphaelsc@scylladb.com>
When compacting sstable, mutation that doesn't belong to current shard
should be filtered out. Otherwise, mutation would be duplicated in
all shards that share the sstable being compacted.
sstable_test will now run with -c1 because arbitrary keys are chosen
for sstables to be compacted, so test could fail because of mutations
being filtered out.
fixes#527.
Signed-off-by: Raphael S. Carvalho <raphaelsc@scylladb.com>
Message-Id: <1acc2e8b9c66fb9c0c601b05e3ae4353e514ead5.1453140657.git.raphaelsc@scylladb.com>
The implementation is about storing generation of compacting sstables
in an unordered set per column family, so before strategy is called,
compaction manager will filter out compacting sstables.
Signed-off-by: Raphael S. Carvalho <raphaelsc@scylladb.com>
Currently, compaction strategy is the responsible for both getting the
sstables selected for compaction and running compaction.
Moving the code that runs compaction from strategy to manager is a big
improvement, which will also make possible for the compaction manager
to keep track of which sstables are being compacted at a moment.
This change will also be needed for cleanup and concurrent compaction
on the same column family.
Signed-off-by: Raphael S. Carvalho <raphaelsc@scylladb.com>
Cleanup is about rewriting a sstable discarding any keys that
are irrelevant, i.e. keys that don't belong to current node.
Parameter cleanup was added to compact_sstables.
If set to true, irrelevant code such as the one that updates
compaction history will be skipped. Logic was also added to
discard irrelevant keys.
Signed-off-by: Raphael S. Carvalho <raphaelsc@scylladb.com>
max_purgeable was being incorrectly calculated because the code
that creates vector of uncompacted sstables was wrong.
This value is used to determine whether or not a tombstone can
be purged.
Operand < is supposed to be used instead in the callback passed
as third parameter to boost::set_difference.
This fix is a step towards closing the issue #676.
Signed-off-by: Raphael S. Carvalho <raphaelsc@scylladb.com>
Use steady_clock instead of high_resolution_clock where monotonic
clock is required. high_resolution_clock is essentially a
system_clock (Wall Clock) therefore may not to be assumed monotonic
since Wall Clock may move backwards due to time/date adjustments.
Fixes issue #638
Signed-off-by: Vlad Zolotarov <vladz@cloudius-systems.com>
I am sure it's a compiler issue but I am not ready to give up and
upgrade just yet:
sstables/compaction.cc:307:55: error: converting to ‘std::unordered_map<int, long int>’ from initializer list would use explicit constructor ‘std::unordered_map<_Key, _Tp, _Hash, _Pred, _Alloc>::unordered_map(std::unordered_map<_Key, _Tp, _Hash, _Pred, _Alloc>::size_type, const hasher&, const key_equal&, const allocator_type&) [with _Key = int; _Tp = long int; _Hash = std::hash<int>; _Pred = std::equal_to<int>; _Alloc = std::allocator<std::pair<const int, long int> >; std::unordered_map<_Key, _Tp, _Hash, _Pred, _Alloc>::size_type = long unsigned int; std::unordered_map<_Key, _Tp, _Hash, _Pred, _Alloc>::hasher = std::hash<int>; std::unordered_map<_Key, _Tp, _Hash, _Pred, _Alloc>::key_equal = std::equal_to<int>; std::unordered_map<_Key, _Tp, _Hash, _Pred, _Alloc>::allocator_type = std::allocator<std::pair<const int, long int> >]’
stats->start_size, stats->end_size, {});
That's important for compaction stats API that will need stats
data of each ongoing compaction.
Signed-off-by: Raphael S. Carvalho <raphaelsc@scylladb.com>
When compaction job finishes, call function to update the system
table COMPACTION_HISTORY. That's also needed for the compaction
history API.
Signed-off-by: Raphael S. Carvalho <raphaelsc@scylladb.com>
Let's move the code that prints that a compaction succeeded only
after the code that catches exception on either read or write
fibers. Let's also get rid of done and use repeat instead in
the read fiber.
Signed-off-by: Raphael S. Carvalho <raphaelsc@scylladb.com>
This assert (in write fiber) would fail if read fiber failed
because the variable done will not be set to true.
The use of assert is very bad, because it prevents scylla
from proceeding, which is possible.
To solve it, let's trigger an exception if done is not true.
We do have code that will wait for both read and write fibers,
and catch exceptions, if any.
Closes#523.
Signed-off-by: Raphael S. Carvalho <raphaelsc@scylladb.com>
Since mutation stores all its data externally and the object itself is
basically just a std::unique_ptr<> there is no need for stdx::optional.
Smart pointer set to nullptr represents a disengaged mutation_opt.
Signed-off-by: Paweł Dziepak <pdziepak@scylladb.com>
Adapt our compaction code to start writing a new sstable if the
one being written reached its maximum size. Leveled strategy works
with that concept. If a strategy other than leveled is being used,
everything will work as before.
Signed-off-by: Raphael S. Carvalho <raphaelsc@cloudius-systems.com>
When we write an SSTable, all its components are already in memory. load() is
to big of a hammer.
We still want to keep the write operation separated from the preparation to
read, but in the case of a newly written SSTable, all we need to do is to open
the index and data file.
Fixes#300
Signed-off-by: Glauber Costa <glommer@scylladb.com>
The compaction strategy was modify to return its compaction type.
The type method calls the virtual impl type method. Each of the
implementations return its type.
A name method was added to the compaction strategy that return the name
according to the strategy type.
And the static type method was modified to recieve a const reference to
the string.
Signed-off-by: Amnon Heiman <amnon@cloudius-systems.com>
Size-tiered compaction strategy works by creating buckets with sstables
of similar size, but if a bucket's size is greater than max_threshold
(defined in schema), it will not be selected for compaction.
Scenario described by issue 298 is facing that. If compaction takes a
long time to finish, more than max_threshold sstables will be created,
and thus there wouldn't be a 'valid' bucket for compaction.
Solution is to not add a sstable for a bucket that reached its limit,
so that bucket will have a chance to be compacted.
Fixes issue #298.
Signed-off-by: Raphael S. Carvalho <raphaelsc@cloudius-systems.com>
Read-ahead will require that we close input_streams. As part of that
we have to close sstables, and mutation_readers (which encapsulate
input_streams). This is part 1 of a patchset series to do that.
(The overarching goal is to enable read-ahead for sstables, see #244)
Conflicts:
sstables/compaction.cc
Using a lambda for implementing a mutation_reader is nifty, but does not
allow us to add methods.
Switch to a class-based implementation in anticipation of adding a close()
method.
"Initial implementation/transposition of commit log replay.
* Changes replay position to be shard aware
* Commit log segment ID:s now follow basically the same scheme as origin;
max(previous ID, wall clock time in ms) + shard info (for us)
* SStables now use the DB definition of replay_position.
* Stores and propagates (compaction) flush replay positions in sstables
* If CL segments are left over from a previous run, they, and existing
sstables are inspected for high water mark, and then replayed from
those marks to amend mutations potentially lost in a crash
* Note that CPU count change is "handled" in so much that shard matching is
per _previous_ runs shards, not current.
Known limitations:
* Mutations deserialized from old CL segments are _not_ fully validated
against existing schemas.
* System::truncated_at (not currently used) does not handle sharding afaik,
so watermark ID:s coming from there are dubious.
* Mutations that fail to apply (invalid, broken) are not placed in blob files
like origin. Partly because I am lazy, but also partly because our serial
format differs, and we currently have no tools to do anything useful with it
* No replay filtering (Origin allows a system property to designate a filter
file, detailing which keyspace/cf:s to replay). Partly because we have no
system properties.
There is no unit test for the commit log replayer (yet).
Because I could not really come up with a good one given the test
infrastructure that exists (tricky to kill stuff just "right").
The functionality is verified by manual testing, i.e. running scylla,
building up data (cassandra-stress), kill -9 + restart.
This of course does not really fully validate whether the resulting DB is
100% valid compared to the one at k-9, but at least it verified that replay
took place, and mutations where applied.
(Note that origin also lacks validity testing)"
Compaction moves output_writer and done out of variables which are then
reused, causing use-after-free. In addition it fails to protect the variables
for the end of the loop.
Fix by avoiding the moves and adding an additional capture.
(endsize / (1024*1024)) is an integer calculation, so if endsize is
lower than 1024^2, the result would be 0.
Signed-off-by: Raphael S. Carvalho <raphaelsc@cloudius-systems.com>
When forcing a compaction on a column family with no sstables, an
assert will fail because there is no sstables to be compacted.
This problem is fixed by ignoring a compaction request when no
sstable is provided.
Fixes#61.
Signed-off-by: Raphael S. Carvalho <raphaelsc@cloudius-systems.com>
Reviewed-by: Nadav Har'El <nyh@cloudius-systems.com>