Skip to content

sql: merge partial and full table statistics#91933

Merged
craig[bot] merged 1 commit intocockroachdb:masterfrom
faizaanmadhani:faizaan/merge-histograms
Dec 21, 2022
Merged

sql: merge partial and full table statistics#91933
craig[bot] merged 1 commit intocockroachdb:masterfrom
faizaanmadhani:faizaan/merge-histograms

Conversation

@faizaanmadhani
Copy link
Copy Markdown
Contributor

@faizaanmadhani faizaanmadhani commented Nov 15, 2022

This commit adds support to the table stats
cache to merge partial statistics with full table
statistics when updating the stats cache, in an
effort to keep more accurate table statistics that
can be used by the optimizer.

Additionally, if forecasting is enabled for the table, these
merged statistics are used for the forecast.

Release note (sql change): The optimizer will
now use table statistics that are merged combinations
of the newest partial statistic and latest full
statistic collection.

Epic: CRDB-19449

@cockroach-teamcity
Copy link
Copy Markdown
Member

This change is Reviewable

@faizaanmadhani faizaanmadhani changed the title Faizaan/merge histograms sql: Write partial statistics into the cache Nov 15, 2022
@faizaanmadhani faizaanmadhani force-pushed the faizaan/merge-histograms branch 8 times, most recently from a1c2d65 to 0699064 Compare November 28, 2022 21:41
@faizaanmadhani faizaanmadhani force-pushed the faizaan/merge-histograms branch 7 times, most recently from c4feccc to 4f68772 Compare December 1, 2022 03:56
@faizaanmadhani faizaanmadhani changed the title sql: Write partial statistics into the cache sql: merge partial and full table statistics Dec 1, 2022
@faizaanmadhani faizaanmadhani force-pushed the faizaan/merge-histograms branch 4 times, most recently from 8bb0939 to a4c0d7e Compare December 1, 2022 05:39
@faizaanmadhani faizaanmadhani marked this pull request as ready for review December 1, 2022 05:42
@faizaanmadhani faizaanmadhani requested review from a team as code owners December 1, 2022 05:42
@faizaanmadhani faizaanmadhani force-pushed the faizaan/merge-histograms branch 2 times, most recently from 93f1eb4 to 31edf0a Compare December 1, 2022 20:54
Copy link
Copy Markdown
Collaborator

@michae2 michae2 left a comment

Choose a reason for hiding this comment

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

Very nice! I'm still reading, just a couple nits so far.

Reviewed 3 of 9 files at r1.
Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @faizaanmadhani, @mgartner, and @rytaft)


-- commits line 16 at r1:
nit: I think it would be helpful to also mention how this merging relates to forecasting. Could be as simple as "before forecasting" somewhere in this sentence.


-- commits line 18 at r1:
I believe everything after "Release note" will get picked up by the release note script, so this "Epic" reference needs to be above the release note.


pkg/sql/stats/merge.go line 170 at r1 (raw file):

		}
	}
	forecastCols, fullStatsMap, colKeyToColIDs := groupObservedStatistics(fullStatsList)

nit: Does this need the full "grouping" behavior now that it's not forecasting and just using the latest full stats? Maybe fullStatsMap should just be map[string]*TableStatistic now instead of map[string][]*TableStatistic.


pkg/sql/stats/new_stat.go line 106 at r1 (raw file):

					"nullCount",
					"avgSize",
					histogram

nit: Is this the same fix sneaking in again? 🙂


pkg/sql/stats/stats_cache.go line 792 at r1 (raw file):

	}

	var forecasts []*TableStatistic

nit: I think this slice can go back to being local to the if forecast branch.


