diff --git a/pkg/cli/statement_bundle.go b/pkg/cli/statement_bundle.go index 28d9ad909b60..19a2835a56f4 100644 --- a/pkg/cli/statement_bundle.go +++ b/pkg/cli/statement_bundle.go @@ -149,6 +149,7 @@ func runBundleRecreate(cmd *cobra.Command, args []string) (resErr error) { if placeholderPairs != nil { placeholderToColMap := make(map[int]string) + placeholderFQColNames := make(map[string]struct{}) for _, placeholderPairStr := range placeholderPairs { pair := strings.Split(placeholderPairStr, "=") if len(pair) != 2 { @@ -159,8 +160,11 @@ func runBundleRecreate(cmd *cobra.Command, args []string) (resErr error) { return err } placeholderToColMap[n] = pair[1] + placeholderFQColNames[pair[1]] = struct{}{} } - inputs, outputs, err := getExplainCombinations(conn, explainPrefix, placeholderToColMap, bundle) + inputs, outputs, err := getExplainCombinations( + conn, explainPrefix, placeholderToColMap, placeholderFQColNames, bundle, + ) if err != nil { return err } @@ -209,6 +213,7 @@ func getExplainCombinations( conn clisqlclient.Conn, explainPrefix string, placeholderToColMap map[int]string, + placeholderFQColNames map[string]struct{}, bundle *statementBundle, ) (inputs [][]string, explainOutputs []string, err error) { @@ -273,6 +278,11 @@ func getExplainCombinations( } col := columns[0] fqColName := fmt.Sprintf("%s.%s", tableName, col) + if _, isPlaceholder := placeholderFQColNames[fqColName]; !isPlaceholder { + // This column is not one of the placeholder values, so simply + // ignore it. + continue + } d, _, err := tree.ParseDTimestamp(nil, stat["created_at"].(string), time.Microsecond) if err != nil { panic(err) @@ -285,7 +295,9 @@ func getExplainCombinations( typ := stat["histo_col_type"].(string) if typ == "" { - fmt.Println("Ignoring column with empty type ", col) + // Empty 'histo_col_type' is used when there is no histogram for + // the column, simply skip this stat (see stats/json.go for more + // details). continue } colTypeRef, err := parser.GetTypeFromValidSQLSyntax(typ) @@ -300,6 +312,10 @@ func getExplainCombinations( continue } buckets := stat["histo_buckets"].([]interface{}) + // addedNonExistent tracks whether we included at least one + // "previous" datum which - according to the histograms - is not + // present in the table. + var addedNonExistent bool var maxUpperBound tree.Datum for _, b := range buckets { bucket := b.(map[string]interface{}) @@ -318,9 +334,24 @@ func getExplainCombinations( if maxUpperBound == nil || maxUpperBound.Compare(&evalCtx, datum) < 0 { maxUpperBound = datum } - if numRange > 0 { - if prev, ok := datum.Prev(&evalCtx); ok { + // If we have any datums within the bucket (i.e. not equal to + // the upper bound), we always attempt to add a "previous" to + // the upper bound datum. + addPrevious := numRange > 0 + if numRange == 0 && !addedNonExistent { + // If our bucket says that there are no values present in + // the table between the current upper bound and the upper + // bound of the previous histogram bucket, then we only + // attempt to add the "previous" non-existent datum if we + // haven't done so already (this is to avoid the redundant + // non-existent values which would get treated in the same + // fashion anyway). + addPrevious = true + } + if addPrevious { + if prev, ok := tree.DatumPrev(datum, &evalCtx, &evalCtx.CollationEnv); ok { bucketMap[key] = append(bucketMap[key], tree.AsStringWithFlags(prev, fmtCtx)) + addedNonExistent = addedNonExistent || numRange == 0 } } } @@ -330,7 +361,7 @@ func getExplainCombinations( } // Create a value that's outside of histogram range by incrementing the // max value that we've seen. - if outside, ok := maxUpperBound.Next(&evalCtx); ok { + if outside, ok := tree.DatumNext(maxUpperBound, &evalCtx, &evalCtx.CollationEnv); ok { colSamples = append(colSamples, tree.AsStringWithFlags(outside, fmtCtx)) } sort.Strings(colSamples) @@ -386,7 +417,8 @@ func getExplainCombinations( func getExplainOutputs( conn clisqlclient.Conn, explainPrefix string, statement string, inputs [][]string, ) (explainStrings []string, err error) { - for _, values := range inputs { + fmt.Printf("trying %d placeholder combinations\n", len(inputs)) + for i, values := range inputs { // Run an explain for each possible input. query := fmt.Sprintf("%s %s", explainPrefix, statement) args := make([]interface{}, len(values)) @@ -409,6 +441,9 @@ func getExplainOutputs( return nil, err } explainStrings = append(explainStrings, explainStr.String()) + if (i+1)%1000 == 0 { + fmt.Printf("%d placeholder combinations are done\n", i+1) + } } return explainStrings, nil } diff --git a/pkg/cli/statement_bundle_test.go b/pkg/cli/statement_bundle_test.go index 5dbcf51daffb..01bd17fd75cf 100644 --- a/pkg/cli/statement_bundle_test.go +++ b/pkg/cli/statement_bundle_test.go @@ -32,10 +32,11 @@ import ( func TestRunExplainCombinations(t *testing.T) { defer leaktest.AfterTest(t)() tests := []struct { - bundlePath string - placeholderToColMap map[int]string - expectedInputs [][]string - expectedOutputs []string + bundlePath string + placeholderToColMap map[int]string + placeholderFQColNames map[string]struct{} + expectedInputs [][]string + expectedOutputs []string }{ { bundlePath: "bundle", @@ -43,6 +44,10 @@ func TestRunExplainCombinations(t *testing.T) { 1: "public.a.a", 2: "public.a.b", }, + placeholderFQColNames: map[string]struct{}{ + "public.a.a": {}, + "public.a.b": {}, + }, expectedInputs: [][]string{{"999", "8"}}, expectedOutputs: []string{`select ├── scan a @@ -82,7 +87,9 @@ func TestRunExplainCombinations(t *testing.T) { } } - inputs, outputs, err := getExplainCombinations(conn, "EXPLAIN(OPT)", test.placeholderToColMap, bundle) + inputs, outputs, err := getExplainCombinations( + conn, "EXPLAIN(OPT)", test.placeholderToColMap, test.placeholderFQColNames, bundle, + ) assert.NoError(t, err) assert.Equal(t, test.expectedInputs, inputs) assert.Equal(t, test.expectedOutputs, outputs) diff --git a/pkg/sql/sem/tree/BUILD.bazel b/pkg/sql/sem/tree/BUILD.bazel index e8dd40e5c1b5..a0893db3d7f6 100644 --- a/pkg/sql/sem/tree/BUILD.bazel +++ b/pkg/sql/sem/tree/BUILD.bazel @@ -174,6 +174,7 @@ go_test( "constant_test.go", "datum_integration_test.go", "datum_invariants_test.go", + "datum_prev_next_test.go", "datum_test.go", "expr_test.go", "format_test.go", diff --git a/pkg/sql/sem/tree/datum.go b/pkg/sql/sem/tree/datum.go index 3efd463354e2..9f92952592ff 100644 --- a/pkg/sql/sem/tree/datum.go +++ b/pkg/sql/sem/tree/datum.go @@ -1629,7 +1629,7 @@ func (d *DEncodedKey) Prev(ctx CompareContext) (Datum, bool) { // Next implements the Datum interface. func (d *DEncodedKey) Next(ctx CompareContext) (Datum, bool) { - return nil, true + return nil, false } // IsMax implements the Datum interface. @@ -5794,3 +5794,120 @@ func AdjustValueToType(typ *types.T, inVal Datum) (outVal Datum, err error) { } return inVal, nil } + +// DatumPrev returns a datum that is "previous" to the given one. For many types +// it just delegates to Datum.Prev, but for some types that don't have an +// implementation of that function this method makes the best effort to come up +// with a reasonable previous datum that is smaller than the given one. +// +// The return value is undefined if Datum.IsMin returns true or if the value is +// NaN of an infinity (for floats and decimals). +func DatumPrev( + datum Datum, cmpCtx CompareContext, collationEnv *CollationEnvironment, +) (Datum, bool) { + datum = UnwrapDOidWrapper(datum) + prevString := func(s string) (string, bool) { + // In order to obtain a previous string we subtract 1 from the last + // non-zero byte. + b := []byte(s) + lastNonZeroByteIdx := len(b) - 1 + for ; lastNonZeroByteIdx >= 0 && b[lastNonZeroByteIdx] == 0; lastNonZeroByteIdx-- { + } + if lastNonZeroByteIdx < 0 { + return "", false + } + b[lastNonZeroByteIdx]-- + return string(b), true + } + switch d := datum.(type) { + case *DDecimal: + var prev DDecimal + var sub apd.Decimal + _, err := sub.SetFloat64(1e-6) + if err != nil { + return nil, false + } + _, err = ExactCtx.Sub(&prev.Decimal, &d.Decimal, &sub) + if err != nil { + return nil, false + } + return &prev, true + case *DString: + prev, ok := prevString(string(*d)) + if !ok { + return nil, false + } + return NewDString(prev), true + case *DCollatedString: + prev, ok := prevString(d.Contents) + if !ok { + return nil, false + } + c, err := NewDCollatedString(prev, d.Locale, collationEnv) + if err != nil { + return nil, false + } + return c, true + case *DBytes: + prev, ok := prevString(string(*d)) + if !ok { + return nil, false + } + return NewDBytes(DBytes(prev)), true + case *DInterval: + // Subtract 1ms. + prev := d.Sub(duration.MakeDuration(1000000 /* nanos */, 0 /* days */, 0 /* months */)) + return NewDInterval(prev, types.DefaultIntervalTypeMetadata), true + default: + // TODO(yuzefovich): consider adding support for other datums that don't + // have Datum.Prev implementation (DBitArray, DGeography, DGeometry, + // DBox2D, DJSON, DArray). + return datum.Prev(cmpCtx) + } +} + +// DatumNext returns a datum that is "next" to the given one. For many types it +// just delegates to Datum.Next, but for some types that don't have an +// implementation of that function this method makes the best effort to come up +// with a reasonable next datum that is greater than the given one. +// +// The return value is undefined if Datum.IsMax returns true or if the value is +// NaN of an infinity (for floats and decimals). +func DatumNext( + datum Datum, cmpCtx CompareContext, collationEnv *CollationEnvironment, +) (Datum, bool) { + datum = UnwrapDOidWrapper(datum) + switch d := datum.(type) { + case *DDecimal: + var next DDecimal + var add apd.Decimal + _, err := add.SetFloat64(1e-6) + if err != nil { + return nil, false + } + _, err = ExactCtx.Add(&next.Decimal, &d.Decimal, &add) + if err != nil { + return nil, false + } + return &next, true + case *DCollatedString: + s := NewDString(d.Contents) + next, ok := s.Next(cmpCtx) + if !ok { + return nil, false + } + c, err := NewDCollatedString(string(*next.(*DString)), d.Locale, collationEnv) + if err != nil { + return nil, false + } + return c, true + case *DInterval: + next := d.Add(duration.MakeDuration(1000000 /* nanos */, 0 /* days */, 0 /* months */)) + return NewDInterval(next, types.DefaultIntervalTypeMetadata), true + default: + // TODO(yuzefovich): consider adding support for other datums that don't + // have Datum.Next implementation (DGeography, DGeometry, DBox2D, + // DJSON). + return datum.Next(cmpCtx) + } +} diff --git a/pkg/sql/sem/tree/datum_prev_next_test.go b/pkg/sql/sem/tree/datum_prev_next_test.go new file mode 100644 index 000000000000..a53c4b34dcba --- /dev/null +++ b/pkg/sql/sem/tree/datum_prev_next_test.go @@ -0,0 +1,66 @@ +// Copyright 2023 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package tree_test + +import ( + "math" + "testing" + + "github.com/cockroachdb/apd/v3" + "github.com/cockroachdb/cockroach/pkg/sql/randgen" + "github.com/cockroachdb/cockroach/pkg/sql/sem/eval" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/cockroach/pkg/util/randutil" + "github.com/stretchr/testify/require" +) + +// TestDatumPrevNext verifies that tree.DatumPrev and tree.DatumNext return +// datums that are smaller and larger, respectively, than the given datum if +// ok=true is returned (modulo some edge cases). +func TestDatumPrevNext(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + rng, _ := randutil.NewTestRand() + var evalCtx eval.Context + const numRuns = 1000 + for i := 0; i < numRuns; i++ { + typ := randgen.RandType(rng) + d := randgen.RandDatum(rng, typ, false /* nullOk */) + // Ignore NaNs and infinities. + if f, ok := d.(*tree.DFloat); ok { + if math.IsNaN(float64(*f)) || math.IsInf(float64(*f), 0) { + continue + } + } + if dec, ok := d.(*tree.DDecimal); ok { + if dec.Form == apd.NaN || dec.Form == apd.Infinite { + continue + } + } + if !d.IsMin(&evalCtx) { + if prev, ok := tree.DatumPrev(d, &evalCtx, &evalCtx.CollationEnv); ok { + cmp, err := d.CompareError(&evalCtx, prev) + require.NoError(t, err) + require.True(t, cmp > 0, "d=%s, prev=%s, type=%s", d.String(), prev.String(), d.ResolvedType().SQLString()) + } + } + if !d.IsMax(&evalCtx) { + if next, ok := tree.DatumNext(d, &evalCtx, &evalCtx.CollationEnv); ok { + cmp, err := d.CompareError(&evalCtx, next) + require.NoError(t, err) + require.True(t, cmp < 0, "d=%s, next=%s, type=%s", d.String(), next.String(), d.ResolvedType().SQLString()) + } + } + } +}