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
Paging on Secondary Index does not end (NodeJS: ReadableStream never ends) #4569
Comments
@denesb please comment |
@thales-gaddini please do provide the dml.cql It will also help if you can provide a tcp dump for the interaction with cassandra and the interaction with scylla capturing port 9042 (the info for no next page should be visible on the last response - and it will help to validate that on the wire its marked correctly flag: Has_more_pages https://github.com/apache/cassandra/blob/trunk/doc/native_protocol_v4.spec#L552) |
Hi @slivne , the dml is attached as a zip (just a really big file with a lot of inserts). I've also implemented the paging with the This is an example of the {
result: ResultSet {
info: {
queriedHost: '172.17.0.2:9042',
triedHosts: [Object],
speculativeExecutions: 0,
achievedConsistency: 10,
traceId: undefined,
warnings: undefined,
customPayload: undefined,
isSchemaInAgreement: true
},
rows: undefined,
rowLength: 20000,
columns: [ [Object], [Object] ],
pageState: '10ee2f4e7f73e043bea633a78769f5dfe300f07fffb1df00',
nextPage: [Function: nextPage]
}
} This is an example of the {
result: ResultSet {
info: {
queriedHost: '172.17.0.2:9042',
triedHosts: [Object],
speculativeExecutions: 0,
achievedConsistency: 10,
traceId: undefined,
warnings: undefined,
customPayload: undefined,
isSchemaInAgreement: true
},
rows: [],
rowLength: 20000,
columns: [ [Object], [Object] ],
pageState: null,
nextPage: undefined
}
} The {
result: ResultSet {
info: {
queriedHost: '172.17.0.2:9042',
triedHosts: [Object],
speculativeExecutions: 0,
achievedConsistency: 10,
traceId: undefined,
warnings: undefined,
customPayload: undefined,
isSchemaInAgreement: true
},
rows: [],
rowLength: 20000,
columns: [ [Object], [Object] ],
pageState: '00000000a40000001000000001000000040000000000271001280000000200000008000000fffc0480a1e1a69410000000c50f3598f6304f7baa73526537dc724cdfb1ff7fee47a20ac473782b0f0c912bde39858e010000003900000001190000001400000001000000080000003aaa8898163098470101190000001400000001000000080000003aaa8898163098470101010000006d41db007affc3b5d4340de9fa2b2a940100',
nextPage: [Function: nextPage]
}
} {
result: ResultSet {
info: {
queriedHost: '172.17.0.2:9042',
triedHosts: [Object],
speculativeExecutions: 0,
achievedConsistency: 10,
traceId: undefined,
warnings: undefined,
customPayload: undefined,
isSchemaInAgreement: true
},
rows: [],
rowLength: 20000,
columns: [ [Object], [Object] ],
pageState: '00000000a40000001000000001000000040000000000271001280000000200000008000000fffc0480a1e1a69410000000c50f3598f6304f7baa73526537dc724cdfb1ff7fee47a20ac473782b0f0c912bde39858e010000003900000001190000001400000001000000080000003aaa8898163098470101190000001400000001000000080000003aaa8898163098470101010000006d41db007affc3b5d4340de9fa2b2a940100',
nextPage: [Function: nextPage]
}
} (they are the same) About the tcp dump, I wasn't able to do it, could you help me with that? I've tried:
But none of them had any output. I've also tried to just tcpdump everything (without specifying a host or port), but nothing related to Scylla was logged. Should I tcpdump inside the Docker container? |
tcpdumping from docker worked. Here is a zip with the pcap files from both Scylla and Cassandra. As you said, the
|
@thales-gaddini thanks (and extra points for doing the initial analysis - nice going) There is one difference between cassandra and scylla when it comes to paging.
I am not sure the paging state is changing correctly between pages - yet will dig further after trying to reproduce this using the sample you shared. |
So trying your schema + dml.cql on a clean 3.0.5 docker container with python I can't reproduce
next the dump :) |
Yet a closer look it does reproduce the issue the last page has the flag Has_next_page set to True |
Is the paging state changing between different iterations? |
The driver should have continued fetching. I don't understand @slivne's capture. |
The paging state does not change frame 17623 result paging state frame 17624 next page request (passes the paging info) frame 17625 result paging state - the same as 17623 |
@avikivity I don;t understand the driver as well not fetching more pages - the program is the one I provided above |
so the query returns a total of 20000 rows split into 5000 per page I changed the pagesize from being 5000 to being 4999 So scylla can return 4999 and the last page not a full page (4 results) in this case the last page Has_more_pages is marked as FALSE So it seems the issue is with the last page and the next query that returns no results returns back the paging information while it shouldn't |
another oddity using the 5000 row page size , when I change the code and not use prepared statements it does not reproduce as well on docker 3.0.7 this has the problem e.g. the last page is an empty page and has Has_more_pages : True
this does not e.g. the last page is an empty page with Has_more_pages set to False
|
Indexed queries need to translate between view table paging state and base table paging state, in order to be able to page the results correctly. One of the stages of this translation is overwriting the paging state obtained from the base query, in order to return view paging state to the user, so it can be used for fetching next pages. Unfortunately, in the original implementation the paging state was overwritten only if more pages were available, while if 'remaining' pages were equal to 0, nothing was done. This is not enough, because the paging state of the base query needs to be overwritten unconditionally - otherwise a guard paging state value of 'remaining == 0' is returned back to the client along with 'has_more_pages = true', which will result in an infinite loop. This patch correctly overwrites the base paging state unconditionally. Fixes scylladb#4569
The test case makes sure that paging indexes does not result in an infinite loop. Refs scylladb#4569
The test case makes sure that paging indexes does not result in an infinite loop. Refs scylladb#4569
" Fixes #4569 This series fixes the infinite paging for indexed queries issue. Before this fix, paging indexes tended to end up in an infinite loop of returning pages with 0 results, but has_more_pages flag set to true, which confused the drivers. Tests: unit(dev) Branches: 3.0, 3.1 " * 'fix_infinite_paging_for_indexed_queries' of https://github.com/psarna/scylla: tests: add test case for finishing index paging cql3: fix infinite paging for indexed queries
Hey guys, thanks for the fix. When does it go from next to master? |
@thales-gaddini before being merged into master, it needs to pass the gating tests. If everything goes smoothly, then it may be a couple of hours, if not, then until I fix the bugs and resend :) |
" Fixes #4569 This series fixes the infinite paging for indexed queries issue. Before this fix, paging indexes tended to end up in an infinite loop of returning pages with 0 results, but has_more_pages flag set to true, which confused the drivers. Tests: unit(dev) Branches: 3.0, 3.1 " * 'fix_infinite_paging_for_indexed_queries' of https://github.com/psarna/scylla: tests: add test case for finishing index paging cql3: fix infinite paging for indexed queries
" Fixes #4569 This series fixes the infinite paging for indexed queries issue. Before this fix, paging indexes tended to end up in an infinite loop of returning pages with 0 results, but has_more_pages flag set to true, which confused the drivers. Tests: unit(dev) Branches: 3.0, 3.1 " * 'fix_infinite_paging_for_indexed_queries' of https://github.com/psarna/scylla: tests: add test case for finishing index paging cql3: fix infinite paging for indexed queries
" Fixes #4569 This series fixes the infinite paging for indexed queries issue. Before this fix, paging indexes tended to end up in an infinite loop of returning pages with 0 results, but has_more_pages flag set to true, which confused the drivers. Tests: unit(dev) Branches: 3.0, 3.1 " * 'fix_infinite_paging_for_indexed_queries' of https://github.com/psarna/scylla: tests: add test case for finishing index paging cql3: fix infinite paging for indexed queries (cherry picked from commit 9229afe)
Indexed queries need to translate between view table paging state and base table paging state, in order to be able to page the results correctly. One of the stages of this translation is overwriting the paging state obtained from the base query, in order to return view paging state to the user, so it can be used for fetching next pages. Unfortunately, in the original implementation the paging state was overwritten only if more pages were available, while if 'remaining' pages were equal to 0, nothing was done. This is not enough, because the paging state of the base query needs to be overwritten unconditionally - otherwise a guard paging state value of 'remaining == 0' is returned back to the client along with 'has_more_pages = true', which will result in an infinite loop. This patch correctly overwrites the base paging state unconditionally. Fixes scylladb#4569 (cherry picked from commit 88f3ade)
The test case makes sure that paging indexes does not result in an infinite loop. Refs scylladb#4569 (cherry picked from commit b8cadc9)
" This series backports fixing infinite paging for indexed queries to branch-3.0. Tests: unit(dev) " Fixes #4569 * 'fix_infinite_paging_for_indexed_queries_for_3.0' of https://github.com/psarna/scylla: tests: add test case for finishing index paging cql3: fix infinite paging for indexed queries
Backported to 3.0 and 3.1, removing from backport candidates. |
" This series backports fixing infinite paging for indexed queries to branch-3.0. Tests: unit(dev) " Fixes #4569 * 'fix_infinite_paging_for_indexed_queries_for_3.0' of https://github.com/psarna/scylla: tests: add test case for finishing index paging cql3: fix infinite paging for indexed queries
Problem
I'm using the npm cassandra client to make a select query using the stream method but the
end
event is never being triggered. This only happens in large collections where pagination is actually necessary. You won't see this behaviour for small sets.Installation details
I'm running scylla locally in docker with
Scylla version (or git commit hash): 3.0.5-0.20190414.403f66eca
Example:
DDL
Code
I've confirmed:
docker run cassandra:latest
)stream.readable
property is never set to false with Scylla, even seconds after the query is finishedend
event is not being triggered with the Scylla imageI can provide a dml.cql with a lot of inserts if it helps.
The text was updated successfully, but these errors were encountered: