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

sql, stats: use the jobs framework for CREATE STATISTICS #34279

Merged
merged 1 commit into from
Jan 30, 2019

Conversation

rytaft
Copy link
Collaborator

@rytaft rytaft commented Jan 27, 2019

Prior to this commit, CREATE STATISTICS was a regular SQL statement
and followed the standard logic of AST -> planNode -> DistSQL physical
plan -> execution. This commit changes CREATE STATISTICS to use the
jobs framework, and as a result the createStatsNode has been
changed to execute a function which starts a CreateStats job. The job
is then responsible for performing DistSQL planning and execution.

There are several advantages to using the jobs framework:

  • Now CREATE STATISTICS jobs can easily be cancelled, paused and resumed,
    and viewed from the Admin UI.
  • Nodes can adopt the job if the original gateway node fails.
  • We will be able to use the JobID to lock creation of automatic
    statistics, so that only one automatic statistics job can run at
    a time. Job adoption will ensure that a dead node never prevents progress
    by holding a lock on stats creation (implementation of locking will be
    saved for the next PR).

Release note (sql change): CREATE STATISTICS now runs as a job instead
of as a regular SQL statement.

@cockroach-teamcity
Copy link
Member

This change is Reviewable

@rytaft rytaft changed the title opt: use the jobs framework for CREATE STATISTICS sql, stats: use the jobs framework for CREATE STATISTICS Jan 27, 2019
Copy link
Member

@RaduBerinde RaduBerinde left a comment

Choose a reason for hiding this comment

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

Cool! It will take some time to unpack this, it involves a lot of infrastructure I'm not familiar with.

The plan node hook stuff is to allow the implementation to live in CCL packages (which is not the case here). Are you sure that part is needed?

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @andy-kimball, @danhhz, @dt, @mjibson, @RaduBerinde, and @rytaft)


pkg/jobs/jobspb/jobs.proto, line 206 at r1 (raw file):

  ];
  sqlbase.TableDescriptor table = 2 [(gogoproto.nullable) = false];
  repeated Columns columns = 3 [(gogoproto.nullable) = false];

This should be a list of lists. Even though there's no syntax, a single CreateStats can in principle create stats for multiple sets of columns. In fact, that's what it does now when no columns are specified (in the future, we may want to populate this with those auto-determined columns).

@rytaft rytaft requested a review from a team as a code owner January 28, 2019 02:03
Copy link
Collaborator Author

@rytaft rytaft left a comment

Choose a reason for hiding this comment

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

Thanks!

The plan node hook stuff is to allow the implementation to live in CCL packages (which is not the case here). Are you sure that part is needed?

I'm not sure, but the reason I did it this way was because we don't want the normal AST -> planNode -> DistSQL plan -> execution pipeline. The planNode returned by planner.newPlan should just create a new job and stop. hookFnNode is a convenient way to do that since it just executes the function registered with AddPlanHook. I could put the createStats planNode back, but I think it would effectively do the same thing as hookFnNode.

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @andy-kimball, @danhhz, @dt, @mjibson, and @RaduBerinde)


pkg/jobs/jobspb/jobs.proto, line 206 at r1 (raw file):

Previously, RaduBerinde wrote…

This should be a list of lists. Even though there's no syntax, a single CreateStats can in principle create stats for multiple sets of columns. In fact, that's what it does now when no columns are specified (in the future, we may want to populate this with those auto-determined columns).

This is actually a list of lists -- it's a repeated element which itself contains a repeated element (I don't know of another way to represent that with protobufs). But I agree the naming doesn't make that obvious... I've changed the names to hopefully make it more clear.

Copy link
Member

@RaduBerinde RaduBerinde left a comment

Choose a reason for hiding this comment

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

Oh, I see, thanks, that makes sense!

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @andy-kimball, @danhhz, @dt, @mjibson, and @rytaft)


pkg/jobs/jobspb/jobs.proto, line 206 at r1 (raw file):

