Skip to content

Conversation

@JunhyungSong
Copy link
Member

@JunhyungSong JunhyungSong commented Jan 26, 2023

Description

Currently, /v1/statement/queued/{queryId}/{slug}/{token} get request returns queued query state even if the query is already planning. It will not be redirected to /v1/statement/executing/{queryId}/{slug}/{token} get request until all planning tasks including plan distribution is completed. When planning takes long time or fails during planning, it can mislead callers because of this. So, it needs to set submitted when transitioning to PLANNING.

Additional context and related issues

Release notes

(v) This is not user-visible or docs only and no release notes are required.
( ) Release notes are required, please propose a release note for me.
( ) Release notes are required, with the following suggested text:

# Section
* Fix some things. ({issue}`issuenumber`)

@cla-bot cla-bot bot added the cla-signed label Jan 26, 2023
@JunhyungSong JunhyungSong requested a review from findepi January 26, 2023 23:00
@martint martint requested a review from dain January 27, 2023 00:40
@findepi
Copy link
Member

findepi commented Jan 27, 2023

@JunhyungSong thanks for a very nice PR description, this is very useful!

Currently, /v1/statement/queued/{queryId}/{slug}/{token} get request returns queued query state even if the query is already planning. It will not be redirected to /v1/statement/executing/{queryId}/{slug}/{token} get request until all planning tasks including plan distribution is completed. When planning takes long time or fails during planning, it can mislead callers because of this.

Callers should not infer any particular meaning from the paths used in nextUri.

Callers (client tools, UIs), should inspect the query state, so we should make sure the query state returned as "PLANNING" if the query is in the planning state.

So, it needs to set submitted when transitioning to PLANNING.

What is the effect of this (from the query protocol perspective)?

@JunhyungSong JunhyungSong force-pushed the transition-to-executing-when-planning branch from 701c9c5 to 96f938d Compare January 30, 2023 10:55
@JunhyungSong
Copy link
Member Author

JunhyungSong commented Jan 30, 2023

Callers (client tools, UIs), should inspect the query state, so we should make sure the query state returned as "PLANNING" if the query is in the planning state.

This PR is for that. Since callers should not infer nextUri as you mentioned, the engine needs to provide a right nextUri within the response for the caller's requests.

What is the effect of this (from the query protocol perspective)?

This will mark dispatched as true. And it will set coordinator location and change the nextUri to ExecutingStatementResource instead of QueuedStatementResource. With that, ExecutingStatementResource will return QueryResult with a correct state(e.g. PLANNING or STARTING instead of QUEUED).

@JunhyungSong JunhyungSong force-pushed the transition-to-executing-when-planning branch 2 times, most recently from fd02941 to 1c74619 Compare January 31, 2023 01:42
@findepi
Copy link
Member

findepi commented Jan 31, 2023

will return QueryResult with a correct state(e.g. PLANNING or STARTING instead of QUEUED).

that's sounds like something observable from end user perspective.

So, are you saying that queries in PLANNING state are not viewed as such by the client tools?
would it be possible to write a test for that?
for example with a mock connector you can insert a pause in some metadata method, allowing planning to take as much time as needed for test purposes.

@JunhyungSong
Copy link
Member Author

So, are you saying that queries in PLANNING state are not viewed as such by the client tools?

Correct.

would it be possible to write a test for that?
for example with a mock connector you can insert a pause in some metadata method, allowing planning to take as much time as needed for test purposes.

Can you recommend tests that I can refer to?

@findepi
Copy link
Member

findepi commented Feb 1, 2023

I do not. @electrum and @dain will know better for protocol-level changes and tests

@JunhyungSong
Copy link
Member Author

@electrum @dain do you have any ideas on how to add tests for this?

@JunhyungSong JunhyungSong requested a review from electrum February 1, 2023 19:55
@github-actions
Copy link

This pull request has gone a while without any activity. Tagging the Trino developer relations team: @bitsondatadev @colebow @mosabua

@github-actions github-actions bot added the stale label Feb 23, 2023
@bitsondatadev
Copy link
Member

@pettyjamesm would you also be able to review this one or approve if you've already done a review internally please?

@github-actions github-actions bot removed the stale label Feb 25, 2023
@sopel39 sopel39 removed their request for review February 27, 2023 14:23
Copy link
Member

@pettyjamesm pettyjamesm Feb 27, 2023

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

A thrown RejectedExecutionException will now trigger query failure instead of a stuck query in the previous implementation- which is good and probably something we could test for. Also note that a failure in minimumWorkerFuture could also have RejectedExecutionException thrown from queryExecutor and should probably just call stateMachine.transitionToFailed(throwable) inline instead of queueing it through queryExecutor.

