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

Make PlanOptimizersStatsCollector thread-safe #16961

Merged
merged 1 commit into from
Apr 12, 2023
Merged

Conversation

Dith3r
Copy link
Member

@Dith3r Dith3r commented Apr 11, 2023

Description

Fix #16960

Additional context and related issues

Release notes

( ) 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`)


import static com.google.common.collect.ImmutableList.toImmutableList;

public class PlanOptimizersStatsCollector
{
private final Map<Class<?>, QueryPlanOptimizerStats> stats = new HashMap<>();
private final Map<Class<?>, QueryPlanOptimizerStats> stats = new ConcurrentHashMap<>();
Copy link
Member

Choose a reason for hiding this comment

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

The class was extracted from IterativeOptimizer in #16365
we perhaps can still be OK with additional overhead this brings

if we want to make this class thread-safe, then

  • mark it as @Threadsafe
  • QueryPlanOptimizerStats should become thread-safe too
  • we no longer need io.trino.sql.planner.iterative.IterativeOptimizer.Context#iterativeOptimizerStatsCollector as a separate field, and we don't need io.trino.execution.querystats.PlanOptimizersStatsCollector#add method.

(I am not totally sure though whether this is the way to go.)

cc @martint

Copy link
Member Author

Choose a reason for hiding this comment

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

add is used when statistics from a single iterative optimizer are merged to collect statistics for a query.

Copy link
Member

Choose a reason for hiding this comment

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

add is used when statistics from a single iterative optimizer are merged to collect statistics for a query.

this i know

however, now that PlanOptimizersStatsCollector is thread-safe, why create a new instance of that thing in IterativeOptimizer.Context?
... actually, it's not directly related to thread-safety -- it was redundant even before, so we can keep it as a follow-up

Copy link
Member Author

Choose a reason for hiding this comment

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

It was created to mirror the behavior that was implemented before. When the iterative optimizer exceeds the time limit, we log the top N rules that the current iterative optimizer has executed, as opposed to a top-level PlanOptimizersStatsCollector which collects statistics from rules across all optimizers.

Copy link
Member

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.

OTOH, when iterative optimizer fails (or query is cancelled), the query completion event top N rules won't include rules from the last iterative optimizer run, which are usually the most important ones.

can you create an issue & follow-up on this?

@Dith3r Dith3r requested a review from findepi April 12, 2023 06:51
The class can be accessed concurrently when query is cancelled during
planning. The planning thread is interrupted but may continue running,
and query completion even contains information got from
`PlanOptimizersStatsCollector`.
@findepi
Copy link
Member

findepi commented Apr 12, 2023

I've squashed commits and added some context to the commit message.

@Dith3r Dith3r changed the title PlanOptimizersStatsCollector threadsafe Make PlanOptimizersStatsCollector thread-safe Apr 12, 2023
@findepi
Copy link
Member

findepi commented Apr 12, 2023

CI #16972 , #15430

@findepi findepi merged commit a04c6df into trinodb:master Apr 12, 2023
@github-actions github-actions bot added this to the 413 milestone Apr 12, 2023
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.

Spurious query failure due to concurrent modification during getTopRuleStats
2 participants