pkg/sql/stats/stats_cache.go line 793 at r1 (raw file):

	var forecasts []*TableStatistic
	if len(partialStatsList) > 0 {

todo: In a future PR we should add a variable to disable merging, just in case we need to turn it off for some reason.

@faizaanmadhani faizaanmadhani requested a review from a team December 16, 2022 15:59
Copy link
Copy Markdown
Contributor Author

@faizaanmadhani faizaanmadhani left a comment

Choose a reason for hiding this comment

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

Hey @michae2 @mgartner! I've addressed the comments, and most notably added a check for StatisticID which validates that the partial stat we are merging was derived from the full statistic we are merging with. To do this, I updated the older migration have that in a different PR which I rebased this branch off of, so its changes are here and #93751 will need to be merged before we merge this, but that's why there are a lot of additional changes to this PR that were also in #93751

I also made some changes to my unit tests. There was an issue with the way I recalculating the avgSize for the merged statistic that has now been fixed.

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (and 1 stale) (waiting on @mgartner, @michae2, and @rytaft)


pkg/sql/stats/merge.go line 70 at r4 (raw file):

Previously, michae2 (Michael Erickson) wrote…

If we don't think we'll ever have buckets in partial and full with the same upper bounds, can we keep the = case but use it to return an error instead of handling it?

Done.


pkg/sql/stats/merge.go line 131 at r12 (raw file):

Previously, michae2 (Michael Erickson) wrote…

Good points. I was hoping that checking the predicate would be simple, but as you pointed out in slack this @ syntax turned out to have weird semantics. I like the idea of referencing a full stat ID, that seems very robust.

So I added a fullStatisticsID column in this PR: #93751 and rebased off of it for this PR so the changes are shared between the two PRs and #93751 will need to be merged first.

I've also added this check here and updated the tests accordingly.


pkg/sql/stats/merge.go line 155 at r12 (raw file):

Previously, michae2 (Michael Erickson) wrote…

I'm a little nervous that this won't work for TIME, TIMETZ, TIMESTAMP, or TIMESTAMPTZ types which actually use this compare context in CompareError. But let me do some research, it might be ok. I remember it was ok for forecasting (which does work on TIMESTAMP and TIMESTAMPTZ).

Sounds good. I've left this here for now if it's ok, but let me know if it should be changed. If the context needs to be set to something, what would we need to do?


pkg/sql/stats/merge.go line 166 at r12 (raw file):

Previously, michae2 (Michael Erickson) wrote…

I'm confused about numOriginalRows. We're adding row counts from both the partial histogram and the full histogram, but then down below we calculate mergedRowCount from the merged histogram, which I think will be the same thing?

This was a leftover from the older merge code and was unnecessary. In fact, changing it helped me catch a few issues in my tests, which I've gone back and fixed because the average calculation wasn't happening correctly.

Copy link
Copy Markdown
Contributor

@mgartner mgartner left a comment

Choose a reason for hiding this comment

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

Reviewed 1 of 2 files at r12, 4 of 27 files at r14, 26 of 28 files at r15, 2 of 2 files at r16, all commit messages.
Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (and 1 stale) (waiting on @faizaanmadhani and @michae2)


pkg/sql/distsql_plan_stats.go line 170 at r16 (raw file):

		t.Name != jobspb.ForecastStatsName &&
		t.Name != jobspb.MergedStatsName &&
		(!colinfo.ColumnTypeIsInvertedIndexable(column.GetType()) || t.HistogramData == nil || t.HistogramData.ColumnType == nil || t.HistogramData.ColumnType.Family() != types.BytesFamily)

I don't understand the ... || t.HistogramData == nil || t.Histogram.ColumnType == nil || .... Doesn't the full stat need to have a histogram in order to create a partial stat from it?


pkg/sql/stats/histogram.go line 199 at r9 (raw file):

Previously, faizaanmadhani (Faizaan Madhani) wrote…

I moved the function above, but my only argument against naming it EquiDepthExtremesHistogram is that the returned histogram is not guaranteed to be equi-depth.

Oh right. I agree, that would be a confusing name.


pkg/sql/stats/merge.go line 216 at r6 (raw file):

Previously, faizaanmadhani (Faizaan Madhani) wrote…

Just noting here that we don't have support for SHOW HISTOGRAM for forecasts yet either. This is because SHOW HISTOGRAM uses the on-disk representation of Histogram Data that it fetches from the system table. Since we don't store merged or forecasted histograms in the system table, we can't support it.

👍


pkg/sql/stats/merge.go line 166 at r16 (raw file):

	j := 0
	// Merge partial stats to prior full statistics.
	for i < len(partialHistogram) && j < len(fullHistogram) {

I don't understand why we need to iterate over the full histogram here. Can we simplify this to just appending from the partial until we reach the lower bucket of the full histogram, and then keep the logic below this loop as-is (adding the full histogram buckets, then the rest of the partial histogram buckets)?


pkg/sql/stats/merge.go line 196 at r16 (raw file):

	var mergedRowCount uint64
	var mergedDistinctCount uint64
	mergedNullCount := partialNullCount

We don't include the full stat's null count because the partial stats collection always scans over all nulls, is that correct? A comment here explaining that would be helpful.


pkg/sql/stats/stats_cache.go line 804 at r16 (raw file):

	}

	// TODO (faizaanmadhani): Wrap merging behind a boolean so

nit: the common format is no space after "TODO"


pkg/sql/opt/exec/execbuilder/testdata/partial_stats line 291 at r16 (raw file):


query T
EXPLAIN (OPT, VERBOSE) SELECT * FROM g WHERE b >= 10

nit: I think one variant of these explains is enough to prove that the merged stats are working as intended (probably the OPT, VERBOSE one since it shows the histogram), unless there's something specific you're trying to show with each that I'm missing.


pkg/sql/opt/exec/execbuilder/testdata/partial_stats line 352 at r16 (raw file):

        {
            "distinct_range": 0,
            "num_eq": 0,

It's weird that forecasting turned this num_eq from 1 to 0... Any ideas why?


pkg/sql/opt/exec/execbuilder/testdata/partial_stats line 415 at r16 (raw file):

    "name": "__forecast__",
    "null_count": 0,
    "row_count": 14

I only count 13 rows based on the histograms. Shouldn't this number match, or is that not guaranteed for forecasted stats? The distinct count above is also 14. cc @michae2


pkg/sql/opt/exec/execbuilder/testdata/partial_stats line 745 at r16 (raw file):


query T
EXPLAIN (OPT, VERBOSE) SELECT * FROM i WHERE i = 6 OR i <= -1

ditto: I think one explain is enough.


pkg/sql/opt/exec/execbuilder/testdata/partial_stats line 777 at r16 (raw file):

       ],
       "created_at":"2022-12-12 15:22:21.988179",
       "distinct_count":25,

nit: This should be 26 to include NULL rows.


pkg/sql/opt/exec/execbuilder/testdata/partial_stats line 814 at r16 (raw file):

       "name":"full_b",
       "null_count":3,
       "row_count":25

nit: I think this should be 28 to include the NULL rows.


pkg/sql/opt/exec/execbuilder/testdata/partial_stats line 822 at r16 (raw file):

       ],
       "created_at":"2022-12-13 15:22:21.988179",
       "distinct_count":31,

nit: 32 to include NULL


pkg/sql/opt/exec/execbuilder/testdata/partial_stats line 871 at r16 (raw file):

       "name":"partial_b",
       "null_count":5,
       "row_count":31,

nit: 36 to include NULLs


pkg/sql/opt/exec/execbuilder/testdata/partial_stats line 880 at r16 (raw file):

       ],
       "created_at":"2022-12-12 15:22:21.988179",
       "distinct_count":25,

nit: 26 to include NULL


pkg/sql/opt/exec/execbuilder/testdata/partial_stats line 917 at r16 (raw file):

       "name":"full_a",
       "null_count":3,
       "row_count":25

nit: 28 to include NULLs


pkg/sql/opt/exec/execbuilder/testdata/partial_stats line 925 at r16 (raw file):

       ],
       "created_at":"2022-12-14 15:22:21.988179",
       "distinct_count":31,

nit: I count 28 inlcuding NULL


pkg/sql/opt/exec/execbuilder/testdata/partial_stats line 968 at r16 (raw file):

       "name":"partial_a",
       "null_count":5,
       "row_count":31,

nit: 36 to include NULLs


pkg/sql/opt/exec/execbuilder/testdata/partial_stats line 1011 at r16 (raw file):

    ],
    "created_at": "2022-12-13 15:22:21.988179",
    "distinct_count": 56,

nit: this should be 57 to include NULL


pkg/sql/opt/exec/execbuilder/testdata/partial_stats line 1098 at r16 (raw file):

    ],
    "created_at": "2022-12-14 15:22:21.988179",
    "distinct_count": 52,

nit: I count 53 including NULL


pkg/sql/stats/bounds/extremes.go line 52 at r16 (raw file):

		},
	}
	return tree.AsStringWithFlags(&pred, tree.FmtSerializable|tree.FmtSymbolicSubqueries)

Why do you need tree.FmtSymbolicSubqueries here?