The above was incorrect, queryExecutor is a cached threadpool executor and will not throw.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Talked offline. DispatchExecutor will not throw RejectedExecutionException unless the server is shutting down.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I'm good with this formulation of submitting a new future to handle queryExecution.start(), but would it be equivalent to call submitted.set(null); queryExecution.start(); here without a shift to another thread in the same threadpool?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Makes sense.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think closeExchangeIfNecessary(QueryInfo) needs to check queryInfo.getState().ordinal() > STARTING logic too, otherwise we could close the exchange before planning completes if that were to be called from another code path.

After that change, we should leave the closeExchangeIfNecessary and removePagesFromExchange as-is without needing to check isStarted at this point explicitly.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

As chatted separately, removePagesFromExchange needs the condition check as well since it needs to return null for columns and types when the query is not started, yet. And the other code path of calling closeExchangeIfNecessary(QueryInfo) already checks a query completion.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I would say that relying on two separate code-paths to check the same condition before calling closeExchangeIfNecessary(QueryInfo) is riskier than just having closeExchangeIfNecessary do the check internally (more robust than refactoring).

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Actually the condition here at https://github.com/trinodb/trino/blob/master/core/trino-main/src/main/java/io/trino/server/protocol/Query.java#L198 is a little different. And there will be three separate code-paths to check the same condition if we move the condition check. The first is inside closeExchangeIfNecessary. The second is inside removePagesFromExchange. And the third is for if (isStarted && (queryInfo.getOutputStage().isEmpty() || exchangeDataSource.isFinished())).

@JunhyungSong JunhyungSong force-pushed the transition-to-executing-when-planning branch 2 times, most recently from c699593 to 42ee128 Compare March 18, 2023 00:26
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I'm good with this formulation of submitting a new future to handle queryExecution.start(), but would it be equivalent to call submitted.set(null); queryExecution.start(); here without a shift to another thread in the same threadpool?

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I would say that relying on two separate code-paths to check the same condition before calling closeExchangeIfNecessary(QueryInfo) is riskier than just having closeExchangeIfNecessary do the check internally (more robust than refactoring).

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

There's a lot of ceremony going on here to override the whole ServerMainModule when we could just pass TestingQueryExecution directly into a LocalDispatchQuery constructor instead and assert off of that, which would be much simpler to verify.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

If we change TestingQueryExecution to use a CountdownLatch inside of QueryExecution#start() then we can just await that countdown instead of having to do a sleep-wait loop here.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This change can be avoided (since it's only to make something overrideable in tests) by just passing the TestQueryExecution directly to the LocalDispatchQuery and skipping all the dependency injection ceremony.

@JunhyungSong JunhyungSong force-pushed the transition-to-executing-when-planning branch 2 times, most recently from 3501f2d to 58d3546 Compare April 4, 2023 15:42
@JunhyungSong JunhyungSong force-pushed the transition-to-executing-when-planning branch from 58d3546 to 2efb1b8 Compare April 5, 2023 02:13
@JunhyungSong
Copy link
Member Author

The test failures seem irrelevant.

Copy link
Member

@dain dain left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

For the Transition to executing when begin planning commit, it is really important for me to be able to understand the diff if the commit contains only the changes absolutely necessary for the desired change to the state transitions (the part of the code base is really complex). The other minor changes can be in commits before of after that one.

@JunhyungSong JunhyungSong force-pushed the transition-to-executing-when-planning branch from 2efb1b8 to 6d0cef0 Compare April 14, 2023 19:08
@JunhyungSong
Copy link
Member Author

The test failure seems irrelevant.

Copy link
Member

@dain dain left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I'm still having some trouble understanding some of the additional changes in this. Also, I think the main commit can be simplified by rewriting the test.

@JunhyungSong JunhyungSong force-pushed the transition-to-executing-when-planning branch from 6d0cef0 to be238c9 Compare April 26, 2023 07:45
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We don't use abbreviations like this

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We should not be making this field mutable for a test. I'll explain how in the test code below.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This test is using TpchQueryRunner, which internally uses DistributedQueryRunner and you should just use that directly. Once you do that, you can add an additional Guice module to the runner. In the module you can register a new DDL task for some mocked up statement (see QueryExecutionFactoryModule). Then submit the fake statement directly using LocalDispatchQuery (which you can get from the injector the coordinator).

Alternatively, you can just create a LocalDispatchQuery directly and test the state changes you expect are happening on the state machine transitions.

@JunhyungSong JunhyungSong force-pushed the transition-to-executing-when-planning branch from be238c9 to d42639d Compare May 2, 2023 09:37
@JunhyungSong JunhyungSong force-pushed the transition-to-executing-when-planning branch from d42639d to 58acd00 Compare May 2, 2023 09:58
@JunhyungSong
Copy link
Member Author

The failure looks irrelevant.

@dain dain merged commit ff0e23a into trinodb:master May 2, 2023
@github-actions github-actions bot added this to the 416 milestone May 2, 2023
@JunhyungSong JunhyungSong deleted the transition-to-executing-when-planning branch May 2, 2023 22:06
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Development

Successfully merging this pull request may close these issues.

5 participants