Skip to content
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

Query Id doesn't use correct offset if the last query in command topic has an associated terminate command #6010

Closed
stevenpyzhang opened this issue Aug 13, 2020 · 0 comments · Fixed by #6278
Assignees
Labels
bug data-integrity P0 Denotes must-have for a given milestone
Milestone

Comments

@stevenpyzhang
Copy link
Member

stevenpyzhang commented Aug 13, 2020

Describe the bug
If the last query in the command topic is terminated and then the server is restarted, we don't correctly set the queryIdGenerator to the correct offset for the next query that's created.

To Reproduce
Start a ksqlDB with a new command topic

ksql> create stream test as select * from KSQL_PROCESSING_LOG;

 Message                           
-----------------------------------
 Created query with ID CSAS_TEST_0 
-----------------------------------
ksql> create stream test1 as select * from test;

 Message                            
------------------------------------
 Created query with ID CSAS_TEST1_3 
------------------------------------
ksql> TERMINATE CSAS_TEST1_3;

 Message           
-------------------
 Query terminated. 
-------------------

Restart server

ksql> create stream test2 as select * from test1;

 Message                            
------------------------------------
 Created query with ID CSAS_TEST2_3 
------------------------------------

Expected behavior
The queryId should be greater than 3 since it was already used for the CSAS for test1

Actual behaviour
The queryId 3 is used again. This violates the guarantee that query id is monotonically increasing as commands are put into the command topic.

Additional context
#5002 removed the query plans from CSAS/CTAS/INSERT statements that were terminated later in the command topic. That means when we execute these command, we don't return a QueryMetadata in the ExecuteResult from the KsqlEngine since there's no QueryPlan.

https://github.com/confluentinc/ksql/blob/master/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/InteractiveStatementExecutor.java#L243
We only set the queryID to the current offset if the ExecuteResult has a query in it. Therefore, if the last command that created a query is terminated, when the server restarts, the next QueryId wouldn't be set to the correct value.

This only affects the next query created after the restart, after it's created, we'll set the QueryIdGenerator to the correct offset.

Having duplicate query id's could lead to data corruption as we use the query id for both internal topics and schema names.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
bug data-integrity P0 Denotes must-have for a given milestone
Projects
None yet
Development

Successfully merging a pull request may close this issue.

5 participants