pkg/sql/stats/bounds/extremes.go line 72 at r16 (raw file):

	if index.GetKeyColumnDirection(0) == catpb.IndexColumn_ASC {
		extremesSpans.InitSingleSpan(&lbSpan)
		extremesSpans.Append(&ubSpan)

AFAIK There is no requirement that constraint.Spans are in any particular order. Did you run into problems adding DESC spans in the same order as ASC spans?


pkg/sql/stats/bounds/extremes.go line 82 at r16 (raw file):

// GetUsingExtremesBounds returns a tree.Datum representing the upper and lower
// bounds of the USING EXTREMES span for partial statistics.

nit: They are both exclusive bounds, correct? It'd be good to make that clear here.


pkg/sql/stats/bounds/extremes.go line 85 at r16 (raw file):

func GetUsingExtremesBounds(
	evalCtx *eval.Context, histogram []cat.HistogramBucket,
) (tree.Datum, tree.Datum, error) {

nit: label the return values like lowerBound, upperBound tree.Datum, _ error)


pkg/sql/stats/bounds/extremes.go line 97 at r16 (raw file):

		}
	}
	if lowerBound.Compare(evalCtx, tree.DNull) == 0 {

nit: you could simplify this to if lowerBound == nil { .. } if you initialize lowerBound as var lowerBound tree.Datum, and I think you can keep the loop as-is.

@mgartner
Copy link
Copy Markdown
Contributor

pkg/sql/distsql_plan_stats.go line 170 at r16 (raw file):

Previously, mgartner (Marcus Gartner) wrote…

I don't understand the ... || t.HistogramData == nil || t.Histogram.ColumnType == nil || .... Doesn't the full stat need to have a histogram in order to create a partial stat from it?

I see - we are guarding against NPEs. It's a bit confusing that this function returns true if the histogram is nil -we clearly need that to do partial stats collection. Is this more clear? cc @michae2

    return len(t.ColumnIDs) == 1 &&
        column.GetID() == t.ColumnIDs[0] &&
        t.PartialPredicate == "" &&
        t.Name != jobspb.ForecastStatsName &&
        t.Name != jobspb.MergedStatsName &&
        (!colinfo.ColumnTypeIsInvertedIndexable(column.GetType()) ||
            (t.HistogramData != nil && t.HistogramData.ColumnType != nil && t.HistogramData.ColumnType.Family() != types.BytesFamily))

Copy link
Copy Markdown
Contributor Author

@faizaanmadhani faizaanmadhani left a comment

Choose a reason for hiding this comment

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

I addressed all the comments and implemented all the changes!

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (and 1 stale) (waiting on @mgartner and @michae2)


pkg/sql/distsql_plan_stats.go line 170 at r16 (raw file):

Previously, mgartner (Marcus Gartner) wrote…

I see - we are guarding against NPEs. It's a bit confusing that this function returns true if the histogram is nil -we clearly need that to do partial stats collection. Is this more clear? cc @michae2

    return len(t.ColumnIDs) == 1 &&
        column.GetID() == t.ColumnIDs[0] &&
        t.PartialPredicate == "" &&
        t.Name != jobspb.ForecastStatsName &&
        t.Name != jobspb.MergedStatsName &&
        (!colinfo.ColumnTypeIsInvertedIndexable(column.GetType()) ||
            (t.HistogramData != nil && t.HistogramData.ColumnType != nil && t.HistogramData.ColumnType.Family() != types.BytesFamily))

Done.


pkg/sql/stats/merge.go line 166 at r16 (raw file):

Previously, mgartner (Marcus Gartner) wrote…

I don't understand why we need to iterate over the full histogram here. Can we simplify this to just appending from the partial until we reach the lower bucket of the full histogram, and then keep the logic below this loop as-is (adding the full histogram buckets, then the rest of the partial histogram buckets)?

Done.


pkg/sql/stats/merge.go line 196 at r16 (raw file):

Previously, mgartner (Marcus Gartner) wrote…

We don't include the full stat's null count because the partial stats collection always scans over all nulls, is that correct? A comment here explaining that would be helpful.

Done.


pkg/sql/opt/exec/execbuilder/testdata/partial_stats line 352 at r16 (raw file):

Previously, mgartner (Marcus Gartner) wrote…

It's weird that forecasting turned this num_eq from 1 to 0... Any ideas why?

I'm not sure... 🤔


pkg/sql/opt/exec/execbuilder/testdata/partial_stats line 415 at r16 (raw file):

Previously, mgartner (Marcus Gartner) wrote…

I only count 13 rows based on the histograms. Shouldn't this number match, or is that not guaranteed for forecasted stats? The distinct count above is also 14. cc @michae2

I would assume it wouldn't be guaranteed because it's forecasted, and the forecast might believe the table is growing in size, but I'm not too sure.


pkg/sql/opt/exec/execbuilder/testdata/partial_stats line 745 at r16 (raw file):

Previously, mgartner (Marcus Gartner) wrote…

ditto: I think one explain is enough.

Done.


pkg/sql/stats/merge_test.go line 32 at r15 (raw file):

	}{
		{
			// Single partial at the extremes of full.

Adding a comment to this file to clarify these changes. Your (@marcus's) corrections above made me realize that I wasn't incrementing the discount count by 1 if the column has NULL values, so I fixed that and reworked these tests to account for that.


pkg/sql/stats/bounds/extremes.go line 52 at r16 (raw file):

Previously, mgartner (Marcus Gartner) wrote…

Why do you need tree.FmtSymbolicSubqueries here?

That was a leftover from when we tried doing ordinals. I've changed it back to normal serializing.


pkg/sql/stats/bounds/extremes.go line 72 at r16 (raw file):

Previously, mgartner (Marcus Gartner) wrote…

AFAIK There is no requirement that constraint.Spans are in any particular order. Did you run into problems adding DESC spans in the same order as ASC spans?

Yes, I did. Added a comment clarify why we need to generate the constraints differently.

Copy link
Copy Markdown
Contributor

@mgartner mgartner left a comment

Choose a reason for hiding this comment

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

:lgtm:

Reviewed 6 of 6 files at r17, all commit messages.
Reviewable status: :shipit: complete! 1 of 0 LGTMs obtained (and 1 stale) (waiting on @faizaanmadhani and @michae2)


pkg/sql/stats/merge.go line 166 at r17 (raw file):

	// Merge partial stats to prior full statistics.
	for i < len(partialHistogram) {
		if val, err := partialHistogram[i].UpperBound.CompareError(cmpCtx, fullHistogram[0].UpperBound); err == nil {

nit: val, err := .... then if err != nil { .. } to handle the err case first.


pkg/sql/stats/merge.go line 181 at r17 (raw file):

	// Iterate through the rest of the full histogram and append it.
	for j := 0; j < len(fullHistogram); j++ {

nit: for j := range fullHistogram { .. }


pkg/sql/stats/merge.go line 195 at r17 (raw file):

	// Since partial statistics at the extremes will always scan over
	// the NULL rows at the lowerbound, we don't include the NULL count
	// of the full statistic.

No need to do anything now, but I'm worried about partial stats scanning all rows where a column is NULL. It's not hard to imagine cases where 50+% of rows have NULL for a column (e.g. a deleted_at TIMESTAMPTZ column on a table where most things are not deleted). That would be a really expensive partial stats collection, relative to most other cases. I think we should consider scanning (\NULL - \lowerbound) instead of [\NULL - \lowerbound) if null_count is already non-zero.

Copy link
Copy Markdown
Contributor Author

@faizaanmadhani faizaanmadhani 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 (and 2 stale) (waiting on @mgartner and @michae2)


pkg/sql/stats/merge.go line 195 at r17 (raw file):

Previously, mgartner (Marcus Gartner) wrote…

No need to do anything now, but I'm worried about partial stats scanning all rows where a column is NULL. It's not hard to imagine cases where 50+% of rows have NULL for a column (e.g. a deleted_at TIMESTAMPTZ column on a table where most things are not deleted). That would be a really expensive partial stats collection, relative to most other cases. I think we should consider scanning (\NULL - \lowerbound) instead of [\NULL - \lowerbound) if null_count is already non-zero.

Sounds good. I will open an issue for this and link it here when I merge this PR.

Copy link
Copy Markdown
Collaborator

@michae2 michae2 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 going to read through testcases after dinner, but this :lgtm_strong:

Nice work, @faizaanmadhani!! You put a ton of effort into this!!

Reviewed 2 of 27 files at r14, 18 of 28 files at r15, 1 of 2 files at r16, 3 of 6 files at r17, 1 of 2 files at r18.
Reviewable status: :shipit: complete! 1 of 0 LGTMs obtained (and 2 stale) (waiting on @faizaanmadhani, @mgartner, and @rytaft)


pkg/sql/distsql_plan_stats.go line 170 at r16 (raw file):

Previously, faizaanmadhani (Faizaan Madhani) wrote…

Done.

Comment for @mgartner: This logic isn't quite the same. Now in some cases where before we would have returned a "prior histogram has no buckets" error from createPartialStatsPlan we are instead creating partial stats based on a stale full statistic. Here's one such case:

SET CLUSTER SETTING sql.stats.automatic_collection.fraction_stale_rows = 0.001;
SET CLUSTER SETTING sql.stats.automatic_collection.min_stale_rows = 5;

CREATE TABLE r (s STRING PRIMARY KEY);

INSERT INTO r VALUES ('a'), ('b'), ('c'), ('d'), ('e');

-- wait a while for full auto stats

SET CLUSTER SETTING sql.stats.histogram_collection.enabled = false;

DELETE FROM r WHERE true;
INSERT INTO r VALUES ('v'), ('w'), ('x'), ('y'), ('z');

-- wait a while for full auto stats

SELECT jsonb_pretty(statistics) FROM [SHOW STATISTICS USING JSON FOR TABLE r];

This produces statistics like the following:

  [
      {
          "avg_size": 0,
          "columns": [
              "s"
          ],
          "created_at": "2022-12-19 23:28:55.124743",
          "distinct_count": 0,
          "histo_col_type": "STRING",
          "histo_version": 2,
          "name": "__auto__",
          "null_count": 0,
          "row_count": 0
      },
      {
          "avg_size": 4,
          "columns": [
              "s"
          ],
          "created_at": "2022-12-19 23:29:55.156286",
          "distinct_count": 5,
          "histo_buckets": [
              {
                  "distinct_range": 0,
                  "num_eq": 1,
                  "num_range": 0,
                  "upper_bound": "a"
              },
              {
                  "distinct_range": 0,
                  "num_eq": 1,
                  "num_range": 0,
                  "upper_bound": "b"
              },
              {
                  "distinct_range": 0,
                  "num_eq": 1,
                  "num_range": 0,
                  "upper_bound": "c"
              },
              {
                  "distinct_range": 0,
                  "num_eq": 1,
                  "num_range": 0,
                  "upper_bound": "d"
              },
              {
                  "distinct_range": 0,
                  "num_eq": 1,
                  "num_range": 0,
                  "upper_bound": "e"
              }
          ],
          "histo_col_type": "STRING",
          "histo_version": 2,
          "name": "__auto__",
          "null_count": 0,
          "row_count": 5
      },
      {
          "avg_size": 4,
          "columns": [
              "s"
          ],
          "created_at": "2022-12-19 23:32:55.222092",
          "distinct_count": 5,
          "histo_col_type": "",
          "name": "__auto__",
          "null_count": 0,
          "row_count": 5
      }
  ]

Before, isFullStatValidToCreatePartial would have returned true for the latest 23:32 stat (which has t.HistogramData == nil) and then createPartialStatsPlan would have returned an error because the histogram is empty. Now, createPartialStatsPlan is building a partial stat based on the stale 23:29 stat. I'm not sure this is better.

In other words, stats with nil HistogramData (and all other conditions true) are "valid" meaning we should not continue searching past them, but then we should return an error because histograms were turned off and have length 0. Does that make sense? WDYT?


pkg/sql/stats/merge.go line 131 at r12 (raw file):

Previously, faizaanmadhani (Faizaan Madhani) wrote…

So I added a fullStatisticsID column in this PR: #93751 and rebased off of it for this PR so the changes are shared between the two PRs and #9371 will need to be merged first.

I've also added this check here and updated the tests accordingly.

Thank you for doing this!!


pkg/sql/stats/merge.go line 155 at r12 (raw file):

Previously, faizaanmadhani (Faizaan Madhani) wrote…

Sounds good. I've left this here for now if it's ok, but let me know if it should be changed. If the context needs to be set to something, what would we need to do?

It looks like all time comparisons will be done using UTC, which should be fine for merging, so this is good.


pkg/sql/stats/merge.go line 81 at r17 (raw file):

//
// For example, consider this case:
// Full Statistic: {row: 3, dist: 4, null: 4, size: 1}

micro-nit: I think this example should have row: 7 (including nulls) and the merge should have row: 15


pkg/sql/stats/merge.go line 182 at r17 (raw file):

	// Iterate through the rest of the full histogram and append it.
	for j := 0; j < len(fullHistogram); j++ {
		mergedHistogram = append(mergedHistogram, fullHistogram[j])

It would be nice to have a defensive break in this loop if i < len(partialHistogram) && partialHistogram[i].UpperBound.CompareError(cmpCtx, fullHistogram[j].UpperBound) is ever <= 0 before we reach the end of the full histogram. (Or maybe instead of a break, return an error?)

Copy link
Copy Markdown
Contributor Author

@faizaanmadhani faizaanmadhani 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! 1 of 0 LGTMs obtained (and 2 stale) (waiting on @mgartner, @michae2, and @rytaft)


pkg/sql/distsql_plan_stats.go line 170 at r16 (raw file):

Previously, michae2 (Michael Erickson) wrote…

Comment for @mgartner: This logic isn't quite the same. Now in some cases where before we would have returned a "prior histogram has no buckets" error from createPartialStatsPlan we are instead creating partial stats based on a stale full statistic. Here's one such case:

SET CLUSTER SETTING sql.stats.automatic_collection.fraction_stale_rows = 0.001;
SET CLUSTER SETTING sql.stats.automatic_collection.min_stale_rows = 5;

CREATE TABLE r (s STRING PRIMARY KEY);

INSERT INTO r VALUES ('a'), ('b'), ('c'), ('d'), ('e');

-- wait a while for full auto stats

SET CLUSTER SETTING sql.stats.histogram_collection.enabled = false;

DELETE FROM r WHERE true;
INSERT INTO r VALUES ('v'), ('w'), ('x'), ('y'), ('z');

-- wait a while for full auto stats

SELECT jsonb_pretty(statistics) FROM [SHOW STATISTICS USING JSON FOR TABLE r];

This produces statistics like the following:

  [
      {
          "avg_size": 0,
          "columns": [
              "s"
          ],
          "created_at": "2022-12-19 23:28:55.124743",
          "distinct_count": 0,
          "histo_col_type": "STRING",
          "histo_version": 2,
          "name": "__auto__",
          "null_count": 0,
          "row_count": 0
      },
      {
          "avg_size": 4,
          "columns": [
              "s"
          ],
          "created_at": "2022-12-19 23:29:55.156286",
          "distinct_count": 5,
          "histo_buckets": [
              {
                  "distinct_range": 0,
                  "num_eq": 1,
                  "num_range": 0,
                  "upper_bound": "a"
              },
              {
                  "distinct_range": 0,
                  "num_eq": 1,
                  "num_range": 0,
                  "upper_bound": "b"
              },
              {
                  "distinct_range": 0,
                  "num_eq": 1,
                  "num_range": 0,
                  "upper_bound": "c"
              },
              {
                  "distinct_range": 0,
                  "num_eq": 1,
                  "num_range": 0,
                  "upper_bound": "d"
              },
              {
                  "distinct_range": 0,
                  "num_eq": 1,
                  "num_range": 0,
                  "upper_bound": "e"
              }
          ],
          "histo_col_type": "STRING",
          "histo_version": 2,
          "name": "__auto__",
          "null_count": 0,
          "row_count": 5
      },
      {
          "avg_size": 4,
          "columns": [
              "s"
          ],
          "created_at": "2022-12-19 23:32:55.222092",
          "distinct_count": 5,
          "histo_col_type": "",
          "name": "__auto__",
          "null_count": 0,
          "row_count": 5
      }
  ]

Before, isFullStatValidToCreatePartial would have returned true for the latest 23:32 stat (which has t.HistogramData == nil) and then createPartialStatsPlan would have returned an error because the histogram is empty. Now, createPartialStatsPlan is building a partial stat based on the stale 23:29 stat. I'm not sure this is better.

In other words, stats with nil HistogramData (and all other conditions true) are "valid" meaning we should not continue searching past them, but then we should return an error because histograms were turned off and have length 0. Does that make sense? WDYT?

Considering this example, I'm leaning on changing this logic back to what it was previously, but I'm going to hold off on changing this and merging after getting @mgartner's input.

Copy link
Copy Markdown
Collaborator

@michae2 michae2 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! 1 of 0 LGTMs obtained (and 2 stale) (waiting on @faizaanmadhani, @mgartner, and @rytaft)


pkg/sql/stats/merge.go line 219 at r20 (raw file):

	}

	mergedAvgSize := (partialStat.AvgSize*partialStat.RowCount + fullStat.AvgSize*fullStat.RowCount) / mergedRowCount

Could mergedRowCount be zero?


pkg/sql/opt/exec/execbuilder/testdata/partial_stats line 352 at r16 (raw file):

Previously, faizaanmadhani (Faizaan Madhani) wrote…

I'm not sure... 🤔

Probably because of the missing 9 in the merged stat + the forecast distance being half a day instead of a full day (or two).


pkg/sql/opt/exec/execbuilder/testdata/partial_stats line 415 at r16 (raw file):

Previously, faizaanmadhani (Faizaan Madhani) wrote…

I would assume it wouldn't be guaranteed because it's forecasted, and the forecast might believe the table is growing in size, but I'm not too sure.

It's probably because the partial stat has the wrong number for row_count and distinct_count (6 instead of 3).


pkg/sql/opt/exec/execbuilder/testdata/partial_stats line 13 at r20 (raw file):


statement ok
CREATE TABLE g (b INT PRIMARY KEY) WITH (sql_stats_automatic_collection_enabled = false);

micro-nit: Statements in our logic tests don't need the terminating semicolon.


pkg/sql/opt/exec/execbuilder/testdata/partial_stats line 126 at r20 (raw file):

      ],
      "created_at": "1988-08-08 00:00:00.000000",
      "distinct_count": 6,

This should be 3.


pkg/sql/opt/exec/execbuilder/testdata/partial_stats line 132 at r20 (raw file):

          "num_eq": 1,
          "num_range": 0,
          "upper_bound": "10"

We'll get less confusing forecasts if this partial stat is {9, 10, 11} rather than {10, 11, 12}.


pkg/sql/opt/exec/execbuilder/testdata/partial_stats line 152 at r20 (raw file):

      "null_count": 0,
      "partial_predicate": "(b < 0:::INT8) OR ((b > 8:::INT8) OR (b IS NULL))",
      "row_count": 6

3 here too


pkg/sql/opt/exec/execbuilder/testdata/partial_stats line 163 at r20 (raw file):

# statistic id is generated when a statistic is added to
# system.table_statistics, and since we don't store it in the JSON statistic
# object, we need to set it manually here.

Oh, hmm. I think we need to file an issue about this. I did not consider that ALTER TABLE INJECT STATISTICS would not work for this new fullStatisticID column. This could be a problem. 🤔 (No need to change this PR.)


pkg/sql/opt/exec/execbuilder/testdata/partial_stats line 304 at r20 (raw file):

partial       {b}  1988-08-08 00:00:00 +0000 +0000  6   6   0  2
__merged__    {b}  1988-08-08 00:00:00 +0000 +0000  12  12  0  1
__forecast__  {b}  1988-08-08 12:00:00 +0000 +0000  14  14  0  1

Beautiful to see it all working together! 🥲

We're forecasting 12 hours into the future because none of these are named __auto__. That's going to give a confusing result because we're growing 3 rows per day, so 12 hours = 1.5 rows. The forecast will still work correctly, but won't be as easy to obviously prove correct at a glance.

If we rename full_0 and full_1 to __auto__ then we should get a forecast 48 hours into the future, which should give a less confusing forecast.


pkg/sql/opt/exec/execbuilder/testdata/partial_stats line 606 at r20 (raw file):

        "num_eq": 1,
        "num_range": 0,
        "upper_bound": "4"

Might be less confusing with {3, 4, 5} instead of {4, 5, 6}?


pkg/sql/opt/exec/execbuilder/testdata/partial_stats line 761 at r20 (raw file):

             "distinct_range":4,
             "num_eq":1,
             "num_range":4,

First bucket always needs to have num_range and distinct_range of zero.


pkg/sql/opt/exec/execbuilder/testdata/partial_stats line 824 at r20 (raw file):

             "distinct_range":4,
             "num_eq":1,
             "num_range":4,

Won't ConstructExtremesHistogram always make upperHist with a first bucket with num_range and distinct_range of zero?


pkg/sql/opt/exec/execbuilder/testdata/partial_stats line 864 at r20 (raw file):

             "distinct_range":4,
             "num_eq":1,
             "num_range":4,

same comment here about num_range and distinct_range


pkg/sql/opt/exec/execbuilder/testdata/partial_stats line 921 at r20 (raw file):

             "distinct_range":3,
             "num_eq":1,
             "num_range":4,

same comment here about upperHist num_range and distinct_range

Copy link
Copy Markdown
Contributor

@mgartner mgartner 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! 1 of 0 LGTMs obtained (and 2 stale) (waiting on @faizaanmadhani, @michae2, and @rytaft)


pkg/sql/distsql_plan_stats.go line 170 at r16 (raw file):

Previously, faizaanmadhani (Faizaan Madhani) wrote…

Considering this example, I'm leaning on changing this logic back to what it was previously, but I'm going to hold off on changing this and merging after getting @mgartner's input.

Thanks for the example, Michael. I would expect partial stats to not occur at all if sql.stats.histogram_collection.enabled = false (@faizaanmadhani let's make an issue to track that). But I believe a slight modification to your example of setting SET CLUSTER SETTING sql.stats.histogram_collection.enabled = true after the non-histogram full collection would lead to the same problem.

I suggest we make this logic more closely match how you've described things should work. It sounds like the logic is:

  1. Is this the most recent full stat for this column? If not, continue.
  2. Does the full stat have a histogram? If not, return an error.
  3. Is the histogram's type invertable or a byes type? If so, return an error.

Making these steps distinct may make the code more clear. Or, if you feel strongly that it should remain as-is, then I'd suggest at least renaming isFullStatValidToCreatePartial because it's suprising that a full stat without a histogram can be "valid" to create a partial.

I'm also curious when we'd return an error from (3) without being able to error before the loop by checking the column's type. Is that specifically for trigram indexes?


pkg/sql/opt/exec/execbuilder/testdata/partial_stats line 163 at r20 (raw file):

Previously, michae2 (Michael Erickson) wrote…

Oh, hmm. I think we need to file an issue about this. I did not consider that ALTER TABLE INJECT STATISTICS would not work for this new fullStatisticID column. This could be a problem. 🤔 (No need to change this PR.)

Why do you think this could be a problem? I think this is fine - these are just test fixtures that we need to setup to mimic a real collection of them.

Copy link
Copy Markdown
Contributor Author

@faizaanmadhani faizaanmadhani 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! 1 of 0 LGTMs obtained (and 2 stale) (waiting on @mgartner and @michae2)


pkg/sql/stats/merge.go line 219 at r20 (raw file):

Previously, michae2 (Michael Erickson) wrote…

Could mergedRowCount be zero?

No, it shouldn't be. The only time that could happen would be if there is no partial and no full statistic histograms, in which case we already have an error above.


pkg/sql/opt/exec/execbuilder/testdata/partial_stats line 352 at r16 (raw file):

Previously, michae2 (Michael Erickson) wrote…

Probably because of the missing 9 in the merged stat + the forecast distance being half a day instead of a full day (or two).

Done.


pkg/sql/opt/exec/execbuilder/testdata/partial_stats line 415 at r16 (raw file):

Previously, michae2 (Michael Erickson) wrote…

It's probably because the partial stat has the wrong number for row_count and distinct_count (6 instead of 3).

Done.


pkg/sql/opt/exec/execbuilder/testdata/partial_stats line 304 at r20 (raw file):

Previously, michae2 (Michael Erickson) wrote…

Beautiful to see it all working together! 🥲

We're forecasting 12 hours into the future because none of these are named __auto__. That's going to give a confusing result because we're growing 3 rows per day, so 12 hours = 1.5 rows. The forecast will still work correctly, but won't be as easy to obviously prove correct at a glance.

If we rename full_0 and full_1 to __auto__ then we should get a forecast 48 hours into the future, which should give a less confusing forecast.

Ok, renamed to auto


pkg/sql/opt/exec/execbuilder/testdata/partial_stats line 824 at r20 (raw file):

Previously, michae2 (Michael Erickson) wrote…

Won't ConstructExtremesHistogram always make upperHist with a first bucket with num_range and distinct_range of zero?

Done.


pkg/sql/opt/exec/execbuilder/testdata/partial_stats line 864 at r20 (raw file):

Previously, michae2 (Michael Erickson) wrote…

same comment here about num_range and distinct_range

Done -- this histogram was constructed by hand from a larger histogram as histograms this small wouldn't have num_range, distinct_range > 0. I've added those buckets back in.

Copy link
Copy Markdown
Contributor Author

@faizaanmadhani faizaanmadhani 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! 1 of 0 LGTMs obtained (and 2 stale) (waiting on @mgartner and @michae2)


pkg/sql/distsql_plan_stats.go line 170 at r16 (raw file):

Previously, mgartner (Marcus Gartner) wrote…

Thanks for the example, Michael. I would expect partial stats to not occur at all if sql.stats.histogram_collection.enabled = false (@faizaanmadhani let's make an issue to track that). But I believe a slight modification to your example of setting SET CLUSTER SETTING sql.stats.histogram_collection.enabled = true after the non-histogram full collection would lead to the same problem.

I suggest we make this logic more closely match how you've described things should work. It sounds like the logic is:

  1. Is this the most recent full stat for this column? If not, continue.
  2. Does the full stat have a histogram? If not, return an error.
  3. Is the histogram's type invertable or a byes type? If so, return an error.

Making these steps distinct may make the code more clear. Or, if you feel strongly that it should remain as-is, then I'd suggest at least renaming isFullStatValidToCreatePartial because it's suprising that a full stat without a histogram can be "valid" to create a partial.

I'm also curious when we'd return an error from (3) without being able to error before the loop by checking the column's type. Is that specifically for trigram indexes?

Ok, I removed the function and moved this logic below. I also modified it so it's easier to read and a bit clearer.

Copy link
Copy Markdown
Contributor

@mgartner mgartner left a comment

Choose a reason for hiding this comment

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

:lgtm:

Can you make an issue to make sure that statement bundles include the latest full stats and partial stats?

Reviewed 1 of 2 files at r18, 1 of 1 files at r20, 1 of 1 files at r21, 1 of 2 files at r22, 1 of 1 files at r24, all commit messages.
Reviewable status: :shipit: complete! 2 of 0 LGTMs obtained (and 1 stale) (waiting on @faizaanmadhani and @michae2)


pkg/sql/logictest/testdata/logic_test/distsql_stats line 2205 at r24 (raw file):


# Verify that a non-inverted index string column with a string histogram
# can have partial statistics

Do you want to add a test for inverted indexes as well, similar to the error cases above?

Copy link
Copy Markdown
Collaborator

@michae2 michae2 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, Faizaan!

Reviewed 1 of 1 files at r24.
Reviewable status: :shipit: complete! 2 of 0 LGTMs obtained (and 1 stale) (waiting on @faizaanmadhani and @mgartner)


pkg/sql/distsql_plan_stats.go line 170 at r16 (raw file):

Previously, faizaanmadhani (Faizaan Madhani) wrote…

Ok, I removed the function and moved this logic below. I also modified it so it's easier to read and a bit clearer.

Really clear. Perfect. Thank you!

@faizaanmadhani
Copy link
Copy Markdown
Contributor Author

TFTR! 🎉
bors r=michae2

@faizaanmadhani
Copy link
Copy Markdown
Contributor Author

bors r-

@craig
Copy link
Copy Markdown
Contributor

craig bot commented Dec 20, 2022

Canceled.

Copy link
Copy Markdown
Contributor

@mgartner mgartner left a comment

Choose a reason for hiding this comment

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

:lgtm:

Reviewed 2 of 2 files at r25, 1 of 1 files at r26, all commit messages.
Reviewable status: :shipit: complete! 2 of 0 LGTMs obtained (and 1 stale) (waiting on @faizaanmadhani and @michae2)

@faizaanmadhani
Copy link
Copy Markdown
Contributor Author

TFTRs! 🔥

bors r=michae2, rytaft, mgartner

@craig
Copy link
Copy Markdown
Contributor

craig bot commented Dec 21, 2022

Build failed (retrying...):

@craig
Copy link
Copy Markdown
Contributor

craig bot commented Dec 21, 2022

Build failed (retrying...):

@erikgrinaker
Copy link
Copy Markdown
Contributor

erikgrinaker commented Dec 21, 2022

Cancelling since this doesn't pass bors, it needs a dev gen bazel

bors r-

diff --git a/pkg/sql/stats/BUILD.bazel b/pkg/sql/stats/BUILD.bazel
index b945a28d23..548af1a8a6 100644
--- a/pkg/sql/stats/BUILD.bazel
+++ b/pkg/sql/stats/BUILD.bazel
@@ -47,6 +47,7 @@ go_library(
         "//pkg/sql/sqlerrors",
         "//pkg/sql/sqlutil",
         "//pkg/sql/types",
+        "//pkg/util",
         "//pkg/util/cache",
         "//pkg/util/encoding",
         "//pkg/util/hlc",

@craig
Copy link
Copy Markdown
Contributor

craig bot commented Dec 21, 2022

Canceled.

This commit adds support to the table stats
cache to merge partial statistics with full table
statistics when updating the stats cache, in an
effort to keep more accurate table statistics that
can be used by the optimizer.

Additionally, if forecasting is enabled for the table, these
merged statistics are used for the forecast.

Epic: CRDB-19449

Release note (sql change): The optimizer will
now use table statistics that are merged combinations
of the newest partial statistic and latest full
statistic collection. And, if forecasting is enabled,
the merged statistic will be used in the forecast.
@faizaanmadhani
Copy link
Copy Markdown
Contributor Author

bors r=michae2,mgartner,rytaft

@craig
Copy link
Copy Markdown
Contributor

craig bot commented Dec 21, 2022

Build succeeded:

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.

6 participants