Compare commits

..

189 Commits

Author SHA1 Message Date
Jenkins
5174b1cd13 release: prepare for 3.0.2 by slivne 2019-01-30 16:14:34 +02:00
Nadav Har'El
9ba608cae4 cql3: really ensure retrieval of columns for filtering
Commit fd422c954e aimed to fix
issue #3803. In that issue, if a query SELECTed only certain columns but
did filtering (ALLOW FILTERING) over other unselected columns, the filtering
didn't work. The fix involved adding the columns being filtered to the set
of columns we read from disk, so they can be filtered.

But that commit included an optimization: If you have clustering keys
c1 and c2, and the query asks for a specific partition key and c1 < 3 and
c2 > 3, the "c1 < 3" part does NOT need to be filtered because it is already
done as a slice (a contiguous read from disk). The committed code erroneously
concluded that both c1 and c2 don't need to be filtered, which was wrong
(c2 *does* need to be read and filtered).

In this patch, we fix this optimization. Previously, we used the "prefix
length", which in the above example was 2 (both c1 and c2 were filtered)
but we need a new and more elaborate function,
num_prefix_columns_that_need_not_be_filtered(), to determine we can only
skip filtering of 1 (c1) and cannot skip the second.

Fixes #4121. This patch also adds a unit test to confirm this.

Signed-off-by: Nadav Har'El <nyh@scylladb.com>

Message-Id: <20190123131212.6269-1-nyh@scylladb.com>
(cherry picked from commit 76f1fcc346)
2019-01-23 21:11:05 +02:00
Avi Kivity
f7c5cbc645 build: fix libdeflate object file corruption during parallel build
libdeflate's build places some object files in the source directory, which is
shared between the debug and release build. If the same object file (for the two
modes) is written concurrently, or if one more reads it while the other writes it,
it will be corrupted.

Fix by not building the executables at all. They aren't needed, and we already
placed the libraries' objects in the build directory (which is unshared). We only
need the libraries anyway.

Fixes #4130.
Branches: master, branch-3.0
Message-Id: <20190123145435.19049-1-avi@scylladb.com>

(cherry picked from commit c83ae62aed)
2019-01-23 21:11:05 +02:00
Duarte Nunes
cf4b4d4878 Merge 'hinted handoff: cache cf mappings' from Vlad
"
Cache cf mappings when breaking in the middle of a segment sending so
that the sender has them the next time it wants to send this segment
for where it left off before.

Also add the "discard" metric so that we can track hints that are being
discarded in the send flow.
"

Fixes #4122

* 'hinted_handoff_cache_cf_mappings-v1' of https://github.com/vladzcloudius/scylla:
  hinted handoff: cache column family mappings for segments that were not sent out in full
  hinted handoff: add a "discarded" metric

(cherry picked from commit 88c7c1e851)
2019-01-23 17:14:29 +02:00
Asias He
45bb1ba1b7 streaming: Futurize estimate_partitions
The loop can take a long time if the number of sstables and/or ranges
are large. To fix, futurize the loop.

Fixes: #4005

Message-Id: <3b05cb84f3f57cc566702142c6365a04b075018e.1545290730.git.asias@scylladb.com>
(cherry picked from commit bcba6b4f4d)
2019-01-22 18:20:21 +02:00
Botond Dénes
28294ed42e auth/service: unregister migration listener on stop()
Otherwise any event that triggers notification to this listener would
trigger a heap-use-after-free.

Refs: #4107

Signed-off-by: Botond Dénes <bdenes@scylladb.com>
Message-Id: <b6bbd609371a2312aed7571b05119d59c7d103d7.1548067626.git.bdenes@scylladb.com>
(cherry picked from commit f229dff210)
2019-01-22 17:54:36 +02:00
Jenkins
3c4f8cf6ed release: prepare for 3.0.1 by hagitsegev 2019-01-20 12:42:00 +02:00
Botond Dénes
7b94264ae5 mutlishard_mutation_query(): use correct reader concurrency semaphore
The multishard mutation query used the semaphore obtained from
`database::user_read_concurrency_sem()` to pause-resume shard readers.
This presented a problem when `multishard_mutation_query()` was reading
from system tables. In this case the readers themselves would obtain
their permits from the system read concurrency semaphore. Since the
pausing of shard readers used the user read semaphore, pausing failed to
fulfill its objective of alleviating pressure on the semaphore the reads
obtained their permits from. In some cases this lead to a deadlock
during system reads.
To ensure the correct semaphore is used for pausing-resuming readers,
obtain the semaphore from the `table` object. To avoid looking up the
table on every pause or resume call, cache the semaphores when readers
are created.

Fixes: #4096

Signed-off-by: Botond Dénes <bdenes@scylladb.com>
Message-Id: <c784a3cd525ce29642d7216fbe92638fa7884e88.1547729119.git.bdenes@scylladb.com>
(cherry picked from commit 4537ec7426)
2019-01-17 18:08:01 +02:00
Duarte Nunes
22a085fbd3 Merge 'Fix filtering with LIMIT and paging' from Piotr
"
Before this series the limit was applied per page instead
of globally, which might have resulted in returning too many
rows.

To fix that:
 1. restrictions filter now has a 'remaining' parameter
    in order to stop accepting rows after enough of them
    have already been accepted
 2. pager passes its row limit to restrictions filter,
    so no more rows than necessary will be served to the client
 3. results no longer need to be trimmed on select_statement
    level

Tests: unit (release)
"

Fixes #4100

* 'fix_filtering_limit_with_paging_3' of https://github.com/psarna/scylla:
  tests: add filtering+limit+paging test case
  tests: allow null paging state in filtering tests
  cql3: fix filtering with LIMIT with regard to paging

(cherry picked from commit 7505815013)
2019-01-17 18:07:41 +02:00
Tomasz Grabiec
2d181da656 row_cache: Fix crash on memtable flush with LCS
Presence checker is constructed and destroyed in the standard
allocator context, but the presence check was invoked in the LSA
context. If the presence checker allocates and caches some managed
objects, there will be alloc-dealloc mismatch.

That is the case with LeveledCompactionStrategy, which uses
incremental_selector.

Fix by invoking the presence check in the standard allocator context.

Fixes #4063.

Message-Id: <1547547700-16599-1-git-send-email-tgrabiec@scylladb.com>
(cherry picked from commit 32f711ce56)
2019-01-15 21:16:13 +02:00
Nadav Har'El
d427a23d42 scylla_util.py: make view_hints_directory setting optional
It is optional to set "view_hints_directory", so we shouldn't insist that
it is defined in scylla.yaml on upgrade.

Fixes #4091.

Signed-off-by: Nadav Har'El <nyh@scylladb.com>
Message-Id: <20190114125225.10794-1-nyh@scylladb.com>
(cherry picked from commit 9062750089)
2019-01-14 16:59:40 +02:00
Shlomi Livne
37ab553f02 release: prepare for 3.0.0
Signed-off-by: Shlomi Livne <shlomi@scylladb.com>
2019-01-12 22:24:08 +02:00
Raphael S. Carvalho
6a3f4fb3f9 database: Fix race condition in sstable snapshot
Race condition takes place when one of the sstables selected by snapshot
is deleted by compaction. Snapshot fails because it tries to link a
sstable that was previously unlinked by compaction's sstable deletion.

Refs #4051.

(master commit 1b7cad3531)
Signed-off-by: Raphael S. Carvalho <raphaelsc@scylladb.com>
Message-Id: <20190110194048.26051-1-raphaelsc@scylladb.com>
2019-01-11 13:48:12 +02:00
Avi Kivity
8168d13887 Merge "Fix UDTs representation in serialization header" from Piotr
"
Tests: unit(release)
"

Fixes #4073.

* commit 'FETCH_HEAD~1':
  Add test for serialization header with UDT
  Fix UDT names in serialization header

(cherry picked from commit 4a6aeced59)
2019-01-11 07:48:23 +02:00
Benny Halevy
13bdec6eb4 sstables: mc: sign-extend serialization_header min_local_deletion_time_base and min_ttl_base
Refs #4074
Refs #3353

Signed-off-by: Benny Halevy <bhalevy@scylladb.com>
Message-Id: <20190110141439.1324-1-bhalevy@scylladb.com>
(cherry picked from commit 2dc3776407)
2019-01-11 07:47:45 +02:00
Benny Halevy
57e7081d86 sstables: mc: sign-extend delta local_deletion_time and delta ttl
Follow Cassandra's encoding so that values that are less than the
baseline encoding_stats will wrap-around in 64-bits rather tham 32.

Fixes #4074
Refs #3353

Signed-off-by: Benny Halevy <bhalevy@scylladb.com>
Message-Id: <20190109192703.18371-1-bhalevy@scylladb.com>
(cherry picked from commit 60323b79d1)
2019-01-09 23:16:00 +02:00
Avi Kivity
2fcae36d96 tests: mutation_source_test: generate valid utf-8 data
test_fast_forwarding_across_partitions_to_empty_range uses an uninitialized
string to populate an sstable, but this can be invalid utf-8 so that sstable
cannot be sstabledumped.

Make it valid by using make_random_string().

Fixes #4040.
Message-Id: <20190107193240.14409-1-avi@scylladb.com>

(cherry picked from commit d8adbeda11)
2019-01-08 14:53:55 +02:00
Avi Kivity
ba62dcd5c7 Update seastar submodule
* seastar 618bc23...5226277 (1):
  > iotune: Initialize io_rates member variables

Fixes #4064.
2019-01-08 11:39:50 +02:00
Nadav Har'El
515399ce17 materialized views: move hints to top-level directory
While we keep ordinary hints in a directory parallel to the data directory,
we decided to keep the materialized view hints in a subdirectory of the data
directory, named "view_pending_updates". But during boot, we expect all
subdirectories of data/ to be keyspace names, and when we notice this one,
we print a warning:

   WARN: database - Skipping undefined keyspace: view_pending_updates

This spurious warning annoyed users. But moreover, we could have bigger
problems if the user actually tries to create a keyspace with that name.

So in this patch, we move the view hints to a separate top-level directory,
which defaults to /var/lib/scylla/view_hints, but as usual can be configured.

Signed-off-by: Nadav Har'El <nyh@scylladb.com>
Message-Id: <20190107142257.16342-1-nyh@scylladb.com>
(cherry picked from commit da090a5458)
2019-01-07 22:01:56 +02:00
Benny Halevy
772c4b5fdc sstables: mc: expired_liveness_ttl should be max int32_t rather than max uint32_t
Corresponding to Cassandra's EXPIRED_LIVENESS_TTL = Integer.MAX_VALUE;

Fixes #4060

Signed-off-by: Benny Halevy <bhalevy@scylladb.com>
Message-Id: <20190107172457.20430-1-bhalevy@scylladb.com>
(cherry picked from commit 40410465d7)
2019-01-07 21:59:59 +02:00
Avi Kivity
874d88c98d Update seastar submodule
* seastar 08f1258...618bc23 (1):
  > perftune.py: tune only active NVMe HW queues on i3 AWS instances

Ref #3831.
2019-01-06 12:59:22 +02:00
Avi Kivity
5a178ff635 compaction_controller: increase minimum shares to 50 (~5%) for small-data workloads
The workload in #3844 has these characteristics:
 - very small data set size (a few gigabytes per shard)
 - large working set size (all the data, enough for high cache miss rate)
 - high overwrite rate (so a compaction results in 12X data reduction)

As a result, the compaction backlog controller assigns very few shares to
compaction (low data set size -> low backlog), so compaction proceeds very slowly.
Meanwhile, we have tons of cache misses, and each cache miss needs to read from a
large number of sstables (since compaction isn't progressing). The end result is
a high read amplification, and in this test, timeouts.

While we could declare that the scenario is very artificial, there are other
real-world scenarios that could trigger it. Consider a 100% write load
(population phase) followed by 100% read. Towards the end of the last compaction,
the backlog will drop more and more until compaction slows to a crawl, and until
it completes, all the data (for that compaction) will have to be read from its
input sstables, resulting in read amplification.

We should probably have read amplification affect the backlog, but for now the
simpler solution is to increase the minimum shares to 50 so that compaction
always makes forward progress. This will result in higher-than-needed compaction
bandwidth in some low write rate scenarios so we will see fluctuations in request
rate (what the controller was designed to avoid), but these fluctioations will be
limited to 5%.

Since the base class backlog_controller has a fixed (0, 0) point, remove it
and add it to derived classes (setting it to (0, 50) for compaction).

Fixes #3844 (or at least improves it).
Message-Id: <20181231162710.29410-1-avi@scylladb.com>

(cherry picked from commit b0980ba7c6)
2019-01-04 13:28:43 +02:00
Avi Kivity
d67439b910 Revert "release: prepare for 3.0-rc4"
This reverts commit 21a5a4c76a. we were already
at rc4, and the commit only changes the syntax (from the incorrect one to
the correct one).
2019-01-04 12:34:38 +02:00
Hagit Segev
21a5a4c76a release: prepare for 3.0-rc4 2019-01-03 23:53:47 +02:00
Tomasz Grabiec
f818d6ee3f tests: cql_test_env: Start the compaction manager
Broken in fee4d2e

Not doing this results in compaction requests being ignored.

One effect of this is that perf_fast_forward produces many sstables instead of one.

Refs #3984
Refs #3983

Message-Id: <1544719540-10178-1-git-send-email-tgrabiec@scylladb.com>
(cherry picked from commit 245a0d953a)
2019-01-03 14:56:42 +01:00
Tomasz Grabiec
20c2745592 Merge "Improve times to start / stop the nodes" from Glauber
If the compaction manager is started, compactions may start (this is
regardless of whether or not we trigger them). The problem with that is
that they start at a time in which we are flushing the commitlog and the
initialization procedure waits for the commitlog to be fully flushed and
the resulting memtables flushed before we move on.

Because there are no incoming writes, the amount of shares in memtable
flushes decrease as memory used decreases and that can cause the startup
procedure to take a long time.

We have recently started to bump the shares manually for manual flushes.
While that guarantees that we will not drive the shares to zero, I will
make the argument that we can do better by making sure that those things
are, at this point, running alone: user experience is affected by
startup times and the bump we give to user-triggered operations will
only do so much. Even if we increase the shares a lot flushes will still
be fighting for resources with compactions and startup will take longer
than it could.

By making sure that flushes are this point running alone we improve the
user experience by making sure the startup is as fast as it can be.

There is a similar problem at the drain level, which is also fixed in this
series.

Fixes #3958

* git@github.com:glommer/scylla.git faster-restart
  compaction_manager: delay initialization of the compaction manager.
  drain: stop compactions early

(cherry picked from commit 3e70ae1d06)
2019-01-03 14:56:16 +01:00
Avi Kivity
cf5c72561c release: prepare for scylla-3.0-rc4 2019-01-03 13:15:58 +02:00
Botond Dénes
53b85e5d32 querier_cache: unregister queriers evicted due to expired TTL
Currently queriers evicted due to their TTL expiring are not
unregistered from the `reader_concurrency_semaphore`. This can cause a
use-after-free when the semaphore tries to evict the same querier at
some later point in time, as the querier entry it has a pointer to is
now invalid.

Fix by unregistering the querier from the semaphore before destroying
the entry.

Refs: #4018
Refs: #4031

Signed-off-by: Botond Dénes <bdenes@scylladb.com>
Message-Id: <4adfd09f5af8a12d73c29d59407a789324cd3d01.1546504034.git.bdenes@scylladb.com>
(cherry picked from commit e5a0ea390a)
2019-01-03 13:14:02 +02:00
Avi Kivity
2456cf63f2 querier_cache: unregister querier from reader_concurrency_semaphore during eviction
In insert_querier(), we may evict older queriers to make room for the new one.
However, we forgot to unregister the evicted queriers from
reader_concurrency_semaphore. As a result, when reader_concurrency_semaphore
eventually wanted to evict something, it saw an inactive_read_handle that was
not connected to a querier_cache::entry, and crashed on use-after-free.

Fix by evicting through the inactive_read_handle associated with the querier
to be evicted. This removes traces of the querier from both
reader_concurrency_semaphore and querier_cache. We also have to massage the
statistics since querier_inactive_read::evict() updates different counters.

Fixes #4018.

Tests: unit(release)
Reviewed-by: Botond Denes <bdenes@scylladb.com>
Message-Id: <20190102175023.26093-1-avi@scylladb.com>
(cherry picked from commit 918d255168)
2019-01-03 13:14:00 +02:00
Pekka Enberg
c1f6ce4251 Merge 'Fixes for the view_update_from_staging_generator' from Duarte
"This series contains a couple of fixes to the
view_update_from_staging_generator, the object responsible for
generating view updates from sstables written through streaming.

Fixes #4021"
* 'materialized-views/staging-generator-fixes/v2' of https://github.com/duarten/scylla:
  db/view/view_update_from_staging_generator: Break semaphore on stop()
  db/view/view_update_from_staging_generator: Restore formatting
  db/view/view_update_from_staging_generator: Avoid creating more than one fiber

(cherry picked from commit 96172b7bca)
2018-12-29 20:22:54 +02:00
Duarte Nunes
fc82eb5586 streaming/stream_session: Only stage sstables for tables with views
When streaming, sstables for which we need to generate view updates
are placed in a special staging directory. However, we only need to do
this for tables that actually have views.

Refs #4021
Message-Id: <20181227215412.5632-1-duarte@scylladb.com>

(cherry picked from commit bab7e6877b)
2018-12-28 20:52:15 +02:00
Avi Kivity
f58e592345 Merge "Fix use-after-free when destroying partition_snapshots in the background"from Tomasz
"
partition_snapshots created in the memtable will keep a reference to
the memtable (as region*) and to memtable::_cleaner. As long as the
reader is alive, the memtable will be kept alive by
partition_snapshot_flat_reader::_container_guard. But after that
nothing prevents it from being destroyed. The snapshot can outlive the
read if mutation_cleaner::merge_and_destroy() defers its destruction
for later. When the read ends after memtable was flushed, the snapshot
will be queued in the cache's cleaner, but internally will reference
memtable's region and cleaner. This will result in a use-after-free
when the snapshot resumes destruction.

The fix is to update snapshots's region and cleaner references at the
time of queueing to point to the cache's region and cleaner.

When memtable is destroyed without being moved to cache there is no
problem because the snapshot would be queued into memtable's cleaner,
which will be drained on destruction from all snapshots.

Introduced in f3da043 (in >= 3.0-rc1)

Fixes #4030.

Tests:

  - mvcc_test (debug)

"

* tag 'fix-snapshot-merging-use-after-free-v1.1' of github.com:tgrabiec/scylla:
  tests: mvcc: Add test_snapshot_merging_after_container_is_destroyed
  tests: mvcc: Introduce mvcc_container::migrate()
  tests: mvcc: Make mvcc_partition move-constructible
  tests: mvcc: Introduce mvcc_container::make_not_evictable()
  tests: mvcc: Allow constructing mvcc_container without a cache_tracker
  mutation_cleaner: Migrate partition_snapshots when queueing for background cleanup
  mvcc: partition_snapshot: Introduce migrate()
  mutation_cleaner: impl: Store a back-reference to the owning mutation_cleaner

(cherry picked from commit 8e2f6d0513)
2018-12-28 13:37:29 +02:00
Duarte Nunes
6375b1e5b7 streaming/stream_session: Don't use table reference across defer points
When creating a sstable from which to generate view updates, we held
on to a table reference across defer points. In case there's a
concurrent schema drop, the table object might be destroyed and we
will incur in a use-after-free. Solve this by holding on to a shared
pointer and pinning the table object.

Refs #4021

Signed-off-by: Duarte Nunes <duarte@scylladb.com>
Message-Id: <20181227105921.3601-1-duarte@scylladb.com>
(cherry picked from commit 66e45469b2)
2018-12-28 10:58:26 +02:00
Gleb Natapov
7ca24efb39 streaming: always read from rpc::source until end-of-stream during mutation sending
rpc::source cannot be abandoned until EOS is reached, but current code
does not obey it if error code is received, it throws exception instead that
aborts the reading loop. Fix it by moving exception throwing out of the
loop.

Fixes: #4025

Message-Id: <20181227135051.GC29458@scylladb.com>
(cherry picked from commit 37b4043677)
2018-12-27 18:59:59 +02:00
Avi Kivity
32ebaaa585 Update libdeflate submodule
* libdeflate 17ec6c9...e7e54ea (1):
  > build: improve out-of-tree build with multiple output trees

(cherry picked from commit d6a22c50cb)
2018-12-25 14:41:24 +02:00
Nadav Har'El
a88c722a4c build_ami.sh: need to check out the right branch of scylla-jmx
This patch is for branch 3.0's build_ami.sh.
It checks out the latest master branch of scylla-jmx, which not only
sounds wrong, it also doesn't work: the latest master of scylla-jmx
can only build a "relocatable package" but branch 3.0 doesn't work with
those.

This patch needs to be applied only in branch 3.0.

It should probably be made more general, though... build_ami.sh should
have been able to figure out what is the *current* branch, and if it is
branch-3.0 or next-3.0, check out branch-3.0 of the other repositories.
But I'm not sure how to do this correctly.

Signed-off-by: Nadav Har'El <nyh@scylladb.com>
Message-Id: <20181217214610.4498-1-nyh@scylladb.com>
2018-12-25 12:37:11 +02:00
Tomasz Grabiec
07582d6c10 sstables: index_reader: Fix abort when _trust_pi == trust_promoted_index::no
data is not moved-from if _trust_pi == trust_promoted_index::no, which
triggers the assert on data.empty(). We should make it empty
unconditionally.

Message-Id: <1545408731-14333-1-git-send-email-tgrabiec@scylladb.com>
(cherry picked from commit 419c771791)
2018-12-24 11:45:14 +02:00
Tomasz Grabiec
18c89edbf7 sstables: mc: reader: Use enum class instead of variant
variant is an overkill here.

Message-Id: <1545409014-16289-1-git-send-email-tgrabiec@scylladb.com>
(cherry picked from commit 07d153c769)
2018-12-24 11:45:09 +02:00
Duarte Nunes
5558fa8c44 service/storage_proxy: Protect against empty mutation when storing hint
mutation_holder::get_mutation_for() can return nullptr's, so protect
against those when storing a hint.

Signed-off-by: Duarte Nunes <duarte@scylladb.com>
Message-Id: <20181221194853.98775-2-duarte@scylladb.com>
(cherry picked from commit e6a8883228)
2018-12-23 12:27:27 +02:00
Duarte Nunes
f678eb52cd service/storage_proxy: Protect against empty mutation in mutation_holder
The per_destination_mutation holder can contain empty mutations,
so make sure release_mutation() skips over those.

Signed-off-by: Duarte Nunes <duarte@scylladb.com>
Message-Id: <20181221194853.98775-1-duarte@scylladb.com>
(cherry picked from commit 6c4a34f378)
2018-12-23 12:27:25 +02:00
Tomasz Grabiec
dfb23f4b38 sstables: mc: index_reader: Handle CK_SIZE split across buffers properly
we incorrectly falled-through to the next state instead of returning
to read more data.

This can manifest in a number of ways, an abort, or incorrect read.

Introduced in 917528c

Fixes #4011.

Message-Id: <1545402032-4114-1-git-send-email-tgrabiec@scylladb.com>
(cherry picked from commit d2f96a60f6)
2018-12-21 20:40:35 +02:00
Tomasz Grabiec
502ddf158a sstables: mc: reader: Avoid unnecessary index reads on fast forwarding
When the next pending fragments are after the start of the new range,
we know there is no need to skip.

Caught by perf_fast_forward --datasets large-part-ds3 \
                            --run-tests=large-partition-slicing

Refs #3984
Message-Id: <1545308006-16389-1-git-send-email-tgrabiec@scylladb.com>

(cherry picked from commit 7afe2bad51)
2018-12-21 20:40:35 +02:00
Paweł Dziepak
0ccb0a127a Merge "Optimize slicing sstable readers" from Tomasz
"
Contains several improvements for fast-forwarding and slicing readers. Mainly
for the MC format, but not only:

  - Exiting the parser early when going out of the fast-forwarding window [MC-format-only]
  - Avoiding reading of the head of the partition when slicing
  - Avoiding parsing rows which are going to be skipped [MC-format-only]
"

* 'sstable-mc-optimize-slicing-reads' of github.com:tgrabiec/scylla:
  sstables: mc: reader: Skip ignored rows before parsing them
  sstables: mc: reader: Call _cells.clear() when row ends rather than when it starts
  sstables: mc: mutation_fragment_filter: Take position_in_partition rather than a clustering_row
  sstables: mc: reader: Do not call consume_row_marker_and_tombstone() for static rows
  sstables: mc: parser: Allow the consumer to skip the whole row
  sstables: continuous_data_consumer: Introduce skip()
  sstables: continuous_data_consumer: Make position() meaningful inside state_processor::process_state()
  sstables: mc: parser: Allocate dynamic_bitset once per read instead of once per row
  sstables: reader: Do not read the head of the partition when index can be used
  sstables: mc: mutation_fragment_filter: Check the fast-forward window first
  sstables: mc: writer: Avoid calling unsigned_vint::serialized_size()

(cherry picked from commit e6d26a528f)
2018-12-21 20:40:35 +02:00
Avi Kivity
b94997be0d Merge " Extract MC sstable writer to a separate compilation unit" from Tomasz
"
The motivation is to keep code related to each format separate, to make it
easier to comprehend and reduce incremental compilation times.

Also reduces dependency on sstable writer code by removing writer bits from
sstales.hh.

The ka/la format writers are still left in sstables.cc, they could be also extracted.
"

* 'extract-sstable-writer-code' of github.com:tgrabiec/scylla:
  sstables: Make variadic write() not picked on substitution error
  sstables: Extract MC format writer to mc/writer.cc
  sstables: Extract maybe_add_summary_entry() out of components_writer
  sstables: Publish functions used by writers in writer.hh
  sstables: Move common write functions to writer.hh
  sstables: Extract sstable_writer_impl to a header
  sstables: Do not include writer.hh from sstables.hh
  sstables: mc: Extract bound_kind_m related stuff into mc/types.hh
  sstables: types: Extract sstable_enabled_features::all()
  sstables: Move components_writer to .cc
  tests: sstable_datafile_test: Avoid dependency on components_writer

(cherry picked from commit b023e8b45d)
2018-12-21 20:40:35 +02:00
Benny Halevy
d3a5b10cb8 sstables_stats: writer_impl: move common members to base class
To be used by sstable_writer for stats collection.

Note that this patch is factored out so it can be verified with no
other change in functionality.

Signed-off-by: Benny Halevy <bhalevy@scylladb.com>
(cherry picked from commit 6853c1677d)
2018-12-21 20:40:35 +02:00
Benny Halevy
48f3f899ac sstable: make write_crc, write_digest, and new_sstable_component_file private methods
Prepare for per-sstable sub directory.
Also, these functions get most of their parameters from the sst at hand so they might
as well be first class members.

Signed-off-by: Benny Halevy <bhalevy@scylladb.com>
(cherry picked from commit ad5f1e4fbb)
2018-12-21 20:40:35 +02:00
Paweł Dziepak
c4f745276c Merge "Optimize sstable writing of large partitions" from Tomasz
"
This series contains several optimizations of the MC format sstable writer, mainly:
  - Avoiding output_stream when serializing into memory (e.g. a row)
  - Faster serialization of primitive types when serializing into memory

I measured the improvement in throughput (frag/s) using perf_fast_forward for
datasets with a single large partition with many small rows:

  - 10% for a row with a single cell of 8 bytes
  - 10% for a row with a single cell of 100 bytes
  -  9% for a row with a single cell of 1000 bytes
  - 13% for a row with 6 cells of 100 bytes
"

* tag 'avoid-output-stream-in-sstable-writer-v2' of github.com:tgrabiec/scylla:
  bytes_ostream: Optimize writing of fixed-size types
  sstables: mc: Write temporary data to bytes_ostream rather than file_writer
  sstables: mc: Avoid double-serialization of a range tombstone marker
  sstables: file_writer: Generalize bytes& writer to accept bytes_view
  sstables: Templetize write() functions on the writer
  sstables: Turn m_format_write_helpers.cc into an impl header
  sstables: De-futurize file_writer
  bytes_ostream: Implement clear()
  bytes_ostream: Make initial chunk size configurable

(cherry picked from commit e3f53542c9)
2018-12-21 20:40:35 +02:00
Hagit Segev
392c7dee3c release: prepare for 3.0-rc3 2018-12-21 20:19:50 +02:00
Gleb Natapov
04e982f909 streaming: hold to sink while close() is running and call close on error as well
Currently if something throws while streaming in mutation sending loop
sink is not closed. Also when close() is running the code does not hold
onto sink object. close() is async, so sink should be kept alive until
it completes. The patch uses do_with() to hold onto sink while close is
running and run close() on error path too.

Fixes #4004.

Message-Id: <20181220155931.GL3075@scylladb.com>
(cherry picked from commit 393269d34b)
2018-12-20 19:50:47 +02:00
Amnon Heiman
97a8cc149e node_exporter_install: switch to node_exporter 0.17
The newer version of node_exporter comes with important bug fixes, that
is especially important for I3.metal is not supported with the older
version of node_exporter.

The dashboards can now support both the new and the old version of
node_exporter.

Fixes #3927

Signed-off-by: Amnon Heiman <amnon@scylladb.com>
Message-Id: <20181210085251.23312-1-amnon@scylladb.com>
(cherry picked from commit 09c2b8b48a)
2018-12-20 19:12:00 +02:00
Avi Kivity
dbe347811c Merge "materialized views: Apply backpressure from view replicas" from Duarte
"
As the amount of pending view updates increases we know that there’s a
mismatch between the rate at which the base receives writes and the
rate at which the view retires them. We react by applying backpressure
to decrease the rate of incoming base writes, allowing the slow view
replicas to catch up. We want to delay the client’s next writes to a
base replica and we use the base’s backlog of view updates to derive
this delay.

To validate this approach we tested a 3 node Scylla cluster on GCE,
using n1-standard-4 instances with NVMEs. A loader running on a
n1-standard-8 instance run cassandra-stress with 100 threads. With the
delay function d(x) set to 1s, we see no base write timeouts. With the
delay function as defined in the series, we see that backlogs stabilize
at some (arbitrary) point, as predicted, but this stabilization
co-exists with base write timeouts. However, the system overall behaves
better than the current version, with the 100 view update limit, and
also better than the version without such limit or any backpressure.

More work is necessary to further stabilize the system. Namely, we want
to keep delaying until we see the backlog is decreasing. This will
require us to add more delay beyond the stabilization point, which in
turn should minimize the base write timeouts, and will also minimize the
amount of memory the backlog takes at each base replica.

Design document:
    https://docs.google.com/document/d/1J6GeLBvN8_c3SbLVp8YsOXHcLc9nOLlRY7pC6MH3JWo

Fixes #2538
"

Reviewed-by: Nadav Har'El <nyh@scylladb.com>

* 'materialized-views/backpressure/v2' of https://github.com/duarten/scylla: (32 commits)
  service/storage_proxy: Release mutation as early as possible
  service/storage_proxy: Delay replica writes based on view update backlog
  service/storage_proxy: Get the backlog of a particular base replica
  service/storage_proxy: Add counters for delayed base writes
  main: Start and stop the view_update_backlog_broker
  service: Distribute a node's view update backlog
  service: Advertise view update backlog over gossip
  service/storage_proxy: Send view update backlog from replicas
  service/storage_proxy: Prepare to receive replica view update backlog
  service/storage_proxy: Expose local view update backlog
  tests/view_schema_test: Add simple test for db::view::node_update_backlog
  db/view: Introduce node_update_backlog class
  db/hints: Initialize current backlog
  database: Add counter for current view backlog
  database: Expose current memory view update backlog
  idl: Add db::view::update_backlog
  db/view: Add view_update_backlog
  database: Wait on view update semaphore for view building
  service/storage_proxy: Use near-infinite timeouts for view updates
  database: generate_and_propagate_view_updates no longer needs a timeout
  ...

(cherry picked from commit b66f59aa3d)
2018-12-20 19:11:56 +02:00
Avi Kivity
8f2d24bb8f config: remove "to be removed before release" notice mc sstable config
The "enable_sstables_mc_format" config item help text wants to remove itself
before release. Since scylla-3.0 did not get enough mc format mileage, we
decided to leave it in, so the notice should be removed.

Fixes #4003.
Message-Id: <20181219082554.23923-1-avi@scylladb.com>

(cherry picked from commit dd51c659f7)
2018-12-19 19:08:36 +02:00
Nadav Har'El
689e11c892 scylla.spec: add libatomic
In some cases which I've yet to understand, build fails without libatomic.
We need to add it to the mock build machine.

Signed-off-by: Nadav Har'El <nyh@scylladb.com>
Message-Id: <20181218154757.25236-1-nyh@scylladb.com>
2018-12-19 12:55:00 +00:00
Nadav Har'El
1766c793a8 build_rpm.sh: put temporary mock in build/, not /var/lib.
build_rpm.sh uses "mock" to build an entire Scylla build environment,
which easily spans more than 15 gigabytes. mock, by defaults, puts this
build directory in a subdirectory of /var/lib/mock. There is no reason
why temporary build products need to be in the root directory: Some machines
(like mine) don't have that much free space in the root directory making it
impossible to use this script on such machines. and it's too easy
to leave temporary files there without noticing.

With this patch, the mock directories are put in build/mock/ instead of
/var/lib/mock.

Signed-off-by: Nadav Har'El <nyh@scylladb.com>
Message-Id: <20181217195952.15154-1-nyh@scylladb.com>
2018-12-19 12:54:37 +02:00
Avi Kivity
0b09008cde Merge "Make sstable reader fail on unknown colum names in MC format" from Piotr
"
Before the reader was just ignoring such columns but this creates a risk of data loss.

Refs #2598
"

* 'haaawk/2598/v3' of github.com:scylladb/seastar-dev:
  sstables: Add test_sstable_reader_on_unknown_column
  sstables: Exception on sstable's column not present in schema
  sstables: store column name in column_translation::column_info
  sstables: Make test_dropped_column_handling test dropped columns

(cherry picked from commit b0cb69ec25)
2018-12-18 16:23:51 +00:00
Piotr Jastrzebski
713e60f690 sstables: Extract mp_row_cosumer_m::check_schema_mismatch
This method will contain common logic used in multiple places
and reduce code duplication.

Signed-off-by: Piotr Jastrzebski <piotr@scylladb.com>
Message-Id: <bbda2f4ea4f9325055f096dc549f63b1bb03d3b6.1543311990.git.piotr@scylladb.com>
(cherry picked from commit 4366302c4c)
2018-12-18 16:23:47 +00:00
Paweł Dziepak
7b6841f947 Merge "Check for schema mismatch after dropping dead cells" from Piotr
"
Previously we were checking for schema incompatibility between current schema and sstable
serialization header before reading any data. This isn't the best approach because
data in sstable may be already irrelevant due to column drop for example.

This patchset moves the check after actual data is read and verified that it has
a timestamp new enough to classify it as nonobsolete.

Fixes #3924
"

* 'haaawk/3924/v3' of github.com:scylladb/seastar-dev:
  sstables: Enable test_schema_change for MC format
  sstables3: Throw error on schema mismatch only for live cells
  sstables: Pass column_info to consume_*_column
  sstables: Add schema_mismatch to column_info
  sstables: Store column data type in column_info
  sstables: Remove code duplication in column_translation

(cherry picked from commit 62ea153629)
2018-12-18 15:27:53 +00:00
Tomasz Grabiec
f124b7026f Merge 'Add tests for schema changes' from Paweł
This series adds a generic test for schema changes that generates
various schema and data before and after an ALTER TABLE operation. It is
then used to check correctness of mutation::upgrade() and sstable
readers and lead to the discovery of #3924 and #3925.

Fixes #3925.

* https://github.com/pdziepak/scylla.git schema-change-test/v3.1
  schema_builder: make member function names less confusing
  converting_mutation_partition_applier: fix collection type changes
  converting_mutation_partition_applier: do not emit empty collections
  sstable: use format() instead of sprint()
  tests/random-utils: make functions and variables inline
  tests: add models for schemas and data
  tests: generate schema changes
  tests/mutation: add test for schema changes
  tests/sstable: add test for schema changes

(cherry picked from commit 564b328b2e)
2018-12-18 14:57:50 +00:00
Avi Kivity
28cca751d1 Merge "Don't binary compare compressed sstables in test_write_many_partitions_* tests" from Piotr
"
Compression is not deterministic so instead of binary comparing the sstable files we just read data back
and make sure everything that was written down is still present.

Tests: unit(release)
"

* 'haaawk/binary-compare-of-compressed-sstables/v3' of github.com:scylladb/seastar-dev:
  sstables: Remove compressed parameter from get_write_test_path
  sstables: Remove unused sstable test files
  sstables: Ensure compare_sstables isn't used for compressed files
  sstables: Don't binary compare compressed sstables
  sstables: Remove debug printout from test_write_many_partitions

(cherry picked from commit 1ff6b8fb96)
2018-12-18 14:53:52 +00:00
Duarte Nunes
21d08aa41e Merge 'Fix evictable shard reader related issues' from Botond
"
Recently some additional issues were discovered related to recent
changes to the way inactive readers are evicted and making shard readers
evictable.
One such issue is that the `querier_cache` is not prepared for the
querier to be immediately evicted by the reader concurrency semaphore,
when registered with it as an inactive read (#3987).
The other issue is that the multishard mutation query code was not
fully prepared for evicted shard readers being re-created, or failing
why being re-created (#3991).

This series fixes both of these issues and adds a unit test which covers
the second one. I am working on a unit test which would cover the second
issue, but it's proving to be a difficult one and I don't want to delay
the fixes for these issues any longer as they also affect 3.0.

Fixes: #3987
Fixes: #3991
"

* 'evictable-reader-related-issues/branch-3.0/v1' of https://github.com/denesb/scylla:
  multishard_mutation_query: reset failed readers to inexistent state
  multishard_mutation_query: handle missing readers when dismantling
  multishard_mutation_query: add support for keeping stats for discarded partitions
  multishard_mutation_query: expect evicted reader state when creating reader
  multishard_mutation_query: pretty-print the reader state in log messages
  querier_cache: check that the query wasn't evicted during registering
  reader_concurrency_semaphore: use the correct types in the constructor
  reader_concurrency_semaphore: add consume_resources()
  reader_concurrency_semaphore::inactive_read_handle: add operator bool()
2018-12-18 13:36:42 +00:00
Botond Dénes
f0b5170fa6 multishard_mutation_query: reset failed readers to inexistent state
When attempting to dismantling readers, some of the to-be-dismantled
readers might be in a failed state. The code waiting on the reader to
stop is expecting failures, however it didn't do anything besides
logging the failure and bumping a counter. Code in the lower layers did
not know how to deal with a failed reader and would trigger
`std::bad_variant_access` when trying to process (save or cleanup) it.
To prevent this, reset the state of failed readers to `inexistent_state`
so code in the lower layers doesn't attempt to further process them.

(cherry picked from commit b4c3aab4a7)
2018-12-18 14:46:56 +02:00
Botond Dénes
3b617e873c multishard_mutation_query: handle missing readers when dismantling
When dismantling the combined buffer and the compaction state we are no
longer guaranteed to have the reader each partition originated from. The
reader might have been evicted and not resumed, or resuming it might
have failed. In any case we can no longer assume the originating reader
of each partition will be present. If a reader no longer exists,
discard the partitions that it emitted.

(cherry picked from commit 9cef043841)
2018-12-18 14:46:56 +02:00
Botond Dénes
4eb9836e64 multishard_mutation_query: add support for keeping stats for discarded partitions
In the next patches we will add code that will have to discard some of
the dismantled partitions/fragments/bytes. Prepare the
`dismantle_buffer_stats` struct for being able to track the discarded
partitions/fragments/bytes in addition to those that were successfully
dismantled.

(cherry picked from commit 438bef333b)
2018-12-18 14:46:56 +02:00
Botond Dénes
46af353209 multishard_mutation_query: expect evicted reader state when creating reader
Previously readers were created once, so `make_remote_reader()` had a
validation to ensure readers were not attempted at being created more
than once. This validation was done by checking that the reader-state is
either `inexistent` or `successful_lookup`. However with the
introduction of pausing shard readers, it is now possible that a reader
will have to be created and then re-created several times, however this
validation was not updated to expect this.
Update the validation so it also expects the reader-state to be
`evicted`, the state the reader will be if it was evicted while paused.

(cherry picked from commit ce52436af4)
2018-12-18 14:46:54 +02:00
Botond Dénes
76f70c676e multishard_mutation_query: pretty-print the reader state in log messages
(cherry picked from commit 1effb1995b)
2018-12-18 14:34:33 +02:00
Botond Dénes
afc9f0e177 querier_cache: check that the query wasn't evicted during registering
The reader concurrency semaphore can evict the querier when it is
registered as an inactive read. Make the `querier_cache` aware of this
so that it doesn't continue to process the inserted querier when this
happens.
Also add a unit test for this.

(cherry picked from commit 5780f2ce7a)
2018-12-18 14:34:33 +02:00
Botond Dénes
c899191ad5 reader_concurrency_semaphore: use the correct types in the constructor
Previously there was a type mismatch for `count` and `memory`, between
the actual type used to store them in the class (signed) and the type
of the parameters in the constructor (unsigned).
Although negative numbers are completely valid for these members,
initializing them to negative numbers don't make sense, this is why they
used unsigned types in the constructor. This restriction can backfire
however when someone intends to give these parameters the maximum
possible value, which, when interpreted as a signed value will be `-1`.
What's worse the caller might not even be aware of this unsigned->signed
conversion and be very suprised when they find out.
So to prevent surprises, expose the real type of these members, trusting
the clients of knowing what they are doing.

Also add a `no_limits` constructor, so clients don't have to make sure
they don't overflow internal types.

(cherry picked from commit e1d8237e6b)
2018-12-18 14:34:33 +02:00
Botond Dénes
a3563e5f7d reader_concurrency_semaphore: add consume_resources()
(cherry picked from commit dfd649a6b4)
2018-12-18 14:34:33 +02:00
Botond Dénes
78c5b09694 reader_concurrency_semaphore::inactive_read_handle: add operator bool()
(cherry picked from commit 21b44adbfe)
2018-12-18 14:34:33 +02:00
Amnon Heiman
a51878205a node-exporter.service: Update command line to fix service startup
The upgrade to node_exporter 0.17 commit
09c2b8b48a ("node_exporter_install: switch
to node_exporter 0.17") caused the service to no longer start. Turns out
node_exported broke backwards compatibility of the command line between
0.15 to 0.16. Fix it up.

While fixing the command line, all the collector that are enabled by
default were removed.

Fixes #3989

Signed-off-by: Amnon Heiman <amnon@scylladb.com>
[ penberg@scylladb.com: edit commit message ]
Message-Id: <20181213114831.27216-1-amnon@scylladb.com>
(cherry picked from commit 571755e117)
2018-12-18 08:33:04 +02:00
Avi Kivity
46efc08882 Update seastar submodule
* seastar 6700dc3...08f1258 (1):
  > reactor: disable nowait aio due to a kernel bug

Fixes #3996.
2018-12-17 17:00:14 +02:00
Avi Kivity
c95433c967 Update seastar submodule
* seastar 1651a2a...6700dc3 (3):
  > build: link against libatomic
  > core/semaphore: Allow combining semaphore_units()
  > core/shared_ptr: Allow releasing a lw_shared_ptr to a non-const object

Fixes #3996.
2018-12-17 15:53:01 +02:00
Piotr Sarna
df3b6fb4a8 cql3: refuse to create index on COMPACT STORAGE with ck
To follow C* compatibility, creating an index on COMPACT STORAGE
table should be disallowed not only on base primary keys,
but also when the base table contains clustering keys.
Message-Id: <ab40c39730aff2e164d11ee5159ff62b8ec9e8e8.1544698186.git.sarna@scylladb.com>

(cherry picked from commit 6743af5dbd)
2018-12-17 09:45:43 +02:00
Piotr Sarna
44ee43bb17 cql3: add refusing to create an index on static column
Secondary indexes on static columns are not yet supported,
so creating such index should return an appropriate error.

Fixes #3993
Message-Id: <700b0a71e80da52d2d5250edacc12626b55681fa.1544785127.git.sarna@scylladb.com>

(cherry picked from commit 63bd43e57e)
2018-12-17 09:44:52 +02:00
Asias He
aac363ca86 storage_service: Notify NEW_NODE only when a node is new node
This is a backport of CASSANDRA-11038.

Before this, a restarted node will be reported as new node with NEW_NODE
cql notification.

To fix, only send NEW_NODE notification when the node was not part of
the cluster

Fixes: #3979
Tests: pushed_notifications_test.py:TestPushedNotifications.restart_node_test
Message-Id: <453d750b98b5af510c4637db25b629f07dd90140.1544583244.git.asias@scylladb.com>
(cherry picked from commit 71c1681f6c)
2018-12-16 13:59:19 +02:00
Duarte Nunes
9e6cc5b024 service/storage_proxy: Embed the expire timer in the response handler
Embedding the expire timer for a write response in the
abstract_write_response_handler simplifies the code as it allows
removing the rh_entry type.

It will also make the timeout easily accessible inside the handler,
for future patches.

Signed-off-by: Duarte Nunes <duarte@scylladb.com>
Message-Id: <20181213111818.39983-1-duarte@scylladb.com>
(cherry picked from commit f8878238ed)
2018-12-13 13:24:09 +00:00
Duarte Nunes
13b72c7b92 Merge branch 'gossip: Send node UP event to cql client after cql server is up' from Asias
"
This is a backport of CASSANDRA-8236.

Before this patch, scylla sends the node UP event to cql client when it
sees a new node joins the cluster, i.e., when a new node's status
becomes NORMAL. The problem is, at this time, the cql server might not
be ready yet. Once the client receives the UP event, it tries to
connect to the new node's cql port and fails.

To fix, a new application_sate::RPC_READY is introduced, new node sets
RPC_READY to false when it starts gossip in the very beginning and sets
RPC_READY to true when the cql server is ready.

The RPC_READY is a bad name but I think it is better to follow Cassandra.

Nodes with or without this patch are supposed to work together with no
problem.

Refs #3843
"

* 'asias/node_up_down.upstream.v4.1' of github.com:scylladb/seastar-dev:
  storage_service: Use cql_ready facility
  storage_service: Handle application_state::RPC_READY
  storage_service: Add notify_cql_change
  storage_service: Add debug log in notify_joined
  storage_service: Add extra check in notify_joined
  storage_service: Add notify_joined
  storage_service: Add debug log in notify_up
  storage_service: Add extra check in notify_up
  storage_service: Add notify_up
  storage_service: Make notify_left log debug level
  storage_service: Introduce notify_left
  storage_service: Add debug log in notify_down
  storage_service: Introduce notify_down
  storage_service: Add set_cql_ready
  gossip: Add gossiper::is_cql_ready
  gms: Add endpoint_state::is_cql_ready
  gms: Add application_state::RPC_READY
  gms: Introduce cql_ready in versioned_value

(cherry picked from commit a42b2895c2)
2018-12-13 12:06:59 +00:00
Avi Kivity
6b011fbe0a build: pass C compiler configuration in dist package build
Just like we allow customizing the C++ compiler, we should allow customizing
the C compiler.

Ref #3978
Message-Id: <20181211172821.30830-1-avi@scylladb.com>

(cherry picked from commit fa96e07e6b)
2018-12-12 14:41:38 +02:00
Tomasz Grabiec
9dd4e1b01f sstables: index_reader: Avoid schema copy in advance_to()
Introduced in 7e15e43.

Exposed by perf_fast_forward:

  running: large-partition-skips on dataset large-part-ds1
  Testing scanning large partition with skips.
  Reads whole range interleaving reads with skips according to read-skip pattern:
  read    skip      time (s)     frags     frag/s (...)
  1       0         5.268780   8000000    1518378

  1       1        31.695985   4000000     126199
Message-Id: <1544614272-21970-1-git-send-email-tgrabiec@scylladb.com>

(cherry picked from commit 0a853b8866)
2018-12-12 14:38:49 +02:00
Nadav Har'El
e91c741ef5 secondary indexes: fail attempts to create a CUSTOM INDEX
Cassandra supports a "CREATE CUSTOM INDEX" to create a secondary index
with a custom implementation. The only custom implementation that Cassandra
supports is SASI. But Scylla doesn't support this, or any other custom
index implementation. If a CREATE CUSTOM INDEX statement is used, we
shouldn't silently ignore the "CUSTOM" tag, we should generate an error.

This patch also includes a regression test that "CREATE CUSTOM INDEX"
statements with valid syntax fail (before this patch, they succeeded).

Fixes #3977

Signed-off-by: Nadav Har'El <nyh@scylladb.com>
Message-Id: <20181211224545.18349-2-nyh@scylladb.com>
(cherry picked from commit a0379209e6)
2018-12-12 00:32:35 +00:00
Nadav Har'El
b18e9e115d Fix typo in error message
Interestingly, this typo was copied from the original Cassandra source
code :-)

Signed-off-by: Nadav Har'El <nyh@scylladb.com>
Message-Id: <20181211224545.18349-1-nyh@scylladb.com>
(cherry picked from commit 36db4fba23)
2018-12-12 00:32:35 +00:00
Avi Kivity
0b86ab0d2a build: build libdeflate with user selected C compiler
If the user specified a C compiler, use it to build libdeflate.

Fixes #3978.
Message-Id: <20181211145604.14847-1-avi@scylladb.com>

(cherry picked from commit 34a31a807d)
2018-12-11 19:24:24 +02:00
Duarte Nunes
97cd9108d6 db/system_distributed_keyspace: Create the schema with min_timestamp
Different nodes can concurrently create the distributed system
keyspace on boot, before the "if not exists" clause can take effect.

However, the resulting schema mutations will be different since
different nodes use different timestamps. This patch forces the
timestamps to be the same across all nodes, so we save some schema
mismatches.

This fixes a bug exposed by ca5dfdf, whereby the initialization of the
distributed system keyspace is done before waiting for schema
agreement. While waiting for schema agreement in
storage_service::join_token_ring(), the node still hasn't joined the
ring and schemas can't be pulled from it, so nodes can deadlock. A
similar situation can happen between a seed node and a non-seed node,
where the seed node progresses to a different "wait for schema
agreement" barrier, but still can't make progress because it can't
pull the schema from the non-seed node still trying to join the ring.

Finally, it is assumed that changes to the schema of the current
distributed system keyspace tables will be protected by a cluster
feature and a subsequent schema synchronization, such that all nodes
will be at a point where schemas can be transferred around.

Fixes #3976

Signed-off-by: Duarte Nunes <duarte@scylladb.com>
Message-Id: <20181211113407.20075-1-duarte@scylladb.com>
(cherry picked from commit 89ae3fbf11)
2018-12-11 14:53:30 +00:00
Hagit Segev
f81fe96b0b release: prepare for 3.0-rc2 2018-12-11 12:32:34 +02:00
Avi Kivity
91ce3a7957 sstables: fix overflow in clustering key blocks header bit access
_ck_blocks_header is a 64-bit variable, so the mask should be 64 bits too.
Otherwise, a shift in the range 32-63 will produce wrong results.

Fix by using a 64-bit mask.

Found by Fedora 29's ubsan.

Fixes #3973.
Message-Id: <20181209120549.21371-1-avi@scylladb.com>

(cherry picked from commit 7c7da0b462)
2018-12-10 14:10:27 +02:00
Takuya ASADA
af7e58f4c5 dist/offline_installer/redhat: fix missing dependencies
Offline installer with Scylla 3.0 causes dependency error on CentOS, added
missing packages.

Fixes #3969

Signed-off-by: Takuya ASADA <syuu@scylladb.com>
Message-Id: <20181207020711.23055-1-syuu@scylladb.com>
(cherry picked from commit a2d0ebf4d9)
2018-12-10 14:10:15 +02:00
Amos Kong
bd3373b511 scylla_setup: only ask for nic in interactive mode
Current scylla_setup still asks for nic even nic is already assigned in cmdline.

Fixes #3908

Signed-off-by: Amos Kong <amos@scylladb.com>
Message-Id: <6b867e17a5583c495c771a37d5fa1e8366b1d61b.1542337635.git.amos@scylladb.com>
(cherry picked from commit 09a3b11c2f)
2018-12-09 19:26:34 +02:00
Gleb Natapov
4820130abe storage_proxy: fix crash during write timeout callback invocation
rh_entry address is captured inside timeout's callback lambda, so the
structure should not be moved after it is created. Change the code to
create rh_entry in-place instead of moving it into the map.

Fixes #3972.

Message-Id: <20181206164043.GN25283@scylladb.com>
(cherry picked from commit 9fb79bf379)
2018-12-09 15:25:52 +02:00
Tomasz Grabiec
9b299241e5 Merge "Fixes for collecting stats in SST3 + more tests" from Vladimir
This patchset fixes several remaining issues found during thorough
testing of SSTables 3.x statistics and enriches ~30 unit tests with
statistics validation against Cassandra-generated golden copies.

* https://github.com/argenet/scylla/tree/projects/sstables-30/sst3-tests-statistics/v1:
  sstables: Enforce estimated_partitions in generate_summary() to be
    always positive.
  sstables: Don't enforce default max_local_deletion_time value for 'mc'
    files.
  sstables: Update TTL/local deletion stats for non-expiring and live
    liveness_info.
  sstables: Collect statistics when writing RT markers to SSTables 3.x.
  tests: Return sstable_assertions from validate_read() helper.
  tests: Introduce helper for validating stats metadata in SSTables 3.x
    tests.
  tests: Add stats metadata validation to test_write_static_row.
  tests: Add stats metadata validation to
    test_write_composite_partition_key.
  tests: Add stats metadata validation to
    test_write_composite_clustering_key.
  tests: Add stats metadata validation to test_write_wide_partitions.
  tests: Add stats metadata validation to write_ttled_row
  tests: Add stats metadata validation to write_ttled_column
  tests: Add stats metadata validation to write_deleted_column
  tests: Add stats metadata validation to write_deleted_row
  tests: Add stats metadata validation to write_collection_wide_update
  tests: Add stats metadata validation to
    write_collection_incremental_update
  tests: Add stats metadata validation to write_multiple_partitions
  tests: Add stats metadata validation to write_multiple_rows
  tests: Add stats metadata validation to
    write_missing_columns_large_set
  tests: Add stats metadata validation to write_different_types
  tests: Add stats metadata validation to write_empty_clustering_values
  tests: Add stats metadata validation to write_large_clustering_key
  tests: Add stats metadata validation to write_compact_table
  tests: Add stats metadata validation to write_user_defined_type_table
  tests: Add stats metadata validation to write_simple_range_tombstone
  tests: Add stats metadata validation to
    write_adjacent_range_tombstones
  tests: Add stats metadata validation to
    write_non_adjacent_range_tombstones
  tests: Add stats metadata validation to
    write_mixed_rows_and_range_tombstones
  tests: Add stats metadata validation to
    write_adjacent_range_tombstones_with_rows
  tests: Add stats metadata validation to
    write_range_tombstone_same_start_with_row
  tests: Add stats metadata validation to
    write_range_tombstone_same_end_with_row
  tests: Add stats metadata validation to
    write_two_non_adjacent_range_tombstones
  tests: Delete unused (bogus) Statistics.db file from write_ SST3
    tests.

(cherry picked from commit bb24d378b2)
2018-12-08 14:08:46 +02:00
Avi Kivity
745a98e151 Merge "Fix deadlocking multishard readers" from Botond
"
Multishard combining readers, running concurrently, with limited
concurrency and no timeout may deadlock, due to inactive shard readers
sitting on permits. To avoid this we have to make sure that all shard
readers belonging to a multishard combining readers, that are not
currently active, can be evicted to free up their permits, ensuring that
all readers can make progress.
Making inactive shard readers evictable is the solution for this
problem, however the original series introducing this solution
(414b14a6bd) did not go all they way and
left some loose ends. These loose ends are tied up by this mini-series.
Namely, two issues remained:
* The last reader to reach EOS was not paused (made evictable).
* Readers created/resumed as part of a read-ahead were not paused
  immediately after finishing the read-ahead.

This series fixes both of these.

Fixes: #3865
Tests: unit(release, debug)
"

* 'fix-multishard-reader-deadlock/v1' of https://github.com/denesb/scylla:
  multishard_combining_reader: pause readers after reading ahead
  multishard_combining_reader: pause *all* EOS'd readers

(cherry picked from commit 21b4b2b9a1)
2018-12-08 14:08:46 +02:00
Avi Kivity
b9c99af18b Merge "Fix tombstone histogram when writing SSTables 3.x" from Vladimir
"
This patchset extends a number of existing tests to check SSTables
statistics for 'mc' format and fixes an issue discovered with the help
of one of the tests.

Tests: unit {release}
"

* 'projects/sstables-30/check-stats/v2' of https://github.com/argenet/scylla:
  tests: Run sstable_timestamp_metadata_correcness_with_negative with all SSTables versions.
  tests: Run sstable_tombstone_histogram_test for all SSTables versions.
  tests: Run min_max_clustering_key_test on all SSTables versions.
  tests: Expand test_sstable_max_local_deletion_time_2 to run for all SSTables versions.
  tests: Run test_sstable_max_local_deletion_time on all SSTables versions.
  tests: Extend test checking tombstones histogram to cover all SSTables versions.
  sstables: Properly track row-level tombstones when writing SSTables 3.x.
  tests: Run min_max_clustering_key_test_2 for all SSTables versions.
  tests: Make reusable_sst() helper accept SSTables version parameter.

(cherry picked from commit f073ea5f87)
2018-12-08 14:08:44 +02:00
Asias He
cded9c7ac7 gossip: Fix race in real_mark_alive and shutdown msg
In dtest, we have

   self.check_rows_on_node(node1, 2000)
   self.check_rows_on_node(node2, 2000)

which introduce the following cluster operations:

1) Initially:

- node1 up
- node2 up

2) self.check_rows_on_node(node1, 2000)
- node2 down
- node2 up (A: node2 will call gossiper::real_mark_alive when node2 boots
up to mark node1 up)

3) self.check_rows_on_node(node2, 2000)
- node1 down (B: node1 will send shutdown gossip message to node2, node2
will mark node1 down)
- node1 up (C: when node1 is up, node2 will call
gossiper::real_mark_alive)

Since there is no guarantee the order of Operation A and Operation B, it
is possible node2 will mark node1 as status=shutdown and mark node1 is
UP.

In Operation C, node2 will call gossiper::real_mark_alive to mark node1
up, but since node2 might think node1 is already up, node2 will exit
early in gossiper::real_mark_alive and not log "InetAddress 127.0.0.1 is
now UP, status={}"

As a result, dtest fails to see node2 reports node1 is up when it boots
node1 and fail the test.

   TimeoutError: 23 Nov 2018 10:44:19 [node2] Missing: ['127.0.0.1.* now UP']

In the log we can see node1 marked as DOWN and UP almost at the same time on node2:

   INFO  2018-11-23 22:31:29,999 [shard 0] gossip - InetAddress 127.0.0.1 is now DOWN, status = shutdown
   INFO  2018-11-23 22:31:30,006 [shard 0] gossip - InetAddress 127.0.0.1 is now UP, status = shutdown

Fixes #3940

Tests: dtest with 20 consecutive succesful runs
Message-Id: <996dc325cbcc3f94fc0b7569217aa65464eaaa1c.1543213511.git.asias@scylladb.com>
(cherry picked from commit eeeb2da7bb)
2018-12-08 13:42:43 +02:00
Gleb Natapov
4acfc5ed8f hints: make hints manager more resilient to unexpected directory content
Currently if hints directory contains unexpected directories Scylla fails to
start with unhandled std::invalid_argument exception. Make the manager
ignore malformed files instead and try to proceed anyway.
Message-Id: <20181121134618.29936-2-gleb@scylladb.com>

(cherry picked from commit b4a8802edc)
2018-12-08 13:42:43 +02:00
Gleb Natapov
cb9199bc7f hints: add auxiliary function for scanning high level hints directory
We scan hints directory in two places: to search for files to replay and
to search for directories to remove after resharding. The code that
translates directory name to a shard is duplicated. It is simple now, so
not a bit issue but in case it grows better have it in one place.
Message-Id: <20181121134618.29936-1-gleb@scylladb.com>

(cherry picked from commit 9433d02624)
2018-12-08 13:42:43 +02:00
Tomasz Grabiec
695ff5383f Merge "Correct the usage of row ttl and add write-read test" from Piotr
Fixes the condition which determines whether a row ttl should be used for a cell
and adds a test that uses each generated mutation to populate mutation source
and then verifies that it can read back the same mutation.

* seastar-dev.git haaawk/sst3/write-read-test/v3:
  Fix use_row_ttl condition
  Add test_all_data_is_read_back

(cherry picked from commit b8c405c019)
2018-12-08 13:42:43 +02:00
Tomasz Grabiec
730e48bf60 configure.py: Always add a rule for building gen_crc_combine_table
Fixes a build failure when only the scylla binary was selected for
building like this:

  ./configure.py --with scylla

In this case the rule for gen_crc_combine_table was missing, but it is
needed to build crc_combine_table.o

Message-Id: <1544010138-21282-1-git-send-email-tgrabiec@scylladb.com>
(cherry picked from commit edbef7400b)
2018-12-08 13:42:43 +02:00
Tomasz Grabiec
af6d4f40e1 utils/gz: Fix compilation on non-x86 archs
gen_crc_combine_table is now executed on every build, so it should not
fail on unsupported archs. The generated file will not contain data,
but this is fine since it should not be used.

Another problem is that u32 and u64 aliases were not visible in the #else
branch in crc_combine.cc
Message-Id: <1543864425-5650-1-git-send-email-tgrabiec@scylladb.com>

(cherry picked from commit 9a4c00beb7)
2018-12-08 13:42:43 +02:00
Avi Kivity
9d8507de09 Merge "Optimize checksum_combine() for CRC32" from Tomek
"
zlib's crc32_combine() is not very efficient. It is faster to re-combine
the buffer using crc32(). It's still substantial amount of work which
could be avoided.

This patch introduces a fast implementation of crc32_combine() which
uses a different algorithm than zlib. It also utilizes intrinsics for
carry-less multiplication instruction to perform the computation faster.
The details of the algorithm can be found in code comments.

Performance results using perf_checksum and second buffer of length 64 KiB:

zlib CRC32 combine:   38'851   ns
libdeflate CRC32:      4'797   ns
fast_crc32_combine():     11   ns

So the new implementation is 3500x faster than zlib's, and 417x faster than
re-checksumming the buffer using libdeflate.

Tested on i7-5960X CPU @ 3.00GHz

Performance was also evaluated using sstable writer benchmark:

  perf_fast_forward --populate --sstable-format=mc --data-directory /tmp/perf-mc \
     --value-size=10000 --rows 1000000 --datasets small-part

It yielded 9% improvement in median frag/s (129'055 vs 117'977).

Refs #3874
"

* tag 'fast-crc32-combine-v2' of github.com:tgrabiec/scylla:
  tests: perf_checksum: Test fast_crc32_combine()
  tests: Rename libdeflate_test to checksum_utils_test
  tests: libdeflate: Add more tests for checksum_combine()
  tests: libdeflate: Check both libdeflate and default checksummers
  sstables: Use fast_crc_combine() in the default checksummer
  utils/gz: Add fast implementation of crc32_combine()
  utils/gz: Add pre-computed polynomials
  utils/gz: Import Barett reduction implementation from libdeflate
  utils: Extract clmul() from crc.hh

(cherry picked from commit b098b5b987)
2018-12-08 13:42:43 +02:00
Tomasz Grabiec
07c980845d utils/crc: Add clmul_u32() implementation
Needed for backporting dependent changes.

Extracted from:

      commit 79136e895f
      Author: Yibo Cai (Arm Technology China) <Yibo.Cai@arm.com>
      Date:   Thu Nov 1 03:26:16 2018 +0000

        utils/crc: calculate crc in parallel
2018-12-08 13:42:43 +02:00
Tomasz Grabiec
c52b8239d0 configure.py: Compile against Westmere on x86
Needed for backporting dependent changes.

Extracted from:

  commit 79136e895f
  Author: Yibo Cai (Arm Technology China) <Yibo.Cai@arm.com>
  Date:   Thu Nov 1 03:26:16 2018 +0000

    utils/crc: calculate crc in parallel
2018-12-08 13:42:43 +02:00
Tomasz Grabiec
5a07a4fac8 configure.py: Use armv8-a+crc+crypto ISA on aarch64
Needed for backporting dependent changes.

Extracted from:

  commit 1c48e3fbec
  Author: Yibo Cai (Arm Technology China) <Yibo.Cai@arm.com>
  Date:   Mon Oct 29 02:58:19 2018 +0000

    utils/crc: leverage arm64 crc extension
2018-12-08 13:42:43 +02:00
Avi Kivity
b9c046b17b Merge "Optimize checksum computation for the MC sstable format" from Tomek
"
One part of the improvement comes from replacing zlib's CRC32 with the one
from libdeflate, which is optimized for modern architecture and utilizes the
PCLMUL instruction.

perf_checksum test was introduced to measure performance of various
checksumming operations.

Results for 514 B (relevant for writing with compression enabled):

    test                                      iterations      median         mad         min         max
    crc_test.perf_deflate_crc32_combine            58414    16.711us     3.483ns    16.708us    16.725us
    crc_test.perf_adler_combine                165788278     6.059ns     0.031ns     6.027ns     7.519ns
    crc_test.perf_zlib_crc32_combine               59546    16.767us    26.191ns    16.741us    16.801us
    ---
    crc_test.perf_deflate_crc32_checksum        12705072    83.267ns     4.580ns    78.687ns    98.964ns
    crc_test.perf_adler_checksum                 3918014   206.701ns    23.469ns   183.231ns   258.859ns
    crc_test.perf_zlib_crc32_checksum            2329682   428.787ns     0.085ns   428.702ns   510.085ns

Results for 64 KB (relevant for writing with compression disabled):

    test                                      iterations      median         mad         min         max
    crc_test.perf_deflate_crc32_combine            25364    38.393us    17.683ns    38.375us    38.545us
    crc_test.perf_adler_combine                169797143     5.842ns     0.009ns     5.833ns     6.901ns
    crc_test.perf_zlib_crc32_combine               26067    38.663us    95.094ns    38.546us    40.523us
    ---
    crc_test.perf_deflate_crc32_checksum          202821     4.937us    14.426ns     4.912us     5.093us
    crc_test.perf_adler_checksum                   44684    22.733us   206.263ns    22.492us    25.258us
    crc_test.perf_zlib_crc32_checksum              18839    53.049us    36.117ns    53.013us    53.274us

The new CRC32 implementation (deflate_crc32) doesn't provide a fast
checksum_combine() yet, it delegates to zlib so it's as slow as the latter.

Because for CRC32 checksum_combine() is several orders of magnitude slower
than checksum(), we avoid calling checksum_combine() completely for this
checksummer. We still do it for adler32, which has combine() which is faster
than checksum().

SStable write performance was evaluated by running:

  perf_fast_forward --populate --data-directory /tmp/perf-mc \
     --rows=10000000 -c1 -m4G --datasets small-part

Below is a summary of the average frag/s for a memtable flush. Each result is
an average of about 20 flushes with stddev of about 4k.

Before:

 [1] MC,lz4: 330'903
 [2] LA,lz4: 450'157
 [3] MC,checksum: 419'716
 [4] LA,checksum: 459'559

After:

 [1'] MC,lz4: 446'917 ([1] + 35%)
 [2'] LA,lz4: 456'046 ([2] + 1.3%)
 [3'] MC,checksum: 462'894 ([3] + 10%)
 [4'] LA,checksum: 467'508 ([4] + 1.7%)

After this series, the performance of the MC format writer is similar to that
of the LA format before the series.

There seems to be a small but consistent improvement for LA too. I'm not sure
why.
"

* tag 'improve-mc-sstable-checksum-libdeflate-v3' of github.com:tgrabiec/scylla:
  tests: perf: Introduce perf_checksum
  tests: Add test for libdeflate CRC32 implementation
  sstables: compress: Use libdeflate for crc32
  sstables: compress: Rename crc32_utils to zlib_crc32_checksummer
  licenses: Add libdeflate license
  Integrate libdeflate with the build system
  Add libdeflate submodule
  sstables: Avoid checksum_combine() for the crc32 checksummer
  sstables: compress: Avoid unnecessary checksum_combine()
  sstables: checksum_utils: Add missing include

(cherry picked from commit 5e759b0c07)
2018-12-08 13:42:43 +02:00
Avi Kivity
979cb636b8 Update seastar submodule
* seastar e64281d...1651a2a (1):
  > tests: perf: Make do_not_optimize() take the argument by const&
2018-12-08 13:42:43 +02:00
Botond Dénes
59cf9d9070 querier: fix evict_one() and evict_all_for_table()
Both of these have the same problem. They remove the to-be-evicted
entries from `_entries` but they don't unregister the `entry` from the
`read_concurrency_semaphore`. This results in the
`reader_concurrency_semaphore` being left with a dangling pointer to the
entries will trigger segfault when it tries to evict the associated
inactive reads.

Also add a unit test for `evict_all_for_table()` to check that it works
properly (`evict_one()` is only used in tests, so no dedicated test for
it).

Fixes: #3962

Signed-off-by: Botond Dénes <bdenes@scylladb.com>
Message-Id: <57001857e3791c6385721b624d33b667ccda2e7d.1544010868.git.bdenes@scylladb.com>
(cherry picked from commit 77dbc7d09a)
2018-12-06 11:38:44 +02:00
Duarte Nunes
c9ec9d4087 Merge seastar upstream
* seastar 880826e...e64281d (2):
  > core/semaphore: Change the access of semaphore_units main ctor
  > Merge "Add semaphore_units<>::split() function" from Duarte

Signed-off-by: Duarte Nunes <duarte@scylladb.com>
2018-12-05 20:25:17 +00:00
Gleb Natapov
2e8fefbc5a storage_proxy: store hint for CL=ANY if all nodes replied with failure
Current code assumes that request failed if all replicas replied with
failure, but this is not true for CL=ANY requests. Take it into account.

Fixed: #3565
(cherry picked from commit 17197fb005)
2018-12-05 20:14:58 +00:00
Gleb Natapov
6be0635029 storage_proxy: complete write request early if all replicas replied with success of failure
Currently if write request reaches CL and all replicas replied, but some
replied with failures, the request will wait for timeout to be retired.
Detect this case and retire request immediately instead.

Fixes #3566

(cherry picked from commit d1d04eae3c)
2018-12-05 20:14:57 +00:00
Gleb Natapov
04a544c0a2 storage_proxy: check that write failure response comes from recognized replica
Before accounting failure response we need to make sure it comes from a
replica that participates in the request.

(cherry picked from commit 76ab3d716b)
2018-12-05 20:14:57 +00:00
Gleb Natapov
028f9b95d1 storage_proxy: move code executed on write timeout into separate function
Currently the callback is in lambda, but we will want to call the code
not only during timer expiration.

(cherry picked from commit 7bc68aa0eb)
2018-12-05 20:14:57 +00:00
Avi Kivity
54258ca8eb Merge "db/hints: Use frozen_mutation in hinted handoff" from Duarte
"
This series changes hinted handoff to work with `frozen_mutation`s
instead of naked `mutation`s. Instead of unfreezing a mutation from
the commitlog entry and then freezing it again for sending, now we'll
just keep the read, frozen mutation.

Tests: unit(release)
"

* 'hh-manager-cleanup/v1' of https://github.com/duarten/scylla:
  db/hints/manager: Use frozen_mutation instead of mutation
  db/hints/manager: Use database::find_schema()
  db/commitlog/commitlog_entry: Allow moving the contained mutation
  service/storage_proxy: send_to_endpoint overload accepting frozen_mutation
  service/storage_proxy: Build a shared_mutation from a frozen_mutation
  service/storage_proxy: Lift frozen_mutation_and_schema
  service/storage_proxy: Allow non-const ranges in mutate_prepare()

(cherry picked from commit 1891779e64)
2018-12-05 20:14:57 +00:00
Gleb Natapov
c9a030f1f0 storage_proxy: count number of timed out write attempts after CL is reached
It is useful to have this counter to investigate the reason for read
repairs. Non zero value means that writes were lost after CL is reached
and RR is expected.

Message-Id: <20181009120900.GF22665@scylladb.com>
(cherry picked from commit 207b57a892)
2018-12-05 20:14:57 +00:00
Gleb Natapov
1c7daef554 storage_proxy: do not pass write_stats down to send_to_live_endpoints
write_stats is referenced from write handler which is available in
send_to_live_endpoints already. No need to pass it down.

Message-Id: <20181009133017.GA14449@scylladb.com>
(cherry picked from commit 319ece8180)
2018-12-05 20:14:57 +00:00
Duarte Nunes
f8195a77b0 db/view/view_builder: Don't timeout waiting for view to be built
Remove the timeout argument to
db::view::view_builder::wait_until_built(), a test-only function to
wait until a given materialized view has finished building.

This change is motivated by the fact that some tests running on slow
environments will timeout. Instead of incrementally increasing the
timeout, remove it completely since tests are already run under an
exterior timeout.

Fixes #3920

Tests: unit release(view_build_test, view_schema_test)

Signed-off-by: Duarte Nunes <duarte@scylladb.com>
Message-Id: <20181115173902.19048-1-duarte@scylladb.com>
(cherry picked from commit 6fbf792777)
2018-12-05 19:20:36 +00:00
Duarte Nunes
5b724c80ab db/view: Don't copy keyspace name
Signed-off-by: Duarte Nunes <duarte@scylladb.com>
Message-Id: <20181022104527.14555-1-duarte@scylladb.com>
(cherry picked from commit f3a5ec0fd9)
2018-12-05 19:19:26 +00:00
Nadav Har'El
4a7ae81b3f materialized views: update stats.write statistics in all cases
mutate_MV usually calls send_to_endpoint() to push view update to remote
view replicas. This function gets passed a statistics object,
service::storage_proxy_stats::write_stats and, in particular, updates
its "writes" statistic which counts the number of ongoing writes.

In the case that the paired view replica happens to be the *same* node,
we avoid calling send_to_endpoint() and call mutate_locally() instead.
That function does not take a write_stats object, so the "writes" statistic
doesn't get incremented for the duration of the write. So we should do
this explicitly.

Co-authored-by: Nadav Har'El <nyh@scylladb.com>
Co-authored-by: Duarte Nunes <duarte@scylladb.com>
(cherry picked from commit 1d5f8d0015)
2018-12-05 19:19:26 +00:00
Piotr Sarna
3cf26a60a2 auth: add abort_source to waiting for schema agreement
When the auth service is requested to stop during bootstrap,
it might have still not reached schema agreement.
Currently, waiting for this agreement is done in an infinite loop,
without taking abort_source into account.
This patch introduces checking if abort was requested
and breaking the loop in such case, so auth service can terminate.

Tests:
unit (release)
dtest (bootstrap_test.py:TestBootstrap.shutdown_wiped_node_cannot_join_test)
Message-Id: <1b7ded14b7c42254f02b5d2e10791eb767aae7fc.1543914769.git.sarna@scylladb.com>

(cherry picked from commit 7b0a3fbf8a)
2018-12-04 14:33:05 +00:00
Tomasz Grabiec
2103d0d52b sstables: Write Statistics.db offset map entries in the same order as Cassandra
Before this patch we were writing offset map enteies in unspecified
order, the one returned by std::unorderd_map. Cassandra writes them
sorted by metadata_type. Use the same order for improved
compatibility.

Fixes #3955.

Message-Id: <1543846649-22861-1-git-send-email-tgrabiec@scylladb.com>
(cherry picked from commit aa19f98d18)
2018-12-04 14:30:19 +02:00
Avi Kivity
16ee3b3ebe Merge "Make inactive shard readers evictable" from Botond
"
This series attempts to solve the regressions recently discovered in
performance of multi-partition range-scans. Namely that they:
* Flood the reader concurrency semaphore's queues, trampling other
  reads.
* Behave very badly when too many of them is running concurrently
  (trashing).
* May deadlock if enough of them is running without a timeout.

The solution for these problems is to make inactive shard readers
evictable. This should address all three issues listed above, to varying
degrees:
* Shard readers will now not cling onto their permits for the entire
  duration of the scan, which might be a lot of time.
* Will be less affected by infinite concurrency (more than the node can
  handle) as each scan now can make progress by evicting inactive shard
  readers belonging to other scans.
* Will not deadlock at all.

In addition to the above fix, this series also bundles two further
improvements:
* Add a mechanism to `reader_concurrecy_semaphore` to be notified of
  newly inserted evictables.
* General cleanups and fixes for `multishard_combining_reader` and
  `foreign_reader`.

I can unbundle these mini series and send them separately, if the
maintainers so prefer, altough considering that this series will have to
be backported to 3.0, I think this present form is better.

Fixes: #3835
"

* 'evictable-inactive-shard-readers/v7' of https://github.com/denesb/scylla: (27 commits)
  tests/multishard_mutation_query_test: test stateless query too
  tests/querier_cache: fail resource-based eviction test gracefully
  tests/querier_cache: simplify resource-based eviction test
  tests/mutation_reader_test: add test_multishard_combining_reader_next_partition
  tests/mutation_reader_test: restore indentation
  tests/mutation_reader_test: enrich pause-related multishard reader test
  multishard_combining_reader: use pause-resume API
  query::partition_slice: add clear_ranges() method
  position_in_partition: add region() accessor
  foreign_reader: add pause-resume API
  tests/mutation_reader_test: implement the pause-resume API
  query_mutations_on_all_shards(): implement pause-resume API
  make_multishard_streaming_reader(): implement the pause-resume API
  database: add accessors for user and streaming concurrency semaphores
  reader_lifecycle_policy: extend with a pause-resume API
  query_mutations_on_all_shards(): restore indentation
  query_mutations_on_all_shards(): simplify the state-machine
  multishard_combining_reader: use the reader lifecycle policy
  multishard_combining_reader: add reader lifecycle policy
  multishard_combining_reader: drop unnecessary `reader_promise` member
  ...

(cherry picked from commit 414b14a6bd)
2018-12-04 12:13:13 +02:00
Duarte Nunes
b0a9c40ab1 service/storage_proxy: Consider target liveness in sent_to_endpoint()
So we don't attempt to send mutations to unreachable endpoints and
instead store a hint for them, we now check the endpoint status and
populate dead_endpoints accordingly in
storage_proxy::send_to_endpoint().

Fixes #3820

Signed-off-by: Duarte Nunes <duarte@scylladb.com>
Message-Id: <20181007100640.2182-1-duarte@scylladb.com>
(cherry picked from commit 30d6ed8f92)
2018-12-03 18:38:05 +00:00
Duarte Nunes
53924e5c7f service/storage_proxy: Fix formatting of send_to_endpoint()
Signed-off-by: Duarte Nunes <duarte@scylladb.com>
Message-Id: <20181006204756.32232-1-duarte@scylladb.com>
(cherry picked from commit a69d468101)
2018-12-03 18:37:59 +00:00
Avi Kivity
befe0012f5 Merge "Fix multiple summary regeneration bugs." from Vladimir
"
This patchset addresses two recently discovered bugs both triggered by
summary regeneration:

Tests: unit {release}

+

Validated with debug build of Scylla (ASAN) that no use-after-free
occurs when re-generating Summary.db.
"

* 'projects/sstables-30/summary-regeneration/v1' of https://github.com/argenet/scylla:
  tests: Add test reading SSTables in 'mc' format with missing summary.
  sstables: When loading, read statistics before summary.
  database: Capture io_priority_class by reference to avoid dangling ref.

(cherry picked from commit 009cbd3dcb)
2018-12-02 13:32:09 +02:00
Duarte Nunes
1953c5fa61 Merge 'Fix filtering with LIMIT' from Piotr
"
This series adds proper handling of filtering queries with LIMIT.
Previously the limit was erroneously applied before filtering,
which leads to truncated results.

To avoid that, paged filtering queries now use an enhanced pager,
which remembers how many rows dropped and uses that information
to fetch for more pages if the limit is not yet reached.

For unpaged filtering queries, paging is done internally as in case
of aggregations to avoid returning keeping huge results in memory.

Also, previously, all limited queries used the page size counted
from max(page size, limit). It's not good for filtering,
because with LIMIT 1 we would then query for rows one-by-one.
To avoid that, filtered queries ask for the whole page and the results
are truncated if need be afterwards.

Tests: unit (release)
"

* 'fix_filtering_with_limit_2' of https://github.com/psarna/scylla:
  tests: add filtering with LIMIT test
  tests: split filtering tests from cql_query_test
  cql3: add proper handling of filtering with LIMIT
  service/pager: use dropped_rows to adjust how many rows to read
  service/pager: virtualize max_rows_to_fetch function
  cql3: add counting dropped rows in filtering pager

(cherry picked from commit 1afda28cf3)
2018-12-02 12:07:46 +02:00
Duarte Nunes
b72a94b53e Merge 'Fix checking if system tables need view updates' from Piotr
"
This miniseries ensures that system tables are not checked
for having view updates, because they never do.
What's more, distributed system table is used in the process,
so it's unsafe to query the table while streaming it.

Tests: unit (release), dtest(update_cluster_layout_tests.py:TestUpdateClusterLayout.simple_decommission_node_2_test)
"

* 'fix_checking_if_system_tables_need_view_updates_3' of https://github.com/psarna/scylla:
  streaming: don't check view building of system tables
  database: add is_internal_keyspace
  streaming: remove unused sstable_is_staging bool class

(cherry picked from commit d09d4bbd91)
2018-11-28 15:39:34 +00:00
Piotr Sarna
3f82b697f2 main: fix deinitialization order for view update generator
View update generator should be stopped only after
drain_on_shutdown() is performed on storage service.
Message-Id: <4d2bda4c73422a2ebf46d6dcd06c95d960839889.1543230849.git.sarna@scylladb.com>

(cherry picked from commit 6ab8235369)
2018-11-27 12:34:50 +00:00
Takuya ASADA
ee1ef853e5 dist/common/systemd/scylla-housekeeping-restart.service.mustache: specify correct repo for Debian variants
We do specify correct repo for both Red Hat/Debian variants on -deily, but
mistakenly don't for -restart, so do same on -restart.

Fixes #3906

Signed-off-by: Takuya ASADA <syuu@scylladb.com>
Message-Id: <20181109224509.27380-1-syuu@scylladb.com>
(cherry picked from commit 7740cd2142)
2018-11-27 09:59:05 +02:00
Raphael S. Carvalho
6e7e7f3822 sstables: deprecate sstable metadata's ancestors
The reason for that is that it's not available in sstable format mc,
so we can no longer rely on it in common code for the currently
supported formats.

Signed-off-by: Raphael S. Carvalho <raphaelsc@scylladb.com>
Message-Id: <20181121170057.20900-1-raphaelsc@scylladb.com>
(cherry picked from commit d29482dce8)
2018-11-24 12:36:40 +02:00
Paweł Dziepak
82a36edc9d Merge "Optimize sstable writing of the MC format" from Tomasz
"
Tested with perf_fast_forward from:

  github.com/tgrabiec/scylla.git perf_fast_forward-for-sst3-opt-write-v1

Using the following command line:

  build/release/tests/perf/perf_fast_forward_g --populate --sstable-format=mc \
     --data-directory /tmp/perf-mc --rows=10000000 -c1 -m4G \
     --datasets small-part

The average reported flush throughput was (stdev for the avergages is around 4k):
  - for mc before the series: 367848 frag/s
  - for lc before the series: 463458 frag/s (= mc.before +25%)
  - for mc after the series: 429276 frag/s (= mc.before +16%)
  - for lc after the series: 466495 frag/s (= mc.before +26%)

Refs #3874.
"

* tag 'sst3-opt-write-v2' of github.com:tgrabiec/scylla:
  sstables: mc: Avoid serialization of promoted index when empty
  sstables: mc: Avoid double serialization of rows
  tests: sstable 3.x: Do not compare Statistics component
  utils: Introduce memory_data_sink
  schema: Optimize column count getters
  sstables: checksummed_file_data_sink_impl: Bypass output_stream

(cherry picked from commit 4aa5d83590)
2018-11-24 12:36:40 +02:00
Avi Kivity
d4efa3c9b2 Update seastar submodule
* seastar d6647df...880826e (1):
  > fstream: Introduce make_file_data_sink()
2018-11-24 12:36:40 +02:00
Avi Kivity
324dae3e12 Merge "compress: Restore lz4 as default compressor" from Duarte
"
Enables sstable compression with LZ4 by default, which was the
long-time behavior until a regression turned off compression by
default.

Fixes #3926
"

* 'restore-default-compression/v2' of https://github.com/duarten/scylla:
  tests/cql_query_test: Assert default compression options
  compress: Restore lz4 as default compressor
  tests: Be explicit about absence of compression

(cherry picked from commit bb85a21a8f)
2018-11-21 16:45:22 +02:00
Tomasz Grabiec
c0ffc9a2b7 utils: phased_barrier: Make advance_and_await() have strong exception guarantees
Currently, when advance_and_await() fails to allocate the new gate
object, it will throw bad_alloc and leave the phased_barrier object in
an invalid state. Calling advance_and_await() again on it will result
in undefined behavior (typically SIGSEGV) beacuse _gate will be
disengaged.

One place affected by this is table::seal_active_memtable(), which
calls _flush_barrier.advance_and_await(). If this throws, subsequent
flush attempts will SIGSEGV.

This patch rearranges the code so that advance_and_await() has strong
exception guarantees.
Message-Id: <1542645562-20932-1-git-send-email-tgrabiec@scylladb.com>

Fixes #3931.

(cherry picked from commit 57e25fa0f8)
2018-11-21 12:17:27 +02:00
Glauber Costa
f81fa5f75c remove monitor if sstable write failed
In (almost) all SSTable write paths, we need to inform the monitor that
the write has failed as well. The monitor will remove the SSTable from
controller's tracking at that point.

Except there is one place where we are not doing that: streaming of big
mutations. Streaming of big mutations is an interesting use case, in
which it is done in 2 parts: if the writing of the SSTable fails right
away, then we do the correct thing.

But the SSTables are not commited at that point and the monitors are
still kept around with the SSTables until a later time, when they are
finally committed. Between those two points in time, it is possible that
the streaming code will detect a failure and manually call
fail_streaming_mutations(), which marks the SSTable for deletions. At
that point we should propagate that information to the monitor as well,
but we don't.

Fixes #3732 (hopefully)
Tests: unit (release)

Signed-off-by: Glauber Costa <glauber@scylladb.com>
Message-Id: <20181114213618.16789-1-glauber@scylladb.com>
(cherry picked from commit 9f403334c8)
2018-11-20 19:27:54 +02:00
Glauber Costa
6fd1cfcfce sstables: correctly parse estimated histograms
In commit a33f0d6, we changed the way we handle arrays during the write
and parse code to avoid reactor stalls. Some potentially big loops were
transformed into futurized loops, and also some calls to vector resizes
were replaced by a reserve + push_back idiom.

The latter broke parsing of the estimated histogram. The reason being
that the vectors that are used here are already initialized internally
by the estimated_histogram object. Therefore, when we push_back, we
don't fill the array all the way from index 0, but end up with a zeroed
beginning and only push back some of the elements we need.

We could revert this array to a resize() call. After all, the reason we
are using reserve + push_back is to avoid calling the constructor member
for each element, but We don't really expect the integer specialization
to do any of that.

However, to avoid confusion with future developers that may feel tempted
to converted this as well for the sake of consistency, it is safer to
just make sure these arrays are zeroed.

Fixes #3918

Signed-off-by: Glauber Costa <glauber@scylladb.com>
Message-Id: <20181116130853.10473-1-glauber@scylladb.com>
(cherry picked from commit c6811bd877)
2018-11-17 17:20:00 +02:00
Nadav Har'El
9d458ffea9 Materialized Views and Secondary Index: no longer experimental
After this patch, the Materialized Views and Secondary Index features
are considered generally-available and no longer require passing an
explicit "--experimental=on" flag to Scylla.

The "--experimental=on" flag and the db::config::check_experimental()
function remain unused, as we graduated the only two features which used
this flag. However, we leave the support for experimental features in
the code, to make it easier to add new experimental features in the future.
Another reason to leave the command-line parameter behind is so existing
scripts that still use it will not break.

Fixes #3917

Signed-off-by: Nadav Har'El <nyh@scylladb.com>
Message-Id: <20181115144456.25518-1-nyh@scylladb.com>
(cherry picked from commit 78ed7d6d0c)
2018-11-15 19:50:30 +02:00
Duarte Nunes
9776a048e7 Merge 'Generating view updates during streaming' from Piotr
During streaming, there are cases when we should invoke the view write
path. In particular, if we're streaming because of repair or if a view
has not yet finished building and we're bootstrapping a new node.

The design constraints are:
1) The streamed writes should be visible to new writes, but the
   sstable should not participate in compaction, or we would lose the
   ability to exclude the streamed writes on a restart;
2) The streamed writes must not be considered when generating view
   updates for them;
3) Resilient to node restarts;
4) Resilient to concurrent stream sessions, possibly streaming mutations for overlapping ranges.

We achieve this by writing the streamed writes to an sstable in a
different folder, call it "staging". We achieve 1) by publishing the
sstable to the column family sstable set, but excluding it from
compactions. We do these steps upon boot, by looking at the staging
directory, thus achieving 3).

Fixes #3275

* 'streaming_view_to_staging_sstables_9' of https://github.com/psarna/scylla: (29 commits)
  tests: add materialized views test
  tests: add view update generator to cql test env
  main: add registering staging sstables read from disk
  database: add a check if loaded sstable is already staging
  database: add get_staging_sstable method
  streaming: stream tables with views through staging sstables
  streaming: add system distributed keyspace ref to streaming
  streaming: add view update generator reference to streaming
  main: add generating missed mv updates from staging sstables
  storage_service: move initializing sys_dist_ks before bootstrap
  db/view: add view_update_from_staging_generator service
  db/view: add view updating consumer
  table: add stream_view_replica_updates
  table: split push_view_replica_updates
  table: add as_mutation_source_excluding
  table: move push_view_replica_updates to table.cc
  database: add populating tables with staging sstables
  database: add creating /staging directory for sstables
  database: add sstable-excluding reader
  table: add move_sstable_from_staging_in_thread function
  ...

(cherry picked from commit a38f6078fb)
2018-11-15 17:46:20 +02:00
Asias He
10cf97375e streaming: Expose reason for streaming
On receiving a mutation_fragment or a mutation triggered by a streaming
operation, we pass an enum stream_reason to notify the receiver what
the streaming is used for. So the receiver can decide further operation,
e.g., send view updates, beyond applying the streaming data on disk.

Fixes #3276
Message-Id: <f15ebcdee25e87a033dcdd066770114a499881c0.1539498866.git.asias@scylladb.com>

(cherry picked from commit 7f826d3343)
2018-11-15 17:45:31 +02:00
Paweł Dziepak
e6355a9a01 Merge "Write static rows for all partitions if there are static columns" from Vladimir
"
It appears that in case when there are any static columns in serialization header,
Cassandra would write a (possibly empty) static row to every partition
in the SSTables file.

This patchset alings Scylla's logic with that of Cassandra.

Note that Scylla optimizes the case when no partition contains a static
row because it keeps track of updated columns that Scylla currently does
not do - see #3901 for details.

Fixes #3900.
"

* 'projects/sstables-30/write-all-static-rows/v1' of https://github.com/argenet/scylla:
  tests: Test writing empty static rows for partitions in tables with static columns.
  sstables: Ignore empty static rows on reading.
  sstables: Write empty static rows when there are static columns in the table.

(cherry picked from commit 6469a1b451)
2018-11-12 15:59:35 -08:00
Raphael S. Carvalho
e57907a1d5 sstables: fix procedure to get fully expired sstables with MC format
MC format lacks ancestors metadata, so we need to workaround it by using
ancestors in metadata collector, which is only available for a sstable
written during this instance. It works fine here because we only want
to know if a sstable recently compacted has an ancestor which wasn't
yet deleted.

Fixes #3852.

Signed-off-by: Raphael S. Carvalho <raphaelsc@scylladb.com>
Reviewed-by: Vladimir Krivopalov <vladimir@scylladb.com>
Message-Id: <20181102154951.22950-1-raphaelsc@scylladb.com>
(cherry picked from commit 1c5934c934)
2018-11-06 16:03:18 +02:00
Pekka Enberg
f94b46e7e0 docker: Switch to 3.0 RPM repository 2018-11-01 19:40:10 +02:00
Avi Kivity
6847c12668 Merge "dist: use perftune.py for disks tuning" from Vlad
"
Use perftune.py for tuning disks:
   - Distribute/pin disks' IRQs:
      - For NVMe drives: evenly among all present CPUs.
      - For non-NVMe drives: according to chosen tuning mode.
   - For all disks used by scylla:
      - Tune nomerges
      - Tune I/O scheduler.

It's important to tune NIC and disks together in order to keep IRQ
pinning in the same mode.

Disk are detected and tuned based on the current content of
/etc/scylla/scylla.yaml configuration file.
"

Fixes #3831.

* 'use_perftune_for_disks-v3' of https://github.com/vladzcloudius/scylla:
  dist: change the sysconfig parameter name to reflect the new semantics
  scylla_util.py::sysconfig_parser: introduce has_option()
  dist: scylla_setup and scylla_sysconfig_setup: change paremeters names to reflect new semantics
  dist: don't distribute posix_net_conf.sh any more
  dist: use perftune.py to tune disks and NIC

(cherry picked from commit f170e3e589)
2018-11-01 19:19:04 +02:00
Avi Kivity
80b86def1f Update seastar submodule
* seastar 0c8a2c8...d6647df (3):
  > scripts: perftune.py: properly merge parameters from the command line and the configuration file
  > scripts: perftune.py: prioritize I/O schedulers
  > Merge "scripts: perftune.py: support different I/O schedulers" from Vlad

Ref #3831.
2018-11-01 19:18:07 +02:00
Vlad Zolotarov
c6de9ea39b config: enable hinted handoff by default
Signed-off-by: Vlad Zolotarov <vladz@scylladb.com>
Message-Id: <20181019180401.12400-1-vladz@scylladb.com>
(cherry picked from commit 4d1bb719a4)
2018-11-01 10:41:44 +02:00
Avi Kivity
94bed81c1d Update seastar submodule
* seastar 39b89de...0c8a2c8 (1):
  > prometheus: Allow preemption between each metric

See scylladb/seastar#469.
2018-10-31 19:21:21 +02:00
Hagit Segev
0f3a21f0bb release: prepare for 3.0-rc1 2018-10-31 12:08:43 +02:00
Tomasz Grabiec
976db7e9e0 Merge "Proper support for static rows in SSTables 3.x" from Vladimir
This patchset addresses two issues with static rows support in SSTables
3.x. ('mc' format):

1. Since collections are allowed in static rows, we need to check for
complex deletion, set corresponding flag and write tombstones, if any.
2. Column indices need to be partitioned for static columns the same way
they are partitioned for regular ones.

 * github.com/argenet/scylla.git projects/sstables-30/columns-proper-order-followup/v1:
  sstables: Partition static columns by atomicity when reading/writing
    SSTables 3.x.
  sstables: Use std::reference_wrapper<> instead of a helper structure.
  sstables: Check for complex deletion when writing static rows.
  tests: Add/fix comments to
    test_write_interleaved_atomic_and_collection_columns.
  tests: Add test covering inverleaved atomic and collection cells in
    static row.

(cherry picked from commit 62c7685b0d)
2018-10-30 14:51:21 +01:00
Nadav Har'El
996b86b804 Materalized views: fix race condition in resharding while view building
When a node reshards (i.e., restarts with a different number of CPUs), and
is in the middle of building a view for a pre-existing table, the view
building needs to find the right token from which to start building on all
shards. We ran the same code on all shards, hoping they would all make
the same decision on which token to continue. But in some cases, one
shard might make the decision, start building, and make progress -
all before a second shard goes to make the decision, which will now
be different.

This resulted, in some rare cases, in the new materialized view missing
a few rows when the build was interrupted with a resharding.

The fix is to add the missing synchronization: All shards should make
the same decision on whether and how to reshard - and only then should
start building the view.

Fixes #3890
Fixes #3452

Signed-off-by: Nadav Har'El <nyh@scylladb.com>
Message-Id: <20181028140549.21200-1-nyh@scylladb.com>
(cherry picked from commit b8337f8c9d)
2018-10-29 09:52:25 +00:00
Avi Kivity
b7b217cc43 Merge "Re-order columns when reading/writing SSTables 3.x" from Vladimir
"
In Cassandra, row columns are stored in a BTree that uses the following
ordering on them:
    - all atomic columns go first, then all multi-cell ones
    - columns of both types (atomic and multi-cell) are
      lexicographically ordered by name regarding each other

Scylla needs to store columns and their respective indices using the
same ordering as well as when reading them back.

Fixes #3853

Tests: unit {release}

+

Checked that the following SSTables are dumped fine using Cassandra's
sstabledump:

cqlsh:sst3> CREATE TABLE atomic_and_collection3 ( pk int, ck int, rc1 text, rc2 list<text>, rc3 text, rc4 list<text>, rc5 text, rc6 list<text>, PRIMARY KEY (pk, ck)) WITH compression = {'sstable_compression': ''};
cqlsh:sst3> INSERT INTO atomic_and_collection3 (pk, ck, rc1, rc4, rc5) VALUES (0, 0, 'hello', ['beautiful','world'], 'here');
<< flush >>

sstabledump:

[
  {
    "partition" : {
      "key" : [ "0" ],
      "position" : 0
    },
    "rows" : [
      {
        "type" : "row",
        "position" : 96,
        "clustering" : [ 0 ],
        "liveness_info" : { "tstamp" : "1540599270139464" },
        "cells" : [
          { "name" : "rc1", "value" : "hello" },
          { "name" : "rc5", "value" : "here" },
          { "name" : "rc4", "deletion_info" : { "marked_deleted" : "1540599270139463", "local_delete_time" : "1540599270" } },
          { "name" : "rc4", "path" : [ "45e22cb0-d97d-11e8-9f07-000000000000" ], "value" : "beautiful" },
          { "name" : "rc4", "path" : [ "45e22cb1-d97d-11e8-9f07-000000000000" ], "value" : "world" }
        ]
      }
    ]
  }
]
"

* 'projects/sstables-30/columns-proper-order/v1' of https://github.com/argenet/scylla:
  tests: Test interleaved atomic and multi-cell columns written to SSTables 3.x.
  sstables: Re-order columns (atomic first, then collections) for SSTables 3.x.
  sstables: Use a compound structure for storing information used for reading columns.

(cherry picked from commit 75dbff984c)
2018-10-28 15:51:47 +02:00
Tomasz Grabiec
c274430933 Merge "Properly write static rows missing columns for SSTables 3.x." from Vladimir
Before this fix, write_missing_columns() helper would always deal with
regular columns even when writing static rows.

This would cause errors on reading those files.

Now, the missing columns are written correctly for regular and static
rows alike.

* github.com/argenet/scylla.git projects/sstables-30/fix-writing-static-missing-columns/v1:
  schema: Add helper method returning the count of columns of specified
    kind.
  sstables: Honour the column kind when writing missing columns in 'mc'
    format.
  tests: Add test for a static row with missing columns (SStables 3.x.).

(cherry picked from commit cf2d5c19fb)
2018-10-26 13:30:12 +03:00
Avi Kivity
893a18a7c4 Merge "Properly writing/reading shadowable deletions with SSTables 3.x." from Vladimir
"
This patchset adddresses two problems with shadowable deletions handling
in SSTables 3.x. ('mc' format).

Firstly, we previously did not set a flag indicating the presence of
extended flags byte with HAS_SHADOWABLE_DELETION bitmask on writing.
This would break subsequent reading and cause all types of failures up
to crash.

Secondly, when reading rows with this extended flag set, we need to
preserve that information and create a shadowable_tombstone for the row.

Tests: unit {release}
+

Verified manually with 'hexdump' and using modified 'sstabledump' that
second (shadowable) tombstone is written for MV tables by Scylla.

+
DTest (materialized_views_test.py:TestMaterializedViews.hundred_mv_concurrent_test)
that originally failed due to this issue has successfully passed locally.
"

* 'projects/sstables-30/shadowable-deletion/v4' of https://github.com/argenet/scylla:
  tests: Add tests writing both regular and shadowable tombstones to SSTables 3.x.
  tests: Add test covering writing and reading a shadowable tombstone with SSTables 3.x.
  sstables: Support Scylla-specific extension for writing shadowable tombstones.
  sstables: Introduce a feature for shadowable tombstones in Scylla.db.
  memtable: Track regular and shadowable tombstones separately in encoding_stats_collector.
  sstables: Error out when reading SSTables 3.x with Cassandra shadowable deletion.
  sstables: Support checking row extension flags for Cassandra shadowable deletion.

(cherry picked from commit 8210f4c982)
2018-10-24 19:32:57 +03:00
Tomasz Grabiec
39b39058fc sstable_mutation_reader: Do not read partition index when scanning
Even when we're using a full clustering range, need_skip() will return
true when we start a new partition and advance_context() will be
called with position_in_partition::before_all_clustered_rows(). We
should detect that there is no need to skip to that position before
the call to advance_to(*_current_partition_key), which will read the
index page.

Fixes #3868.

Message-Id: <1539881775-8578-1-git-send-email-tgrabiec@scylladb.com>
(cherry picked from commit 9e756d3863)
2018-10-24 19:32:40 +03:00
Avi Kivity
6bf4a73d88 thrift: limit message size
Limit message size according to the configuration, to avoid a huge message from
allocating all of the server's memory.

We also need to limit memory used in aggregate by thrift, but that is left to
another patch.

Fixes #3878.
Message-Id: <20181024081042.13067-1-avi@scylladb.com>

(cherry picked from commit a9836ad758)
2018-10-24 19:32:25 +03:00
Gleb Natapov
ca4846dd63 stream_session: remove unused capture
'Consumer function' parameter for distribute_reader_and_consume_on_shards()
captures schema_ptr (which is a seastar::shared_ptr), but the function
is later copied on another shard at which point schema_ptr is also copied
and its counter is incremented by the wrong shard. The capture is not
even used, so lets just drop it.

Fixes #3838

Message-Id: <20181011075500.GN14449@scylladb.com>
(cherry picked from commit ceb361544a)
2018-10-24 09:47:02 +03:00
Takuya ASADA
2663ff7bc1 dist/common/sysctl.d: add new conf file to set fs.aio-max-nr
We need raise fs.aio-max-nr to larger value since Seastar may allocates
more then 65535 AIO events (= kernel default value)

Fixes #3842

Signed-off-by: Takuya ASADA <syuu@scylladb.com>
Message-Id: <20181023030449.15445-1-syuu@scylladb.com>
(cherry picked from commit 950dbdb466)
2018-10-24 09:45:51 +03:00
Tomasz Grabiec
043a575fcd Merge "Correctly handle dropped columns in SSTable 3" from Piotr J.
Previously we were making assumptions about missing columns
(the size of its value, whether it's a collection or a counter) but
they didn't have to be always true. Now we're using column type
from serialization header to use the right values.

Fixes #3859

* seastar-dev.git haaawk/projects/sstables-30/handling-dropped-columns/v4:
  sstables 3: Correctly handle dropped columns in column_translation
  sstables 3: Add test for dropped columns handling

(cherry picked from commit fc37b80d24)
2018-10-24 09:45:25 +03:00
Vlad Zolotarov
00dc400993 storage_proxy::query_result_local: create a single tracing span on a replica shard
Every call of a tracing::global_trace_state_ptr object instead of a
tracing::tracing_state_ptr or a call to tracing::global_trace_state_ptr::get()
creates a new tracing session (span) object.

This should never be done unless query handling moves to a different shard.

Fixes #3862

Signed-off-by: Vlad Zolotarov <vladz@scylladb.com>
Message-Id: <20181018003500.10030-1-vladz@scylladb.com>
(cherry picked from commit a87c11bad2)
2018-10-24 09:45:14 +03:00
Duarte Nunes
522a48a244 Merge 'Fix for a select statement with filtered columns' from Eliran
"
This patchset fixes #3803. When a select statement with filtering
is executed and the column that is needed for the filtering is not
present in the select clause, rows that should have been filtered out
according to this column will still be present in the result set.

Tests:
 1. The testcase from the issue.
 2. Unit tests (release) including the
 newly added test from this patchset.
"

* 'issues/3803/v10' of https://github.com/eliransin/scylla:
  unit test: add test for filtering queries without the filtered column
  cql3 unit test: add assertion for the number of serialized columns
  cql3: ensure retrieval of columns for filtering
  cql3: refactor find_idx to be part of statement restrictions object
  cql3: add prefix size common functionality to all clustering restrictions
  cql3: rename selection metadata manipulation functions

(cherry picked from commit 3fe92663d4)
2018-10-24 09:44:46 +03:00
Paweł Dziepak
5faa28ce45 cql3: restore original timeout behaviour for aggregate queries
Commit 1d34ef38a8 "cql3: make pagers use
time_point instead of duration" has unintentionally altered the timeout
semantics for aggregate queries. Such requests fetch multiple pages before
sending a response to the client. Originally, each of those fetches had
a timeout-duration to finish, after the problematic commit the whole
request needs to complete in a single timeout-duration. This,
unsurprisingly, makes some queries that were successful before fail with
a timeout. This patch restores the original behaviour.

Fixes #3877.

Message-Id: <20181022125318.4384-1-pdziepak@scylladb.com>
(cherry picked from commit c94d2b6aa6)
2018-10-24 09:43:59 +03:00
Avi Kivity
52be02558e config: mark range_request_timeout_in_ms and request_timeout_in_ms as Used
This makes them available in scylla --help.

Fixes #3884.
Message-Id: <20181023101150.29856-1-avi@scylladb.com>

(cherry picked from commit d9e0ea6bb0)
2018-10-24 09:43:54 +03:00
Avi Kivity
a7cbfbe63f Merge "hinted handoff: give a sender a low priority" from Vlad
"
Hinted handoff should not overpower regular flows like READs, WRITEs or
background activities like memtable flushes or compactions.

In order to achieve this put its sending in the STEAMING CPU scheduling
group and its commitlog object into the STREAMING I/O scheduling group.

Fixes #3817
"

* 'hinted_handoff_scheduling_groups-v2' of https://github.com/vladzcloudius/scylla:
  db::hints::manager: use "streaming" I/O scheduling class for reads
  commitlog::read_log_file(): set the a read I/O priority class explicitly
  db::hints::manager: add hints sender to the "streaming" CPU scheduling group

(cherry picked from commit 1533487ba8)
2018-10-24 09:43:39 +03:00
Duarte Nunes
28fd2044d2 Merge 'hinted handoff: add manager::state and split storing and replaying enablement' from Vlad
"
Refs #3828
(Probably fixes it)

We found a few flaws in a way we enable hints replaying.
First of all it was allowed before manager::start() is complete.
Then, since manager::start() is called after messaging_service is
initialized there was a time window when hints are rejected and this
creates an issue for MV.

Both issues above were found in the context of #3828.

This series fixes them both.

Tested {release}:
dtest: materialized_views_test.py:TestMaterializedViews.write_to_hinted_handoff_for_views_test
dtest: hintedhandoff_additional_test.py
"

* 'hinted_handoff_dont_create_hints_until_started-v1' of https://github.com/vladzcloudius/scylla:
  hinted handoff: enable storing hints before starting messaging_service
  db::hints::manager: add a "started" state
  db::hints::manager: introduce a _state

(cherry picked from commit 3a53b3cebc)
2018-10-24 09:43:03 +03:00
Calle Wilund
76ff2e5c3d messaging_service: Make rpc streaming sink respect tls connection
Fixes #3787

Message service streaming sink was created using direct call to
rpc::client::make_sink. This in turn needs a new socker, which it
creates completely ignoring what underlying transport is active for the
client in question.

Fix by retaining the tls credential pointer in the client wrapper, and
using this in a sink method to determine whether to create a new tls
socker, or just go ahead with a plain one.

Message-Id: <20181010003249.30526-1-calle@scylladb.com>
(cherry picked from commit 3cb50c861d)
2018-10-23 07:36:21 +00:00
Avi Kivity
7b34d54a96 locator: fix abstract_replication_strategy::get_ranges() and friends violating sort order
get_ranges() is supposed to return ranges in sorted order. However, a35136533d
broke this and returned the range that was supposed to be last in the second
position (e.g. [0, 10, 1, 2, 3, 4, 5, 6, 7, 8, 9]). The broke cleanup, which
relied on the sort order to perform a binary search. Other users of the
get_ranges() family did not rely on the sort order.

Fixes #3872.
Message-Id: <20181019113613.1895-1-avi@scylladb.com>

(cherry picked from commit 1ce52d5432)
2018-10-23 07:36:21 +00:00
Duarte Nunes
26c31f6798 Merge "db/hints: Expose current backlog" from Duarte
"
Hints are stored on disk by a hints::manager, ensuring they are
eventually sent. A hints::resource_manager ensures the hints::managers
it tracks don't consume more than their allocated resources by
monitoring disk space and disabling new hints if needed. This series
fixes some bugs related to the backlog calculation, but mainly exposes
the backlog through a hints::manager so upper layers can apply flow
control.

Refs #2538
"

* 'hh-manager-backlog/v3' of https://github.com/duarten/scylla:
  db/hints/manager: Expose current backlog
  db/hints/manager: Move decision about blocking hints to the manager
  db/hints/resource_manager: Correctly account resources in space_watchdog
  db/hints/resource_manager: Replace timer with seastar::thread
  db/hints/resource_manager: Ensure managers are correctly registered
  db/hints/resource_manager: Fix formatting
  db/hints: Disallow moving or copying the managers
2018-10-23 07:36:21 +00:00
Glauber Costa
28fa66591a sstables: print sstable path in case of an exception
Without that, we don't know where to look for the problems

Before:

 compaction failed: sstables::malformed_sstable_exception (Too big ttl: 3163676957)

After:

 compaction_manager - compaction failed: sstables::malformed_sstable_exception (Too big ttl: 4294967295 in sstable /var/lib/scylla/data/system_traces/events-8826e8e9e16a372887533bc1fc713c25/mc-832-big-Data.db)

Signed-off-by: Glauber Costa <glauber@scylladb.com>
Message-Id: <20181016181004.17838-1-glauber@scylladb.com>
(cherry picked from commit 7edae5421d)
2018-10-23 07:36:14 +00:00
Piotr Sarna
0fee1d9e43 cql3: add asking for pk/ck in the base query
Base query partition and clustering keys are used to generate
paging state for an index query, so they always need to be present
when a paged base query is processed.
Message-Id: <f3bf69453a6fd2bc842c8bdbd602d62c91cf9218.1538568953.git.sarna@scylladb.com>

Fixes #3855.
(cherry picked from commit 4a23297117)
2018-10-16 19:59:42 +03:00
Piotr Sarna
76e72e28f4 cql3: add checking for may_need_paging when executing base query
It's not sufficient to check for positive page_size when preparing
a base query for indexed select statement - may_need_paging() should
be called as well.
Message-Id: <d435820019e4082a64ca9807541f0c9ad334e6a8.1538568953.git.sarna@scylladb.com>

(cherry picked from commit 50d3de0693)
2018-10-16 19:58:58 +03:00
Piotr Sarna
f969e80965 cql3: move base query command creation to a separate function
Message-Id: <6b48b8cbd6312da4a17bfd3c85af628b4215e9f4.1538568953.git.sarna@scylladb.com>
(cherry picked from commit 11b8831c04)
2018-10-16 19:58:56 +03:00
Vladimir Krivopalov
2029134063 sstables: Reset opened range tombstone when moving to another partition.
Signed-off-by: Vladimir Krivopalov <vladimir@scylladb.com>
Message-Id: <f6dc6b0bd88ca44f2ef84c2a8bee43fde82c89cc.1539396572.git.vladimir@scylladb.com>
(cherry picked from commit 092276b13d)
2018-10-15 13:26:22 +03:00
Vladimir Krivopalov
f30fe7bd17 sstables: Factor out code resetting values for a new partition.
Signed-off-by: Vladimir Krivopalov <vladimir@scylladb.com>
Message-Id: <83a3a4ce6942b036be447bcfeb66142828e75293.1539396572.git.vladimir@scylladb.com>
(cherry picked from commit 926b6430fd)
2018-10-15 13:26:20 +03:00
Piotr Sarna
aeb418af9e service/pager: avoid dereferencing null partition key
The pager::state() function returns a valid paging object even
if the pager itself is exhausted. It may also not contain the partition
key, so using it unconditionally was a bug - now, in case there is no
partition key present, paging state will contain an empty partition key.

Fixes #3829

Message-Id: <28401eb21ab8f12645c0a33d9e92ada9de83e96b.1539074813.git.sarna@scylladb.com>
(cherry picked from commit b3685342a6)
2018-10-15 12:47:25 +03:00
Glauber Costa
714e6d741f api: use longs instead of ints for snapshot sizes
Int types in json will be serialized to int types in C++. They will then
only be able to handle 4GB, and we tend to store more data than that.

Without this patch, listsnapshots is broken in all versions.

Fixes: #3845

Signed-off-by: Glauber Costa <glauber@scylladb.com>
Message-Id: <20181012155902.7573-1-glauber@scylladb.com>
(cherry picked from commit 98332de268)
2018-10-12 22:01:59 +03:00
Tomasz Grabiec
95c5872450 Merge "Enable sstable_mutation_test with SSTables 3.x." from Vladimir
Introduce uppermost_bound() method instead of upper_bound() in mutation_fragment_filter and clustering_ranges_walker.

For now, this has been only used to produce the final range tombstone
for sliced reads inside consume_partition_end().

Usage of the upper bound of the current range causes problems of two
kinds:
    1. If not all the slicing ranges have been traversed with the
    clustering range walker, which is possible when the last read
    mutation fragment was before some of the ranges and reading was limited
    to a specific range of positions taken from index, the emitted range
    tombstone will not cover the untraversed slices.

    2. At the same time, if all ranges have been walked past, the end
    bound is set to after_all_clustered_rows and the emitted RT may span
    more data than it should.

To avoid both situations, the uppermost bound is used instead, which
refers to the upper bound of the last range in the sequence.

* github.com/scylladb/seastar-dev.git haaawk/projects/sstables-30/enable-mc-with-sstable-mutation-test/v2
  sstables: Use uppermost_bound() instead of upper_bound() in
    mutation_fragment_filter.
  tests: Enable sstable_mutation_test for SSTables 'mc' format.

Rebased by Piotr J.

(cherry picked from commit b89556512a)
2018-10-12 17:46:49 +03:00
Tomasz Grabiec
87f8968553 Merge "Make SST3 pass test_clustering_slices test" from Piotr
* seastar-dev.git haaawk/sst3/test_clustering_slices/v8:
  sstables: Extract on_end_of_stream from consume_partition_end
  sstables: Don't call consume_range_tombstone_end in
    consume_partition_end
  sstables: Change the way fragments are returned from consumer

(cherry picked from commit 193efef950)
2018-10-12 17:46:46 +03:00
Tomasz Grabiec
2895428d44 Merge "Handle dead row markers when writing to SSTables 3.x" from Vladimir
There is a mismatch between row markers used in SSTables 2.x (ka/la) and
liveness_info used by SSTables 3.x (mc) in that a row marker can be
written as a deleted cell but liveness_info cannot.

To handle this, for a dead row marker the corresponding liveness_info is
written as expiring liveness_info with a fake TTL set to 1.
This approach is adapted from the solution for CASSANDRA-13395 that
exercised similar issue during SSTables upgrades.

* github.com/argenet/scylla.git projects/sstables-30/dead-row-marker/v7:
  sstables: Introduce TTL limitation and special 'expired TTL' value.
  sstables: Write dead row marker as expired liveness info.
  tests: Add test covering dead row marker writing to SSTables 3.x.

(cherry picked from commit a7a14e3af2)
2018-10-11 15:03:58 +03:00
Botond Dénes
e18f182cfc multishard_mutation_query(): don't attempt to stop broken readers
Currently, when stopping a reader fails, it simply won't be attempted to
be saved, and it will be left in the `_readers` array as-is. This can
lead to an assertion failure as the reader state will contain futures
that were already waited upon, and that the cleanup code will attempt to
wait on again. To prevent this, when stopping a reader fails, reset it
to nonexistent state, so that the cleanup code doesn't attempt to do
anything with it.

Refs: #3830

Signed-off-by: Botond Dénes <bdenes@scylladb.com>
Message-Id: <a1afc1d3d74f196b772e6c218999c57c15ca05be.1539088164.git.bdenes@scylladb.com>
(cherry picked from commit d467b518bc)
2018-10-10 10:12:00 +03:00
Vladimir Krivopalov
cf8cdbf87d sstables: Add missing 'mc' format into format strings map in sstable::filename().
Fixes #3832.

Signed-off-by: Vladimir Krivopalov <vladimir@scylladb.com>
Message-Id: <269421fb2ac8ab389231cbe9ed501da7e7ff936a.1539048008.git.vladimir@scylladb.com>
(cherry picked from commit e9aba6a9c3)
2018-10-10 05:53:28 +03:00
Avi Kivity
eb2814067d Update seastar submodule
* seastar 5712816...39b89de (1):
  > prometheus: Fix histogram text representation

Fixes #3827.
2018-10-09 16:35:54 +03:00
Avi Kivity
0c722d4547 Point seastar submodule at scylla-seastar.git
This allows us to freeze this branch's Seastar and only backport selected fixes.
2018-10-09 16:29:53 +03:00
Nadav Har'El
54cf463430 materialized views: refuse to filter by non-key column
A materialized views can provide a filter so as to pick up only a subset
of the rows from the base table. Usually, the filter operates on columns
from the base table's primary key. If we use a filter on regular (non-key)
columns, things get hairy, and as issue #3430 showed, wrong: merely updating
this column in the base table may require us to delete, or resurrect, the
view row. But normally we need to do the above when the "new view key column"
was updated, when there is one. We use shadowable tombstones with one
timestamp to do this, so it cannot take into account the two timestamp from
those two columns (the filtered column and the new key column).

So in the current code, filtering by a non-key column does not work correctly.
In this patch we provide two test cases (one involving TTLs, and one involves
only normal updates), which demonstrate vividly that it does *not* work
correctly. With normal updates, trying to resurect a view row that has
previously disappeared, fails. With TTLs, things are even worse, and the view
row fails to disappear when the filtered column is TTLed.

In Cassandra, the same thing doesn't work correctly as well (see
CASSANDRA-13798 and CASSANDRA-13832) so they decided to refuse creating
a materialized view filtering a non-key column. In this patch we also
do this - fail the creation of such an unsupported view. For this reason,
the two tests mentioned above are commented out in a "#if", with, instead,
a trivial test verifying a failure to create such a view.

Note that as explained above, when the filtered column and new view key
column are *different* we have a problem. But when they are the *same* - namely
we filter by a non-key base column which actually *is* a key in the view -
we are actually fine. This patch includes additional test cases verifying
that this case is really fine and provides correct results. Accordingly,
this case is *not* forbidden in the view creation code.

Fixes #3430.

Signed-off-by: Nadav Har'El <nyh@scylladb.com>
Message-Id: <20181008185633.24616-1-nyh@scylladb.com>
(cherry picked from commit b8668dc0f8)
2018-10-09 10:18:58 +01:00
Nadav Har'El
d2a0622edd materialized views: enable two tests in view_schema_test
We had two commented out tests based on Cassandra's MV unit tests, for
the case that the view's filter (the "SELECT" clause used to define the
view) filtered by a non-primary-key column. These tests used to fail
because of problems we had in the filtering code, but they now succeed,
so we can enable them. This patch also adds some comments about what
the tests do, and adds a few more cases to one of the tests.

Refs #3430.

However, note that the success of these tests does not really prove that
the non-PK-column filtering feature works fully correctly and that issue
forbidding it, as explained in
https://issues.apache.org/jira/browse/CASSANDRA-13798. We can probably
fix this feature with our "virtual cells" mechanism, but will need to add
a test to confirm the possible problem and its (probably needed fix).
We do not add such a test in this patch.

In the meantime, issue #3430 should remain open: we still *allow* users
to create MV with such a filter, and, as the tests in this patch show,
this "mostly" works correctly. We just need to prove and/or fix what happens
with the complex row liveness issues a la issue #3362.

Signed-off-by: Nadav Har'El <nyh@scylladb.com>
Message-Id: <20181004213637.32330-1-nyh@scylladb.com>
(cherry picked from commit e4ef7fc40a)
2018-10-09 10:18:54 +01:00
Duarte Nunes
60edaec757 Merge 'Fix issues with endpoint state replication to other shards' from Tomasz
Fixes #3798
Fixes #3694

Tests:

  unit(release), dtest([new] cql_tests.py:TruncateTester.truncate_after_restart_test)

* tag 'fix-gossip-shard-replication-v1' of github.com:tgrabiec/scylla:
  gms/gossiper: Replicate enpoint states in add_saved_endpoint()
  gms/gossiper: Make reset_endpoint_state_map() have effect on all shards
  gms/gossiper: Replicate STATUS change from mark_as_shutdown() to other shards
  gms/gossiper: Always override states from older generations

(cherry picked from commit 48ebe6552c)
2018-10-09 10:14:30 +03:00
Avi Kivity
5802532cb3 Merge "Fix mutation fragments clobbering on fast_forward" from Vladimir
"
This patchset fixes a bug in SSTables 3.x reading when fast-forwarding
is enabled. It is possible that a mutation fragment, row or RT marker,
is read and then stored because it falls outside the current
fast-forwarding range.

If the reader is further fast-forwarded but the
row still falls outside of it, the reader would still continue reading
and get the next fragment, if any, that would clobber the currently
stored one. With this fix, the reader does not attempt to read on
after storing the current fragment.

Tests: unit {release}
"

* 'projects/sstables-30/row-skipped-on-double-ff/v2' of https://github.com/argenet/scylla:
  tests: Add test for reading rows after multiple fast-forwarding with SSTables 3.x.
  sstables: mp_row_consumer_m to notify reader on end of stream when storing a mutation fragment.
  sstables: In mp_row_consumer_m::push_mutation_fragments(), return the called helper's value.

(cherry picked from commit 0fa60660b8)
2018-10-09 09:35:51 +03:00
Eliran Sinvani
83ea91055e cql3 : add workaround to antlr3 null dereference bug
The Antlr3 exception class has a null dereference bug that crashes
the system when trying to extract the exception message using
ANTLR_Exception<...>::displayRecognitionError(...) function. When
a parsing error occurs the CqlParser throws an exception which in
turn processesed for some special cases in scylla to generate a custom
message. The default case however, creates the message using
displayRecognitionError, causing the system to crash.
The fix is a simple workaround, making sure the pointer is not null
before the call to the function. A "proper" fix can't be implemented
because the exception class itself is implemented outside scylla
in antlr headers that resides on the host machine os.

Tested manualy 2 testcases, a typo causing scylla to crash and
a cql comment without a newline at the end also caused scylla to crash.
Ran unit tests (release).

Fixes #3740
Fixes #3764

Signed-off-by: Eliran Sinvani <eliransin@scylladb.com>
Message-Id: <cfc7e0d758d7a855d113bb7c8191b0fd7d2e8921.1538566542.git.eliransin@scylladb.com>
(cherry picked from commit 20f49566a2)
2018-10-04 14:07:25 +03:00
Piotr Sarna
e7863d3d54 tests: add missing get() calls in threaded context
One test case missed a few get() calls in order to wait
for continuations, which only accidentally worked,
because it was followed by 'eventually()' blocks.
Message-Id: <69c145575ac81154c4b5f500d01c6b045a267088.1536839959.git.sarna@scylladb.com>

(cherry picked from commit a5570cb288)
2018-10-04 14:06:50 +03:00
Piotr Sarna
57f124b905 tests: add collections test for secondary indexing
Test case regarding creating indexes on collection columns
is added to the suite.

Refs #3654
Refs #2962
Message-Id: <1b6844634b6e9a353028545813571647c92fb330.1536839959.git.sarna@scylladb.com>

(cherry picked from commit 8a2abd45fb)
2018-10-04 14:06:48 +03:00
Piotr Sarna
40d8de5784 cql3: prevent creation of indexes on non-frozen collections
Until indexes for non-frozen collections is implemented,
creating such indexes should be disallowed to prevent unnecessary
errors on insertions/selections.

Fixes #3653
Refs #2962
Message-Id: <218cf96d5e38340806fb9446b8282d2296ba5f43.1536839959.git.sarna@scylladb.com>

(cherry picked from commit 2d355bdf47)
2018-10-04 14:06:47 +03:00
Avi Kivity
1468ec62de Merge "Handle simple column type schema changes in SST3" from Piotr
"
This patchset enables very simple column type conversions.
It covers only handling variable and fixed size type differences.
Two types still have to be compatiple on bits level to be able to convert a field from one to the other.
"

* 'haaawk/sst3/column_type_schema_change/v4' of github.com:scylladb/seastar-dev:
  Fix check_multi_schema to actually check the column type change
  Handle very basic column type conversions in SST3
  Enable check_multi_schema for SST3

(cherry picked from commit b9702222f8)
2018-10-03 17:44:26 +03:00
Avi Kivity
c6ef56ae1e Revert "compaction: demote compaction start/end messages to DEBUG level"
This reverts commit b443a9b930. The compaction
history table doesn't have enough information to be a replacement for this
log message yet.

(cherry picked from commit 7c8143c3c4)
2018-10-03 17:44:21 +03:00
Avi Kivity
ad62313b86 utils: crc32: mark power crc32 assembly as not requiring an executable stack
The linker uses an opt-in system for non-executable stack: if all object files
opt into a non-executable stack, the binary will have a non-executable stack,
which is very desirable for security. The compiler cooperates by opting into
a non-executable stack whenever possible (always for our code).

However, we also have an assembly file (for fast power crc32 computations).
Since it doesn't opt into a non-executable stack, we get a binary with
executable stack, which Gentoo's build system rightly complains about.

Fix by adding the correct incantation to the file.

Fixes #3799.

Reported-by: Alexys Jacob <ultrabug@gmail.com>
Message-Id: <20181002151251.26383-1-avi@scylladb.com>
(cherry picked from commit aaab8a3f46)
2018-10-02 23:22:56 +03:00
Avi Kivity
de87f798e1 release: prepare for 3.0-rc0 2018-10-02 12:00:50 +03:00
6052 changed files with 181003 additions and 520310 deletions

View File

@@ -1,4 +0,0 @@
.git
build
seastar/build
testlog

1
.gitattributes vendored
View File

@@ -1,3 +1,2 @@
*.cc diff=cpp
*.hh diff=cpp
*.svg binary

101
.github/CODEOWNERS vendored
View File

@@ -1,101 +0,0 @@
# AUTH
auth/* @elcallio @vladzcloudius
# CACHE
row_cache* @tgrabiec
*mutation* @tgrabiec
test/boost/mvcc* @tgrabiec
# CDC
cdc/* @kbr- @elcallio @piodul @jul-stas
test/cql/cdc_* @kbr- @elcallio @piodul @jul-stas
test/boost/cdc_* @kbr- @elcallio @piodul @jul-stas
# COMMITLOG / BATCHLOG
db/commitlog/* @elcallio @eliransin
db/batch* @elcallio
# COORDINATOR
service/storage_proxy* @gleb-cloudius
# COMPACTION
compaction/* @raphaelsc @nyh
# CQL TRANSPORT LAYER
transport/*
# CQL QUERY LANGUAGE
cql3/* @tgrabiec @cvybhu @nyh
# COUNTERS
counters* @jul-stas
tests/counter_test* @jul-stas
# DOCS
docs/* @annastuchlik @tzach
docs/alternator @annastuchlik @tzach @nyh @havaker @nuivall
# GOSSIP
gms/* @tgrabiec @asias
# DOCKER
dist/docker/*
# LSA
utils/logalloc* @tgrabiec
# MATERIALIZED VIEWS
db/view/* @nyh @cvybhu @piodul
cql3/statements/*view* @nyh @cvybhu @piodul
test/boost/view_* @nyh @cvybhu @piodul
# PACKAGING
dist/* @syuu1228
# REPAIR
repair/* @tgrabiec @asias @nyh
# SCHEMA MANAGEMENT
db/schema_tables* @tgrabiec @nyh
db/legacy_schema_migrator* @tgrabiec @nyh
service/migration* @tgrabiec @nyh
schema* @tgrabiec @nyh
# SECONDARY INDEXES
index/* @nyh @cvybhu @piodul
cql3/statements/*index* @nyh @cvybhu @piodul
test/boost/*index* @nyh @cvybhu @piodul
# SSTABLES
sstables/* @tgrabiec @raphaelsc @nyh
# STREAMING
streaming/* @tgrabiec @asias
service/storage_service.* @tgrabiec @asias
# ALTERNATOR
alternator/* @nyh @havaker @nuivall
test/alternator/* @nyh @havaker @nuivall
# HINTED HANDOFF
db/hints/* @piodul @vladzcloudius @eliransin
# REDIS
redis/* @nyh @syuu1228
test/redis/* @nyh @syuu1228
# READERS
reader_* @denesb
querier* @denesb
test/boost/mutation_reader_test.cc @denesb
test/boost/querier_cache_test.cc @denesb
# PYTEST-BASED CQL TESTS
test/cql-pytest/* @nyh
# RAFT
raft/* @kbr- @gleb-cloudius @kostja
test/raft/* @kbr- @gleb-cloudius @kostja
# HEAT-WEIGHTED LOAD BALANCING
db/heat_load_balance.* @nyh @gleb-cloudius

4
.github/PULL_REQUEST_TEMPLATE.md vendored Normal file
View File

@@ -0,0 +1,4 @@
Scylla doesn't use pull-requests, please send a patch to the [mailing list](mailto:scylladb-dev@googlegroups.com) instead.
See our [contributing guidelines](../CONTRIBUTING.md) and our [Scylla development guidelines](../HACKING.md) for more information.
If you have any questions please don't hesitate to send a mail to the [dev list](mailto:scylladb-dev@googlegroups.com).

View File

@@ -1,17 +0,0 @@
name: "Docs / Amplify enhanced"
on: issue_comment
jobs:
build:
runs-on: ubuntu-latest
if: ${{ github.event.issue.pull_request }}
steps:
- name: Checkout
uses: actions/checkout@v3
with:
fetch-depth: 0
- name: Amplify enhanced
env:
TOKEN: ${{ secrets.GITHUB_TOKEN }}
uses: scylladb/sphinx-scylladb-theme/.github/actions/amplify-enhanced@master

View File

@@ -1,40 +0,0 @@
name: "Docs / Publish"
# For more information,
# see https://sphinx-theme.scylladb.com/stable/deployment/production.html#available-workflows
env:
FLAG: ${{ github.repository == 'scylladb/scylla-enterprise' && 'enterprise' || 'opensource' }}
on:
push:
branches:
- 'master'
- 'enterprise'
paths:
- "docs/**"
workflow_dispatch:
jobs:
release:
runs-on: ubuntu-latest
steps:
- name: Checkout
uses: actions/checkout@v3
with:
persist-credentials: false
fetch-depth: 0
- name: Set up Python
uses: actions/setup-python@v3
with:
python-version: 3.7
- name: Set up env
run: make -C docs FLAG="${{ env.FLAG }}" setupenv
- name: Build docs
run: make -C docs FLAG="${{ env.FLAG }}" multiversion
- name: Build redirects
run: make -C docs FLAG="${{ env.FLAG }}" redirects
- name: Deploy docs to GitHub Pages
run: ./docs/_utils/deploy.sh
if: (github.ref_name == 'master' && env.FLAG == 'opensource') || (github.ref_name == 'enterprise' && env.FLAG == 'enterprise')
env:
GITHUB_TOKEN: ${{ secrets.GITHUB_TOKEN }}

View File

@@ -1,32 +0,0 @@
name: "Docs / Build PR"
# For more information,
# see https://sphinx-theme.scylladb.com/stable/deployment/production.html#available-workflows
env:
FLAG: ${{ github.repository == 'scylladb/scylla-enterprise' && 'enterprise' || 'opensource' }}
on:
pull_request:
branches:
- master
- enterprise
paths:
- "docs/**"
jobs:
build:
runs-on: ubuntu-latest
steps:
- name: Checkout
uses: actions/checkout@v3
with:
persist-credentials: false
fetch-depth: 0
- name: Set up Python
uses: actions/setup-python@v3
with:
python-version: 3.7
- name: Set up env
run: make -C docs FLAG="${{ env.FLAG }}" setupenv
- name: Build docs
run: make -C docs FLAG="${{ env.FLAG }}" test

13
.gitignore vendored
View File

@@ -19,16 +19,3 @@ CMakeLists.txt.user
__pycache__CMakeLists.txt.user
.gdbinit
resources
.pytest_cache
/expressions.tokens
tags
!db/tags/
testlog
test/*/*.reject
.vscode
docs/_build
docs/poetry.lock
compile_commands.json
.ccls-cache/
.mypy_cache
.envrc

20
.gitmodules vendored
View File

@@ -1,20 +1,14 @@
[submodule "seastar"]
path = seastar
url = ../seastar
url = ../scylla-seastar
ignore = dirty
[submodule "swagger-ui"]
path = swagger-ui
url = ../scylla-swagger-ui
ignore = dirty
[submodule "scylla-jmx"]
path = tools/jmx
url = ../scylla-jmx
[submodule "scylla-tools"]
path = tools/java
url = ../scylla-tools-java
[submodule "scylla-python3"]
path = tools/python3
url = ../scylla-python3
[submodule "tools/cqlsh"]
path = tools/cqlsh
url = ../scylla-cqlsh
[submodule "xxHash"]
path = xxHash
url = ../xxHash
[submodule "libdeflate"]
path = libdeflate
url = ../libdeflate

View File

@@ -1,3 +0,0 @@
Avi Kivity <avi@scylladb.com> Avi Kivity' via ScyllaDB development <scylladb-dev@googlegroups.com>
Raphael S. Carvalho <raphaelsc@scylladb.com> Raphael S. Carvalho' via ScyllaDB development <scylladb-dev@googlegroups.com>
Pavel Emelyanov <xemul@scylladb.com> Pavel Emelyanov' via ScyllaDB development <scylladb-dev@googlegroups.com>

View File

@@ -1,201 +1,142 @@
cmake_minimum_required(VERSION 3.18)
##
## For best results, first compile the project using the Ninja build-system.
##
cmake_minimum_required(VERSION 3.7)
project(scylla)
include(CTest)
if (NOT DEFINED FOR_IDE AND NOT DEFINED ENV{FOR_IDE} AND NOT DEFINED ENV{CLION_IDE})
message(FATAL_ERROR "This CMakeLists.txt file is only valid for use in IDEs, please define FOR_IDE to acknowledge this.")
endif()
list(APPEND CMAKE_MODULE_PATH
${CMAKE_CURRENT_SOURCE_DIR}/cmake
${CMAKE_CURRENT_SOURCE_DIR}/seastar/cmake)
# Default value. A more accurate list is populated through `pkg-config` below if `seastar.pc` is available.
set(SEASTAR_INCLUDE_DIRS "seastar")
set(CMAKE_BUILD_TYPE "${CMAKE_BUILD_TYPE}" CACHE
STRING "Choose the type of build." FORCE)
# Set the possible values of build type for cmake-gui
set_property(CACHE CMAKE_BUILD_TYPE PROPERTY STRINGS
"Debug" "Release" "Dev" "Sanitize")
string(TOUPPER "${CMAKE_BUILD_TYPE}" build_mode)
include(mode.${build_mode})
include(mode.common)
add_compile_definitions(
${Seastar_DEFINITIONS_${build_mode}}
FMT_DEPRECATED_OSTREAM)
include(limit_jobs)
# Configure Seastar compile options to align with Scylla
set(CMAKE_CXX_STANDARD "20" CACHE INTERNAL "")
set(CMAKE_CXX_EXTENSIONS ON CACHE INTERNAL "")
set(CMAKE_CXX_VISIBILITY_PRESET hidden)
# These paths are always available, since they're included in the repository. Additional DPDK headers are placed while
# Seastar is built, and are captured in `SEASTAR_INCLUDE_DIRS` through parsing the Seastar pkg-config file (below).
set(SEASTAR_DPDK_INCLUDE_DIRS
seastar/dpdk/lib/librte_eal/common/include
seastar/dpdk/lib/librte_eal/common/include/generic
seastar/dpdk/lib/librte_eal/common/include/x86
seastar/dpdk/lib/librte_ether)
set(Seastar_TESTING ON CACHE BOOL "" FORCE)
add_subdirectory(seastar)
find_package(PkgConfig REQUIRED)
# System libraries dependencies
find_package(Boost REQUIRED
COMPONENTS filesystem program_options system thread regex unit_test_framework)
find_package(Lua REQUIRED)
find_package(ZLIB REQUIRED)
find_package(ICU COMPONENTS uc i18n REQUIRED)
find_package(absl COMPONENTS hash raw_hash_set REQUIRED)
find_package(libdeflate REQUIRED)
find_package(libxcrypt REQUIRED)
find_package(Snappy REQUIRED)
find_package(RapidJSON REQUIRED)
find_package(Thrift REQUIRED)
find_package(xxHash REQUIRED)
set(ENV{PKG_CONFIG_PATH} "${CMAKE_SOURCE_DIR}/seastar/build/release:$ENV{PKG_CONFIG_PATH}")
pkg_check_modules(SEASTAR seastar)
set(scylla_gen_build_dir "${CMAKE_BINARY_DIR}/gen")
file(MAKE_DIRECTORY "${scylla_gen_build_dir}")
find_package(Boost COMPONENTS filesystem program_options system thread)
##
## Populate the names of all source and header files in the indicated paths in a designated variable.
##
## When RECURSIVE is specified, directories are traversed recursively.
##
## Use: scan_scylla_source_directories(VAR my_result_var [RECURSIVE] PATHS [path1 path2 ...])
##
function (scan_scylla_source_directories)
set(options RECURSIVE)
set(oneValueArgs VAR)
set(multiValueArgs PATHS)
cmake_parse_arguments(args "${options}" "${oneValueArgs}" "${multiValueArgs}" "${ARGN}")
add_library(scylla-main STATIC)
target_sources(scylla-main
PRIVATE
absl-flat_hash_map.cc
bytes.cc
client_data.cc
clocks-impl.cc
collection_mutation.cc
compress.cc
converting_mutation_partition_applier.cc
counters.cc
direct_failure_detector/failure_detector.cc
duration.cc
exceptions/exceptions.cc
frozen_schema.cc
generic_server.cc
debug.cc
init.cc
keys.cc
message/messaging_service.cc
multishard_mutation_query.cc
mutation_query.cc
partition_slice_builder.cc
querier.cc
query.cc
query_ranges_to_vnodes.cc
query-result-set.cc
tombstone_gc_options.cc
tombstone_gc.cc
reader_concurrency_semaphore.cc
row_cache.cc
schema_mutations.cc
serializer.cc
sstables_loader.cc
table_helper.cc
tasks/task_manager.cc
timeout_config.cc
unimplemented.cc
validation.cc
vint-serialization.cc
zstd.cc)
target_link_libraries(scylla-main
PRIVATE
db
absl::hash
absl::raw_hash_set
Seastar::seastar
Snappy::snappy
systemd
ZLIB::ZLIB)
add_subdirectory(api)
add_subdirectory(alternator)
add_subdirectory(db)
add_subdirectory(auth)
add_subdirectory(cdc)
add_subdirectory(compaction)
add_subdirectory(cql3)
add_subdirectory(data_dictionary)
add_subdirectory(dht)
add_subdirectory(gms)
add_subdirectory(idl)
add_subdirectory(index)
add_subdirectory(interface)
add_subdirectory(lang)
add_subdirectory(locator)
add_subdirectory(mutation)
add_subdirectory(mutation_writer)
add_subdirectory(readers)
add_subdirectory(redis)
add_subdirectory(replica)
add_subdirectory(raft)
add_subdirectory(repair)
add_subdirectory(rust)
add_subdirectory(schema)
add_subdirectory(service)
add_subdirectory(sstables)
add_subdirectory(streaming)
add_subdirectory(test)
add_subdirectory(thrift)
add_subdirectory(tools)
add_subdirectory(tracing)
add_subdirectory(transport)
add_subdirectory(types)
add_subdirectory(utils)
include(add_version_library)
add_version_library(scylla_version
release.cc)
set(globs "")
foreach (dir ${args_PATHS})
list(APPEND globs "${dir}/*.cc" "${dir}/*.hh")
endforeach()
if (args_RECURSIVE)
set(glob_kind GLOB_RECURSE)
else()
set(glob_kind GLOB)
endif()
file(${glob_kind} var
${globs})
set(${args_VAR} ${var} PARENT_SCOPE)
endfunction()
## Although Seastar is an external project, it is common enough to explore the sources while doing
## Scylla development that we'll treat the Seastar sources as part of this project for easier navigation.
scan_scylla_source_directories(
VAR SEASTAR_SOURCE_FILES
RECURSIVE
PATHS
seastar/core
seastar/http
seastar/json
seastar/net
seastar/rpc
seastar/tests
seastar/util)
scan_scylla_source_directories(
VAR SCYLLA_ROOT_SOURCE_FILES
PATHS .)
scan_scylla_source_directories(
VAR SCYLLA_SUB_SOURCE_FILES
RECURSIVE
PATHS
api
auth
cql3
db
dht
exceptions
gms
index
io
locator
message
repair
service
sstables
streaming
tests
thrift
tracing
transport
utils)
scan_scylla_source_directories(
VAR SCYLLA_GEN_SOURCE_FILES
RECURSIVE
PATHS build/release/gen)
set(SCYLLA_SOURCE_FILES
${SCYLLA_ROOT_SOURCE_FILES}
${SCYLLA_GEN_SOURCE_FILES}
${SCYLLA_SUB_SOURCE_FILES})
add_executable(scylla
main.cc)
target_link_libraries(scylla PRIVATE
scylla-main
api
auth
alternator
db
cdc
compaction
cql3
data_dictionary
dht
gms
idl
index
lang
locator
mutation
mutation_writer
raft
readers
redis
repair
replica
schema
scylla_version
service
sstables
streaming
test-perf
thrift
tools
tracing
transport
types
utils)
target_link_libraries(Boost::regex
INTERFACE
ICU::i18n
ICU::uc)
${SEASTAR_SOURCE_FILES}
${SCYLLA_SOURCE_FILES})
target_link_libraries(scylla PRIVATE
seastar
Boost::program_options)
# Note that since CLion does not undestand GCC6 concepts, we always disable them (even if users configure otherwise).
# CLion seems to have trouble with `-U` (macro undefinition), so we do it this way instead.
list(REMOVE_ITEM SEASTAR_CFLAGS "-DHAVE_GCC6_CONCEPTS")
# Force SHA1 build-id generation
set(default_linker_flags "-Wl,--build-id=sha1")
include(CheckLinkerFlag)
foreach(linker "lld" "gold")
set(linker_flag "-fuse-ld=${linker}")
check_linker_flag(CXX ${linker_flag} "CXX_LINKER_HAVE_${linker}")
if(CXX_LINKER_HAVE_${linker})
string(APPEND default_linker_flags " ${linker_flag}")
break()
endif()
endforeach()
# If the Seastar pkg-config information is available, append to the default flags.
#
# For ease of browsing the source code, we always pretend that DPDK is enabled.
target_compile_options(scylla PUBLIC
-std=gnu++1z
-DHAVE_DPDK
-DHAVE_HWLOC
"${SEASTAR_CFLAGS}")
set(CMAKE_EXE_LINKER_FLAGS "${default_linker_flags}" CACHE INTERNAL "")
# TODO: patch dynamic linker to match configure.py behavior
target_include_directories(scylla PRIVATE
"${CMAKE_CURRENT_SOURCE_DIR}"
"${scylla_gen_build_dir}")
# The order matters here: prefer the "static" DPDK directories to any dynamic paths from pkg-config. Some files are only
# available dynamically, though.
target_include_directories(scylla PUBLIC
.
${SEASTAR_DPDK_INCLUDE_DIRS}
${SEASTAR_INCLUDE_DIRS}
${Boost_INCLUDE_DIRS}
xxhash
libdeflate
build/release/gen)

View File

@@ -1,22 +1,11 @@
# Contributing to Scylla
# Asking questions or requesting help
## Asking questions or requesting help
Use the [ScyllaDB user mailing list](https://groups.google.com/forum/#!forum/scylladb-users) for general questions and help.
Use the [ScyllaDB Community Forum](https://forum.scylladb.com) or the [Slack workspace](http://slack.scylladb.com) for general questions and help.
# Reporting an issue
Join the [Scylla Developers mailing list](https://groups.google.com/g/scylladb-dev) for deeper technical discussions and to discuss your ideas for contributions.
Please use the [Issue Tracker](https://github.com/scylladb/scylla/issues/) to report issues. Fill in as much information as you can in the issue template, especially for performance problems.
## Reporting an issue
# Contributing Code to Scylla
Please use the [issue tracker](https://github.com/scylladb/scylla/issues/) to report issues or to suggest features. Fill in as much information as you can in the issue template, especially for performance problems.
## Contributing code to Scylla
Before you can contribute code to Scylla for the first time, you should sign the [Contributor License Agreement](https://www.scylladb.com/open-source/contributor-agreement/) and send the signed form cla@scylladb.com. You can then submit your changes as patches to the to the [scylladb-dev mailing list](https://groups.google.com/forum/#!forum/scylladb-dev) or as a pull request to the [Scylla project on github](https://github.com/scylladb/scylla).
If you need help formatting or sending patches, [check out these instructions](https://github.com/scylladb/scylla/wiki/Formatting-and-sending-patches).
The Scylla C++ source code uses the [Seastar coding style](https://github.com/scylladb/seastar/blob/master/coding-style.md) so please adhere to that in your patches. Note that Scylla code is written with `using namespace seastar`, so should not explicitly add the `seastar::` prefix to Seastar symbols. You will usually not need to add `using namespace seastar` to new source files, because most Scylla header files have `#include "seastarx.hh"`, which does this.
Header files in Scylla must be self-contained, i.e., each can be included without having to include specific other headers first. To verify that your change did not break this property, run `ninja dev-headers`. If you added or removed header files, you must `touch configure.py` first - this will cause `configure.py` to be automatically re-run to generate a fresh list of header files.
For more criteria on what reviewers consider good code, see the [review checklist](https://github.com/scylladb/scylla/blob/master/docs/dev/review-checklist.md).
To contribute code to Scylla, you need to sign the [Contributor License Agreement](http://www.scylladb.com/opensource/cla/) and send your changes as [patches](https://github.com/scylladb/scylla/wiki/Formatting-and-sending-patches) to the [mailing list](https://groups.google.com/forum/#!forum/scylladb-dev). We don't accept pull requests on GitHub.

View File

@@ -18,36 +18,13 @@ $ git submodule update --init --recursive
### Dependencies
Scylla is fairly fussy about its build environment, requiring a very recent
version of the C++20 compiler and numerous tools and libraries to build.
Scylla depends on the system package manager for its development dependencies.
Run `./install-dependencies.sh` (as root) to use your Linux distributions's
package manager to install the appropriate packages on your build machine.
However, this will only work on very recent distributions. For example,
currently Fedora users must upgrade to Fedora 32 otherwise the C++ compiler
will be too old, and not support the new C++20 standard that Scylla uses.
Alternatively, to avoid having to upgrade your build machine or install
various packages on it, we provide another option - the **frozen toolchain**.
This is a script, `./tools/toolchain/dbuild`, that can execute build or run
commands inside a Docker image that contains exactly the right build tools and
libraries. The `dbuild` technique is useful for beginners, but is also the way
in which ScyllaDB produces official releases, so it is highly recommended.
To use `dbuild`, you simply prefix any build or run command with it. Building
and running Scylla becomes as easy as:
```bash
$ ./tools/toolchain/dbuild ./configure.py
$ ./tools/toolchain/dbuild ninja build/release/scylla
$ ./tools/toolchain/dbuild ./build/release/scylla --developer-mode 1
```
Running `./install-dependencies.sh` (as root) installs the appropriate packages based on your Linux distribution.
### Build system
**Note**: Compiling Scylla requires, conservatively, 2 GB of memory per native
thread, and up to 3 GB per native thread while linking. GCC >= 10 is
required.
**Note**: Compiling Scylla requires, conservatively, 2 GB of memory per native thread, and up to 3 GB per native thread while linking.
Scylla is built with [Ninja](https://ninja-build.org/), a low-level rule-based system. A Python script, `configure.py`, generates a Ninja file (`build.ninja`) based on configuration options.
@@ -66,9 +43,11 @@ The full suite of options for project configuration is available via
$ ./configure.py --help
```
The most important option is:
The most important options are:
- `--enable-dpdk`: [DPDK](http://dpdk.org/) is a set of libraries and drivers for fast packet processing. During development, it's not necessary to enable support even if it is supported by your platform.
- `--mode={release,debug,all}`: Debug mode enables [AddressSanitizer](https://github.com/google/sanitizers/wiki/AddressSanitizer) and allows for debugging with tools like GDB. Debugging builds are generally slower and generate much larger object files than release builds.
- `--{enable,disable}-dpdk`: [DPDK](http://dpdk.org/) is a set of libraries and drivers for fast packet processing. During development, it's not necessary to enable support even if it is supported by your platform.
Source files and build targets are tracked manually in `configure.py`, so the script needs to be updated when new files or targets are added or removed.
@@ -76,30 +55,6 @@ To save time -- for instance, to avoid compiling all unit tests -- you can also
```bash
$ ninja-build build/release/tests/schema_change_test
$ ninja-build build/release/service/storage_proxy.o
```
You can also specify a single mode. For example
```bash
$ ninja-build release
```
Will build everytihng in release mode. The valid modes are
* Debug: Enables [AddressSanitizer](https://github.com/google/sanitizers/wiki/AddressSanitizer)
and other sanity checks. It has no optimizations, which allows for debugging with tools like
GDB. Debugging builds are generally slower and generate much larger object files than release builds.
* Release: Fewer checks and more optimizations. It still has debug info.
* Dev: No optimizations or debug info. The objective is to compile and link as fast as possible.
This is useful for the first iterations of a patch.
Note that by default unit tests binaries are stripped so they can't be used with gdb or seastar-addr2line.
To include debug information in the unit test binary, build the test binary with a `_g` suffix. For example,
```bash
$ ninja-build build/release/tests/schema_change_test_g
```
### Unit testing
@@ -128,7 +83,7 @@ The `-c1 -m1G` arguments limit this Seastar-based test to a single system thread
### Preparing patches
All changes to Scylla are submitted as patches to the public [mailing list](mailto:scylladb-dev@googlegroups.com). Once a patch is approved by one of the maintainers of the project, it is committed to the maintainers' copy of the repository at https://github.com/scylladb/scylla.
All changes to Scylla are submitted as patches to the public mailing list. Once a patch is approved by one of the maintainers of the project, it is committed to the maintainers' copy of the repository at https://github.com/scylladb/scylla.
Detailed instructions for formatting patches for the mailing list and advice on preparing good patches are available at the [ScyllaDB website](http://docs.scylladb.com/contribute/). There are also some guidelines that can help you make the patch review process smoother:
@@ -153,12 +108,10 @@ In v3:
"Tests: unit ({mode}), dtest ({smp})"
```
The usual is "Tests: unit (dev)", although running debug tests is encouraged.
The usual is "Tests: unit (release)", although running debug tests is encouraged.
5. When answering review comments, prefer inline quotes as they make it easier to track the conversation across multiple e-mails.
6. The Linux kernel's [Submitting Patches](https://www.kernel.org/doc/html/v4.19/process/submitting-patches.html) document offers excellent advice on how to prepare patches and patchsets for review. Since the Scylla development process is derived from the kernel's, almost all of the advice there is directly applicable.
### Finding a person to review and merge your patches
You can use the `scripts/find-maintainer` script to find a subsystem maintainer and/or reviewer for your patches. The script accepts a filename in the git source tree as an argument and outputs a list of subsystems the file belongs to and their respective maintainers and reviewers. For example, if you changed the `cql3/statements/create_view_statement.hh` file, run the script as follows:
@@ -172,8 +125,12 @@ and you will get output like this:
```
CQL QUERY LANGUAGE
Tomasz Grabiec <tgrabiec@scylladb.com> [maintainer]
Pekka Enberg <penberg@scylladb.com> [maintainer]
MATERIALIZED VIEWS
Pekka Enberg <penberg@scylladb.com> [maintainer]
Duarte Nunes <duarte@scylladb.com> [maintainer]
Nadav Har'El <nyh@scylladb.com> [reviewer]
Duarte Nunes <duarte@scylladb.com> [reviewer]
```
### Running Scylla
@@ -195,7 +152,7 @@ $ # Edit configuration options as appropriate
$ SCYLLA_HOME=$HOME/scylla build/release/scylla
```
The `scylla.yaml` file in the repository by default writes all database data to `/var/lib/scylla`, which likely requires root access. Change the `data_file_directories`, `commitlog_directory` and `schema_commitlog_directory` fields as appropriate.
The `scylla.yaml` file in the repository by default writes all database data to `/var/lib/scylla`, which likely requires root access. Change the `data_file_directories` and `commitlog_directory` fields as appropriate.
Scylla has a number of requirements for the file-system and operating system to operate ideally and at peak performance. However, during development, these requirements can be relaxed with the `--developer-mode` flag.
@@ -207,29 +164,6 @@ On a development machine, one might run Scylla as
$ SCYLLA_HOME=$HOME/scylla build/release/scylla --overprovisioned --developer-mode=yes
```
To interact with scylla it is recommended to build our versions of
cqlsh and nodetool. They are available at
https://github.com/scylladb/scylla-tools-java and can be built with
```bash
$ sudo ./install-dependencies.sh
$ ant jar
```
cqlsh should work out of the box, but nodetool depends on a running
scylla-jmx (https://github.com/scylladb/scylla-jmx). It can be build
with
```bash
$ mvn package
```
and must be started with
```bash
$ ./scripts/scylla-jmx
```
### Branches and tags
Multiple release branches are maintained on the Git repository at https://github.com/scylladb/scylla. Release 1.5, for instance, is tracked on the `branch-1.5` branch.
@@ -320,7 +254,7 @@ In this example, `10.0.0.2` will be sent up to 16 jobs and the local machine wil
When a compilation is in progress, the status of jobs on all remote machines can be visualized in the terminal with `distccmon-text` or graphically as a GTK application with `distccmon-gnome`.
One thing to keep in mind is that linking object files happens on the coordinating machine, which can be a bottleneck. See the next sections speeding up this process.
One thing to keep in mind is that linking object files happens on the coordinating machine, which can be a bottleneck. See the next section speeding up this process.
### Using the `gold` linker
@@ -330,24 +264,6 @@ Linking Scylla can be slow. The gold linker can replace GNU ld and often speeds
$ sudo alternatives --config ld
```
### Using split dwarf
With debug info enabled, most of the link time is spent copying and
relocating it. It is possible to leave most of the debug info out of
the link by writing it to a side .dwo file. This is done by passing
`-gsplit-dwarf` to gcc.
Unfortunately just `-gsplit-dwarf` would slow down `gdb` startup. To
avoid that the gold linker can be told to create an index with
`--gdb-index`.
More info at https://gcc.gnu.org/wiki/DebugFission.
Both options can be enable by passing `--split-dwarf` to configure.py.
Note that distcc is *not* compatible with it, but icecream
(https://github.com/icecc/icecream) is.
### Testing changes in Seastar with Scylla
Sometimes Scylla development is closely tied with a feature being developed in Seastar. It can be useful to compile Scylla with a particular check-out of Seastar.
@@ -361,62 +277,3 @@ $ git remote add local /home/tsmith/src/seastar
$ git remote update
$ git checkout -t local/my_local_seastar_branch
```
### Generating code coverage report
Install dependencies:
$ dnf install llvm # for llvm-profdata and llvm-cov
$ dnf install lcov # for genhtml
Instruct `configure.py` to generate build files for `coverage` mode:
$ ./configure.py --mode=coverage
Build the tests you want to run, then run them via `test.py` (important!):
$ ./test.py --mode=coverage [...]
Alternatively, you can run individual tests via `./scripts/coverage.py --run`.
Open the link printed at the end. Be horrified. Go and write more tests.
For more details see `./scripts/coverage.py --help`.
### Resolving stack backtraces
Scylla may print stack backtraces to the log for several reasons.
For example:
- When aborting (e.g. due to assertion failure, internal error, or segfault)
- When detecting seastar reactor stalls (where a seastar task runs for a long time without yielding the cpu to other tasks on that shard)
The backtraces contain code pointers so they are not very helpful without resolving into code locations.
To resolve the backtraces, one needs the scylla relocatable package that contains the scylla binary (with debug information),
as well as the dynamic libraries it is linked against.
Builds from our automated build system are uploaded to the cloud
and can be searched on http://backtrace.scylladb.com/
Make sure you have the scylla server exact `build-id` to locate
its respective relocatable package, required for decoding backtraces it prints.
The build-id is printed to the system log when scylla starts.
It can also be found by executing `scylla --build-id`, or
by using the `file` utility, for example:
```
$ scylla --build-id
4cba12e6eb290a406bfa4930918db23941fd4be3
$ file scylla
scylla: ELF 64-bit LSB executable, x86-64, version 1 (SYSV), dynamically linked, interpreter /////////////////////////////////////////////////////////////////////////////////////////////////////////////////////////////////////////////////////////////////////////////////////////////////////////////////////////////////////////////////////////////////////////////////////////////////////////////////////////////////////////////////////////////////////////////////////////////////////////////////////////////////////////////////////////////////////////////////////////////////////lib64/ld-linux-x86-64.so.2, for GNU/Linux 3.2.0, BuildID[sha1]=4cba12e6eb290a406bfa4930918db23941fd4be3, with debug_info, not stripped, too many notes (256)
```
To find the build-id of a coredump, use the `eu-unstrip` utility as follows:
```
$ eu-unstrip -n --core <coredump> | awk '/scylla$/ { s=$2; sub(/@.*$/, "", s); print s; exit(0); }'
4cba12e6eb290a406bfa4930918db23941fd4be3
```
### Core dump debugging
See [debugging.md](docs/dev/debugging.md).

103
IDL.md Normal file
View File

@@ -0,0 +1,103 @@
#IDL definition
The schema we use similar to c++ schema.
Use class or struct similar to the object you need the serializer for.
Use namespace when applicable.
##keywords
* class/struct - a class or a struct like C++
class/struct can have final or stub marker
* namespace - has the same C++ meaning
* enum class - has the same C++ meaning
* final modifier for class - when a class mark as final it will not contain a size parameter. Note that final class cannot be extended by future version, so use with care
* stub class - when a class is mark as stub, it means that no code will be generated for this class and it is only there as a documentation.
* version attributes - mark with [[version id ]] mark that a field is available from a specific version
* template - A template class definition like C++
##Syntax
###Namespace
```
namespace ns_name { namespace-body }
```
* ns_name: either a previously unused identifier, in which case this is original-namespace-definition or the name of a namespace, in which case this is extension-namespace-definition
* namespace-body: possibly empty sequence of declarations of any kind (including class and struct definitions as well as nested namespaces)
###class/struct
`
class-key class-name final(optional) stub(optional) { member-specification } ;(optional)
`
* class-key: one of class or struct.
* class-name: the name of the class that's being defined. optionally followed by keyword final, optionally followed by keyword stub
* final: when a class mark as final, it means it can not be extended and there is no need to serialize its size, use with care.
* stub: when a class is mark as stub, it means no code will generate for it and it is added for documentation only.
* member-specification: list of access specifiers, and public member accessor see class member below.
* to be compatible with C++ a class definition can be followed by a semicolon.
###enum
`enum-key identifier enum-base { enumerator-list(optional) }`
* enum-key: only enum class is supported
* identifier: the name of the enumeration that's being declared.
* enum-base: colon (:), followed by a type-specifier-seq that names an integral type (see the C++ standard for the full list of all possible integral types).
* enumerator-list: comma-separated list of enumerator definitions, each of which is either simply an identifier, which becomes the name of the enumerator, or an identifier with an initializer: identifier = integral value.
Note that though C++ allows constexpr as an initialize value, it makes the documentation less readable, hence is not permitted.
###class member
`type member-access attributes(optional) default-value(optional);`
* type: Any valid C++ type, following the C++ notation. note that there should be a serializer for the type, but deceleration order is not mandatory
* member-access: is the way the member can be access. If the member is public it can be the name itself. if not it could be a getter function that should be followed by braces. Note that getter can (and probably should) be const methods.
* attributes: Attributes define by square brackets. Currently are use to mark a version in which a specific member was added [ [ version version-number] ] would mark that the specific member was added in the given version number.
###template
`template < parameter-list > class-declaration`
* parameter-list - a non-empty comma-separated list of the template parameters.
* class-decleration - (See class section) The class name declared become a template name.
##IDL example
Forward slashes comments are ignored until the end of the line.
```
namespace utils {
// An example of a stub class
class UUID stub {
int64_t most_sig_bits;
int64_t least_sig_bits;
}
}
namespace gms {
//an enum example
enum class application_state:int {STATUS = 0,
LOAD,
SCHEMA,
DC};
// example of final class
class versioned_value final {
// getter and setter as public member
int version;
sstring value;
}
class heart_beat_state {
//getter as function
int32_t get_generation();
//default value example
int32_t get_heart_beat_version() = 1;
}
class endpoint_state {
heart_beat_state get_heart_beat_state();
std::map<application_state, versioned_value> get_application_state_map();
}
class gossip_digest {
inet_address get_endpoint();
int32_t get_generation();
//mark that a field was added on a specific version
int32_t get_max_version() [ [version 0.14.2] ];
}
class gossip_digest_ack {
std::vector<gossip_digest> digests();
std::map<inet_address, gms::endpoint_state> get_endpoint_state_map();
}
}
```

131
MAINTAINERS Normal file
View File

@@ -0,0 +1,131 @@
M: Maintainer with commit access
R: Reviewer with subsystem expertise
F: Filename, directory, or pattern for the subsystem
---
AUTH
M: Paweł Dziepak <pdziepak@scylladb.com>
M: Duarte Nunes <duarte@scylladb.com>
R: Calle Wilund <calle@scylladb.com>
R: Vlad Zolotarov <vladz@scylladb.com>
R: Jesse Haber-Kucharsky <jhaberku@scylladb.com>
F: auth/*
CACHE
M: Tomasz Grabiec <tgrabiec@scylladb.com>
M: Paweł Dziepak <pdziepak@scylladb.com>
R: Piotr Jastrzebski <piotr@scylladb.com>
F: row_cache*
F: *mutation*
F: tests/mvcc*
COMMITLOG / BATCHLOGa
M: Paweł Dziepak <pdziepak@scylladb.com>
M: Duarte Nunes <duarte@scylladb.com>
R: Calle Wilund <calle@scylladb.com>
F: db/commitlog/*
F: db/batch*
COORDINATOR
M: Paweł Dziepak <pdziepak@scylladb.com>
M: Duarte Nunes <duarte@scylladb.com>
R: Gleb Natapov <gleb@scylladb.com>
F: service/storage_proxy*
COMPACTION
R: Raphael S. Carvalho <raphaelsc@scylladb.com>
R: Glauber Costa <glauber@scylladb.com>
R: Nadav Har'El <nyh@scylladb.com>
F: sstables/compaction*
CQL TRANSPORT LAYER
M: Pekka Enberg <penberg@scylladb.com>
F: transport/*
CQL QUERY LANGUAGE
M: Tomasz Grabiec <tgrabiec@scylladb.com>
M: Pekka Enberg <penberg@scylladb.com>
F: cql3/*
COUNTERS
M: Paweł Dziepak <pdziepak@scylladb.com>
F: counters*
F: tests/counter_test*
GOSSIP
M: Duarte Nunes <duarte@scylladb.com>
M: Tomasz Grabiec <tgrabiec@scylladb.com>
R: Asias He <asias@scylladb.com>
F: gms/*
DOCKER
M: Pekka Enberg <penberg@scylladb.com>
F: dist/docker/*
LSA
M: Tomasz Grabiec <tgrabiec@scylladb.com>
M: Paweł Dziepak <pdziepak@scylladb.com>
F: utils/logalloc*
MATERIALIZED VIEWS
M: Duarte Nunes <duarte@scylladb.com>
M: Pekka Enberg <penberg@scylladb.com>
R: Nadav Har'El <nyh@scylladb.com>
R: Duarte Nunes <duarte@scylladb.com>
F: db/view/*
F: cql3/statements/*view*
PACKAGING
R: Takuya ASADA <syuu@scylladb.com>
F: dist/*
REPAIR
M: Tomasz Grabiec <tgrabiec@scylladb.com>
M: Duarte Nunes <duarte@scylladb.com>
R: Asias He <asias@scylladb.com>
R: Nadav Har'El <nyh@scylladb.com>
F: repair/*
SCHEMA MANAGEMENT
M: Tomasz Grabiec <tgrabiec@scylladb.com>
M: Duarte Nunes <duarte@scylladb.com>
M: Pekka Enberg <penberg@scylladb.com>
F: db/schema_tables*
F: db/legacy_schema_migrator*
F: service/migration*
F: schema*
SECONDARY INDEXES
M: Pekka Enberg <penberg@scylladb.com>
M: Duarte Nunes <duarte@scylladb.com>
R: Nadav Har'El <nyh@scylladb.com>
R: Pekka Enberg <penberg@scylladb.com>
F: db/index/*
F: cql3/statements/*index*
SSTABLES
M: Tomasz Grabiec <tgrabiec@scylladb.com>
M: Duarte Nunes <duarte@scylladb.com>
R: Raphael S. Carvalho <raphaelsc@scylladb.com>
R: Glauber Costa <glauber@scylladb.com>
R: Nadav Har'El <nyh@scylladb.com>
F: sstables/*
STREAMING
M: Tomasz Grabiec <tgrabiec@scylladb.com>
M: Duarte Nunes <duarte@scylladb.com>
R: Asias He <asias@scylladb.com>
F: streaming/*
F: service/storage_service.*
THRIFT TRANSPORT LAYER
M: Duarte Nunes <duarte@scylladb.com>
F: thrift/*
THE REST
M: Avi Kivity <avi@scylladb.com>
M: Paweł Dziepak <pdziepak@scylladb.com>
M: Duarte Nunes <duarte@scylladb.com>
M: Tomasz Grabiec <tgrabiec@scylladb.com>
F: *

View File

@@ -1,11 +1,5 @@
This project includes code developed by the Apache Software Foundation (http://www.apache.org/),
especially Apache Cassandra.
It includes files from https://github.com/antonblanchard/crc32-vpmsum (author Anton Blanchard <anton@au.ibm.com>, IBM).
It also includes files from https://github.com/antonblanchard/crc32-vpmsum (author Anton Blanchard <anton@au.ibm.com>, IBM).
These files are located in utils/arch/powerpc/crc32-vpmsum. Their license may be found in licenses/LICENSE-crc32-vpmsum.TXT.
It includes modified code from https://gitbox.apache.org/repos/asf?p=cassandra-dtest.git (owned by The Apache Software Foundation)
It includes modified tests from https://github.com/etcd-io/etcd.git (owned by The etcd Authors)
It includes files from https://github.com/bytecodealliance/wasmtime-cpp (owned by Bytecode Alliance), licensed with Apache License 2.0.

29
README-DPDK.md Normal file
View File

@@ -0,0 +1,29 @@
Seastar and DPDK
================
Seastar uses the Data Plane Development Kit to drive NIC hardware directly. This
provides an enormous performance boost.
To enable DPDK, specify `--enable-dpdk` to `./configure.py`, and `--dpdk-pmd` as a
run-time parameter. This will use the DPDK package provided as a git submodule with the
seastar sources.
To use your own self-compiled DPDK package, follow this procedure:
1. Setup host to compile DPDK:
- Ubuntu
`sudo apt-get install -y build-essential linux-image-extra-$(uname -r)`
2. Prepare a DPDK SDK:
- Download the latest DPDK release: `wget http://dpdk.org/browse/dpdk/snapshot/dpdk-1.8.0.tar.gz`
- Untar it.
- Edit config/common_linuxapp: set CONFIG_RTE_MBUF_REFCNT and CONFIG_RTE_LIBRTE_KNI to 'n'.
- For DPDK 1.7.x: edit config/common_linuxapp:
- Set CONFIG_RTE_LIBRTE_PMD_BOND to 'n'.
- Set CONFIG_RTE_MBUF_SCATTER_GATHER to 'n'.
- Set CONFIG_RTE_LIBRTE_IP_FRAG to 'n'.
- Start the tools/setup.sh script as root.
- Compile a linuxapp target (option 9).
- Install IGB_UIO module (option 11).
- Bind some physical port to IGB_UIO (option 17).
- Configure hugepage mappings (option 14/15).
3. Run a configure.py: `./configure.py --dpdk-target <Path to untared dpdk-1.8.0 above>/x86_64-native-linuxapp-gcc`.

140
README.md
View File

@@ -1,110 +1,78 @@
# Scylla
[![Slack](https://img.shields.io/badge/slack-scylla-brightgreen.svg?logo=slack)](http://slack.scylladb.com)
[![Twitter](https://img.shields.io/twitter/follow/ScyllaDB.svg?style=social&label=Follow)](https://twitter.com/intent/follow?screen_name=ScyllaDB)
## What is Scylla?
Scylla is the real-time big data database that is API-compatible with Apache Cassandra and Amazon DynamoDB.
Scylla embraces a shared-nothing approach that increases throughput and storage capacity to realize order-of-magnitude performance improvements and reduce hardware costs.
For more information, please see the [ScyllaDB web site].
[ScyllaDB web site]: https://www.scylladb.com
## Build Prerequisites
Scylla is fairly fussy about its build environment, requiring very recent
versions of the C++20 compiler and of many libraries to build. The document
[HACKING.md](HACKING.md) includes detailed information on building and
developing Scylla, but to get Scylla building quickly on (almost) any build
machine, Scylla offers a [frozen toolchain](tools/toolchain/README.md),
This is a pre-configured Docker image which includes recent versions of all
the required compilers, libraries and build tools. Using the frozen toolchain
allows you to avoid changing anything in your build machine to meet Scylla's
requirements - you just need to meet the frozen toolchain's prerequisites
(mostly, Docker or Podman being available).
## Building Scylla
Building Scylla with the frozen toolchain `dbuild` is as easy as:
## Quick-start
```bash
$ git submodule update --init --force --recursive
$ ./tools/toolchain/dbuild ./configure.py
$ ./tools/toolchain/dbuild ninja build/release/scylla
$ git submodule update --init --recursive
$ sudo ./install-dependencies.sh
$ ./configure.py --mode=release
$ ninja-build -j4 # Assuming 4 system threads.
$ ./build/release/scylla
$ # Rejoice!
```
For further information, please see:
* [Developer documentation] for more information on building Scylla.
* [Build documentation] on how to build Scylla binaries, tests, and packages.
* [Docker image build documentation] for information on how to build Docker images.
[developer documentation]: HACKING.md
[build documentation]: docs/dev/building.md
[docker image build documentation]: dist/docker/debian/README.md
Please see [HACKING.md](HACKING.md) for detailed information on building and developing Scylla.
## Running Scylla
To start Scylla server, run:
* Run Scylla
```
./build/release/scylla
```bash
$ ./tools/toolchain/dbuild ./build/release/scylla --workdir tmp --smp 1 --developer-mode 1
```
This will start a Scylla node with one CPU core allocated to it and data files stored in the `tmp` directory.
The `--developer-mode` is needed to disable the various checks Scylla performs at startup to ensure the machine is configured for maximum performance (not relevant on development workstations).
Please note that you need to run Scylla with `dbuild` if you built it with the frozen toolchain.
* run Scylla with one CPU and ./tmp as data directory
For more run options, run:
```bash
$ ./tools/toolchain/dbuild ./build/release/scylla --help
```
./build/release/scylla --datadir tmp --commitlog-directory tmp --smp 1
```
## Testing
* For more run options:
```
./build/release/scylla --help
```
See [test.py manual](docs/dev/testing.md).
## Building Fedora RPM
## Scylla APIs and compatibility
By default, Scylla is compatible with Apache Cassandra and its APIs - CQL and
Thrift. There is also support for the API of Amazon DynamoDB™,
which needs to be enabled and configured in order to be used. For more
information on how to enable the DynamoDB™ API in Scylla,
and the current compatibility of this feature as well as Scylla-specific extensions, see
[Alternator](docs/alternator/alternator.md) and
[Getting started with Alternator](docs/alternator/getting-started.md).
As a pre-requisite, you need to install [Mock](https://fedoraproject.org/wiki/Mock) on your machine:
## Documentation
```
# Install mock:
sudo yum install mock
Documentation can be found [here](docs/dev/README.md).
Seastar documentation can be found [here](http://docs.seastar.io/master/index.html).
User documentation can be found [here](https://docs.scylladb.com/).
# Add user to the "mock" group:
usermod -a -G mock $USER && newgrp mock
```
## Training
Then, to build an RPM, run:
Training material and online courses can be found at [Scylla University](https://university.scylladb.com/).
The courses are free, self-paced and include hands-on examples. They cover a variety of topics including Scylla data modeling,
administration, architecture, basic NoSQL concepts, using drivers for application development, Scylla setup, failover, compactions,
multi-datacenters and how Scylla integrates with third-party applications.
```
./dist/redhat/build_rpm.sh
```
The built RPM is stored in the ``build/mock/<configuration>/result`` directory.
For example, on Fedora 21 mock reports the following:
```
INFO: Done(scylla-server-0.00-1.fc21.src.rpm) Config(default) 20 minutes 7 seconds
INFO: Results and/or logs in: build/mock/fedora-21-x86_64/result
```
## Building Fedora-based Docker image
Build a Docker image with:
```
cd dist/docker
docker build -t <image-name> .
```
Run the image with:
```
docker run -p $(hostname -i):9042:9042 -i -t <image name>
```
## Contributing to Scylla
If you want to report a bug or submit a pull request or a patch, please read the [contribution guidelines].
If you are a developer working on Scylla, please read the [developer guidelines].
[contribution guidelines]: CONTRIBUTING.md
[developer guidelines]: HACKING.md
## Contact
* The [community forum] and [Slack channel] are for users to discuss configuration, management, and operations of the ScyllaDB open source.
* The [developers mailing list] is for developers and people interested in following the development of ScyllaDB to discuss technical topics.
[Community forum]: https://forum.scylladb.com/
[Slack channel]: http://slack.scylladb.com/
[Developers mailing list]: https://groups.google.com/forum/#!forum/scylladb-dev
[Guidelines for contributing](CONTRIBUTING.md)

View File

@@ -1,109 +1,24 @@
#!/bin/sh
USAGE=$(cat <<-END
Usage: $(basename "$0") [-h|--help] [-o|--output-dir PATH] [--date-stamp DATE] -- generate Scylla version and build information files.
Options:
-h|--help show this help message.
-o|--output-dir PATH specify destination path at which the version files are to be created.
-d|--date-stamp DATE manually set date for release parameter
By default, the script will attempt to parse 'version' file
in the current directory, which should contain a string of
'\$version-\$release' form.
Otherwise, it will call 'git log' on the source tree (the
directory, which contains the script) to obtain current
commit hash and use it for building the version and release
strings.
The script assumes that it's called from the Scylla source
tree.
The files created are:
SCYLLA-VERSION-FILE
SCYLLA-RELEASE-FILE
SCYLLA-PRODUCT-FILE
By default, these files are created in the 'build'
subdirectory under the directory containing the script.
The destination directory can be overriden by
using '-o PATH' option.
END
)
DATE=""
while [ $# -gt 0 ]; do
opt="$1"
case $opt in
-h|--help)
echo "$USAGE"
exit 0
;;
-o|--output-dir)
OUTPUT_DIR="$2"
shift
shift
;;
--date-stamp)
DATE="$2"
shift
shift
;;
*)
echo "Unexpected argument found: $1"
echo
echo "$USAGE"
exit 1
;;
esac
done
SCRIPT_DIR="$(dirname "$0")"
if [ -z "$OUTPUT_DIR" ]; then
OUTPUT_DIR="$SCRIPT_DIR/build"
fi
if [ -z "$DATE" ]; then
DATE=$(date --utc +%Y%m%d)
fi
# Default scylla product/version tags
PRODUCT=scylla
VERSION=5.3.0-dev
VERSION=3.0.2
if test -f version
then
SCYLLA_VERSION=$(cat version | awk -F'-' '{print $1}')
SCYLLA_RELEASE=$(cat version | awk -F'-' '{print $2}')
else
DATE=$(date +%Y%m%d)
GIT_COMMIT=$(git log --pretty=format:'%h' -n 1)
SCYLLA_VERSION=$VERSION
if [ -z "$SCYLLA_RELEASE" ]; then
DATE=$(date --utc +%Y%m%d)
GIT_COMMIT=$(git -C "$SCRIPT_DIR" log --pretty=format:'%h' -n 1 --abbrev=12)
# For custom package builds, replace "0" with "counter.your_name",
# where counter starts at 1 and increments for successive versions.
# This ensures that the package manager will select your custom
# package over the standard release.
SCYLLA_BUILD=0
SCYLLA_RELEASE=$SCYLLA_BUILD.$DATE.$GIT_COMMIT
elif [ -f "$OUTPUT_DIR/SCYLLA-RELEASE-FILE" ]; then
echo "setting SCYLLA_RELEASE only makes sense in clean builds" 1>&2
exit 1
fi
fi
if [ -f "$OUTPUT_DIR/SCYLLA-RELEASE-FILE" ]; then
GIT_COMMIT_FILE=$(cat "$OUTPUT_DIR/SCYLLA-RELEASE-FILE" |cut -d . -f 3)
if [ "$GIT_COMMIT" = "$GIT_COMMIT_FILE" ]; then
exit 0
fi
# For custom package builds, replace "0" with "counter.your_name",
# where counter starts at 1 and increments for successive versions.
# This ensures that the package manager will select your custom
# package over the standard release.
SCYLLA_BUILD=0
SCYLLA_RELEASE=$SCYLLA_BUILD.$DATE.$GIT_COMMIT
fi
echo "$SCYLLA_VERSION-$SCYLLA_RELEASE"
mkdir -p "$OUTPUT_DIR"
echo "$SCYLLA_VERSION" > "$OUTPUT_DIR/SCYLLA-VERSION-FILE"
echo "$SCYLLA_RELEASE" > "$OUTPUT_DIR/SCYLLA-RELEASE-FILE"
echo "$PRODUCT" > "$OUTPUT_DIR/SCYLLA-PRODUCT-FILE"
mkdir -p build
echo "$SCYLLA_VERSION" > build/SCYLLA-VERSION-FILE
echo "$SCYLLA_RELEASE" > build/SCYLLA-RELEASE-FILE

View File

@@ -1,13 +0,0 @@
/*
* Copyright (C) 2020-present ScyllaDB
*/
/*
* SPDX-License-Identifier: AGPL-3.0-or-later
*/
#include "absl-flat_hash_map.hh"
size_t sstring_hash::operator()(std::string_view v) const noexcept {
return absl::Hash<std::string_view>{}(v);
}

View File

@@ -1,34 +0,0 @@
/*
* Copyright (C) 2020-present ScyllaDB
*/
/*
* SPDX-License-Identifier: AGPL-3.0-or-later
*/
#pragma once
#include <absl/container/flat_hash_map.h>
#include <seastar/core/sstring.hh>
using namespace seastar;
struct sstring_hash {
using is_transparent = void;
size_t operator()(std::string_view v) const noexcept;
};
struct sstring_eq {
using is_transparent = void;
bool operator()(std::string_view a, std::string_view b) const noexcept {
return a == b;
}
};
template <typename K, typename V, typename... Ts>
struct flat_hash_map : public absl::flat_hash_map<K, V, Ts...> {
};
template <typename V>
struct flat_hash_map<sstring, V>
: public absl::flat_hash_map<sstring, V, sstring_hash, sstring_eq> {};

View File

@@ -1,30 +0,0 @@
include(generate_cql_grammar)
generate_cql_grammar(
GRAMMAR expressions.g
SOURCES cql_grammar_srcs)
add_library(alternator STATIC)
target_sources(alternator
PRIVATE
controller.cc
server.cc
executor.cc
stats.cc
serialization.cc
expressions.cc
conditions.cc
auth.cc
streams.cc
ttl.cc
${cql_grammar_srcs})
target_include_directories(alternator
PUBLIC
${CMAKE_SOURCE_DIR}
${CMAKE_BINARY_DIR}
PRIVATE
${RAPIDJSON_INCLUDE_DIRS})
target_link_libraries(alternator
cql3
idl
Seastar::seastar
xxHash::xxhash)

View File

@@ -1,63 +0,0 @@
/*
* Copyright 2019-present ScyllaDB
*/
/*
* SPDX-License-Identifier: AGPL-3.0-or-later
*/
#include "alternator/error.hh"
#include "log.hh"
#include <string>
#include <string_view>
#include "bytes.hh"
#include "alternator/auth.hh"
#include <fmt/format.h>
#include "auth/common.hh"
#include "auth/password_authenticator.hh"
#include "auth/roles-metadata.hh"
#include "service/storage_proxy.hh"
#include "alternator/executor.hh"
#include "cql3/selection/selection.hh"
#include "query-result-set.hh"
#include "cql3/result_set.hh"
#include <seastar/core/coroutine.hh>
namespace alternator {
static logging::logger alogger("alternator-auth");
future<std::string> get_key_from_roles(service::storage_proxy& proxy, std::string username) {
schema_ptr schema = proxy.data_dictionary().find_schema("system_auth", "roles");
partition_key pk = partition_key::from_single_value(*schema, utf8_type->decompose(username));
dht::partition_range_vector partition_ranges{dht::partition_range(dht::decorate_key(*schema, pk))};
std::vector<query::clustering_range> bounds{query::clustering_range::make_open_ended_both_sides()};
const column_definition* salted_hash_col = schema->get_column_definition(bytes("salted_hash"));
if (!salted_hash_col) {
co_await coroutine::return_exception(api_error::unrecognized_client(format("Credentials cannot be fetched for: {}", username)));
}
auto selection = cql3::selection::selection::for_columns(schema, {salted_hash_col});
auto partition_slice = query::partition_slice(std::move(bounds), {}, query::column_id_vector{salted_hash_col->id}, selection->get_query_options());
auto command = ::make_lw_shared<query::read_command>(schema->id(), schema->version(), partition_slice,
proxy.get_max_result_size(partition_slice), query::tombstone_limit(proxy.get_tombstone_limit()));
auto cl = auth::password_authenticator::consistency_for_user(username);
service::client_state client_state{service::client_state::internal_tag()};
service::storage_proxy::coordinator_query_result qr = co_await proxy.query(schema, std::move(command), std::move(partition_ranges), cl,
service::storage_proxy::coordinator_query_options(executor::default_timeout(), empty_service_permit(), client_state));
cql3::selection::result_set_builder builder(*selection, gc_clock::now());
query::result_view::consume(*qr.query_result, partition_slice, cql3::selection::result_set_builder::visitor(builder, *schema, *selection));
auto result_set = builder.build();
if (result_set->empty()) {
co_await coroutine::return_exception(api_error::unrecognized_client(format("User not found: {}", username)));
}
const bytes_opt& salted_hash = result_set->rows().front().front(); // We only asked for 1 row and 1 column
if (!salted_hash) {
co_await coroutine::return_exception(api_error::unrecognized_client(format("No password found for user: {}", username)));
}
co_return value_cast<sstring>(utf8_type->deserialize(*salted_hash));
}
}

View File

@@ -1,27 +0,0 @@
/*
* Copyright 2019-present ScyllaDB
*/
/*
* SPDX-License-Identifier: AGPL-3.0-or-later
*/
#pragma once
#include <string>
#include <string_view>
#include <array>
#include "gc_clock.hh"
#include "utils/loading_cache.hh"
namespace service {
class storage_proxy;
}
namespace alternator {
using key_cache = utils::loading_cache<std::string, std::string, 1>;
future<std::string> get_key_from_roles(service::storage_proxy& proxy, std::string username);
}

View File

@@ -1,761 +0,0 @@
/*
* Copyright 2019-present ScyllaDB
*/
/*
* SPDX-License-Identifier: AGPL-3.0-or-later
*/
#include <list>
#include <map>
#include <string_view>
#include "alternator/conditions.hh"
#include "alternator/error.hh"
#include "cql3/constants.hh"
#include <unordered_map>
#include "utils/rjson.hh"
#include "serialization.hh"
#include "utils/base64.hh"
#include "utils/rjson.hh"
#include <stdexcept>
#include <boost/algorithm/cxx11/all_of.hpp>
#include <boost/algorithm/cxx11/any_of.hpp>
#include "utils/overloaded_functor.hh"
#include "expressions.hh"
namespace alternator {
static logging::logger clogger("alternator-conditions");
comparison_operator_type get_comparison_operator(const rjson::value& comparison_operator) {
static std::unordered_map<std::string, comparison_operator_type> ops = {
{"EQ", comparison_operator_type::EQ},
{"NE", comparison_operator_type::NE},
{"LE", comparison_operator_type::LE},
{"LT", comparison_operator_type::LT},
{"GE", comparison_operator_type::GE},
{"GT", comparison_operator_type::GT},
{"IN", comparison_operator_type::IN},
{"NULL", comparison_operator_type::IS_NULL},
{"NOT_NULL", comparison_operator_type::NOT_NULL},
{"BETWEEN", comparison_operator_type::BETWEEN},
{"BEGINS_WITH", comparison_operator_type::BEGINS_WITH},
{"CONTAINS", comparison_operator_type::CONTAINS},
{"NOT_CONTAINS", comparison_operator_type::NOT_CONTAINS},
};
if (!comparison_operator.IsString()) {
throw api_error::validation(format("Invalid comparison operator definition {}", rjson::print(comparison_operator)));
}
std::string op = comparison_operator.GetString();
auto it = ops.find(op);
if (it == ops.end()) {
throw api_error::validation(format("Unsupported comparison operator {}", op));
}
return it->second;
}
namespace {
struct size_check {
// True iff size passes this check.
virtual bool operator()(rapidjson::SizeType size) const = 0;
// Check description, such that format("expected array {}", check.what()) is human-readable.
virtual sstring what() const = 0;
};
class exact_size : public size_check {
rapidjson::SizeType _expected;
public:
explicit exact_size(rapidjson::SizeType expected) : _expected(expected) {}
bool operator()(rapidjson::SizeType size) const override { return size == _expected; }
sstring what() const override { return format("of size {}", _expected); }
};
struct empty : public size_check {
bool operator()(rapidjson::SizeType size) const override { return size < 1; }
sstring what() const override { return "to be empty"; }
};
struct nonempty : public size_check {
bool operator()(rapidjson::SizeType size) const override { return size > 0; }
sstring what() const override { return "to be non-empty"; }
};
} // anonymous namespace
// Check that array has the expected number of elements
static void verify_operand_count(const rjson::value* array, const size_check& expected, const rjson::value& op) {
if (!array && expected(0)) {
// If expected() allows an empty AttributeValueList, it is also fine
// that it is missing.
return;
}
if (!array || !array->IsArray()) {
throw api_error::validation("With ComparisonOperator, AttributeValueList must be given and an array");
}
if (!expected(array->Size())) {
throw api_error::validation(
format("{} operator requires AttributeValueList {}, instead found list size {}",
op, expected.what(), array->Size()));
}
}
struct rjson_engaged_ptr_comp {
bool operator()(const rjson::value* p1, const rjson::value* p2) const {
return rjson::single_value_comp()(*p1, *p2);
}
};
// It's not enough to compare underlying JSON objects when comparing sets,
// as internally they're stored in an array, and the order of elements is
// not important in set equality. See issue #5021
static bool check_EQ_for_sets(const rjson::value& set1, const rjson::value& set2) {
if (!set1.IsArray() || !set2.IsArray() || set1.Size() != set2.Size()) {
return false;
}
std::set<const rjson::value*, rjson_engaged_ptr_comp> set1_raw;
for (auto it = set1.Begin(); it != set1.End(); ++it) {
set1_raw.insert(&*it);
}
for (const auto& a : set2.GetArray()) {
if (!set1_raw.contains(&a)) {
return false;
}
}
return true;
}
// Moreover, the JSON being compared can be a nested document with outer
// layers of lists and maps and some inner set - and we need to get to that
// inner set to compare it correctly with check_EQ_for_sets() (issue #8514).
static bool check_EQ(const rjson::value* v1, const rjson::value& v2);
static bool check_EQ_for_lists(const rjson::value& list1, const rjson::value& list2) {
if (!list1.IsArray() || !list2.IsArray() || list1.Size() != list2.Size()) {
return false;
}
auto it1 = list1.Begin();
auto it2 = list2.Begin();
while (it1 != list1.End()) {
// Note: Alternator limits an item's depth (rjson::parse() limits
// it to around 37 levels), so this recursion is safe.
if (!check_EQ(&*it1, *it2)) {
return false;
}
++it1;
++it2;
}
return true;
}
static bool check_EQ_for_maps(const rjson::value& list1, const rjson::value& list2) {
if (!list1.IsObject() || !list2.IsObject() || list1.MemberCount() != list2.MemberCount()) {
return false;
}
for (auto it1 = list1.MemberBegin(); it1 != list1.MemberEnd(); ++it1) {
auto it2 = list2.FindMember(it1->name);
if (it2 == list2.MemberEnd() || !check_EQ(&it1->value, it2->value)) {
return false;
}
}
return true;
}
// Check if two JSON-encoded values match with the EQ relation
static bool check_EQ(const rjson::value* v1, const rjson::value& v2) {
if (v1 && v1->IsObject() && v1->MemberCount() == 1 && v2.IsObject() && v2.MemberCount() == 1) {
auto it1 = v1->MemberBegin();
auto it2 = v2.MemberBegin();
if (it1->name != it2->name) {
return false;
}
if (it1->name == "SS" || it1->name == "NS" || it1->name == "BS") {
return check_EQ_for_sets(it1->value, it2->value);
} else if(it1->name == "L") {
return check_EQ_for_lists(it1->value, it2->value);
} else if(it1->name == "M") {
return check_EQ_for_maps(it1->value, it2->value);
} else {
// Other, non-nested types (number, string, etc.) can be compared
// literally, comparing their JSON representation.
return it1->value == it2->value;
}
} else {
// If v1 and/or v2 are missing (IsNull()) the result should be false.
// In the unlikely case that the object is malformed (issue #8070),
// let's also return false.
return false;
}
}
// Check if two JSON-encoded values match with the NE relation
static bool check_NE(const rjson::value* v1, const rjson::value& v2) {
return !check_EQ(v1, v2);
}
// Check if two JSON-encoded values match with the BEGINS_WITH relation
bool check_BEGINS_WITH(const rjson::value* v1, const rjson::value& v2,
bool v1_from_query, bool v2_from_query) {
bool bad = false;
if (!v1 || !v1->IsObject() || v1->MemberCount() != 1) {
if (v1_from_query) {
throw api_error::validation("begins_with() encountered malformed argument");
} else {
bad = true;
}
} else if (v1->MemberBegin()->name != "S" && v1->MemberBegin()->name != "B") {
if (v1_from_query) {
throw api_error::validation(format("begins_with supports only string or binary type, got: {}", *v1));
} else {
bad = true;
}
}
if (!v2.IsObject() || v2.MemberCount() != 1) {
if (v2_from_query) {
throw api_error::validation("begins_with() encountered malformed argument");
} else {
bad = true;
}
} else if (v2.MemberBegin()->name != "S" && v2.MemberBegin()->name != "B") {
if (v2_from_query) {
throw api_error::validation(format("begins_with() supports only string or binary type, got: {}", v2));
} else {
bad = true;
}
}
if (bad) {
return false;
}
auto it1 = v1->MemberBegin();
auto it2 = v2.MemberBegin();
if (it1->name != it2->name) {
return false;
}
if (it2->name == "S") {
return rjson::to_string_view(it1->value).starts_with(rjson::to_string_view(it2->value));
} else /* it2->name == "B" */ {
try {
return base64_begins_with(rjson::to_string_view(it1->value), rjson::to_string_view(it2->value));
} catch(std::invalid_argument&) {
// determine if any of the malformed values is from query and raise an exception if so
unwrap_bytes(it1->value, v1_from_query);
unwrap_bytes(it2->value, v2_from_query);
return false;
}
}
}
static bool is_set_of(const rjson::value& type1, const rjson::value& type2) {
return (type2 == "S" && type1 == "SS") || (type2 == "N" && type1 == "NS") || (type2 == "B" && type1 == "BS");
}
// Check if two JSON-encoded values match with the CONTAINS relation
bool check_CONTAINS(const rjson::value* v1, const rjson::value& v2, bool v1_from_query, bool v2_from_query) {
if (!v1) {
return false;
}
const auto& kv1 = *v1->MemberBegin();
const auto& kv2 = *v2.MemberBegin();
if (kv1.name == "S" && kv2.name == "S") {
return rjson::to_string_view(kv1.value).find(rjson::to_string_view(kv2.value)) != std::string_view::npos;
} else if (kv1.name == "B" && kv2.name == "B") {
auto d_kv1 = unwrap_bytes(kv1.value, v1_from_query);
auto d_kv2 = unwrap_bytes(kv2.value, v2_from_query);
if (!d_kv1 || !d_kv2) {
return false;
}
return d_kv1->find(*d_kv2) != bytes::npos;
} else if (is_set_of(kv1.name, kv2.name)) {
for (auto i = kv1.value.Begin(); i != kv1.value.End(); ++i) {
if (*i == kv2.value) {
return true;
}
}
} else if (kv1.name == "L") {
for (auto i = kv1.value.Begin(); i != kv1.value.End(); ++i) {
if (!i->IsObject() || i->MemberCount() != 1) {
clogger.error("check_CONTAINS received a list whose element is malformed");
return false;
}
const auto& el = *i->MemberBegin();
if (el.name == kv2.name && el.value == kv2.value) {
return true;
}
}
}
return false;
}
// Check if two JSON-encoded values match with the NOT_CONTAINS relation
static bool check_NOT_CONTAINS(const rjson::value* v1, const rjson::value& v2, bool v1_from_query, bool v2_from_query) {
if (!v1) {
return false;
}
return !check_CONTAINS(v1, v2, v1_from_query, v2_from_query);
}
// Check if a JSON-encoded value equals any element of an array, which must have at least one element.
static bool check_IN(const rjson::value* val, const rjson::value& array) {
if (!array[0].IsObject() || array[0].MemberCount() != 1) {
throw api_error::validation(
format("IN operator encountered malformed AttributeValue: {}", array[0]));
}
const auto& type = array[0].MemberBegin()->name;
if (type != "S" && type != "N" && type != "B") {
throw api_error::validation(
"IN operator requires AttributeValueList elements to be of type String, Number, or Binary ");
}
if (!val) {
return false;
}
bool have_match = false;
for (const auto& elem : array.GetArray()) {
if (!elem.IsObject() || elem.MemberCount() != 1 || elem.MemberBegin()->name != type) {
throw api_error::validation(
"IN operator requires all AttributeValueList elements to have the same type ");
}
if (!have_match && *val == elem) {
// Can't return yet, must check types of all array elements. <sigh>
have_match = true;
}
}
return have_match;
}
// Another variant of check_IN, this one for ConditionExpression. It needs to
// check whether the first element in the given vector is equal to any of the
// others.
static bool check_IN(const std::vector<rjson::value>& array) {
const rjson::value* first = &array[0];
for (unsigned i = 1; i < array.size(); i++) {
if (check_EQ(first, array[i])) {
return true;
}
}
return false;
}
static bool check_NULL(const rjson::value* val) {
return val == nullptr;
}
static bool check_NOT_NULL(const rjson::value* val) {
return val != nullptr;
}
// Only types S, N or B (string, number or bytes) may be compared by the
// various comparion operators - lt, le, gt, ge, and between.
// Note that in particular, if the value is missing (v->IsNull()), this
// check returns false.
static bool check_comparable_type(const rjson::value& v) {
if (!v.IsObject() || v.MemberCount() != 1) {
return false;
}
const rjson::value& type = v.MemberBegin()->name;
return type == "S" || type == "N" || type == "B";
}
// Check if two JSON-encoded values match with cmp.
template <typename Comparator>
bool check_compare(const rjson::value* v1, const rjson::value& v2, const Comparator& cmp,
bool v1_from_query, bool v2_from_query) {
bool bad = false;
if (!v1 || !check_comparable_type(*v1)) {
if (v1_from_query) {
throw api_error::validation(format("{} allow only the types String, Number, or Binary", cmp.diagnostic));
}
bad = true;
}
if (!check_comparable_type(v2)) {
if (v2_from_query) {
throw api_error::validation(format("{} allow only the types String, Number, or Binary", cmp.diagnostic));
}
bad = true;
}
if (bad) {
return false;
}
const auto& kv1 = *v1->MemberBegin();
const auto& kv2 = *v2.MemberBegin();
if (kv1.name != kv2.name) {
return false;
}
if (kv1.name == "N") {
return cmp(unwrap_number(*v1, cmp.diagnostic), unwrap_number(v2, cmp.diagnostic));
}
if (kv1.name == "S") {
return cmp(std::string_view(kv1.value.GetString(), kv1.value.GetStringLength()),
std::string_view(kv2.value.GetString(), kv2.value.GetStringLength()));
}
if (kv1.name == "B") {
auto d_kv1 = unwrap_bytes(kv1.value, v1_from_query);
auto d_kv2 = unwrap_bytes(kv2.value, v2_from_query);
if(!d_kv1 || !d_kv2) {
return false;
}
return cmp(*d_kv1, *d_kv2);
}
// cannot reach here, as check_comparable_type() verifies the type is one
// of the above options.
return false;
}
struct cmp_lt {
template <typename T> bool operator()(const T& lhs, const T& rhs) const { return lhs < rhs; }
// We cannot use the normal comparison operators like "<" on the bytes
// type, because they treat individual bytes as signed but we need to
// compare them as *unsigned*. So we need a specialization for bytes.
bool operator()(const bytes& lhs, const bytes& rhs) const { return compare_unsigned(lhs, rhs) < 0; }
static constexpr const char* diagnostic = "LT operator";
};
struct cmp_le {
template <typename T> bool operator()(const T& lhs, const T& rhs) const { return lhs <= rhs; }
bool operator()(const bytes& lhs, const bytes& rhs) const { return compare_unsigned(lhs, rhs) <= 0; }
static constexpr const char* diagnostic = "LE operator";
};
struct cmp_ge {
template <typename T> bool operator()(const T& lhs, const T& rhs) const { return lhs >= rhs; }
bool operator()(const bytes& lhs, const bytes& rhs) const { return compare_unsigned(lhs, rhs) >= 0; }
static constexpr const char* diagnostic = "GE operator";
};
struct cmp_gt {
template <typename T> bool operator()(const T& lhs, const T& rhs) const { return lhs > rhs; }
bool operator()(const bytes& lhs, const bytes& rhs) const { return compare_unsigned(lhs, rhs) > 0; }
static constexpr const char* diagnostic = "GT operator";
};
// True if v is between lb and ub, inclusive. Throws or returns false
// (depending on bounds_from_query parameter) if lb > ub.
template <typename T>
static bool check_BETWEEN(const T& v, const T& lb, const T& ub, bool bounds_from_query) {
if (cmp_lt()(ub, lb)) {
if (bounds_from_query) {
throw api_error::validation(
format("BETWEEN operator requires lower_bound <= upper_bound, but {} > {}", lb, ub));
} else {
return false;
}
}
return cmp_ge()(v, lb) && cmp_le()(v, ub);
}
static bool check_BETWEEN(const rjson::value* v, const rjson::value& lb, const rjson::value& ub,
bool v_from_query, bool lb_from_query, bool ub_from_query) {
if ((v && v_from_query && !check_comparable_type(*v)) ||
(lb_from_query && !check_comparable_type(lb)) ||
(ub_from_query && !check_comparable_type(ub))) {
throw api_error::validation("between allow only the types String, Number, or Binary");
}
if (!v || !v->IsObject() || v->MemberCount() != 1 ||
!lb.IsObject() || lb.MemberCount() != 1 ||
!ub.IsObject() || ub.MemberCount() != 1) {
return false;
}
const auto& kv_v = *v->MemberBegin();
const auto& kv_lb = *lb.MemberBegin();
const auto& kv_ub = *ub.MemberBegin();
bool bounds_from_query = lb_from_query && ub_from_query;
if (kv_lb.name != kv_ub.name) {
if (bounds_from_query) {
throw api_error::validation(
format("BETWEEN operator requires the same type for lower and upper bound; instead got {} and {}",
kv_lb.name, kv_ub.name));
} else {
return false;
}
}
if (kv_v.name != kv_lb.name) { // Cannot compare different types, so v is NOT between lb and ub.
return false;
}
if (kv_v.name == "N") {
const char* diag = "BETWEEN operator";
return check_BETWEEN(unwrap_number(*v, diag), unwrap_number(lb, diag), unwrap_number(ub, diag), bounds_from_query);
}
if (kv_v.name == "S") {
return check_BETWEEN(std::string_view(kv_v.value.GetString(), kv_v.value.GetStringLength()),
std::string_view(kv_lb.value.GetString(), kv_lb.value.GetStringLength()),
std::string_view(kv_ub.value.GetString(), kv_ub.value.GetStringLength()),
bounds_from_query);
}
if (kv_v.name == "B") {
auto d_kv_v = unwrap_bytes(kv_v.value, v_from_query);
auto d_kv_lb = unwrap_bytes(kv_lb.value, lb_from_query);
auto d_kv_ub = unwrap_bytes(kv_ub.value, ub_from_query);
if(!d_kv_v || !d_kv_lb || !d_kv_ub) {
return false;
}
return check_BETWEEN(*d_kv_v, *d_kv_lb, *d_kv_ub, bounds_from_query);
}
if (v_from_query) {
throw api_error::validation(
format("BETWEEN operator requires AttributeValueList elements to be of type String, Number, or Binary; instead got {}",
kv_lb.name));
} else {
return false;
}
}
// Verify one Expect condition on one attribute (whose content is "got")
// for the verify_expected() below.
// This function returns true or false depending on whether the condition
// succeeded - it does not throw ConditionalCheckFailedException.
// However, it may throw ValidationException on input validation errors.
static bool verify_expected_one(const rjson::value& condition, const rjson::value* got) {
const rjson::value* comparison_operator = rjson::find(condition, "ComparisonOperator");
const rjson::value* attribute_value_list = rjson::find(condition, "AttributeValueList");
const rjson::value* value = rjson::find(condition, "Value");
const rjson::value* exists = rjson::find(condition, "Exists");
// There are three types of conditions that Expected supports:
// A value, not-exists, and a comparison of some kind. Each allows
// and requires a different combinations of parameters in the request
if (value) {
if (exists && (!exists->IsBool() || exists->GetBool() != true)) {
throw api_error::validation("Cannot combine Value with Exists!=true");
}
if (comparison_operator) {
throw api_error::validation("Cannot combine Value with ComparisonOperator");
}
return check_EQ(got, *value);
} else if (exists) {
if (comparison_operator) {
throw api_error::validation("Cannot combine Exists with ComparisonOperator");
}
if (!exists->IsBool() || exists->GetBool() != false) {
throw api_error::validation("Exists!=false requires Value");
}
// Remember Exists=false, so we're checking that the attribute does *not* exist:
return !got;
} else {
if (!comparison_operator) {
throw api_error::validation("Missing ComparisonOperator, Value or Exists");
}
comparison_operator_type op = get_comparison_operator(*comparison_operator);
switch (op) {
case comparison_operator_type::EQ:
verify_operand_count(attribute_value_list, exact_size(1), *comparison_operator);
return check_EQ(got, (*attribute_value_list)[0]);
case comparison_operator_type::NE:
verify_operand_count(attribute_value_list, exact_size(1), *comparison_operator);
return check_NE(got, (*attribute_value_list)[0]);
case comparison_operator_type::LT:
verify_operand_count(attribute_value_list, exact_size(1), *comparison_operator);
return check_compare(got, (*attribute_value_list)[0], cmp_lt{}, false, true);
case comparison_operator_type::LE:
verify_operand_count(attribute_value_list, exact_size(1), *comparison_operator);
return check_compare(got, (*attribute_value_list)[0], cmp_le{}, false, true);
case comparison_operator_type::GT:
verify_operand_count(attribute_value_list, exact_size(1), *comparison_operator);
return check_compare(got, (*attribute_value_list)[0], cmp_gt{}, false, true);
case comparison_operator_type::GE:
verify_operand_count(attribute_value_list, exact_size(1), *comparison_operator);
return check_compare(got, (*attribute_value_list)[0], cmp_ge{}, false, true);
case comparison_operator_type::BEGINS_WITH:
verify_operand_count(attribute_value_list, exact_size(1), *comparison_operator);
return check_BEGINS_WITH(got, (*attribute_value_list)[0], false, true);
case comparison_operator_type::IN:
verify_operand_count(attribute_value_list, nonempty(), *comparison_operator);
return check_IN(got, *attribute_value_list);
case comparison_operator_type::IS_NULL:
verify_operand_count(attribute_value_list, empty(), *comparison_operator);
return check_NULL(got);
case comparison_operator_type::NOT_NULL:
verify_operand_count(attribute_value_list, empty(), *comparison_operator);
return check_NOT_NULL(got);
case comparison_operator_type::BETWEEN:
verify_operand_count(attribute_value_list, exact_size(2), *comparison_operator);
return check_BETWEEN(got, (*attribute_value_list)[0], (*attribute_value_list)[1],
false, true, true);
case comparison_operator_type::CONTAINS:
{
verify_operand_count(attribute_value_list, exact_size(1), *comparison_operator);
// Expected's "CONTAINS" has this artificial limitation.
// ConditionExpression's "contains()" does not...
const rjson::value& arg = (*attribute_value_list)[0];
const auto& argtype = (*arg.MemberBegin()).name;
if (argtype != "S" && argtype != "N" && argtype != "B") {
throw api_error::validation(
format("CONTAINS operator requires a single AttributeValue of type String, Number, or Binary, "
"got {} instead", argtype));
}
return check_CONTAINS(got, arg, false, true);
}
case comparison_operator_type::NOT_CONTAINS:
{
verify_operand_count(attribute_value_list, exact_size(1), *comparison_operator);
// Expected's "NOT_CONTAINS" has this artificial limitation.
// ConditionExpression's "contains()" does not...
const rjson::value& arg = (*attribute_value_list)[0];
const auto& argtype = (*arg.MemberBegin()).name;
if (argtype != "S" && argtype != "N" && argtype != "B") {
throw api_error::validation(
format("CONTAINS operator requires a single AttributeValue of type String, Number, or Binary, "
"got {} instead", argtype));
}
return check_NOT_CONTAINS(got, arg, false, true);
}
}
throw std::logic_error(format("Internal error: corrupted operator enum: {}", int(op)));
}
}
conditional_operator_type get_conditional_operator(const rjson::value& req) {
const rjson::value* conditional_operator = rjson::find(req, "ConditionalOperator");
if (!conditional_operator) {
return conditional_operator_type::MISSING;
}
if (!conditional_operator->IsString()) {
throw api_error::validation("'ConditionalOperator' parameter, if given, must be a string");
}
auto s = rjson::to_string_view(*conditional_operator);
if (s == "AND") {
return conditional_operator_type::AND;
} else if (s == "OR") {
return conditional_operator_type::OR;
} else {
throw api_error::validation(
format("'ConditionalOperator' parameter must be AND, OR or missing. Found {}.", s));
}
}
// Check if the existing values of the item (previous_item) match the
// conditions given by the Expected and ConditionalOperator parameters
// (if they exist) in the request (an UpdateItem, PutItem or DeleteItem).
// This function can throw an ValidationException API error if there
// are errors in the format of the condition itself.
bool verify_expected(const rjson::value& req, const rjson::value* previous_item) {
const rjson::value* expected = rjson::find(req, "Expected");
auto conditional_operator = get_conditional_operator(req);
if (conditional_operator != conditional_operator_type::MISSING &&
(!expected || (expected->IsObject() && expected->GetObject().ObjectEmpty()))) {
throw api_error::validation("'ConditionalOperator' parameter cannot be specified for missing or empty Expression");
}
if (!expected) {
return true;
}
if (!expected->IsObject()) {
throw api_error::validation("'Expected' parameter, if given, must be an object");
}
bool require_all = conditional_operator != conditional_operator_type::OR;
return verify_condition(*expected, require_all, previous_item);
}
bool verify_condition(const rjson::value& condition, bool require_all, const rjson::value* previous_item) {
for (auto it = condition.MemberBegin(); it != condition.MemberEnd(); ++it) {
const rjson::value* got = nullptr;
if (previous_item) {
got = rjson::find(*previous_item, rjson::to_string_view(it->name));
}
bool success = verify_expected_one(it->value, got);
if (success && !require_all) {
// When !require_all, one success is enough!
return true;
} else if (!success && require_all) {
// When require_all, one failure is enough!
return false;
}
}
// If we got here and require_all, none of the checks failed, so succeed.
// If we got here and !require_all, all of the checks failed, so fail.
return require_all;
}
static bool calculate_primitive_condition(const parsed::primitive_condition& cond,
const rjson::value* previous_item) {
std::vector<rjson::value> calculated_values;
calculated_values.reserve(cond._values.size());
for (const parsed::value& v : cond._values) {
calculated_values.push_back(calculate_value(v,
cond._op == parsed::primitive_condition::type::VALUE ?
calculate_value_caller::ConditionExpressionAlone :
calculate_value_caller::ConditionExpression,
previous_item));
}
switch (cond._op) {
case parsed::primitive_condition::type::BETWEEN:
if (calculated_values.size() != 3) {
// Shouldn't happen unless we have a bug in the parser
throw std::logic_error(format("Wrong number of values {} in BETWEEN primitive_condition", cond._values.size()));
}
return check_BETWEEN(&calculated_values[0], calculated_values[1], calculated_values[2],
cond._values[0].is_constant(), cond._values[1].is_constant(), cond._values[2].is_constant());
case parsed::primitive_condition::type::IN:
return check_IN(calculated_values);
case parsed::primitive_condition::type::VALUE:
if (calculated_values.size() != 1) {
// Shouldn't happen unless we have a bug in the parser
throw std::logic_error(format("Unexpected values in primitive_condition", cond._values.size()));
}
// Unwrap the boolean wrapped as the value (if it is a boolean)
if (calculated_values[0].IsObject() && calculated_values[0].MemberCount() == 1) {
auto it = calculated_values[0].MemberBegin();
if (it->name == "BOOL" && it->value.IsBool()) {
return it->value.GetBool();
}
}
throw api_error::validation(
format("ConditionExpression: condition results in a non-boolean value: {}",
calculated_values[0]));
default:
// All the rest of the operators have exactly two parameters (and unless
// we have a bug in the parser, that's what we have in the parsed object:
if (calculated_values.size() != 2) {
throw std::logic_error(format("Wrong number of values {} in primitive_condition object", cond._values.size()));
}
}
switch (cond._op) {
case parsed::primitive_condition::type::EQ:
return check_EQ(&calculated_values[0], calculated_values[1]);
case parsed::primitive_condition::type::NE:
return check_NE(&calculated_values[0], calculated_values[1]);
case parsed::primitive_condition::type::GT:
return check_compare(&calculated_values[0], calculated_values[1], cmp_gt{},
cond._values[0].is_constant(), cond._values[1].is_constant());
case parsed::primitive_condition::type::GE:
return check_compare(&calculated_values[0], calculated_values[1], cmp_ge{},
cond._values[0].is_constant(), cond._values[1].is_constant());
case parsed::primitive_condition::type::LT:
return check_compare(&calculated_values[0], calculated_values[1], cmp_lt{},
cond._values[0].is_constant(), cond._values[1].is_constant());
case parsed::primitive_condition::type::LE:
return check_compare(&calculated_values[0], calculated_values[1], cmp_le{},
cond._values[0].is_constant(), cond._values[1].is_constant());
default:
// Shouldn't happen unless we have a bug in the parser
throw std::logic_error(format("Unknown type {} in primitive_condition object", (int)(cond._op)));
}
}
// Check if the existing values of the item (previous_item) match the
// conditions given by the given parsed ConditionExpression.
bool verify_condition_expression(
const parsed::condition_expression& condition_expression,
const rjson::value* previous_item) {
if (condition_expression.empty()) {
return true;
}
bool ret = std::visit(overloaded_functor {
[&] (const parsed::primitive_condition& cond) -> bool {
return calculate_primitive_condition(cond, previous_item);
},
[&] (const parsed::condition_expression::condition_list& list) -> bool {
auto verify_condition = [&] (const parsed::condition_expression& e) {
return verify_condition_expression(e, previous_item);
};
switch (list.op) {
case '&':
return boost::algorithm::all_of(list.conditions, verify_condition);
case '|':
return boost::algorithm::any_of(list.conditions, verify_condition);
default:
// Shouldn't happen unless we have a bug in the parser
throw std::logic_error("bad operator in condition_list");
}
}
}, condition_expression._expression);
return condition_expression._negated ? !ret : ret;
}
}

View File

@@ -1,48 +0,0 @@
/*
* Copyright 2019-present ScyllaDB
*/
/*
* SPDX-License-Identifier: AGPL-3.0-or-later
*/
/*
* This file contains definitions and functions related to placing conditions
* on Alternator queries (equivalent of CQL's restrictions).
*
* With conditions, it's possible to add criteria to selection requests (Scan, Query)
* and use them for narrowing down the result set, by means of filtering or indexing.
*
* Ref: https://docs.aws.amazon.com/amazondynamodb/latest/APIReference/API_Condition.html
*/
#pragma once
#include "cql3/restrictions/statement_restrictions.hh"
#include "serialization.hh"
#include "expressions_types.hh"
namespace alternator {
enum class comparison_operator_type {
EQ, NE, LE, LT, GE, GT, IN, BETWEEN, CONTAINS, NOT_CONTAINS, IS_NULL, NOT_NULL, BEGINS_WITH
};
comparison_operator_type get_comparison_operator(const rjson::value& comparison_operator);
enum class conditional_operator_type {
AND, OR, MISSING
};
conditional_operator_type get_conditional_operator(const rjson::value& req);
bool verify_expected(const rjson::value& req, const rjson::value* previous_item);
bool verify_condition(const rjson::value& condition, bool require_all, const rjson::value* previous_item);
bool check_CONTAINS(const rjson::value* v1, const rjson::value& v2, bool v1_from_query, bool v2_from_query);
bool check_BEGINS_WITH(const rjson::value* v1, const rjson::value& v2, bool v1_from_query, bool v2_from_query);
bool verify_condition_expression(
const parsed::condition_expression& condition_expression,
const rjson::value* previous_item);
}

View File

@@ -1,159 +0,0 @@
/*
* Copyright (C) 2021-present ScyllaDB
*/
/*
* SPDX-License-Identifier: AGPL-3.0-or-later
*/
#include <seastar/net/dns.hh>
#include "controller.hh"
#include "server.hh"
#include "executor.hh"
#include "rmw_operation.hh"
#include "db/config.hh"
#include "cdc/generation_service.hh"
#include "service/memory_limiter.hh"
#include "auth/service.hh"
#include "service/qos/service_level_controller.hh"
using namespace seastar;
namespace alternator {
static logging::logger logger("alternator_controller");
controller::controller(
sharded<gms::gossiper>& gossiper,
sharded<service::storage_proxy>& proxy,
sharded<service::migration_manager>& mm,
sharded<db::system_distributed_keyspace>& sys_dist_ks,
sharded<cdc::generation_service>& cdc_gen_svc,
sharded<service::memory_limiter>& memory_limiter,
sharded<auth::service>& auth_service,
sharded<qos::service_level_controller>& sl_controller,
const db::config& config)
: _gossiper(gossiper)
, _proxy(proxy)
, _mm(mm)
, _sys_dist_ks(sys_dist_ks)
, _cdc_gen_svc(cdc_gen_svc)
, _memory_limiter(memory_limiter)
, _auth_service(auth_service)
, _sl_controller(sl_controller)
, _config(config)
{
}
sstring controller::name() const {
return "alternator";
}
sstring controller::protocol() const {
return "dynamodb";
}
sstring controller::protocol_version() const {
return version;
}
std::vector<socket_address> controller::listen_addresses() const {
return _listen_addresses;
}
future<> controller::start_server() {
return seastar::async([this] {
_listen_addresses.clear();
auto preferred = _config.listen_interface_prefer_ipv6() ? std::make_optional(net::inet_address::family::INET6) : std::nullopt;
auto family = _config.enable_ipv6_dns_lookup() || preferred ? std::nullopt : std::make_optional(net::inet_address::family::INET);
// Create an smp_service_group to be used for limiting the
// concurrency when forwarding Alternator request between
// shards - if necessary for LWT.
smp_service_group_config c;
c.max_nonlocal_requests = 5000;
_ssg = create_smp_service_group(c).get0();
rmw_operation::set_default_write_isolation(_config.alternator_write_isolation());
executor::set_default_timeout(std::chrono::milliseconds(_config.alternator_timeout_in_ms()));
net::inet_address addr = utils::resolve(_config.alternator_address, family).get0();
auto get_cdc_metadata = [] (cdc::generation_service& svc) { return std::ref(svc.get_cdc_metadata()); };
_executor.start(std::ref(_gossiper), std::ref(_proxy), std::ref(_mm), std::ref(_sys_dist_ks), sharded_parameter(get_cdc_metadata, std::ref(_cdc_gen_svc)), _ssg.value()).get();
_server.start(std::ref(_executor), std::ref(_proxy), std::ref(_gossiper), std::ref(_auth_service), std::ref(_sl_controller)).get();
// Note: from this point on, if start_server() throws for any reason,
// it must first call stop_server() to stop the executor and server
// services we just started - or Scylla will cause an assertion
// failure when the controller object is destroyed in the exception
// unwinding.
std::optional<uint16_t> alternator_port;
if (_config.alternator_port()) {
alternator_port = _config.alternator_port();
_listen_addresses.push_back({addr, *alternator_port});
}
std::optional<uint16_t> alternator_https_port;
std::optional<tls::credentials_builder> creds;
if (_config.alternator_https_port()) {
alternator_https_port = _config.alternator_https_port();
_listen_addresses.push_back({addr, *alternator_https_port});
creds.emplace();
auto opts = _config.alternator_encryption_options();
if (opts.empty()) {
// Earlier versions mistakenly configured Alternator's
// HTTPS parameters via the "server_encryption_option"
// configuration parameter. We *temporarily* continue
// to allow this, for backward compatibility.
opts = _config.server_encryption_options();
if (!opts.empty()) {
logger.warn("Setting server_encryption_options to configure "
"Alternator's HTTPS encryption is deprecated. Please "
"switch to setting alternator_encryption_options instead.");
}
}
opts.erase("require_client_auth");
opts.erase("truststore");
try {
utils::configure_tls_creds_builder(creds.value(), std::move(opts)).get();
} catch(...) {
logger.error("Failed to set up Alternator TLS credentials: {}", std::current_exception());
stop_server().get();
std::throw_with_nested(std::runtime_error("Failed to set up Alternator TLS credentials"));
}
}
bool alternator_enforce_authorization = _config.alternator_enforce_authorization();
_server.invoke_on_all(
[this, addr, alternator_port, alternator_https_port, creds = std::move(creds), alternator_enforce_authorization] (server& server) mutable {
return server.init(addr, alternator_port, alternator_https_port, creds, alternator_enforce_authorization,
&_memory_limiter.local().get_semaphore(),
_config.max_concurrent_requests_per_shard);
}).handle_exception([this, addr, alternator_port, alternator_https_port] (std::exception_ptr ep) {
logger.error("Failed to set up Alternator HTTP server on {} port {}, TLS port {}: {}",
addr, alternator_port ? std::to_string(*alternator_port) : "OFF", alternator_https_port ? std::to_string(*alternator_https_port) : "OFF", ep);
return stop_server().then([ep = std::move(ep)] { return make_exception_future<>(ep); });
}).then([addr, alternator_port, alternator_https_port] {
logger.info("Alternator server listening on {}, HTTP port {}, HTTPS port {}",
addr, alternator_port ? std::to_string(*alternator_port) : "OFF", alternator_https_port ? std::to_string(*alternator_https_port) : "OFF");
}).get();
});
}
future<> controller::stop_server() {
return seastar::async([this] {
if (!_ssg) {
return;
}
_server.stop().get();
_executor.stop().get();
_listen_addresses.clear();
destroy_smp_service_group(_ssg.value()).get();
});
}
future<> controller::request_stop_server() {
return stop_server();
}
}

View File

@@ -1,94 +0,0 @@
/*
* Copyright (C) 2021-present ScyllaDB
*/
/*
* SPDX-License-Identifier: AGPL-3.0-or-later
*/
#pragma once
#include <seastar/core/sharded.hh>
#include <seastar/core/smp.hh>
#include "protocol_server.hh"
namespace service {
class storage_proxy;
class migration_manager;
class memory_limiter;
}
namespace db {
class system_distributed_keyspace;
class config;
}
namespace cdc {
class generation_service;
}
namespace gms {
class gossiper;
}
namespace auth {
class service;
}
namespace qos {
class service_level_controller;
}
namespace alternator {
// This is the official DynamoDB API version.
// It represents the last major reorganization of that API, and all the features
// that were added since did NOT increment this version string.
constexpr const char* version = "2012-08-10";
using namespace seastar;
class executor;
class server;
class controller : public protocol_server {
sharded<gms::gossiper>& _gossiper;
sharded<service::storage_proxy>& _proxy;
sharded<service::migration_manager>& _mm;
sharded<db::system_distributed_keyspace>& _sys_dist_ks;
sharded<cdc::generation_service>& _cdc_gen_svc;
sharded<service::memory_limiter>& _memory_limiter;
sharded<auth::service>& _auth_service;
sharded<qos::service_level_controller>& _sl_controller;
const db::config& _config;
std::vector<socket_address> _listen_addresses;
sharded<executor> _executor;
sharded<server> _server;
std::optional<smp_service_group> _ssg;
public:
controller(
sharded<gms::gossiper>& gossiper,
sharded<service::storage_proxy>& proxy,
sharded<service::migration_manager>& mm,
sharded<db::system_distributed_keyspace>& sys_dist_ks,
sharded<cdc::generation_service>& cdc_gen_svc,
sharded<service::memory_limiter>& memory_limiter,
sharded<auth::service>& auth_service,
sharded<qos::service_level_controller>& sl_controller,
const db::config& config);
virtual sstring name() const override;
virtual sstring protocol() const override;
virtual sstring protocol_version() const override;
virtual std::vector<socket_address> listen_addresses() const override;
virtual future<> start_server() override;
virtual future<> stop_server() override;
virtual future<> request_stop_server() override;
};
}

View File

@@ -1,93 +0,0 @@
/*
* Copyright 2019-present ScyllaDB
*/
/*
* SPDX-License-Identifier: AGPL-3.0-or-later
*/
#pragma once
#include <seastar/http/httpd.hh>
#include "seastarx.hh"
namespace alternator {
// api_error contains a DynamoDB error message to be returned to the user.
// It can be returned by value (see executor::request_return_type) or thrown.
// The DynamoDB's error messages are described in detail in
// https://docs.aws.amazon.com/amazondynamodb/latest/developerguide/Programming.Errors.html
// An error message has an HTTP code (almost always 400), a type, e.g.,
// "ResourceNotFoundException", and a human readable message.
// Eventually alternator::api_handler will convert a returned or thrown
// api_error into a JSON object, and that is returned to the user.
class api_error final : public std::exception {
public:
using status_type = http::reply::status_type;
status_type _http_code;
std::string _type;
std::string _msg;
api_error(std::string type, std::string msg, status_type http_code = status_type::bad_request)
: _http_code(std::move(http_code))
, _type(std::move(type))
, _msg(std::move(msg))
{ }
// Factory functions for some common types of DynamoDB API errors
static api_error validation(std::string msg) {
return api_error("ValidationException", std::move(msg));
}
static api_error resource_not_found(std::string msg) {
return api_error("ResourceNotFoundException", std::move(msg));
}
static api_error resource_in_use(std::string msg) {
return api_error("ResourceInUseException", std::move(msg));
}
static api_error invalid_signature(std::string msg) {
return api_error("InvalidSignatureException", std::move(msg));
}
static api_error missing_authentication_token(std::string msg) {
return api_error("MissingAuthenticationTokenException", std::move(msg));
}
static api_error unrecognized_client(std::string msg) {
return api_error("UnrecognizedClientException", std::move(msg));
}
static api_error unknown_operation(std::string msg) {
return api_error("UnknownOperationException", std::move(msg));
}
static api_error access_denied(std::string msg) {
return api_error("AccessDeniedException", std::move(msg));
}
static api_error conditional_check_failed(std::string msg) {
return api_error("ConditionalCheckFailedException", std::move(msg));
}
static api_error expired_iterator(std::string msg) {
return api_error("ExpiredIteratorException", std::move(msg));
}
static api_error trimmed_data_access_exception(std::string msg) {
return api_error("TrimmedDataAccessException", std::move(msg));
}
static api_error request_limit_exceeded(std::string msg) {
return api_error("RequestLimitExceeded", std::move(msg));
}
static api_error serialization(std::string msg) {
return api_error("SerializationException", std::move(msg));
}
static api_error table_not_found(std::string msg) {
return api_error("TableNotFoundException", std::move(msg));
}
static api_error internal(std::string msg) {
return api_error("InternalServerError", std::move(msg), http::reply::status_type::internal_server_error);
}
// Provide the "std::exception" interface, to make it easier to print this
// exception in log messages. Note that this function is *not* used to
// format the error to send it back to the client - server.cc has
// generate_error_reply() to format an api_error as the DynamoDB protocol
// requires.
virtual const char* what() const noexcept override;
mutable std::string _what_string;
};
}

File diff suppressed because it is too large Load Diff

View File

@@ -1,253 +0,0 @@
/*
* Copyright 2019-present ScyllaDB
*/
/*
* SPDX-License-Identifier: AGPL-3.0-or-later
*/
#pragma once
#include <seastar/core/future.hh>
#include <seastar/http/httpd.hh>
#include "seastarx.hh"
#include <seastar/json/json_elements.hh>
#include <seastar/core/sharded.hh>
#include "service/migration_manager.hh"
#include "service/client_state.hh"
#include "service_permit.hh"
#include "db/timeout_clock.hh"
#include "alternator/error.hh"
#include "stats.hh"
#include "utils/rjson.hh"
namespace db {
class system_distributed_keyspace;
}
namespace query {
class partition_slice;
class result;
}
namespace cql3::selection {
class selection;
}
namespace service {
class storage_proxy;
}
namespace cdc {
class metadata;
}
namespace gms {
class gossiper;
}
namespace alternator {
class rmw_operation;
struct make_jsonable : public json::jsonable {
rjson::value _value;
public:
explicit make_jsonable(rjson::value&& value);
std::string to_json() const override;
};
/**
* Make return type for serializing the object "streamed",
* i.e. direct to HTTP output stream. Note: only useful for
* (very) large objects as there are overhead issues with this
* as well, but for massive lists of return objects this can
* help avoid large allocations/many re-allocs
*/
json::json_return_type make_streamed(rjson::value&&);
struct json_string : public json::jsonable {
std::string _value;
public:
explicit json_string(std::string&& value);
std::string to_json() const override;
};
namespace parsed {
class path;
};
schema_ptr get_table(service::storage_proxy& proxy, const rjson::value& request);
bool is_alternator_keyspace(const sstring& ks_name);
// Wraps the db::get_tags_of_table and throws if the table is missing the tags extension.
const std::map<sstring, sstring>& get_tags_of_table_or_throw(schema_ptr schema);
// An attribute_path_map object is used to hold data for various attributes
// paths (parsed::path) in a hierarchy of attribute paths. Each attribute path
// has a root attribute, and then modified by member and index operators -
// for example in "a.b[2].c" we have "a" as the root, then ".b" member, then
// "[2]" index, and finally ".c" member.
// Data can be added to an attribute_path_map using the add() function, but
// requires that attributes with data not be *overlapping* or *conflicting*:
//
// 1. Two attribute paths which are identical or an ancestor of one another
// are considered *overlapping* and not allowed. If a.b.c has data,
// we can't add more data in a.b.c or any of its descendants like a.b.c.d.
//
// 2. Two attribute paths which need the same parent to have both a member and
// an index are considered *conflicting* and not allowed. E.g., if a.b has
// data, you can't add a[1]. The meaning of adding both would be that the
// attribute a is both a map and an array, which isn't sensible.
//
// These two requirements are common to the two places where Alternator uses
// this abstraction to describe how a hierarchical item is to be transformed:
//
// 1. In ProjectExpression: for filtering from a full top-level attribute
// only the parts for which user asked in ProjectionExpression.
//
// 2. In UpdateExpression: for taking the previous value of a top-level
// attribute, and modifying it based on the instructions in the user
// wrote in UpdateExpression.
template<typename T>
class attribute_path_map_node {
public:
using data_t = T;
// We need the extra unique_ptr<> here because libstdc++ unordered_map
// doesn't work with incomplete types :-(
using members_t = std::unordered_map<std::string, std::unique_ptr<attribute_path_map_node<T>>>;
// The indexes list is sorted because DynamoDB requires handling writes
// beyond the end of a list in index order.
using indexes_t = std::map<unsigned, std::unique_ptr<attribute_path_map_node<T>>>;
// The prohibition on "overlap" and "conflict" explained above means
// That only one of data, members or indexes is non-empty.
std::optional<std::variant<data_t, members_t, indexes_t>> _content;
bool is_empty() const { return !_content; }
bool has_value() const { return _content && std::holds_alternative<data_t>(*_content); }
bool has_members() const { return _content && std::holds_alternative<members_t>(*_content); }
bool has_indexes() const { return _content && std::holds_alternative<indexes_t>(*_content); }
// get_members() assumes that has_members() is true
members_t& get_members() { return std::get<members_t>(*_content); }
const members_t& get_members() const { return std::get<members_t>(*_content); }
indexes_t& get_indexes() { return std::get<indexes_t>(*_content); }
const indexes_t& get_indexes() const { return std::get<indexes_t>(*_content); }
T& get_value() { return std::get<T>(*_content); }
const T& get_value() const { return std::get<T>(*_content); }
};
template<typename T>
using attribute_path_map = std::unordered_map<std::string, attribute_path_map_node<T>>;
using attrs_to_get_node = attribute_path_map_node<std::monostate>;
// attrs_to_get lists which top-level attribute are needed, and possibly also
// which part of the top-level attribute is really needed (when nested
// attribute paths appeared in the query).
// Most code actually uses optional<attrs_to_get>. There, a disengaged
// optional means we should get all attributes, not specific ones.
using attrs_to_get = attribute_path_map<std::monostate>;
class executor : public peering_sharded_service<executor> {
gms::gossiper& _gossiper;
service::storage_proxy& _proxy;
service::migration_manager& _mm;
db::system_distributed_keyspace& _sdks;
cdc::metadata& _cdc_metadata;
// An smp_service_group to be used for limiting the concurrency when
// forwarding Alternator request between shards - if necessary for LWT.
smp_service_group _ssg;
public:
using client_state = service::client_state;
using request_return_type = std::variant<json::json_return_type, api_error>;
stats _stats;
static constexpr auto ATTRS_COLUMN_NAME = ":attrs";
static constexpr auto KEYSPACE_NAME_PREFIX = "alternator_";
static constexpr std::string_view INTERNAL_TABLE_PREFIX = ".scylla.alternator.";
executor(gms::gossiper& gossiper, service::storage_proxy& proxy, service::migration_manager& mm, db::system_distributed_keyspace& sdks, cdc::metadata& cdc_metadata, smp_service_group ssg)
: _gossiper(gossiper), _proxy(proxy), _mm(mm), _sdks(sdks), _cdc_metadata(cdc_metadata), _ssg(ssg) {}
future<request_return_type> create_table(client_state& client_state, tracing::trace_state_ptr trace_state, service_permit permit, rjson::value request);
future<request_return_type> describe_table(client_state& client_state, tracing::trace_state_ptr trace_state, service_permit permit, rjson::value request);
future<request_return_type> delete_table(client_state& client_state, tracing::trace_state_ptr trace_state, service_permit permit, rjson::value request);
future<request_return_type> update_table(client_state& client_state, tracing::trace_state_ptr trace_state, service_permit permit, rjson::value request);
future<request_return_type> put_item(client_state& client_state, tracing::trace_state_ptr trace_state, service_permit permit, rjson::value request);
future<request_return_type> get_item(client_state& client_state, tracing::trace_state_ptr trace_state, service_permit permit, rjson::value request);
future<request_return_type> delete_item(client_state& client_state, tracing::trace_state_ptr trace_state, service_permit permit, rjson::value request);
future<request_return_type> update_item(client_state& client_state, tracing::trace_state_ptr trace_state, service_permit permit, rjson::value request);
future<request_return_type> list_tables(client_state& client_state, service_permit permit, rjson::value request);
future<request_return_type> scan(client_state& client_state, tracing::trace_state_ptr trace_state, service_permit permit, rjson::value request);
future<request_return_type> describe_endpoints(client_state& client_state, service_permit permit, rjson::value request, std::string host_header);
future<request_return_type> batch_write_item(client_state& client_state, tracing::trace_state_ptr trace_state, service_permit permit, rjson::value request);
future<request_return_type> batch_get_item(client_state& client_state, tracing::trace_state_ptr trace_state, service_permit permit, rjson::value request);
future<request_return_type> query(client_state& client_state, tracing::trace_state_ptr trace_state, service_permit permit, rjson::value request);
future<request_return_type> tag_resource(client_state& client_state, service_permit permit, rjson::value request);
future<request_return_type> untag_resource(client_state& client_state, service_permit permit, rjson::value request);
future<request_return_type> list_tags_of_resource(client_state& client_state, service_permit permit, rjson::value request);
future<request_return_type> update_time_to_live(client_state& client_state, service_permit permit, rjson::value request);
future<request_return_type> describe_time_to_live(client_state& client_state, service_permit permit, rjson::value request);
future<request_return_type> list_streams(client_state& client_state, service_permit permit, rjson::value request);
future<request_return_type> describe_stream(client_state& client_state, service_permit permit, rjson::value request);
future<request_return_type> get_shard_iterator(client_state& client_state, service_permit permit, rjson::value request);
future<request_return_type> get_records(client_state& client_state, tracing::trace_state_ptr, service_permit permit, rjson::value request);
future<request_return_type> describe_continuous_backups(client_state& client_state, service_permit permit, rjson::value request);
future<> start();
future<> stop() { return make_ready_future<>(); }
static sstring table_name(const schema&);
static db::timeout_clock::time_point default_timeout();
static void set_default_timeout(db::timeout_clock::duration timeout);
private:
static db::timeout_clock::duration s_default_timeout;
public:
static schema_ptr find_table(service::storage_proxy&, const rjson::value& request);
private:
friend class rmw_operation;
static void describe_key_schema(rjson::value& parent, const schema&, std::unordered_map<std::string,std::string> * = nullptr);
static void describe_key_schema(rjson::value& parent, const schema& schema, std::unordered_map<std::string,std::string>&);
public:
static std::optional<rjson::value> describe_single_item(schema_ptr,
const query::partition_slice&,
const cql3::selection::selection&,
const query::result&,
const std::optional<attrs_to_get>&);
static std::vector<rjson::value> describe_multi_item(schema_ptr schema,
const query::partition_slice& slice,
const cql3::selection::selection& selection,
const query::result& query_result,
const std::optional<attrs_to_get>& attrs_to_get);
static void describe_single_item(const cql3::selection::selection&,
const std::vector<bytes_opt>&,
const std::optional<attrs_to_get>&,
rjson::value&,
bool = false);
static void add_stream_options(const rjson::value& stream_spec, schema_builder&, service::storage_proxy& sp);
static void supplement_table_info(rjson::value& descr, const schema& schema, service::storage_proxy& sp);
static void supplement_table_stream_info(rjson::value& descr, const schema& schema, service::storage_proxy& sp);
};
// is_big() checks approximately if the given JSON value is "bigger" than
// the given big_size number of bytes. The goal is to *quickly* detect
// oversized JSON that, for example, is too large to be serialized to a
// contiguous string - we don't need an accurate size for that. Moreover,
// as soon as we detect that the JSON is indeed "big", we can return true
// and don't need to continue calculating its exact size.
// For simplicity, we use a recursive implementation. This is fine because
// Alternator limits the depth of JSONs it reads from inputs, and doesn't
// add more than a couple of levels in its own output construction.
bool is_big(const rjson::value& val, int big_size = 100'000);
}

View File

@@ -1,741 +0,0 @@
/*
* Copyright 2019-present ScyllaDB
*/
/*
* SPDX-License-Identifier: AGPL-3.0-or-later
*/
#include "expressions.hh"
#include "serialization.hh"
#include "utils/base64.hh"
#include "conditions.hh"
#include "alternator/expressionsLexer.hpp"
#include "alternator/expressionsParser.hpp"
#include "utils/overloaded_functor.hh"
#include "error.hh"
#include "seastarx.hh"
#include <seastar/core/print.hh>
#include <seastar/util/log.hh>
#include <boost/algorithm/cxx11/any_of.hpp>
#include <boost/algorithm/cxx11/all_of.hpp>
#include <functional>
#include <unordered_map>
namespace alternator {
template <typename Func, typename Result = std::result_of_t<Func(expressionsParser&)>>
Result do_with_parser(std::string_view input, Func&& f) {
expressionsLexer::InputStreamType input_stream{
reinterpret_cast<const ANTLR_UINT8*>(input.data()),
ANTLR_ENC_UTF8,
static_cast<ANTLR_UINT32>(input.size()),
nullptr };
expressionsLexer lexer(&input_stream);
expressionsParser::TokenStreamType tstream(ANTLR_SIZE_HINT, lexer.get_tokSource());
expressionsParser parser(&tstream);
auto result = f(parser);
return result;
}
parsed::update_expression
parse_update_expression(std::string_view query) {
try {
return do_with_parser(query, std::mem_fn(&expressionsParser::update_expression));
} catch (...) {
throw expressions_syntax_error(format("Failed parsing UpdateExpression '{}': {}", query, std::current_exception()));
}
}
std::vector<parsed::path>
parse_projection_expression(std::string_view query) {
try {
return do_with_parser(query, std::mem_fn(&expressionsParser::projection_expression));
} catch (...) {
throw expressions_syntax_error(format("Failed parsing ProjectionExpression '{}': {}", query, std::current_exception()));
}
}
parsed::condition_expression
parse_condition_expression(std::string_view query) {
try {
return do_with_parser(query, std::mem_fn(&expressionsParser::condition_expression));
} catch (...) {
throw expressions_syntax_error(format("Failed parsing ConditionExpression '{}': {}", query, std::current_exception()));
}
}
namespace parsed {
void update_expression::add(update_expression::action a) {
std::visit(overloaded_functor {
[&] (action::set&) { seen_set = true; },
[&] (action::remove&) { seen_remove = true; },
[&] (action::add&) { seen_add = true; },
[&] (action::del&) { seen_del = true; }
}, a._action);
_actions.push_back(std::move(a));
}
void update_expression::append(update_expression other) {
if ((seen_set && other.seen_set) ||
(seen_remove && other.seen_remove) ||
(seen_add && other.seen_add) ||
(seen_del && other.seen_del)) {
throw expressions_syntax_error("Each of SET, REMOVE, ADD, DELETE may only appear once in UpdateExpression");
}
std::move(other._actions.begin(), other._actions.end(), std::back_inserter(_actions));
seen_set |= other.seen_set;
seen_remove |= other.seen_remove;
seen_add |= other.seen_add;
seen_del |= other.seen_del;
}
void condition_expression::append(condition_expression&& a, char op) {
std::visit(overloaded_functor {
[&] (condition_list& x) {
// If 'a' has a single condition, we could, instead of inserting
// it insert its single condition (possibly negated if a._negated)
// But considering it we don't evaluate these expressions many
// times, this optimization is not worth extra code complexity.
if (!x.conditions.empty() && x.op != op) {
// Shouldn't happen unless we have a bug in the parser
throw std::logic_error("condition_expression::append called with mixed operators");
}
x.conditions.push_back(std::move(a));
x.op = op;
},
[&] (primitive_condition& x) {
// Shouldn't happen unless we have a bug in the parser
throw std::logic_error("condition_expression::append called on primitive_condition");
}
}, _expression);
}
void path::check_depth_limit() {
if (1 + _operators.size() > depth_limit) {
throw expressions_syntax_error(format("Document path exceeded {} nesting levels", depth_limit));
}
}
std::ostream& operator<<(std::ostream& os, const path& p) {
os << p.root();
for (const auto& op : p.operators()) {
std::visit(overloaded_functor {
[&] (const std::string& member) {
os << '.' << member;
},
[&] (unsigned index) {
os << '[' << index << ']';
}
}, op);
}
return os;
}
} // namespace parsed
// The following resolve_*() functions resolve references in parsed
// expressions of different types. Resolving a parsed expression means
// replacing:
// 1. In parsed::path objects, replace references like "#name" with the
// attribute name from ExpressionAttributeNames,
// 2. In parsed::constant objects, replace references like ":value" with
// the value from ExpressionAttributeValues.
// These function also track which name and value references were used, to
// allow complaining if some remain unused.
// Note that the resolve_*() functions modify the expressions in-place,
// so if we ever intend to cache parsed expression, we need to pass a copy
// into this function.
//
// Doing the "resolving" stage before the evaluation stage has two benefits.
// First, it allows us to be compatible with DynamoDB in catching unused
// names and values (see issue #6572). Second, in the FilterExpression case,
// we need to resolve the expression just once but then use it many times
// (once for each item to be filtered).
static std::optional<std::string> resolve_path_component(const std::string& column_name,
const rjson::value* expression_attribute_names,
std::unordered_set<std::string>& used_attribute_names) {
if (column_name.size() > 0 && column_name.front() == '#') {
if (!expression_attribute_names) {
throw api_error::validation(
format("ExpressionAttributeNames missing, entry '{}' required by expression", column_name));
}
const rjson::value* value = rjson::find(*expression_attribute_names, column_name);
if (!value || !value->IsString()) {
throw api_error::validation(
format("ExpressionAttributeNames missing entry '{}' required by expression", column_name));
}
used_attribute_names.emplace(column_name);
return std::string(rjson::to_string_view(*value));
}
return std::nullopt;
}
static void resolve_path(parsed::path& p,
const rjson::value* expression_attribute_names,
std::unordered_set<std::string>& used_attribute_names) {
std::optional<std::string> r = resolve_path_component(p.root(), expression_attribute_names, used_attribute_names);
if (r) {
p.set_root(std::move(*r));
}
for (auto& op : p.operators()) {
std::visit(overloaded_functor {
[&] (std::string& s) {
r = resolve_path_component(s, expression_attribute_names, used_attribute_names);
if (r) {
s = std::move(*r);
}
},
[&] (unsigned index) {
// nothing to resolve
}
}, op);
}
}
static void resolve_constant(parsed::constant& c,
const rjson::value* expression_attribute_values,
std::unordered_set<std::string>& used_attribute_values) {
std::visit(overloaded_functor {
[&] (const std::string& valref) {
if (!expression_attribute_values) {
throw api_error::validation(
format("ExpressionAttributeValues missing, entry '{}' required by expression", valref));
}
const rjson::value* value = rjson::find(*expression_attribute_values, valref);
if (!value) {
throw api_error::validation(
format("ExpressionAttributeValues missing entry '{}' required by expression", valref));
}
if (value->IsNull()) {
throw api_error::validation(
format("ExpressionAttributeValues null value for entry '{}' required by expression", valref));
}
validate_value(*value, "ExpressionAttributeValues");
used_attribute_values.emplace(valref);
c.set(*value);
},
[&] (const parsed::constant::literal& lit) {
// Nothing to do, already resolved
}
}, c._value);
}
void resolve_value(parsed::value& rhs,
const rjson::value* expression_attribute_names,
const rjson::value* expression_attribute_values,
std::unordered_set<std::string>& used_attribute_names,
std::unordered_set<std::string>& used_attribute_values) {
std::visit(overloaded_functor {
[&] (parsed::constant& c) {
resolve_constant(c, expression_attribute_values, used_attribute_values);
},
[&] (parsed::value::function_call& f) {
for (parsed::value& value : f._parameters) {
resolve_value(value, expression_attribute_names, expression_attribute_values,
used_attribute_names, used_attribute_values);
}
},
[&] (parsed::path& p) {
resolve_path(p, expression_attribute_names, used_attribute_names);
}
}, rhs._value);
}
void resolve_set_rhs(parsed::set_rhs& rhs,
const rjson::value* expression_attribute_names,
const rjson::value* expression_attribute_values,
std::unordered_set<std::string>& used_attribute_names,
std::unordered_set<std::string>& used_attribute_values) {
resolve_value(rhs._v1, expression_attribute_names, expression_attribute_values,
used_attribute_names, used_attribute_values);
if (rhs._op != 'v') {
resolve_value(rhs._v2, expression_attribute_names, expression_attribute_values,
used_attribute_names, used_attribute_values);
}
}
void resolve_update_expression(parsed::update_expression& ue,
const rjson::value* expression_attribute_names,
const rjson::value* expression_attribute_values,
std::unordered_set<std::string>& used_attribute_names,
std::unordered_set<std::string>& used_attribute_values) {
for (parsed::update_expression::action& action : ue.actions()) {
resolve_path(action._path, expression_attribute_names, used_attribute_names);
std::visit(overloaded_functor {
[&] (parsed::update_expression::action::set& a) {
resolve_set_rhs(a._rhs, expression_attribute_names, expression_attribute_values,
used_attribute_names, used_attribute_values);
},
[&] (parsed::update_expression::action::remove& a) {
// nothing to do
},
[&] (parsed::update_expression::action::add& a) {
resolve_constant(a._valref, expression_attribute_values, used_attribute_values);
},
[&] (parsed::update_expression::action::del& a) {
resolve_constant(a._valref, expression_attribute_values, used_attribute_values);
}
}, action._action);
}
}
static void resolve_primitive_condition(parsed::primitive_condition& pc,
const rjson::value* expression_attribute_names,
const rjson::value* expression_attribute_values,
std::unordered_set<std::string>& used_attribute_names,
std::unordered_set<std::string>& used_attribute_values) {
for (parsed::value& value : pc._values) {
resolve_value(value,
expression_attribute_names, expression_attribute_values,
used_attribute_names, used_attribute_values);
}
}
void resolve_condition_expression(parsed::condition_expression& ce,
const rjson::value* expression_attribute_names,
const rjson::value* expression_attribute_values,
std::unordered_set<std::string>& used_attribute_names,
std::unordered_set<std::string>& used_attribute_values) {
std::visit(overloaded_functor {
[&] (parsed::primitive_condition& cond) {
resolve_primitive_condition(cond,
expression_attribute_names, expression_attribute_values,
used_attribute_names, used_attribute_values);
},
[&] (parsed::condition_expression::condition_list& list) {
for (parsed::condition_expression& cond : list.conditions) {
resolve_condition_expression(cond,
expression_attribute_names, expression_attribute_values,
used_attribute_names, used_attribute_values);
}
}
}, ce._expression);
}
void resolve_projection_expression(std::vector<parsed::path>& pe,
const rjson::value* expression_attribute_names,
std::unordered_set<std::string>& used_attribute_names) {
for (parsed::path& p : pe) {
resolve_path(p, expression_attribute_names, used_attribute_names);
}
}
// condition_expression_on() checks whether a condition_expression places any
// condition on the given attribute. It can be useful, for example, for
// checking whether the condition tries to restrict a key column.
static bool value_on(const parsed::value& v, std::string_view attribute) {
return std::visit(overloaded_functor {
[&] (const parsed::constant& c) {
return false;
},
[&] (const parsed::value::function_call& f) {
for (const parsed::value& value : f._parameters) {
if (value_on(value, attribute)) {
return true;
}
}
return false;
},
[&] (const parsed::path& p) {
return p.root() == attribute;
}
}, v._value);
}
static bool primitive_condition_on(const parsed::primitive_condition& pc, std::string_view attribute) {
for (const parsed::value& value : pc._values) {
if (value_on(value, attribute)) {
return true;
}
}
return false;
}
bool condition_expression_on(const parsed::condition_expression& ce, std::string_view attribute) {
return std::visit(overloaded_functor {
[&] (const parsed::primitive_condition& cond) {
return primitive_condition_on(cond, attribute);
},
[&] (const parsed::condition_expression::condition_list& list) {
for (const parsed::condition_expression& cond : list.conditions) {
if (condition_expression_on(cond, attribute)) {
return true;
}
}
return false;
}
}, ce._expression);
}
// for_condition_expression_on() runs a given function over all the attributes
// mentioned in the expression. If the same attribute is mentioned more than
// once, the function will be called more than once for the same attribute.
static void for_value_on(const parsed::value& v, const noncopyable_function<void(std::string_view)>& func) {
std::visit(overloaded_functor {
[&] (const parsed::constant& c) { },
[&] (const parsed::value::function_call& f) {
for (const parsed::value& value : f._parameters) {
for_value_on(value, func);
}
},
[&] (const parsed::path& p) {
func(p.root());
}
}, v._value);
}
void for_condition_expression_on(const parsed::condition_expression& ce, const noncopyable_function<void(std::string_view)>& func) {
std::visit(overloaded_functor {
[&] (const parsed::primitive_condition& cond) {
for (const parsed::value& value : cond._values) {
for_value_on(value, func);
}
},
[&] (const parsed::condition_expression::condition_list& list) {
for (const parsed::condition_expression& cond : list.conditions) {
for_condition_expression_on(cond, func);
}
}
}, ce._expression);
}
// The following calculate_value() functions calculate, or evaluate, a parsed
// expression. The parsed expression is assumed to have been "resolved", with
// the matching resolve_* function.
// calculate_size() is ConditionExpression's size() function, i.e., it takes
// a JSON-encoded value and returns its "size" as defined differently for the
// different types - also as a JSON-encoded number.
// It return a JSON-encoded "null" value if this value's type has no size
// defined. Comparisons against this non-numeric value will later fail.
static rjson::value calculate_size(const rjson::value& v) {
// NOTE: If v is improperly formatted for our JSON value encoding, it
// must come from the request itself, not from the database, so it makes
// sense to throw a ValidationException if we see such a problem.
if (!v.IsObject() || v.MemberCount() != 1) {
throw api_error::validation(format("invalid object: {}", v));
}
auto it = v.MemberBegin();
int ret;
if (it->name == "S") {
if (!it->value.IsString()) {
throw api_error::validation(format("invalid string: {}", v));
}
ret = it->value.GetStringLength();
} else if (it->name == "NS" || it->name == "SS" || it->name == "BS" || it->name == "L") {
if (!it->value.IsArray()) {
throw api_error::validation(format("invalid set: {}", v));
}
ret = it->value.Size();
} else if (it->name == "M") {
if (!it->value.IsObject()) {
throw api_error::validation(format("invalid map: {}", v));
}
ret = it->value.MemberCount();
} else if (it->name == "B") {
if (!it->value.IsString()) {
throw api_error::validation(format("invalid byte string: {}", v));
}
ret = base64_decoded_len(rjson::to_string_view(it->value));
} else {
rjson::value json_ret = rjson::empty_object();
rjson::add(json_ret, "null", rjson::value(true));
return json_ret;
}
rjson::value json_ret = rjson::empty_object();
rjson::add(json_ret, "N", rjson::from_string(std::to_string(ret)));
return json_ret;
}
static const rjson::value& calculate_value(const parsed::constant& c) {
return std::visit(overloaded_functor {
[&] (const parsed::constant::literal& v) -> const rjson::value& {
return *v;
},
[&] (const std::string& valref) -> const rjson::value& {
// Shouldn't happen, we should have called resolve_value() earlier
// and replaced the value reference by the literal constant.
throw std::logic_error("calculate_value() called before resolve_value()");
}
}, c._value);
}
static rjson::value to_bool_json(bool b) {
rjson::value json_ret = rjson::empty_object();
rjson::add(json_ret, "BOOL", rjson::value(b));
return json_ret;
}
static bool known_type(std::string_view type) {
static thread_local const std::unordered_set<std::string_view> types = {
"N", "S", "B", "NS", "SS", "BS", "L", "M", "NULL", "BOOL"
};
return types.contains(type);
}
using function_handler_type = rjson::value(calculate_value_caller, const rjson::value*, const parsed::value::function_call&);
static const
std::unordered_map<std::string_view, function_handler_type*> function_handlers {
{"list_append", [] (calculate_value_caller caller, const rjson::value* previous_item, const parsed::value::function_call& f) {
if (caller != calculate_value_caller::UpdateExpression) {
throw api_error::validation(
format("{}: list_append() not allowed here", caller));
}
if (f._parameters.size() != 2) {
throw api_error::validation(
format("{}: list_append() accepts 2 parameters, got {}", caller, f._parameters.size()));
}
rjson::value v1 = calculate_value(f._parameters[0], caller, previous_item);
rjson::value v2 = calculate_value(f._parameters[1], caller, previous_item);
rjson::value ret = list_concatenate(v1, v2);
if (ret.IsNull()) {
throw api_error::validation("UpdateExpression: list_append() given a non-list");
}
return ret;
}
},
{"if_not_exists", [] (calculate_value_caller caller, const rjson::value* previous_item, const parsed::value::function_call& f) {
if (caller != calculate_value_caller::UpdateExpression) {
throw api_error::validation(
format("{}: if_not_exists() not allowed here", caller));
}
if (f._parameters.size() != 2) {
throw api_error::validation(
format("{}: if_not_exists() accepts 2 parameters, got {}", caller, f._parameters.size()));
}
if (!std::holds_alternative<parsed::path>(f._parameters[0]._value)) {
throw api_error::validation(
format("{}: if_not_exists() must include path as its first argument", caller));
}
rjson::value v1 = calculate_value(f._parameters[0], caller, previous_item);
rjson::value v2 = calculate_value(f._parameters[1], caller, previous_item);
return v1.IsNull() ? std::move(v2) : std::move(v1);
}
},
{"size", [] (calculate_value_caller caller, const rjson::value* previous_item, const parsed::value::function_call& f) {
if (caller != calculate_value_caller::ConditionExpression) {
throw api_error::validation(
format("{}: size() not allowed here", caller));
}
if (f._parameters.size() != 1) {
throw api_error::validation(
format("{}: size() accepts 1 parameter, got {}", caller, f._parameters.size()));
}
rjson::value v = calculate_value(f._parameters[0], caller, previous_item);
return calculate_size(v);
}
},
{"attribute_exists", [] (calculate_value_caller caller, const rjson::value* previous_item, const parsed::value::function_call& f) {
if (caller != calculate_value_caller::ConditionExpressionAlone) {
throw api_error::validation(
format("{}: attribute_exists() not allowed here", caller));
}
if (f._parameters.size() != 1) {
throw api_error::validation(
format("{}: attribute_exists() accepts 1 parameter, got {}", caller, f._parameters.size()));
}
if (!std::holds_alternative<parsed::path>(f._parameters[0]._value)) {
throw api_error::validation(
format("{}: attribute_exists()'s parameter must be a path", caller));
}
rjson::value v = calculate_value(f._parameters[0], caller, previous_item);
return to_bool_json(!v.IsNull());
}
},
{"attribute_not_exists", [] (calculate_value_caller caller, const rjson::value* previous_item, const parsed::value::function_call& f) {
if (caller != calculate_value_caller::ConditionExpressionAlone) {
throw api_error::validation(
format("{}: attribute_not_exists() not allowed here", caller));
}
if (f._parameters.size() != 1) {
throw api_error::validation(
format("{}: attribute_not_exists() accepts 1 parameter, got {}", caller, f._parameters.size()));
}
if (!std::holds_alternative<parsed::path>(f._parameters[0]._value)) {
throw api_error::validation(
format("{}: attribute_not_exists()'s parameter must be a path", caller));
}
rjson::value v = calculate_value(f._parameters[0], caller, previous_item);
return to_bool_json(v.IsNull());
}
},
{"attribute_type", [] (calculate_value_caller caller, const rjson::value* previous_item, const parsed::value::function_call& f) {
if (caller != calculate_value_caller::ConditionExpressionAlone) {
throw api_error::validation(
format("{}: attribute_type() not allowed here", caller));
}
if (f._parameters.size() != 2) {
throw api_error::validation(
format("{}: attribute_type() accepts 2 parameters, got {}", caller, f._parameters.size()));
}
// There is no real reason for the following check (not
// allowing the type to come from a document attribute), but
// DynamoDB does this check, so we do too...
if (!f._parameters[1].is_constant()) {
throw api_error::validation(
format("{}: attribute_types()'s first parameter must be an expression attribute", caller));
}
rjson::value v0 = calculate_value(f._parameters[0], caller, previous_item);
rjson::value v1 = calculate_value(f._parameters[1], caller, previous_item);
if (v1.IsObject() && v1.MemberCount() == 1 && v1.MemberBegin()->name == "S") {
// If the type parameter is not one of the legal types
// we should generate an error, not a failed condition:
if (!known_type(rjson::to_string_view(v1.MemberBegin()->value))) {
throw api_error::validation(
format("{}: attribute_types()'s second parameter, {}, is not a known type",
caller, v1.MemberBegin()->value));
}
if (v0.IsObject() && v0.MemberCount() == 1) {
return to_bool_json(v1.MemberBegin()->value == v0.MemberBegin()->name);
} else {
return to_bool_json(false);
}
} else {
throw api_error::validation(
format("{}: attribute_type() second parameter must refer to a string, got {}", caller, v1));
}
}
},
{"begins_with", [] (calculate_value_caller caller, const rjson::value* previous_item, const parsed::value::function_call& f) {
if (caller != calculate_value_caller::ConditionExpressionAlone) {
throw api_error::validation(
format("{}: begins_with() not allowed here", caller));
}
if (f._parameters.size() != 2) {
throw api_error::validation(
format("{}: begins_with() accepts 2 parameters, got {}", caller, f._parameters.size()));
}
rjson::value v1 = calculate_value(f._parameters[0], caller, previous_item);
rjson::value v2 = calculate_value(f._parameters[1], caller, previous_item);
return to_bool_json(check_BEGINS_WITH(v1.IsNull() ? nullptr : &v1, v2,
f._parameters[0].is_constant(), f._parameters[1].is_constant()));
}
},
{"contains", [] (calculate_value_caller caller, const rjson::value* previous_item, const parsed::value::function_call& f) {
if (caller != calculate_value_caller::ConditionExpressionAlone) {
throw api_error::validation(
format("{}: contains() not allowed here", caller));
}
if (f._parameters.size() != 2) {
throw api_error::validation(
format("{}: contains() accepts 2 parameters, got {}", caller, f._parameters.size()));
}
rjson::value v1 = calculate_value(f._parameters[0], caller, previous_item);
rjson::value v2 = calculate_value(f._parameters[1], caller, previous_item);
return to_bool_json(check_CONTAINS(v1.IsNull() ? nullptr : &v1, v2,
f._parameters[0].is_constant(), f._parameters[1].is_constant()));
}
},
};
// Given a parsed::path and an item read from the table, extract the value
// of a certain attribute path, such as "a" or "a.b.c[3]". Returns a null
// value if the item or the requested attribute does not exist.
// Note that the item is assumed to be encoded in JSON using DynamoDB
// conventions - each level of a nested document is a map with one key -
// a type (e.g., "M" for map) - and its value is the representation of
// that value.
static rjson::value extract_path(const rjson::value* item,
const parsed::path& p, calculate_value_caller caller) {
if (!item) {
return rjson::null_value();
}
const rjson::value* v = rjson::find(*item, p.root());
if (!v) {
return rjson::null_value();
}
for (const auto& op : p.operators()) {
if (!v->IsObject() || v->MemberCount() != 1) {
// This shouldn't happen. We shouldn't have stored malformed
// objects. But today Alternator does not validate the structure
// of nested documents before storing them, so this can happen on
// read.
throw api_error::validation(format("{}: malformed item read: {}", *item));
}
const char* type = v->MemberBegin()->name.GetString();
v = &(v->MemberBegin()->value);
std::visit(overloaded_functor {
[&] (const std::string& member) {
if (type[0] == 'M' && v->IsObject()) {
v = rjson::find(*v, member);
} else {
v = nullptr;
}
},
[&] (unsigned index) {
if (type[0] == 'L' && v->IsArray() && index < v->Size()) {
v = &(v->GetArray()[index]);
} else {
v = nullptr;
}
}
}, op);
if (!v) {
return rjson::null_value();
}
}
return rjson::copy(*v);
}
// Given a parsed::value, which can refer either to a constant value from
// ExpressionAttributeValues, to the value of some attribute, or to a function
// of other values, this function calculates the resulting value.
// "caller" determines which expression - ConditionExpression or
// UpdateExpression - is asking for this value. We need to know this because
// DynamoDB allows a different choice of functions for different expressions.
rjson::value calculate_value(const parsed::value& v,
calculate_value_caller caller,
const rjson::value* previous_item) {
return std::visit(overloaded_functor {
[&] (const parsed::constant& c) -> rjson::value {
return rjson::copy(calculate_value(c));
},
[&] (const parsed::value::function_call& f) -> rjson::value {
auto function_it = function_handlers.find(std::string_view(f._function_name));
if (function_it == function_handlers.end()) {
throw api_error::validation(
format("{}: unknown function '{}' called.", caller, f._function_name));
}
return function_it->second(caller, previous_item, f);
},
[&] (const parsed::path& p) -> rjson::value {
return extract_path(previous_item, p, caller);
}
}, v._value);
}
// Same as calculate_value() above, except takes a set_rhs, which may be
// either a single value, or v1+v2 or v1-v2.
rjson::value calculate_value(const parsed::set_rhs& rhs,
const rjson::value* previous_item) {
switch (rhs._op) {
case 'v':
return calculate_value(rhs._v1, calculate_value_caller::UpdateExpression, previous_item);
case '+': {
rjson::value v1 = calculate_value(rhs._v1, calculate_value_caller::UpdateExpression, previous_item);
rjson::value v2 = calculate_value(rhs._v2, calculate_value_caller::UpdateExpression, previous_item);
return number_add(v1, v2);
}
case '-': {
rjson::value v1 = calculate_value(rhs._v1, calculate_value_caller::UpdateExpression, previous_item);
rjson::value v2 = calculate_value(rhs._v2, calculate_value_caller::UpdateExpression, previous_item);
return number_subtract(v1, v2);
}
}
// Can't happen
return rjson::null_value();
}
} // namespace alternator

View File

@@ -1,249 +0,0 @@
/*
* Copyright 2019-present ScyllaDB
*/
/*
* SPDX-License-Identifier: AGPL-3.0-or-later
*/
/*
* The DynamoDB protocol is based on JSON, and most DynamoDB requests
* describe the operation and its parameters via JSON objects such as maps
* and lists. Nevertheless, in some types of requests an "expression" is
* passed as a single string, and we need to parse this string. These
* cases include:
* 1. Attribute paths, such as "a[3].b.c", are used in projection
* expressions as well as inside other expressions described below.
* 2. Condition expressions, such as "(NOT (a=b OR c=d)) AND e=f",
* used in conditional updates, filters, and other places.
* 3. Update expressions, such as "SET #a.b = :x, c = :y DELETE d"
*
* All these expression syntaxes are very simple: Most of them could be
* parsed as regular expressions, and the parenthesized condition expression
* could be done with a simple hand-written lexical analyzer and recursive-
* descent parser. Nevertheless, we decided to specify these parsers in the
* ANTLR3 language already used in the Scylla project, hopefully making these
* parsers easier to reason about, and easier to change if needed - and
* reducing the amount of boiler-plate code.
*/
grammar expressions;
options {
language = Cpp;
}
@parser::namespace{alternator}
@lexer::namespace{alternator}
/* TODO: explain what these traits things are. I haven't seen them explained
* in any document... Compilation fails without these fail because a definition
* of "expressionsLexerTraits" and "expressionParserTraits" is needed.
*/
@lexer::traits {
class expressionsLexer;
class expressionsParser;
typedef antlr3::Traits<expressionsLexer, expressionsParser> expressionsLexerTraits;
}
@parser::traits {
typedef expressionsLexerTraits expressionsParserTraits;
}
@lexer::header {
#include "alternator/expressions.hh"
// ANTLR generates a bunch of unused variables and functions. Yuck...
#pragma GCC diagnostic ignored "-Wunused-variable"
#pragma GCC diagnostic ignored "-Wunused-function"
}
@parser::header {
#include "expressionsLexer.hpp"
}
/* By default, ANTLR3 composes elaborate syntax-error messages, saying which
* token was unexpected, where, and so on on, but then dutifully writes these
* error messages to the standard error, and returns from the parser as if
* everything was fine, with a half-constructed output object! If we define
* the "displayRecognitionError" method, it will be called upon to build this
* error message, and we can instead throw an exception to stop the parsing
* immediately. This is good enough for now, for our simple needs, but if
* we ever want to show more information about the syntax error, Cql3.g
* contains an elaborate implementation (it would be nice if we could reuse
* it, not duplicate it).
* Unfortunately, we have to repeat the same definition twice - once for the
* parser, and once for the lexer.
*/
@parser::context {
void displayRecognitionError(ANTLR_UINT8** token_names, ExceptionBaseType* ex) {
throw expressions_syntax_error("syntax error");
}
}
@lexer::context {
void displayRecognitionError(ANTLR_UINT8** token_names, ExceptionBaseType* ex) {
throw expressions_syntax_error("syntax error");
}
}
/*
* Lexical analysis phase, i.e., splitting the input up to tokens.
* Lexical analyzer rules have names starting in capital letters.
* "fragment" rules do not generate tokens, and are just aliases used to
* make other rules more readable.
* Characters *not* listed here, e.g., '=', '(', etc., will be handled
* as individual tokens on their own right.
* Whitespace spans are skipped, so do not generate tokens.
*/
WHITESPACE: (' ' | '\t' | '\n' | '\r')+ { skip(); };
/* shortcuts for case-insensitive keywords */
fragment A:('a'|'A');
fragment B:('b'|'B');
fragment C:('c'|'C');
fragment D:('d'|'D');
fragment E:('e'|'E');
fragment F:('f'|'F');
fragment G:('g'|'G');
fragment H:('h'|'H');
fragment I:('i'|'I');
fragment J:('j'|'J');
fragment K:('k'|'K');
fragment L:('l'|'L');
fragment M:('m'|'M');
fragment N:('n'|'N');
fragment O:('o'|'O');
fragment P:('p'|'P');
fragment Q:('q'|'Q');
fragment R:('r'|'R');
fragment S:('s'|'S');
fragment T:('t'|'T');
fragment U:('u'|'U');
fragment V:('v'|'V');
fragment W:('w'|'W');
fragment X:('x'|'X');
fragment Y:('y'|'Y');
fragment Z:('z'|'Z');
/* These keywords must be appear before the generic NAME token below,
* because NAME matches too, and the first to match wins.
*/
SET: S E T;
REMOVE: R E M O V E;
ADD: A D D;
DELETE: D E L E T E;
AND: A N D;
OR: O R;
NOT: N O T;
BETWEEN: B E T W E E N;
IN: I N;
fragment ALPHA: 'A'..'Z' | 'a'..'z';
fragment DIGIT: '0'..'9';
fragment ALNUM: ALPHA | DIGIT | '_';
INTEGER: DIGIT+;
NAME: ALPHA ALNUM*;
NAMEREF: '#' ALNUM+;
VALREF: ':' ALNUM+;
/*
* Parsing phase - parsing the string of tokens generated by the lexical
* analyzer defined above.
*/
path_component: NAME | NAMEREF;
path returns [parsed::path p]:
root=path_component { $p.set_root($root.text); }
( '.' name=path_component { $p.add_dot($name.text); }
| '[' INTEGER ']' { $p.add_index(std::stoi($INTEGER.text)); }
)*;
value returns [parsed::value v]:
VALREF { $v.set_valref($VALREF.text); }
| path { $v.set_path($path.p); }
| NAME { $v.set_func_name($NAME.text); }
'(' x=value { $v.add_func_parameter($x.v); }
(',' x=value { $v.add_func_parameter($x.v); })*
')'
;
update_expression_set_rhs returns [parsed::set_rhs rhs]:
v=value { $rhs.set_value(std::move($v.v)); }
( '+' v=value { $rhs.set_plus(std::move($v.v)); }
| '-' v=value { $rhs.set_minus(std::move($v.v)); }
)?
;
update_expression_set_action returns [parsed::update_expression::action a]:
path '=' rhs=update_expression_set_rhs { $a.assign_set($path.p, $rhs.rhs); };
update_expression_remove_action returns [parsed::update_expression::action a]:
path { $a.assign_remove($path.p); };
update_expression_add_action returns [parsed::update_expression::action a]:
path VALREF { $a.assign_add($path.p, $VALREF.text); };
update_expression_delete_action returns [parsed::update_expression::action a]:
path VALREF { $a.assign_del($path.p, $VALREF.text); };
update_expression_clause returns [parsed::update_expression e]:
SET s=update_expression_set_action { $e.add(s); }
(',' s=update_expression_set_action { $e.add(s); })*
| REMOVE r=update_expression_remove_action { $e.add(r); }
(',' r=update_expression_remove_action { $e.add(r); })*
| ADD a=update_expression_add_action { $e.add(a); }
(',' a=update_expression_add_action { $e.add(a); })*
| DELETE d=update_expression_delete_action { $e.add(d); }
(',' d=update_expression_delete_action { $e.add(d); })*
;
// Note the "EOF" token at the end of the update expression. We want to the
// parser to match the entire string given to it - not just its beginning!
update_expression returns [parsed::update_expression e]:
(update_expression_clause { e.append($update_expression_clause.e); })* EOF;
projection_expression returns [std::vector<parsed::path> v]:
p=path { $v.push_back(std::move($p.p)); }
(',' p=path { $v.push_back(std::move($p.p)); } )* EOF;
primitive_condition returns [parsed::primitive_condition c]:
v=value { $c.add_value(std::move($v.v));
$c.set_operator(parsed::primitive_condition::type::VALUE); }
( ( '=' { $c.set_operator(parsed::primitive_condition::type::EQ); }
| '<' '>' { $c.set_operator(parsed::primitive_condition::type::NE); }
| '<' { $c.set_operator(parsed::primitive_condition::type::LT); }
| '<' '=' { $c.set_operator(parsed::primitive_condition::type::LE); }
| '>' { $c.set_operator(parsed::primitive_condition::type::GT); }
| '>' '=' { $c.set_operator(parsed::primitive_condition::type::GE); }
)
v=value { $c.add_value(std::move($v.v)); }
| BETWEEN { $c.set_operator(parsed::primitive_condition::type::BETWEEN); }
v=value { $c.add_value(std::move($v.v)); }
AND
v=value { $c.add_value(std::move($v.v)); }
| IN '(' { $c.set_operator(parsed::primitive_condition::type::IN); }
v=value { $c.add_value(std::move($v.v)); }
(',' v=value { $c.add_value(std::move($v.v)); })*
')'
)?
;
// The following rules for parsing boolean expressions are verbose and
// somewhat strange because of Antlr 3's limitations on recursive rules,
// common rule prefixes, and (lack of) support for operator precedence.
// These rules could have been written more clearly using a more powerful
// parser generator - such as Yacc.
boolean_expression returns [parsed::condition_expression e]:
b=boolean_expression_1 { $e.append(std::move($b.e), '|'); }
(OR b=boolean_expression_1 { $e.append(std::move($b.e), '|'); } )*
;
boolean_expression_1 returns [parsed::condition_expression e]:
b=boolean_expression_2 { $e.append(std::move($b.e), '&'); }
(AND b=boolean_expression_2 { $e.append(std::move($b.e), '&'); } )*
;
boolean_expression_2 returns [parsed::condition_expression e]:
p=primitive_condition { $e.set_primitive(std::move($p.c)); }
| NOT b=boolean_expression_2 { $e = std::move($b.e); $e.apply_not(); }
| '(' b=boolean_expression ')' { $e = std::move($b.e); }
;
condition_expression returns [parsed::condition_expression e]:
boolean_expression { e=std::move($boolean_expression.e); } EOF;

View File

@@ -1,89 +0,0 @@
/*
* Copyright 2019-present ScyllaDB
*/
/*
* SPDX-License-Identifier: AGPL-3.0-or-later
*/
#pragma once
#include <string>
#include <stdexcept>
#include <vector>
#include <unordered_set>
#include <string_view>
#include <seastar/util/noncopyable_function.hh>
#include "expressions_types.hh"
#include "utils/rjson.hh"
namespace alternator {
class expressions_syntax_error : public std::runtime_error {
public:
using runtime_error::runtime_error;
};
parsed::update_expression parse_update_expression(std::string_view query);
std::vector<parsed::path> parse_projection_expression(std::string_view query);
parsed::condition_expression parse_condition_expression(std::string_view query);
void resolve_update_expression(parsed::update_expression& ue,
const rjson::value* expression_attribute_names,
const rjson::value* expression_attribute_values,
std::unordered_set<std::string>& used_attribute_names,
std::unordered_set<std::string>& used_attribute_values);
void resolve_projection_expression(std::vector<parsed::path>& pe,
const rjson::value* expression_attribute_names,
std::unordered_set<std::string>& used_attribute_names);
void resolve_condition_expression(parsed::condition_expression& ce,
const rjson::value* expression_attribute_names,
const rjson::value* expression_attribute_values,
std::unordered_set<std::string>& used_attribute_names,
std::unordered_set<std::string>& used_attribute_values);
void validate_value(const rjson::value& v, const char* caller);
bool condition_expression_on(const parsed::condition_expression& ce, std::string_view attribute);
// for_condition_expression_on() runs the given function on the attributes
// that the expression uses. It may run for the same attribute more than once
// if the same attribute is used more than once in the expression.
void for_condition_expression_on(const parsed::condition_expression& ce, const noncopyable_function<void(std::string_view)>& func);
// calculate_value() behaves slightly different (especially, different
// functions supported) when used in different types of expressions, as
// enumerated in this enum:
enum class calculate_value_caller {
UpdateExpression, ConditionExpression, ConditionExpressionAlone
};
inline std::ostream& operator<<(std::ostream& out, calculate_value_caller caller) {
switch (caller) {
case calculate_value_caller::UpdateExpression:
out << "UpdateExpression";
break;
case calculate_value_caller::ConditionExpression:
out << "ConditionExpression";
break;
case calculate_value_caller::ConditionExpressionAlone:
out << "ConditionExpression";
break;
default:
out << "unknown type of expression";
break;
}
return out;
}
rjson::value calculate_value(const parsed::value& v,
calculate_value_caller caller,
const rjson::value* previous_item);
rjson::value calculate_value(const parsed::set_rhs& rhs,
const rjson::value* previous_item);
} /* namespace alternator */

View File

@@ -1,257 +0,0 @@
/*
* Copyright 2019-present ScyllaDB
*/
/*
* SPDX-License-Identifier: AGPL-3.0-or-later
*/
#pragma once
#include <vector>
#include <string>
#include <variant>
#include <seastar/core/shared_ptr.hh>
#include "utils/rjson.hh"
/*
* Parsed representation of expressions and their components.
*
* Types in alternator::parsed namespace are used for holding the parse
* tree - objects generated by the Antlr rules after parsing an expression.
* Because of the way Antlr works, all these objects are default-constructed
* first, and then assigned when the rule is completed, so all these types
* have only default constructors - but setter functions to set them later.
*/
namespace alternator {
namespace parsed {
// "path" is an attribute's path in a document, e.g., a.b[3].c.
class path {
// All paths have a "root", a top-level attribute, and any number of
// "dereference operators" - each either an index (e.g., "[2]") or a
// dot (e.g., ".xyz").
std::string _root;
std::vector<std::variant<std::string, unsigned>> _operators;
// It is useful to limit the depth of a user-specified path, because is
// allows us to use recursive algorithms without worrying about recursion
// depth. DynamoDB officially limits the length of paths to 32 components
// (including the root) so let's use the same limit.
static constexpr unsigned depth_limit = 32;
void check_depth_limit();
public:
void set_root(std::string root) {
_root = std::move(root);
}
void add_index(unsigned i) {
_operators.emplace_back(i);
check_depth_limit();
}
void add_dot(std::string(name)) {
_operators.emplace_back(std::move(name));
check_depth_limit();
}
const std::string& root() const {
return _root;
}
bool has_operators() const {
return !_operators.empty();
}
const std::vector<std::variant<std::string, unsigned>>& operators() const {
return _operators;
}
std::vector<std::variant<std::string, unsigned>>& operators() {
return _operators;
}
friend std::ostream& operator<<(std::ostream&, const path&);
};
// When an expression is first parsed, all constants are references, like
// ":val1", into ExpressionAttributeValues. This uses std::string() variant.
// The resolve_value() function replaces these constants by the JSON item
// extracted from the ExpressionAttributeValues.
struct constant {
// We use lw_shared_ptr<rjson::value> just to make rjson::value copyable,
// to make this entire object copyable as ANTLR needs.
using literal = lw_shared_ptr<rjson::value>;
std::variant<std::string, literal> _value;
void set(const rjson::value& v) {
_value = make_lw_shared<rjson::value>(rjson::copy(v));
}
void set(std::string& s) {
_value = s;
}
};
// "value" is is a value used in the right hand side of an assignment
// expression, "SET a = ...". It can be a constant (a reference to a value
// included in the request, e.g., ":val"), a path to an attribute from the
// existing item (e.g., "a.b[3].c"), or a function of other such values.
// Note that the real right-hand-side of an assignment is actually a bit
// more general - it allows either a value, or a value+value or value-value -
// see class set_rhs below.
struct value {
struct function_call {
std::string _function_name;
std::vector<value> _parameters;
};
std::variant<constant, path, function_call> _value;
void set_constant(constant c) {
_value = std::move(c);
}
void set_valref(std::string s) {
_value = constant { std::move(s) };
}
void set_path(path p) {
_value = std::move(p);
}
void set_func_name(std::string s) {
_value = function_call {std::move(s), {}};
}
void add_func_parameter(value v) {
std::get<function_call>(_value)._parameters.emplace_back(std::move(v));
}
bool is_constant() const {
return std::holds_alternative<constant>(_value);
}
bool is_path() const {
return std::holds_alternative<path>(_value);
}
bool is_func() const {
return std::holds_alternative<function_call>(_value);
}
};
// The right-hand-side of a SET in an update expression can be either a
// single value (see above), or value+value, or value-value.
class set_rhs {
public:
char _op; // '+', '-', or 'v''
value _v1;
value _v2;
void set_value(value&& v1) {
_op = 'v';
_v1 = std::move(v1);
}
void set_plus(value&& v2) {
_op = '+';
_v2 = std::move(v2);
}
void set_minus(value&& v2) {
_op = '-';
_v2 = std::move(v2);
}
};
class update_expression {
public:
struct action {
path _path;
struct set {
set_rhs _rhs;
};
struct remove {
};
struct add {
constant _valref;
};
struct del {
constant _valref;
};
std::variant<set, remove, add, del> _action;
void assign_set(path p, set_rhs rhs) {
_path = std::move(p);
_action = set { std::move(rhs) };
}
void assign_remove(path p) {
_path = std::move(p);
_action = remove { };
}
void assign_add(path p, std::string v) {
_path = std::move(p);
_action = add { constant { std::move(v) } };
}
void assign_del(path p, std::string v) {
_path = std::move(p);
_action = del { constant { std::move(v) } };
}
};
private:
std::vector<action> _actions;
bool seen_set = false;
bool seen_remove = false;
bool seen_add = false;
bool seen_del = false;
public:
void add(action a);
void append(update_expression other);
bool empty() const {
return _actions.empty();
}
const std::vector<action>& actions() const {
return _actions;
}
std::vector<action>& actions() {
return _actions;
}
};
// A primitive_condition is a condition expression involving one condition,
// while the full condition_expression below adds boolean logic over these
// primitive conditions.
// The supported primitive conditions are:
// 1. Binary operators - v1 OP v2, where OP is =, <>, <, <=, >, or >= and
// v1 and v2 are values - from the item (an attribute path), the query
// (a ":val" reference), or a function of the the above (only the size()
// function is supported).
// 2. Ternary operator - v1 BETWEEN v2 and v3 (means v1 >= v2 AND v1 <= v3).
// 3. N-ary operator - v1 IN ( v2, v3, ... )
// 4. A single function call (attribute_exists etc.). The parser actually
// accepts a more general "value" here but later stages reject a value
// which is not a function call (because DynamoDB does it too).
class primitive_condition {
public:
enum class type {
UNDEFINED, VALUE, EQ, NE, LT, LE, GT, GE, BETWEEN, IN
};
type _op = type::UNDEFINED;
std::vector<value> _values;
void set_operator(type op) {
_op = op;
}
void add_value(value&& v) {
_values.push_back(std::move(v));
}
bool empty() const {
return _op == type::UNDEFINED;
}
};
class condition_expression {
public:
bool _negated = false; // If true, the entire condition is negated
struct condition_list {
char op = '|'; // '&' or '|'
std::vector<condition_expression> conditions;
};
std::variant<primitive_condition, condition_list> _expression = condition_list();
void set_primitive(primitive_condition&& p) {
_expression = std::move(p);
}
void append(condition_expression&& c, char op);
void apply_not() {
_negated = !_negated;
}
bool empty() const {
return std::holds_alternative<condition_list>(_expression) &&
std::get<condition_list>(_expression).conditions.empty();
}
};
} // namespace parsed
} // namespace alternator

View File

@@ -1,114 +0,0 @@
/*
* Copyright 2020-present ScyllaDB
*/
/*
* SPDX-License-Identifier: AGPL-3.0-or-later
*/
#pragma once
#include "seastarx.hh"
#include "service/paxos/cas_request.hh"
#include "utils/rjson.hh"
#include "executor.hh"
namespace alternator {
// An rmw_operation encapsulates the common logic of all the item update
// operations which may involve a read of the item before the write
// (so-called Read-Modify-Write operations). These operations include PutItem,
// UpdateItem and DeleteItem: All of these may be conditional operations (the
// "Expected" parameter) which requir a read before the write, and UpdateItem
// may also have an update expression which refers to the item's old value.
//
// The code below supports running the read and the write together as one
// transaction using LWT (this is why rmw_operation is a subclass of
// cas_request, as required by storage_proxy::cas()), but also has optional
// modes not using LWT.
class rmw_operation : public service::cas_request, public enable_shared_from_this<rmw_operation> {
public:
// The following options choose which mechanism to use for isolating
// parallel write operations:
// * The FORBID_RMW option forbids RMW (read-modify-write) operations
// such as conditional updates. For the remaining write-only
// operations, ordinary quorum writes are isolated enough.
// * The LWT_ALWAYS option always uses LWT (lightweight transactions)
// for any write operation - whether or not it also has a read.
// * The LWT_RMW_ONLY option uses LWT only for RMW operations, and uses
// ordinary quorum writes for write-only operations.
// This option is not safe if the user may send both RMW and write-only
// operations on the same item.
// * The UNSAFE_RMW option does read-modify-write operations as separate
// read and write. It is unsafe - concurrent RMW operations are not
// isolated at all. This option will likely be removed in the future.
enum class write_isolation {
FORBID_RMW, LWT_ALWAYS, LWT_RMW_ONLY, UNSAFE_RMW
};
static constexpr auto WRITE_ISOLATION_TAG_KEY = "system:write_isolation";
static write_isolation get_write_isolation_for_schema(schema_ptr schema);
static write_isolation default_write_isolation;
public:
static void set_default_write_isolation(std::string_view mode);
protected:
// The full request JSON
rjson::value _request;
// All RMW operations involve a single item with a specific partition
// and optional clustering key, in a single table, so the following
// information is common to all of them:
schema_ptr _schema;
partition_key _pk = partition_key::make_empty();
clustering_key _ck = clustering_key::make_empty();
write_isolation _write_isolation;
// All RMW operations can have a ReturnValues parameter from the following
// choices. But note that only UpdateItem actually supports all of them:
enum class returnvalues {
NONE, ALL_OLD, UPDATED_OLD, ALL_NEW, UPDATED_NEW
} _returnvalues;
static returnvalues parse_returnvalues(const rjson::value& request);
// When _returnvalues != NONE, apply() should store here, in JSON form,
// the values which are to be returned in the "Attributes" field.
// The default null JSON means do not return an Attributes field at all.
// This field is marked "mutable" so that the const apply() can modify
// it (see explanation below), but note that because apply() may be
// called more than once, if apply() will sometimes set this field it
// must set it (even if just to the default empty value) every time.
mutable rjson::value _return_attributes;
public:
// The constructor of a rmw_operation subclass should parse the request
// and try to discover as many input errors as it can before really
// attempting the read or write operations.
rmw_operation(service::storage_proxy& proxy, rjson::value&& request);
// rmw_operation subclasses (update_item_operation, put_item_operation
// and delete_item_operation) shall implement an apply() function which
// takes the previous value of the item (if it was read) and creates the
// write mutation. If the previous value of item does not pass the needed
// conditional expression, apply() should return an empty optional.
// apply() may throw if it encounters input errors not discovered during
// the constructor.
// apply() may be called more than once in case of contention, so it must
// not change the state saved in the object (issue #7218 was caused by
// violating this). We mark apply() "const" to let the compiler validate
// this for us. The output-only field _return_attributes is marked
// "mutable" above so that apply() can still write to it.
virtual std::optional<mutation> apply(std::unique_ptr<rjson::value> previous_item, api::timestamp_type ts) const = 0;
// Convert the above apply() into the signature needed by cas_request:
virtual std::optional<mutation> apply(foreign_ptr<lw_shared_ptr<query::result>> qr, const query::partition_slice& slice, api::timestamp_type ts) override;
virtual ~rmw_operation() = default;
schema_ptr schema() const { return _schema; }
const rjson::value& request() const { return _request; }
rjson::value&& move_request() && { return std::move(_request); }
future<executor::request_return_type> execute(service::storage_proxy& proxy,
service::client_state& client_state,
tracing::trace_state_ptr trace_state,
service_permit permit,
bool needs_read_before_write,
stats& stats);
std::optional<shard_id> shard_for_execute(bool needs_read_before_write);
};
} // namespace alternator

View File

@@ -1,456 +0,0 @@
/*
* Copyright 2019-present ScyllaDB
*/
/*
* SPDX-License-Identifier: AGPL-3.0-or-later
*/
#include "utils/base64.hh"
#include "utils/rjson.hh"
#include "log.hh"
#include "serialization.hh"
#include "error.hh"
#include "rapidjson/writer.h"
#include "concrete_types.hh"
#include "cql3/type_json.hh"
#include "mutation/position_in_partition.hh"
static logging::logger slogger("alternator-serialization");
namespace alternator {
bool is_alternator_keyspace(const sstring& ks_name);
type_info type_info_from_string(std::string_view type) {
static thread_local const std::unordered_map<std::string_view, type_info> type_infos = {
{"S", {alternator_type::S, utf8_type}},
{"B", {alternator_type::B, bytes_type}},
{"BOOL", {alternator_type::BOOL, boolean_type}},
{"N", {alternator_type::N, decimal_type}}, //FIXME: Replace with custom Alternator type when implemented
};
auto it = type_infos.find(type);
if (it == type_infos.end()) {
return {alternator_type::NOT_SUPPORTED_YET, utf8_type};
}
return it->second;
}
type_representation represent_type(alternator_type atype) {
static thread_local const std::unordered_map<alternator_type, type_representation> type_representations = {
{alternator_type::S, {"S", utf8_type}},
{alternator_type::B, {"B", bytes_type}},
{alternator_type::BOOL, {"BOOL", boolean_type}},
{alternator_type::N, {"N", decimal_type}}, //FIXME: Replace with custom Alternator type when implemented
};
auto it = type_representations.find(atype);
if (it == type_representations.end()) {
throw std::runtime_error(format("Unknown alternator type {}", int8_t(atype)));
}
return it->second;
}
struct from_json_visitor {
const rjson::value& v;
bytes_ostream& bo;
void operator()(const reversed_type_impl& t) const { visit(*t.underlying_type(), from_json_visitor{v, bo}); };
void operator()(const string_type_impl& t) {
bo.write(t.from_string(rjson::to_string_view(v)));
}
void operator()(const bytes_type_impl& t) const {
// FIXME: it's difficult at this point to get information if value was provided
// in request or comes from the storage, for now we assume it's user's fault.
bo.write(*unwrap_bytes(v, true));
}
void operator()(const boolean_type_impl& t) const {
bo.write(boolean_type->decompose(v.GetBool()));
}
void operator()(const decimal_type_impl& t) const {
try {
bo.write(t.from_string(rjson::to_string_view(v)));
} catch (const marshal_exception& e) {
throw api_error::validation(format("The parameter cannot be converted to a numeric value: {}", v));
}
}
// default
void operator()(const abstract_type& t) const {
bo.write(from_json_object(t, v));
}
};
bytes serialize_item(const rjson::value& item) {
if (item.IsNull() || item.MemberCount() != 1) {
throw api_error::validation(format("An item can contain only one attribute definition: {}", item));
}
auto it = item.MemberBegin();
type_info type_info = type_info_from_string(rjson::to_string_view(it->name)); // JSON keys are guaranteed to be strings
if (type_info.atype == alternator_type::NOT_SUPPORTED_YET) {
slogger.trace("Non-optimal serialization of type {}", it->name);
return bytes{int8_t(type_info.atype)} + to_bytes(rjson::print(item));
}
bytes_ostream bo;
bo.write(bytes{int8_t(type_info.atype)});
visit(*type_info.dtype, from_json_visitor{it->value, bo});
return bytes(bo.linearize());
}
struct to_json_visitor {
rjson::value& deserialized;
const std::string& type_ident;
bytes_view bv;
void operator()(const reversed_type_impl& t) const { visit(*t.underlying_type(), to_json_visitor{deserialized, type_ident, bv}); };
void operator()(const decimal_type_impl& t) const {
auto s = to_json_string(*decimal_type, bytes(bv));
//FIXME(sarna): unnecessary copy
rjson::add_with_string_name(deserialized, type_ident, rjson::from_string(s));
}
void operator()(const string_type_impl& t) {
rjson::add_with_string_name(deserialized, type_ident, rjson::from_string(reinterpret_cast<const char *>(bv.data()), bv.size()));
}
void operator()(const bytes_type_impl& t) const {
std::string b64 = base64_encode(bv);
rjson::add_with_string_name(deserialized, type_ident, rjson::from_string(b64));
}
// default
void operator()(const abstract_type& t) const {
rjson::add_with_string_name(deserialized, type_ident, rjson::parse(to_json_string(t, bytes(bv))));
}
};
rjson::value deserialize_item(bytes_view bv) {
rjson::value deserialized(rapidjson::kObjectType);
if (bv.empty()) {
throw api_error::validation("Serialized value empty");
}
alternator_type atype = alternator_type(bv[0]);
bv.remove_prefix(1);
if (atype == alternator_type::NOT_SUPPORTED_YET) {
slogger.trace("Non-optimal deserialization of alternator type {}", int8_t(atype));
return rjson::parse(std::string_view(reinterpret_cast<const char *>(bv.data()), bv.size()));
}
type_representation type_representation = represent_type(atype);
visit(*type_representation.dtype, to_json_visitor{deserialized, type_representation.ident, bv});
return deserialized;
}
std::string type_to_string(data_type type) {
static thread_local std::unordered_map<data_type, std::string> types = {
{utf8_type, "S"},
{bytes_type, "B"},
{boolean_type, "BOOL"},
{decimal_type, "N"}, // FIXME: use a specialized Alternator number type instead of the general decimal_type
};
auto it = types.find(type);
if (it == types.end()) {
// fall back to string, in order to be able to present
// internal Scylla types in a human-readable way
return "S";
}
return it->second;
}
bytes get_key_column_value(const rjson::value& item, const column_definition& column) {
std::string column_name = column.name_as_text();
const rjson::value* key_typed_value = rjson::find(item, column_name);
if (!key_typed_value) {
throw api_error::validation(format("Key column {} not found", column_name));
}
return get_key_from_typed_value(*key_typed_value, column);
}
// Parses the JSON encoding for a key value, which is a map with a single
// entry whose key is the type and the value is the encoded value.
// If this type does not match the desired "type_str", an api_error::validation
// error is thrown (the "name" parameter is the name of the column which will
// mentioned in the exception message).
// If the type does match, a reference to the encoded value is returned.
static const rjson::value& get_typed_value(const rjson::value& key_typed_value, std::string_view type_str, std::string_view name, std::string_view value_name) {
if (!key_typed_value.IsObject() || key_typed_value.MemberCount() != 1 ||
!key_typed_value.MemberBegin()->value.IsString()) {
throw api_error::validation(
format("Malformed value object for {} {}: {}",
value_name, name, key_typed_value));
}
auto it = key_typed_value.MemberBegin();
if (rjson::to_string_view(it->name) != type_str) {
throw api_error::validation(
format("Type mismatch: expected type {} for {} {}, got type {}",
type_str, value_name, name, it->name));
}
return it->value;
}
// Parses the JSON encoding for a key value, which is a map with a single
// entry, whose key is the type (expected to match the key column's type)
// and the value is the encoded value.
bytes get_key_from_typed_value(const rjson::value& key_typed_value, const column_definition& column) {
auto& value = get_typed_value(key_typed_value, type_to_string(column.type), column.name_as_text(), "key column");
std::string_view value_view = rjson::to_string_view(value);
if (value_view.empty()) {
throw api_error::validation(
format("The AttributeValue for a key attribute cannot contain an empty string value. Key: {}", column.name_as_text()));
}
if (column.type == bytes_type) {
// FIXME: it's difficult at this point to get information if value was provided
// in request or comes from the storage, for now we assume it's user's fault.
return *unwrap_bytes(value, true);
} else {
return column.type->from_string(value_view);
}
}
rjson::value json_key_column_value(bytes_view cell, const column_definition& column) {
if (column.type == bytes_type) {
std::string b64 = base64_encode(cell);
return rjson::from_string(b64);
} if (column.type == utf8_type) {
return rjson::from_string(reinterpret_cast<const char*>(cell.data()), cell.size());
} else if (column.type == decimal_type) {
// FIXME: use specialized Alternator number type, not the more
// general "decimal_type". A dedicated type can be more efficient
// in storage space and in parsing speed.
auto s = to_json_string(*decimal_type, bytes(cell));
return rjson::from_string(s);
} else {
// Support for arbitrary key types is useful for parsing values of virtual tables,
// which can involve any type supported by Scylla.
// In order to guarantee that the returned type is parsable by alternator clients,
// they are represented simply as strings.
return rjson::from_string(column.type->to_string(bytes(cell)));
}
}
partition_key pk_from_json(const rjson::value& item, schema_ptr schema) {
std::vector<bytes> raw_pk;
// FIXME: this is a loop, but we really allow only one partition key column.
for (const column_definition& cdef : schema->partition_key_columns()) {
bytes raw_value = get_key_column_value(item, cdef);
raw_pk.push_back(std::move(raw_value));
}
return partition_key::from_exploded(raw_pk);
}
clustering_key ck_from_json(const rjson::value& item, schema_ptr schema) {
if (schema->clustering_key_size() == 0) {
return clustering_key::make_empty();
}
std::vector<bytes> raw_ck;
// FIXME: this is a loop, but we really allow only one clustering key column.
for (const column_definition& cdef : schema->clustering_key_columns()) {
bytes raw_value = get_key_column_value(item, cdef);
raw_ck.push_back(std::move(raw_value));
}
return clustering_key::from_exploded(raw_ck);
}
position_in_partition pos_from_json(const rjson::value& item, schema_ptr schema) {
auto ck = ck_from_json(item, schema);
if (is_alternator_keyspace(schema->ks_name())) {
return position_in_partition::for_key(std::move(ck));
}
const auto region_item = rjson::find(item, scylla_paging_region);
const auto weight_item = rjson::find(item, scylla_paging_weight);
if (bool(region_item) != bool(weight_item)) {
throw api_error::validation("Malformed value object: region and weight has to be either both missing or both present");
}
bound_weight weight;
if (region_item) {
auto region_view = rjson::to_string_view(get_typed_value(*region_item, "S", scylla_paging_region, "key region"));
auto weight_view = rjson::to_string_view(get_typed_value(*weight_item, "N", scylla_paging_weight, "key weight"));
auto region = parse_partition_region(region_view);
if (weight_view == "-1") {
weight = bound_weight::before_all_prefixed;
} else if (weight_view == "0") {
weight = bound_weight::equal;
} else if (weight_view == "1") {
weight = bound_weight::after_all_prefixed;
} else {
throw std::runtime_error(fmt::format("Invalid value for weight: {}", weight_view));
}
return position_in_partition(region, weight, region == partition_region::clustered ? std::optional(std::move(ck)) : std::nullopt);
}
if (ck.is_empty()) {
return position_in_partition::for_partition_start();
}
return position_in_partition::for_key(std::move(ck));
}
big_decimal unwrap_number(const rjson::value& v, std::string_view diagnostic) {
if (!v.IsObject() || v.MemberCount() != 1) {
throw api_error::validation(format("{}: invalid number object", diagnostic));
}
auto it = v.MemberBegin();
if (it->name != "N") {
throw api_error::validation(format("{}: expected number, found type '{}'", diagnostic, it->name));
}
try {
if (!it->value.IsString()) {
// We shouldn't reach here. Callers normally validate their input
// earlier with validate_value().
throw api_error::validation(format("{}: improperly formatted number constant", diagnostic));
}
return big_decimal(rjson::to_string_view(it->value));
} catch (const marshal_exception& e) {
throw api_error::validation(format("The parameter cannot be converted to a numeric value: {}", it->value));
}
}
std::optional<big_decimal> try_unwrap_number(const rjson::value& v) {
if (!v.IsObject() || v.MemberCount() != 1) {
return std::nullopt;
}
auto it = v.MemberBegin();
if (it->name != "N" || !it->value.IsString()) {
return std::nullopt;
}
try {
return big_decimal(rjson::to_string_view(it->value));
} catch (const marshal_exception& e) {
return std::nullopt;
}
}
std::optional<bytes> unwrap_bytes(const rjson::value& value, bool from_query) {
try {
return rjson::base64_decode(value);
} catch (...) {
if (from_query) {
throw api_error::serialization(format("Invalid base64 data"));
}
return std::nullopt;
}
}
const std::pair<std::string, const rjson::value*> unwrap_set(const rjson::value& v) {
if (!v.IsObject() || v.MemberCount() != 1) {
return {"", nullptr};
}
auto it = v.MemberBegin();
const std::string it_key = it->name.GetString();
if (it_key != "SS" && it_key != "BS" && it_key != "NS") {
return {std::move(it_key), nullptr};
}
return std::make_pair(it_key, &(it->value));
}
const rjson::value* unwrap_list(const rjson::value& v) {
if (!v.IsObject() || v.MemberCount() != 1) {
return nullptr;
}
auto it = v.MemberBegin();
if (it->name != std::string("L")) {
return nullptr;
}
return &(it->value);
}
// Take two JSON-encoded numeric values ({"N": "thenumber"}) and return the
// sum, again as a JSON-encoded number.
rjson::value number_add(const rjson::value& v1, const rjson::value& v2) {
auto n1 = unwrap_number(v1, "UpdateExpression");
auto n2 = unwrap_number(v2, "UpdateExpression");
rjson::value ret = rjson::empty_object();
sstring str_ret = (n1 + n2).to_string();
rjson::add(ret, "N", rjson::from_string(str_ret));
return ret;
}
rjson::value number_subtract(const rjson::value& v1, const rjson::value& v2) {
auto n1 = unwrap_number(v1, "UpdateExpression");
auto n2 = unwrap_number(v2, "UpdateExpression");
rjson::value ret = rjson::empty_object();
sstring str_ret = (n1 - n2).to_string();
rjson::add(ret, "N", rjson::from_string(str_ret));
return ret;
}
// Take two JSON-encoded set values (e.g. {"SS": [...the actual set]}) and
// return the sum of both sets, again as a set value.
rjson::value set_sum(const rjson::value& v1, const rjson::value& v2) {
auto [set1_type, set1] = unwrap_set(v1);
auto [set2_type, set2] = unwrap_set(v2);
if (set1_type != set2_type) {
throw api_error::validation(format("Mismatched set types: {} and {}", set1_type, set2_type));
}
if (!set1 || !set2) {
throw api_error::validation("UpdateExpression: ADD operation for sets must be given sets as arguments");
}
rjson::value sum = rjson::copy(*set1);
std::set<rjson::value, rjson::single_value_comp> set1_raw;
for (auto it = sum.Begin(); it != sum.End(); ++it) {
set1_raw.insert(rjson::copy(*it));
}
for (const auto& a : set2->GetArray()) {
if (!set1_raw.contains(a)) {
rjson::push_back(sum, rjson::copy(a));
}
}
rjson::value ret = rjson::empty_object();
rjson::add_with_string_name(ret, set1_type, std::move(sum));
return ret;
}
// Take two JSON-encoded set values (e.g. {"SS": [...the actual list]}) and
// return the difference of s1 - s2, again as a set value.
// DynamoDB does not allow empty sets, so if resulting set is empty, return
// an unset optional instead.
std::optional<rjson::value> set_diff(const rjson::value& v1, const rjson::value& v2) {
auto [set1_type, set1] = unwrap_set(v1);
auto [set2_type, set2] = unwrap_set(v2);
if (set1_type != set2_type) {
throw api_error::validation(format("Set DELETE type mismatch: {} and {}", set1_type, set2_type));
}
if (!set1 || !set2) {
throw api_error::validation("UpdateExpression: DELETE operation can only be performed on a set");
}
std::set<rjson::value, rjson::single_value_comp> set1_raw;
for (auto it = set1->Begin(); it != set1->End(); ++it) {
set1_raw.insert(rjson::copy(*it));
}
for (const auto& a : set2->GetArray()) {
set1_raw.erase(a);
}
if (set1_raw.empty()) {
return std::nullopt;
}
rjson::value ret = rjson::empty_object();
rjson::add_with_string_name(ret, set1_type, rjson::empty_array());
rjson::value& result_set = ret[set1_type];
for (const auto& a : set1_raw) {
rjson::push_back(result_set, rjson::copy(a));
}
return ret;
}
// Take two JSON-encoded list values (remember that a list value is
// {"L": [...the actual list]}) and return the concatenation, again as
// a list value.
// Returns a null value if one of the arguments is not actually a list.
rjson::value list_concatenate(const rjson::value& v1, const rjson::value& v2) {
const rjson::value* list1 = unwrap_list(v1);
const rjson::value* list2 = unwrap_list(v2);
if (!list1 || !list2) {
return rjson::null_value();
}
rjson::value cat = rjson::copy(*list1);
for (const auto& a : list2->GetArray()) {
rjson::push_back(cat, rjson::copy(a));
}
rjson::value ret = rjson::empty_object();
rjson::add(ret, "L", std::move(cat));
return ret;
}
}

View File

@@ -1,98 +0,0 @@
/*
* Copyright 2019-present ScyllaDB
*/
/*
* SPDX-License-Identifier: AGPL-3.0-or-later
*/
#pragma once
#include <string>
#include <string_view>
#include <optional>
#include "types/types.hh"
#include "schema/schema_fwd.hh"
#include "keys.hh"
#include "utils/rjson.hh"
#include "utils/big_decimal.hh"
class position_in_partition;
namespace alternator {
enum class alternator_type : int8_t {
S, B, BOOL, N, NOT_SUPPORTED_YET
};
struct type_info {
alternator_type atype;
data_type dtype;
};
struct type_representation {
std::string ident;
data_type dtype;
};
inline constexpr std::string_view scylla_paging_region(":scylla:paging:region");
inline constexpr std::string_view scylla_paging_weight(":scylla:paging:weight");
type_info type_info_from_string(std::string_view type);
type_representation represent_type(alternator_type atype);
bytes serialize_item(const rjson::value& item);
rjson::value deserialize_item(bytes_view bv);
std::string type_to_string(data_type type);
bytes get_key_column_value(const rjson::value& item, const column_definition& column);
bytes get_key_from_typed_value(const rjson::value& key_typed_value, const column_definition& column);
rjson::value json_key_column_value(bytes_view cell, const column_definition& column);
partition_key pk_from_json(const rjson::value& item, schema_ptr schema);
clustering_key ck_from_json(const rjson::value& item, schema_ptr schema);
position_in_partition pos_from_json(const rjson::value& item, schema_ptr schema);
// If v encodes a number (i.e., it is a {"N": [...]}, returns an object representing it. Otherwise,
// raises ValidationException with diagnostic.
big_decimal unwrap_number(const rjson::value& v, std::string_view diagnostic);
// try_unwrap_number is like unwrap_number, but returns an unset optional
// when the given v does not encode a number.
std::optional<big_decimal> try_unwrap_number(const rjson::value& v);
// unwrap_bytes decodes byte value, on decoding failure it either raises api_error::serialization
// iff from_query is true or returns unset optional iff from_query is false.
// Therefore it's safe to dereference returned optional when called with from_query equal true.
std::optional<bytes> unwrap_bytes(const rjson::value& value, bool from_query);
// Check if a given JSON object encodes a set (i.e., it is a {"SS": [...]}, or "NS", "BS"
// and returns set's type and a pointer to that set. If the object does not encode a set,
// returned value is {"", nullptr}
const std::pair<std::string, const rjson::value*> unwrap_set(const rjson::value& v);
// Check if a given JSON object encodes a list (i.e., it is a {"L": [...]}
// and returns a pointer to that list.
const rjson::value* unwrap_list(const rjson::value& v);
// Take two JSON-encoded numeric values ({"N": "thenumber"}) and return the
// sum, again as a JSON-encoded number.
rjson::value number_add(const rjson::value& v1, const rjson::value& v2);
rjson::value number_subtract(const rjson::value& v1, const rjson::value& v2);
// Take two JSON-encoded set values (e.g. {"SS": [...the actual set]}) and
// return the sum of both sets, again as a set value.
rjson::value set_sum(const rjson::value& v1, const rjson::value& v2);
// Take two JSON-encoded set values (e.g. {"SS": [...the actual list]}) and
// return the difference of s1 - s2, again as a set value.
// DynamoDB does not allow empty sets, so if resulting set is empty, return
// an unset optional instead.
std::optional<rjson::value> set_diff(const rjson::value& v1, const rjson::value& v2);
// Take two JSON-encoded list values (remember that a list value is
// {"L": [...the actual list]}) and return the concatenation, again as
// a list value.
// Returns a null value if one of the arguments is not actually a list.
rjson::value list_concatenate(const rjson::value& v1, const rjson::value& v2);
}

View File

@@ -1,640 +0,0 @@
/*
* Copyright 2019-present ScyllaDB
*/
/*
* SPDX-License-Identifier: AGPL-3.0-or-later
*/
#include "alternator/server.hh"
#include "log.hh"
#include <seastar/http/function_handlers.hh>
#include <seastar/http/short_streams.hh>
#include <seastar/core/coroutine.hh>
#include <seastar/json/json_elements.hh>
#include <seastar/util/defer.hh>
#include <seastar/util/short_streams.hh>
#include "seastarx.hh"
#include "error.hh"
#include "service/qos/service_level_controller.hh"
#include "utils/rjson.hh"
#include "auth.hh"
#include <cctype>
#include "service/storage_proxy.hh"
#include "gms/gossiper.hh"
#include "utils/overloaded_functor.hh"
#include "utils/fb_utilities.hh"
#include "utils/aws_sigv4.hh"
static logging::logger slogger("alternator-server");
using namespace httpd;
using request = http::request;
using reply = http::reply;
namespace alternator {
static constexpr auto TARGET = "X-Amz-Target";
inline std::vector<std::string_view> split(std::string_view text, char separator) {
std::vector<std::string_view> tokens;
if (text == "") {
return tokens;
}
while (true) {
auto pos = text.find_first_of(separator);
if (pos != std::string_view::npos) {
tokens.emplace_back(text.data(), pos);
text.remove_prefix(pos + 1);
} else {
tokens.emplace_back(text);
break;
}
}
return tokens;
}
// Handle CORS (Cross-origin resource sharing) in the HTTP request:
// If the request has the "Origin" header specifying where the script which
// makes this request comes from, we need to reply with the header
// "Access-Control-Allow-Origin: *" saying that this (and any) origin is fine.
// Additionally, if preflight==true (i.e., this is an OPTIONS request),
// the script can also "request" in headers that the server allows it to use
// some HTTP methods and headers in the followup request, and the server
// should respond by "allowing" them in the response headers.
// We also add the header "Access-Control-Expose-Headers" to let the script
// access additional headers in the response.
// This handle_CORS() should be used when handling any HTTP method - both the
// usual GET and POST, and also the "preflight" OPTIONS method.
static void handle_CORS(const request& req, reply& rep, bool preflight) {
if (!req.get_header("origin").empty()) {
rep.add_header("Access-Control-Allow-Origin", "*");
// This is the list that DynamoDB returns for expose headers. I am
// not sure why not just return "*" here, what's the risk?
rep.add_header("Access-Control-Expose-Headers", "x-amzn-RequestId,x-amzn-ErrorType,x-amzn-ErrorMessage,Date");
if (preflight) {
sstring s = req.get_header("Access-Control-Request-Headers");
if (!s.empty()) {
rep.add_header("Access-Control-Allow-Headers", std::move(s));
}
s = req.get_header("Access-Control-Request-Method");
if (!s.empty()) {
rep.add_header("Access-Control-Allow-Methods", std::move(s));
}
// Our CORS response never change anyway, let the browser cache it
// for two hours (Chrome's maximum):
rep.add_header("Access-Control-Max-Age", "7200");
}
}
}
// DynamoDB HTTP error responses are structured as follows
// https://docs.aws.amazon.com/amazondynamodb/latest/developerguide/Programming.Errors.html
// Our handlers throw an exception to report an error. If the exception
// is of type alternator::api_error, it unwrapped and properly reported to
// the user directly. Other exceptions are unexpected, and reported as
// Internal Server Error.
class api_handler : public handler_base {
public:
api_handler(const std::function<future<executor::request_return_type>(std::unique_ptr<request> req)>& _handle) : _f_handle(
[this, _handle](std::unique_ptr<request> req, std::unique_ptr<reply> rep) {
return seastar::futurize_invoke(_handle, std::move(req)).then_wrapped([this, rep = std::move(rep)](future<executor::request_return_type> resf) mutable {
if (resf.failed()) {
// Exceptions of type api_error are wrapped as JSON and
// returned to the client as expected. Other types of
// exceptions are unexpected, and returned to the user
// as an internal server error:
try {
resf.get();
} catch (api_error &ae) {
generate_error_reply(*rep, ae);
} catch (rjson::error & re) {
generate_error_reply(*rep,
api_error::validation(re.what()));
} catch (...) {
generate_error_reply(*rep,
api_error::internal(format("Internal server error: {}", std::current_exception())));
}
return make_ready_future<std::unique_ptr<reply>>(std::move(rep));
}
auto res = resf.get0();
std::visit(overloaded_functor {
[&] (const json::json_return_type& json_return_value) {
slogger.trace("api_handler success case");
if (json_return_value._body_writer) {
// Unfortunately, write_body() forces us to choose
// from a fixed and irrelevant list of "mime-types"
// at this point. But we'll override it with the
// one (application/x-amz-json-1.0) below.
rep->write_body("json", std::move(json_return_value._body_writer));
} else {
rep->_content += json_return_value._res;
}
},
[&] (const api_error& err) {
generate_error_reply(*rep, err);
}
}, res);
return make_ready_future<std::unique_ptr<reply>>(std::move(rep));
});
}) { }
api_handler(const api_handler&) = default;
future<std::unique_ptr<reply>> handle(const sstring& path,
std::unique_ptr<request> req, std::unique_ptr<reply> rep) override {
handle_CORS(*req, *rep, false);
return _f_handle(std::move(req), std::move(rep)).then(
[](std::unique_ptr<reply> rep) {
rep->set_mime_type("application/x-amz-json-1.0");
rep->done();
return make_ready_future<std::unique_ptr<reply>>(std::move(rep));
});
}
protected:
void generate_error_reply(reply& rep, const api_error& err) {
rjson::value results = rjson::empty_object();
rjson::add(results, "__type", rjson::from_string("com.amazonaws.dynamodb.v20120810#" + err._type));
rjson::add(results, "message", err._msg);
rep._content = rjson::print(std::move(results));
rep._status = err._http_code;
slogger.trace("api_handler error case: {}", rep._content);
}
future_handler_function _f_handle;
};
class gated_handler : public handler_base {
seastar::gate& _gate;
public:
gated_handler(seastar::gate& gate) : _gate(gate) {}
virtual future<std::unique_ptr<reply>> do_handle(const sstring& path, std::unique_ptr<request> req, std::unique_ptr<reply> rep) = 0;
virtual future<std::unique_ptr<reply>> handle(const sstring& path, std::unique_ptr<request> req, std::unique_ptr<reply> rep) final override {
return with_gate(_gate, [this, &path, req = std::move(req), rep = std::move(rep)] () mutable {
return do_handle(path, std::move(req), std::move(rep));
});
}
};
class health_handler : public gated_handler {
public:
health_handler(seastar::gate& pending_requests) : gated_handler(pending_requests) {}
protected:
virtual future<std::unique_ptr<reply>> do_handle(const sstring& path, std::unique_ptr<request> req, std::unique_ptr<reply> rep) override {
handle_CORS(*req, *rep, false);
rep->set_status(reply::status_type::ok);
rep->write_body("txt", format("healthy: {}", req->get_header("Host")));
return make_ready_future<std::unique_ptr<reply>>(std::move(rep));
}
};
class local_nodelist_handler : public gated_handler {
service::storage_proxy& _proxy;
gms::gossiper& _gossiper;
public:
local_nodelist_handler(seastar::gate& pending_requests, service::storage_proxy& proxy, gms::gossiper& gossiper)
: gated_handler(pending_requests)
, _proxy(proxy)
, _gossiper(gossiper) {}
protected:
virtual future<std::unique_ptr<reply>> do_handle(const sstring& path, std::unique_ptr<request> req, std::unique_ptr<reply> rep) override {
rjson::value results = rjson::empty_array();
// It's very easy to get a list of all live nodes on the cluster,
// using _gossiper().get_live_members(). But getting
// just the list of live nodes in this DC needs more elaborate code:
auto& topology = _proxy.get_token_metadata_ptr()->get_topology();
sstring local_dc = topology.get_datacenter();
std::unordered_set<gms::inet_address> local_dc_nodes = topology.get_datacenter_endpoints().at(local_dc);
for (auto& ip : local_dc_nodes) {
if (_gossiper.is_alive(ip)) {
rjson::push_back(results, rjson::from_string(ip.to_sstring()));
}
}
rep->set_status(reply::status_type::ok);
rep->set_content_type("json");
rep->_content = rjson::print(results);
return make_ready_future<std::unique_ptr<reply>>(std::move(rep));
}
};
// The CORS (Cross-origin resource sharing) protocol can send an OPTIONS
// request before ("pre-flight") the main request. The response to this
// request can be empty, but needs to have the right headers (which we
// fill with handle_CORS())
class options_handler : public gated_handler {
public:
options_handler(seastar::gate& pending_requests) : gated_handler(pending_requests) {}
protected:
virtual future<std::unique_ptr<reply>> do_handle(const sstring& path, std::unique_ptr<request> req, std::unique_ptr<reply> rep) override {
handle_CORS(*req, *rep, true);
rep->set_status(reply::status_type::ok);
rep->write_body("txt", sstring(""));
return make_ready_future<std::unique_ptr<reply>>(std::move(rep));
}
};
future<std::string> server::verify_signature(const request& req, const chunked_content& content) {
if (!_enforce_authorization) {
slogger.debug("Skipping authorization");
return make_ready_future<std::string>();
}
auto host_it = req._headers.find("Host");
if (host_it == req._headers.end()) {
throw api_error::invalid_signature("Host header is mandatory for signature verification");
}
auto authorization_it = req._headers.find("Authorization");
if (authorization_it == req._headers.end()) {
throw api_error::missing_authentication_token("Authorization header is mandatory for signature verification");
}
std::string host = host_it->second;
std::string_view authorization_header = authorization_it->second;
auto pos = authorization_header.find_first_of(' ');
if (pos == std::string_view::npos || authorization_header.substr(0, pos) != "AWS4-HMAC-SHA256") {
throw api_error::invalid_signature(format("Authorization header must use AWS4-HMAC-SHA256 algorithm: {}", authorization_header));
}
authorization_header.remove_prefix(pos+1);
std::string credential;
std::string user_signature;
std::string signed_headers_str;
std::vector<std::string_view> signed_headers;
do {
// Either one of a comma or space can mark the end of an entry
pos = authorization_header.find_first_of(" ,");
std::string_view entry = authorization_header.substr(0, pos);
if (pos != std::string_view::npos) {
authorization_header.remove_prefix(pos + 1);
}
if (entry.empty()) {
continue;
}
std::vector<std::string_view> entry_split = split(entry, '=');
if (entry_split.size() != 2) {
continue;
}
std::string_view auth_value = entry_split[1];
if (entry_split[0] == "Credential") {
credential = std::string(auth_value);
} else if (entry_split[0] == "Signature") {
user_signature = std::string(auth_value);
} else if (entry_split[0] == "SignedHeaders") {
signed_headers_str = std::string(auth_value);
signed_headers = split(auth_value, ';');
std::sort(signed_headers.begin(), signed_headers.end());
}
} while (pos != std::string_view::npos);
std::vector<std::string_view> credential_split = split(credential, '/');
if (credential_split.size() != 5) {
throw api_error::validation(format("Incorrect credential information format: {}", credential));
}
std::string user(credential_split[0]);
std::string datestamp(credential_split[1]);
std::string region(credential_split[2]);
std::string service(credential_split[3]);
std::map<std::string_view, std::string_view> signed_headers_map;
for (const auto& header : signed_headers) {
signed_headers_map.emplace(header, std::string_view());
}
for (auto& header : req._headers) {
std::string header_str;
header_str.resize(header.first.size());
std::transform(header.first.begin(), header.first.end(), header_str.begin(), ::tolower);
auto it = signed_headers_map.find(header_str);
if (it != signed_headers_map.end()) {
it->second = std::string_view(header.second);
}
}
auto cache_getter = [&proxy = _proxy] (std::string username) {
return get_key_from_roles(proxy, std::move(username));
};
return _key_cache.get_ptr(user, cache_getter).then([this, &req, &content,
user = std::move(user),
host = std::move(host),
datestamp = std::move(datestamp),
signed_headers_str = std::move(signed_headers_str),
signed_headers_map = std::move(signed_headers_map),
region = std::move(region),
service = std::move(service),
user_signature = std::move(user_signature)] (key_cache::value_ptr key_ptr) {
std::string signature;
try {
signature = utils::aws::get_signature(user, *key_ptr, std::string_view(host), "/", req._method,
datestamp, signed_headers_str, signed_headers_map, &content, region, service, "");
} catch (const std::exception& e) {
throw api_error::invalid_signature(e.what());
}
if (signature != std::string_view(user_signature)) {
_key_cache.remove(user);
throw api_error::unrecognized_client("The security token included in the request is invalid.");
}
return user;
});
}
static tracing::trace_state_ptr create_tracing_session(tracing::tracing& tracing_instance) {
tracing::trace_state_props_set props;
props.set<tracing::trace_state_props::full_tracing>();
props.set_if<tracing::trace_state_props::log_slow_query>(tracing_instance.slow_query_tracing_enabled());
return tracing_instance.create_session(tracing::trace_type::QUERY, props);
}
// truncated_content_view() prints a potentially long chunked_content for
// debugging purposes. In the common case when the content is not excessively
// long, it just returns a view into the given content, without any copying.
// But when the content is very long, it is truncated after some arbitrary
// max_len (or one chunk, whichever comes first), with "<truncated>" added at
// the end. To do this modification to the string, we need to create a new
// std::string, so the caller must pass us a reference to one, "buf", where
// we can store the content. The returned view is only alive for as long this
// buf is kept alive.
static std::string_view truncated_content_view(const chunked_content& content, std::string& buf) {
constexpr size_t max_len = 1024;
if (content.empty()) {
return std::string_view();
} else if (content.size() == 1 && content.begin()->size() <= max_len) {
return std::string_view(content.begin()->get(), content.begin()->size());
} else {
buf = std::string(content.begin()->get(), std::min(content.begin()->size(), max_len)) + "<truncated>";
return std::string_view(buf);
}
}
static tracing::trace_state_ptr maybe_trace_query(service::client_state& client_state, std::string_view username, sstring_view op, const chunked_content& query) {
tracing::trace_state_ptr trace_state;
tracing::tracing& tracing_instance = tracing::tracing::get_local_tracing_instance();
if (tracing_instance.trace_next_query() || tracing_instance.slow_query_tracing_enabled()) {
trace_state = create_tracing_session(tracing_instance);
std::string buf;
tracing::add_session_param(trace_state, "alternator_op", op);
tracing::add_query(trace_state, truncated_content_view(query, buf));
tracing::begin(trace_state, format("Alternator {}", op), client_state.get_client_address());
if (!username.empty()) {
tracing::set_username(trace_state, auth::authenticated_user(username));
}
}
return trace_state;
}
future<executor::request_return_type> server::handle_api_request(std::unique_ptr<request> req) {
_executor._stats.total_operations++;
sstring target = req->get_header(TARGET);
std::vector<std::string_view> split_target = split(target, '.');
//NOTICE(sarna): Target consists of Dynamo API version followed by a dot '.' and operation type (e.g. CreateTable)
std::string op = split_target.empty() ? std::string() : std::string(split_target.back());
// JSON parsing can allocate up to roughly 2x the size of the raw
// document, + a couple of bytes for maintenance.
// TODO: consider the case where req->content_length is missing. Maybe
// we need to take the content_length_limit and return some of the units
// when we finish read_content_and_verify_signature?
size_t mem_estimate = req->content_length * 2 + 8000;
auto units_fut = get_units(*_memory_limiter, mem_estimate);
if (_memory_limiter->waiters()) {
++_executor._stats.requests_blocked_memory;
}
auto units = co_await std::move(units_fut);
assert(req->content_stream);
chunked_content content = co_await util::read_entire_stream(*req->content_stream);
auto username = co_await verify_signature(*req, content);
if (slogger.is_enabled(log_level::trace)) {
std::string buf;
slogger.trace("Request: {} {} {}", op, truncated_content_view(content, buf), req->_headers);
}
auto callback_it = _callbacks.find(op);
if (callback_it == _callbacks.end()) {
_executor._stats.unsupported_operations++;
co_return api_error::unknown_operation(format("Unsupported operation {}", op));
}
if (_pending_requests.get_count() >= _max_concurrent_requests) {
_executor._stats.requests_shed++;
co_return api_error::request_limit_exceeded(format("too many in-flight requests (configured via max_concurrent_requests_per_shard): {}", _pending_requests.get_count()));
}
_pending_requests.enter();
auto leave = defer([this] () noexcept { _pending_requests.leave(); });
//FIXME: Client state can provide more context, e.g. client's endpoint address
// We use unique_ptr because client_state cannot be moved or copied
executor::client_state client_state = username.empty()
? service::client_state{service::client_state::internal_tag()}
: service::client_state{service::client_state::internal_tag(), _auth_service, _sl_controller, username};
co_await client_state.maybe_update_per_service_level_params();
tracing::trace_state_ptr trace_state = maybe_trace_query(client_state, username, op, content);
tracing::trace(trace_state, op);
rjson::value json_request = co_await _json_parser.parse(std::move(content));
co_return co_await callback_it->second(_executor, client_state, trace_state,
make_service_permit(std::move(units)), std::move(json_request), std::move(req));
}
void server::set_routes(routes& r) {
api_handler* req_handler = new api_handler([this] (std::unique_ptr<request> req) mutable {
return handle_api_request(std::move(req));
});
r.put(operation_type::POST, "/", req_handler);
r.put(operation_type::GET, "/", new health_handler(_pending_requests));
// The "/localnodes" request is a new Alternator feature, not supported by
// DynamoDB and not required for DynamoDB compatibility. It allows a
// client to enquire - using a trivial HTTP request without requiring
// authentication - the list of all live nodes in the same data center of
// the Alternator cluster. The client can use this list to balance its
// request load to all the nodes in the same geographical region.
// Note that this API exposes - openly without authentication - the
// information on the cluster's members inside one data center. We do not
// consider this to be a security risk, because an attacker can already
// scan an entire subnet for nodes responding to the health request,
// or even just scan for open ports.
r.put(operation_type::GET, "/localnodes", new local_nodelist_handler(_pending_requests, _proxy, _gossiper));
r.put(operation_type::OPTIONS, "/", new options_handler(_pending_requests));
}
//FIXME: A way to immediately invalidate the cache should be considered,
// e.g. when the system table which stores the keys is changed.
// For now, this propagation may take up to 1 minute.
server::server(executor& exec, service::storage_proxy& proxy, gms::gossiper& gossiper, auth::service& auth_service, qos::service_level_controller& sl_controller)
: _http_server("http-alternator")
, _https_server("https-alternator")
, _executor(exec)
, _proxy(proxy)
, _gossiper(gossiper)
, _auth_service(auth_service)
, _sl_controller(sl_controller)
, _key_cache(1024, 1min, slogger)
, _enforce_authorization(false)
, _enabled_servers{}
, _pending_requests{}
, _callbacks{
{"CreateTable", [] (executor& e, executor::client_state& client_state, tracing::trace_state_ptr trace_state, service_permit permit, rjson::value json_request, std::unique_ptr<request> req) {
return e.create_table(client_state, std::move(trace_state), std::move(permit), std::move(json_request));
}},
{"DescribeTable", [] (executor& e, executor::client_state& client_state, tracing::trace_state_ptr trace_state, service_permit permit, rjson::value json_request, std::unique_ptr<request> req) {
return e.describe_table(client_state, std::move(trace_state), std::move(permit), std::move(json_request));
}},
{"DeleteTable", [] (executor& e, executor::client_state& client_state, tracing::trace_state_ptr trace_state, service_permit permit, rjson::value json_request, std::unique_ptr<request> req) {
return e.delete_table(client_state, std::move(trace_state), std::move(permit), std::move(json_request));
}},
{"UpdateTable", [] (executor& e, executor::client_state& client_state, tracing::trace_state_ptr trace_state, service_permit permit, rjson::value json_request, std::unique_ptr<request> req) {
return e.update_table(client_state, std::move(trace_state), std::move(permit), std::move(json_request));
}},
{"PutItem", [] (executor& e, executor::client_state& client_state, tracing::trace_state_ptr trace_state, service_permit permit, rjson::value json_request, std::unique_ptr<request> req) {
return e.put_item(client_state, std::move(trace_state), std::move(permit), std::move(json_request));
}},
{"UpdateItem", [] (executor& e, executor::client_state& client_state, tracing::trace_state_ptr trace_state, service_permit permit, rjson::value json_request, std::unique_ptr<request> req) {
return e.update_item(client_state, std::move(trace_state), std::move(permit), std::move(json_request));
}},
{"GetItem", [] (executor& e, executor::client_state& client_state, tracing::trace_state_ptr trace_state, service_permit permit, rjson::value json_request, std::unique_ptr<request> req) {
return e.get_item(client_state, std::move(trace_state), std::move(permit), std::move(json_request));
}},
{"DeleteItem", [] (executor& e, executor::client_state& client_state, tracing::trace_state_ptr trace_state, service_permit permit, rjson::value json_request, std::unique_ptr<request> req) {
return e.delete_item(client_state, std::move(trace_state), std::move(permit), std::move(json_request));
}},
{"ListTables", [] (executor& e, executor::client_state& client_state, tracing::trace_state_ptr trace_state, service_permit permit, rjson::value json_request, std::unique_ptr<request> req) {
return e.list_tables(client_state, std::move(permit), std::move(json_request));
}},
{"Scan", [] (executor& e, executor::client_state& client_state, tracing::trace_state_ptr trace_state, service_permit permit, rjson::value json_request, std::unique_ptr<request> req) {
return e.scan(client_state, std::move(trace_state), std::move(permit), std::move(json_request));
}},
{"DescribeEndpoints", [] (executor& e, executor::client_state& client_state, tracing::trace_state_ptr trace_state, service_permit permit, rjson::value json_request, std::unique_ptr<request> req) {
return e.describe_endpoints(client_state, std::move(permit), std::move(json_request), req->get_header("Host"));
}},
{"BatchWriteItem", [] (executor& e, executor::client_state& client_state, tracing::trace_state_ptr trace_state, service_permit permit, rjson::value json_request, std::unique_ptr<request> req) {
return e.batch_write_item(client_state, std::move(trace_state), std::move(permit), std::move(json_request));
}},
{"BatchGetItem", [] (executor& e, executor::client_state& client_state, tracing::trace_state_ptr trace_state, service_permit permit, rjson::value json_request, std::unique_ptr<request> req) {
return e.batch_get_item(client_state, std::move(trace_state), std::move(permit), std::move(json_request));
}},
{"Query", [] (executor& e, executor::client_state& client_state, tracing::trace_state_ptr trace_state, service_permit permit, rjson::value json_request, std::unique_ptr<request> req) {
return e.query(client_state, std::move(trace_state), std::move(permit), std::move(json_request));
}},
{"TagResource", [] (executor& e, executor::client_state& client_state, tracing::trace_state_ptr trace_state, service_permit permit, rjson::value json_request, std::unique_ptr<request> req) {
return e.tag_resource(client_state, std::move(permit), std::move(json_request));
}},
{"UntagResource", [] (executor& e, executor::client_state& client_state, tracing::trace_state_ptr trace_state, service_permit permit, rjson::value json_request, std::unique_ptr<request> req) {
return e.untag_resource(client_state, std::move(permit), std::move(json_request));
}},
{"ListTagsOfResource", [] (executor& e, executor::client_state& client_state, tracing::trace_state_ptr trace_state, service_permit permit, rjson::value json_request, std::unique_ptr<request> req) {
return e.list_tags_of_resource(client_state, std::move(permit), std::move(json_request));
}},
{"UpdateTimeToLive", [] (executor& e, executor::client_state& client_state, tracing::trace_state_ptr trace_state, service_permit permit, rjson::value json_request, std::unique_ptr<request> req) {
return e.update_time_to_live(client_state, std::move(permit), std::move(json_request));
}},
{"DescribeTimeToLive", [] (executor& e, executor::client_state& client_state, tracing::trace_state_ptr trace_state, service_permit permit, rjson::value json_request, std::unique_ptr<request> req) {
return e.describe_time_to_live(client_state, std::move(permit), std::move(json_request));
}},
{"ListStreams", [] (executor& e, executor::client_state& client_state, tracing::trace_state_ptr trace_state, service_permit permit, rjson::value json_request, std::unique_ptr<request> req) {
return e.list_streams(client_state, std::move(permit), std::move(json_request));
}},
{"DescribeStream", [] (executor& e, executor::client_state& client_state, tracing::trace_state_ptr trace_state, service_permit permit, rjson::value json_request, std::unique_ptr<request> req) {
return e.describe_stream(client_state, std::move(permit), std::move(json_request));
}},
{"GetShardIterator", [] (executor& e, executor::client_state& client_state, tracing::trace_state_ptr trace_state, service_permit permit, rjson::value json_request, std::unique_ptr<request> req) {
return e.get_shard_iterator(client_state, std::move(permit), std::move(json_request));
}},
{"GetRecords", [] (executor& e, executor::client_state& client_state, tracing::trace_state_ptr trace_state, service_permit permit, rjson::value json_request, std::unique_ptr<request> req) {
return e.get_records(client_state, std::move(trace_state), std::move(permit), std::move(json_request));
}},
{"DescribeContinuousBackups", [] (executor& e, executor::client_state& client_state, tracing::trace_state_ptr trace_state, service_permit permit, rjson::value json_request, std::unique_ptr<request> req) {
return e.describe_continuous_backups(client_state, std::move(permit), std::move(json_request));
}},
} {
}
future<> server::init(net::inet_address addr, std::optional<uint16_t> port, std::optional<uint16_t> https_port, std::optional<tls::credentials_builder> creds,
bool enforce_authorization, semaphore* memory_limiter, utils::updateable_value<uint32_t> max_concurrent_requests) {
_memory_limiter = memory_limiter;
_enforce_authorization = enforce_authorization;
_max_concurrent_requests = std::move(max_concurrent_requests);
if (!port && !https_port) {
return make_exception_future<>(std::runtime_error("Either regular port or TLS port"
" must be specified in order to init an alternator HTTP server instance"));
}
return seastar::async([this, addr, port, https_port, creds] {
_executor.start().get();
if (port) {
set_routes(_http_server._routes);
_http_server.set_content_length_limit(server::content_length_limit);
_http_server.set_content_streaming(true);
_http_server.listen(socket_address{addr, *port}).get();
_enabled_servers.push_back(std::ref(_http_server));
}
if (https_port) {
set_routes(_https_server._routes);
_https_server.set_content_length_limit(server::content_length_limit);
_https_server.set_content_streaming(true);
_https_server.set_tls_credentials(creds->build_reloadable_server_credentials([](const std::unordered_set<sstring>& files, std::exception_ptr ep) {
if (ep) {
slogger.warn("Exception loading {}: {}", files, ep);
} else {
slogger.info("Reloaded {}", files);
}
}).get0());
_https_server.listen(socket_address{addr, *https_port}).get();
_enabled_servers.push_back(std::ref(_https_server));
}
});
}
future<> server::stop() {
return parallel_for_each(_enabled_servers, [] (http_server& server) {
return server.stop();
}).then([this] {
return _pending_requests.close();
}).then([this] {
return _json_parser.stop();
});
}
server::json_parser::json_parser() : _run_parse_json_thread(async([this] {
while (true) {
_document_waiting.wait().get();
if (_as.abort_requested()) {
return;
}
try {
_parsed_document = rjson::parse_yieldable(std::move(_raw_document));
_current_exception = nullptr;
} catch (...) {
_current_exception = std::current_exception();
}
_document_parsed.signal();
}
})) {
}
future<rjson::value> server::json_parser::parse(chunked_content&& content) {
if (content.size() < yieldable_parsing_threshold) {
return make_ready_future<rjson::value>(rjson::parse(std::move(content)));
}
return with_semaphore(_parsing_sem, 1, [this, content = std::move(content)] () mutable {
_raw_document = std::move(content);
_document_waiting.signal();
return _document_parsed.wait().then([this] {
if (_current_exception) {
return make_exception_future<rjson::value>(_current_exception);
}
return make_ready_future<rjson::value>(std::move(_parsed_document));
});
});
}
future<> server::json_parser::stop() {
_as.request_abort();
_document_waiting.signal();
_document_parsed.broken();
return std::move(_run_parse_json_thread);
}
const char* api_error::what() const noexcept {
if (_what_string.empty()) {
_what_string = format("{} {}: {}", static_cast<int>(_http_code), _type, _msg);
}
return _what_string.c_str();
}
}

View File

@@ -1,84 +0,0 @@
/*
* Copyright 2019-present ScyllaDB
*/
/*
* SPDX-License-Identifier: AGPL-3.0-or-later
*/
#pragma once
#include "alternator/executor.hh"
#include <seastar/core/future.hh>
#include <seastar/core/condition-variable.hh>
#include <seastar/http/httpd.hh>
#include <seastar/net/tls.hh>
#include <optional>
#include "alternator/auth.hh"
#include "service/qos/service_level_controller.hh"
#include "utils/small_vector.hh"
#include "utils/updateable_value.hh"
#include <seastar/core/units.hh>
namespace alternator {
using chunked_content = rjson::chunked_content;
class server {
static constexpr size_t content_length_limit = 16*MB;
using alternator_callback = std::function<future<executor::request_return_type>(executor&, executor::client_state&,
tracing::trace_state_ptr, service_permit, rjson::value, std::unique_ptr<http::request>)>;
using alternator_callbacks_map = std::unordered_map<std::string_view, alternator_callback>;
httpd::http_server _http_server;
httpd::http_server _https_server;
executor& _executor;
service::storage_proxy& _proxy;
gms::gossiper& _gossiper;
auth::service& _auth_service;
qos::service_level_controller& _sl_controller;
key_cache _key_cache;
bool _enforce_authorization;
utils::small_vector<std::reference_wrapper<seastar::httpd::http_server>, 2> _enabled_servers;
gate _pending_requests;
alternator_callbacks_map _callbacks;
semaphore* _memory_limiter;
utils::updateable_value<uint32_t> _max_concurrent_requests;
class json_parser {
static constexpr size_t yieldable_parsing_threshold = 16*KB;
chunked_content _raw_document;
rjson::value _parsed_document;
std::exception_ptr _current_exception;
semaphore _parsing_sem{1};
condition_variable _document_waiting;
condition_variable _document_parsed;
abort_source _as;
future<> _run_parse_json_thread;
public:
json_parser();
// Moving a chunked_content into parse() allows parse() to free each
// chunk as soon as it is parsed, so when chunks are relatively small,
// we don't need to store the sum of unparsed and parsed sizes.
future<rjson::value> parse(chunked_content&& content);
future<> stop();
};
json_parser _json_parser;
public:
server(executor& executor, service::storage_proxy& proxy, gms::gossiper& gossiper, auth::service& service, qos::service_level_controller& sl_controller);
future<> init(net::inet_address addr, std::optional<uint16_t> port, std::optional<uint16_t> https_port, std::optional<tls::credentials_builder> creds,
bool enforce_authorization, semaphore* memory_limiter, utils::updateable_value<uint32_t> max_concurrent_requests);
future<> stop();
private:
void set_routes(seastar::httpd::routes& r);
// If verification succeeds, returns the authenticated user's username
future<std::string> verify_signature(const seastar::http::request&, const chunked_content&);
future<executor::request_return_type> handle_api_request(std::unique_ptr<http::request> req);
};
}

View File

@@ -1,99 +0,0 @@
/*
* Copyright 2019-present ScyllaDB
*/
/*
* SPDX-License-Identifier: AGPL-3.0-or-later
*/
#include "stats.hh"
#include "utils/histogram_metrics_helper.hh"
#include <seastar/core/metrics.hh>
namespace alternator {
const char* ALTERNATOR_METRICS = "alternator";
stats::stats() : api_operations{} {
// Register the
seastar::metrics::label op("op");
_metrics.add_group("alternator", {
#define OPERATION(name, CamelCaseName) \
seastar::metrics::make_total_operations("operation", api_operations.name, \
seastar::metrics::description("number of operations via Alternator API"), {op(CamelCaseName)}),
#define OPERATION_LATENCY(name, CamelCaseName) \
seastar::metrics::make_histogram("op_latency", \
seastar::metrics::description("Latency histogram of an operation via Alternator API"), {op(CamelCaseName)}, [this]{return to_metrics_histogram(api_operations.name);}),
OPERATION(batch_get_item, "BatchGetItem")
OPERATION(batch_write_item, "BatchWriteItem")
OPERATION(create_backup, "CreateBackup")
OPERATION(create_global_table, "CreateGlobalTable")
OPERATION(create_table, "CreateTable")
OPERATION(delete_backup, "DeleteBackup")
OPERATION(delete_item, "DeleteItem")
OPERATION(delete_table, "DeleteTable")
OPERATION(describe_backup, "DescribeBackup")
OPERATION(describe_continuous_backups, "DescribeContinuousBackups")
OPERATION(describe_endpoints, "DescribeEndpoints")
OPERATION(describe_global_table, "DescribeGlobalTable")
OPERATION(describe_global_table_settings, "DescribeGlobalTableSettings")
OPERATION(describe_limits, "DescribeLimits")
OPERATION(describe_table, "DescribeTable")
OPERATION(describe_time_to_live, "DescribeTimeToLive")
OPERATION(get_item, "GetItem")
OPERATION(list_backups, "ListBackups")
OPERATION(list_global_tables, "ListGlobalTables")
OPERATION(list_tables, "ListTables")
OPERATION(list_tags_of_resource, "ListTagsOfResource")
OPERATION(put_item, "PutItem")
OPERATION(query, "Query")
OPERATION(restore_table_from_backup, "RestoreTableFromBackup")
OPERATION(restore_table_to_point_in_time, "RestoreTableToPointInTime")
OPERATION(scan, "Scan")
OPERATION(tag_resource, "TagResource")
OPERATION(transact_get_items, "TransactGetItems")
OPERATION(transact_write_items, "TransactWriteItems")
OPERATION(untag_resource, "UntagResource")
OPERATION(update_continuous_backups, "UpdateContinuousBackups")
OPERATION(update_global_table, "UpdateGlobalTable")
OPERATION(update_global_table_settings, "UpdateGlobalTableSettings")
OPERATION(update_item, "UpdateItem")
OPERATION(update_table, "UpdateTable")
OPERATION(update_time_to_live, "UpdateTimeToLive")
OPERATION_LATENCY(put_item_latency, "PutItem")
OPERATION_LATENCY(get_item_latency, "GetItem")
OPERATION_LATENCY(delete_item_latency, "DeleteItem")
OPERATION_LATENCY(update_item_latency, "UpdateItem")
OPERATION(list_streams, "ListStreams")
OPERATION(describe_stream, "DescribeStream")
OPERATION(get_shard_iterator, "GetShardIterator")
OPERATION(get_records, "GetRecords")
OPERATION_LATENCY(get_records_latency, "GetRecords")
});
_metrics.add_group("alternator", {
seastar::metrics::make_total_operations("unsupported_operations", unsupported_operations,
seastar::metrics::description("number of unsupported operations via Alternator API")),
seastar::metrics::make_total_operations("total_operations", total_operations,
seastar::metrics::description("number of total operations via Alternator API")),
seastar::metrics::make_total_operations("reads_before_write", reads_before_write,
seastar::metrics::description("number of performed read-before-write operations")),
seastar::metrics::make_total_operations("write_using_lwt", write_using_lwt,
seastar::metrics::description("number of writes that used LWT")),
seastar::metrics::make_total_operations("shard_bounce_for_lwt", shard_bounce_for_lwt,
seastar::metrics::description("number writes that had to be bounced from this shard because of LWT requirements")),
seastar::metrics::make_total_operations("requests_blocked_memory", requests_blocked_memory,
seastar::metrics::description("Counts a number of requests blocked due to memory pressure.")),
seastar::metrics::make_total_operations("requests_shed", requests_shed,
seastar::metrics::description("Counts a number of requests shed due to overload.")),
seastar::metrics::make_total_operations("filtered_rows_read_total", cql_stats.filtered_rows_read_total,
seastar::metrics::description("number of rows read during filtering operations")),
seastar::metrics::make_total_operations("filtered_rows_matched_total", cql_stats.filtered_rows_matched_total,
seastar::metrics::description("number of rows read and matched during filtering operations")),
seastar::metrics::make_total_operations("filtered_rows_dropped_total", [this] { return cql_stats.filtered_rows_read_total - cql_stats.filtered_rows_matched_total; },
seastar::metrics::description("number of rows read and dropped during filtering operations")),
});
}
}

View File

@@ -1,91 +0,0 @@
/*
* Copyright 2019-present ScyllaDB
*/
/*
* SPDX-License-Identifier: AGPL-3.0-or-later
*/
#pragma once
#include <cstdint>
#include <seastar/core/metrics_registration.hh>
#include "seastarx.hh"
#include "utils/estimated_histogram.hh"
#include "cql3/stats.hh"
namespace alternator {
// Object holding per-shard statistics related to Alternator.
// While this object is alive, these metrics are also registered to be
// visible by the metrics REST API, with the "alternator" prefix.
class stats {
public:
stats();
// Count of DynamoDB API operations by types
struct {
uint64_t batch_get_item = 0;
uint64_t batch_write_item = 0;
uint64_t create_backup = 0;
uint64_t create_global_table = 0;
uint64_t create_table = 0;
uint64_t delete_backup = 0;
uint64_t delete_item = 0;
uint64_t delete_table = 0;
uint64_t describe_backup = 0;
uint64_t describe_continuous_backups = 0;
uint64_t describe_endpoints = 0;
uint64_t describe_global_table = 0;
uint64_t describe_global_table_settings = 0;
uint64_t describe_limits = 0;
uint64_t describe_table = 0;
uint64_t describe_time_to_live = 0;
uint64_t get_item = 0;
uint64_t list_backups = 0;
uint64_t list_global_tables = 0;
uint64_t list_tables = 0;
uint64_t list_tags_of_resource = 0;
uint64_t put_item = 0;
uint64_t query = 0;
uint64_t restore_table_from_backup = 0;
uint64_t restore_table_to_point_in_time = 0;
uint64_t scan = 0;
uint64_t tag_resource = 0;
uint64_t transact_get_items = 0;
uint64_t transact_write_items = 0;
uint64_t untag_resource = 0;
uint64_t update_continuous_backups = 0;
uint64_t update_global_table = 0;
uint64_t update_global_table_settings = 0;
uint64_t update_item = 0;
uint64_t update_table = 0;
uint64_t update_time_to_live = 0;
uint64_t list_streams = 0;
uint64_t describe_stream = 0;
uint64_t get_shard_iterator = 0;
uint64_t get_records = 0;
utils::time_estimated_histogram put_item_latency;
utils::time_estimated_histogram get_item_latency;
utils::time_estimated_histogram delete_item_latency;
utils::time_estimated_histogram update_item_latency;
utils::time_estimated_histogram get_records_latency;
} api_operations;
// Miscellaneous event counters
uint64_t total_operations = 0;
uint64_t unsupported_operations = 0;
uint64_t reads_before_write = 0;
uint64_t write_using_lwt = 0;
uint64_t shard_bounce_for_lwt = 0;
uint64_t requests_blocked_memory = 0;
uint64_t requests_shed = 0;
// CQL-derived stats
cql3::cql_stats cql_stats;
private:
// The metric_groups object holds this stat object's metrics registered
// as long as the stats object is alive.
seastar::metrics::metric_groups _metrics;
};
}

File diff suppressed because it is too large Load Diff

View File

@@ -1,849 +0,0 @@
/*
* Copyright 2021-present ScyllaDB
*/
/*
* SPDX-License-Identifier: AGPL-3.0-or-later
*/
#include <chrono>
#include <cstdint>
#include <exception>
#include <optional>
#include <seastar/core/sstring.hh>
#include <seastar/core/coroutine.hh>
#include <seastar/core/sleep.hh>
#include <seastar/core/future.hh>
#include <seastar/core/lowres_clock.hh>
#include <seastar/coroutine/maybe_yield.hh>
#include <boost/multiprecision/cpp_int.hpp>
#include "exceptions/exceptions.hh"
#include "gms/gossiper.hh"
#include "gms/inet_address.hh"
#include "inet_address_vectors.hh"
#include "locator/abstract_replication_strategy.hh"
#include "log.hh"
#include "gc_clock.hh"
#include "replica/database.hh"
#include "service_permit.hh"
#include "timestamp.hh"
#include "service/storage_proxy.hh"
#include "service/pager/paging_state.hh"
#include "service/pager/query_pagers.hh"
#include "gms/feature_service.hh"
#include "sstables/types.hh"
#include "mutation/mutation.hh"
#include "types/types.hh"
#include "types/map.hh"
#include "utils/rjson.hh"
#include "utils/big_decimal.hh"
#include "utils/fb_utilities.hh"
#include "cql3/selection/selection.hh"
#include "cql3/values.hh"
#include "cql3/query_options.hh"
#include "cql3/column_identifier.hh"
#include "alternator/executor.hh"
#include "alternator/controller.hh"
#include "alternator/serialization.hh"
#include "dht/sharder.hh"
#include "db/config.hh"
#include "db/tags/utils.hh"
#include "ttl.hh"
static logging::logger tlogger("alternator_ttl");
namespace alternator {
// We write the expiration-time attribute enabled on a table using a
// tag TTL_TAG_KEY.
// Currently, the *value* of this tag is simply the name of the attribute,
// and the expiration scanner interprets it as an Alternator attribute name -
// It can refer to a real column or if that doesn't exist, to a member of
// the ":attrs" map column. Although this is designed for Alternator, it may
// be good enough for CQL as well (there, the ":attrs" column won't exist).
static const sstring TTL_TAG_KEY("system:ttl_attribute");
future<executor::request_return_type> executor::update_time_to_live(client_state& client_state, service_permit permit, rjson::value request) {
_stats.api_operations.update_time_to_live++;
if (!_proxy.data_dictionary().features().alternator_ttl) {
co_return api_error::unknown_operation("UpdateTimeToLive not yet supported. Experimental support is available if the 'alternator-ttl' experimental feature is enabled on all nodes.");
}
schema_ptr schema = get_table(_proxy, request);
rjson::value* spec = rjson::find(request, "TimeToLiveSpecification");
if (!spec || !spec->IsObject()) {
co_return api_error::validation("UpdateTimeToLive missing mandatory TimeToLiveSpecification");
}
const rjson::value* v = rjson::find(*spec, "Enabled");
if (!v || !v->IsBool()) {
co_return api_error::validation("UpdateTimeToLive requires boolean Enabled");
}
bool enabled = v->GetBool();
v = rjson::find(*spec, "AttributeName");
if (!v || !v->IsString()) {
co_return api_error::validation("UpdateTimeToLive requires string AttributeName");
}
// Although the DynamoDB documentation specifies that attribute names
// should be between 1 and 64K bytes, in practice, it only allows
// between 1 and 255 bytes. There are no other limitations on which
// characters are allowed in the name.
if (v->GetStringLength() < 1 || v->GetStringLength() > 255) {
co_return api_error::validation("The length of AttributeName must be between 1 and 255");
}
sstring attribute_name(v->GetString(), v->GetStringLength());
co_await db::modify_tags(_mm, schema->ks_name(), schema->cf_name(), [&](std::map<sstring, sstring>& tags_map) {
if (enabled) {
if (tags_map.contains(TTL_TAG_KEY)) {
throw api_error::validation("TTL is already enabled");
}
tags_map[TTL_TAG_KEY] = attribute_name;
} else {
auto i = tags_map.find(TTL_TAG_KEY);
if (i == tags_map.end()) {
throw api_error::validation("TTL is already disabled");
} else if (i->second != attribute_name) {
throw api_error::validation(format(
"Requested to disable TTL on attribute {}, but a different attribute {} is enabled.",
attribute_name, i->second));
}
tags_map.erase(TTL_TAG_KEY);
}
});
// Prepare the response, which contains a TimeToLiveSpecification
// basically identical to the request's
rjson::value response = rjson::empty_object();
rjson::add(response, "TimeToLiveSpecification", std::move(*spec));
co_return make_jsonable(std::move(response));
}
future<executor::request_return_type> executor::describe_time_to_live(client_state& client_state, service_permit permit, rjson::value request) {
_stats.api_operations.describe_time_to_live++;
schema_ptr schema = get_table(_proxy, request);
std::map<sstring, sstring> tags_map = get_tags_of_table_or_throw(schema);
rjson::value desc = rjson::empty_object();
auto i = tags_map.find(TTL_TAG_KEY);
if (i == tags_map.end()) {
rjson::add(desc, "TimeToLiveStatus", "DISABLED");
} else {
rjson::add(desc, "TimeToLiveStatus", "ENABLED");
rjson::add(desc, "AttributeName", rjson::from_string(i->second));
}
rjson::value response = rjson::empty_object();
rjson::add(response, "TimeToLiveDescription", std::move(desc));
co_return make_jsonable(std::move(response));
}
// expiration_service is a sharded service responsible for cleaning up expired
// items in all tables with per-item expiration enabled. Currently, this means
// Alternator tables with TTL configured via a UpdateTimeToLive request.
//
// Here is a brief overview of how the expiration service works:
//
// An expiration thread on each shard periodically scans the items (i.e.,
// rows) owned by this shard, looking for items whose chosen expiration-time
// attribute indicates they are expired, and deletes those items.
// The expiration-time "attribute" can be either an actual Scylla column
// (must be numeric) or an Alternator "attribute" - i.e., an element in
// the ATTRS_COLUMN_NAME map<utf8,bytes> column where the numeric expiration
// time is encoded in DynamoDB's JSON encoding inside the bytes value.
// To avoid scanning the same items RF times in RF replicas, only one node is
// responsible for scanning a token range at a time. Normally, this is the
// node owning this range as a "primary range" (the first node in the ring
// with this range), but when this node is down, the secondary owner (the
// second in the ring) may take over.
// An expiration thread is reponsible for all tables which need expiration
// scans. Currently, the different tables are scanned sequentially (not in
// parallel).
// The expiration thread scans item using CL=QUORUM to ensures that it reads
// a consistent expiration-time attribute. This means that the items are read
// locally and in addition QUORUM-1 additional nodes (one additional node
// when RF=3) need to read the data and send digests.
// When the expiration thread decides that an item has expired and wants
// to delete it, it does it using a CL=QUORUM write. This allows this
// deletion to be visible for consistent (quorum) reads. The deletion,
// like user deletions, will also appear on the CDC log and therefore
// Alternator Streams if enabled - currently as ordinary deletes (the
// userIdentity flag is currently missing this is issue #11523).
expiration_service::expiration_service(data_dictionary::database db, service::storage_proxy& proxy, gms::gossiper& g)
: _db(db)
, _proxy(proxy)
, _gossiper(g)
{
}
// Convert the big_decimal used to represent expiration time to an integer.
// Any fractional part is dropped. If the number is negative or invalid,
// 0 is returned, and if it's too high, the maximum unsigned long is returned.
static unsigned long bigdecimal_to_ul(const big_decimal& bd) {
// The big_decimal format has an integer mantissa of arbitrary length
// "unscaled_value" and then a (power of 10) exponent "scale".
if (bd.unscaled_value() <= 0) {
return 0;
}
if (bd.scale() == 0) {
// The fast path, when the expiration time is an integer, scale==0.
return static_cast<unsigned long>(bd.unscaled_value());
}
// Because the mantissa can be of arbitrary length, we work on it
// as a string. TODO: find a less ugly algorithm.
auto str = bd.unscaled_value().str();
if (bd.scale() > 0) {
int len = str.length();
if (len < bd.scale()) {
return 0;
}
str = str.substr(0, len-bd.scale());
} else {
if (bd.scale() < -20) {
return std::numeric_limits<unsigned long>::max();
}
for (int i = 0; i < -bd.scale(); i++) {
str.push_back('0');
}
}
// strtoul() returns ULONG_MAX if the number is too large, or 0 if not
// a number.
return strtoul(str.c_str(), nullptr, 10);
}
// The following is_expired() functions all check if an item with the given
// expiration time has expired, according to the DynamoDB API rules.
// The rules are:
// 1. If the expiration time attribute's value is not a number type,
// the item is not expired.
// 2. The expiration time is measured in seconds since the UNIX epoch.
// 3. If the expiration time is more than 5 years in the past, it is assumed
// to be malformed and ignored - and the item does not expire.
static bool is_expired(gc_clock::time_point expiration_time, gc_clock::time_point now) {
return expiration_time <= now &&
expiration_time > now - std::chrono::years(5);
}
static bool is_expired(const big_decimal& expiration_time, gc_clock::time_point now) {
unsigned long t = bigdecimal_to_ul(expiration_time);
// We assume - and the assumption turns out to be correct - that the
// epoch of gc_clock::time_point and the one used by the DynamoDB protocol
// are the same (the UNIX epoch in UTC). The resolution (seconds) is also
// the same.
return is_expired(gc_clock::time_point(gc_clock::duration(std::chrono::seconds(t))), now);
}
static bool is_expired(const rjson::value& expiration_time, gc_clock::time_point now) {
std::optional<big_decimal> n = try_unwrap_number(expiration_time);
return n && is_expired(*n, now);
}
// expire_item() expires an item - i.e., deletes it as appropriate for
// expiration - with CL=QUORUM and (FIXME!) in a way Alternator Streams
// understands it is an expiration event - not a user-initiated deletion.
static future<> expire_item(service::storage_proxy& proxy,
const service::query_state& qs,
const std::vector<bytes_opt>& row,
schema_ptr schema,
api::timestamp_type ts) {
// Prepare the row key to delete
// NOTICE: the order of columns is guaranteed by the fact that selection::wildcard
// is used, which indicates that columns appear in the order defined by
// schema::all_columns_in_select_order() - partition key columns goes first,
// immediately followed by clustering key columns
std::vector<bytes> exploded_pk;
const unsigned pk_size = schema->partition_key_size();
const unsigned ck_size = schema->clustering_key_size();
for (unsigned c = 0; c < pk_size; ++c) {
const auto& row_c = row[c];
if (!row_c) {
// This shouldn't happen - all key columns must have values.
// But if it ever happens, let's just *not* expire the item.
// FIXME: log or increment a metric if this happens.
return make_ready_future<>();
}
exploded_pk.push_back(*row_c);
}
auto pk = partition_key::from_exploded(exploded_pk);
mutation m(schema, pk);
// If there's no clustering key, a tombstone should be created directly
// on a partition, not on a clustering row - otherwise it will look like
// an open-ended range tombstone, which will crash on KA/LA sstable format.
// See issue #6035
if (ck_size == 0) {
m.partition().apply(tombstone(ts, gc_clock::now()));
} else {
std::vector<bytes> exploded_ck;
for (unsigned c = pk_size; c < pk_size + ck_size; ++c) {
const auto& row_c = row[c];
if (!row_c) {
// This shouldn't happen - all key columns must have values.
// But if it ever happens, let's just *not* expire the item.
// FIXME: log or increment a metric if this happens.
return make_ready_future<>();
}
exploded_ck.push_back(*row_c);
}
auto ck = clustering_key::from_exploded(exploded_ck);
m.partition().clustered_row(*schema, ck).apply(tombstone(ts, gc_clock::now()));
}
std::vector<mutation> mutations;
mutations.push_back(std::move(m));
return proxy.mutate(std::move(mutations),
db::consistency_level::LOCAL_QUORUM,
executor::default_timeout(), // FIXME - which timeout?
qs.get_trace_state(), qs.get_permit(),
db::allow_per_partition_rate_limit::no);
}
static size_t random_offset(size_t min, size_t max) {
static thread_local std::default_random_engine re{std::random_device{}()};
std::uniform_int_distribution<size_t> dist(min, max);
return dist(re);
}
// Get a list of secondary token ranges for the given node, and the primary
// node responsible for each of these token ranges.
// A "secondary range" is a range of tokens where for each token, the second
// node (in ring order) out of the RF replicas that hold this token is the
// given node.
// In the expiration scanner, we want to scan a secondary range but only if
// this range's primary node is down. For this we need to return not just
// a list of this node's secondary ranges - but also the primary owner of
// each of those ranges.
static std::vector<std::pair<dht::token_range, gms::inet_address>> get_secondary_ranges(
const locator::effective_replication_map_ptr& erm,
gms::inet_address ep) {
const auto& tm = *erm->get_token_metadata_ptr();
const auto& sorted_tokens = tm.sorted_tokens();
std::vector<std::pair<dht::token_range, gms::inet_address>> ret;
if (sorted_tokens.empty()) {
on_internal_error(tlogger, "Token metadata is empty");
}
auto prev_tok = sorted_tokens.back();
for (const auto& tok : sorted_tokens) {
inet_address_vector_replica_set eps = erm->get_natural_endpoints(tok);
if (eps.size() <= 1 || eps[1] != ep) {
prev_tok = tok;
continue;
}
// Add the range (prev_tok, tok] to ret. However, if the range wraps
// around, split it to two non-wrapping ranges.
if (prev_tok < tok) {
ret.emplace_back(
dht::token_range{
dht::token_range::bound(prev_tok, false),
dht::token_range::bound(tok, true)},
eps[0]);
} else {
ret.emplace_back(
dht::token_range{
dht::token_range::bound(prev_tok, false),
std::nullopt},
eps[0]);
ret.emplace_back(
dht::token_range{
std::nullopt,
dht::token_range::bound(tok, true)},
eps[0]);
}
prev_tok = tok;
}
return ret;
}
// A class for iterating over all the token ranges *owned* by this shard.
// To avoid code duplication, it is a template with two distinct cases -
// <primary> and <secondary>:
//
// In the <primary> case, we consider a token *owned* by this shard if:
// 1. This node is a replica for this token.
// 2. Moreover, this node is the *primary* replica of the token (i.e., the
// first replica in the ring).
// 3. In this node, this shard is responsible for this token.
// We will use this definition of which shard in the cluster owns which tokens
// to split the expiration scanner's work between all the shards of the
// system.
//
// In the <secondary> case, we consider a token *owned* by this shard if:
// 1. This node is the *secondary* replica for this token (i.e., the second
// replica in the ring).
// 2. The primary replica for this token is currently marked down.
// 3. In this node, this shard is responsible for this token.
// We use the <secondary> case to handle the possibility that some of the
// nodes in the system are down. A dead node will not be expiring
// the tokens owned by it, so we want the secondary owner to take over its
// primary ranges.
//
// FIXME: need to decide how to choose primary ranges in multi-DC setup!
// We could call get_primary_ranges_within_dc() below instead of get_primary_ranges().
// NOTICE: Iteration currently starts from a random token range in order to improve
// the chances of covering all ranges during a scan when restarts occur.
// A more deterministic way would be to regularly persist the scanning state,
// but that incurs overhead that we want to avoid if not needed.
enum primary_or_secondary_t {primary, secondary};
template<primary_or_secondary_t primary_or_secondary>
class token_ranges_owned_by_this_shard {
// ranges_holder_primary holds just the primary ranges themselves
class ranges_holder_primary {
const dht::token_range_vector _token_ranges;
public:
ranges_holder_primary(const locator::effective_replication_map_ptr& erm, gms::gossiper& g, gms::inet_address ep)
: _token_ranges(erm->get_primary_ranges(ep)) {}
std::size_t size() const { return _token_ranges.size(); }
const dht::token_range& operator[](std::size_t i) const {
return _token_ranges[i];
}
bool should_skip(std::size_t i) const {
return false;
}
};
// ranges_holder<secondary> holds the secondary token ranges plus each
// range's primary owner, needed to implement should_skip().
class ranges_holder_secondary {
std::vector<std::pair<dht::token_range, gms::inet_address>> _token_ranges;
gms::gossiper& _gossiper;
public:
ranges_holder_secondary(const locator::effective_replication_map_ptr& erm, gms::gossiper& g, gms::inet_address ep)
: _token_ranges(get_secondary_ranges(erm, ep))
, _gossiper(g) {}
std::size_t size() const { return _token_ranges.size(); }
const dht::token_range& operator[](std::size_t i) const {
return _token_ranges[i].first;
}
// range i should be skipped if its primary owner is alive.
bool should_skip(std::size_t i) const {
return _gossiper.is_alive(_token_ranges[i].second);
}
};
schema_ptr _s;
// _token_ranges will contain a list of token ranges owned by this node.
// We'll further need to split each such range to the pieces owned by
// the current shard, using _intersecter.
using ranges_holder = std::conditional_t<
primary_or_secondary == primary_or_secondary_t::primary,
ranges_holder_primary,
ranges_holder_secondary>;
const ranges_holder _token_ranges;
// NOTICE: _range_idx is used modulo _token_ranges size when accessing
// the data to ensure that it doesn't go out of bounds
size_t _range_idx;
size_t _end_idx;
std::optional<dht::selective_token_range_sharder> _intersecter;
public:
token_ranges_owned_by_this_shard(replica::database& db, gms::gossiper& g, schema_ptr s)
: _s(s)
, _token_ranges(db.find_keyspace(s->ks_name()).get_effective_replication_map(),
g, utils::fb_utilities::get_broadcast_address())
, _range_idx(random_offset(0, _token_ranges.size() - 1))
, _end_idx(_range_idx + _token_ranges.size())
{
tlogger.debug("Generating token ranges starting from base range {} of {}", _range_idx, _token_ranges.size());
}
// Return the next token_range owned by this shard, or nullopt when the
// iteration ends.
std::optional<dht::token_range> next() {
// We may need three or more iterations in the following loop if a
// vnode doesn't intersect with the given shard at all (such a small
// vnode is unlikely, but possible). The loop cannot be infinite
// because each iteration of the loop advances _range_idx.
for (;;) {
if (_intersecter) {
std::optional<dht::token_range> ret = _intersecter->next();
if (ret) {
return ret;
}
// done with this range, go to next one
++_range_idx;
_intersecter = std::nullopt;
}
if (_range_idx == _end_idx) {
return std::nullopt;
}
// If should_skip(), the range should be skipped. This happens for
// a secondary range whose primary owning node is still alive.
while (_token_ranges.should_skip(_range_idx % _token_ranges.size())) {
++_range_idx;
if (_range_idx == _end_idx) {
return std::nullopt;
}
}
_intersecter.emplace(_s->get_sharder(), _token_ranges[_range_idx % _token_ranges.size()], this_shard_id());
}
}
// Same as next(), just return a partition_range instead of token_range
std::optional<dht::partition_range> next_partition_range() {
std::optional<dht::token_range> ret = next();
if (ret) {
return dht::to_partition_range(*ret);
} else {
return std::nullopt;
}
}
};
// Precomputed information needed to perform a scan on partition ranges
struct scan_ranges_context {
schema_ptr s;
bytes column_name;
std::optional<std::string> member;
::shared_ptr<cql3::selection::selection> selection;
std::unique_ptr<service::query_state> query_state_ptr;
std::unique_ptr<cql3::query_options> query_options;
::lw_shared_ptr<query::read_command> command;
scan_ranges_context(schema_ptr s, service::storage_proxy& proxy, bytes column_name, std::optional<std::string> member)
: s(s)
, column_name(column_name)
, member(member)
{
// FIXME: don't read the entire items - read only parts of it.
// We must read the key columns (to be able to delete) and also
// the requested attribute. If the requested attribute is a map's
// member we may be forced to read the entire map - but it would
// be good if we can read only the single item of the map - it
// should be possible (and a must for issue #7751!).
lw_shared_ptr<service::pager::paging_state> paging_state = nullptr;
auto regular_columns = boost::copy_range<query::column_id_vector>(
s->regular_columns() | boost::adaptors::transformed([] (const column_definition& cdef) { return cdef.id; }));
selection = cql3::selection::selection::wildcard(s);
query::partition_slice::option_set opts = selection->get_query_options();
opts.set<query::partition_slice::option::allow_short_read>();
// It is important that the scan bypass cache to avoid polluting it:
opts.set<query::partition_slice::option::bypass_cache>();
std::vector<query::clustering_range> ck_bounds{query::clustering_range::make_open_ended_both_sides()};
auto partition_slice = query::partition_slice(std::move(ck_bounds), {}, std::move(regular_columns), opts);
command = ::make_lw_shared<query::read_command>(s->id(), s->version(), partition_slice, proxy.get_max_result_size(partition_slice), query::tombstone_limit(proxy.get_tombstone_limit()));
executor::client_state client_state{executor::client_state::internal_tag()};
tracing::trace_state_ptr trace_state;
// NOTICE: empty_service_permit is used because the TTL service has fixed parallelism
query_state_ptr = std::make_unique<service::query_state>(client_state, trace_state, empty_service_permit());
// FIXME: What should we do on multi-DC? Will we run the expiration on the same ranges on all
// DCs or only once for each range? If the latter, we need to change the CLs in the
// scanner and deleter.
db::consistency_level cl = db::consistency_level::LOCAL_QUORUM;
query_options = std::make_unique<cql3::query_options>(cl, std::vector<cql3::raw_value>{});
query_options = std::make_unique<cql3::query_options>(std::move(query_options), std::move(paging_state));
}
};
// Scan data in a list of token ranges in one table, looking for expired
// items and deleting them.
// Because of issue #9167, partition_ranges must have a single partition
// range for this code to work correctly.
static future<> scan_table_ranges(
service::storage_proxy& proxy,
const scan_ranges_context& scan_ctx,
dht::partition_range_vector&& partition_ranges,
abort_source& abort_source,
named_semaphore& page_sem,
expiration_service::stats& expiration_stats)
{
const schema_ptr& s = scan_ctx.s;
assert (partition_ranges.size() == 1); // otherwise issue #9167 will cause incorrect results.
auto p = service::pager::query_pagers::pager(proxy, s, scan_ctx.selection, *scan_ctx.query_state_ptr,
*scan_ctx.query_options, scan_ctx.command, std::move(partition_ranges), nullptr);
while (!p->is_exhausted()) {
if (abort_source.abort_requested()) {
co_return;
}
auto units = co_await get_units(page_sem, 1);
// We don't need to limit page size in number of rows because there is
// a builtin limit of the page's size in bytes. Setting this limit to
// 1 is useful for debugging the paging code with moderate-size data.
uint32_t limit = std::numeric_limits<uint32_t>::max();
// Read a page, and if that times out, try again after a small sleep.
// If we didn't catch the timeout exception, it would cause the scan
// be aborted and only be restarted at the next scanning period.
// If we retry too many times, give up and restart the scan later.
std::unique_ptr<cql3::result_set> rs;
for (int retries=0; ; retries++) {
try {
// FIXME: which timeout?
rs = co_await p->fetch_page(limit, gc_clock::now(), executor::default_timeout());
break;
} catch(exceptions::read_timeout_exception&) {
tlogger.warn("expiration scanner read timed out, will retry: {}",
std::current_exception());
}
// If we didn't break out of this loop, add a minimal sleep
if (retries >= 10) {
// Don't get stuck forever asking the same page, maybe there's
// a bug or a real problem in several replicas. Give up on
// this scan an retry the scan from a random position later,
// in the next scan period.
throw runtime_exception("scanner thread failed after too many timeouts for the same page");
}
co_await sleep_abortable(std::chrono::seconds(1), abort_source);
}
auto rows = rs->rows();
auto meta = rs->get_metadata().get_names();
std::optional<unsigned> expiration_column;
for (unsigned i = 0; i < meta.size(); i++) {
const cql3::column_specification& col = *meta[i];
if (col.name->name() == scan_ctx.column_name) {
expiration_column = i;
break;
}
}
if (!expiration_column) {
continue;
}
for (const auto& row : rows) {
const bytes_opt& cell = row[*expiration_column];
if (!cell) {
continue;
}
auto v = meta[*expiration_column]->type->deserialize(*cell);
bool expired = false;
// FIXME: don't recalculate "now" all the time
auto now = gc_clock::now();
if (scan_ctx.member) {
// In this case, the expiration-time attribute we're
// looking for is a member in a map, saved serialized
// into bytes using Alternator's serialization (basically
// a JSON serialized into bytes)
// FIXME: is it possible to find a specific member of a map
// without iterating through it like we do here and compare
// the key?
for (const auto& entry : value_cast<map_type_impl::native_type>(v)) {
std::string attr_name = value_cast<sstring>(entry.first);
if (value_cast<sstring>(entry.first) == *scan_ctx.member) {
bytes value = value_cast<bytes>(entry.second);
rjson::value json = deserialize_item(value);
expired = is_expired(json, now);
break;
}
}
} else {
// For a real column to contain an expiration time, it
// must be a numeric type.
// FIXME: Currently we only support decimal_type (which is
// what Alternator uses), but other numeric types can be
// supported as well to make this feature more useful in CQL.
// Note that kind::decimal is also checked above.
big_decimal n = value_cast<big_decimal>(v);
expired = is_expired(n, now);
}
if (expired) {
expiration_stats.items_deleted++;
// FIXME: maybe don't recalculate new_timestamp() all the time
// FIXME: if expire_item() throws on timeout, we need to retry it.
auto ts = api::new_timestamp();
co_await expire_item(proxy, *scan_ctx.query_state_ptr, row, s, ts);
}
}
// FIXME: once in a while, persist p->state(), so on reboot
// we don't start from scratch.
}
}
// scan_table() scans, in one table, data "owned" by this shard, looking for
// expired items and deleting them.
// We consider each node to "own" its primary token ranges, i.e., the tokens
// that this node is their first replica in the ring. Inside the node, each
// shard "owns" subranges of the node's token ranges - according to the node's
// sharding algorithm.
// When a node goes down, the token ranges owned by it will not be scanned
// and items in those token ranges will not expire, so in the future (FIXME)
// this function should additionally work on token ranges whose primary owner
// is down and this node is the range's secondary owner.
// If the TTL (expiration-time scanning) feature is not enabled for this
// table, scan_table() returns false without doing anything. Remember that the
// TTL feature may be enabled later so this function will need to be called
// again when the feature is enabled.
// Currently this function scans the entire table (or, rather the parts owned
// by this shard) at full rate, once. In the future (FIXME) we should consider
// how to pace this scan, how and when to repeat it, how to interleave or
// parallelize scanning of multiple tables, and how to continue scans after a
// reboot.
static future<bool> scan_table(
service::storage_proxy& proxy,
data_dictionary::database db,
gms::gossiper& gossiper,
schema_ptr s,
abort_source& abort_source,
named_semaphore& page_sem,
expiration_service::stats& expiration_stats)
{
// Check if an expiration-time attribute is enabled for this table.
// If not, just return false immediately.
// FIXME: the setting of the TTL may change in the middle of a long scan!
std::optional<std::string> attribute_name = db::find_tag(*s, TTL_TAG_KEY);
if (!attribute_name) {
co_return false;
}
// attribute_name may be one of the schema's columns (in Alternator, this
// means it's a key column), or an element in Alternator's attrs map
// encoded in Alternator's JSON encoding.
// FIXME: To make this less Alternators-specific, we should encode in the
// single key's value three things:
// 1. The name of a column
// 2. Optionally if column is a map, a member in the map
// 3. The deserializer for the value: CQL or Alternator (JSON).
// The deserializer can be guessed: If the given column or map item is
// numeric, it can be used directly. If it is a "bytes" type, it needs to
// be deserialized using Alternator's deserializer.
bytes column_name = to_bytes(*attribute_name);
const column_definition *cd = s->get_column_definition(column_name);
std::optional<std::string> member;
if (!cd) {
member = std::move(attribute_name);
column_name = bytes(executor::ATTRS_COLUMN_NAME);
cd = s->get_column_definition(column_name);
tlogger.info("table {} TTL enabled with attribute {} in {}", s->cf_name(), *member, executor::ATTRS_COLUMN_NAME);
} else {
tlogger.info("table {} TTL enabled with attribute {}", s->cf_name(), *attribute_name);
}
if (!cd) {
tlogger.info("table {} TTL column is missing, not scanning", s->cf_name());
co_return false;
}
data_type column_type = cd->type;
// Verify that the column has the right type: If "member" exists
// the column must be a map, and if it doesn't, the column must
// (currently) be a decimal_type. If the column has the wrong type
// nothing can get expired in this table, and it's pointless to
// scan it.
if ((member && column_type->get_kind() != abstract_type::kind::map) ||
(!member && column_type->get_kind() != abstract_type::kind::decimal)) {
tlogger.info("table {} TTL column has unsupported type, not scanning", s->cf_name());
co_return false;
}
expiration_stats.scan_table++;
// FIXME: need to pace the scan, not do it all at once.
scan_ranges_context scan_ctx{s, proxy, std::move(column_name), std::move(member)};
token_ranges_owned_by_this_shard<primary> my_ranges(db.real_database(), gossiper, s);
while (std::optional<dht::partition_range> range = my_ranges.next_partition_range()) {
// Note that because of issue #9167 we need to run a separate
// query on each partition range, and can't pass several of
// them into one partition_range_vector.
dht::partition_range_vector partition_ranges;
partition_ranges.push_back(std::move(*range));
// FIXME: if scanning a single range fails, including network errors,
// we fail the entire scan (and rescan from the beginning). Need to
// reconsider this. Saving the scan position might be a good enough
// solution for this problem.
co_await scan_table_ranges(proxy, scan_ctx, std::move(partition_ranges), abort_source, page_sem, expiration_stats);
}
// If each node only scans its own primary ranges, then when any node is
// down part of the token range will not get scanned. This can be viewed
// as acceptable (when the comes back online, it will resume its scan),
// but as noted in issue #9787, we can allow more prompt expiration
// by tasking another node to take over scanning of the dead node's primary
// ranges. What we do here is that this node will also check expiration
// on its *secondary* ranges - but only those whose primary owner is down.
token_ranges_owned_by_this_shard<secondary> my_secondary_ranges(db.real_database(), gossiper, s);
while (std::optional<dht::partition_range> range = my_secondary_ranges.next_partition_range()) {
expiration_stats.secondary_ranges_scanned++;
dht::partition_range_vector partition_ranges;
partition_ranges.push_back(std::move(*range));
co_await scan_table_ranges(proxy, scan_ctx, std::move(partition_ranges), abort_source, page_sem, expiration_stats);
}
co_return true;
}
future<> expiration_service::run() {
// FIXME: don't just tight-loop, think about timing, pace, and
// store position in durable storage, etc.
// FIXME: think about working on different tables in parallel.
// also need to notice when a new table is added, a table is
// deleted or when ttl is enabled or disabled for a table!
for (;;) {
auto start = lowres_clock::now();
// _db.tables() may change under our feet during a
// long-living loop, so we must keep our own copy of the list of
// schemas.
std::vector<schema_ptr> schemas;
for (auto cf : _db.get_tables()) {
schemas.push_back(cf.schema());
}
for (schema_ptr s : schemas) {
co_await coroutine::maybe_yield();
if (shutting_down()) {
co_return;
}
try {
co_await scan_table(_proxy, _db, _gossiper, s, _abort_source, _page_sem, _expiration_stats);
} catch (...) {
// The scan of a table may fail in the middle for many
// reasons, including network failure and even the table
// being removed. We'll continue scanning this table later
// (if it still exists). In any case it's important to catch
// the exception and not let the scanning service die for
// good.
// If the table has been deleted, it is expected that the scan
// will fail at some point, and even a warning is excessive.
if (_db.has_schema(s->ks_name(), s->cf_name())) {
tlogger.warn("table {}.{} expiration scan failed: {}",
s->ks_name(), s->cf_name(), std::current_exception());
} else {
tlogger.info("expiration scan failed when table {}.{} was deleted",
s->ks_name(), s->cf_name());
}
}
}
_expiration_stats.scan_passes++;
// The TTL scanner runs above once over all tables, at full steam.
// After completing such a scan, we sleep until it's time start
// another scan. TODO: If the scan went too fast, we can slow it down
// in the next iteration by reducing the scanner's scheduling-group
// share (if using a separate scheduling group), or introduce
// finer-grain sleeps into the scanning code.
std::chrono::milliseconds scan_duration(std::chrono::duration_cast<std::chrono::milliseconds>(lowres_clock::now() - start));
std::chrono::milliseconds period(long(_db.get_config().alternator_ttl_period_in_seconds() * 1000));
if (scan_duration < period) {
try {
tlogger.info("sleeping {} seconds until next period", (period - scan_duration).count()/1000.0);
co_await seastar::sleep_abortable(period - scan_duration, _abort_source);
} catch(seastar::sleep_aborted&) {}
} else {
tlogger.warn("scan took {} seconds, longer than period - not sleeping", scan_duration.count()/1000.0);
}
}
}
future<> expiration_service::start() {
// Called by main() on each shard to start the expiration-service
// thread. Just runs run() in the background and allows stop().
if (_db.features().alternator_ttl) {
if (!shutting_down()) {
_end = run().handle_exception([] (std::exception_ptr ep) {
tlogger.error("expiration_service failed: {}", ep);
});
}
}
return make_ready_future<>();
}
future<> expiration_service::stop() {
if (_abort_source.abort_requested()) {
throw std::logic_error("expiration_service::stop() called a second time");
}
_abort_source.request_abort();
if (!_end) {
// if _end is was not set, start() was never called
return make_ready_future<>();
}
return std::move(*_end);
}
expiration_service::stats::stats() {
_metrics.add_group("expiration", {
seastar::metrics::make_total_operations("scan_passes", scan_passes,
seastar::metrics::description("number of passes over the database")),
seastar::metrics::make_total_operations("scan_table", scan_table,
seastar::metrics::description("number of table scans (counting each scan of each table that enabled expiration)")),
seastar::metrics::make_total_operations("items_deleted", items_deleted,
seastar::metrics::description("number of items deleted after expiration")),
seastar::metrics::make_total_operations("secondary_ranges_scanned", secondary_ranges_scanned,
seastar::metrics::description("number of token ranges scanned by this node while their primary owner was down")),
});
}
} // namespace alternator

View File

@@ -1,80 +0,0 @@
/*
* Copyright 2021-present ScyllaDB
*/
/*
* SPDX-License-Identifier: AGPL-3.0-or-later
*/
#pragma once
#include "seastarx.hh"
#include <seastar/core/sharded.hh>
#include <seastar/core/abort_source.hh>
#include <seastar/core/semaphore.hh>
#include "data_dictionary/data_dictionary.hh"
namespace gms {
class gossiper;
}
namespace replica {
class database;
}
namespace service {
class storage_proxy;
}
namespace alternator {
// expiration_service is a sharded service responsible for cleaning up expired
// items in all tables with per-item expiration enabled. Currently, this means
// Alternator tables with TTL configured via a UpdateTimeToLeave request.
class expiration_service final : public seastar::peering_sharded_service<expiration_service> {
public:
// Object holding per-shard statistics related to the expiration service.
// While this object is alive, these metrics are also registered to be
// visible by the metrics REST API, with the "expiration_" prefix.
class stats {
public:
stats();
uint64_t scan_passes = 0;
uint64_t scan_table = 0;
uint64_t items_deleted = 0;
uint64_t secondary_ranges_scanned = 0;
private:
// The metric_groups object holds this stat object's metrics registered
// as long as the stats object is alive.
seastar::metrics::metric_groups _metrics;
};
private:
data_dictionary::database _db;
service::storage_proxy& _proxy;
gms::gossiper& _gossiper;
// _end is set by start(), and resolves when the the background service
// started by it ends. To ask the background service to end, _abort_source
// should be triggered. stop() below uses both _abort_source and _end.
std::optional<future<>> _end;
abort_source _abort_source;
// Ensures that at most 1 page of scan results at a time is processed by the TTL service
named_semaphore _page_sem{1, named_semaphore_exception_factory{"alternator_ttl"}};
bool shutting_down() { return _abort_source.abort_requested(); }
stats _expiration_stats;
public:
// sharded_service<expiration_service>::start() creates this object on
// all shards, so calls this constructor on each shard. Later, the
// additional start() function should be invoked on all shards.
expiration_service(data_dictionary::database, service::storage_proxy&, gms::gossiper&);
future<> start();
future<> run();
// sharded_service<expiration_service>::stop() calls the following stop()
// method on each shard. This stop() asks the service on this shard to
// shut down as quickly as it can. The returned future indicates when the
// service is no longer running.
// stop() may be called even before start(), but may only be called once -
// calling it twice will result in an exception.
future<> stop();
};
} // namespace alternator

View File

@@ -1,15 +0,0 @@
version: 1
applications:
- frontend:
phases:
build:
commands:
- make setupenv
- make dirhtml
artifacts:
baseDirectory: _build/dirhtml
files:
- '**/*'
cache:
paths: []
appRoot: docs

View File

@@ -1,70 +0,0 @@
# Generate C++ sources from Swagger definitions
set(swagger_files
api-doc/authorization_cache.json
api-doc/cache_service.json
api-doc/collectd.json
api-doc/column_family.json
api-doc/commitlog.json
api-doc/compaction_manager.json
api-doc/config.json
api-doc/endpoint_snitch_info.json
api-doc/error_injection.json
api-doc/failure_detector.json
api-doc/gossiper.json
api-doc/hinted_handoff.json
api-doc/lsa.json
api-doc/messaging_service.json
api-doc/storage_proxy.json
api-doc/storage_service.json
api-doc/stream_manager.json
api-doc/system.json
api-doc/task_manager.json
api-doc/task_manager_test.json
api-doc/utils.json)
foreach(f ${swagger_files})
get_filename_component(fname "${f}" NAME_WE)
get_filename_component(dir "${f}" DIRECTORY)
seastar_generate_swagger(
TARGET scylla_swagger_gen_${fname}
VAR scylla_swagger_gen_${fname}_files
IN_FILE "${CMAKE_CURRENT_SOURCE_DIR}/${f}"
OUT_DIR "${scylla_gen_build_dir}/api/${dir}")
list(APPEND swagger_gen_files "${scylla_swagger_gen_${fname}_files}")
endforeach()
add_library(api)
target_sources(api
PRIVATE
api.cc
cache_service.cc
collectd.cc
column_family.cc
commitlog.cc
compaction_manager.cc
config.cc
endpoint_snitch.cc
error_injection.cc
authorization_cache.cc
failure_detector.cc
gossiper.cc
hinted_handoff.cc
lsa.cc
messaging_service.cc
storage_proxy.cc
storage_service.cc
stream_manager.cc
system.cc
task_manager.cc
task_manager_test.cc
${swagger_gen_files})
target_include_directories(api
PUBLIC
${CMAKE_SOURCE_DIR}
${scylla_gen_build_dir})
target_link_libraries(api
idl
wasmtime_bindings
Seastar::seastar
xxHash::xxhash)

View File

@@ -1,29 +0,0 @@
{
"apiVersion":"0.0.1",
"swaggerVersion":"1.2",
"basePath":"{{Protocol}}://{{Host}}",
"resourcePath":"/authorization_cache",
"produces":[
"application/json"
],
"apis":[
{
"path":"/authorization_cache/reset",
"operations":[
{
"method":"POST",
"summary":"Reset cache",
"type":"void",
"nickname":"authorization_cache_reset",
"produces":[
"application/json"
],
"parameters":[
]
}
]
}
],
"models":{
}
}

View File

@@ -13,7 +13,7 @@
{
"method":"GET",
"summary":"get row cache save period in seconds",
"type": "long",
"type":"int",
"nickname":"get_row_cache_save_period_in_seconds",
"produces":[
"application/json"
@@ -35,7 +35,7 @@
"description":"row cache save period in seconds",
"required":true,
"allowMultiple":false,
"type": "long",
"type":"int",
"paramType":"query"
}
]
@@ -48,7 +48,7 @@
{
"method":"GET",
"summary":"get key cache save period in seconds",
"type": "long",
"type":"int",
"nickname":"get_key_cache_save_period_in_seconds",
"produces":[
"application/json"
@@ -70,7 +70,7 @@
"description":"key cache save period in seconds",
"required":true,
"allowMultiple":false,
"type": "long",
"type":"int",
"paramType":"query"
}
]
@@ -83,7 +83,7 @@
{
"method":"GET",
"summary":"get counter cache save period in seconds",
"type": "long",
"type":"int",
"nickname":"get_counter_cache_save_period_in_seconds",
"produces":[
"application/json"
@@ -105,7 +105,7 @@
"description":"counter cache save period in seconds",
"required":true,
"allowMultiple":false,
"type": "long",
"type":"int",
"paramType":"query"
}
]
@@ -118,7 +118,7 @@
{
"method":"GET",
"summary":"get row cache keys to save",
"type": "long",
"type":"int",
"nickname":"get_row_cache_keys_to_save",
"produces":[
"application/json"
@@ -140,7 +140,7 @@
"description":"row cache keys to save",
"required":true,
"allowMultiple":false,
"type": "long",
"type":"int",
"paramType":"query"
}
]
@@ -153,7 +153,7 @@
{
"method":"GET",
"summary":"get key cache keys to save",
"type": "long",
"type":"int",
"nickname":"get_key_cache_keys_to_save",
"produces":[
"application/json"
@@ -175,7 +175,7 @@
"description":"key cache keys to save",
"required":true,
"allowMultiple":false,
"type": "long",
"type":"int",
"paramType":"query"
}
]
@@ -188,7 +188,7 @@
{
"method":"GET",
"summary":"get counter cache keys to save",
"type": "long",
"type":"int",
"nickname":"get_counter_cache_keys_to_save",
"produces":[
"application/json"
@@ -210,7 +210,7 @@
"description":"counter cache keys to save",
"required":true,
"allowMultiple":false,
"type": "long",
"type":"int",
"paramType":"query"
}
]
@@ -448,7 +448,7 @@
{
"method": "GET",
"summary": "Get key entries",
"type": "long",
"type": "int",
"nickname": "get_key_entries",
"produces": [
"application/json"
@@ -568,7 +568,7 @@
{
"method": "GET",
"summary": "Get row entries",
"type": "long",
"type": "int",
"nickname": "get_row_entries",
"produces": [
"application/json"
@@ -688,7 +688,7 @@
{
"method": "GET",
"summary": "Get counter entries",
"type": "long",
"type": "int",
"nickname": "get_counter_entries",
"produces": [
"application/json"

View File

@@ -70,7 +70,7 @@
{
"method":"POST",
"summary":"Force a major compaction of this column family",
"type":"void",
"type":"string",
"nickname":"force_major_compaction",
"produces":[
"application/json"
@@ -89,7 +89,7 @@
"description":"true if the output of the major compaction should be split in several sstables",
"required":false,
"allowMultiple":false,
"type":"boolean",
"type":"bool",
"paramType":"query"
}
]
@@ -121,7 +121,7 @@
"description":"The minimum number of sstables in queue before compaction kicks off",
"required":true,
"allowMultiple":false,
"type": "long",
"type":"int",
"paramType":"query"
}
]
@@ -172,7 +172,7 @@
"description":"The maximum number of sstables in queue before compaction kicks off",
"required":true,
"allowMultiple":false,
"type": "long",
"type":"int",
"paramType":"query"
}
]
@@ -223,7 +223,7 @@
"description":"The maximum number of sstables in queue before compaction kicks off",
"required":true,
"allowMultiple":false,
"type": "long",
"type":"int",
"paramType":"query"
},
{
@@ -231,7 +231,7 @@
"description":"The minimum number of sstables in queue before compaction kicks off",
"required":true,
"allowMultiple":false,
"type": "long",
"type":"int",
"paramType":"query"
}
]
@@ -380,54 +380,16 @@
"operations":[
{
"method":"GET",
"summary":"check if the auto_compaction property is enabled for a given table",
"summary":"check if the auto compaction disabled",
"type":"boolean",
"nickname":"get_auto_compaction",
"nickname":"is_auto_compaction_disabled",
"produces":[
"application/json"
],
"parameters":[
{
"name":"name",
"description":"The table name in keyspace:name format",
"required":true,
"allowMultiple":false,
"type":"string",
"paramType":"path"
}
]
},
{
"method":"POST",
"summary":"Enable table auto compaction",
"type":"void",
"nickname":"enable_auto_compaction",
"produces":[
"application/json"
],
"parameters":[
{
"name":"name",
"description":"The table name in keyspace:name format",
"required":true,
"allowMultiple":false,
"type":"string",
"paramType":"path"
}
]
},
{
"method":"DELETE",
"summary":"Disable table auto compaction",
"type":"void",
"nickname":"disable_auto_compaction",
"produces":[
"application/json"
],
"parameters":[
{
"name":"name",
"description":"The table name in keyspace:name format",
"description":"The column family name in keyspace:name format",
"required":true,
"allowMultiple":false,
"type":"string",
@@ -582,7 +544,7 @@
"summary":"sstable count for each level. empty unless leveled compaction is used",
"type":"array",
"items":{
"type": "long"
"type":"int"
},
"nickname":"get_sstable_count_per_level",
"produces":[
@@ -649,54 +611,6 @@
}
]
},
{
"path":"/column_family/toppartitions/{name}",
"operations":[
{
"method":"GET",
"summary":"Toppartitions query",
"type":"toppartitions_query_results",
"nickname":"toppartitions",
"produces":[
"application/json"
],
"parameters":[
{
"name":"name",
"description":"The column family name in keyspace:name format",
"required":true,
"allowMultiple":false,
"type":"string",
"paramType":"path"
},
{
"name":"duration",
"description":"Duration (in milliseconds) of monitoring operation",
"required":true,
"allowMultiple":false,
"type": "long",
"paramType":"query"
},
{
"name":"list_size",
"description":"number of the top partitions to list",
"required":false,
"allowMultiple":false,
"type": "long",
"paramType":"query"
},
{
"name":"capacity",
"description":"capacity of stream summary: determines amount of resources used in query processing",
"required":false,
"allowMultiple":false,
"type": "long",
"paramType":"query"
}
]
}
]
},
{
"path":"/column_family/metrics/memtable_columns_count/",
"operations":[
@@ -959,7 +873,7 @@
{
"method":"GET",
"summary":"Get memtable switch count",
"type": "long",
"type":"int",
"nickname":"get_memtable_switch_count",
"produces":[
"application/json"
@@ -983,7 +897,7 @@
{
"method":"GET",
"summary":"Get all memtable switch count",
"type": "long",
"type":"int",
"nickname":"get_all_memtable_switch_count",
"produces":[
"application/json"
@@ -1120,7 +1034,7 @@
{
"method":"GET",
"summary":"Get read latency",
"type": "long",
"type":"int",
"nickname":"get_read_latency",
"produces":[
"application/json"
@@ -1273,7 +1187,7 @@
{
"method":"GET",
"summary":"Get all read latency",
"type": "long",
"type":"int",
"nickname":"get_all_read_latency",
"produces":[
"application/json"
@@ -1289,7 +1203,7 @@
{
"method":"GET",
"summary":"Get range latency",
"type": "long",
"type":"int",
"nickname":"get_range_latency",
"produces":[
"application/json"
@@ -1313,7 +1227,7 @@
{
"method":"GET",
"summary":"Get all range latency",
"type": "long",
"type":"int",
"nickname":"get_all_range_latency",
"produces":[
"application/json"
@@ -1329,7 +1243,7 @@
{
"method":"GET",
"summary":"Get write latency",
"type": "long",
"type":"int",
"nickname":"get_write_latency",
"produces":[
"application/json"
@@ -1482,7 +1396,7 @@
{
"method":"GET",
"summary":"Get all write latency",
"type": "long",
"type":"int",
"nickname":"get_all_write_latency",
"produces":[
"application/json"
@@ -1498,7 +1412,7 @@
{
"method":"GET",
"summary":"Get pending flushes",
"type": "long",
"type":"int",
"nickname":"get_pending_flushes",
"produces":[
"application/json"
@@ -1522,7 +1436,7 @@
{
"method":"GET",
"summary":"Get all pending flushes",
"type": "long",
"type":"int",
"nickname":"get_all_pending_flushes",
"produces":[
"application/json"
@@ -1538,7 +1452,7 @@
{
"method":"GET",
"summary":"Get pending compactions",
"type": "long",
"type":"int",
"nickname":"get_pending_compactions",
"produces":[
"application/json"
@@ -1562,7 +1476,7 @@
{
"method":"GET",
"summary":"Get all pending compactions",
"type": "long",
"type":"int",
"nickname":"get_all_pending_compactions",
"produces":[
"application/json"
@@ -1578,7 +1492,7 @@
{
"method":"GET",
"summary":"Get live ss table count",
"type": "long",
"type":"int",
"nickname":"get_live_ss_table_count",
"produces":[
"application/json"
@@ -1602,7 +1516,7 @@
{
"method":"GET",
"summary":"Get all live ss table count",
"type": "long",
"type":"int",
"nickname":"get_all_live_ss_table_count",
"produces":[
"application/json"
@@ -1618,7 +1532,7 @@
{
"method":"GET",
"summary":"Get live disk space used",
"type": "long",
"type":"int",
"nickname":"get_live_disk_space_used",
"produces":[
"application/json"
@@ -1642,7 +1556,7 @@
{
"method":"GET",
"summary":"Get all live disk space used",
"type": "long",
"type":"int",
"nickname":"get_all_live_disk_space_used",
"produces":[
"application/json"
@@ -1658,7 +1572,7 @@
{
"method":"GET",
"summary":"Get total disk space used",
"type": "long",
"type":"int",
"nickname":"get_total_disk_space_used",
"produces":[
"application/json"
@@ -1682,7 +1596,7 @@
{
"method":"GET",
"summary":"Get all total disk space used",
"type": "long",
"type":"int",
"nickname":"get_all_total_disk_space_used",
"produces":[
"application/json"
@@ -2138,7 +2052,7 @@
{
"method":"GET",
"summary":"Get speculative retries",
"type": "long",
"type":"int",
"nickname":"get_speculative_retries",
"produces":[
"application/json"
@@ -2162,7 +2076,7 @@
{
"method":"GET",
"summary":"Get all speculative retries",
"type": "long",
"type":"int",
"nickname":"get_all_speculative_retries",
"produces":[
"application/json"
@@ -2242,7 +2156,7 @@
{
"method":"GET",
"summary":"Get row cache hit out of range",
"type": "long",
"type":"int",
"nickname":"get_row_cache_hit_out_of_range",
"produces":[
"application/json"
@@ -2266,7 +2180,7 @@
{
"method":"GET",
"summary":"Get all row cache hit out of range",
"type": "long",
"type":"int",
"nickname":"get_all_row_cache_hit_out_of_range",
"produces":[
"application/json"
@@ -2282,7 +2196,7 @@
{
"method":"GET",
"summary":"Get row cache hit",
"type": "long",
"type":"int",
"nickname":"get_row_cache_hit",
"produces":[
"application/json"
@@ -2306,7 +2220,7 @@
{
"method":"GET",
"summary":"Get all row cache hit",
"type": "long",
"type":"int",
"nickname":"get_all_row_cache_hit",
"produces":[
"application/json"
@@ -2322,7 +2236,7 @@
{
"method":"GET",
"summary":"Get row cache miss",
"type": "long",
"type":"int",
"nickname":"get_row_cache_miss",
"produces":[
"application/json"
@@ -2346,7 +2260,7 @@
{
"method":"GET",
"summary":"Get all row cache miss",
"type": "long",
"type":"int",
"nickname":"get_all_row_cache_miss",
"produces":[
"application/json"
@@ -2362,7 +2276,7 @@
{
"method":"GET",
"summary":"Get cas prepare",
"type": "long",
"type":"int",
"nickname":"get_cas_prepare",
"produces":[
"application/json"
@@ -2386,7 +2300,7 @@
{
"method":"GET",
"summary":"Get cas propose",
"type": "long",
"type":"int",
"nickname":"get_cas_propose",
"produces":[
"application/json"
@@ -2410,7 +2324,7 @@
{
"method":"GET",
"summary":"Get cas commit",
"type": "long",
"type":"int",
"nickname":"get_cas_commit",
"produces":[
"application/json"
@@ -2902,52 +2816,6 @@
"description":"The column family type"
}
}
},
"toppartitions_record":{
"id":"toppartitions_record",
"description":"nodetool toppartitions query record",
"properties":{
"partition":{
"type":"string",
"description":"Partition key"
},
"count":{
"type":"long",
"description":"Number of read/write operations"
},
"error":{
"type":"long",
"description":"Indication of inaccuracy in counting PKs"
}
}
},
"toppartitions_query_results":{
"id":"toppartitions_query_results",
"description":"nodetool toppartitions query results",
"properties":{
"read_cardinality":{
"type":"long",
"description":"Number of the unique operations in the sample set"
},
"read":{
"type":"array",
"items":{
"type":"toppartitions_record"
},
"description":"Read results"
},
"write_cardinality":{
"type":"long",
"description":"Number of the unique operations in the sample set"
},
"write":{
"type":"array",
"items":{
"type":"toppartitions_record"
},
"description":"Write results"
}
}
}
}
}

View File

@@ -102,47 +102,7 @@
"parameters":[
{
"name":"type",
"description":"The type of compaction to stop. Can be one of: COMPACTION | CLEANUP | SCRUB | UPGRADE | RESHAPE",
"required":true,
"allowMultiple":false,
"type":"string",
"paramType":"query"
}
]
}
]
},
{
"path":"/compaction_manager/stop_keyspace_compaction/{keyspace}",
"operations":[
{
"method":"POST",
"summary":"Stop all running compaction-like tasks in the given keyspace and tables having the provided type.",
"type":"void",
"nickname":"stop_keyspace_compaction",
"produces":[
"application/json"
],
"parameters":[
{
"name":"keyspace",
"description":"The keyspace to stop compaction in",
"required":true,
"allowMultiple":false,
"type":"string",
"paramType":"path"
},
{
"name":"tables",
"description":"Comma-separated tables to stop compaction in",
"required":false,
"allowMultiple":false,
"type":"string",
"paramType":"query"
},
{
"name":"type",
"description":"The type of compaction to stop. Can be one of: COMPACTION | CLEANUP | SCRUB | UPGRADE | RESHAPE",
"description":"the type of compaction to stop. Can be one of: - COMPACTION - VALIDATION - CLEANUP - SCRUB - INDEX_BUILD",
"required":true,
"allowMultiple":false,
"type":"string",
@@ -158,7 +118,7 @@
{
"method": "GET",
"summary": "Get pending tasks",
"type": "long",
"type": "int",
"nickname": "get_pending_tasks",
"produces": [
"application/json"
@@ -167,24 +127,6 @@
}
]
},
{
"path": "/compaction_manager/metrics/pending_tasks_by_table",
"operations": [
{
"method": "GET",
"summary": "Get pending tasks by table name",
"type": "array",
"items": {
"type": "pending_compaction"
},
"nickname": "get_pending_tasks_by_table",
"produces": [
"application/json"
],
"parameters": []
}
]
},
{
"path": "/compaction_manager/metrics/completed_tasks",
"operations": [
@@ -221,7 +163,7 @@
{
"method": "GET",
"summary": "Get bytes compacted",
"type": "long",
"type": "int",
"nickname": "get_bytes_compacted",
"produces": [
"application/json"
@@ -237,7 +179,7 @@
"description":"A row merged information",
"properties":{
"key":{
"type": "long",
"type":"int",
"description":"The number of sstable"
},
"value":{
@@ -302,23 +244,6 @@
}
}
},
"pending_compaction": {
"id": "pending_compaction",
"properties": {
"cf": {
"type": "string",
"description": "The column family name"
},
"ks": {
"type":"string",
"description": "The keyspace name"
},
"task": {
"type":"long",
"description": "The number of pending tasks"
}
}
},
"history": {
"id":"history",
"description":"Compaction history information",

View File

@@ -1,90 +0,0 @@
{
"apiVersion":"0.0.1",
"swaggerVersion":"1.2",
"basePath":"{{Protocol}}://{{Host}}",
"resourcePath":"/error_injection",
"produces":[
"application/json"
],
"apis":[
{
"path":"/v2/error_injection/injection/{injection}",
"operations":[
{
"method":"POST",
"summary":"Activate an injection that triggers an error in code",
"type":"void",
"nickname":"enable_injection",
"produces":[
"application/json"
],
"parameters":[
{
"name":"injection",
"description":"injection name, should correspond to an injection added in code",
"required":true,
"allowMultiple":false,
"type":"string",
"paramType":"path"
},
{
"name":"one_shot",
"description":"boolean flag indicating whether the injection should be enabled to trigger only once",
"required":false,
"allowMultiple":false,
"type":"boolean",
"paramType":"query"
}
]
},
{
"method":"DELETE",
"summary":"Deactivate an injection previously activated by the API",
"type":"void",
"nickname":"disable_injection",
"produces":[
"application/json"
],
"parameters":[
{
"name":"injection",
"description":"injection name",
"required":true,
"allowMultiple":false,
"type":"string",
"paramType":"path"
}
]
}
]
},
{
"path":"/v2/error_injection/injection",
"operations":[
{
"method":"GET",
"summary":"List all enabled injections on all shards, i.e. injections that will trigger an error in the code",
"type":"array",
"items":{
"type":"string"
},
"nickname":"get_enabled_injections_on_all",
"produces":[
"application/json"
],
"parameters":[]
},
{
"method":"DELETE",
"summary":"Deactivate all injections previously activated on all shards by the API",
"type":"void",
"nickname":"disable_on_all",
"produces":[
"application/json"
],
"parameters":[]
}
]
}
]
}

View File

@@ -110,7 +110,7 @@
{
"method":"GET",
"summary":"Get count down endpoint",
"type": "long",
"type":"int",
"nickname":"get_down_endpoint_count",
"produces":[
"application/json"
@@ -126,7 +126,7 @@
{
"method":"GET",
"summary":"Get count up endpoint",
"type": "long",
"type":"int",
"nickname":"get_up_endpoint_count",
"produces":[
"application/json"
@@ -180,11 +180,11 @@
"description": "The endpoint address"
},
"generation": {
"type": "long",
"type": "int",
"description": "The heart beat generation"
},
"version": {
"type": "long",
"type": "int",
"description": "The heart beat version"
},
"update_time": {
@@ -209,7 +209,7 @@
"description": "Holds a version value for an application state",
"properties": {
"application_state": {
"type": "long",
"type": "int",
"description": "The application state enum index"
},
"value": {
@@ -217,7 +217,7 @@
"description": "The version value"
},
"version": {
"type": "long",
"type": "int",
"description": "The application state version"
}
}

View File

@@ -75,7 +75,7 @@
{
"method":"GET",
"summary":"Returns files which are pending for archival attempt. Does NOT include failed archive attempts",
"type": "long",
"type":"int",
"nickname":"get_current_generation_number",
"produces":[
"application/json"
@@ -99,7 +99,7 @@
{
"method":"GET",
"summary":"Get heart beat version for a node",
"type": "long",
"type":"int",
"nickname":"get_current_heart_beat_version",
"produces":[
"application/json"
@@ -148,30 +148,6 @@
]
}
]
},
{
"path":"/gossiper/force_remove_endpoint/{addr}",
"operations":[
{
"method":"POST",
"summary":"Force remove an endpoint from gossip",
"type":"void",
"nickname":"force_remove_endpoint",
"produces":[
"application/json"
],
"parameters":[
{
"name":"addr",
"description":"The endpoint address",
"required":true,
"allowMultiple":false,
"type":"string",
"paramType":"path"
}
]
}
]
}
]
}

View File

@@ -7,61 +7,6 @@
"application/json"
],
"apis":[
{
"path":"/hinted_handoff/sync_point",
"operations":[
{
"method":"POST",
"summary":"Creates a hints sync point. It can be used to wait until hints between given nodes are replayed. A sync point allows you to wait for hints accumulated at the moment of its creation - it won't wait for hints generated later. A sync point is described entirely by its ID - there is no state kept server-side, so there is no need to delete it.",
"type":"string",
"nickname":"create_hints_sync_point",
"produces":[
"application/json"
],
"parameters":[
{
"name":"target_hosts",
"description":"A list of nodes towards which hints should be replayed. Multiple hosts can be listed by separating them with commas. If not provided or empty, the point will resolve when current hints towards all nodes in the cluster are sent.",
"required":false,
"allowMultiple":false,
"type":"string",
"paramType":"query"
}
]
},
{
"method":"GET",
"summary":"Get the status of a hints sync point, possibly waiting for it to be reached.",
"type":"string",
"enum":[
"DONE",
"IN_PROGRESS"
],
"nickname":"get_hints_sync_point",
"produces":[
"application/json"
],
"parameters":[
{
"name":"id",
"description":"The ID of the hint sync point which should be checked or waited on",
"required":true,
"allowMultiple":false,
"type":"string",
"paramType":"query"
},
{
"name":"timeout",
"description":"Timeout in seconds after which the query returns even if hints are still being replayed. No value or 0 will cause the query to return immediately. A negative value will cause the query to wait until the sync point is reached",
"required":false,
"allowMultiple":false,
"type":"long",
"paramType":"query"
}
]
}
]
},
{
"path":"/hinted_handoff/hints",
"operations":[
@@ -154,7 +99,7 @@
{
"method": "GET",
"summary": "Get create hint count",
"type": "long",
"type": "int",
"nickname": "get_create_hint_count",
"produces": [
"application/json"
@@ -178,7 +123,7 @@
{
"method": "GET",
"summary": "Get not stored hints count",
"type": "long",
"type": "int",
"nickname": "get_not_stored_hints_count",
"produces": [
"application/json"

View File

@@ -76,7 +76,7 @@
"items":{
"type":"message_counter"
},
"nickname":"get_replied_messages",
"nickname":"get_completed_messages",
"produces":[
"application/json"
],
@@ -191,7 +191,7 @@
{
"method":"GET",
"summary":"Get the version number",
"type": "long",
"type":"int",
"nickname":"get_version",
"produces":[
"application/json"
@@ -249,10 +249,10 @@
"MIGRATION_REQUEST",
"PREPARE_MESSAGE",
"PREPARE_DONE_MESSAGE",
"UNUSED__STREAM_MUTATION",
"STREAM_MUTATION",
"STREAM_MUTATION_DONE",
"COMPLETE_MESSAGE",
"UNUSED__REPAIR_CHECKSUM_RANGE",
"REPAIR_CHECKSUM_RANGE",
"GET_SCHEMA_VERSION"
]
}

View File

@@ -68,7 +68,7 @@
"summary":"Get the hinted handoff enabled by dc",
"type":"array",
"items":{
"type":"array"
"type":"mapper_list"
},
"nickname":"get_hinted_handoff_enabled_by_dc",
"produces":[
@@ -105,7 +105,7 @@
{
"method":"GET",
"summary":"Get the max hint window",
"type": "long",
"type":"int",
"nickname":"get_max_hint_window",
"produces":[
"application/json"
@@ -128,7 +128,7 @@
"description":"max hint window in ms",
"required":true,
"allowMultiple":false,
"type": "long",
"type":"int",
"paramType":"query"
}
]
@@ -141,7 +141,7 @@
{
"method":"GET",
"summary":"Get max hints in progress",
"type": "long",
"type":"int",
"nickname":"get_max_hints_in_progress",
"produces":[
"application/json"
@@ -164,7 +164,7 @@
"description":"max hints in progress",
"required":true,
"allowMultiple":false,
"type": "long",
"type":"int",
"paramType":"query"
}
]
@@ -177,7 +177,7 @@
{
"method":"GET",
"summary":"get hints in progress",
"type": "long",
"type":"int",
"nickname":"get_hints_in_progress",
"produces":[
"application/json"
@@ -602,7 +602,7 @@
{
"method": "GET",
"summary": "Get cas write metrics",
"type": "long",
"type": "int",
"nickname": "get_cas_write_metrics_unfinished_commit",
"produces": [
"application/json"
@@ -632,7 +632,7 @@
{
"method": "GET",
"summary": "Get cas write metrics",
"type": "long",
"type": "int",
"nickname": "get_cas_write_metrics_condition_not_met",
"produces": [
"application/json"
@@ -641,28 +641,13 @@
}
]
},
{
"path": "/storage_proxy/metrics/cas_write/failed_read_round_optimization",
"operations": [
{
"method": "GET",
"summary": "Get cas write metrics",
"type": "long",
"nickname": "get_cas_write_metrics_failed_read_round_optimization",
"produces": [
"application/json"
],
"parameters": []
}
]
},
{
"path": "/storage_proxy/metrics/cas_read/unfinished_commit",
"operations": [
{
"method": "GET",
"summary": "Get cas read metrics",
"type": "long",
"type": "int",
"nickname": "get_cas_read_metrics_unfinished_commit",
"produces": [
"application/json"
@@ -686,13 +671,28 @@
}
]
},
{
"path": "/storage_proxy/metrics/cas_read/condition_not_met",
"operations": [
{
"method": "GET",
"summary": "Get cas read metrics",
"type": "int",
"nickname": "get_cas_read_metrics_condition_not_met",
"produces": [
"application/json"
],
"parameters": []
}
]
},
{
"path": "/storage_proxy/metrics/read/timeouts",
"operations": [
{
"method": "GET",
"summary": "Get read metrics",
"type": "long",
"type": "int",
"nickname": "get_read_metrics_timeouts",
"produces": [
"application/json"
@@ -707,7 +707,7 @@
{
"method": "GET",
"summary": "Get read metrics",
"type": "long",
"type": "int",
"nickname": "get_read_metrics_unavailables",
"produces": [
"application/json"
@@ -791,36 +791,6 @@
}
]
},
{
"path": "/storage_proxy/metrics/cas_read/moving_average_histogram",
"operations": [
{
"method": "GET",
"summary": "Get CAS read rate and latency histogram",
"$ref": "#/utils/rate_moving_average_and_histogram",
"nickname": "get_cas_read_metrics_latency_histogram",
"produces": [
"application/json"
],
"parameters": []
}
]
},
{
"path": "/storage_proxy/metrics/view_write/moving_average_histogram",
"operations": [
{
"method": "GET",
"summary": "Get view write rate and latency histogram",
"$ref": "#/utils/rate_moving_average_and_histogram",
"nickname": "get_view_write_metrics_latency_histogram",
"produces": [
"application/json"
],
"parameters": []
}
]
},
{
"path": "/storage_proxy/metrics/range/moving_average_histogram",
"operations": [
@@ -842,7 +812,7 @@
{
"method": "GET",
"summary": "Get range metrics",
"type": "long",
"type": "int",
"nickname": "get_range_metrics_timeouts",
"produces": [
"application/json"
@@ -857,7 +827,7 @@
{
"method": "GET",
"summary": "Get range metrics",
"type": "long",
"type": "int",
"nickname": "get_range_metrics_unavailables",
"produces": [
"application/json"
@@ -902,7 +872,7 @@
{
"method": "GET",
"summary": "Get write metrics",
"type": "long",
"type": "int",
"nickname": "get_write_metrics_timeouts",
"produces": [
"application/json"
@@ -917,7 +887,7 @@
{
"method": "GET",
"summary": "Get write metrics",
"type": "long",
"type": "int",
"nickname": "get_write_metrics_unavailables",
"produces": [
"application/json"
@@ -986,21 +956,6 @@
}
]
},
{
"path": "/storage_proxy/metrics/cas_write/moving_average_histogram",
"operations": [
{
"method": "GET",
"summary": "Get CAS write rate and latency histogram",
"$ref": "#/utils/rate_moving_average_and_histogram",
"nickname": "get_cas_write_metrics_latency_histogram",
"produces": [
"application/json"
],
"parameters": []
}
]
},
{
"path":"/storage_proxy/metrics/read/estimated_histogram/",
"operations":[
@@ -1023,7 +978,7 @@
{
"method":"GET",
"summary":"Get read latency",
"type": "long",
"type":"int",
"nickname":"get_read_latency",
"produces":[
"application/json"
@@ -1055,7 +1010,7 @@
{
"method":"GET",
"summary":"Get write latency",
"type": "long",
"type":"int",
"nickname":"get_write_latency",
"produces":[
"application/json"
@@ -1087,7 +1042,7 @@
{
"method":"GET",
"summary":"Get range latency",
"type": "long",
"type":"int",
"nickname":"get_range_latency",
"produces":[
"application/json"

View File

@@ -104,68 +104,6 @@
}
]
},
{
"path":"/storage_service/toppartitions/",
"operations":[
{
"method":"GET",
"summary":"Toppartitions query",
"type":"toppartitions_query_results",
"nickname":"toppartitions_generic",
"produces":[
"application/json"
],
"parameters":[
{
"name":"table_filters",
"description":"Optional list of table name filters in keyspace:name format",
"required":false,
"allowMultiple":false,
"type":"array",
"items":{
"type":"string"
},
"paramType":"query"
},
{
"name":"keyspace_filters",
"description":"Optional list of keyspace filters",
"required":false,
"allowMultiple":false,
"type":"array",
"items":{
"type":"string"
},
"paramType":"query"
},
{
"name":"duration",
"description":"Duration (in milliseconds) of monitoring operation",
"required":true,
"allowMultiple":false,
"type": "long",
"paramType":"query"
},
{
"name":"list_size",
"description":"number of the top partitions to list",
"required":false,
"allowMultiple":false,
"type": "long",
"paramType":"query"
},
{
"name":"capacity",
"description":"capacity of stream summary: determines amount of resources used in query processing",
"required":false,
"allowMultiple":false,
"type": "long",
"paramType":"query"
}
]
}
]
},
{
"path":"/storage_service/nodes/leaving",
"operations":[
@@ -520,7 +458,7 @@
{
"method":"GET",
"summary":"Return the generation value for this node.",
"type": "long",
"type":"int",
"nickname":"get_current_generation_number",
"produces":[
"application/json"
@@ -573,21 +511,6 @@
}
]
},
{
"path":"/storage_service/cdc_streams_check_and_repair",
"operations":[
{
"method":"POST",
"summary":"Checks that CDC streams reflect current cluster topology and regenerates them if not.",
"type":"void",
"nickname":"cdc_streams_check_and_repair",
"produces":[
"application/json"
],
"parameters":[]
}
]
},
{
"path":"/storage_service/snapshots",
"operations":[
@@ -624,7 +547,7 @@
},
{
"name":"kn",
"description":"Keyspace(s) to snapshot. Multiple keyspaces can be provided using a comma-separated list. If omitted, snapshot all keyspaces.",
"description":"Comma seperated keyspaces name to snapshot",
"required":false,
"allowMultiple":false,
"type":"string",
@@ -632,19 +555,11 @@
},
{
"name":"cf",
"description":"Table(s) to snapshot. Multiple tables (in a single keyspace) can be provided using a comma-separated list. If omitted, snapshot all tables in the given keyspace(s).",
"description":"the column family to snapshot",
"required":false,
"allowMultiple":false,
"type":"string",
"paramType":"query"
},
{
"name":"sf",
"description":"Skip flush. When set to \"true\", do not flush memtables before snapshotting (snapshot will not contain unflushed data)",
"required":false,
"allowMultiple":false,
"type":"boolean",
"paramType":"query"
}
]
},
@@ -667,15 +582,7 @@
},
{
"name":"kn",
"description":"Comma-separated keyspaces name that their snapshot will be deleted",
"required":false,
"allowMultiple":false,
"type":"string",
"paramType":"query"
},
{
"name":"cf",
"description":"an optional table name that its snapshot will be deleted",
"description":"Comma seperated keyspaces name to snapshot",
"required":false,
"allowMultiple":false,
"type":"string",
@@ -723,7 +630,7 @@
},
{
"name":"cf",
"description":"Comma-separated column family names",
"description":"Comma seperated column family names",
"required":false,
"allowMultiple":false,
"type":"string",
@@ -739,7 +646,7 @@
{
"method":"POST",
"summary":"Trigger a cleanup of keys on a single keyspace",
"type": "long",
"type":"int",
"nickname":"force_keyspace_cleanup",
"produces":[
"application/json"
@@ -755,39 +662,7 @@
},
{
"name":"cf",
"description":"Comma-separated column family names",
"required":false,
"allowMultiple":false,
"type":"string",
"paramType":"query"
}
]
}
]
},
{
"path":"/storage_service/keyspace_offstrategy_compaction/{keyspace}",
"operations":[
{
"method":"POST",
"summary":"Perform offstrategy compaction, if needed, in a single keyspace",
"type":"boolean",
"nickname":"perform_keyspace_offstrategy_compaction",
"produces":[
"application/json"
],
"parameters":[
{
"name":"keyspace",
"description":"The keyspace to operate on",
"required":true,
"allowMultiple":false,
"type":"string",
"paramType":"path"
},
{
"name":"cf",
"description":"Comma-separated table names",
"description":"Comma seperated column family names",
"required":false,
"allowMultiple":false,
"type":"string",
@@ -802,8 +677,8 @@
"operations":[
{
"method":"GET",
"summary":"Scrub (deserialize + reserialize at the latest version, resolving corruptions if any) the given keyspace. If columnFamilies array is empty, all CFs are scrubbed. Scrubbed CFs will be snapshotted first, if disableSnapshot is false. Scrub has the following modes: Abort (default) - abort scrub if corruption is detected; Skip (same as `skip_corrupted=true`) skip over corrupt data, omitting them from the output; Segregate - segregate data into multiple sstables if needed, such that each sstable contains data with valid order; Validate - read (no rewrite) and validate data, logging any problems found.",
"type": "long",
"summary":"Scrub (deserialize + reserialize at the latest version, skipping bad rows if any) the given keyspace. If columnFamilies array is empty, all CFs are scrubbed. Scrubbed CFs will be snapshotted first, if disableSnapshot is false",
"type":"int",
"nickname":"scrub",
"produces":[
"application/json"
@@ -825,33 +700,6 @@
"type":"boolean",
"paramType":"query"
},
{
"name":"scrub_mode",
"description":"How to handle corrupt data (overrides 'skip_corrupted'); ",
"required":false,
"allowMultiple":false,
"type":"string",
"enum":[
"ABORT",
"SKIP",
"SEGREGATE",
"VALIDATE"
],
"paramType":"query"
},
{
"name":"quarantine_mode",
"description":"Controls whether to scrub quarantined sstables (default INCLUDE)",
"required":false,
"allowMultiple":false,
"type":"string",
"enum":[
"INCLUDE",
"EXCLUDE",
"ONLY"
],
"paramType":"query"
},
{
"name":"keyspace",
"description":"The keyspace to query about",
@@ -862,7 +710,7 @@
},
{
"name":"cf",
"description":"Comma-separated column family names",
"description":"Comma seperated column family names",
"required":false,
"allowMultiple":false,
"type":"string",
@@ -878,7 +726,7 @@
{
"method":"GET",
"summary":"Rewrite all sstables to the latest version. Unlike scrub, it doesn't skip bad rows and do not snapshot sstables first.",
"type": "long",
"type":"int",
"nickname":"upgrade_sstables",
"produces":[
"application/json"
@@ -902,7 +750,7 @@
},
{
"name":"cf",
"description":"Comma-separated column family names",
"description":"Comma seperated column family names",
"required":false,
"allowMultiple":false,
"type":"string",
@@ -934,7 +782,7 @@
},
{
"name":"cf",
"description":"Comma-separated column family names",
"description":"Comma seperated column family names",
"required":false,
"allowMultiple":false,
"type":"string",
@@ -952,7 +800,7 @@
"summary":"Return an array with the ids of the currently active repairs",
"type":"array",
"items":{
"type": "long"
"type":"int"
},
"nickname":"get_active_repair_async",
"produces":[
@@ -962,50 +810,13 @@
}
]
},
{
"path":"/storage_service/repair_status/",
"operations":[
{
"method":"GET",
"summary":"Query the repair status and return when the repair is finished or timeout",
"type":"string",
"enum":[
"RUNNING",
"SUCCESSFUL",
"FAILED"
],
"nickname":"repair_await_completion",
"produces":[
"application/json"
],
"parameters":[
{
"name":"id",
"description":"The repair ID to check for status",
"required":true,
"allowMultiple":false,
"type": "long",
"paramType":"query"
},
{
"name":"timeout",
"description":"Seconds to wait before the query returns even if the repair is not finished. The value -1 or not providing this parameter means no timeout",
"required":false,
"allowMultiple":false,
"type": "long",
"paramType":"query"
}
]
}
]
},
{
"path":"/storage_service/repair_async/{keyspace}",
"operations":[
{
"method":"POST",
"summary":"Invoke repair asynchronously. You can track repair progress by using the get supplying id",
"type": "long",
"type":"int",
"nickname":"repair_async",
"produces":[
"application/json"
@@ -1099,14 +910,6 @@
"type":"string",
"paramType":"query"
},
{
"name":"ignore_nodes",
"description":"Which hosts are to ignore in this repair. Multiple hosts can be listed separated by commas.",
"required":false,
"allowMultiple":false,
"type":"string",
"paramType":"query"
},
{
"name":"trace",
"description":"If the value is the string 'true' with any capitalization, enable tracing of the repair.",
@@ -1144,7 +947,7 @@
"description":"The repair ID to check for status",
"required":true,
"allowMultiple":false,
"type": "long",
"type":"int",
"paramType":"query"
}
]
@@ -1228,7 +1031,7 @@
"operations":[
{
"method":"POST",
"summary":"Removes a node from the cluster. Replicated data that logically belonged to this node is redistributed among the remaining nodes.",
"summary":"Removes token (and all data associated with enpoint that had it) from the ring",
"type":"void",
"nickname":"remove_node",
"produces":[
@@ -1242,14 +1045,6 @@
"allowMultiple":false,
"type":"string",
"paramType":"query"
},
{
"name":"ignore_nodes",
"description":"Comma-separated list of dead nodes to ignore in removenode operation. Use the same method for all nodes to ignore: either Host IDs or ip addresses.",
"required":false,
"allowMultiple":false,
"type":"string",
"paramType":"query"
}
]
}
@@ -1482,18 +1277,18 @@
},
{
"name":"dynamic_update_interval",
"description":"interval in ms (default 100)",
"description":"integer, in ms (default 100)",
"required":false,
"allowMultiple":false,
"type":"long",
"type":"integer",
"paramType":"query"
},
{
"name":"dynamic_reset_interval",
"description":"interval in ms (default 600,000)",
"description":"integer, in ms (default 600,000)",
"required":false,
"allowMultiple":false,
"type":"long",
"type":"integer",
"paramType":"query"
},
{
@@ -1698,7 +1493,7 @@
"description":"Stream throughput",
"required":true,
"allowMultiple":false,
"type": "long",
"type":"int",
"paramType":"query"
}
]
@@ -1706,7 +1501,7 @@
{
"method":"GET",
"summary":"Get stream throughput mb per sec",
"type": "long",
"type":"int",
"nickname":"get_stream_throughput_mb_per_sec",
"produces":[
"application/json"
@@ -1722,7 +1517,7 @@
{
"method":"GET",
"summary":"get compaction throughput mb per sec",
"type": "long",
"type":"int",
"nickname":"get_compaction_throughput_mb_per_sec",
"produces":[
"application/json"
@@ -1744,7 +1539,7 @@
"description":"compaction throughput",
"required":true,
"allowMultiple":false,
"type": "long",
"type":"int",
"paramType":"query"
}
]
@@ -1901,22 +1696,6 @@
"allowMultiple":false,
"type":"string",
"paramType":"query"
},
{
"name":"load_and_stream",
"description":"Load the sstables and stream to all replica nodes that owns the data",
"required":false,
"allowMultiple":false,
"type":"string",
"paramType":"query"
},
{
"name":"primary_replica_only",
"description":"Load the sstables and stream to primary replica node that owns the data. Repair is needed after the load and stream process",
"required":false,
"allowMultiple":false,
"type":"string",
"paramType":"query"
}
]
}
@@ -2027,14 +1806,6 @@
"allowMultiple":false,
"type":"long",
"paramType":"query"
},
{
"name":"fast",
"description":"Lightweight tracing mode: if true, slow queries tracing records only session headers",
"required":false,
"allowMultiple":false,
"type":"boolean",
"paramType":"query"
}
]
},
@@ -2073,7 +1844,7 @@
},
{
"name":"cf",
"description":"Comma-separated column family names",
"description":"Comma seperated column family names",
"required":false,
"allowMultiple":false,
"type":"string",
@@ -2100,7 +1871,7 @@
},
{
"name":"cf",
"description":"Comma-separated column family names",
"description":"Comma seperated column family names",
"required":false,
"allowMultiple":false,
"type":"string",
@@ -2172,7 +1943,7 @@
{
"method":"GET",
"summary":"Returns the threshold for warning of queries with many tombstones",
"type": "long",
"type":"int",
"nickname":"get_tombstone_warn_threshold",
"produces":[
"application/json"
@@ -2194,7 +1965,7 @@
"description":"tombstone debug threshold",
"required":true,
"allowMultiple":false,
"type": "long",
"type":"int",
"paramType":"query"
}
]
@@ -2207,7 +1978,7 @@
{
"method":"GET",
"summary":"",
"type": "long",
"type":"int",
"nickname":"get_tombstone_failure_threshold",
"produces":[
"application/json"
@@ -2229,7 +2000,7 @@
"description":"tombstone debug threshold",
"required":true,
"allowMultiple":false,
"type": "long",
"type":"int",
"paramType":"query"
}
]
@@ -2242,7 +2013,7 @@
{
"method":"GET",
"summary":"Returns the threshold for rejecting queries due to a large batch size",
"type": "long",
"type":"int",
"nickname":"get_batch_size_failure_threshold",
"produces":[
"application/json"
@@ -2264,7 +2035,7 @@
"description":"batch size debug threshold",
"required":true,
"allowMultiple":false,
"type": "long",
"type":"int",
"paramType":"query"
}
]
@@ -2288,7 +2059,7 @@
"description":"throttle in kb",
"required":true,
"allowMultiple":false,
"type": "long",
"type":"int",
"paramType":"query"
}
]
@@ -2301,7 +2072,7 @@
{
"method":"GET",
"summary":"Get load",
"type": "long",
"type":"int",
"nickname":"get_metrics_load",
"produces":[
"application/json"
@@ -2317,7 +2088,7 @@
{
"method":"GET",
"summary":"Get exceptions",
"type": "long",
"type":"int",
"nickname":"get_exceptions",
"produces":[
"application/json"
@@ -2333,7 +2104,7 @@
{
"method":"GET",
"summary":"Get total hints in progress",
"type": "long",
"type":"int",
"nickname":"get_total_hints_in_progress",
"produces":[
"application/json"
@@ -2349,7 +2120,7 @@
{
"method":"GET",
"summary":"Get total hints",
"type": "long",
"type":"int",
"nickname":"get_total_hints",
"produces":[
"application/json"
@@ -2393,42 +2164,7 @@
]
}
]
},
{
"path":"/storage_service/sstable_info",
"operations":[
{
"method":"GET",
"summary":"SSTable information",
"type":"array",
"items":{
"type":"table_sstables"
},
"nickname":"sstable_info",
"produces":[
"application/json"
],
"parameters":[
{
"name":"keyspace",
"description":"The keyspace",
"required":false,
"allowMultiple":false,
"type":"string",
"paramType":"query"
},
{
"name":"cf",
"description":"column family name",
"required":false,
"allowMultiple":false,
"type":"string",
"paramType":"query"
}
]
}
]
}
}
],
"models":{
"mapper":{
@@ -2533,10 +2269,6 @@
"threshold":{
"type":"long",
"description":"The slow query logging threshold in microseconds. Queries that takes longer, will be logged"
},
"fast":{
"type":"boolean",
"description":"Is lightweight tracing mode enabled. In that mode tracing ignore events and tracks only sessions."
}
}
},
@@ -2592,92 +2324,6 @@
"description":"The endpoint details"
}
}
},
"named_maps":{
"id":"named_maps",
"properties":{
"group":{
"type":"string"
},
"attributes":{
"type":"array",
"items":{
"type":"mapper"
}
}
}
},
"sstable":{
"id":"sstable",
"properties":{
"size":{
"type":"long",
"description":"Total size in bytes of sstable"
},
"data_size":{
"type":"long",
"description":"The size in bytes on disk of data"
},
"index_size":{
"type":"long",
"description":"The size in bytes on disk of index"
},
"filter_size":{
"type":"long",
"description":"The size in bytes on disk of filter"
},
"timestamp":{
"type":"datetime",
"description":"File creation time"
},
"generation":{
"type":"long",
"description":"SSTable generation"
},
"level":{
"type":"long",
"description":"SSTable level"
},
"version":{
"type":"string",
"enum":[
"ka", "la", "mc", "md", "me"
],
"description":"SSTable version"
},
"properties":{
"type":"array",
"description":"SSTable attributes",
"items":{
"type":"mapper"
}
},
"extended_properties":{
"type":"array",
"description":"SSTable extended attributes",
"items":{
"type":"named_maps"
}
}
}
},
"table_sstables":{
"id":"table_sstables",
"description":"Per-table SSTable info and attributes",
"properties":{
"keyspace":{
"type":"string"
},
"table":{
"type":"string"
},
"sstables":{
"type":"array",
"items":{
"$ref":"sstable"
}
}
}
}
}
}

View File

@@ -32,7 +32,7 @@
{
"method":"GET",
"summary":"Get number of active outbound streams",
"type": "long",
"type":"int",
"nickname":"get_all_active_streams_outbound",
"produces":[
"application/json"
@@ -48,7 +48,7 @@
{
"method":"GET",
"summary":"Get total incoming bytes",
"type": "long",
"type":"int",
"nickname":"get_total_incoming_bytes",
"produces":[
"application/json"
@@ -72,7 +72,7 @@
{
"method":"GET",
"summary":"Get all total incoming bytes",
"type": "long",
"type":"int",
"nickname":"get_all_total_incoming_bytes",
"produces":[
"application/json"
@@ -88,7 +88,7 @@
{
"method":"GET",
"summary":"Get total outgoing bytes",
"type": "long",
"type":"int",
"nickname":"get_total_outgoing_bytes",
"produces":[
"application/json"
@@ -112,7 +112,7 @@
{
"method":"GET",
"summary":"Get all total outgoing bytes",
"type": "long",
"type":"int",
"nickname":"get_all_total_outgoing_bytes",
"produces":[
"application/json"
@@ -154,7 +154,7 @@
"description":"The peer"
},
"session_index":{
"type": "long",
"type":"int",
"description":"The session index"
},
"connecting":{
@@ -211,7 +211,7 @@
"description":"The ID"
},
"files":{
"type": "long",
"type":"int",
"description":"Number of files to transfer. Can be 0 if nothing to transfer for some streaming request."
},
"total_size":{
@@ -242,7 +242,7 @@
"description":"The peer address"
},
"session_index":{
"type": "long",
"type":"int",
"description":"The session index"
},
"file_name":{

View File

@@ -52,76 +52,6 @@
}
]
},
{
"path":"/system/log",
"operations":[
{
"method":"POST",
"summary":"Write a message to the Scylla log",
"type":"void",
"nickname":"write_log_message",
"produces":[
"application/json"
],
"parameters":[
{
"name":"message",
"description":"The message to write to the log",
"required":true,
"allowMultiple":false,
"type":"string",
"paramType":"query"
},
{
"name":"level",
"description":"The logging level to use",
"required":true,
"allowMultiple":false,
"type":"string",
"enum":[
"error",
"warn",
"info",
"debug",
"trace"
],
"paramType":"query"
}
]
}
]
},
{
"path":"/system/drop_sstable_caches",
"operations":[
{
"method":"POST",
"summary":"Drop in-memory caches for data which is in sstables",
"type":"void",
"nickname":"drop_sstable_caches",
"produces":[
"application/json"
],
"parameters":[
]
}
]
},
{
"path":"/system/uptime_ms",
"operations":[
{
"method":"GET",
"summary":"Get system uptime, in milliseconds",
"type":"long",
"nickname":"get_system_uptime",
"produces":[
"application/json"
],
"parameters":[]
}
]
},
{
"path":"/system/logger/{name}",
"operations":[

View File

@@ -1,329 +0,0 @@
{
"apiVersion":"0.0.1",
"swaggerVersion":"1.2",
"basePath":"{{Protocol}}://{{Host}}",
"resourcePath":"/task_manager",
"produces":[
"application/json"
],
"apis":[
{
"path":"/task_manager/list_modules",
"operations":[
{
"method":"GET",
"summary":"Get all modules names",
"type":"array",
"items":{
"type":"string"
},
"nickname":"get_modules",
"produces":[
"application/json"
],
"parameters":[
]
}
]
},
{
"path":"/task_manager/list_module_tasks/{module}",
"operations":[
{
"method":"GET",
"summary":"Get a list of tasks",
"type":"array",
"items":{
"type":"task_stats"
},
"nickname":"get_tasks",
"produces":[
"application/json"
],
"parameters":[
{
"name":"module",
"description":"The module to query about",
"required":true,
"allowMultiple":false,
"type":"string",
"paramType":"path"
},
{
"name":"internal",
"description":"Boolean flag indicating whether internal tasks should be shown (false by default)",
"required":false,
"allowMultiple":false,
"type":"boolean",
"paramType":"query"
},
{
"name":"keyspace",
"description":"The keyspace to query about",
"required":false,
"allowMultiple":false,
"type":"string",
"paramType":"query"
},
{
"name":"table",
"description":"The table to query about",
"required":false,
"allowMultiple":false,
"type":"string",
"paramType":"query"
}
]
}
]
},
{
"path":"/task_manager/task_status/{task_id}",
"operations":[
{
"method":"GET",
"summary":"Get task status",
"type":"task_status",
"nickname":"get_task_status",
"produces":[
"application/json"
],
"parameters":[
{
"name":"task_id",
"description":"The uuid of a task to query about",
"required":true,
"allowMultiple":false,
"type":"string",
"paramType":"path"
}
]
}
]
},
{
"path":"/task_manager/abort_task/{task_id}",
"operations":[
{
"method":"POST",
"summary":"Abort running task and its descendants",
"type":"void",
"nickname":"abort_task",
"produces":[
"application/json"
],
"parameters":[
{
"name":"task_id",
"description":"The uuid of a task to abort",
"required":true,
"allowMultiple":false,
"type":"string",
"paramType":"path"
}
]
}
]
},
{
"path":"/task_manager/wait_task/{task_id}",
"operations":[
{
"method":"GET",
"summary":"Wait for a task to complete",
"type":"task_status",
"nickname":"wait_task",
"produces":[
"application/json"
],
"parameters":[
{
"name":"task_id",
"description":"The uuid of a task to wait for",
"required":true,
"allowMultiple":false,
"type":"string",
"paramType":"path"
}
]
}
]
},
{
"path":"/task_manager/task_status_recursive/{task_id}",
"operations":[
{
"method":"GET",
"summary":"Get statuses of the task and all its descendants",
"type":"array",
"items":{
"type":"task_status"
},
"nickname":"get_task_status_recursively",
"produces":[
"application/json"
],
"parameters":[
{
"name":"task_id",
"description":"The uuid of a task to query about",
"required":true,
"allowMultiple":false,
"type":"string",
"paramType":"path"
}
]
}
]
},
{
"path":"/task_manager/ttl",
"operations":[
{
"method":"POST",
"summary":"Set ttl in seconds and get last value",
"type":"long",
"nickname":"get_and_update_ttl",
"produces":[
"application/json"
],
"parameters":[
{
"name":"ttl",
"description":"The number of seconds for which the tasks will be kept in memory after it finishes",
"required":true,
"allowMultiple":false,
"type":"long",
"paramType":"query"
}
]
}
]
}
],
"models":{
"task_stats" :{
"id": "task_stats",
"description":"A task statistics object",
"properties":{
"task_id":{
"type":"string",
"description":"The uuid of a task"
},
"state":{
"type":"string",
"enum":[
"created",
"running",
"done",
"failed"
],
"description":"The state of a task"
},
"type":{
"type":"string",
"description":"The description of the task"
},
"keyspace":{
"type":"string",
"description":"The keyspace the task is working on (if applicable)"
},
"table":{
"type":"string",
"description":"The table the task is working on (if applicable)"
},
"entity":{
"type":"string",
"description":"Task-specific entity description"
},
"sequence_number":{
"type":"long",
"description":"The running sequence number of the task"
}
}
},
"task_status":{
"id":"task_status",
"description":"A task status object",
"properties":{
"id":{
"type":"string",
"description":"The uuid of the task"
},
"type":{
"type":"string",
"description":"The description of the task"
},
"state":{
"type":"string",
"enum":[
"created",
"running",
"done",
"failed"
],
"description":"The state of the task"
},
"is_abortable":{
"type":"boolean",
"description":"Boolean flag indicating whether the task can be aborted"
},
"start_time":{
"type":"datetime",
"description":"The start time of the task"
},
"end_time":{
"type":"datetime",
"description":"The end time of the task (unspecified when the task is not completed)"
},
"error":{
"type":"string",
"description":"Error string, if the task failed"
},
"parent_id":{
"type":"string",
"description":"The uuid of the parent task"
},
"sequence_number":{
"type":"long",
"description":"The running sequence number of the task"
},
"shard":{
"type":"long",
"description":"The number of a shard the task is running on"
},
"keyspace":{
"type":"string",
"description":"The keyspace the task is working on (if applicable)"
},
"table":{
"type":"string",
"description":"The table the task is working on (if applicable)"
},
"entity":{
"type":"string",
"description":"Task-specific entity description"
},
"progress_units":{
"type":"string",
"description":"A description of the progress units"
},
"progress_total":{
"type":"double",
"description":"The total number of units to complete for the task"
},
"progress_completed":{
"type":"double",
"description":"The number of units completed so far"
},
"children_ids":{
"type":"array",
"items":{
"type":"string"
},
"description":"Task IDs of children of this task"
}
}
}
}
}

View File

@@ -1,153 +0,0 @@
{
"apiVersion":"0.0.1",
"swaggerVersion":"1.2",
"basePath":"{{Protocol}}://{{Host}}",
"resourcePath":"/task_manager_test",
"produces":[
"application/json"
],
"apis":[
{
"path":"/task_manager_test/test_module",
"operations":[
{
"method":"POST",
"summary":"Register test module in task manager",
"type":"void",
"nickname":"register_test_module",
"produces":[
"application/json"
],
"parameters":[
]
},
{
"method":"DELETE",
"summary":"Unregister test module in task manager",
"type":"void",
"nickname":"unregister_test_module",
"produces":[
"application/json"
],
"parameters":[
]
}
]
},
{
"path":"/task_manager_test/test_task",
"operations":[
{
"method":"POST",
"summary":"Register test task",
"type":"string",
"nickname":"register_test_task",
"produces":[
"application/json"
],
"parameters":[
{
"name":"task_id",
"description":"The uuid of a task to register",
"required":false,
"allowMultiple":false,
"type":"string",
"paramType":"query"
},
{
"name":"shard",
"description":"The shard of the task",
"required":false,
"allowMultiple":false,
"type":"long",
"paramType":"query"
},
{
"name":"parent_id",
"description":"The uuid of a parent task",
"required":false,
"allowMultiple":false,
"type":"string",
"paramType":"query"
},
{
"name":"keyspace",
"description":"The keyspace the task is working on",
"required":false,
"allowMultiple":false,
"type":"string",
"paramType":"query"
},
{
"name":"table",
"description":"The table the task is working on",
"required":false,
"allowMultiple":false,
"type":"string",
"paramType":"query"
},
{
"name":"entity",
"description":"Task-specific entity description",
"required":false,
"allowMultiple":false,
"type":"string",
"paramType":"query"
}
]
},
{
"method":"DELETE",
"summary":"Unregister test task",
"type":"void",
"nickname":"unregister_test_task",
"produces":[
"application/json"
],
"parameters":[
{
"name":"task_id",
"description":"The uuid of a task to register",
"required":true,
"allowMultiple":false,
"type":"string",
"paramType":"query"
}
]
}
]
},
{
"path":"/task_manager_test/finish_test_task/{task_id}",
"operations":[
{
"method":"POST",
"summary":"Finish test task",
"type":"void",
"nickname":"finish_test_task",
"produces":[
"application/json"
],
"parameters":[
{
"name":"task_id",
"description":"The uuid of a task to finish",
"required":true,
"allowMultiple":false,
"type":"string",
"paramType":"path"
},
{
"name":"error",
"description":"The error with which task fails (if it does)",
"required":false,
"allowMultiple":false,
"type":"string",
"paramType":"query"
}
]
}
]
}
]
}

View File

@@ -1,15 +1,28 @@
/*
* Copyright 2015-present ScyllaDB
* Copyright 2015 ScyllaDB
*/
/*
* SPDX-License-Identifier: AGPL-3.0-or-later
* 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 "api.hh"
#include <seastar/http/file_handler.hh>
#include <seastar/http/transformers.hh>
#include <seastar/http/api_docs.hh>
#include "http/file_handler.hh"
#include "http/transformers.hh"
#include "http/api_docs.hh"
#include "storage_service.hh"
#include "commitlog.hh"
#include "gossiper.hh"
@@ -23,24 +36,17 @@
#include "endpoint_snitch.hh"
#include "compaction_manager.hh"
#include "hinted_handoff.hh"
#include "error_injection.hh"
#include "authorization_cache.hh"
#include <seastar/http/exception.hh>
#include "http/exception.hh"
#include "stream_manager.hh"
#include "system.hh"
#include "api/config.hh"
#include "task_manager.hh"
#include "task_manager_test.hh"
logging::logger apilog("api");
namespace api {
using namespace seastar::httpd;
static std::unique_ptr<reply> exception_reply(std::exception_ptr eptr) {
try {
std::rethrow_exception(eptr);
} catch (const replica::no_such_keyspace& ex) {
} catch (const no_such_keyspace& ex) {
throw bad_param_exception(ex.what());
}
// We never going to get here
@@ -60,19 +66,13 @@ future<> set_server_init(http_context& ctx) {
rb->set_api_doc(r);
rb02->set_api_doc(r);
rb02->register_api_file(r, "swagger20_header");
set_config(rb02, ctx, r);
rb->register_function(r, "system",
"The system related API");
set_system(ctx, r);
});
}
future<> set_server_config(http_context& ctx, const db::config& cfg) {
auto rb02 = std::make_shared < api_registry_builder20 > (ctx.api_doc, "/v2");
return ctx.http_server.set_routes([&ctx, &cfg, rb02](routes& r) {
set_config(rb02, ctx, r, cfg);
});
}
static future<> register_api(http_context& ctx, const sstring& api_name,
const sstring api_desc,
std::function<void(http_context& ctx, routes& r)> f) {
@@ -84,129 +84,37 @@ static future<> register_api(http_context& ctx, const sstring& api_name,
});
}
future<> set_transport_controller(http_context& ctx, cql_transport::controller& ctl) {
return ctx.http_server.set_routes([&ctx, &ctl] (routes& r) { set_transport_controller(ctx, r, ctl); });
future<> set_server_storage_service(http_context& ctx) {
return register_api(ctx, "storage_service", "The storage service API", set_storage_service);
}
future<> unset_transport_controller(http_context& ctx) {
return ctx.http_server.set_routes([&ctx] (routes& r) { unset_transport_controller(ctx, r); });
future<> set_server_snitch(http_context& ctx) {
return register_api(ctx, "endpoint_snitch_info", "The endpoint snitch info API", set_endpoint_snitch);
}
future<> set_rpc_controller(http_context& ctx, thrift_controller& ctl) {
return ctx.http_server.set_routes([&ctx, &ctl] (routes& r) { set_rpc_controller(ctx, r, ctl); });
}
future<> unset_rpc_controller(http_context& ctx) {
return ctx.http_server.set_routes([&ctx] (routes& r) { unset_rpc_controller(ctx, r); });
}
future<> set_server_storage_service(http_context& ctx, sharded<service::storage_service>& ss, sharded<gms::gossiper>& g, sharded<cdc::generation_service>& cdc_gs, sharded<db::system_keyspace>& sys_ks) {
return register_api(ctx, "storage_service", "The storage service API", [&ss, &g, &cdc_gs, &sys_ks] (http_context& ctx, routes& r) {
set_storage_service(ctx, r, ss, g.local(), cdc_gs, sys_ks);
});
}
future<> set_server_sstables_loader(http_context& ctx, sharded<sstables_loader>& sst_loader) {
return ctx.http_server.set_routes([&ctx, &sst_loader] (routes& r) { set_sstables_loader(ctx, r, sst_loader); });
}
future<> unset_server_sstables_loader(http_context& ctx) {
return ctx.http_server.set_routes([&ctx] (routes& r) { unset_sstables_loader(ctx, r); });
}
future<> set_server_view_builder(http_context& ctx, sharded<db::view::view_builder>& vb) {
return ctx.http_server.set_routes([&ctx, &vb] (routes& r) { set_view_builder(ctx, r, vb); });
}
future<> unset_server_view_builder(http_context& ctx) {
return ctx.http_server.set_routes([&ctx] (routes& r) { unset_view_builder(ctx, r); });
}
future<> set_server_repair(http_context& ctx, sharded<repair_service>& repair) {
return ctx.http_server.set_routes([&ctx, &repair] (routes& r) { set_repair(ctx, r, repair); });
}
future<> unset_server_repair(http_context& ctx) {
return ctx.http_server.set_routes([&ctx] (routes& r) { unset_repair(ctx, r); });
}
future<> set_server_authorization_cache(http_context &ctx, sharded<auth::service> &auth_service) {
return register_api(ctx, "authorization_cache",
"The authorization cache API", [&auth_service] (http_context &ctx, routes &r) {
set_authorization_cache(ctx, r, auth_service);
});
}
future<> unset_server_authorization_cache(http_context& ctx) {
return ctx.http_server.set_routes([&ctx] (routes& r) { unset_authorization_cache(ctx, r); });
}
future<> set_server_snapshot(http_context& ctx, sharded<db::snapshot_ctl>& snap_ctl) {
return ctx.http_server.set_routes([&ctx, &snap_ctl] (routes& r) { set_snapshot(ctx, r, snap_ctl); });
}
future<> unset_server_snapshot(http_context& ctx) {
return ctx.http_server.set_routes([&ctx] (routes& r) { unset_snapshot(ctx, r); });
}
future<> set_server_snitch(http_context& ctx, sharded<locator::snitch_ptr>& snitch) {
return register_api(ctx, "endpoint_snitch_info", "The endpoint snitch info API", [&snitch] (http_context& ctx, routes& r) {
set_endpoint_snitch(ctx, r, snitch);
});
}
future<> unset_server_snitch(http_context& ctx) {
return ctx.http_server.set_routes([&ctx] (routes& r) { unset_endpoint_snitch(ctx, r); });
}
future<> set_server_gossip(http_context& ctx, sharded<gms::gossiper>& g) {
future<> set_server_gossip(http_context& ctx) {
return register_api(ctx, "gossiper",
"The gossiper API", [&g] (http_context& ctx, routes& r) {
set_gossiper(ctx, r, g.local());
});
"The gossiper API", set_gossiper);
}
future<> set_server_load_sstable(http_context& ctx, sharded<db::system_keyspace>& sys_ks) {
future<> set_server_load_sstable(http_context& ctx) {
return register_api(ctx, "column_family",
"The column family API", [&sys_ks] (http_context& ctx, routes& r) {
set_column_family(ctx, r, sys_ks);
});
"The column family API", set_column_family);
}
future<> unset_server_load_sstable(http_context& ctx) {
return ctx.http_server.set_routes([&ctx] (routes& r) { unset_column_family(ctx, r); });
}
future<> set_server_messaging_service(http_context& ctx, sharded<netw::messaging_service>& ms) {
future<> set_server_messaging_service(http_context& ctx) {
return register_api(ctx, "messaging_service",
"The messaging service API", [&ms] (http_context& ctx, routes& r) {
set_messaging_service(ctx, r, ms);
});
}
future<> unset_server_messaging_service(http_context& ctx) {
return ctx.http_server.set_routes([&ctx] (routes& r) { unset_messaging_service(ctx, r); });
"The messaging service API", set_messaging_service);
}
future<> set_server_storage_proxy(http_context& ctx, sharded<service::storage_service>& ss) {
future<> set_server_storage_proxy(http_context& ctx) {
return register_api(ctx, "storage_proxy",
"The storage proxy API", [&ss] (http_context& ctx, routes& r) {
set_storage_proxy(ctx, r, ss);
});
"The storage proxy API", set_storage_proxy);
}
future<> unset_server_storage_proxy(http_context& ctx) {
return ctx.http_server.set_routes([&ctx] (routes& r) { unset_storage_proxy(ctx, r); });
}
future<> set_server_stream_manager(http_context& ctx, sharded<streaming::stream_manager>& sm) {
future<> set_server_stream_manager(http_context& ctx) {
return register_api(ctx, "stream_manager",
"The stream manager API", [&sm] (http_context& ctx, routes& r) {
set_stream_manager(ctx, r, sm);
});
}
future<> unset_server_stream_manager(http_context& ctx) {
return ctx.http_server.set_routes([&ctx] (routes& r) { unset_stream_manager(ctx, r); });
"The stream manager API", set_stream_manager);
}
future<> set_server_cache(http_context& ctx) {
@@ -214,34 +122,13 @@ future<> set_server_cache(http_context& ctx) {
"The cache service API", set_cache_service);
}
future<> set_hinted_handoff(http_context& ctx, sharded<gms::gossiper>& g) {
return register_api(ctx, "hinted_handoff",
"The hinted handoff API", [&g] (http_context& ctx, routes& r) {
set_hinted_handoff(ctx, r, g.local());
});
}
future<> unset_hinted_handoff(http_context& ctx) {
return ctx.http_server.set_routes([&ctx] (routes& r) { unset_hinted_handoff(ctx, r); });
}
future<> set_server_gossip_settle(http_context& ctx, sharded<gms::gossiper>& g) {
auto rb = std::make_shared < api_registry_builder > (ctx.api_doc);
return ctx.http_server.set_routes([rb, &ctx, &g](routes& r) {
rb->register_function(r, "failure_detector",
"The failure detector API");
set_failure_detector(ctx, r, g.local());
});
}
future<> set_server_compaction_manager(http_context& ctx) {
future<> set_server_gossip_settle(http_context& ctx) {
auto rb = std::make_shared < api_registry_builder > (ctx.api_doc);
return ctx.http_server.set_routes([rb, &ctx](routes& r) {
rb->register_function(r, "compaction_manager",
"The Compaction manager API");
set_compaction_manager(ctx, r);
rb->register_function(r, "failure_detector",
"The failure detector API");
set_failure_detector(ctx,r);
});
}
@@ -249,71 +136,23 @@ future<> set_server_done(http_context& ctx) {
auto rb = std::make_shared < api_registry_builder > (ctx.api_doc);
return ctx.http_server.set_routes([rb, &ctx](routes& r) {
rb->register_function(r, "compaction_manager",
"The Compaction manager API");
set_compaction_manager(ctx, r);
rb->register_function(r, "lsa", "Log-structured allocator API");
set_lsa(ctx, r);
rb->register_function(r, "commitlog",
"The commit log API");
set_commitlog(ctx,r);
rb->register_function(r, "hinted_handoff",
"The hinted handoff API");
set_hinted_handoff(ctx, r);
rb->register_function(r, "collectd",
"The collectd API");
set_collectd(ctx, r);
rb->register_function(r, "error_injection",
"The error injection API");
set_error_injection(ctx, r);
});
}
future<> set_server_task_manager(http_context& ctx, lw_shared_ptr<db::config> cfg) {
auto rb = std::make_shared < api_registry_builder > (ctx.api_doc);
return ctx.http_server.set_routes([rb, &ctx, &cfg = *cfg](routes& r) {
rb->register_function(r, "task_manager",
"The task manager API");
set_task_manager(ctx, r, cfg);
});
}
#ifndef SCYLLA_BUILD_MODE_RELEASE
future<> set_server_task_manager_test(http_context& ctx) {
auto rb = std::make_shared < api_registry_builder > (ctx.api_doc);
return ctx.http_server.set_routes([rb, &ctx](routes& r) mutable {
rb->register_function(r, "task_manager_test",
"The task manager test API");
set_task_manager_test(ctx, r);
});
}
#endif
void req_params::process(const request& req) {
// Process mandatory parameters
for (auto& [name, ent] : params) {
if (!ent.is_mandatory) {
continue;
}
try {
ent.value = req.param[name];
} catch (std::out_of_range&) {
throw httpd::bad_param_exception(fmt::format("Mandatory parameter '{}' was not provided", name));
}
}
// Process optional parameters
for (auto& [name, value] : req.query_parameters) {
try {
auto& ent = params.at(name);
if (ent.is_mandatory) {
throw httpd::bad_param_exception(fmt::format("Parameter '{}' is expected to be provided as part of the request url", name));
}
ent.value = value;
} catch (std::out_of_range&) {
throw httpd::bad_param_exception(fmt::format("Unsupported optional parameter '{}'", name));
}
}
}
}

View File

@@ -1,23 +1,33 @@
/*
* Copyright 2015-present ScyllaDB
* Copyright 2015 ScyllaDB
*/
/*
* SPDX-License-Identifier: AGPL-3.0-or-later
* 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 <seastar/json/json_elements.hh>
#include <type_traits>
#include "json/json_elements.hh"
#include <boost/lexical_cast.hpp>
#include <boost/algorithm/string/split.hpp>
#include <boost/algorithm/string/classification.hpp>
#include <boost/units/detail/utility.hpp>
#include "api/api-doc/utils.json.hh"
#include "utils/histogram.hh"
#include "utils/estimated_histogram.hh"
#include <seastar/http/exception.hh>
#include "http/exception.hh"
#include "api_init.hh"
#include "seastarx.hh"
@@ -27,7 +37,7 @@ template<class T>
std::vector<sstring> container_to_vec(const T& container) {
std::vector<sstring> res;
for (auto i : container) {
res.push_back(fmt::to_string(i));
res.push_back(boost::lexical_cast<std::string>(i));
}
return res;
}
@@ -47,8 +57,8 @@ template<class T, class MAP>
std::vector<T>& map_to_key_value(const MAP& map, std::vector<T>& res) {
for (auto i : map) {
T val;
val.key = fmt::to_string(i.first);
val.value = fmt::to_string(i.second);
val.key = boost::lexical_cast<std::string>(i.first);
val.value = boost::lexical_cast<std::string>(i.second);
res.push_back(val);
}
return res;
@@ -58,14 +68,14 @@ T map_sum(T&& dest, const S& src) {
for (auto i : src) {
dest[i.first] += i.second;
}
return std::move(dest);
return dest;
}
template <typename MAP>
std::vector<sstring> map_keys(const MAP& map) {
std::vector<sstring> res;
for (const auto& i : map) {
res.push_back(fmt::to_string(i.first));
res.push_back(boost::lexical_cast<std::string>(i.first));
}
return res;
}
@@ -81,6 +91,13 @@ inline std::vector<sstring> split(const sstring& text, const char* separator) {
return boost::split(tokens, text, boost::is_any_of(separator));
}
/**
* Split a column family parameter
*/
inline std::vector<sstring> split_cf(const sstring& cf) {
return split(cf, ",");
}
/**
* A helper function to sum values on an a distributed object that
* has a get_stats method.
@@ -137,14 +154,6 @@ future<json::json_return_type> sum_timer_stats(distributed<T>& d, utils::timed_
});
}
template<class T, class F>
future<json::json_return_type> sum_timer_stats(distributed<T>& d, utils::timed_rate_moving_average_summary_and_histogram F::*f) {
return d.map_reduce0([f](const T& p) {return (p.get_stats().*f).rate();}, utils::rate_moving_average_and_histogram(),
std::plus<utils::rate_moving_average_and_histogram>()).then([](const utils::rate_moving_average_and_histogram& val) {
return make_ready_future<json::json_return_type>(timer_to_json(val));
});
}
inline int64_t min_int64(int64_t a, int64_t b) {
return std::min(a,b);
}
@@ -189,7 +198,7 @@ struct basic_ratio_holder : public json::jsonable {
typedef basic_ratio_holder<double> ratio_holder;
typedef basic_ratio_holder<int64_t> integral_ratio_holder;
class unimplemented_exception : public httpd::base_exception {
class unimplemented_exception : public base_exception {
public:
unimplemented_exception()
: base_exception("API call is not supported yet", reply::status_type::internal_server_error) {
@@ -207,105 +216,4 @@ std::vector<T> concat(std::vector<T> a, std::vector<T>&& b) {
return a;
}
template <class T, class Base = T>
class req_param {
public:
sstring name;
sstring param;
T value;
req_param(const request& req, sstring name, T default_val) : name(name) {
param = req.get_query_param(name);
if (param.empty()) {
value = default_val;
return;
}
try {
// boost::lexical_cast does not use boolalpha. Converting a
// true/false throws exceptions. We don't want that.
if constexpr (std::is_same_v<Base, bool>) {
// Cannot use boolalpha because we (probably) want to
// accept 1 and 0 as well as true and false. And True. And fAlse.
std::transform(param.begin(), param.end(), param.begin(), ::tolower);
if (param == "true" || param == "1") {
value = T(true);
} else if (param == "false" || param == "0") {
value = T(false);
} else {
throw boost::bad_lexical_cast{};
}
} else {
value = T{boost::lexical_cast<Base>(param)};
}
} catch (boost::bad_lexical_cast&) {
throw httpd::bad_param_exception(format("{} ({}): type error - should be {}", name, param, boost::units::detail::demangle(typeid(Base).name())));
}
}
operator T() const { return value; }
};
using mandatory = bool_class<struct mandatory_tag>;
class req_params {
public:
struct def {
std::optional<sstring> value;
mandatory is_mandatory = mandatory::no;
def(std::optional<sstring> value_ = std::nullopt, mandatory is_mandatory_ = mandatory::no)
: value(std::move(value_))
, is_mandatory(is_mandatory_)
{ }
def(mandatory is_mandatory_)
: is_mandatory(is_mandatory_)
{ }
};
private:
std::unordered_map<sstring, def> params;
public:
req_params(std::initializer_list<std::pair<sstring, def>> l) {
for (const auto& [name, ent] : l) {
add(std::move(name), std::move(ent));
}
}
void add(sstring name, def ent) {
params.emplace(std::move(name), std::move(ent));
}
void process(const request& req);
const std::optional<sstring>& get(const char* name) const {
return params.at(name).value;
}
template <typename T = sstring>
const std::optional<T> get_as(const char* name) const {
return get(name);
}
template <typename T = sstring>
requires std::same_as<T, bool>
const std::optional<bool> get_as(const char* name) const {
auto value = get(name);
if (!value) {
return std::nullopt;
}
std::transform(value->begin(), value->end(), value->begin(), ::tolower);
if (value == "true" || value == "yes" || value == "1") {
return true;
}
if (value == "false" || value == "no" || value == "0") {
return false;
}
throw boost::bad_lexical_cast{};
}
};
httpd::utils_json::estimated_histogram time_to_json_histogram(const utils::time_estimated_histogram& val);
}

View File

@@ -3,63 +3,25 @@
*/
/*
* SPDX-License-Identifier: AGPL-3.0-or-later
* 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 <seastar/http/httpd.hh>
#include <seastar/core/future.hh>
#include "replica/database_fwd.hh"
#include "tasks/task_manager.hh"
#include "seastarx.hh"
using request = http::request;
using reply = http::reply;
namespace service {
class load_meter;
class storage_proxy;
class storage_service;
} // namespace service
class sstables_loader;
namespace streaming {
class stream_manager;
}
namespace locator {
class token_metadata;
class shared_token_metadata;
class snitch_ptr;
} // namespace locator
namespace cql_transport { class controller; }
class thrift_controller;
namespace db {
class snapshot_ctl;
class config;
namespace view {
class view_builder;
}
class system_keyspace;
}
namespace netw { class messaging_service; }
class repair_service;
namespace cdc { class generation_service; }
namespace gms {
class gossiper;
}
namespace auth { class service; }
#include "database.hh"
#include "service/storage_proxy.hh"
#include "http/httpd.hh"
namespace api {
@@ -67,56 +29,24 @@ struct http_context {
sstring api_dir;
sstring api_doc;
httpd::http_server_control http_server;
distributed<replica::database>& db;
distributed<database>& db;
distributed<service::storage_proxy>& sp;
service::load_meter& lmeter;
const sharded<locator::shared_token_metadata>& shared_token_metadata;
sharded<tasks::task_manager>& tm;
http_context(distributed<replica::database>& _db,
distributed<service::storage_proxy>& _sp,
service::load_meter& _lm, const sharded<locator::shared_token_metadata>& _stm, sharded<tasks::task_manager>& _tm)
: db(_db), sp(_sp), lmeter(_lm), shared_token_metadata(_stm), tm(_tm) {
http_context(distributed<database>& _db,
distributed<service::storage_proxy>& _sp)
: db(_db), sp(_sp) {
}
const locator::token_metadata& get_token_metadata();
};
future<> set_server_init(http_context& ctx);
future<> set_server_config(http_context& ctx, const db::config& cfg);
future<> set_server_snitch(http_context& ctx, sharded<locator::snitch_ptr>& snitch);
future<> unset_server_snitch(http_context& ctx);
future<> set_server_storage_service(http_context& ctx, sharded<service::storage_service>& ss, sharded<gms::gossiper>& g, sharded<cdc::generation_service>& cdc_gs, sharded<db::system_keyspace>& sys_ks);
future<> set_server_sstables_loader(http_context& ctx, sharded<sstables_loader>& sst_loader);
future<> unset_server_sstables_loader(http_context& ctx);
future<> set_server_view_builder(http_context& ctx, sharded<db::view::view_builder>& vb);
future<> unset_server_view_builder(http_context& ctx);
future<> set_server_repair(http_context& ctx, sharded<repair_service>& repair);
future<> unset_server_repair(http_context& ctx);
future<> set_transport_controller(http_context& ctx, cql_transport::controller& ctl);
future<> unset_transport_controller(http_context& ctx);
future<> set_rpc_controller(http_context& ctx, thrift_controller& ctl);
future<> unset_rpc_controller(http_context& ctx);
future<> set_server_authorization_cache(http_context& ctx, sharded<auth::service> &auth_service);
future<> unset_server_authorization_cache(http_context& ctx);
future<> set_server_snapshot(http_context& ctx, sharded<db::snapshot_ctl>& snap_ctl);
future<> unset_server_snapshot(http_context& ctx);
future<> set_server_gossip(http_context& ctx, sharded<gms::gossiper>& g);
future<> set_server_load_sstable(http_context& ctx, sharded<db::system_keyspace>& sys_ks);
future<> unset_server_load_sstable(http_context& ctx);
future<> set_server_messaging_service(http_context& ctx, sharded<netw::messaging_service>& ms);
future<> unset_server_messaging_service(http_context& ctx);
future<> set_server_storage_proxy(http_context& ctx, sharded<service::storage_service>& ss);
future<> unset_server_storage_proxy(http_context& ctx);
future<> set_server_stream_manager(http_context& ctx, sharded<streaming::stream_manager>& sm);
future<> unset_server_stream_manager(http_context& ctx);
future<> set_hinted_handoff(http_context& ctx, sharded<gms::gossiper>& g);
future<> unset_hinted_handoff(http_context& ctx);
future<> set_server_gossip_settle(http_context& ctx, sharded<gms::gossiper>& g);
future<> set_server_snitch(http_context& ctx);
future<> set_server_storage_service(http_context& ctx);
future<> set_server_gossip(http_context& ctx);
future<> set_server_load_sstable(http_context& ctx);
future<> set_server_messaging_service(http_context& ctx);
future<> set_server_storage_proxy(http_context& ctx);
future<> set_server_stream_manager(http_context& ctx);
future<> set_server_gossip_settle(http_context& ctx);
future<> set_server_cache(http_context& ctx);
future<> set_server_compaction_manager(http_context& ctx);
future<> set_server_done(http_context& ctx);
future<> set_server_task_manager(http_context& ctx, lw_shared_ptr<db::config> cfg);
future<> set_server_task_manager_test(http_context& ctx);
}

View File

@@ -1,34 +0,0 @@
/*
* Copyright (C) 2022-present ScyllaDB
*/
/*
* SPDX-License-Identifier: AGPL-3.0-or-later
*/
#include "api/api-doc/authorization_cache.json.hh"
#include "api/authorization_cache.hh"
#include "api/api.hh"
#include "auth/common.hh"
namespace api {
using namespace json;
using namespace seastar::httpd;
void set_authorization_cache(http_context& ctx, routes& r, sharded<auth::service> &auth_service) {
httpd::authorization_cache_json::authorization_cache_reset.set(r, [&auth_service] (std::unique_ptr<http::request> req) -> future<json::json_return_type> {
co_await auth_service.invoke_on_all([] (auth::service& auth) -> future<> {
auth.reset_authorization_cache();
return make_ready_future<>();
});
co_return json_void();
});
}
void unset_authorization_cache(http_context& ctx, routes& r) {
httpd::authorization_cache_json::authorization_cache_reset.unset(r);
}
}

View File

@@ -1,18 +0,0 @@
/*
* Copyright (C) 2022-present ScyllaDB
*/
/*
* SPDX-License-Identifier: AGPL-3.0-or-later
*/
#pragma once
#include "api.hh"
namespace api {
void set_authorization_cache(http_context& ctx, httpd::routes& r, sharded<auth::service> &auth_service);
void unset_authorization_cache(http_context& ctx, httpd::routes& r);
}

View File

@@ -1,9 +1,22 @@
/*
* Copyright (C) 2015-present ScyllaDB
* Copyright (C) 2015 ScyllaDB
*/
/*
* SPDX-License-Identifier: AGPL-3.0-or-later
* 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 "cache_service.hh"
@@ -12,128 +25,127 @@
namespace api {
using namespace json;
using namespace seastar::httpd;
namespace cs = httpd::cache_service_json;
void set_cache_service(http_context& ctx, routes& r) {
cs::get_row_cache_save_period_in_seconds.set(r, [](std::unique_ptr<http::request> req) {
cs::get_row_cache_save_period_in_seconds.set(r, [](std::unique_ptr<request> req) {
// We never save the cache
// Origin uses 0 for never
return make_ready_future<json::json_return_type>(0);
});
cs::set_row_cache_save_period_in_seconds.set(r, [](std::unique_ptr<http::request> req) {
cs::set_row_cache_save_period_in_seconds.set(r, [](std::unique_ptr<request> req) {
// TBD
unimplemented();
auto period = req->get_query_param("period");
return make_ready_future<json::json_return_type>(json_void());
});
cs::get_key_cache_save_period_in_seconds.set(r, [](std::unique_ptr<http::request> req) {
cs::get_key_cache_save_period_in_seconds.set(r, [](std::unique_ptr<request> req) {
// We never save the cache
// Origin uses 0 for never
return make_ready_future<json::json_return_type>(0);
});
cs::set_key_cache_save_period_in_seconds.set(r, [](std::unique_ptr<http::request> req) {
cs::set_key_cache_save_period_in_seconds.set(r, [](std::unique_ptr<request> req) {
// TBD
unimplemented();
auto period = req->get_query_param("period");
return make_ready_future<json::json_return_type>(json_void());
});
cs::get_counter_cache_save_period_in_seconds.set(r, [](std::unique_ptr<http::request> req) {
cs::get_counter_cache_save_period_in_seconds.set(r, [](std::unique_ptr<request> req) {
// We never save the cache
// Origin uses 0 for never
return make_ready_future<json::json_return_type>(0);
});
cs::set_counter_cache_save_period_in_seconds.set(r, [](std::unique_ptr<http::request> req) {
cs::set_counter_cache_save_period_in_seconds.set(r, [](std::unique_ptr<request> req) {
// TBD
unimplemented();
auto ccspis = req->get_query_param("ccspis");
return make_ready_future<json::json_return_type>(json_void());
});
cs::get_row_cache_keys_to_save.set(r, [](std::unique_ptr<http::request> req) {
cs::get_row_cache_keys_to_save.set(r, [](std::unique_ptr<request> req) {
// TBD
unimplemented();
return make_ready_future<json::json_return_type>(0);
});
cs::set_row_cache_keys_to_save.set(r, [](std::unique_ptr<http::request> req) {
cs::set_row_cache_keys_to_save.set(r, [](std::unique_ptr<request> req) {
// TBD
unimplemented();
auto rckts = req->get_query_param("rckts");
return make_ready_future<json::json_return_type>(json_void());
});
cs::get_key_cache_keys_to_save.set(r, [](std::unique_ptr<http::request> req) {
cs::get_key_cache_keys_to_save.set(r, [](std::unique_ptr<request> req) {
// TBD
unimplemented();
return make_ready_future<json::json_return_type>(0);
});
cs::set_key_cache_keys_to_save.set(r, [](std::unique_ptr<http::request> req) {
cs::set_key_cache_keys_to_save.set(r, [](std::unique_ptr<request> req) {
// TBD
unimplemented();
auto kckts = req->get_query_param("kckts");
return make_ready_future<json::json_return_type>(json_void());
});
cs::get_counter_cache_keys_to_save.set(r, [](std::unique_ptr<http::request> req) {
cs::get_counter_cache_keys_to_save.set(r, [](std::unique_ptr<request> req) {
// TBD
unimplemented();
return make_ready_future<json::json_return_type>(0);
});
cs::set_counter_cache_keys_to_save.set(r, [](std::unique_ptr<http::request> req) {
cs::set_counter_cache_keys_to_save.set(r, [](std::unique_ptr<request> req) {
// TBD
unimplemented();
auto cckts = req->get_query_param("cckts");
return make_ready_future<json::json_return_type>(json_void());
});
cs::invalidate_key_cache.set(r, [](std::unique_ptr<http::request> req) {
cs::invalidate_key_cache.set(r, [](std::unique_ptr<request> req) {
// TBD
unimplemented();
return make_ready_future<json::json_return_type>(json_void());
});
cs::invalidate_counter_cache.set(r, [](std::unique_ptr<http::request> req) {
cs::invalidate_counter_cache.set(r, [](std::unique_ptr<request> req) {
// TBD
unimplemented();
return make_ready_future<json::json_return_type>(json_void());
});
cs::set_row_cache_capacity_in_mb.set(r, [](std::unique_ptr<http::request> req) {
cs::set_row_cache_capacity_in_mb.set(r, [](std::unique_ptr<request> req) {
// TBD
unimplemented();
auto capacity = req->get_query_param("capacity");
return make_ready_future<json::json_return_type>(json_void());
});
cs::set_key_cache_capacity_in_mb.set(r, [](std::unique_ptr<http::request> req) {
cs::set_key_cache_capacity_in_mb.set(r, [](std::unique_ptr<request> req) {
// TBD
unimplemented();
auto period = req->get_query_param("period");
return make_ready_future<json::json_return_type>(json_void());
});
cs::set_counter_cache_capacity_in_mb.set(r, [](std::unique_ptr<http::request> req) {
cs::set_counter_cache_capacity_in_mb.set(r, [](std::unique_ptr<request> req) {
// TBD
unimplemented();
auto capacity = req->get_query_param("capacity");
return make_ready_future<json::json_return_type>(json_void());
});
cs::save_caches.set(r, [](std::unique_ptr<http::request> req) {
cs::save_caches.set(r, [](std::unique_ptr<request> req) {
// TBD
unimplemented();
return make_ready_future<json::json_return_type>(json_void());
});
cs::get_key_capacity.set(r, [] (std::unique_ptr<http::request> req) {
cs::get_key_capacity.set(r, [] (std::unique_ptr<request> req) {
// TBD
// FIXME
// we don't support keys cache,
@@ -141,7 +153,7 @@ void set_cache_service(http_context& ctx, routes& r) {
return make_ready_future<json::json_return_type>(0);
});
cs::get_key_hits.set(r, [] (std::unique_ptr<http::request> req) {
cs::get_key_hits.set(r, [] (std::unique_ptr<request> req) {
// TBD
// FIXME
// we don't support keys cache,
@@ -149,7 +161,7 @@ void set_cache_service(http_context& ctx, routes& r) {
return make_ready_future<json::json_return_type>(0);
});
cs::get_key_requests.set(r, [] (std::unique_ptr<http::request> req) {
cs::get_key_requests.set(r, [] (std::unique_ptr<request> req) {
// TBD
// FIXME
// we don't support keys cache,
@@ -157,7 +169,7 @@ void set_cache_service(http_context& ctx, routes& r) {
return make_ready_future<json::json_return_type>(0);
});
cs::get_key_hit_rate.set(r, [] (std::unique_ptr<http::request> req) {
cs::get_key_hit_rate.set(r, [] (std::unique_ptr<request> req) {
// TBD
// FIXME
// we don't support keys cache,
@@ -165,21 +177,21 @@ void set_cache_service(http_context& ctx, routes& r) {
return make_ready_future<json::json_return_type>(0);
});
cs::get_key_hits_moving_avrage.set(r, [] (std::unique_ptr<http::request> req) {
cs::get_key_hits_moving_avrage.set(r, [&ctx] (std::unique_ptr<request> req) {
// TBD
// FIXME
// See above
return make_ready_future<json::json_return_type>(meter_to_json(utils::rate_moving_average()));
});
cs::get_key_requests_moving_avrage.set(r, [] (std::unique_ptr<http::request> req) {
cs::get_key_requests_moving_avrage.set(r, [&ctx] (std::unique_ptr<request> req) {
// TBD
// FIXME
// See above
return make_ready_future<json::json_return_type>(meter_to_json(utils::rate_moving_average()));
});
cs::get_key_size.set(r, [] (std::unique_ptr<http::request> req) {
cs::get_key_size.set(r, [] (std::unique_ptr<request> req) {
// TBD
// FIXME
// we don't support keys cache,
@@ -187,7 +199,7 @@ void set_cache_service(http_context& ctx, routes& r) {
return make_ready_future<json::json_return_type>(0);
});
cs::get_key_entries.set(r, [] (std::unique_ptr<http::request> req) {
cs::get_key_entries.set(r, [] (std::unique_ptr<request> req) {
// TBD
// FIXME
// we don't support keys cache,
@@ -195,68 +207,62 @@ void set_cache_service(http_context& ctx, routes& r) {
return make_ready_future<json::json_return_type>(0);
});
cs::get_row_capacity.set(r, [&ctx] (std::unique_ptr<http::request> req) {
return ctx.db.map_reduce0([](replica::database& db) -> uint64_t {
return db.row_cache_tracker().region().occupancy().used_space();
}, uint64_t(0), std::plus<uint64_t>()).then([](const int64_t& res) {
return make_ready_future<json::json_return_type>(res);
});
cs::get_row_capacity.set(r, [&ctx] (std::unique_ptr<request> req) {
return map_reduce_cf(ctx, uint64_t(0), [](const column_family& cf) {
return cf.get_row_cache().get_cache_tracker().region().occupancy().used_space();
}, std::plus<uint64_t>());
});
cs::get_row_hits.set(r, [&ctx] (std::unique_ptr<http::request> req) {
return map_reduce_cf(ctx, uint64_t(0), [](const replica::column_family& cf) {
cs::get_row_hits.set(r, [&ctx] (std::unique_ptr<request> req) {
return map_reduce_cf(ctx, uint64_t(0), [](const column_family& cf) {
return cf.get_row_cache().stats().hits.count();
}, std::plus<uint64_t>());
});
cs::get_row_requests.set(r, [&ctx] (std::unique_ptr<http::request> req) {
return map_reduce_cf(ctx, uint64_t(0), [](const replica::column_family& cf) {
cs::get_row_requests.set(r, [&ctx] (std::unique_ptr<request> req) {
return map_reduce_cf(ctx, uint64_t(0), [](const column_family& cf) {
return cf.get_row_cache().stats().hits.count() + cf.get_row_cache().stats().misses.count();
}, std::plus<uint64_t>());
});
cs::get_row_hit_rate.set(r, [&ctx] (std::unique_ptr<http::request> req) {
return map_reduce_cf(ctx, ratio_holder(), [](const replica::column_family& cf) {
cs::get_row_hit_rate.set(r, [&ctx] (std::unique_ptr<request> req) {
return map_reduce_cf(ctx, ratio_holder(), [](const column_family& cf) {
return ratio_holder(cf.get_row_cache().stats().hits.count() + cf.get_row_cache().stats().misses.count(),
cf.get_row_cache().stats().hits.count());
}, std::plus<ratio_holder>());
});
cs::get_row_hits_moving_avrage.set(r, [&ctx] (std::unique_ptr<http::request> req) {
return map_reduce_cf_raw(ctx, utils::rate_moving_average(), [](const replica::column_family& cf) {
cs::get_row_hits_moving_avrage.set(r, [&ctx] (std::unique_ptr<request> req) {
return map_reduce_cf_raw(ctx, utils::rate_moving_average(), [](const column_family& cf) {
return cf.get_row_cache().stats().hits.rate();
}, std::plus<utils::rate_moving_average>()).then([](const utils::rate_moving_average& m) {
return make_ready_future<json::json_return_type>(meter_to_json(m));
});
});
cs::get_row_requests_moving_avrage.set(r, [&ctx] (std::unique_ptr<http::request> req) {
return map_reduce_cf_raw(ctx, utils::rate_moving_average(), [](const replica::column_family& cf) {
cs::get_row_requests_moving_avrage.set(r, [&ctx] (std::unique_ptr<request> req) {
return map_reduce_cf_raw(ctx, utils::rate_moving_average(), [](const column_family& cf) {
return cf.get_row_cache().stats().hits.rate() + cf.get_row_cache().stats().misses.rate();
}, std::plus<utils::rate_moving_average>()).then([](const utils::rate_moving_average& m) {
return make_ready_future<json::json_return_type>(meter_to_json(m));
});
});
cs::get_row_size.set(r, [&ctx] (std::unique_ptr<http::request> req) {
cs::get_row_size.set(r, [&ctx] (std::unique_ptr<request> req) {
// In origin row size is the weighted size.
// We currently do not support weights, so we use num entries instead
return ctx.db.map_reduce0([](replica::database& db) -> uint64_t {
return db.row_cache_tracker().partitions();
}, uint64_t(0), std::plus<uint64_t>()).then([](const int64_t& res) {
return make_ready_future<json::json_return_type>(res);
});
return map_reduce_cf(ctx, 0, [](const column_family& cf) {
return cf.get_row_cache().partitions();
}, std::plus<uint64_t>());
});
cs::get_row_entries.set(r, [&ctx] (std::unique_ptr<http::request> req) {
return ctx.db.map_reduce0([](replica::database& db) -> uint64_t {
return db.row_cache_tracker().partitions();
}, uint64_t(0), std::plus<uint64_t>()).then([](const int64_t& res) {
return make_ready_future<json::json_return_type>(res);
});
cs::get_row_entries.set(r, [&ctx] (std::unique_ptr<request> req) {
return map_reduce_cf(ctx, 0, [](const column_family& cf) {
return cf.get_row_cache().partitions();
}, std::plus<uint64_t>());
});
cs::get_counter_capacity.set(r, [] (std::unique_ptr<http::request> req) {
cs::get_counter_capacity.set(r, [] (std::unique_ptr<request> req) {
// TBD
// FIXME
// we don't support counter cache,
@@ -264,7 +270,7 @@ void set_cache_service(http_context& ctx, routes& r) {
return make_ready_future<json::json_return_type>(0);
});
cs::get_counter_hits.set(r, [] (std::unique_ptr<http::request> req) {
cs::get_counter_hits.set(r, [] (std::unique_ptr<request> req) {
// TBD
// FIXME
// we don't support counter cache,
@@ -272,7 +278,7 @@ void set_cache_service(http_context& ctx, routes& r) {
return make_ready_future<json::json_return_type>(0);
});
cs::get_counter_requests.set(r, [] (std::unique_ptr<http::request> req) {
cs::get_counter_requests.set(r, [] (std::unique_ptr<request> req) {
// TBD
// FIXME
// we don't support counter cache,
@@ -280,7 +286,7 @@ void set_cache_service(http_context& ctx, routes& r) {
return make_ready_future<json::json_return_type>(0);
});
cs::get_counter_hit_rate.set(r, [] (std::unique_ptr<http::request> req) {
cs::get_counter_hit_rate.set(r, [] (std::unique_ptr<request> req) {
// TBD
// FIXME
// we don't support counter cache,
@@ -288,21 +294,21 @@ void set_cache_service(http_context& ctx, routes& r) {
return make_ready_future<json::json_return_type>(0);
});
cs::get_counter_hits_moving_avrage.set(r, [] (std::unique_ptr<http::request> req) {
cs::get_counter_hits_moving_avrage.set(r, [&ctx] (std::unique_ptr<request> req) {
// TBD
// FIXME
// See above
return make_ready_future<json::json_return_type>(meter_to_json(utils::rate_moving_average()));
});
cs::get_counter_requests_moving_avrage.set(r, [] (std::unique_ptr<http::request> req) {
cs::get_counter_requests_moving_avrage.set(r, [&ctx] (std::unique_ptr<request> req) {
// TBD
// FIXME
// See above
return make_ready_future<json::json_return_type>(meter_to_json(utils::rate_moving_average()));
});
cs::get_counter_size.set(r, [] (std::unique_ptr<http::request> req) {
cs::get_counter_size.set(r, [] (std::unique_ptr<request> req) {
// TBD
// FIXME
// we don't support counter cache,
@@ -310,7 +316,7 @@ void set_cache_service(http_context& ctx, routes& r) {
return make_ready_future<json::json_return_type>(0);
});
cs::get_counter_entries.set(r, [] (std::unique_ptr<http::request> req) {
cs::get_counter_entries.set(r, [] (std::unique_ptr<request> req) {
// TBD
// FIXME
// we don't support counter cache,

View File

@@ -1,9 +1,22 @@
/*
* Copyright (C) 2015-present ScyllaDB
* Copyright (C) 2015 ScyllaDB
*/
/*
* SPDX-License-Identifier: AGPL-3.0-or-later
* 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
@@ -12,6 +25,6 @@
namespace api {
void set_cache_service(http_context& ctx, httpd::routes& r);
void set_cache_service(http_context& ctx, routes& r);
}

View File

@@ -1,15 +1,28 @@
/*
* Copyright (C) 2015-present ScyllaDB
* Copyright (C) 2015 ScyllaDB
*/
/*
* SPDX-License-Identifier: AGPL-3.0-or-later
* 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 "collectd.hh"
#include "api/api-doc/collectd.json.hh"
#include <seastar/core/scollectd.hh>
#include <seastar/core/scollectd_api.hh>
#include "core/scollectd.hh"
#include "core/scollectd_api.hh"
#include "endian.h"
#include <boost/range/irange.hpp>
#include <regex>
@@ -29,11 +42,8 @@ static auto transformer(const std::vector<collectd_value>& values) {
case scollectd::data_type::GAUGE:
collected_value.values.push(v.d());
break;
case scollectd::data_type::COUNTER:
collected_value.values.push(v.ui());
break;
case scollectd::data_type::REAL_COUNTER:
collected_value.values.push(v.d());
case scollectd::data_type::DERIVE:
collected_value.values.push(v.i());
break;
default:
collected_value.values.push(v.ui());
@@ -52,9 +62,9 @@ static const char* str_to_regex(const sstring& v) {
}
void set_collectd(http_context& ctx, routes& r) {
cd::get_collectd.set(r, [](std::unique_ptr<request> req) {
cd::get_collectd.set(r, [&ctx](std::unique_ptr<request> req) {
auto id = ::make_shared<scollectd::type_instance_id>(req->param["pluginid"],
auto id = make_shared<scollectd::type_instance_id>(req->param["pluginid"],
req->get_query_param("instance"), req->get_query_param("type"),
req->get_query_param("type_instance"));

View File

@@ -1,9 +1,22 @@
/*
* Copyright (C) 2015-present ScyllaDB
* Copyright (C) 2015 ScyllaDB
*/
/*
* SPDX-License-Identifier: AGPL-3.0-or-later
* 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
@@ -12,6 +25,6 @@
namespace api {
void set_collectd(http_context& ctx, httpd::routes& r);
void set_collectd(http_context& ctx, routes& r);
}

File diff suppressed because it is too large Load Diff

View File

@@ -1,44 +1,51 @@
/*
* Copyright (C) 2015-present ScyllaDB
* Copyright (C) 2015 ScyllaDB
*/
/*
* SPDX-License-Identifier: AGPL-3.0-or-later
* 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 "api.hh"
#include "api/api-doc/column_family.json.hh"
#include "replica/database.hh"
#include <seastar/core/future-util.hh>
#include "database.hh"
#include <any>
namespace db {
class system_keyspace;
}
namespace api {
void set_column_family(http_context& ctx, httpd::routes& r, sharded<db::system_keyspace>& sys_ks);
void unset_column_family(http_context& ctx, httpd::routes& r);
void set_column_family(http_context& ctx, routes& r);
const table_id& get_uuid(const sstring& name, const replica::database& db);
future<> foreach_column_family(http_context& ctx, const sstring& name, std::function<void(replica::column_family&)> f);
const utils::UUID& get_uuid(const sstring& name, const database& db);
future<> foreach_column_family(http_context& ctx, const sstring& name, std::function<void(column_family&)> f);
template<class Mapper, class I, class Reducer>
future<I> map_reduce_cf_raw(http_context& ctx, const sstring& name, I init,
Mapper mapper, Reducer reducer) {
auto uuid = get_uuid(name, ctx.db.local());
using mapper_type = std::function<std::unique_ptr<std::any>(replica::database&)>;
using reducer_type = std::function<std::unique_ptr<std::any>(std::unique_ptr<std::any>, std::unique_ptr<std::any>)>;
return ctx.db.map_reduce0(mapper_type([mapper, uuid](replica::database& db) {
return std::make_unique<std::any>(I(mapper(db.find_column_family(uuid))));
}), std::make_unique<std::any>(std::move(init)), reducer_type([reducer = std::move(reducer)] (std::unique_ptr<std::any> a, std::unique_ptr<std::any> b) mutable {
return std::make_unique<std::any>(I(reducer(std::any_cast<I>(std::move(*a)), std::any_cast<I>(std::move(*b)))));
})).then([] (std::unique_ptr<std::any> r) {
return std::any_cast<I>(std::move(*r));
using mapper_type = std::function<std::any (database&)>;
using reducer_type = std::function<std::any (std::any, std::any)>;
return ctx.db.map_reduce0(mapper_type([mapper, uuid](database& db) {
return I(mapper(db.find_column_family(uuid)));
}), std::any(std::move(init)), reducer_type([reducer = std::move(reducer)] (std::any a, std::any b) mutable {
return I(reducer(std::any_cast<I>(std::move(a)), std::any_cast<I>(std::move(b))));
})).then([] (std::any r) {
return std::any_cast<I>(std::move(r));
});
}
@@ -60,36 +67,32 @@ future<json::json_return_type> map_reduce_cf(http_context& ctx, const sstring& n
});
}
future<json::json_return_type> map_reduce_cf_time_histogram(http_context& ctx, const sstring& name, std::function<utils::time_estimated_histogram(const replica::column_family&)> f);
struct map_reduce_column_families_locally {
std::any init;
std::function<std::unique_ptr<std::any>(replica::column_family&)> mapper;
std::function<std::unique_ptr<std::any>(std::unique_ptr<std::any>, std::unique_ptr<std::any>)> reducer;
future<std::unique_ptr<std::any>> operator()(replica::database& db) const {
auto res = seastar::make_lw_shared<std::unique_ptr<std::any>>(std::make_unique<std::any>(init));
return do_for_each(db.get_column_families(), [res, this](const std::pair<table_id, seastar::lw_shared_ptr<replica::table>>& i) {
*res = reducer(std::move(*res), mapper(*i.second.get()));
}).then([res] {
return std::move(*res);
});
std::function<std::any (column_family&)> mapper;
std::function<std::any (std::any, std::any)> reducer;
std::any operator()(database& db) const {
auto res = init;
for (auto i : db.get_column_families()) {
res = reducer(res, mapper(*i.second.get()));
}
return res;
}
};
template<class Mapper, class I, class Reducer>
future<I> map_reduce_cf_raw(http_context& ctx, I init,
Mapper mapper, Reducer reducer) {
using mapper_type = std::function<std::unique_ptr<std::any>(replica::column_family&)>;
using reducer_type = std::function<std::unique_ptr<std::any>(std::unique_ptr<std::any>, std::unique_ptr<std::any>)>;
auto wrapped_mapper = mapper_type([mapper = std::move(mapper)] (replica::column_family& cf) mutable {
return std::make_unique<std::any>(I(mapper(cf)));
using mapper_type = std::function<std::any (column_family&)>;
using reducer_type = std::function<std::any (std::any, std::any)>;
auto wrapped_mapper = mapper_type([mapper = std::move(mapper)] (column_family& cf) mutable {
return I(mapper(cf));
});
auto wrapped_reducer = reducer_type([reducer = std::move(reducer)] (std::unique_ptr<std::any> a, std::unique_ptr<std::any> b) mutable {
return std::make_unique<std::any>(I(reducer(std::any_cast<I>(std::move(*a)), std::any_cast<I>(std::move(*b)))));
auto wrapped_reducer = reducer_type([reducer = std::move(reducer)] (std::any a, std::any b) mutable {
return I(reducer(std::any_cast<I>(std::move(a)), std::any_cast<I>(std::move(b))));
});
return ctx.db.map_reduce0(map_reduce_column_families_locally{init,
std::move(wrapped_mapper), wrapped_reducer}, std::make_unique<std::any>(init), wrapped_reducer).then([] (std::unique_ptr<std::any> res) {
return std::any_cast<I>(std::move(*res));
return ctx.db.map_reduce0(map_reduce_column_families_locally{init, std::move(wrapped_mapper), wrapped_reducer}, std::any(init), wrapped_reducer).then([] (std::any res) {
return std::any_cast<I>(std::move(res));
});
}
@@ -103,12 +106,9 @@ future<json::json_return_type> map_reduce_cf(http_context& ctx, I init,
}
future<json::json_return_type> get_cf_stats(http_context& ctx, const sstring& name,
int64_t replica::column_family_stats::*f);
int64_t column_family::stats::*f);
future<json::json_return_type> get_cf_stats(http_context& ctx,
int64_t replica::column_family_stats::*f);
std::tuple<sstring, sstring> parse_fully_qualified_cf_name(sstring name);
int64_t column_family::stats::*f);
}

View File

@@ -1,25 +1,36 @@
/*
* Copyright (C) 2015-present ScyllaDB
* Copyright (C) 2015 ScyllaDB
*/
/*
* SPDX-License-Identifier: AGPL-3.0-or-later
* 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 "commitlog.hh"
#include "db/commitlog/commitlog.hh"
#include <db/commitlog/commitlog.hh>
#include "api/api-doc/commitlog.json.hh"
#include "replica/database.hh"
#include <vector>
namespace api {
using namespace seastar::httpd;
template<typename T>
static auto acquire_cl_metric(http_context& ctx, std::function<T (db::commitlog*)> func) {
typedef T ret_type;
template<typename Func>
static auto acquire_cl_metric(http_context& ctx, Func&& func) {
typedef std::result_of_t<Func(db::commitlog *)> ret_type;
return ctx.db.map_reduce0([func = std::move(func)](replica::database& db) {
return ctx.db.map_reduce0([func = std::forward<Func>(func)](database& db) {
if (db.commitlog() == nullptr) {
return make_ready_future<ret_type>();
}
@@ -35,7 +46,7 @@ void set_commitlog(http_context& ctx, routes& r) {
auto res = make_shared<std::vector<sstring>>();
return ctx.db.map_reduce([res](std::vector<sstring> names) {
res->insert(res->end(), names.begin(), names.end());
}, [](replica::database& db) {
}, [](database& db) {
if (db.commitlog() == nullptr) {
return make_ready_future<std::vector<sstring>>(std::vector<sstring>());
}
@@ -52,15 +63,15 @@ void set_commitlog(http_context& ctx, routes& r) {
});
httpd::commitlog_json::get_completed_tasks.set(r, [&ctx](std::unique_ptr<request> req) {
return acquire_cl_metric<uint64_t>(ctx, std::bind(&db::commitlog::get_completed_tasks, std::placeholders::_1));
return acquire_cl_metric(ctx, std::bind(&db::commitlog::get_completed_tasks, std::placeholders::_1));
});
httpd::commitlog_json::get_pending_tasks.set(r, [&ctx](std::unique_ptr<request> req) {
return acquire_cl_metric<uint64_t>(ctx, std::bind(&db::commitlog::get_pending_tasks, std::placeholders::_1));
return acquire_cl_metric(ctx, std::bind(&db::commitlog::get_pending_tasks, std::placeholders::_1));
});
httpd::commitlog_json::get_total_commit_log_size.set(r, [&ctx](std::unique_ptr<request> req) {
return acquire_cl_metric<uint64_t>(ctx, std::bind(&db::commitlog::get_total_size, std::placeholders::_1));
return acquire_cl_metric(ctx, std::bind(&db::commitlog::get_total_size, std::placeholders::_1));
});
}

View File

@@ -1,9 +1,22 @@
/*
* Copyright (C) 2015-present ScyllaDB
* Copyright (C) 2015 ScyllaDB
*/
/*
* SPDX-License-Identifier: AGPL-3.0-or-later
* 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
@@ -12,6 +25,6 @@
namespace api {
void set_commitlog(http_context& ctx, httpd::routes& r);
void set_commitlog(http_context& ctx, routes& r);
}

View File

@@ -1,62 +1,58 @@
/*
* Copyright (C) 2015-present ScyllaDB
* Copyright (C) 2015 ScyllaDB
*/
/*
* SPDX-License-Identifier: AGPL-3.0-or-later
* 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 <seastar/core/coroutine.hh>
#include "compaction_manager.hh"
#include "compaction/compaction_manager.hh"
#include "sstables/compaction_manager.hh"
#include "api/api-doc/compaction_manager.json.hh"
#include "db/system_keyspace.hh"
#include "column_family.hh"
#include "unimplemented.hh"
#include "storage_service.hh"
#include <utility>
namespace api {
namespace cm = httpd::compaction_manager_json;
using namespace json;
using namespace seastar::httpd;
static future<json::json_return_type> get_cm_stats(http_context& ctx,
int64_t compaction_manager::stats::*f) {
return ctx.db.map_reduce0([f](replica::database& db) {
return ctx.db.map_reduce0([f](database& db) {
return db.get_compaction_manager().get_stats().*f;
}, int64_t(0), std::plus<int64_t>()).then([](const int64_t& res) {
return make_ready_future<json::json_return_type>(res);
});
}
static std::unordered_map<std::pair<sstring, sstring>, uint64_t, utils::tuple_hash> sum_pending_tasks(std::unordered_map<std::pair<sstring, sstring>, uint64_t, utils::tuple_hash>&& a,
const std::unordered_map<std::pair<sstring, sstring>, uint64_t, utils::tuple_hash>& b) {
for (auto&& i : b) {
if (i.second) {
a[i.first] += i.second;
}
}
return std::move(a);
}
void set_compaction_manager(http_context& ctx, routes& r) {
cm::get_compactions.set(r, [&ctx] (std::unique_ptr<http::request> req) {
return ctx.db.map_reduce0([](replica::database& db) {
cm::get_compactions.set(r, [&ctx] (std::unique_ptr<request> req) {
return ctx.db.map_reduce0([](database& db) {
std::vector<cm::summary> summaries;
const compaction_manager& cm = db.get_compaction_manager();
for (const auto& c : cm.get_compactions()) {
cm::summary s;
s.id = c.compaction_uuid.to_sstring();
s.ks = c.ks_name;
s.cf = c.cf_name;
s.ks = c->ks;
s.cf = c->cf;
s.unit = "keys";
s.task_type = sstables::compaction_name(c.type);
s.completed = c.total_keys_written;
s.total = c.total_partitions;
s.task_type = sstables::compaction_name(c->type);
s.completed = c->total_keys_written;
s.total = c->total_partitions;
summaries.push_back(std::move(s));
}
return summaries;
@@ -65,127 +61,74 @@ void set_compaction_manager(http_context& ctx, routes& r) {
});
});
cm::get_pending_tasks_by_table.set(r, [&ctx] (std::unique_ptr<http::request> req) {
return ctx.db.map_reduce0([](replica::database& db) {
return do_with(std::unordered_map<std::pair<sstring, sstring>, uint64_t, utils::tuple_hash>(), [&db](std::unordered_map<std::pair<sstring, sstring>, uint64_t, utils::tuple_hash>& tasks) {
return do_for_each(db.get_column_families(), [&tasks](const std::pair<table_id, seastar::lw_shared_ptr<replica::table>>& i) -> future<> {
replica::table& cf = *i.second.get();
tasks[std::make_pair(cf.schema()->ks_name(), cf.schema()->cf_name())] = cf.estimate_pending_compactions();
return make_ready_future<>();
}).then([&tasks] {
return std::move(tasks);
});
});
}, std::unordered_map<std::pair<sstring, sstring>, uint64_t, utils::tuple_hash>(), sum_pending_tasks).then(
[](const std::unordered_map<std::pair<sstring, sstring>, uint64_t, utils::tuple_hash>& task_map) {
std::vector<cm::pending_compaction> res;
res.reserve(task_map.size());
for (auto i : task_map) {
cm::pending_compaction task;
task.ks = i.first.first;
task.cf = i.first.second;
task.task = i.second;
res.emplace_back(std::move(task));
}
return make_ready_future<json::json_return_type>(res);
});
});
cm::force_user_defined_compaction.set(r, [] (std::unique_ptr<http::request> req) {
cm::force_user_defined_compaction.set(r, [] (std::unique_ptr<request> req) {
//TBD
// FIXME
warn(unimplemented::cause::API);
return make_ready_future<json::json_return_type>(json_void());
});
cm::stop_compaction.set(r, [&ctx] (std::unique_ptr<http::request> req) {
cm::stop_compaction.set(r, [&ctx] (std::unique_ptr<request> req) {
auto type = req->get_query_param("type");
return ctx.db.invoke_on_all([type] (replica::database& db) {
return ctx.db.invoke_on_all([type] (database& db) {
auto& cm = db.get_compaction_manager();
return cm.stop_compaction(type);
cm.stop_compaction(type);
}).then([] {
return make_ready_future<json::json_return_type>(json_void());
});
});
cm::stop_keyspace_compaction.set(r, [&ctx] (std::unique_ptr<http::request> req) -> future<json::json_return_type> {
auto ks_name = validate_keyspace(ctx, req->param);
auto table_names = parse_tables(ks_name, ctx, req->query_parameters, "tables");
if (table_names.empty()) {
table_names = map_keys(ctx.db.local().find_keyspace(ks_name).metadata().get()->cf_meta_data());
}
auto type = req->get_query_param("type");
co_await ctx.db.invoke_on_all([&ks_name, &table_names, type] (replica::database& db) {
auto& cm = db.get_compaction_manager();
return parallel_for_each(table_names, [&db, &cm, &ks_name, type] (sstring& table_name) {
auto& t = db.find_column_family(ks_name, table_name);
return t.parallel_foreach_table_state([&] (compaction::table_state& ts) {
return cm.stop_compaction(type, &ts);
});
});
});
co_return json_void();
});
cm::get_pending_tasks.set(r, [&ctx] (std::unique_ptr<http::request> req) {
return map_reduce_cf(ctx, int64_t(0), [](replica::column_family& cf) {
return cf.estimate_pending_compactions();
cm::get_pending_tasks.set(r, [&ctx] (std::unique_ptr<request> req) {
return map_reduce_cf(ctx, int64_t(0), [](column_family& cf) {
return cf.get_compaction_strategy().estimated_pending_compactions(cf);
}, std::plus<int64_t>());
});
cm::get_completed_tasks.set(r, [&ctx] (std::unique_ptr<http::request> req) {
cm::get_completed_tasks.set(r, [&ctx] (std::unique_ptr<request> req) {
return get_cm_stats(ctx, &compaction_manager::stats::completed_tasks);
});
cm::get_total_compactions_completed.set(r, [] (std::unique_ptr<http::request> req) {
cm::get_total_compactions_completed.set(r, [] (std::unique_ptr<request> req) {
// FIXME
// We are currently dont have an API for compaction
// so returning a 0 as the number of total compaction is ok
return make_ready_future<json::json_return_type>(0);
});
cm::get_bytes_compacted.set(r, [] (std::unique_ptr<http::request> req) {
cm::get_bytes_compacted.set(r, [] (std::unique_ptr<request> req) {
//TBD
// FIXME
warn(unimplemented::cause::API);
return make_ready_future<json::json_return_type>(0);
});
cm::get_compaction_history.set(r, [&ctx] (std::unique_ptr<http::request> req) {
std::function<future<>(output_stream<char>&&)> f = [&ctx](output_stream<char>&& s) {
return do_with(output_stream<char>(std::move(s)), true, [&ctx] (output_stream<char>& s, bool& first){
return s.write("[").then([&ctx, &s, &first] {
return ctx.db.local().get_compaction_manager().get_compaction_history([&s, &first](const db::compaction_history_entry& entry) mutable {
cm::history h;
h.id = entry.id.to_sstring();
h.ks = std::move(entry.ks);
h.cf = std::move(entry.cf);
h.compacted_at = entry.compacted_at;
h.bytes_in = entry.bytes_in;
h.bytes_out = entry.bytes_out;
for (auto it : entry.rows_merged) {
httpd::compaction_manager_json::row_merged e;
e.key = it.first;
e.value = it.second;
h.rows_merged.push(std::move(e));
}
auto fut = first ? make_ready_future<>() : s.write(", ");
first = false;
return fut.then([&s, h = std::move(h)] {
return formatter::write(s, h);
});
}).then([&s] {
return s.write("]").then([&s] {
return s.close();
});
});
});
});
};
return make_ready_future<json::json_return_type>(std::move(f));
cm::get_compaction_history.set(r, [] (std::unique_ptr<request> req) {
return db::system_keyspace::get_compaction_history().then([] (std::vector<db::system_keyspace::compaction_history_entry> history) {
std::vector<cm::history> res;
res.reserve(history.size());
for (auto& entry : history) {
cm::history h;
h.id = entry.id.to_sstring();
h.ks = std::move(entry.ks);
h.cf = std::move(entry.cf);
h.compacted_at = entry.compacted_at;
h.bytes_in = entry.bytes_in;
h.bytes_out = entry.bytes_out;
for (auto it : entry.rows_merged) {
httpd::compaction_manager_json::row_merged e;
e.key = it.first;
e.value = it.second;
h.rows_merged.push(std::move(e));
}
res.push_back(std::move(h));
}
return make_ready_future<json::json_return_type>(res);
});
});
cm::get_compaction_info.set(r, [] (std::unique_ptr<http::request> req) {
cm::get_compaction_info.set(r, [] (std::unique_ptr<request> req) {
//TBD
// FIXME
warn(unimplemented::cause::API);

View File

@@ -1,9 +1,22 @@
/*
* Copyright (C) 2015-present ScyllaDB
* Copyright (C) 2015 ScyllaDB
*/
/*
* SPDX-License-Identifier: AGPL-3.0-or-later
* 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
@@ -12,6 +25,6 @@
namespace api {
void set_compaction_manager(http_context& ctx, httpd::routes& r);
void set_compaction_manager(http_context& ctx, routes& r);
}

View File

@@ -1,9 +1,22 @@
/*
* Copyright 2018-present ScyllaDB
* Copyright 2018 ScyllaDB
*/
/*
* SPDX-License-Identifier: AGPL-3.0-or-later
* 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 "api/config.hh"
@@ -13,7 +26,6 @@
#include <boost/algorithm/string/replace.hpp>
namespace api {
using namespace seastar::httpd;
template<class T>
json::json_return_type get_json_return_type(const T& val) {
@@ -31,14 +43,14 @@ json::json_return_type get_json_return_type(const db::seed_provider_type& val) {
return json::json_return_type(val.class_name);
}
std::string_view format_type(std::string_view type) {
std::string format_type(const std::string& type) {
if (type == "int") {
return "integer";
}
return type;
}
future<> get_config_swagger_entry(std::string_view name, const std::string& description, std::string_view type, bool& first, output_stream<char>& os) {
future<> get_config_swagger_entry(const std::string& name, const std::string& description, const std::string& type, bool& first, output_stream<char>& os) {
std::stringstream ss;
if (first) {
first=false;
@@ -75,29 +87,23 @@ future<> get_config_swagger_entry(std::string_view name, const std::string& desc
}
namespace cs = httpd::config_json;
#define _get_config_value(name, type, deflt, status, desc, ...) if (id == #name) {return get_json_return_type(ctx.db.local().get_config().name());}
void set_config(std::shared_ptr < api_registry_builder20 > rb, http_context& ctx, routes& r, const db::config& cfg) {
rb->register_function(r, [&cfg] (output_stream<char>& os) {
return do_with(true, [&os, &cfg] (bool& first) {
#define _get_config_description(name, type, deflt, status, desc, ...) f = f.then([&os, &first] {return get_config_swagger_entry(#name, desc, #type, first, os);});
void set_config(std::shared_ptr < api_registry_builder20 > rb, http_context& ctx, routes& r) {
rb->register_function(r, [] (output_stream<char>& os) {
return do_with(true, [&os] (bool& first) {
auto f = make_ready_future();
for (auto&& cfg_ref : cfg.values()) {
auto&& cfg = cfg_ref.get();
f = f.then([&os, &first, &cfg] {
return get_config_swagger_entry(cfg.name(), std::string(cfg.desc()), cfg.type_name(), first, os);
});
}
_make_config_values(_get_config_description)
return f;
});
});
cs::find_config_id.set(r, [&cfg] (const_req r) {
cs::find_config_id.set(r, [&ctx] (const_req r) {
auto id = r.param["id"];
for (auto&& cfg_ref : cfg.values()) {
auto&& cfg = cfg_ref.get();
if (id == cfg.name()) {
return cfg.value_as_json();
}
}
_make_config_values(_get_config_value)
throw bad_param_exception(sstring("No such config entry: ") + id);
});
}

View File

@@ -1,9 +1,22 @@
/*
* Copyright (C) 2018-present ScyllaDB
* Copyright (C) 2018 ScyllaDB
*/
/*
* SPDX-License-Identifier: AGPL-3.0-or-later
* 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
@@ -13,5 +26,5 @@
namespace api {
void set_config(std::shared_ptr<httpd::api_registry_builder20> rb, http_context& ctx, httpd::routes& r, const db::config& cfg);
void set_config(std::shared_ptr<api_registry_builder20> rb, http_context& ctx, routes& r);
}

View File

@@ -1,69 +1,48 @@
/*
* Copyright (C) 2015-present ScyllaDB
* Copyright (C) 2015 ScyllaDB
*/
/*
* SPDX-License-Identifier: AGPL-3.0-or-later
* 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 "locator/token_metadata.hh"
#include "locator/snitch_base.hh"
#include "locator/production_snitch_base.hh"
#include "endpoint_snitch.hh"
#include "api/api-doc/endpoint_snitch_info.json.hh"
#include "api/api-doc/storage_service.json.hh"
#include "utils/fb_utilities.hh"
namespace api {
using namespace seastar::httpd;
void set_endpoint_snitch(http_context& ctx, routes& r, sharded<locator::snitch_ptr>& snitch) {
void set_endpoint_snitch(http_context& ctx, routes& r) {
static auto host_or_broadcast = [](const_req req) {
auto host = req.get_query_param("host");
return host.empty() ? gms::inet_address(utils::fb_utilities::get_broadcast_address()) : gms::inet_address(host);
};
httpd::endpoint_snitch_info_json::get_datacenter.set(r, [&ctx](const_req req) {
auto& topology = ctx.shared_token_metadata.local().get()->get_topology();
auto ep = host_or_broadcast(req);
if (!topology.has_endpoint(ep)) {
// Cannot return error here, nodetool status can race, request
// info about just-left node and not handle it nicely
return locator::endpoint_dc_rack::default_location.dc;
}
return topology.get_datacenter(ep);
httpd::endpoint_snitch_info_json::get_datacenter.set(r, [](const_req req) {
return locator::i_endpoint_snitch::get_local_snitch_ptr()->get_datacenter(host_or_broadcast(req));
});
httpd::endpoint_snitch_info_json::get_rack.set(r, [&ctx](const_req req) {
auto& topology = ctx.shared_token_metadata.local().get()->get_topology();
auto ep = host_or_broadcast(req);
if (!topology.has_endpoint(ep)) {
// Cannot return error here, nodetool status can race, request
// info about just-left node and not handle it nicely
return locator::endpoint_dc_rack::default_location.rack;
}
return topology.get_rack(ep);
httpd::endpoint_snitch_info_json::get_rack.set(r, [](const_req req) {
return locator::i_endpoint_snitch::get_local_snitch_ptr()->get_rack(host_or_broadcast(req));
});
httpd::endpoint_snitch_info_json::get_snitch_name.set(r, [&snitch] (const_req req) {
return snitch.local()->get_name();
httpd::endpoint_snitch_info_json::get_snitch_name.set(r, [] (const_req req) {
return locator::i_endpoint_snitch::get_local_snitch_ptr()->get_name();
});
httpd::storage_service_json::update_snitch.set(r, [&snitch](std::unique_ptr<request> req) {
locator::snitch_config cfg;
cfg.name = req->get_query_param("ep_snitch_class_name");
return locator::i_endpoint_snitch::reset_snitch(snitch, cfg).then([] {
return make_ready_future<json::json_return_type>(json::json_void());
});
});
}
void unset_endpoint_snitch(http_context& ctx, routes& r) {
httpd::endpoint_snitch_info_json::get_datacenter.unset(r);
httpd::endpoint_snitch_info_json::get_rack.unset(r);
httpd::endpoint_snitch_info_json::get_snitch_name.unset(r);
httpd::storage_service_json::update_snitch.unset(r);
}
}

View File

@@ -1,22 +1,30 @@
/*
* Copyright (C) 2015-present ScyllaDB
* Copyright (C) 2015 ScyllaDB
*/
/*
* SPDX-License-Identifier: AGPL-3.0-or-later
* 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 "api.hh"
namespace locator {
class snitch_ptr;
}
namespace api {
void set_endpoint_snitch(http_context& ctx, httpd::routes& r, sharded<locator::snitch_ptr>&);
void unset_endpoint_snitch(http_context& ctx, httpd::routes& r);
void set_endpoint_snitch(http_context& ctx, routes& r);
}

View File

@@ -1,57 +0,0 @@
/*
* Copyright (C) 2020-present ScyllaDB
*/
/*
* SPDX-License-Identifier: AGPL-3.0-or-later
*/
#include "api/api-doc/error_injection.json.hh"
#include "api/api.hh"
#include <seastar/http/exception.hh>
#include "log.hh"
#include "utils/error_injection.hh"
#include <seastar/core/future-util.hh>
namespace api {
using namespace seastar::httpd;
namespace hf = httpd::error_injection_json;
void set_error_injection(http_context& ctx, routes& r) {
hf::enable_injection.set(r, [](std::unique_ptr<request> req) {
sstring injection = req->param["injection"];
bool one_shot = req->get_query_param("one_shot") == "True";
auto& errinj = utils::get_local_injector();
return errinj.enable_on_all(injection, one_shot).then([] {
return make_ready_future<json::json_return_type>(json::json_void());
});
});
hf::get_enabled_injections_on_all.set(r, [](std::unique_ptr<request> req) {
auto& errinj = utils::get_local_injector();
auto ret = errinj.enabled_injections_on_all();
return make_ready_future<json::json_return_type>(ret);
});
hf::disable_injection.set(r, [](std::unique_ptr<request> req) {
sstring injection = req->param["injection"];
auto& errinj = utils::get_local_injector();
return errinj.disable_on_all(injection).then([] {
return make_ready_future<json::json_return_type>(json::json_void());
});
});
hf::disable_on_all.set(r, [](std::unique_ptr<request> req) {
auto& errinj = utils::get_local_injector();
return errinj.disable_on_all().then([] {
return make_ready_future<json::json_return_type>(json::json_void());
});
});
}
} // namespace api

View File

@@ -1,17 +0,0 @@
/*
* Copyright (C) 2019-present ScyllaDB
*/
/*
* SPDX-License-Identifier: AGPL-3.0-or-later
*/
#pragma once
#include "api.hh"
namespace api {
void set_error_injection(http_context& ctx, httpd::routes& r);
}

View File

@@ -1,38 +1,50 @@
/*
* Copyright (C) 2015-present ScyllaDB
* Copyright (C) 2015 ScyllaDB
*/
/*
* SPDX-License-Identifier: AGPL-3.0-or-later
* 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 "failure_detector.hh"
#include "api/api-doc/failure_detector.json.hh"
#include "gms/failure_detector.hh"
#include "gms/application_state.hh"
#include "gms/gossiper.hh"
namespace api {
using namespace seastar::httpd;
namespace fd = httpd::failure_detector_json;
void set_failure_detector(http_context& ctx, routes& r, gms::gossiper& g) {
fd::get_all_endpoint_states.set(r, [&g](std::unique_ptr<request> req) {
void set_failure_detector(http_context& ctx, routes& r) {
fd::get_all_endpoint_states.set(r, [](std::unique_ptr<request> req) {
std::vector<fd::endpoint_state> res;
for (auto i : g.get_endpoint_states()) {
for (auto i : gms::get_local_gossiper().endpoint_state_map) {
fd::endpoint_state val;
val.addrs = fmt::to_string(i.first);
val.addrs = boost::lexical_cast<std::string>(i.first);
val.is_alive = i.second.is_alive();
val.generation = i.second.get_heart_beat_state().get_generation().value();
val.version = i.second.get_heart_beat_state().get_heart_beat_version().value();
val.generation = i.second.get_heart_beat_state().get_generation();
val.version = i.second.get_heart_beat_state().get_heart_beat_version();
val.update_time = i.second.get_update_timestamp().time_since_epoch().count();
for (auto a : i.second.get_application_state_map()) {
fd::version_value version_val;
// We return the enum index and not it's name to stay compatible to origin
// method that the state index are static but the name can be changed.
version_val.application_state = static_cast<std::underlying_type<gms::application_state>::type>(a.first);
version_val.value = a.second.value();
version_val.version = a.second.version().value();
version_val.value = a.second.value;
version_val.version = a.second.version;
val.application_state.push(version_val);
}
res.push_back(val);
@@ -40,50 +52,55 @@ void set_failure_detector(http_context& ctx, routes& r, gms::gossiper& g) {
return make_ready_future<json::json_return_type>(res);
});
fd::get_up_endpoint_count.set(r, [&g](std::unique_ptr<request> req) {
int res = g.get_up_endpoint_count();
return make_ready_future<json::json_return_type>(res);
fd::get_up_endpoint_count.set(r, [](std::unique_ptr<request> req) {
return gms::get_up_endpoint_count().then([](int res) {
return make_ready_future<json::json_return_type>(res);
});
});
fd::get_down_endpoint_count.set(r, [&g](std::unique_ptr<request> req) {
int res = g.get_down_endpoint_count();
return make_ready_future<json::json_return_type>(res);
fd::get_down_endpoint_count.set(r, [](std::unique_ptr<request> req) {
return gms::get_down_endpoint_count().then([](int res) {
return make_ready_future<json::json_return_type>(res);
});
});
fd::get_phi_convict_threshold.set(r, [] (std::unique_ptr<request> req) {
return make_ready_future<json::json_return_type>(8);
return gms::get_phi_convict_threshold().then([](double res) {
return make_ready_future<json::json_return_type>(res);
});
});
fd::get_simple_states.set(r, [&g] (std::unique_ptr<request> req) {
std::map<sstring, sstring> nodes_status;
for (auto& entry : g.get_endpoint_states()) {
nodes_status.emplace(entry.first.to_sstring(), entry.second.is_alive() ? "UP" : "DOWN");
}
return make_ready_future<json::json_return_type>(map_to_key_value<fd::mapper>(nodes_status));
fd::get_simple_states.set(r, [] (std::unique_ptr<request> req) {
return gms::get_simple_states().then([](const std::map<sstring, sstring>& map) {
return make_ready_future<json::json_return_type>(map_to_key_value<fd::mapper>(map));
});
});
fd::set_phi_convict_threshold.set(r, [](std::unique_ptr<request> req) {
// TBD
unimplemented();
std::ignore = atof(req->get_query_param("phi").c_str());
return make_ready_future<json::json_return_type>("");
double phi = atof(req->get_query_param("phi").c_str());
return gms::set_phi_convict_threshold(phi).then([]() {
return make_ready_future<json::json_return_type>("");
});
});
fd::get_endpoint_state.set(r, [&g] (std::unique_ptr<request> req) {
auto* state = g.get_endpoint_state_for_endpoint_ptr(gms::inet_address(req->param["addr"]));
if (!state) {
return make_ready_future<json::json_return_type>(format("unknown endpoint {}", req->param["addr"]));
}
std::stringstream ss;
g.append_endpoint_state(ss, *state);
return make_ready_future<json::json_return_type>(sstring(ss.str()));
fd::get_endpoint_state.set(r, [](std::unique_ptr<request> req) {
return gms::get_endpoint_state(req->param["addr"]).then([](const sstring& state) {
return make_ready_future<json::json_return_type>(state);
});
});
fd::get_endpoint_phi_values.set(r, [](std::unique_ptr<request> req) {
// We no longer have a phi failure detector,
// just returning the empty value is good enough.
std::vector<fd::endpoint_phi_value> res;
return make_ready_future<json::json_return_type>(res);
return gms::get_arrival_samples().then([](std::map<gms::inet_address, gms::arrival_window> map) {
std::vector<fd::endpoint_phi_value> res;
auto now = gms::arrival_window::clk::now();
for (auto& p : map) {
fd::endpoint_phi_value val;
val.endpoint = p.first.to_sstring();
val.phi = p.second.phi(now);
res.emplace_back(std::move(val));
}
return make_ready_future<json::json_return_type>(res);
});
});
}

View File

@@ -1,23 +1,30 @@
/*
* Copyright (C) 2015-present ScyllaDB
* Copyright (C) 2015 ScyllaDB
*/
/*
* SPDX-License-Identifier: AGPL-3.0-or-later
* 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 "api.hh"
namespace gms {
class gossiper;
}
namespace api {
void set_failure_detector(http_context& ctx, httpd::routes& r, gms::gossiper& g);
void set_failure_detector(http_context& ctx, routes& r);
}

View File

@@ -1,65 +1,68 @@
/*
* Copyright (C) 2015-present ScyllaDB
* Copyright (C) 2015 ScyllaDB
*/
/*
* SPDX-License-Identifier: AGPL-3.0-or-later
* 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 "gossiper.hh"
#include "api/api-doc/gossiper.json.hh"
#include "gms/gossiper.hh"
#include <gms/gossiper.hh>
namespace api {
using namespace seastar::httpd;
using namespace json;
void set_gossiper(http_context& ctx, routes& r, gms::gossiper& g) {
httpd::gossiper_json::get_down_endpoint.set(r, [&g] (const_req req) {
auto res = g.get_unreachable_members();
void set_gossiper(http_context& ctx, routes& r) {
httpd::gossiper_json::get_down_endpoint.set(r, [] (const_req req) {
auto res = gms::get_local_gossiper().get_unreachable_members();
return container_to_vec(res);
});
httpd::gossiper_json::get_live_endpoint.set(r, [&g] (std::unique_ptr<request> req) {
return g.get_live_members_synchronized().then([] (auto res) {
return make_ready_future<json::json_return_type>(container_to_vec(res));
});
httpd::gossiper_json::get_live_endpoint.set(r, [] (const_req req) {
auto res = gms::get_local_gossiper().get_live_members();
return container_to_vec(res);
});
httpd::gossiper_json::get_endpoint_downtime.set(r, [&g] (const_req req) {
httpd::gossiper_json::get_endpoint_downtime.set(r, [] (const_req req) {
gms::inet_address ep(req.param["addr"]);
return g.get_endpoint_downtime(ep);
return gms::get_local_gossiper().get_endpoint_downtime(ep);
});
httpd::gossiper_json::get_current_generation_number.set(r, [&g] (std::unique_ptr<http::request> req) {
httpd::gossiper_json::get_current_generation_number.set(r, [] (std::unique_ptr<request> req) {
gms::inet_address ep(req->param["addr"]);
return g.get_current_generation_number(ep).then([] (gms::generation_type res) {
return make_ready_future<json::json_return_type>(res.value());
return gms::get_local_gossiper().get_current_generation_number(ep).then([] (int res) {
return make_ready_future<json::json_return_type>(res);
});
});
httpd::gossiper_json::get_current_heart_beat_version.set(r, [&g] (std::unique_ptr<http::request> req) {
httpd::gossiper_json::get_current_heart_beat_version.set(r, [] (std::unique_ptr<request> req) {
gms::inet_address ep(req->param["addr"]);
return g.get_current_heart_beat_version(ep).then([] (gms::version_type res) {
return make_ready_future<json::json_return_type>(res.value());
return gms::get_local_gossiper().get_current_heart_beat_version(ep).then([] (int res) {
return make_ready_future<json::json_return_type>(res);
});
});
httpd::gossiper_json::assassinate_endpoint.set(r, [&g](std::unique_ptr<http::request> req) {
httpd::gossiper_json::assassinate_endpoint.set(r, [](std::unique_ptr<request> req) {
if (req->get_query_param("unsafe") != "True") {
return g.assassinate_endpoint(req->param["addr"]).then([] {
return gms::get_local_gossiper().assassinate_endpoint(req->param["addr"]).then([] {
return make_ready_future<json::json_return_type>(json_void());
});
}
return g.unsafe_assassinate_endpoint(req->param["addr"]).then([] {
return make_ready_future<json::json_return_type>(json_void());
});
});
httpd::gossiper_json::force_remove_endpoint.set(r, [&g](std::unique_ptr<http::request> req) {
gms::inet_address ep(req->param["addr"]);
return g.force_remove_endpoint(ep).then([] {
return gms::get_local_gossiper().unsafe_assassinate_endpoint(req->param["addr"]).then([] {
return make_ready_future<json::json_return_type>(json_void());
});
});

View File

@@ -1,23 +1,30 @@
/*
* Copyright (C) 2015-present ScyllaDB
* Copyright (C) 2015 ScyllaDB
*/
/*
* SPDX-License-Identifier: AGPL-3.0-or-later
* 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 "api.hh"
namespace gms {
class gossiper;
}
namespace api {
void set_gossiper(http_context& ctx, httpd::routes& r, gms::gossiper& g);
void set_gossiper(http_context& ctx, routes& r);
}

View File

@@ -1,135 +1,69 @@
/*
* Copyright (C) 2015-present ScyllaDB
* Copyright (C) 2015 ScyllaDB
*/
/*
* SPDX-License-Identifier: AGPL-3.0-or-later
* 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 <algorithm>
#include <vector>
#include "hinted_handoff.hh"
#include "api/api-doc/hinted_handoff.json.hh"
#include "gms/inet_address.hh"
#include "gms/gossiper.hh"
#include "service/storage_proxy.hh"
namespace api {
using namespace json;
using namespace seastar::httpd;
namespace hh = httpd::hinted_handoff_json;
void set_hinted_handoff(http_context& ctx, routes& r, gms::gossiper& g) {
hh::create_hints_sync_point.set(r, [&ctx, &g] (std::unique_ptr<http::request> req) -> future<json::json_return_type> {
auto parse_hosts_list = [&g] (sstring arg) {
std::vector<sstring> hosts_str = split(arg, ",");
std::vector<gms::inet_address> hosts;
hosts.reserve(hosts_str.size());
if (hosts_str.empty()) {
// No target_hosts specified means that we should wait for hints for all nodes to be sent
const auto members_set = g.get_live_members();
std::copy(members_set.begin(), members_set.end(), std::back_inserter(hosts));
} else {
for (const auto& host_str : hosts_str) {
try {
gms::inet_address host;
host = gms::inet_address(host_str);
hosts.push_back(host);
} catch (std::exception& e) {
throw httpd::bad_param_exception(format("Failed to parse host address {}: {}", host_str, e.what()));
}
}
}
return hosts;
};
std::vector<gms::inet_address> target_hosts = parse_hosts_list(req->get_query_param("target_hosts"));
return ctx.sp.local().create_hint_sync_point(std::move(target_hosts)).then([] (db::hints::sync_point sync_point) {
return json::json_return_type(sync_point.encode());
});
});
hh::get_hints_sync_point.set(r, [&ctx] (std::unique_ptr<http::request> req) -> future<json::json_return_type> {
db::hints::sync_point sync_point;
const sstring encoded = req->get_query_param("id");
try {
sync_point = db::hints::sync_point::decode(encoded);
} catch (std::exception& e) {
throw httpd::bad_param_exception(format("Failed to parse the sync point description {}: {}", encoded, e.what()));
}
lowres_clock::time_point deadline;
const sstring timeout_str = req->get_query_param("timeout");
try {
deadline = [&] {
if (timeout_str.empty()) {
// Empty string - don't wait at all, just check the status
return lowres_clock::time_point::min();
} else {
const auto timeout = std::stoll(timeout_str);
if (timeout >= 0) {
// Wait until the point is reached, or until `timeout` seconds elapse
return lowres_clock::now() + std::chrono::seconds(timeout);
} else {
// Negative value indicates infinite timeout
return lowres_clock::time_point::max();
}
}
} ();
} catch (std::exception& e) {
throw httpd::bad_param_exception(format("Failed to parse the timeout parameter {}: {}", timeout_str, e.what()));
}
using return_type = hh::ns_get_hints_sync_point::get_hints_sync_point_return_type;
using return_type_wrapper = hh::ns_get_hints_sync_point::return_type_wrapper;
return ctx.sp.local().wait_for_hint_sync_point(std::move(sync_point), deadline).then([] {
return json::json_return_type(return_type_wrapper(return_type::DONE));
}).handle_exception_type([] (const timed_out_error&) {
return json::json_return_type(return_type_wrapper(return_type::IN_PROGRESS));
});
});
hh::list_endpoints_pending_hints.set(r, [] (std::unique_ptr<http::request> req) {
void set_hinted_handoff(http_context& ctx, routes& r) {
hh::list_endpoints_pending_hints.set(r, [] (std::unique_ptr<request> req) {
//TBD
unimplemented();
std::vector<sstring> res;
return make_ready_future<json::json_return_type>(res);
});
hh::truncate_all_hints.set(r, [] (std::unique_ptr<http::request> req) {
hh::truncate_all_hints.set(r, [] (std::unique_ptr<request> req) {
//TBD
unimplemented();
sstring host = req->get_query_param("host");
return make_ready_future<json::json_return_type>(json_void());
});
hh::schedule_hint_delivery.set(r, [] (std::unique_ptr<http::request> req) {
hh::schedule_hint_delivery.set(r, [] (std::unique_ptr<request> req) {
//TBD
unimplemented();
sstring host = req->get_query_param("host");
return make_ready_future<json::json_return_type>(json_void());
});
hh::pause_hints_delivery.set(r, [] (std::unique_ptr<http::request> req) {
hh::pause_hints_delivery.set(r, [] (std::unique_ptr<request> req) {
//TBD
unimplemented();
sstring pause = req->get_query_param("pause");
return make_ready_future<json::json_return_type>(json_void());
});
hh::get_create_hint_count.set(r, [] (std::unique_ptr<http::request> req) {
hh::get_create_hint_count.set(r, [] (std::unique_ptr<request> req) {
//TBD
unimplemented();
sstring host = req->get_query_param("host");
return make_ready_future<json::json_return_type>(0);
});
hh::get_not_stored_hints_count.set(r, [] (std::unique_ptr<http::request> req) {
hh::get_not_stored_hints_count.set(r, [] (std::unique_ptr<request> req) {
//TBD
unimplemented();
sstring host = req->get_query_param("host");
@@ -137,16 +71,5 @@ void set_hinted_handoff(http_context& ctx, routes& r, gms::gossiper& g) {
});
}
void unset_hinted_handoff(http_context& ctx, routes& r) {
hh::create_hints_sync_point.unset(r);
hh::get_hints_sync_point.unset(r);
hh::list_endpoints_pending_hints.unset(r);
hh::truncate_all_hints.unset(r);
hh::schedule_hint_delivery.unset(r);
hh::pause_hints_delivery.unset(r);
hh::get_create_hint_count.unset(r);
hh::get_not_stored_hints_count.unset(r);
}
}

View File

@@ -1,24 +1,30 @@
/*
* Copyright (C) 2015-present ScyllaDB
* Copyright (C) 2015 ScyllaDB
*/
/*
* SPDX-License-Identifier: AGPL-3.0-or-later
* 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 "api.hh"
namespace gms {
class gossiper;
}
namespace api {
void set_hinted_handoff(http_context& ctx, httpd::routes& r, gms::gossiper& g);
void unset_hinted_handoff(http_context& ctx, httpd::routes& r);
void set_hinted_handoff(http_context& ctx, routes& r);
}

View File

@@ -1,29 +1,40 @@
/*
* Copyright (C) 2015-present ScyllaDB
* Copyright (C) 2015 ScyllaDB
*/
/*
* SPDX-License-Identifier: AGPL-3.0-or-later
* 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 "api/api-doc/lsa.json.hh"
#include "api/lsa.hh"
#include "api/api.hh"
#include <seastar/http/exception.hh>
#include "http/exception.hh"
#include "utils/logalloc.hh"
#include "log.hh"
#include "replica/database.hh"
namespace api {
using namespace seastar::httpd;
static logging::logger alogger("lsa-api");
void set_lsa(http_context& ctx, routes& r) {
httpd::lsa_json::lsa_compact.set(r, [&ctx](std::unique_ptr<request> req) {
alogger.info("Triggering compaction");
return ctx.db.invoke_on_all([] (replica::database&) {
return ctx.db.invoke_on_all([] (database&) {
logalloc::shard_tracker().reclaim(std::numeric_limits<size_t>::max());
}).then([] {
return json::json_return_type(json::json_void());

View File

@@ -1,9 +1,22 @@
/*
* Copyright (C) 2015-present ScyllaDB
* Copyright (C) 2015 ScyllaDB
*/
/*
* SPDX-License-Identifier: AGPL-3.0-or-later
* 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
@@ -12,6 +25,6 @@
namespace api {
void set_lsa(http_context& ctx, httpd::routes& r);
void set_lsa(http_context& ctx, routes& r);
}

View File

@@ -1,19 +1,31 @@
/*
* Copyright (C) 2015-present ScyllaDB
* Copyright (C) 2015 ScyllaDB
*/
/*
* SPDX-License-Identifier: AGPL-3.0-or-later
* 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 "messaging_service.hh"
#include "message/messaging_service.hh"
#include <seastar/rpc/rpc_types.hh>
#include "rpc/rpc_types.hh"
#include "api/api-doc/messaging_service.json.hh"
#include <iostream>
#include <sstream>
using namespace seastar::httpd;
using namespace httpd::messaging_service_json;
using namespace netw;
@@ -29,7 +41,7 @@ std::vector<message_counter> map_to_message_counters(
std::vector<message_counter> res;
for (auto i : map) {
res.push_back(message_counter());
res.back().key = fmt::to_string(i.first);
res.back().key = boost::lexical_cast<sstring>(i.first);
res.back().value = i.second;
}
return res;
@@ -41,8 +53,8 @@ std::vector<message_counter> map_to_message_counters(
* according to a function that it gets as a parameter.
*
*/
future_json_function get_client_getter(sharded<netw::messaging_service>& ms, std::function<uint64_t(const shard_info&)> f) {
return [&ms, f](std::unique_ptr<request> req) {
future_json_function get_client_getter(std::function<uint64_t(const shard_info&)> f) {
return [f](std::unique_ptr<request> req) {
using map_type = std::unordered_map<gms::inet_address, uint64_t>;
auto get_shard_map = [f](messaging_service& ms) {
std::unordered_map<gms::inet_address, unsigned long> map;
@@ -51,74 +63,70 @@ future_json_function get_client_getter(sharded<netw::messaging_service>& ms, std
});
return map;
};
return ms.map_reduce0(get_shard_map, map_type(), map_sum<map_type>).
return get_messaging_service().map_reduce0(get_shard_map, map_type(), map_sum<map_type>).
then([](map_type&& map) {
return make_ready_future<json::json_return_type>(map_to_message_counters(map));
});
};
}
future_json_function get_server_getter(sharded<netw::messaging_service>& ms, std::function<uint64_t(const rpc::stats&)> f) {
return [&ms, f](std::unique_ptr<request> req) {
future_json_function get_server_getter(std::function<uint64_t(const rpc::stats&)> f) {
return [f](std::unique_ptr<request> req) {
using map_type = std::unordered_map<gms::inet_address, uint64_t>;
auto get_shard_map = [f](messaging_service& ms) {
std::unordered_map<gms::inet_address, unsigned long> map;
ms.foreach_server_connection_stats([&map, f] (const rpc::client_info& info, const rpc::stats& stats) mutable {
map[gms::inet_address(info.addr.addr())] = f(stats);
map[gms::inet_address(net::ipv4_address(info.addr))] = f(stats);
});
return map;
};
return ms.map_reduce0(get_shard_map, map_type(), map_sum<map_type>).
return get_messaging_service().map_reduce0(get_shard_map, map_type(), map_sum<map_type>).
then([](map_type&& map) {
return make_ready_future<json::json_return_type>(map_to_message_counters(map));
});
};
}
void set_messaging_service(http_context& ctx, routes& r, sharded<netw::messaging_service>& ms) {
get_timeout_messages.set(r, get_client_getter(ms, [](const shard_info& c) {
void set_messaging_service(http_context& ctx, routes& r) {
get_timeout_messages.set(r, get_client_getter([](const shard_info& c) {
return c.get_stats().timeout;
}));
get_sent_messages.set(r, get_client_getter(ms, [](const shard_info& c) {
get_sent_messages.set(r, get_client_getter([](const shard_info& c) {
return c.get_stats().sent_messages;
}));
get_replied_messages.set(r, get_client_getter(ms, [](const shard_info& c) {
return c.get_stats().replied;
}));
get_dropped_messages.set(r, get_client_getter(ms, [](const shard_info& c) {
get_dropped_messages.set(r, get_client_getter([](const shard_info& c) {
// We don't have the same drop message mechanism
// as origin has.
// hence we can always return 0
return 0;
}));
get_exception_messages.set(r, get_client_getter(ms, [](const shard_info& c) {
get_exception_messages.set(r, get_client_getter([](const shard_info& c) {
return c.get_stats().exception_received;
}));
get_pending_messages.set(r, get_client_getter(ms, [](const shard_info& c) {
get_pending_messages.set(r, get_client_getter([](const shard_info& c) {
return c.get_stats().pending;
}));
get_respond_pending_messages.set(r, get_server_getter(ms, [](const rpc::stats& c) {
get_respond_pending_messages.set(r, get_server_getter([](const rpc::stats& c) {
return c.pending;
}));
get_respond_completed_messages.set(r, get_server_getter(ms, [](const rpc::stats& c) {
get_respond_completed_messages.set(r, get_server_getter([](const rpc::stats& c) {
return c.sent_messages;
}));
get_version.set(r, [&ms](const_req req) {
return ms.local().get_raw_version(req.get_query_param("addr"));
get_version.set(r, [](const_req req) {
return netw::get_local_messaging_service().get_raw_version(req.get_query_param("addr"));
});
get_dropped_messages_by_ver.set(r, [&ms](std::unique_ptr<request> req) {
get_dropped_messages_by_ver.set(r, [](std::unique_ptr<request> req) {
shared_ptr<std::vector<uint64_t>> map = make_shared<std::vector<uint64_t>>(num_verb);
return ms.map_reduce([map](const uint64_t* local_map) mutable {
return netw::get_messaging_service().map_reduce([map](const uint64_t* local_map) mutable {
for (auto i = 0; i < num_verb; i++) {
(*map)[i]+= local_map[i];
}
@@ -131,7 +139,7 @@ void set_messaging_service(http_context& ctx, routes& r, sharded<netw::messaging
messaging_verb v = i; // for type safety we use messaging_verb values
auto idx = static_cast<uint32_t>(v);
if (idx >= map->size()) {
throw std::runtime_error(format("verb index out of bounds: {:d}, map size: {:d}", idx, map->size()));
throw std::runtime_error(sprint("verb index out of bounds: %lu, map size: %lu", idx, map->size()));
}
if ((*map)[idx] > 0) {
c.count = (*map)[idx];
@@ -143,19 +151,5 @@ void set_messaging_service(http_context& ctx, routes& r, sharded<netw::messaging
});
});
}
void unset_messaging_service(http_context& ctx, routes& r) {
get_timeout_messages.unset(r);
get_sent_messages.unset(r);
get_replied_messages.unset(r);
get_dropped_messages.unset(r);
get_exception_messages.unset(r);
get_pending_messages.unset(r);
get_respond_pending_messages.unset(r);
get_respond_completed_messages.unset(r);
get_version.unset(r);
get_dropped_messages_by_ver.unset(r);
}
}

View File

@@ -1,20 +1,30 @@
/*
* Copyright (C) 2015-present ScyllaDB
* Copyright (C) 2015 ScyllaDB
*/
/*
* SPDX-License-Identifier: AGPL-3.0-or-later
* 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 "api.hh"
namespace netw { class messaging_service; }
namespace api {
void set_messaging_service(http_context& ctx, httpd::routes& r, sharded<netw::messaging_service>& ms);
void unset_messaging_service(http_context& ctx, httpd::routes& r);
void set_messaging_service(http_context& ctx, routes& r);
}

View File

@@ -1,9 +1,22 @@
/*
* Copyright (C) 2015-present ScyllaDB
* Copyright (C) 2015 ScyllaDB
*/
/*
* SPDX-License-Identifier: AGPL-3.0-or-later
* 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 "storage_proxy.hh"
@@ -13,83 +26,19 @@
#include "service/storage_service.hh"
#include "db/config.hh"
#include "utils/histogram.hh"
#include "replica/database.hh"
#include <seastar/core/scheduling_specific.hh>
namespace api {
namespace sp = httpd::storage_proxy_json;
using proxy = service::storage_proxy;
using namespace seastar::httpd;
using namespace json;
utils::time_estimated_histogram timed_rate_moving_average_summary_merge(utils::time_estimated_histogram a, const utils::timed_rate_moving_average_summary_and_histogram& b) {
return a.merge(b.histogram());
static future<utils::rate_moving_average> sum_timed_rate(distributed<proxy>& d, utils::timed_rate_moving_average proxy::stats::*f) {
return d.map_reduce0([f](const proxy& p) {return (p.get_stats().*f).rate();}, utils::rate_moving_average(),
std::plus<utils::rate_moving_average>());
}
/**
* This function implement a two dimentional map reduce where
* the first level is a distributed storage_proxy class and the
* second level is the stats per scheduling group class.
* @param d - a reference to the storage_proxy distributed class.
* @param mapper - the internal mapper that is used to map the internal
* stat class into a value of type `V`.
* @param reducer - the reducer that is used in both outer and inner
* aggregations.
* @param initial_value - the initial value to use for both aggregations
* @return A future that resolves to the result of the aggregation.
*/
template<typename V, typename Reducer, typename InnerMapper>
future<V> two_dimensional_map_reduce(distributed<service::storage_proxy>& d,
InnerMapper mapper, Reducer reducer, V initial_value) {
return d.map_reduce0( [mapper, reducer, initial_value] (const service::storage_proxy& sp) {
return map_reduce_scheduling_group_specific<service::storage_proxy_stats::stats>(
mapper, reducer, initial_value, sp.get_stats_key());
}, initial_value, reducer);
}
/**
* This function implement a two dimentional map reduce where
* the first level is a distributed storage_proxy class and the
* second level is the stats per scheduling group class.
* @param d - a reference to the storage_proxy distributed class.
* @param f - a field pointer which is the implicit internal reducer.
* @param reducer - the reducer that is used in both outer and inner
* aggregations.
* @param initial_value - the initial value to use for both aggregations* @return
* @return A future that resolves to the result of the aggregation.
*/
template<typename V, typename Reducer, typename F, typename C>
future<V> two_dimensional_map_reduce(distributed<service::storage_proxy>& d,
C F::*f, Reducer reducer, V initial_value) {
return two_dimensional_map_reduce(d, [f] (F& stats) -> V {
return stats.*f;
}, reducer, initial_value);
}
/**
* A partial Specialization of sum_stats for the storage proxy
* case where the get stats function doesn't return a
* stats object with fields but a per scheduling group
* stats object, the name was also changed since functions
* partial specialization is not supported in C++.
*
*/
template<typename V, typename F>
future<json::json_return_type> sum_stats_storage_proxy(distributed<proxy>& d, V F::*f) {
return two_dimensional_map_reduce(d, [f] (F& stats) { return stats.*f; }, std::plus<V>(), V(0)).then([] (V val) {
return make_ready_future<json::json_return_type>(val);
});
}
static future<utils::rate_moving_average> sum_timed_rate(distributed<proxy>& d, utils::timed_rate_moving_average service::storage_proxy_stats::stats::*f) {
return two_dimensional_map_reduce(d, [f] (service::storage_proxy_stats::stats& stats) {
return (stats.*f).rate();
}, std::plus<utils::rate_moving_average>(), utils::rate_moving_average());
}
static future<json::json_return_type> sum_timed_rate_as_obj(distributed<proxy>& d, utils::timed_rate_moving_average service::storage_proxy_stats::stats::*f) {
static future<json::json_return_type> sum_timed_rate_as_obj(distributed<proxy>& d, utils::timed_rate_moving_average proxy::stats::*f) {
return sum_timed_rate(d, f).then([](const utils::rate_moving_average& val) {
httpd::utils_json::rate_moving_average m;
m = val;
@@ -97,163 +46,92 @@ static future<json::json_return_type> sum_timed_rate_as_obj(distributed<proxy>&
});
}
httpd::utils_json::rate_moving_average_and_histogram get_empty_moving_average() {
return timer_to_json(utils::rate_moving_average_and_histogram());
}
static future<json::json_return_type> sum_timed_rate_as_long(distributed<proxy>& d, utils::timed_rate_moving_average service::storage_proxy_stats::stats::*f) {
static future<json::json_return_type> sum_timed_rate_as_long(distributed<proxy>& d, utils::timed_rate_moving_average proxy::stats::*f) {
return sum_timed_rate(d, f).then([](const utils::rate_moving_average& val) {
return make_ready_future<json::json_return_type>(val.count);
});
}
utils_json::estimated_histogram time_to_json_histogram(const utils::time_estimated_histogram& val) {
utils_json::estimated_histogram res;
for (size_t i = 0; i < val.size(); i++) {
res.buckets.push(val.get(i));
res.bucket_offsets.push(val.get_bucket_lower_limit(i));
}
return res;
}
static future<json::json_return_type> sum_estimated_histogram(http_context& ctx, utils::timed_rate_moving_average_summary_and_histogram service::storage_proxy_stats::stats::*f) {
return two_dimensional_map_reduce(ctx.sp, [f] (service::storage_proxy_stats::stats& stats) {
return (stats.*f).histogram();
}, utils::time_estimated_histogram_merge, utils::time_estimated_histogram()).then([](const utils::time_estimated_histogram& val) {
return make_ready_future<json::json_return_type>(time_to_json_histogram(val));
});
}
static future<json::json_return_type> sum_estimated_histogram(http_context& ctx, utils::estimated_histogram service::storage_proxy_stats::stats::*f) {
return two_dimensional_map_reduce(ctx.sp, f, utils::estimated_histogram_merge,
utils::estimated_histogram()).then([](const utils::estimated_histogram& val) {
static future<json::json_return_type> sum_estimated_histogram(http_context& ctx, utils::estimated_histogram proxy::stats::*f) {
return ctx.sp.map_reduce0([f](const proxy& p) {return p.get_stats().*f;}, utils::estimated_histogram(),
utils::estimated_histogram_merge).then([](const utils::estimated_histogram& val) {
utils_json::estimated_histogram res;
res = val;
return make_ready_future<json::json_return_type>(res);
});
}
static future<json::json_return_type> total_latency(http_context& ctx, utils::timed_rate_moving_average_summary_and_histogram service::storage_proxy_stats::stats::*f) {
return two_dimensional_map_reduce(ctx.sp, [f] (service::storage_proxy_stats::stats& stats) {
return (stats.*f).hist.mean * (stats.*f).hist.count;
}, std::plus<double>(), 0.0).then([](double val) {
static future<json::json_return_type> total_latency(http_context& ctx, utils::timed_rate_moving_average_and_histogram proxy::stats::*f) {
return ctx.sp.map_reduce0([f](const proxy& p) {return (p.get_stats().*f).hist.mean * (p.get_stats().*f).hist.count;}, 0.0,
std::plus<double>()).then([](double val) {
int64_t res = val;
return make_ready_future<json::json_return_type>(res);
});
}
/**
* A partial Specialization of sum_histogram_stats
* for the storage proxy case where the get stats
* function doesn't return a stats object with
* fields but a per scheduling group stats object,
* the name was also changed since function partial
* specialization is not supported in C++.
*/
template<typename F>
future<json::json_return_type>
sum_histogram_stats_storage_proxy(distributed<proxy>& d,
utils::timed_rate_moving_average_summary_and_histogram F::*f) {
return two_dimensional_map_reduce(d, [f] (service::storage_proxy_stats::stats& stats) {
return (stats.*f).hist;
}, std::plus<utils::ihistogram>(), utils::ihistogram()).
then([](const utils::ihistogram& val) {
return make_ready_future<json::json_return_type>(to_json(val));
});
}
/**
* A partial Specialization of sum_timer_stats for the
* storage proxy case where the get stats function
* doesn't return a stats object with fields but a
* per scheduling group stats object, the name
* was also changed since partial function specialization
* is not supported in C++.
*/
template<typename F>
future<json::json_return_type>
sum_timer_stats_storage_proxy(distributed<proxy>& d,
utils::timed_rate_moving_average_summary_and_histogram F::*f) {
return two_dimensional_map_reduce(d, [f] (service::storage_proxy_stats::stats& stats) {
return (stats.*f).rate();
}, std::plus<utils::rate_moving_average_and_histogram>(),
utils::rate_moving_average_and_histogram()).then([](const utils::rate_moving_average_and_histogram& val) {
return make_ready_future<json::json_return_type>(timer_to_json(val));
});
}
void set_storage_proxy(http_context& ctx, routes& r, sharded<service::storage_service>& ss) {
sp::get_total_hints.set(r, [](std::unique_ptr<http::request> req) {
void set_storage_proxy(http_context& ctx, routes& r) {
sp::get_total_hints.set(r, [](std::unique_ptr<request> req) {
//TBD
unimplemented();
return make_ready_future<json::json_return_type>(0);
});
sp::get_hinted_handoff_enabled.set(r, [&ctx](std::unique_ptr<http::request> req) {
const auto& filter = ctx.sp.local().get_hints_host_filter();
return make_ready_future<json::json_return_type>(!filter.is_disabled_for_all());
sp::get_hinted_handoff_enabled.set(r, [](std::unique_ptr<request> req) {
//TBD
// FIXME
// hinted handoff is not supported currently,
// so we should return false
return make_ready_future<json::json_return_type>(false);
});
sp::set_hinted_handoff_enabled.set(r, [&ctx](std::unique_ptr<http::request> req) {
sp::set_hinted_handoff_enabled.set(r, [](std::unique_ptr<request> req) {
//TBD
unimplemented();
auto enable = req->get_query_param("enable");
auto filter = (enable == "true" || enable == "1")
? db::hints::host_filter(db::hints::host_filter::enabled_for_all_tag {})
: db::hints::host_filter(db::hints::host_filter::disabled_for_all_tag {});
return ctx.sp.invoke_on_all([filter = std::move(filter)] (service::storage_proxy& sp) {
return sp.change_hints_host_filter(filter);
}).then([] {
return make_ready_future<json::json_return_type>(json_void());
});
return make_ready_future<json::json_return_type>(json_void());
});
sp::get_hinted_handoff_enabled_by_dc.set(r, [&ctx](std::unique_ptr<http::request> req) {
std::vector<sstring> res;
const auto& filter = ctx.sp.local().get_hints_host_filter();
const auto& dcs = filter.get_dcs();
res.reserve(res.size());
std::copy(dcs.begin(), dcs.end(), std::back_inserter(res));
sp::get_hinted_handoff_enabled_by_dc.set(r, [](std::unique_ptr<request> req) {
//TBD
unimplemented();
std::vector<sp::mapper_list> res;
return make_ready_future<json::json_return_type>(res);
});
sp::set_hinted_handoff_enabled_by_dc_list.set(r, [&ctx](std::unique_ptr<http::request> req) {
auto dcs = req->get_query_param("dcs");
auto filter = db::hints::host_filter::parse_from_dc_list(std::move(dcs));
return ctx.sp.invoke_on_all([filter = std::move(filter)] (service::storage_proxy& sp) {
return sp.change_hints_host_filter(filter);
}).then([] {
return make_ready_future<json::json_return_type>(json_void());
});
sp::set_hinted_handoff_enabled_by_dc_list.set(r, [](std::unique_ptr<request> req) {
//TBD
unimplemented();
auto enable = req->get_query_param("dcs");
return make_ready_future<json::json_return_type>(json_void());
});
sp::get_max_hint_window.set(r, [](std::unique_ptr<http::request> req) {
sp::get_max_hint_window.set(r, [](std::unique_ptr<request> req) {
//TBD
unimplemented();
return make_ready_future<json::json_return_type>(0);
});
sp::set_max_hint_window.set(r, [](std::unique_ptr<http::request> req) {
sp::set_max_hint_window.set(r, [](std::unique_ptr<request> req) {
//TBD
unimplemented();
auto enable = req->get_query_param("ms");
return make_ready_future<json::json_return_type>(json_void());
});
sp::get_max_hints_in_progress.set(r, [](std::unique_ptr<http::request> req) {
sp::get_max_hints_in_progress.set(r, [](std::unique_ptr<request> req) {
//TBD
unimplemented();
return make_ready_future<json::json_return_type>(1);
});
sp::set_max_hints_in_progress.set(r, [](std::unique_ptr<http::request> req) {
sp::set_max_hints_in_progress.set(r, [](std::unique_ptr<request> req) {
//TBD
unimplemented();
auto enable = req->get_query_param("qs");
return make_ready_future<json::json_return_type>(json_void());
});
sp::get_hints_in_progress.set(r, [](std::unique_ptr<http::request> req) {
sp::get_hints_in_progress.set(r, [](std::unique_ptr<request> req) {
//TBD
unimplemented();
return make_ready_future<json::json_return_type>(0);
@@ -263,7 +141,7 @@ void set_storage_proxy(http_context& ctx, routes& r, sharded<service::storage_se
return ctx.db.local().get_config().request_timeout_in_ms()/1000.0;
});
sp::set_rpc_timeout.set(r, [](std::unique_ptr<http::request> req) {
sp::set_rpc_timeout.set(r, [](std::unique_ptr<request> req) {
//TBD
unimplemented();
auto enable = req->get_query_param("timeout");
@@ -274,7 +152,7 @@ void set_storage_proxy(http_context& ctx, routes& r, sharded<service::storage_se
return ctx.db.local().get_config().read_request_timeout_in_ms()/1000.0;
});
sp::set_read_rpc_timeout.set(r, [](std::unique_ptr<http::request> req) {
sp::set_read_rpc_timeout.set(r, [](std::unique_ptr<request> req) {
//TBD
unimplemented();
auto enable = req->get_query_param("timeout");
@@ -285,7 +163,7 @@ void set_storage_proxy(http_context& ctx, routes& r, sharded<service::storage_se
return ctx.db.local().get_config().write_request_timeout_in_ms()/1000.0;
});
sp::set_write_rpc_timeout.set(r, [](std::unique_ptr<http::request> req) {
sp::set_write_rpc_timeout.set(r, [](std::unique_ptr<request> req) {
//TBD
unimplemented();
auto enable = req->get_query_param("timeout");
@@ -296,7 +174,7 @@ void set_storage_proxy(http_context& ctx, routes& r, sharded<service::storage_se
return ctx.db.local().get_config().counter_write_request_timeout_in_ms()/1000.0;
});
sp::set_counter_write_rpc_timeout.set(r, [](std::unique_ptr<http::request> req) {
sp::set_counter_write_rpc_timeout.set(r, [](std::unique_ptr<request> req) {
//TBD
unimplemented();
auto enable = req->get_query_param("timeout");
@@ -307,7 +185,7 @@ void set_storage_proxy(http_context& ctx, routes& r, sharded<service::storage_se
return ctx.db.local().get_config().cas_contention_timeout_in_ms()/1000.0;
});
sp::set_cas_contention_timeout.set(r, [](std::unique_ptr<http::request> req) {
sp::set_cas_contention_timeout.set(r, [](std::unique_ptr<request> req) {
//TBD
unimplemented();
auto enable = req->get_query_param("timeout");
@@ -318,7 +196,7 @@ void set_storage_proxy(http_context& ctx, routes& r, sharded<service::storage_se
return ctx.db.local().get_config().range_request_timeout_in_ms()/1000.0;
});
sp::set_range_rpc_timeout.set(r, [](std::unique_ptr<http::request> req) {
sp::set_range_rpc_timeout.set(r, [](std::unique_ptr<request> req) {
//TBD
unimplemented();
auto enable = req->get_query_param("timeout");
@@ -329,33 +207,33 @@ void set_storage_proxy(http_context& ctx, routes& r, sharded<service::storage_se
return ctx.db.local().get_config().truncate_request_timeout_in_ms()/1000.0;
});
sp::set_truncate_rpc_timeout.set(r, [](std::unique_ptr<http::request> req) {
sp::set_truncate_rpc_timeout.set(r, [](std::unique_ptr<request> req) {
//TBD
unimplemented();
auto enable = req->get_query_param("timeout");
return make_ready_future<json::json_return_type>(json_void());
});
sp::reload_trigger_classes.set(r, [](std::unique_ptr<http::request> req) {
sp::reload_trigger_classes.set(r, [](std::unique_ptr<request> req) {
//TBD
unimplemented();
return make_ready_future<json::json_return_type>(json_void());
});
sp::get_read_repair_attempted.set(r, [&ctx](std::unique_ptr<http::request> req) {
return sum_stats_storage_proxy(ctx.sp, &service::storage_proxy_stats::stats::read_repair_attempts);
sp::get_read_repair_attempted.set(r, [&ctx](std::unique_ptr<request> req) {
return sum_stats(ctx.sp, &proxy::stats::read_repair_attempts);
});
sp::get_read_repair_repaired_blocking.set(r, [&ctx](std::unique_ptr<http::request> req) {
return sum_stats_storage_proxy(ctx.sp, &service::storage_proxy_stats::stats::read_repair_repaired_blocking);
sp::get_read_repair_repaired_blocking.set(r, [&ctx](std::unique_ptr<request> req) {
return sum_stats(ctx.sp, &proxy::stats::read_repair_repaired_blocking);
});
sp::get_read_repair_repaired_background.set(r, [&ctx](std::unique_ptr<http::request> req) {
return sum_stats_storage_proxy(ctx.sp, &service::storage_proxy_stats::stats::read_repair_repaired_background);
sp::get_read_repair_repaired_background.set(r, [&ctx](std::unique_ptr<request> req) {
return sum_stats(ctx.sp, &proxy::stats::read_repair_repaired_background);
});
sp::get_schema_versions.set(r, [&ss](std::unique_ptr<http::request> req) {
return ss.local().describe_schema_versions().then([] (auto result) {
sp::get_schema_versions.set(r, [](std::unique_ptr<request> req) {
return service::get_local_storage_service().describe_schema_versions().then([] (auto result) {
std::vector<sp::mapper_list> res;
for (auto e : result) {
sp::mapper_list entry;
@@ -367,224 +245,164 @@ void set_storage_proxy(http_context& ctx, routes& r, sharded<service::storage_se
});
});
sp::get_cas_read_timeouts.set(r, [&ctx](std::unique_ptr<http::request> req) {
return sum_timed_rate_as_long(ctx.sp, &proxy::stats::cas_read_timeouts);
});
sp::get_cas_read_unavailables.set(r, [&ctx](std::unique_ptr<http::request> req) {
return sum_timed_rate_as_long(ctx.sp, &proxy::stats::cas_read_unavailables);
});
sp::get_cas_write_timeouts.set(r, [&ctx](std::unique_ptr<http::request> req) {
return sum_timed_rate_as_long(ctx.sp, &proxy::stats::cas_write_timeouts);
});
sp::get_cas_write_unavailables.set(r, [&ctx](std::unique_ptr<http::request> req) {
return sum_timed_rate_as_long(ctx.sp, &proxy::stats::cas_write_unavailables);
});
sp::get_cas_write_metrics_unfinished_commit.set(r, [&ctx](std::unique_ptr<http::request> req) {
return sum_stats(ctx.sp, &proxy::stats::cas_write_unfinished_commit);
});
sp::get_cas_write_metrics_contention.set(r, [&ctx](std::unique_ptr<http::request> req) {
return sum_estimated_histogram(ctx, &proxy::stats::cas_write_contention);
});
sp::get_cas_write_metrics_condition_not_met.set(r, [&ctx](std::unique_ptr<http::request> req) {
return sum_stats(ctx.sp, &proxy::stats::cas_write_condition_not_met);
});
sp::get_cas_write_metrics_failed_read_round_optimization.set(r, [&ctx](std::unique_ptr<http::request> req) {
return sum_stats(ctx.sp, &proxy::stats::cas_failed_read_round_optimization);
});
sp::get_cas_read_metrics_unfinished_commit.set(r, [&ctx](std::unique_ptr<http::request> req) {
return sum_stats(ctx.sp, &proxy::stats::cas_read_unfinished_commit);
});
sp::get_cas_read_metrics_contention.set(r, [&ctx](std::unique_ptr<http::request> req) {
return sum_estimated_histogram(ctx, &proxy::stats::cas_read_contention);
});
sp::get_read_metrics_timeouts.set(r, [&ctx](std::unique_ptr<http::request> req) {
return sum_timed_rate_as_long(ctx.sp, &service::storage_proxy_stats::stats::read_timeouts);
});
sp::get_read_metrics_unavailables.set(r, [&ctx](std::unique_ptr<http::request> req) {
return sum_timed_rate_as_long(ctx.sp, &service::storage_proxy_stats::stats::read_unavailables);
});
sp::get_range_metrics_timeouts.set(r, [&ctx](std::unique_ptr<http::request> req) {
return sum_timed_rate_as_long(ctx.sp, &service::storage_proxy_stats::stats::range_slice_timeouts);
});
sp::get_range_metrics_unavailables.set(r, [&ctx](std::unique_ptr<http::request> req) {
return sum_timed_rate_as_long(ctx.sp, &service::storage_proxy_stats::stats::range_slice_unavailables);
});
sp::get_write_metrics_timeouts.set(r, [&ctx](std::unique_ptr<http::request> req) {
return sum_timed_rate_as_long(ctx.sp, &service::storage_proxy_stats::stats::write_timeouts);
});
sp::get_write_metrics_unavailables.set(r, [&ctx](std::unique_ptr<http::request> req) {
return sum_timed_rate_as_long(ctx.sp, &service::storage_proxy_stats::stats::write_unavailables);
});
sp::get_read_metrics_timeouts_rates.set(r, [&ctx](std::unique_ptr<http::request> req) {
return sum_timed_rate_as_obj(ctx.sp, &service::storage_proxy_stats::stats::read_timeouts);
});
sp::get_read_metrics_unavailables_rates.set(r, [&ctx](std::unique_ptr<http::request> req) {
return sum_timed_rate_as_obj(ctx.sp, &service::storage_proxy_stats::stats::read_unavailables);
});
sp::get_range_metrics_timeouts_rates.set(r, [&ctx](std::unique_ptr<http::request> req) {
return sum_timed_rate_as_obj(ctx.sp, &service::storage_proxy_stats::stats::range_slice_timeouts);
});
sp::get_range_metrics_unavailables_rates.set(r, [&ctx](std::unique_ptr<http::request> req) {
return sum_timed_rate_as_obj(ctx.sp, &service::storage_proxy_stats::stats::range_slice_unavailables);
});
sp::get_write_metrics_timeouts_rates.set(r, [&ctx](std::unique_ptr<http::request> req) {
return sum_timed_rate_as_obj(ctx.sp, &service::storage_proxy_stats::stats::write_timeouts);
});
sp::get_write_metrics_unavailables_rates.set(r, [&ctx](std::unique_ptr<http::request> req) {
return sum_timed_rate_as_obj(ctx.sp, &service::storage_proxy_stats::stats::write_unavailables);
});
sp::get_range_metrics_latency_histogram_depricated.set(r, [&ctx](std::unique_ptr<http::request> req) {
return sum_histogram_stats_storage_proxy(ctx.sp, &service::storage_proxy_stats::stats::range);
});
sp::get_write_metrics_latency_histogram_depricated.set(r, [&ctx](std::unique_ptr<http::request> req) {
return sum_histogram_stats_storage_proxy(ctx.sp, &service::storage_proxy_stats::stats::write);
});
sp::get_read_metrics_latency_histogram_depricated.set(r, [&ctx](std::unique_ptr<http::request> req) {
return sum_histogram_stats_storage_proxy(ctx.sp, &service::storage_proxy_stats::stats::read);
});
sp::get_range_metrics_latency_histogram.set(r, [&ctx](std::unique_ptr<http::request> req) {
return sum_timer_stats_storage_proxy(ctx.sp, &service::storage_proxy_stats::stats::range);
});
sp::get_write_metrics_latency_histogram.set(r, [&ctx](std::unique_ptr<http::request> req) {
return sum_timer_stats_storage_proxy(ctx.sp, &service::storage_proxy_stats::stats::write);
});
sp::get_cas_write_metrics_latency_histogram.set(r, [&ctx](std::unique_ptr<http::request> req) {
return sum_timer_stats(ctx.sp, &proxy::stats::cas_write);
});
sp::get_cas_read_metrics_latency_histogram.set(r, [&ctx](std::unique_ptr<http::request> req) {
return sum_timer_stats(ctx.sp, &proxy::stats::cas_read);
});
sp::get_view_write_metrics_latency_histogram.set(r, [](std::unique_ptr<http::request> req) {
sp::get_cas_read_timeouts.set(r, [](std::unique_ptr<request> req) {
//TBD
// FIXME
// No View metrics are available, so just return empty moving average
return make_ready_future<json::json_return_type>(get_empty_moving_average());
// cas is not supported yet, so just return 0
return make_ready_future<json::json_return_type>(0);
});
sp::get_read_metrics_latency_histogram.set(r, [&ctx](std::unique_ptr<http::request> req) {
return sum_timer_stats_storage_proxy(ctx.sp, &service::storage_proxy_stats::stats::read);
sp::get_cas_read_unavailables.set(r, [](std::unique_ptr<request> req) {
//TBD
// FIXME
// cas is not supported yet, so just return 0
return make_ready_future<json::json_return_type>(0);
});
sp::get_read_estimated_histogram.set(r, [&ctx](std::unique_ptr<http::request> req) {
return sum_estimated_histogram(ctx, &service::storage_proxy_stats::stats::read);
sp::get_cas_write_timeouts.set(r, [](std::unique_ptr<request> req) {
//TBD
// FIXME
// cas is not supported yet, so just return 0
return make_ready_future<json::json_return_type>(0);
});
sp::get_read_latency.set(r, [&ctx](std::unique_ptr<http::request> req) {
return total_latency(ctx, &service::storage_proxy_stats::stats::read);
});
sp::get_write_estimated_histogram.set(r, [&ctx](std::unique_ptr<http::request> req) {
return sum_estimated_histogram(ctx, &service::storage_proxy_stats::stats::write);
sp::get_cas_write_unavailables.set(r, [](std::unique_ptr<request> req) {
//TBD
// FIXME
// cas is not supported yet, so just return 0
return make_ready_future<json::json_return_type>(0);
});
sp::get_write_latency.set(r, [&ctx](std::unique_ptr<http::request> req) {
return total_latency(ctx, &service::storage_proxy_stats::stats::write);
sp::get_cas_write_metrics_unfinished_commit.set(r, [](std::unique_ptr<request> req) {
//TBD
unimplemented();
return make_ready_future<json::json_return_type>(0);
});
sp::get_range_estimated_histogram.set(r, [&ctx](std::unique_ptr<http::request> req) {
return sum_timer_stats_storage_proxy(ctx.sp, &service::storage_proxy_stats::stats::range);
sp::get_cas_write_metrics_contention.set(r, [](std::unique_ptr<request> req) {
//TBD
unimplemented();
return make_ready_future<json::json_return_type>(0);
});
sp::get_range_latency.set(r, [&ctx](std::unique_ptr<http::request> req) {
return total_latency(ctx, &service::storage_proxy_stats::stats::range);
sp::get_cas_write_metrics_condition_not_met.set(r, [](std::unique_ptr<request> req) {
//TBD
unimplemented();
return make_ready_future<json::json_return_type>(0);
});
sp::get_cas_read_metrics_unfinished_commit.set(r, [](std::unique_ptr<request> req) {
//TBD
unimplemented();
return make_ready_future<json::json_return_type>(0);
});
sp::get_cas_read_metrics_contention.set(r, [](std::unique_ptr<request> req) {
//TBD
unimplemented();
return make_ready_future<json::json_return_type>(0);
});
sp::get_cas_read_metrics_condition_not_met.set(r, [](std::unique_ptr<request> req) {
//TBD
unimplemented();
return make_ready_future<json::json_return_type>(0);
});
sp::get_read_metrics_timeouts.set(r, [&ctx](std::unique_ptr<request> req) {
return sum_timed_rate_as_long(ctx.sp, &proxy::stats::read_timeouts);
});
sp::get_read_metrics_unavailables.set(r, [&ctx](std::unique_ptr<request> req) {
return sum_timed_rate_as_long(ctx.sp, &proxy::stats::read_unavailables);
});
sp::get_range_metrics_timeouts.set(r, [&ctx](std::unique_ptr<request> req) {
return sum_timed_rate_as_long(ctx.sp, &proxy::stats::range_slice_timeouts);
});
sp::get_range_metrics_unavailables.set(r, [&ctx](std::unique_ptr<request> req) {
return sum_timed_rate_as_long(ctx.sp, &proxy::stats::range_slice_unavailables);
});
sp::get_write_metrics_timeouts.set(r, [&ctx](std::unique_ptr<request> req) {
return sum_timed_rate_as_long(ctx.sp, &proxy::stats::write_timeouts);
});
sp::get_write_metrics_unavailables.set(r, [&ctx](std::unique_ptr<request> req) {
return sum_timed_rate_as_long(ctx.sp, &proxy::stats::write_unavailables);
});
sp::get_read_metrics_timeouts_rates.set(r, [&ctx](std::unique_ptr<request> req) {
return sum_timed_rate_as_obj(ctx.sp, &proxy::stats::read_timeouts);
});
sp::get_read_metrics_unavailables_rates.set(r, [&ctx](std::unique_ptr<request> req) {
return sum_timed_rate_as_obj(ctx.sp, &proxy::stats::read_unavailables);
});
sp::get_range_metrics_timeouts_rates.set(r, [&ctx](std::unique_ptr<request> req) {
return sum_timed_rate_as_obj(ctx.sp, &proxy::stats::range_slice_timeouts);
});
sp::get_range_metrics_unavailables_rates.set(r, [&ctx](std::unique_ptr<request> req) {
return sum_timed_rate_as_obj(ctx.sp, &proxy::stats::range_slice_unavailables);
});
sp::get_write_metrics_timeouts_rates.set(r, [&ctx](std::unique_ptr<request> req) {
return sum_timed_rate_as_obj(ctx.sp, &proxy::stats::write_timeouts);
});
sp::get_write_metrics_unavailables_rates.set(r, [&ctx](std::unique_ptr<request> req) {
return sum_timed_rate_as_obj(ctx.sp, &proxy::stats::write_unavailables);
});
sp::get_range_metrics_latency_histogram_depricated.set(r, [&ctx](std::unique_ptr<request> req) {
return sum_histogram_stats(ctx.sp, &proxy::stats::range);
});
sp::get_write_metrics_latency_histogram_depricated.set(r, [&ctx](std::unique_ptr<request> req) {
return sum_histogram_stats(ctx.sp, &proxy::stats::write);
});
sp::get_read_metrics_latency_histogram_depricated.set(r, [&ctx](std::unique_ptr<request> req) {
return sum_histogram_stats(ctx.sp, &proxy::stats::read);
});
sp::get_range_metrics_latency_histogram.set(r, [&ctx](std::unique_ptr<request> req) {
return sum_timer_stats(ctx.sp, &proxy::stats::range);
});
sp::get_write_metrics_latency_histogram.set(r, [&ctx](std::unique_ptr<request> req) {
return sum_timer_stats(ctx.sp, &proxy::stats::write);
});
sp::get_read_metrics_latency_histogram.set(r, [&ctx](std::unique_ptr<request> req) {
return sum_timer_stats(ctx.sp, &proxy::stats::read);
});
sp::get_read_estimated_histogram.set(r, [&ctx](std::unique_ptr<request> req) {
return sum_estimated_histogram(ctx, &proxy::stats::estimated_read);
});
sp::get_read_latency.set(r, [&ctx](std::unique_ptr<request> req) {
return total_latency(ctx, &proxy::stats::read);
});
sp::get_write_estimated_histogram.set(r, [&ctx](std::unique_ptr<request> req) {
return sum_estimated_histogram(ctx, &proxy::stats::estimated_write);
});
sp::get_write_latency.set(r, [&ctx](std::unique_ptr<request> req) {
return total_latency(ctx, &proxy::stats::write);
});
sp::get_range_estimated_histogram.set(r, [&ctx](std::unique_ptr<request> req) {
return sum_timer_stats(ctx.sp, &proxy::stats::range);
});
sp::get_range_latency.set(r, [&ctx](std::unique_ptr<request> req) {
return total_latency(ctx, &proxy::stats::range);
});
}
void unset_storage_proxy(http_context& ctx, routes& r) {
sp::get_total_hints.unset(r);
sp::get_hinted_handoff_enabled.unset(r);
sp::set_hinted_handoff_enabled.unset(r);
sp::get_hinted_handoff_enabled_by_dc.unset(r);
sp::set_hinted_handoff_enabled_by_dc_list.unset(r);
sp::get_max_hint_window.unset(r);
sp::set_max_hint_window.unset(r);
sp::get_max_hints_in_progress.unset(r);
sp::set_max_hints_in_progress.unset(r);
sp::get_hints_in_progress.unset(r);
sp::get_rpc_timeout.unset(r);
sp::set_rpc_timeout.unset(r);
sp::get_read_rpc_timeout.unset(r);
sp::set_read_rpc_timeout.unset(r);
sp::get_write_rpc_timeout.unset(r);
sp::set_write_rpc_timeout.unset(r);
sp::get_counter_write_rpc_timeout.unset(r);
sp::set_counter_write_rpc_timeout.unset(r);
sp::get_cas_contention_timeout.unset(r);
sp::set_cas_contention_timeout.unset(r);
sp::get_range_rpc_timeout.unset(r);
sp::set_range_rpc_timeout.unset(r);
sp::get_truncate_rpc_timeout.unset(r);
sp::set_truncate_rpc_timeout.unset(r);
sp::reload_trigger_classes.unset(r);
sp::get_read_repair_attempted.unset(r);
sp::get_read_repair_repaired_blocking.unset(r);
sp::get_read_repair_repaired_background.unset(r);
sp::get_schema_versions.unset(r);
sp::get_cas_read_timeouts.unset(r);
sp::get_cas_read_unavailables.unset(r);
sp::get_cas_write_timeouts.unset(r);
sp::get_cas_write_unavailables.unset(r);
sp::get_cas_write_metrics_unfinished_commit.unset(r);
sp::get_cas_write_metrics_contention.unset(r);
sp::get_cas_write_metrics_condition_not_met.unset(r);
sp::get_cas_write_metrics_failed_read_round_optimization.unset(r);
sp::get_cas_read_metrics_unfinished_commit.unset(r);
sp::get_cas_read_metrics_contention.unset(r);
sp::get_read_metrics_timeouts.unset(r);
sp::get_read_metrics_unavailables.unset(r);
sp::get_range_metrics_timeouts.unset(r);
sp::get_range_metrics_unavailables.unset(r);
sp::get_write_metrics_timeouts.unset(r);
sp::get_write_metrics_unavailables.unset(r);
sp::get_read_metrics_timeouts_rates.unset(r);
sp::get_read_metrics_unavailables_rates.unset(r);
sp::get_range_metrics_timeouts_rates.unset(r);
sp::get_range_metrics_unavailables_rates.unset(r);
sp::get_write_metrics_timeouts_rates.unset(r);
sp::get_write_metrics_unavailables_rates.unset(r);
sp::get_range_metrics_latency_histogram_depricated.unset(r);
sp::get_write_metrics_latency_histogram_depricated.unset(r);
sp::get_read_metrics_latency_histogram_depricated.unset(r);
sp::get_range_metrics_latency_histogram.unset(r);
sp::get_write_metrics_latency_histogram.unset(r);
sp::get_cas_write_metrics_latency_histogram.unset(r);
sp::get_cas_read_metrics_latency_histogram.unset(r);
sp::get_view_write_metrics_latency_histogram.unset(r);
sp::get_read_metrics_latency_histogram.unset(r);
sp::get_read_estimated_histogram.unset(r);
sp::get_read_latency.unset(r);
sp::get_write_estimated_histogram.unset(r);
sp::get_write_latency.unset(r);
sp::get_range_estimated_histogram.unset(r);
sp::get_range_latency.unset(r);
}
}

View File

@@ -1,21 +1,30 @@
/*
* Copyright (C) 2015-present ScyllaDB
* Copyright (C) 2015 ScyllaDB
*/
/*
* SPDX-License-Identifier: AGPL-3.0-or-later
* 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 <seastar/core/sharded.hh>
#include "api.hh"
namespace service { class storage_service; }
namespace api {
void set_storage_proxy(http_context& ctx, httpd::routes& r, sharded<service::storage_service>& ss);
void unset_storage_proxy(http_context& ctx, httpd::routes& r);
void set_storage_proxy(http_context& ctx, routes& r);
}

File diff suppressed because it is too large Load Diff

View File

@@ -1,87 +1,30 @@
/*
* Copyright (C) 2015-present ScyllaDB
* Copyright (C) 2015 ScyllaDB
*/
/*
* SPDX-License-Identifier: AGPL-3.0-or-later
* 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 <iostream>
#include <seastar/core/sharded.hh>
#include "api.hh"
#include "db/data_listeners.hh"
namespace cql_transport { class controller; }
class thrift_controller;
namespace db {
class snapshot_ctl;
namespace view {
class view_builder;
}
class system_keyspace;
}
namespace netw { class messaging_service; }
class repair_service;
namespace cdc { class generation_service; }
class sstables_loader;
namespace gms {
class gossiper;
}
namespace api {
// verify that the keyspace is found, otherwise a bad_param_exception exception is thrown
// containing the description of the respective keyspace error.
sstring validate_keyspace(http_context& ctx, sstring ks_name);
void set_storage_service(http_context& ctx, routes& r);
// verify that the keyspace parameter is found, otherwise a bad_param_exception exception is thrown
// containing the description of the respective keyspace error.
sstring validate_keyspace(http_context& ctx, const httpd::parameters& param);
// splits a request parameter assumed to hold a comma-separated list of table names
// verify that the tables are found, otherwise a bad_param_exception exception is thrown
// containing the description of the respective no_such_column_family error.
// Returns an empty vector if no parameter was found.
// If the parameter is found and empty, returns a list of all table names in the keyspace.
std::vector<sstring> parse_tables(const sstring& ks_name, http_context& ctx, const std::unordered_map<sstring, sstring>& query_params, sstring param_name);
struct table_info {
sstring name;
table_id id;
};
// splits a request parameter assumed to hold a comma-separated list of table names
// verify that the tables are found, otherwise a bad_param_exception exception is thrown
// containing the description of the respective no_such_column_family error.
// Returns a vector of all table infos given by the parameter, or
// if the parameter is not found or is empty, returns a list of all table infos in the keyspace.
std::vector<table_info> parse_table_infos(const sstring& ks_name, http_context& ctx, const std::unordered_map<sstring, sstring>& query_params, sstring param_name);
void set_storage_service(http_context& ctx, httpd::routes& r, sharded<service::storage_service>& ss, gms::gossiper& g, sharded<cdc::generation_service>& cdc_gs, sharded<db::system_keyspace>& sys_ls);
void set_sstables_loader(http_context& ctx, httpd::routes& r, sharded<sstables_loader>& sst_loader);
void unset_sstables_loader(http_context& ctx, httpd::routes& r);
void set_view_builder(http_context& ctx, httpd::routes& r, sharded<db::view::view_builder>& vb);
void unset_view_builder(http_context& ctx, httpd::routes& r);
void set_repair(http_context& ctx, httpd::routes& r, sharded<repair_service>& repair);
void unset_repair(http_context& ctx, httpd::routes& r);
void set_transport_controller(http_context& ctx, httpd::routes& r, cql_transport::controller& ctl);
void unset_transport_controller(http_context& ctx, httpd::routes& r);
void set_rpc_controller(http_context& ctx, httpd::routes& r, thrift_controller& ctl);
void unset_rpc_controller(http_context& ctx, httpd::routes& r);
void set_snapshot(http_context& ctx, httpd::routes& r, sharded<db::snapshot_ctl>& snap_ctl);
void unset_snapshot(http_context& ctx, httpd::routes& r);
seastar::future<json::json_return_type> run_toppartitions_query(db::toppartitions_query& q, http_context &ctx, bool legacy_request = false);
} // namespace api
namespace std {
std::ostream& operator<<(std::ostream& os, const api::table_info& ti);
} // namespace std
}

View File

@@ -1,9 +1,22 @@
/*
* Copyright (C) 2015-present ScyllaDB
* Copyright (C) 2015 ScyllaDB
*/
/*
* SPDX-License-Identifier: AGPL-3.0-or-later
* 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 "stream_manager.hh"
@@ -14,7 +27,6 @@
#include "gms/gossiper.hh"
namespace api {
using namespace seastar::httpd;
namespace hs = httpd::stream_manager_json;
@@ -22,7 +34,7 @@ static void set_summaries(const std::vector<streaming::stream_summary>& from,
json::json_list<hs::stream_summary>& to) {
if (!from.empty()) {
hs::stream_summary res;
res.cf_id = fmt::to_string(from.front().cf_id);
res.cf_id = boost::lexical_cast<std::string>(from.front().cf_id);
// For each stream_session, we pretend we are sending/receiving one
// file, to make it compatible with nodetool.
res.files = 1;
@@ -39,7 +51,7 @@ static hs::progress_info get_progress_info(const streaming::progress_info& info)
res.current_bytes = info.current_bytes;
res.direction = info.dir;
res.file_name = info.file_name;
res.peer = fmt::to_string(info.peer);
res.peer = boost::lexical_cast<std::string>(info.peer);
res.session_index = 0;
res.total_bytes = info.total_bytes;
return res;
@@ -62,7 +74,7 @@ static hs::stream_state get_state(
state.plan_id = result_future.plan_id.to_sstring();
for (auto info : result_future.get_coordinator().get()->get_all_session_info()) {
hs::stream_info si;
si.peer = fmt::to_string(info.peer);
si.peer = boost::lexical_cast<std::string>(info.peer);
si.session_index = 0;
si.state = info.state;
si.connecting = si.peer;
@@ -75,13 +87,13 @@ static hs::stream_state get_state(
return state;
}
void set_stream_manager(http_context& ctx, routes& r, sharded<streaming::stream_manager>& sm) {
void set_stream_manager(http_context& ctx, routes& r) {
hs::get_current_streams.set(r,
[&sm] (std::unique_ptr<request> req) {
return sm.invoke_on_all([] (auto& sm) {
[] (std::unique_ptr<request> req) {
return streaming::get_stream_manager().invoke_on_all([] (auto& sm) {
return sm.update_all_progress_info();
}).then([&sm] {
return sm.map_reduce0([](streaming::stream_manager& stream) {
}).then([] {
return streaming::get_stream_manager().map_reduce0([](streaming::stream_manager& stream) {
std::vector<hs::stream_state> res;
for (auto i : stream.get_initiated_streams()) {
res.push_back(get_state(*i.second.get()));
@@ -97,17 +109,17 @@ void set_stream_manager(http_context& ctx, routes& r, sharded<streaming::stream_
});
});
hs::get_all_active_streams_outbound.set(r, [&sm](std::unique_ptr<request> req) {
return sm.map_reduce0([](streaming::stream_manager& stream) {
hs::get_all_active_streams_outbound.set(r, [](std::unique_ptr<request> req) {
return streaming::get_stream_manager().map_reduce0([](streaming::stream_manager& stream) {
return stream.get_initiated_streams().size();
}, 0, std::plus<int64_t>()).then([](int64_t res) {
return make_ready_future<json::json_return_type>(res);
});
});
hs::get_total_incoming_bytes.set(r, [&sm](std::unique_ptr<request> req) {
hs::get_total_incoming_bytes.set(r, [](std::unique_ptr<request> req) {
gms::inet_address peer(req->param["peer"]);
return sm.map_reduce0([peer](streaming::stream_manager& sm) {
return streaming::get_stream_manager().map_reduce0([peer](streaming::stream_manager& sm) {
return sm.get_progress_on_all_shards(peer).then([] (auto sbytes) {
return sbytes.bytes_received;
});
@@ -116,8 +128,8 @@ void set_stream_manager(http_context& ctx, routes& r, sharded<streaming::stream_
});
});
hs::get_all_total_incoming_bytes.set(r, [&sm](std::unique_ptr<request> req) {
return sm.map_reduce0([](streaming::stream_manager& sm) {
hs::get_all_total_incoming_bytes.set(r, [](std::unique_ptr<request> req) {
return streaming::get_stream_manager().map_reduce0([](streaming::stream_manager& sm) {
return sm.get_progress_on_all_shards().then([] (auto sbytes) {
return sbytes.bytes_received;
});
@@ -126,9 +138,9 @@ void set_stream_manager(http_context& ctx, routes& r, sharded<streaming::stream_
});
});
hs::get_total_outgoing_bytes.set(r, [&sm](std::unique_ptr<request> req) {
hs::get_total_outgoing_bytes.set(r, [](std::unique_ptr<request> req) {
gms::inet_address peer(req->param["peer"]);
return sm.map_reduce0([peer] (streaming::stream_manager& sm) {
return streaming::get_stream_manager().map_reduce0([peer] (streaming::stream_manager& sm) {
return sm.get_progress_on_all_shards(peer).then([] (auto sbytes) {
return sbytes.bytes_sent;
});
@@ -137,8 +149,8 @@ void set_stream_manager(http_context& ctx, routes& r, sharded<streaming::stream_
});
});
hs::get_all_total_outgoing_bytes.set(r, [&sm](std::unique_ptr<request> req) {
return sm.map_reduce0([](streaming::stream_manager& sm) {
hs::get_all_total_outgoing_bytes.set(r, [](std::unique_ptr<request> req) {
return streaming::get_stream_manager().map_reduce0([](streaming::stream_manager& sm) {
return sm.get_progress_on_all_shards().then([] (auto sbytes) {
return sbytes.bytes_sent;
});
@@ -148,13 +160,4 @@ void set_stream_manager(http_context& ctx, routes& r, sharded<streaming::stream_
});
}
void unset_stream_manager(http_context& ctx, routes& r) {
hs::get_current_streams.unset(r);
hs::get_all_active_streams_outbound.unset(r);
hs::get_total_incoming_bytes.unset(r);
hs::get_all_total_incoming_bytes.unset(r);
hs::get_total_outgoing_bytes.unset(r);
hs::get_all_total_outgoing_bytes.unset(r);
}
}

View File

@@ -1,9 +1,22 @@
/*
* Copyright (C) 2015-present ScyllaDB
* Copyright (C) 2015 ScyllaDB
*/
/*
* SPDX-License-Identifier: AGPL-3.0-or-later
* 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
@@ -12,7 +25,6 @@
namespace api {
void set_stream_manager(http_context& ctx, httpd::routes& r, sharded<streaming::stream_manager>& sm);
void unset_stream_manager(http_context& ctx, httpd::routes& r);
void set_stream_manager(http_context& ctx, routes& r);
}

View File

@@ -1,31 +1,35 @@
/*
* Copyright (C) 2015-present ScyllaDB
* Copyright (C) 2015 ScyllaDB
*/
/*
* SPDX-License-Identifier: AGPL-3.0-or-later
* 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 "api/api-doc/system.json.hh"
#include "api/api.hh"
#include <seastar/core/reactor.hh>
#include <seastar/http/exception.hh>
#include "http/exception.hh"
#include "log.hh"
#include "replica/database.hh"
extern logging::logger apilog;
namespace api {
using namespace seastar::httpd;
namespace hs = httpd::system_json;
void set_system(http_context& ctx, routes& r) {
hs::get_system_uptime.set(r, [](const_req req) {
return std::chrono::duration_cast<std::chrono::milliseconds>(engine().uptime()).count();
});
hs::get_all_logger_names.set(r, [](const_req req) {
return logging::logger_registry().get_all_logger_names();
});
@@ -61,26 +65,6 @@ void set_system(http_context& ctx, routes& r) {
}
return json::json_void();
});
hs::write_log_message.set(r, [](const_req req) {
try {
logging::log_level level = boost::lexical_cast<logging::log_level>(std::string(req.get_query_param("level")));
apilog.log(level, "/system/log: {}", std::string(req.get_query_param("message")));
} catch (boost::bad_lexical_cast& e) {
throw bad_param_exception("Unknown logging level " + req.get_query_param("level"));
}
return json::json_void();
});
hs::drop_sstable_caches.set(r, [&ctx](std::unique_ptr<request> req) {
apilog.info("Dropping sstable caches");
return ctx.db.invoke_on_all([] (replica::database& db) {
return db.drop_caches();
}).then([] {
apilog.info("Caches dropped");
return json::json_return_type(json::json_void());
});
});
}
}

Some files were not shown because too many files have changed in this diff Show More