Skip to content

Commit

Permalink
sql: add more session variables to statement bundles
Browse files Browse the repository at this point in the history
Add some missing session variables to statement diagnostic bundles. This
isn't quite everything, but it's better than what we had before.

Release note: None
  • Loading branch information
michae2 committed Aug 24, 2022
1 parent e3601c5 commit 129f1de
Show file tree
Hide file tree
Showing 5 changed files with 157 additions and 41 deletions.
67 changes: 56 additions & 11 deletions pkg/sql/explain_bundle.go
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/sessiondata"
"github.com/cockroachdb/cockroach/pkg/sql/sessiondatapb"
"github.com/cockroachdb/cockroach/pkg/sql/stmtdiagnostics"
"github.com/cockroachdb/cockroach/pkg/util/duration"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/memzipper"
"github.com/cockroachdb/cockroach/pkg/util/tracing"
Expand Down Expand Up @@ -503,6 +504,22 @@ func (c *stmtEnvCollector) PrintSessionSettings(w io.Writer) error {
return boolStr
}

datestyleConv := func(enumVal string) string {
n, err := strconv.ParseInt(enumVal, 10, 32)
if err != nil || n < 0 || n >= int64(len(dateStyleEnumMap)) {
return enumVal
}
return dateStyleEnumMap[n]
}

intervalstyleConv := func(enumVal string) string {
n, err := strconv.ParseInt(enumVal, 10, 32)
if err != nil || n < 0 || n >= int64(len(duration.IntervalStyle_name)) {
return enumVal
}
return strings.ToLower(duration.IntervalStyle(n).String())
}

