Skip to content

Conversation

@JunhyungSong
Copy link
Member

Description

Before the change HttpRemotTask will send all pending splits in one taskUpdateRequest, sometimes causing large request for each task in coordinator.

This change will enable the ability to dynamically split large taskUpdateRequest into several smaller batch to send.

Introduced 4 parameters regarding to this feature:

query.remote-task.enable-adaptive-request-size : to control the enable/disable of the feature
query.remote-task.max-request-size: the max size of taskUpdateRequest
query.remote-task.request-size-headroom: the expected mean value of the taskUpdateRequest will be (request-size-limit - request-size-limit-headroom)
query.remote-task.guaranteed-splits-per-task: guaranteed splits that will be sent in taskUpdateRequest for each task, to prevent deadlock

New revision of #10013.

Additional context and related issues

The problematic situation:
Table has 3884 columns.
Table schema has a “parameters” field with the whole table schema string(~ 300k characters)
Size of each split object(in task update request) is big due to large strings in table properties.
If taskUpdateRequest contains a lot of huge splits like above and send it to the worker through http, it causes coordinator crash due to out of memory.
So, this PR tried to limit the size of taskUpdateRequest not only using max_unacknowledged_splits_per_task, but also using max_remote_task_request_size.

Release notes

( ) This is not user-visible or docs only and no release notes are required.
(x) 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`)

@JunhyungSong
Copy link
Member Author

JunhyungSong commented Jan 14, 2023

This can mitigate concerns mentioned by @pettyjamesm in #15579.

Copy link
Member

Choose a reason for hiding this comment

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

This condition was always evaluated as false due to difference in types.

Copy link
Member

Choose a reason for hiding this comment

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

nit. numSplits != 0

Copy link
Member

@Dith3r Dith3r Jan 16, 2023

Choose a reason for hiding this comment

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

If we exceed request size, we should always end up with different number of splits if we did not hit guaranteedSplitsPerRequest. Doing operations on doubles and round by casting, we could end up with infinite loop, as size would be bigger than expected and newSplitBatchSize could stay the same.

Also, source splits could be only a part of requests, and we don't take this into account during limitation of request size.

Copy link
Member Author

Choose a reason for hiding this comment

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

Parts that can increase significantly is splitAssignments. fragment can increase as well, but it cannot be divided into smaller pieces. That's the reason why there is requestSizeHeadroom.

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 reschedule if splitAssignments size is equal guaranteedSplitsPerRequest and such request should be processed as it is.

Copy link
Member Author

Choose a reason for hiding this comment

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

If oldSplitBatchSize == guaranteedSplitsPerRequest, then newSplitBatchSize < oldSplitBatchSize will be always false.

Copy link
Member

@Dith3r Dith3r Jan 19, 2023

Choose a reason for hiding this comment

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

Shouldn't we test here (line 709) taskUpdateRequestJson.length > (maxRequestSize - requestSizeHeadroom)?

If we take requestSize = 6 mb, maxRequestSize = 7 mb, requestSizeHeadroom = 2 mb then in adjustSplitBatchSize expected size will be 5mb, and algorithm will trigger limiting of splits, but we will not trigger scheduleUpdate. Next request will be adjusted to newSplitBatchSize.

Copy link
Member Author

Choose a reason for hiding this comment

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

If requestSize = 6 mb and maxRequestSize = 7 mb, since requestSize < maxRequestSize, it doesn't need to reschedule. requestSizeHeadroom is for calculating the number of splits only when requestSize > maxRequestSize (given that a request contains not only splits, but also other data). Let me change the logic to call adjustSplitBatchSize only when requestSize > maxRequestSize.

Copy link
Member

Choose a reason for hiding this comment

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

How adjustment is tested here?

Copy link
Member Author

@JunhyungSong JunhyungSong Jan 19, 2023

Choose a reason for hiding this comment

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

Even though splits is divided into smaller requests, it will eventually send all 1000 splits.

Let me add another poll to check whether splits is actually divided or not.

Copy link
Member

Choose a reason for hiding this comment

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

Can we test if request size did not exceed our maximum?

Copy link
Member

Choose a reason for hiding this comment

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

Is this variable required, as its purpose is only to present log?

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. To prevent repetitive logs. But, let me remove this, since the log would be more fit to debug log level.

Copy link
Member

Choose a reason for hiding this comment

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

Why is this object property?

Copy link
Member

Choose a reason for hiding this comment

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

Why is this required in processTaskUpdate? This looks more like a way around not sending all splits during sendUpdate. Why pendingSplits instead of pendingSourceSplitsWeight?

Btw. Do you test how this waiting for task update impacts latency to send more splits if we hit request size limit?

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. I think pendingSourceSplitCount works as well.

Yes. I tested it and there was no noticeable performance impacts. Since sendUpdate is just assigning splits to workers, workers will have sufficient amount of works(splits) to do. guaranteedSplitsPerRequest will guarantee that as well.

Copy link
Member

Choose a reason for hiding this comment

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

Maybe use of linkedHashSet would be a better approach here?

Copy link
Member

Choose a reason for hiding this comment

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

My intention was to avoid sorting splits by having splits already inserted into a set which is sorted by sequenceId and just do poll of first splits from this set.

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 don't understand what you meant. (especially just do poll of first splits from this set) AFAIK, splits does not guarantee the sequenceId order.

Copy link
Contributor

@arhimondr arhimondr left a comment

Choose a reason for hiding this comment

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

Some comments

Copy link
Contributor

Choose a reason for hiding this comment

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

Should it be done under the if condition?

Copy link
Contributor

Choose a reason for hiding this comment

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

debug

Copy link
Contributor

Choose a reason for hiding this comment

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

debug

Copy link
Contributor

Choose a reason for hiding this comment

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

We should minimize the scope of syncrhonization

Copy link
Member Author

Choose a reason for hiding this comment

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

getSplitAssignments and adjustSplitBatchSize should be synchronized together.

Copy link
Member

Choose a reason for hiding this comment

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

Cannot we pass currentSplitBatchSize to getSplitAssignments?

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 me change splitBatchSize to AtomicInteger, then.

Copy link
Contributor

Choose a reason for hiding this comment

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

debug

Copy link
Contributor

Choose a reason for hiding this comment

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

debug

Copy link
Contributor

Choose a reason for hiding this comment

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

It would be great if it worked for any number / type of sources. Otherwise some queries (such as collocated joins) might still experience the same problem. Would it be possible to have something like targetSplitsPerSource and adjust it according to request size?

Copy link
Member Author

Choose a reason for hiding this comment

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

We can. But, can we do that as a follow-up of this PR?

Copy link
Contributor

Choose a reason for hiding this comment

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

It feels like you would need to change quite a bit, but maybe it is more incremental than it seems. Do you think it is worth having two increments?

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. I think most of cases there will be one partitioned source (assuming that collocated joins are not common). So, this PR will cover most of cases and unblock #15579.

@JunhyungSong JunhyungSong force-pushed the adaptive-remote-task-size branch from b567913 to 4a678f0 Compare January 19, 2023 01:57
Copy link
Member

Choose a reason for hiding this comment

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

Why not use ImmutableSet.Builder here?

Copy link
Member

Choose a reason for hiding this comment

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

nit. oldSplitBatchSize -> currentSplitBatchSize

@JunhyungSong JunhyungSong force-pushed the adaptive-remote-task-size branch from 4a678f0 to 9f274b8 Compare January 23, 2023 07:32
Copy link
Member

Choose a reason for hiding this comment

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

Testing here taskUpdateRequestJson.length > maxRequestSize disallow to increase currentSplitBatchSize if request size is smaller than expectedSize and we didn't exceed maxUnacknowledgedSplits. Cannot we have taskUpdateRequestJson.length > (maxRequestSize - headroom) in if at 704 line?

Also, it would be nice to test increment behavior.

Copy link
Member Author

Choose a reason for hiding this comment

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

taskUpdateRequestJson is symmetric of maxRequestSize. the condition in adjustSplitBatchSize should check maxRequestSizeInBytes instead of maxRequestSizeInBytes - requestSizeHeadroomInBytes. Let me update it accordingly.

Copy link
Contributor

@arhimondr arhimondr left a comment

Choose a reason for hiding this comment

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

Generally looks good, some small comments

Copy link
Contributor

Choose a reason for hiding this comment

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

nit: generally we prefer boolean property names to answer is question, e.g.: ADAPTIVE_PARTIAL_AGGREGATION_ENABLED, SPILL_ENABLED. There are some properties that answer "do" (e.g.: ENABLE_DYNAMIC_FILTERING) but generally it is not preffered. I would recommend calling this property (and respective config properties / variables) as REMOTE_TASK_ADAPTIVE_UPDATE_REQUEST_SIZE_ENABLED

Copy link
Contributor

Choose a reason for hiding this comment

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

maybe REMOTE_TASK_MAX_REQUEST_SIZE to be consistent with other two

Copy link
Contributor

Choose a reason for hiding this comment

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

nit: maxRequestSizeInBytes

Copy link
Contributor

Choose a reason for hiding this comment

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

nit: requestSizeHeadroomInBytes

Copy link
Contributor

Choose a reason for hiding this comment

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

nit: adaptiveUpdateRequestSizeEnabled

Copy link
Contributor

Choose a reason for hiding this comment

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

Please create a TODO issue and link it here

Copy link
Contributor

Choose a reason for hiding this comment

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

Why wasn't it necessary before?

Copy link
Member Author

Choose a reason for hiding this comment

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

Before this feature, always all pending splits were added to a request.

Copy link
Contributor

Choose a reason for hiding this comment

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

Why is the (requestSize < expectedSize && currentSplitBatchSize < maxUnacknowledgedSplits) condition necessary? adjustSplitBatchSize is only called when taskUpdateRequestJson.length > maxRequestSize, right?

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's a good point. I mistakenly moved taskUpdateRequestJson.length > maxRequestSize to the upper condition for addressing Kamil's comment. Let me fix it.

Before the change HttpRemotTask will send all pending splits in one
taskUpdateRequest, sometimes causing large request for each task in
coordinator.

This change will enable the ability to dynamically split large
taskUpdateRequest into several smaller batch to send.

Introduced 4 parameters regarding to this feature:

query.remote-task.enable-adaptive-request-size : to control the enable/disable of the feature
query.remote-task.max-request-size: the max size of taskUpdateRequest
query.remote-task.request-size-headroom: the expected mean value of the taskUpdateRequest will be (request-size-limit - request-size-limit-headroom)
query.remote-task.guaranteed-splits-per-task: guaranteed splits that will be sent in taskUpdateRequest for each task, to prevent deadlock
@JunhyungSong JunhyungSong force-pushed the adaptive-remote-task-size branch from 9f274b8 to 03cbc77 Compare January 24, 2023 06:56
@Dith3r
Copy link
Member

Dith3r commented Jan 24, 2023

LGTM % comments

@JunhyungSong
Copy link
Member Author

Can someone please help merge this?

@arhimondr arhimondr merged commit 19f18fb into trinodb:master Jan 30, 2023
@arhimondr
Copy link
Contributor

Thanks @JunhyungSong !

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.

3 participants