Previously, rytaft wrote…

This is actually a list of lists -- it's a repeated element which itself contains a repeated element (I don't know of another way to represent that with protobufs). But I agree the naming doesn't make that obvious... I've changed the names to hopefully make it more clear.

Sorry, my bad, missed the inner "repeated". I like the new name though, thanks!

Copy link
Contributor

@danhhz danhhz left a comment

Choose a reason for hiding this comment

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

All the pieces here look good! I didn't do a detailed review, but I can if you need an additional pair of eyes.

From a high level, my two large concerns with any new job are 1) overloading the jobs row with progress updates, 2) zombie jobs and 3) the user UX of the jobs entries.

  1. It's really easy to gum things up with txn contention updating the job progress. Changefeeds, for example, funnel progress info to a singleton distsql processor, which then rate limits and does the updates. I don't think it's worth the complexity in your case of a processor just for that, but given that the number of updates here scales with the number of processors, it's worth a few preventative measures to mitigate the risk.

First, you could rate limit the progress updates based on time. No need to update more than every second or so.

Even easier here, if your TODO is going to stick around for a while, since you're not actually updating the progress, you're just checking whether the job is still live, is to use a read-only txn. I don't recall offhand if the jobs framework already has an accessor to grab the state, but if it doesn't, feel free to add one. This would be used instead of the progress update in your worker loop. If it also surfaced the error, you could also use it the error handling bit in your planner code.

  1. Dunno the details of the lock you describe, but important to know that you can end up with zombie jobs. The framework does its best to let only one copy of each job run at a time, but there are various conditions around failures where we can end up with two copies of the job running at once: the blessed one and a zombie one. I think you'll be fine, since the worst thing that happens here is probably some wasted work (as opposed to restore/changefeeds/etc where we have correctness issues to worry about), but important to keep in mind.

  2. How often do we make these jobs? What does that look like in the jobs admin ui page? I've only been following the statistics stuff at a high level, but the number of these we run scales with traffic right? Maybe with num tables, too? Unless something has changed since I last looked, we don't garbage collect old jobs, so these could clutter things up. On the other hand, that page already has a dropdown for job type, so maybe that's sufficient. I just want to make sure this is considered.

I could put the createStats planNode back, but I think it would effectively do the same thing as hookFnNode.

This is likely what I'd recommend, make the planNode submit the job and then either return the jobID (so the client can poll progress) or have it wait until the job finishes. Most (all?) existing jobs do the former. We tend to avoid planHook when we can (which is any non-ccl code).

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @andy-kimball, @danhhz, @dt, @mjibson, and @rytaft)

Copy link
Member

@RaduBerinde RaduBerinde left a comment

Choose a reason for hiding this comment

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

@danhhz How does changefeed funnel progress info? Is it through a special ProducerMetadata? We already have a single "final" processor (SampleAggregator), we should be able to use the same mechanism.

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @andy-kimball, @dt, @mjibson, and @rytaft)

@danhhz
Copy link
Contributor

danhhz commented Jan 28, 2019

Changefeed progress is both used for some user facing stuff and super complicated and as a result, ended up needing stronger ordering guarantees than distsql metadata provides.

For this job, you have two "progress" concerns. One is stopping the workers when the job is paused or fails. This can be done with the read only txn I mentioned or (now that I think about it) handled with ctx cancellation. The latter is what changefeeds do. The second concern is actually updating the progress. Distsql metadata and doing it on SampleAggregator sgtm

Copy link
Collaborator Author

@rytaft rytaft left a comment

Choose a reason for hiding this comment

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

Thanks, @danhhz!

  1. Regarding the read-only txn: In my first attempt I actually took this approach and stored the transaction info in the CreateStatsDetails. Not sure I understand the suggestion to create an accessor for the jobs framework, but that sounds like it might be better -- can you explain a bit more? Alternatively, ctx cancellation also sounds like a good option. Is there a reason to prefer one over the other?

  2. Good to know about the possibility of zombie jobs, but I think you're right that the only consequence would be some wasted work and perhaps temporarily degraded performance. The purpose of the lock is for performance reasons -- there is no correctness issue with running multiple CREATE STATISTICS jobs at once.

  3. You're right that there could be a lot of CREATE STATISTICS jobs cluttering the admin UI. The TYPE dropdown does help, though (speaking of which, looks like I need to add CREATE STATISTICS to the dropdown....). Should I create an issue to garbage collect old jobs (at least CreateStats jobs)?

  4. Good to know about avoiding planHook -- I'll put the createStats node back.

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @andy-kimball, @dt, and @mjibson)

@danhhz
Copy link
Contributor

danhhz commented Jan 28, 2019

Regarding the read-only txn: In my first attempt I actually took this approach and stored the transaction info in the CreateStatsDetails. Not sure I understand the suggestion to create an accessor for the jobs framework, but that sounds like it might be better -- can you explain a bit more? Alternatively, ctx cancellation also sounds like a good option. Is there a reason to prefer one over the other?

Right now, in (*samplerProcessor).mainLoop you're using job.FractionProgressed (via progFn) to "periodically check that the job has not been paused or canceled." My original suggestion was to add a method to Job that fetches and returns the current status without also doing the write that FractionProgressed does. However, I think that's moot because it's better to use ctx cancellation. This is because the case of a zombie job, where you could end up with both the zombie job and the real job seeing the status of "running", so the zombie job wouldn't shut down. But the context for the zombie job will be cancelled, so it's a better signal. Basically, where you currently have if err := progFn(0); err != nil {, you'd replace it with if err := ctx.Err(); err != nil { and everything should work 🤞

Should I create an issue to garbage collect old jobs (at least CreateStats jobs)?

Jobs is unstaffed at this point, so if this needs to be done as a result of making stats a job, then you might have to pick this one off yourself : - ).

Copy link
Collaborator Author

@rytaft rytaft left a comment

Choose a reason for hiding this comment

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

@danhhz -- It seems like the context is not getting cancelled automatically. I could add a function to cancel the job's context and call it from OnFailOrCancel, but that wouldn't work for PAUSE JOB. Perhaps I should do both of your suggestions: add a function to get the status and add a function to cancel the context? Does that seem reasonable?

Jobs is unstaffed at this point, so if this needs to be done as a result of making stats a job, then you might have to pick this one off yourself : - ).

No problem! I'll create an issue and assign to myself. Doesn't seem urgent (the UI seems to work fine, especially given the TYPE dropdown), but I'll try to get it done before the release...

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @andy-kimball, @dt, and @mjibson)

@danhhz
Copy link
Contributor

danhhz commented Jan 28, 2019

The context cancellation does work, changefeeds rely on it. What test are you running that cause you to say it doesn't?

Usually, the tricky bit is getting the distsql error handling right, but it's worth the effort to investigate what's going on. Basically, if the job is paused or cancelled, the ctx your Resume function gets will be cancelled. If the job coordinator node is killed, then your SampleAggregator should get an error back from job.FractionProgressed and it should use that error to tear down the processor, which will cause distsql to propagate the ctx cancellation to the rest of your flow.

@rytaft rytaft force-pushed the auto-stats-job branch 2 times, most recently from 0c8a512 to 67be11f Compare January 29, 2019 16:15
@rytaft rytaft force-pushed the auto-stats-job branch 2 times, most recently from c32abce to 6642726 Compare January 29, 2019 21:17
Copy link
Contributor

@danhhz danhhz left a comment

Choose a reason for hiding this comment

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

jobs stuff lgtm modulo a few last comments. great work!

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @andy-kimball, @dt, @mjibson, and @rytaft)


pkg/sql/distsqlrun/sample_aggregator.go, line 19 at r2 (raw file):

import (
	"context"
	time "time"

nit: this is odd


pkg/sql/distsqlrun/sample_aggregator.go, line 145 at r2 (raw file):

	var job *jobs.Job
	jobID := s.spec.JobID
	if jobID != 0 {

are there times that you won't have a job id? maybe stick a comment here with when that happens


pkg/sql/distsqlrun/sample_aggregator.go, line 173 at r2 (raw file):

			// rows have been processed. The purpose of this function call is to
			// periodically check that the job has not been paused or canceled.
			if err := progFn(0); err != nil {

I was imagining doing this in response to getting distsql metadata with progress info filled in (rate limited with a util.Every(sampleAggregatorProgressInterval)). This works too, but my suggestion would save you the goroutine (and what you have here leaks the timer).


pkg/sql/distsqlrun/sample_aggregator.go, line 246 at r2 (raw file):

	// Report progress one last time so we don't write results if the job was
	// canceled.
	if err = progFn(1.0); err != nil {

Sorry if i've missed it, but do you have a job.Succeeded somewhere? You want to call it when the job finishes successfully to put the job into the terminal "succeeded" status

@rytaft rytaft force-pushed the auto-stats-job branch 2 times, most recently from 378eda9 to adb8afd Compare January 30, 2019 15:54
Copy link
Collaborator Author

@rytaft rytaft left a comment

Choose a reason for hiding this comment

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

Thanks! Updated.

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @andy-kimball, @danhhz, @dt, and @mjibson)


pkg/sql/distsqlrun/sample_aggregator.go, line 19 at r2 (raw file):

Previously, danhhz (Daniel Harrison) wrote…

nit: this is odd

Done.


pkg/sql/distsqlrun/sample_aggregator.go, line 145 at r2 (raw file):

Previously, danhhz (Daniel Harrison) wrote…

are there times that you won't have a job id? maybe stick a comment here with when that happens

Done.


pkg/sql/distsqlrun/sample_aggregator.go, line 173 at r2 (raw file):

Previously, danhhz (Daniel Harrison) wrote…

I was imagining doing this in response to getting distsql metadata with progress info filled in (rate limited with a util.Every(sampleAggregatorProgressInterval)). This works too, but my suggestion would save you the goroutine (and what you have here leaks the timer).

Oh nice! Didn't know that function existed. Done.


pkg/sql/distsqlrun/sample_aggregator.go, line 246 at r2 (raw file):

Previously, danhhz (Daniel Harrison) wrote…

Sorry if i've missed it, but do you have a job.Succeeded somewhere? You want to call it when the job finishes successfully to put the job into the terminal "succeeded" status

Seems like that's getting called inside Registry.resume (in jobs/registry.go) once createStatsResumer.Resume finishes successfully.

Copy link
Contributor

@danhhz danhhz left a comment

Choose a reason for hiding this comment

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

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @andy-kimball, @danhhz, @dt, @mjibson, and @rytaft)


pkg/sql/distsqlrun/sample_aggregator.go, line 246 at r2 (raw file):

Previously, rytaft wrote…

Seems like that's getting called inside Registry.resume (in jobs/registry.go) once createStatsResumer.Resume finishes successfully.

🤦‍♂️ carry on

Copy link
Collaborator Author

@rytaft rytaft left a comment

Choose a reason for hiding this comment

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

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @andy-kimball, @danhhz, @dt, and @mjibson)


pkg/sql/distsqlrun/sample_aggregator.go, line 246 at r2 (raw file):

Previously, danhhz (Daniel Harrison) wrote…

🤦‍♂️ carry on

👍 No worries!

@rytaft rytaft force-pushed the auto-stats-job branch 2 times, most recently from b7f559f to 13822ac Compare January 30, 2019 19:47
@RaduBerinde RaduBerinde mentioned this pull request Jan 30, 2019
Copy link
Member

@RaduBerinde RaduBerinde left a comment

Choose a reason for hiding this comment

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

:lgtm: Nice work!

Reviewable status: :shipit: complete! 1 of 0 LGTMs obtained (waiting on @andy-kimball, @danhhz, @dt, @mjibson, and @rytaft)


pkg/jobs/jobspb/jobs.proto, line 195 at r3 (raw file):

}

message CreateStatsDetails {

[nit] could use a comment mentioning this is related to table stats and CREATE STATS statement.


pkg/sql/create_stats.go, line 80 at r3 (raw file):

func (p *planner) CreateStatistics(ctx context.Context, n *tree.CreateStats) (planNode, error) {
	fn := func(ctx context.Context, resultsCh chan<- tree.Datums) error {

Do we need to store this as a function? Can't it just be a separate method?


pkg/sql/create_stats.go, line 221 at r3 (raw file):

}

type createStatsResumer struct {

I would add a description here and explain how the object is used. Especially that an instance of this is associated with a single job (because the methods in the interface suggest otherwise)


pkg/sql/create_stats.go, line 249 at r3 (raw file):

	dsp := p.DistSQLPlanner()
	err := r.evalCtx.ExecCfg.DB.Txn(ctx, func(ctx context.Context, txn *client.Txn) error {

[nit] can just return so we don't have yet another err instance


pkg/sql/create_stats.go, line 276 at r3 (raw file):

				return err
			}
			return err

We have two different errs here which makes this kind of confusing, I'd give the inner one a different name (eg jobErr)


pkg/sql/create_stats.go, line 293 at r3 (raw file):

// OnSuccess is part of the jobs.Resumer interface.
func (r *createStatsResumer) OnSuccess(ctx context.Context, txn *client.Txn, job *jobs.Job) error {

nit: maybe use _ instead of txn here to make it clear we're not using that


pkg/sql/create_stats.go, line 323 at r3 (raw file):

}

func createStatsResumeHook(typ jobspb.Type, settings *cluster.Settings) jobs.Resumer {

[nit] maybe add a comment, or if not I'd just declare this directly in the AddResumeHook call.

Prior to this commit, CREATE STATISTICS was a regular SQL statement
and followed the standard logic of AST -> planNode -> DistSQL physical
plan -> execution. This commit changes CREATE STATISTICS to use the
jobs framework, and as a result the createStats planNode has been
changed to execute a function which starts a CreateStats job. The job
is then responsible for performing DistSQL planning and execution.

There are several advantages to using the jobs framework:
- Now CREATE STATISTICS jobs can easily be cancelled, paused and resumed,
  and viewed from the Admin UI.
- Nodes can adopt the job if the original gateway node fails.
- We will be able to use the JobID to lock creation of automatic
  statistics, so that only one automatic statistics job can run at
  a time. Job adoption will ensure that a dead node never prevents progress
  by holding a lock on stats creation (implementation of locking will be
  saved for the next PR).

Release note (sql change): CREATE STATISTICS now runs as a job instead
of as a regular SQL statement.
Copy link
Collaborator Author

@rytaft rytaft left a comment

Choose a reason for hiding this comment

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

TFTR! Updated.

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (and 1 stale) (waiting on @andy-kimball, @dt, and @mjibson)


pkg/jobs/jobspb/jobs.proto, line 195 at r3 (raw file):

Previously, RaduBerinde wrote…

[nit] could use a comment mentioning this is related to table stats and CREATE STATS statement.

Done.


pkg/sql/create_stats.go, line 80 at r3 (raw file):

Previously, RaduBerinde wrote…

Do we need to store this as a function? Can't it just be a separate method?

Done.


pkg/sql/create_stats.go, line 221 at r3 (raw file):

Previously, RaduBerinde wrote…

I would add a description here and explain how the object is used. Especially that an instance of this is associated with a single job (because the methods in the interface suggest otherwise)

Done.


pkg/sql/create_stats.go, line 249 at r3 (raw file):

Previously, RaduBerinde wrote…

[nit] can just return so we don't have yet another err instance

Done.


pkg/sql/create_stats.go, line 276 at r3 (raw file):

Previously, RaduBerinde wrote…

We have two different errs here which makes this kind of confusing, I'd give the inner one a different name (eg jobErr)

Done.


pkg/sql/create_stats.go, line 293 at r3 (raw file):

Previously, RaduBerinde wrote…

nit: maybe use _ instead of txn here to make it clear we're not using that

Done.


pkg/sql/create_stats.go, line 323 at r3 (raw file):

Previously, RaduBerinde wrote…

[nit] maybe add a comment, or if not I'd just declare this directly in the AddResumeHook call.

Done.

@rytaft
Copy link
Collaborator Author

rytaft commented Jan 30, 2019

bors r+

craig bot pushed a commit that referenced this pull request Jan 30, 2019
34279: sql, stats: use the jobs framework for CREATE STATISTICS r=rytaft a=rytaft

Prior to this commit, `CREATE STATISTICS` was a regular SQL statement
and followed the standard logic of AST -> `planNode` -> DistSQL physical
plan -> execution. This commit changes `CREATE STATISTICS` to use the
jobs framework, and as a result the `createStatsNode` has been
changed to execute a function which starts a `CreateStats` job. The job
is then responsible for performing DistSQL planning and execution.

There are several advantages to using the jobs framework:
- Now `CREATE STATISTICS` jobs can easily be cancelled, paused and resumed,
  and viewed from the Admin UI.
- Nodes can adopt the job if the original gateway node fails.
- We will be able to use the JobID to lock creation of automatic
  statistics, so that only one automatic statistics job can run at
  a time. Job adoption will ensure that a dead node never prevents progress
  by holding a lock on stats creation (implementation of locking will be
  saved for the next PR).

Release note (sql change): CREATE STATISTICS now runs as a job instead
of as a regular SQL statement.

Co-authored-by: Rebecca Taft <[email protected]>
@craig
Copy link
Contributor

craig bot commented Jan 30, 2019

Build succeeded

@craig craig bot merged commit 9ef8bb2 into cockroachdb:master Jan 30, 2019
celiala added a commit to celiala/cockroach that referenced this pull request Mar 12, 2019
With cockroachdb#34279, enabling the cluster setting
`sql.stats.experimental_automatic_collection.enabled` has the potential
to create many CreateStats jobs, which can cause the Jobs view on the
AdminUI to become cluttered.

This commit creates a new `AutoCreateStats` job type for these auto-created
CreateStats jobs, so that users are able to still see their own manual runs
of CREATE STATISTICS, via the pre-existing `CreateStats` type.

![jobs-auto-create-stats](https://user-images.githubusercontent.com/3051672/54212467-5cea2c80-44b9-11e9-9c11-db749814f019.gif)

Release note (admin ui change): AutoCreateStats type added to
Jobs page to filter automatic statistics jobs.

Fixes cockroachdb#34377.
celiala added a commit to celiala/cockroach that referenced this pull request Mar 12, 2019
With cockroachdb#34279, enabling the cluster setting
`sql.stats.experimental_automatic_collection.enabled` has the potential
to create many CreateStats jobs, which can cause the Jobs view on the
AdminUI to become cluttered.

This commit creates a new `AutoCreateStats` job type for these auto-created
CreateStats jobs, so that users are able to still see their own manual runs
of CREATE STATISTICS, via the pre-existing `CreateStats` type.

![jobs-auto-create-stats](https://user-images.githubusercontent.com/3051672/54212467-5cea2c80-44b9-11e9-9c11-db749814f019.gif)

Release note (admin ui change): AutoCreateStats type added to
Jobs page to filter automatic statistics jobs.

Fixes cockroachdb#34377.
craig bot pushed a commit that referenced this pull request Mar 12, 2019
35321: opt: propagate set operation output types to input columns r=rytaft a=rytaft

This commit updates the `optbuilder` logic for set operations in which
the types of the input columns do not match the types of the output
columns. This can happen if a column on one side has type Unknown,
but the corresponding column on the other side has a known type such
as Int. The known type must be propagated to the side with the unknown
type to prevent errors in the execution engine related to decoding
types.

If there are any column types on either side that don't match the output,
then the `optbuilder` propagates the output types of the set operation down
to the input columns by wrapping the side with mismatched types in a
Project operation. The Project operation passes through columns that
already have the correct type, and creates cast expressions for those
that don't.

Fixes #34524

Release note (bug fix): Fixed an error that happened when executing
some set operations containing only nulls in one of the input columns.

35587: opt: add more cost for lookup joins with more ON conditions r=RaduBerinde a=RaduBerinde

This is a very limited fix for #34810.

The core problem is that we don't take into account that if we have an
ON condition, not only there's a cost to evaluate it on each row, but
we are generating more internal rows to get a given number of output
rows.

I attempted to do a more general fix (for all join types), where I
tried to estimate the "internal" number of rows using
`unknownFilterSelectivity` for each ON condition. There were two
problems:
 - in some cases (especially with lookup joins) we have an extra ON
   condition that doesn't actually do anything:
     `ab JOIN xy ON a=x AND a=10` becomes
     `ab JOIN xy ON a=x AND a=10 AND x=10` becomes
   and `a=10` could remain as an ON condition. This results in bad
   query plans in important cases (e.g. TPCC) where it prefers to do
   an extra lookup join (due to a non-covering index) just because of
   that condition.

 - we don't have the equality columns readily available for hash join
   (and didn't want to extract them each time we cost). In the future
   we may split the planning into a logical and physical stage, and we
   should then separate the logical joins from hash join.

For 19.1, we simply simply add a cost for lookup joins that is
proportional to the number of remaining ON conditions. This is the
least disruptive method that still fixes the case observed in #34810.
I will leave the issue open to address this properly in the next
release.

Note that although hash joins and merge joins have the same issue in
principle, in practice we always generate these expressions with
equality on all possible columns.

Release note: None

35630: storage/tscache: Pick up andy-kimball/arenaskl fix r=nvanbenschoten a=nvanbenschoten

Fixes #31624.
Fixes #35557.

This commit picks up andy-kimball/arenaskl#4.

I strongly suspect that the uint32 overflow fixed in that PR was the
cause of the two index out of bounds panics. See that commit for more
details.

The PR also fixes a bug in memory recylcling within the tscache. I confirmed
on adriatic that over 900 64MB arenas had been allocated since it was last
wiped.

35644: opt: use correct ordering for insert input in execbuilder r=RaduBerinde a=RaduBerinde

We were setting up a projection on the Insert's input but we were
accidentally using the parent Insert's ordering instead of that of the
input.

Fixes #35564.

Release note (bug fix): Fixed a "column not in input" crash when
`INSERT ... RETURNING` is used inside a clause that requires an
ordering.

35651: jobs, sql, ui: Create `AutoCreateStats` job type r=celiala a=celiala

With #34279, enabling the cluster setting
`sql.stats.experimental_automatic_collection.enabled` has the potential
to create many CreateStats jobs, which can cause the Jobs view on the
AdminUI to become cluttered.

This commit creates a new `AutoCreateStats` job type for these auto-created
CreateStats jobs, so that users are able to still see their own manual runs
of CREATE STATISTICS, via the pre-existing `CreateStats` type.

cc @danhhz, @piyush-singh, @rolandcrosby 

![jobs-auto-create-stats](https://user-images.githubusercontent.com/3051672/54212467-5cea2c80-44b9-11e9-9c11-db749814f019.gif)

Release note (admin ui change): AutoCreateStats type added to
Jobs page to filter automatic statistics jobs.

Fixes #34377.

Co-authored-by: Rebecca Taft <[email protected]>
Co-authored-by: Radu Berinde <[email protected]>
Co-authored-by: Nathan VanBenschoten <[email protected]>
Co-authored-by: Celia La <[email protected]>
@rytaft rytaft deleted the auto-stats-job branch April 2, 2020 22:16
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

4 participants