distsqlConv := func(enumVal string) string {
n, err := strconv.ParseInt(enumVal, 10, 32)
if err != nil {
Expand All @@ -526,18 +543,42 @@ func (c *stmtEnvCollector) PrintSessionSettings(w io.Writer) error {
clusterSetting settings.NonMaskedSetting
convFunc func(string) string
}{
{sessionSetting: "reorder_joins_limit", clusterSetting: ReorderJoinsLimitClusterValue},
{sessionSetting: "allow_prepare_as_opt_plan"},
{sessionSetting: "copy_fast_path_enabled"},
{sessionSetting: "cost_scans_with_default_col_size", clusterSetting: costScansWithDefaultColSize, convFunc: boolToOnOff},
{sessionSetting: "datestyle", clusterSetting: dateStyle, convFunc: datestyleConv},
{sessionSetting: "default_int_size", clusterSetting: defaultIntSize},
{sessionSetting: "default_transaction_priority"},
{sessionSetting: "default_transaction_quality_of_service"},
{sessionSetting: "default_transaction_read_only"},
{sessionSetting: "default_transaction_use_follower_reads"},
{sessionSetting: "disallow_full_table_scans", clusterSetting: disallowFullTableScans, convFunc: boolToOnOff},
{sessionSetting: "distsql", clusterSetting: DistSQLClusterExecMode, convFunc: distsqlConv},
{sessionSetting: "enable_implicit_select_for_update", clusterSetting: implicitSelectForUpdateClusterMode, convFunc: boolToOnOff},
{sessionSetting: "enable_implicit_transaction_for_batch_statements"},
{sessionSetting: "enable_insert_fast_path", clusterSetting: insertFastPathClusterMode, convFunc: boolToOnOff},
{sessionSetting: "enable_multiple_modifications_of_table"},
{sessionSetting: "enable_zigzag_join", clusterSetting: zigzagJoinClusterMode, convFunc: boolToOnOff},
{sessionSetting: "expect_and_ignore_not_visible_columns_in_copy"},
{sessionSetting: "intervalstyle", clusterSetting: intervalStyle, convFunc: intervalstyleConv},
{sessionSetting: "large_full_scan_rows", clusterSetting: largeFullScanRows},
{sessionSetting: "locality_optimized_partitioned_index_scan", clusterSetting: localityOptimizedSearchMode, convFunc: boolToOnOff},
{sessionSetting: "null_ordered_last"},
{sessionSetting: "on_update_rehome_row_enabled", clusterSetting: onUpdateRehomeRowEnabledClusterMode, convFunc: boolToOnOff},
{sessionSetting: "opt_split_scan_limit"},
{sessionSetting: "optimizer_use_histograms", clusterSetting: optUseHistogramsClusterMode, convFunc: boolToOnOff},
{sessionSetting: "optimizer_use_multicol_stats", clusterSetting: optUseMultiColStatsClusterMode, convFunc: boolToOnOff},
{sessionSetting: "locality_optimized_partitioned_index_scan", clusterSetting: localityOptimizedSearchMode, convFunc: boolToOnOff},
{sessionSetting: "propagate_input_ordering", clusterSetting: propagateInputOrdering, convFunc: boolToOnOff},
{sessionSetting: "optimizer_use_not_visible_indexes"},
{sessionSetting: "pg_trgm.similarity_threshold"},
{sessionSetting: "prefer_lookup_joins_for_fks", clusterSetting: preferLookupJoinsForFKs, convFunc: boolToOnOff},
{sessionSetting: "disallow_full_table_scans", clusterSetting: disallowFullTableScans, convFunc: boolToOnOff},
{sessionSetting: "large_full_scan_rows", clusterSetting: largeFullScanRows},
{sessionSetting: "cost_scans_with_default_col_size", clusterSetting: costScansWithDefaultColSize, convFunc: boolToOnOff},
{sessionSetting: "default_transaction_quality_of_service"},
{sessionSetting: "distsql", clusterSetting: DistSQLClusterExecMode, convFunc: distsqlConv},
{sessionSetting: "propagate_input_ordering", clusterSetting: propagateInputOrdering, convFunc: boolToOnOff},
{sessionSetting: "reorder_joins_limit", clusterSetting: ReorderJoinsLimitClusterValue},
{sessionSetting: "sql_safe_updates"},
{sessionSetting: "testing_optimizer_cost_perturbation"},
{sessionSetting: "testing_optimizer_disable_rule_probability"},
{sessionSetting: "testing_optimizer_random_seed"},
{sessionSetting: "timezone"},
{sessionSetting: "unconstrained_non_covering_index_scan_enabled"},
{sessionSetting: "vectorize", clusterSetting: VectorizeClusterMode, convFunc: vectorizeConv},
}

Expand All @@ -549,9 +590,11 @@ func (c *stmtEnvCollector) PrintSessionSettings(w io.Writer) error {
// Get the default value for the cluster setting.
var def string
if s.clusterSetting == nil {
// Special handling for default_transaction_quality_of_service since it
// has no cluster setting.
def = sessiondatapb.Normal.String()
if ok, v, _ := getSessionVar(s.sessionSetting, true); ok {
if v.GlobalDefault != nil {
def = v.GlobalDefault(nil /* *settings.Values */)
}
}
} else {
def = s.clusterSetting.EncodedDefault()
}
Expand All @@ -571,6 +614,8 @@ func (c *stmtEnvCollector) PrintSessionSettings(w io.Writer) error {
}

func (c *stmtEnvCollector) PrintClusterSettings(w io.Writer) error {
// TODO(michae2): We should also query system.database_role_settings.

rows, err := c.ie.QueryBufferedEx(
c.ctx,
"stmtEnvCollector",
Expand Down
100 changes: 85 additions & 15 deletions pkg/sql/explain_bundle_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -60,7 +60,7 @@ CREATE TABLE s.a (a INT PRIMARY KEY);`)
t.Run("basic", func(t *testing.T) {
rows := r.QueryStr(t, "EXPLAIN ANALYZE (DEBUG) SELECT * FROM abc WHERE c=1")
checkBundle(
t, fmt.Sprint(rows), "public.abc",
t, fmt.Sprint(rows), "public.abc", nil,
base, plans, "stats-defaultdb.public.abc.sql", "distsql.html vec.txt vec-v.txt",
)
})
Expand All @@ -69,15 +69,15 @@ CREATE TABLE s.a (a INT PRIMARY KEY);`)
t.Run("subqueries", func(t *testing.T) {
rows := r.QueryStr(t, "EXPLAIN ANALYZE (DEBUG) SELECT EXISTS (SELECT * FROM abc WHERE c=1)")
checkBundle(
t, fmt.Sprint(rows), "public.abc",
t, fmt.Sprint(rows), "public.abc", nil,
base, plans, "stats-defaultdb.public.abc.sql", "distsql-2-main-query.html distsql-1-subquery.html vec-1-subquery-v.txt vec-1-subquery.txt vec-2-main-query-v.txt vec-2-main-query.txt",
)
})

t.Run("user-defined schema", func(t *testing.T) {
rows := r.QueryStr(t, "EXPLAIN ANALYZE (DEBUG) SELECT * FROM s.a WHERE a=1")
checkBundle(
t, fmt.Sprint(rows), "s.a",
t, fmt.Sprint(rows), "s.a", nil,
base, plans, "stats-defaultdb.s.a.sql", "distsql.html vec.txt vec-v.txt",
)
})
Expand All @@ -91,7 +91,7 @@ CREATE TABLE s.a (a INT PRIMARY KEY);`)
// The bundle url is inside the error detail.
var pqErr *pq.Error
_ = errors.As(err, &pqErr)
checkBundle(t, fmt.Sprintf("%+v", pqErr.Detail), "", base)
checkBundle(t, fmt.Sprintf("%+v", pqErr.Detail), "", nil, base)
})

// Verify that we can issue the statement with prepare (which can happen
Expand All @@ -116,7 +116,7 @@ CREATE TABLE s.a (a INT PRIMARY KEY);`)
rowsBuf.WriteByte('\n')
}
checkBundle(
t, rowsBuf.String(), "public.abc",
t, rowsBuf.String(), "public.abc", nil,
base, plans, "stats-defaultdb.public.abc.sql", "distsql.html vec.txt vec-v.txt",
)
})
Expand All @@ -135,7 +135,7 @@ CREATE TABLE users(id UUID DEFAULT gen_random_uuid() PRIMARY KEY, promo_id INT R
`)
rows := r.QueryStr(t, "EXPLAIN ANALYZE (DEBUG) INSERT INTO users (promo_id) VALUES (642606224929619969);")
checkBundle(
t, fmt.Sprint(rows), "public.users", base, plans,
t, fmt.Sprint(rows), "public.users", nil, base, plans,
"stats-defaultdb.public.users.sql", "stats-defaultdb.public.promos.sql",
"distsql-1-main-query.html distsql-2-postquery.html vec-1-main-query-v.txt vec-1-main-query.txt vec-2-postquery-v.txt vec-2-postquery.txt",
)
Expand All @@ -147,17 +147,82 @@ CREATE TABLE users(id UUID DEFAULT gen_random_uuid() PRIMARY KEY, promo_id INT R
defer r.Exec(t, "SET CLUSTER SETTING sql.trace.txn.enable_threshold='0ms';")
rows := r.QueryStr(t, "EXPLAIN ANALYZE (DEBUG) SELECT * FROM abc WHERE c=1")
checkBundle(
t, fmt.Sprint(rows), "public.abc",
t, fmt.Sprint(rows), "public.abc", nil,
base, plans, "stats-defaultdb.public.abc.sql", "distsql.html vec.txt vec-v.txt",
)
})

t.Run("session-settings", func(t *testing.T) {
testcases := []struct {
sessionVar, value string
}{
{"allow_prepare_as_opt_plan", "on"},
{"cost_scans_with_default_col_size", "on"},
{"datestyle", "'ISO, DMY'"},
{"default_int_size", "4"},
{"default_transaction_priority", "low"},
{"default_transaction_quality_of_service", "background"},
{"default_transaction_read_only", "on"},
{"disallow_full_table_scans", "on"},
{"distsql", "always"},
{"enable_implicit_select_for_update", "off"},
{"enable_implicit_transaction_for_batch_statements", "off"},
{"enable_insert_fast_path", "off"},
{"enable_multiple_modifications_of_table", "on"},
{"enable_zigzag_join", "off"},
{"expect_and_ignore_not_visible_columns_in_copy", "on"},
{"intervalstyle", "iso_8601"},
{"large_full_scan_rows", "2000"},
{"locality_optimized_partitioned_index_scan", "off"},
{"null_ordered_last", "on"},
{"on_update_rehome_row_enabled", "off"},
{"opt_split_scan_limit", "1000"},
{"optimizer_use_histograms", "off"},
{"optimizer_use_multicol_stats", "off"},
{"optimizer_use_not_visible_indexes", "on"},
{"pg_trgm.similarity_threshold", "0.6"},
{"prefer_lookup_joins_for_fks", "on"},
{"propagate_input_ordering", "on"},
{"reorder_joins_limit", "3"},
{"sql_safe_updates", "on"},
{"testing_optimizer_cost_perturbation", "0.3"},
{"testing_optimizer_disable_rule_probability", "0.00000000001"},
{"testing_optimizer_random_seed", "123"},
{"timezone", "+8"},
{"unconstrained_non_covering_index_scan_enabled", "on"},
}
for _, tc := range testcases {
t.Run(tc.sessionVar, func(t *testing.T) {
r.Exec(t, fmt.Sprintf("SET %s = %s", tc.sessionVar, tc.value))
defer r.Exec(t, fmt.Sprintf("RESET %s", tc.sessionVar))
rows := r.QueryStr(t, "EXPLAIN ANALYZE (DEBUG) SELECT * FROM abc WHERE c=1")
checkBundle(
t, fmt.Sprint(rows), "public.abc", func(name, contents string) error {
if name == "env.sql" {
reg := regexp.MustCompile(fmt.Sprintf("SET %s.*-- default value", tc.sessionVar))
if reg.FindString(contents) == "" {
return errors.Errorf("could not find 'SET %s' in env.sql", tc.sessionVar)
}
}
return nil
},
base, plans, "stats-defaultdb.public.abc.sql", "distsql.html vec.txt vec-v.txt",
)
})
}
})
}

// checkBundle searches text strings for a bundle URL and then verifies that the
// bundle contains the expected files. The expected files are passed as an
// arbitrary number of strings; each string contains one or more filenames
// separated by a space.
func checkBundle(t *testing.T, text, tableName string, expectedFiles ...string) {
func checkBundle(
t *testing.T,
text, tableName string,
contentCheck func(name, contents string) error,
expectedFiles ...string,
) {
httpClient := httputil.NewClientWithTimeout(30 * time.Second)

t.Helper()
Expand Down Expand Up @@ -194,28 +259,33 @@ func checkBundle(t *testing.T, text, tableName string, expectedFiles ...string)
t.Fatal(err)
}
defer r.Close()
contents, err := ioutil.ReadAll(r)
bytes, err := ioutil.ReadAll(r)
if err != nil {
t.Fatal(err)
}
contents := string(bytes)

if strings.Contains(string(contents), "-- error") {
if strings.Contains(contents, "-- error") {
t.Errorf(
"expected no errors in %s, file contents:\n%s",
f.Name,
string(contents),
f.Name, contents,
)
}

if f.Name == "schema.sql" {
if !strings.Contains(string(contents), tableName) {
if !strings.Contains(contents, tableName) {
t.Errorf(
"expected table name to appear in schema.sql. tableName: %s\nfile contents:\n%s",
tableName,
string(contents),
tableName, contents,
)
}
}

if contentCheck != nil {
if err := contentCheck(f.Name, contents); err != nil {
t.Error(err)
}
}
}

var expList []string
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/logictest/testdata/logic_test/pg_catalog
Original file line number Diff line number Diff line change
Expand Up @@ -4354,7 +4354,7 @@ optimizer_use_not_visible_indexes off NULL
override_multi_region_zone_config off NULL user NULL off off
parallelize_multi_key_lookup_joins_enabled off NULL user NULL false false
password_encryption scram-sha-256 NULL user NULL scram-sha-256 scram-sha-256
pg_trgm.similarity_threshold 0.3 NULL user NULL .3 .3
pg_trgm.similarity_threshold 0.3 NULL user NULL 0.3 0.3
prefer_lookup_joins_for_fks off NULL user NULL off off
propagate_input_ordering off NULL user NULL off off
reorder_joins_limit 8 NULL user NULL 8 8
Expand Down
Loading

0 comments on commit 129f1de

Please sign in to comment.