-
-
Notifications
You must be signed in to change notification settings - Fork 1.2k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Group CQL keys into a single query [cql-tests] [tp-tests] #3879
Group CQL keys into a single query [cql-tests] [tp-tests] #3879
Conversation
3438649
to
94981f3
Compare
95bf7f4
to
4ffdd10
Compare
397d65e
to
7227a24
Compare
1d5e7bc
to
3fad6ae
Compare
docs/configs/janusgraph-cfg.md
Outdated
| storage.cql.grouping.slice-allowed | If `true` this allows multiple Slice queries which are allowed to be performed as non-range queries (i.e. direct equality operation) to be grouped together into a single CQL query via `IN` operator. Notice, currently only operations to fetch properties with Cardinality.SINGLE are allowed to be performed as non-range queries (edges fetching or properties with Cardinality SET or LIST won't be grouped together). | ||
If this option if `false` then each Slice query will be executed in a separate asynchronous CQL query even when grouping is allowed. | Boolean | true | MASKABLE | | ||
| storage.cql.grouping.slice-limit | Maximum amount of grouped together slice queries into a single CQL query. | ||
Notice, for ScyllaDB this option should not exceed the maximum number of distinct clustering key restrictions per query which can be changed by ScyllaDB configuration option `max-partition-key-restrictions-per-query` (https://enterprise.docs.scylladb.com/branch-2022.2/faq.html#how-can-i-change-the-maximum-number-of-in-restrictions). For AstraDB this limit is set to 20 and should be asked to be changed by AstraDB side via `partition_keys_in_select_failure_threshold` and `in_select_cartesian_product_failure_threshold` threshold configurations (https://docs.datastax.com/en/astra-serverless/docs/plan/planning.html#_cassandra_yaml). |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Users might be confused about should be asked to be changed by AstraDB side
. My first impression was one can ask AstraDB customer support to change this, but it seems the value is a fixed number and cannot be configured.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
The value is generally fixed, but you may ask AstraDB customer support and they may modify some threshold restrictions for your own use case.
That’s of course case by case, and it’s not guaranteed they will increase those numbers for your specific case.
However, it’s possible.
I will try to make it more clear in this message that it’s generally fixed, but there is a chance they will increase those limits per your request.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Changed the description, so that it would be clear for AstraDB side.
ExecutorService executorService, | ||
QueryBackPressure queryBackPressure) { | ||
this.session = session; | ||
sliceGroupingLimit = configuration.get(SLICE_GROUPING_LIMIT) < 1 ? 1 : configuration.get(SLICE_GROUPING_LIMIT); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
sliceGroupingLimit = configuration.get(SLICE_GROUPING_LIMIT) < 1 ? 1 : configuration.get(SLICE_GROUPING_LIMIT); | |
sliceGroupingLimit = Math.min(configuration.get(SLICE_GROUPING_LIMIT), 1); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I think it should be ‘max’ instead of ‘min’ here, but I agree that it looks better like you suggested. I will change it.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Oh yeah it should be max
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Applied Math.max
clopen(configuration); | ||
|
||
// test batching for `values()` | ||
TraversalMetrics profile = testLimitedBatch(() -> graph.traversal().V(cs).barrier(elementsAmount).values("singleProperty1", "singleProperty2", "singleProperty3", "setProperty", "listProperty"), configuration); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I have two questions:
testLimitedBatch
only compares results with and without batching. In the context of this PR, we would want to compare the results with and without key/column grouping.- Can one have one query that requests different properties (columns) from different vertices (keys)? For example,
g.V('v1').values('prop1').as('v1Prop1').V('v2').values('prop2').as('v2Prop2').select('v1Prop1', 'v2Prop2')
. I am not sure if a query like this will trigger your new logic at all, but I would like to make sure the grouping doesn't assume we request the same columns for different keys.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
- That makes sense. I will change to use a new method which compares results also with and without keys grouping.
- This logic is applied to a single query like ‘values(“prop1”, “prop2”)’ only because that’s how multi-query is working. However, we may have some data already cached for some vertices in transaction level cache of db-level cache which will modify what we are actually querying from the storage backend. Nevertheless, those will be different query groups. The grouping logic is aware that there might be different columns asked for different set of keys (thats what query group means there). Grouping logic won’t ever fetch columns which were not explicitly requested for specific keys.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Nevertheless, those will be different query groups. The grouping logic is aware that there might be different columns asked for different set of keys (thats what query group means there)
Cool 👍
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Updated this test to test each configuration with enabled / disabled grouping.
We should test this feature against as many cloud providers as possible. I can test it with Amazon Keyspace. |
Thank you! That will be very helpful! The one thing to notice, is queries with small number of keys. Generally you queries could be routed to different replicas. If you request 5 keys (for example) and assuming each separate CQL query takes 1 ms, then grouped query could potentially take longer than 1 ms because it is executed on the single node. I’m not sure how IN queries is executed inside that node (i.e. if it uses more than one CPU or not for evaluation), but this also could affect performance. Nevertheless, with queries where same token range keys amount be more than ‘CPUs over all replicas of that token range * grouping limit size’ - performance should not suffer theoretically. Now I also realised that for ‘janusgraph-scylla’ driver, token range only grouping won’t be enough because ScyllaDB driver is also shard aware. We should provide different grouping strategy for ScyllaDB and for token-aware only implementations (like Cassandra, AstraDB, etc.). Thus, I will update the code slightly to be able to select different grouping strategies (i.e. Token-Aware strategy or Token-And-Shard-Aware strategy or user provided custom strategy). I.e. when ScyllaDB driver is used we should group keys by token range + shard id. |
For ScyllaDB driver there is no an easy way to determine Shard id without making this API on ScyllaDB public. The issue to track this feature on ScyllaDB is: scylladb/java-driver#232 Discussion thread: https://forum.scylladb.com/t/how-to-get-node-sharding-information-using-scylla-java-driver/696 I propose to just making non-optimized strategy versions of grouping keys for ScyllaDB as for now (i.e. always execute each key separately for ScyllaDB driver), but when ScyllaDB has this API public, we can improve grouping implementation for ScyllaDB to also group tokens by shards in addition to grouping by Tokens. |
3fad6ae
to
f40ad5b
Compare
@li-boxuan I added a new configuration option |
The feature
but as long as this option is turned off, everything seems to be working as usual, so I am happy with this PR. It would be even better if you could mention that this feature does not work for Amazon Keyspace. |
Thank you @li-boxuan for checking it with Amazon Keyspace. I will surely reference that this feature doesn't work with Amazon Keyspaces. Screenshots below show the moment of enabling grouping feature. |
f40ad5b
to
93d0c43
Compare
@li-boxuan I added information about Amazon Keyspace as well as added one more grouping strategy which groups by same replica sets. The difference between grouping partition keys by token ranges and replica sets doesn't have a big difference, but knowing that a single node may have more than 1 token range grouping by same nodes may be slightly better (i.e. less groups will be created). However grouping by same token ranges means that those tokens leave close to each other on a disk (potentially less disk seeks). Overall the difference between these grouping strategy is small, but one grouping strategy might be preferred to another in some situations. |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I did another pass on the comments and most of them are just nitpicks.
.../java/org/janusgraph/diskstorage/cql/function/slice/AsyncCQLMultiKeyMultiColumnFunction.java
Outdated
Show resolved
Hide resolved
...c/main/java/org/janusgraph/diskstorage/cql/function/slice/AsyncCQLMultiKeySliceFunction.java
Outdated
Show resolved
Hide resolved
...src/main/java/org/janusgraph/diskstorage/cql/service/GroupingAsyncQueryExecutionService.java
Outdated
Show resolved
Hide resolved
93d0c43
to
f984038
Compare
Thank you @li-boxuan ! I force pushed the changes to resolve your comments here: https://github.com/JanusGraph/janusgraph/compare/93d0c4357e0af0d3072c89ba050c907b605c2f6e..f984038d6d7c056d4d9ad7397bf4153c683031cf Btw. The failing test |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
LGTM 👍
…ests] Fixes JanusGraph#3863 Signed-off-by: Oleksandr Porunov <alexandr.porunov@gmail.com>
f984038
to
a069074
Compare
Fixes #3863
Benchmarking
Local single instance Cassandra node
CQL benchmarks for local single instance Cassandra node is located here: https://gist.github.com/porunov/b0804e789ad6af56625230785ae8698b
There 2 tests for
master
branch (with enabled and disabledslice
query grouping) as well as 4 tests with different configurations of this PR.The observations from
local
testing is that generally performance increases 10-20% for cases when there are onlykeys
grouping used (i.e. multi-key cases but not multi-slice cases).I noticed that when there are few keys to be grouped (less than 500) then performance decreases (up to 20% performance decrease). I imagine it's due to less parallelism on deserialization side (1 thread instead of
CPUs * 2
threads) as well as increased amount of data to be fetched (when grouping keys we need to returnkey
beck per each column fetched).Another observation is that when both
slice
andkeys
grouping is triggered for a specific query the performance also drops about 5%. It could be that it's harder for Cassandra to process a query with twoIN
operators, but it's hard to verify the real reason.Overall, from the above tests it's clearly seen that:
keys
grouping is disabled - the performance is the same as inmaster
branch (i.e. no regression and no improvements).keys
grouping is enabled most of the queries with more than 700 keys gain additional performance improvements (mostly 10-20%, but goes up to 40% for some cases).keys
grouping.Distributed cluster
At home I started a distributed ScyllaDB cluster on 3 different machines and started JanusGraph benchmarking on the 4th machine via:
Unfortunately, processors, disks, and RAM are different and have different performance characteristics. Moreover this was a home WIFI network up to 1800 Mbps with possible network congestion from neighbor devices. Thus, this testing should be taken with a grain of salt because it could be influenced by a number of factors.
I couldn't finish all the tests yet, but quick first tests of
getAdjacentVerticesLocalCounts
show that whenkeys
grouping is enabled the performance actually drops 5-8%. I did run this test multiple times and the results are consistent (5%-8% percent performance decrease).It seems that the amount of queries in the distributed environment becomes less important that the amount of data fetched. Seems the additional
key
fetching per eachcolumn
influence performance more than I expected.It's hard to verify if the performance decrease is really related to the fact that additional data is being fetched or the way
IN
operator is processed on the storage node side.After a meeting with AstraDB developers they ensured me that this way of grouping partition keys together is legit and they usually don't recommend using
IN
operator because users usually don't group partition keys by same token ranges. However, in our case, it should be OK. Nevertheless, they warned that the checksum verification by coordinator node could still potentially be executed by multiple queries which could potentially affect performance, but should be verified case by case.Another grey area for now is internal parallelism on the
IN
query when executed by a node. It isn't obvious weather Cassandra / ScyllaDB utilizes multiple cores for processing a singleIN
query or not. In case only a single CPU is utilized forIN
query but multiple CPUs are utilized to process separate queries then the internal parallelism of separate queries could potentially be higher (in non overloaded clusters).Conclusion
Performance improvement of the
keys
grouping feature is questionable. It might improve performance in some cases and decrease performance in other cases.However, the obvious benefit is that this gives users a choice of enabling keys grouping for Serverless cases when users pay per read for each CQL query. In some cases performance might be not the main factor and the price reduction could be more needed.
I propose to have this option be disabled by default and users should enable it only for cases when they are sure they will gain benefits from grouping CQL queries.
Thank you for contributing to JanusGraph!
In order to streamline the review of the contribution we ask you
to ensure the following steps have been taken:
For all changes:
master
)?For code changes:
For documentation related changes: