Skip to content

add Init stage to SqlTask#19962

Closed
jklamer wants to merge 3 commits intotrinodb:masterfrom
jklamer:jklamer/TaskInitingState
Closed

add Init stage to SqlTask#19962
jklamer wants to merge 3 commits intotrinodb:masterfrom
jklamer:jklamer/TaskInitingState

Conversation

@jklamer
Copy link
Member

@jklamer jklamer commented Nov 30, 2023

Description

Additional context and related issues

Release notes

( ) This is not user-visible or is 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 Nov 30, 2023
// notify that task state changed (apart from initial RUNNING state notification)
if (newState != RUNNING) {
// notify that task state changed (apart from initial INITIALIZING state notification)
if (newState != INITIALIZING) {
Copy link
Member

Choose a reason for hiding this comment

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

do we want to notify about RUNNING now?

Copy link
Member Author

Choose a reason for hiding this comment

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

good question, if notifying useful for terminating states only then probably not?

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 don't care about notification from changing state from INITIALIZING -> RUNNING, then why an extra state is needed?

Copy link
Member Author

Choose a reason for hiding this comment

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

Extra state is needed to block execution of the query before all catalogs are loaded on the worker.

Copy link
Member Author

@jklamer jklamer Dec 12, 2023

Choose a reason for hiding this comment

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

The reasoning for not notifying on certain changes is not something I fully understand

{
requireNonNull(catalogs, "catalogs is null");
return this.catalogs.compareAndSet(null, requireNonNull(catalogs, "catalogs is null"));
return this.catalogs.compareAndSet(null, catalogs);
Copy link
Member

Choose a reason for hiding this comment

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

can it be called twice? Should you verify that compareAndSet returns true

Copy link
Member Author

Choose a reason for hiding this comment

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

This is allowed to be called more than once. In case of coordinator -> worker connection drop, or just timeout, or polling while waiting for task state == RUNNING

Copy link
Member Author

Choose a reason for hiding this comment

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

I think split assignments also come through here?

ReentrantReadWriteLock.ReadLock catalogInitLock = catalogsLock.readLock();
catalogInitLock.lock();
try {
connectorServicesProvider.ensureCatalogsLoaded(session, activeCatalogs);
Copy link
Member

Choose a reason for hiding this comment

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

looks like ensureCatalogsLoaded could return ListenableFuture

Copy link
Member Author

Choose a reason for hiding this comment

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

Wasn't sure how to ensure that the lock is acquired for the execution of the Future only when its needed

Copy link
Member

Choose a reason for hiding this comment

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

Or it cannot because it has to be complete its work with the lock boundary? what are those locks gaurding. Should't those be internal detail of ensureCatalogsLoaded

Copy link
Member Author

Choose a reason for hiding this comment

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

There is a race condition with pruneCatalog, where a catalog can be pruned immediately after creation, but before use. This lock guards against that

Copy link
Member

Choose a reason for hiding this comment

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

add code comment

Comment on lines +553 to +559
try {
catalogLoading.get(5, SECONDS);
sqlTask.setCatalogsLoaded(immediateVoidFuture());
}
Copy link
Member

Choose a reason for hiding this comment

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

why special casing here with active wait. Why not always use the listener of future.

Copy link
Member Author

Choose a reason for hiding this comment

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

To avoid adding a round trip to most task creates mostly.

Copy link
Member Author

Choose a reason for hiding this comment

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

Most cases catalogs will be "loaded" in well under 5 seconds, because they are already loaded.

Copy link
Member

Choose a reason for hiding this comment

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

I am not a big fan of blocking http server thread for 5 seconds. But maybe this is not a big deal. I will let others chime in.

cc: @dain, @sopel39, @findepi

@jklamer jklamer force-pushed the jklamer/TaskInitingState branch 3 times, most recently from 98549d3 to ed5fc1d Compare December 6, 2023 23:47
Copy link
Member

@losipiuk losipiuk left a comment

Choose a reason for hiding this comment

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

This is probably mostly fine. But I want more eyes on this one.

@sopel39 @dain @findepi

Copy link
Member

Choose a reason for hiding this comment

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

Update comment what this state means

Copy link
Member

Choose a reason for hiding this comment

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

nit: use proper variable name in error message

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 this will be triggered on failure on future too. You do not want that.

Replace those two lines with:

            addCallback(catalogsLoadedFuture, new FutureCallback<>()
            {
                @Override
                public void onSuccess(Void result)
                {
                    taskStateMachine.transitionToRunning();
                }

                @Override
                public void onFailure(Throwable t)
                {
                    taskStateMachine.failed(t);
                }
            }, directExecutor());

Copy link
Member Author

Choose a reason for hiding this comment

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

ah got it.

Comment on lines +553 to +559
try {
catalogLoading.get(5, SECONDS);
sqlTask.setCatalogsLoaded(immediateVoidFuture());
}
Copy link
Member

Choose a reason for hiding this comment

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

I am not a big fan of blocking http server thread for 5 seconds. But maybe this is not a big deal. I will let others chime in.

cc: @dain, @sopel39, @findepi

ReentrantReadWriteLock.ReadLock catalogInitLock = catalogsLock.readLock();
catalogInitLock.lock();
try {
connectorServicesProvider.ensureCatalogsLoaded(session, activeCatalogs);
Copy link
Member

Choose a reason for hiding this comment

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

add code comment

Copy link
Member

Choose a reason for hiding this comment

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

+ 1 is not obvious here. Explain in code comment what is going on here.

@losipiuk
Copy link
Member

@pettyjamesm you can take a look too

// notify that task state changed (apart from initial RUNNING state notification)
if (newState != RUNNING) {
// notify that task state changed (apart from initial INITIALIZING state notification)
if (newState != INITIALIZING) {
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 don't care about notification from changing state from INITIALIZING -> RUNNING, then why an extra state is needed?

Copy link
Member

Choose a reason for hiding this comment

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

throw an error if compareAndSet failed. Can it be called multiple times?

Copy link
Member Author

Choose a reason for hiding this comment

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

yes. Update task can be called multiple times. Believe split assignments also come through this endpoint

Copy link
Member

Choose a reason for hiding this comment

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

When using this factory without SqlTaskManager

Is it only for testing? add a comment

Copy link
Member Author

Choose a reason for hiding this comment

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

Correct. Will do.

Copy link
Member

Choose a reason for hiding this comment

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

nit: Opt -> Optional

Copy link
Member

@sopel39 sopel39 Dec 12, 2023

Choose a reason for hiding this comment

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

you shouldn't update when pendingSourceSplitCount == 0, should you?

BTW: why if (taskStatusFetcher.getTaskStatus().getState() == INITIALIZING) is needed? It seems it would work without it. Also, ignoring splitAssignments looks wrong. Assertion (that it's empty) would be useful at very least

Copy link
Member Author

@jklamer jklamer Dec 12, 2023

Choose a reason for hiding this comment

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

the task isn't running, and coordinator shouldn't treat it as running. So an update loop until it is running was my plan

I felt continuing to assign splits to a task that is still in init stage didn't make sense, because they won't get processes.

Copy link
Member

Choose a reason for hiding this comment

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

Add a comment why you return here. Is it needed?

Additionally, you need to schedule update when task transitions from INITIALIZE to RUNNING.

You should add a test that query won't deadlock, e.g:

  1. Task is scheduled in INITIALIZE state
  2. All splits (single split) is scheduled in HttpRemoteTask
  3. Task transitions to RUNNING state
  4. Expected: HttpRemoteTask will schedule task update on transition from INITIALIZE -> RUNNING

Copy link
Member Author

Choose a reason for hiding this comment

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

I believe incrementing pendingRequestsCounter.incrementAndGet(); causes a reschedule of the update and will spin the update until the state is transitioned away. (The call site for this function has that logic).

Copy link
Member

@sopel39 sopel39 Dec 12, 2023

Choose a reason for hiding this comment

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

pendingRequestsCounter.incrementAndGet();

It won't cause reschedule on it's own. You need to call triggerUpdate.

and will spin the update until the state is transitioned away

We shouldn't spin TaskUpdateRequest actively until stage goes away, because TaskUpdateRequest (and TaskInfo responses) are expensive. The proper way to do it is to listen for state changes from taskStatusFetcher and act upon these (e.g. call triggerUpdate)

BTW: is it OK to send splits while task is initializing? If so, maybe you don't need special handling in HttpRemoteTask

Copy link
Member Author

Choose a reason for hiding this comment

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

Is the taskStatusFetcher always polling? And can used to block query execution until state ~= Initializing?

Copy link
Member Author

Choose a reason for hiding this comment

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

I think I found a way to block stage execution until initialized is complete. I can probably remove this. But our latency would be limited by the latency of ContinuousTaskStatusFetcher?

Copy link
Member Author

@jklamer jklamer Dec 19, 2023

Choose a reason for hiding this comment

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

Is it a problem if a task fails to initialize after accepting splits?

Copy link
Member

Choose a reason for hiding this comment

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

Is the taskStatusFetcher always polling? And can used to block query execution until state ~= Initializing?

Yes. It's using long pulling.

Is it a problem if a task fails to initialize after accepting splits?

Maybe not, but I don't know exactly how dynamic catalogs work

Copy link
Member

Choose a reason for hiding this comment

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

why lock is needed? Can catalogs be updated concurrently? If so, how do you guarantee that they are updated in correct order?

Copy link
Member Author

Choose a reason for hiding this comment

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

The lock is needed for a race condition with catalog pruning. The lock ensures proper ordering

Copy link
Member

Choose a reason for hiding this comment

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

The lock ensures proper ordering

What does prevent execution of:

init of catalogs
prune catalogs

or

prune catalogs
init of catalogs

lock on it's own doesn't enforce order. Do I miss something?

Copy link
Member Author

Choose a reason for hiding this comment

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

This is the PR for it: was able to replicate in unit test. The general order was that while prune was collecting the catalogs for a task, it was possible for a task to be assigned catalogs, and load them, only for the pruning to run right after before the query executed.

Copy link
Member

Choose a reason for hiding this comment

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

#19683 was able to replicate in unit test.

How does that PR relates to change here?

Copy link
Member Author

Choose a reason for hiding this comment

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

That PR introduced the lock. This PR just move the location of the lock

Copy link
Member

Choose a reason for hiding this comment

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

this will introduce additional query latency (task won't be updated with splits for 5s as only one request at a time is allowed). We need to figure another way to do this.

Copy link
Member Author

Choose a reason for hiding this comment

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

Let's chat with @dain. This comes directly from conversations with him. My understanding is that this will only wait 5 seconds if the catalogs are still loading ( and the query cannot execute anyway)

Copy link
Member

Choose a reason for hiding this comment

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

My understanding is that this will only wait 5 seconds if the catalogs are still loading ( and the query cannot execute anyway)

Do I understand correctly that catalogs can be different per query? What if catalogs load in 10 milliseconds? In that case split assignments will be blocked for remainder of 5s.

Lastly, is 5s always sufficient?

Copy link
Member Author

Choose a reason for hiding this comment

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

Is that how the future timeout works? It always blocks 5s?

5s is not always sufficient for loading all catalogs, which is why I need to block execution until the task has initialized all catalogs.

Copy link
Member Author

Choose a reason for hiding this comment

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

Just tested. I believe this early terminates

Copy link
Member

Choose a reason for hiding this comment

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

5s is not always sufficient for loading all catalogs, which is why I need to block execution until the task has initialized all catalogs.

I think this means it should not be time based, but event based (e.g. some future)

@losipiuk
Copy link
Member

@dain can you please take a look a this one?

@losipiuk losipiuk requested a review from dain December 14, 2023 14:46
@findepi
Copy link
Member

findepi commented Dec 15, 2023

I don't have opinion about the code changes yet, but would be nice to fill PR Description to explain why it's a good change to have.

@jklamer
Copy link
Member Author

jklamer commented Dec 19, 2023

Will try and get some testing started to prove this out as a POC and see what's missing.

Copy link
Member Author

Choose a reason for hiding this comment

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

These execute but I can't tell if this is sufficient to test

@jklamer jklamer force-pushed the jklamer/TaskInitingState branch 3 times, most recently from e17ba4c to d1492fa Compare December 20, 2023 16:23
@jklamer jklamer force-pushed the jklamer/TaskInitingState branch from d1492fa to c18746d Compare December 26, 2023 22:52
@jklamer jklamer force-pushed the jklamer/TaskInitingState branch from c18746d to 1c9b52d Compare December 26, 2023 22:55
@jklamer
Copy link
Member Author

jklamer commented Jan 26, 2024

scrapping

@jklamer jklamer closed this Jan 26, 2024
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.

4 participants