From f5c22b0082167fc66cf66a83342e6f59c3ead309 Mon Sep 17 00:00:00 2001 From: Stan Rosenberg Date: Wed, 19 Jan 2022 00:17:05 -0500 Subject: [PATCH 1/4] roachprod: move default env. vars into config.go and remove the duplicates Both, COCKROACH_ENABLE_RPC_COMPRESSION=false and COCKROACH_UI_RELEASE_NOTES_SIGNUP_DISMISSED=true are passed into ENV_VARS of the systemd startup script (start.sh) by default. The same behavior is preserved _modulo_ duplication. Since roachprod can now be initialized either via CLI or API, default env. vars have been consolidated into config.go (used by both). Release note: None --- pkg/cmd/roachprod/flags.go | 52 +++++++++-------------- pkg/cmd/roachtest/cluster.go | 13 ------ pkg/roachprod/config/config.go | 14 ++++++ pkg/roachprod/install/cluster_settings.go | 7 +-- 4 files changed, 37 insertions(+), 49 deletions(-) diff --git a/pkg/cmd/roachprod/flags.go b/pkg/cmd/roachprod/flags.go index 37117039b692..0dd54276de84 100644 --- a/pkg/cmd/roachprod/flags.go +++ b/pkg/cmd/roachprod/flags.go @@ -45,36 +45,26 @@ var ( listPattern string secure = false extraSSHOptions = "" - nodeEnv = []string{ - // NOTE: The defaults are also copied in roachtest's invocation of roachprod - // (which overrides the default). On changes, consider updating that one - // too. - - // RPC compressions costs around 5% on kv95, so we disable it. It might help - // when moving snapshots around, though. - "COCKROACH_ENABLE_RPC_COMPRESSION=false", - // Get rid of an annoying popup in the UI. - "COCKROACH_UI_RELEASE_NOTES_SIGNUP_DISMISSED=true", - } - tag string - external = false - pgurlCertsDir string - adminurlOpen = false - adminurlPath = "" - adminurlIPs = false - useTreeDist = true - sig = 9 - waitFlag = false - createVMOpts = vm.DefaultCreateOpts() - startOpts = roachprod.DefaultStartOpts() - stageOS string - stageDir string - logsDir string - logsFilter string - logsProgramFilter string - logsFrom time.Time - logsTo time.Time - logsInterval time.Duration + nodeEnv []string + tag string + external = false + pgurlCertsDir string + adminurlOpen = false + adminurlPath = "" + adminurlIPs = false + useTreeDist = true + sig = 9 + waitFlag = false + createVMOpts = vm.DefaultCreateOpts() + startOpts = roachprod.DefaultStartOpts() + stageOS string + stageDir string + logsDir string + logsFilter string + logsProgramFilter string + logsFrom time.Time + logsTo time.Time + logsInterval time.Duration monitorOpts install.MonitorOpts cachedHostsCluster string @@ -188,7 +178,7 @@ func initFlags() { startCmd.Flags().StringArrayVarP(&startOpts.ExtraArgs, "args", "a", nil, "node arguments") startCmd.Flags().StringArrayVarP(&nodeEnv, - "env", "e", nodeEnv, "node environment variables") + "env", "e", config.DefaultEnvVars(), "node environment variables") startCmd.Flags().BoolVar(&startOpts.EncryptedStores, "encrypt", startOpts.EncryptedStores, "start nodes with encryption at rest turned on") startCmd.Flags().BoolVar(&startOpts.SkipInit, diff --git a/pkg/cmd/roachtest/cluster.go b/pkg/cmd/roachtest/cluster.go index acb1466c227b..f98caa5117ff 100644 --- a/pkg/cmd/roachtest/cluster.go +++ b/pkg/cmd/roachtest/cluster.go @@ -1842,19 +1842,6 @@ func (c *clusterImpl) StartE( } } - // Set some env vars. The first two also the default for `roachprod start`, - // but we have to add them so that the third one doesn't wipe them out. - if !envExists(settings.Env, "COCKROACH_ENABLE_RPC_COMPRESSION") { - // RPC compressions costs around 5% on kv95, so we disable it. It might help - // when moving snapshots around, though. - settings.Env = append(settings.Env, "COCKROACH_ENABLE_RPC_COMPRESSION=false") - } - - if !envExists(settings.Env, "COCKROACH_UI_RELEASE_NOTES_SIGNUP_DISMISSED") { - // Get rid of an annoying popup in the UI. - settings.Env = append(settings.Env, "COCKROACH_UI_RELEASE_NOTES_SIGNUP_DISMISSED=true") - } - if !envExists(settings.Env, "COCKROACH_CRASH_ON_SPAN_USE_AFTER_FINISH") { // Panic on span use-after-Finish, so we catch such bugs. settings.Env = append(settings.Env, "COCKROACH_CRASH_ON_SPAN_USE_AFTER_FINISH=true") diff --git a/pkg/roachprod/config/config.go b/pkg/roachprod/config/config.go index 4391b90f4198..f584b233c260 100644 --- a/pkg/roachprod/config/config.go +++ b/pkg/roachprod/config/config.go @@ -73,6 +73,20 @@ const ( DefaultAdminUIPort = 26258 ) +// DefaultEnvVars returns default environment variables used in conjunction with CLI and MakeClusterSettings. +// These can be overriden by specifying different values (last one wins). +// See 'generateStartCmd' which sets 'ENV_VARS' for the systemd startup script (start.sh). +func DefaultEnvVars() []string { + return []string{ + // RPC compressions costs around 5% on kv95, so we disable it. It might help + // when moving snapshots around, though. + // (For other perf. related knobs, see https://github.com/cockroachdb/cockroach/issues/17165) + "COCKROACH_ENABLE_RPC_COMPRESSION=false", + // Get rid of an annoying popup in the UI. + "COCKROACH_UI_RELEASE_NOTES_SIGNUP_DISMISSED=true", + } +} + // IsLocalClusterName returns true if the given name is a valid name for a local // cluster. // diff --git a/pkg/roachprod/install/cluster_settings.go b/pkg/roachprod/install/cluster_settings.go index 77bb7bebb7f9..02a8e92e472f 100644 --- a/pkg/roachprod/install/cluster_settings.go +++ b/pkg/roachprod/install/cluster_settings.go @@ -100,11 +100,8 @@ func MakeClusterSettings(opts ...ClusterSettingOption) ClusterSettings { PGUrlCertsDir: "./certs", Secure: false, UseTreeDist: true, - Env: []string{ - "COCKROACH_ENABLE_RPC_COMPRESSION=false", - "COCKROACH_UI_RELEASE_NOTES_SIGNUP_DISMISSED=true", - }, - NumRacks: 0, + Env: config.DefaultEnvVars(), + NumRacks: 0, } // Override default values using the passed options (if any). for _, opt := range opts { From a57aa7c8f42c254b94feb36a0383419c41df2384 Mon Sep 17 00:00:00 2001 From: Xin Hao Zhang Date: Wed, 23 Mar 2022 11:22:47 -0400 Subject: [PATCH 2/4] sql: record index usage stats in index joins Fixes #76173 Previously, the construction of index joins skipped the recording of the primary key for index usage stats. This commit records the use of the primary key if the statement is not of the EXPLAIN variety. Release note (bug fix): Index usage stats are now properly captured for index joins. --- .../serverccl/statusccl/tenant_status_test.go | 6 ++-- pkg/server/index_usage_stats_test.go | 29 ++++++++++++++++--- pkg/sql/opt_exec_factory.go | 11 ++++++- 3 files changed, 39 insertions(+), 7 deletions(-) diff --git a/pkg/ccl/serverccl/statusccl/tenant_status_test.go b/pkg/ccl/serverccl/statusccl/tenant_status_test.go index e1a64ece7f7a..4f7e4299fdc2 100644 --- a/pkg/ccl/serverccl/statusccl/tenant_status_test.go +++ b/pkg/ccl/serverccl/statusccl/tenant_status_test.go @@ -439,6 +439,7 @@ VALUES (1, 10, 100), (2, 20, 200), (3, 30, 300) Exec(t, "SELECT * FROM test") // Record scan on secondary index. + // Note that this is an index join and will also read from the primary index. cluster.tenantConn(randomServer). Exec(t, "SELECT * FROM test@test_a_idx") testTableIDStr := cluster.tenantConn(randomServer). @@ -463,7 +464,7 @@ WHERE table_id = ` + testTableIDStr // Assert index usage data was inserted. expected := [][]string{ - {testTableIDStr, "1", "1", "true"}, + {testTableIDStr, "1", "2", "true"}, // Primary index {testTableIDStr, "2", "1", "true"}, } cluster.tenantConn(randomServer).CheckQueryResults(t, query, expected) @@ -771,6 +772,7 @@ VALUES (1, 10, 100), (2, 20, 200), (3, 30, 300) testingCluster.tenantConn(0).Exec(t, "SELECT * FROM idx_test.test") // Record scan on secondary index. + // Note that this is an index join and will also read from the primary index. testingCluster.tenantConn(1).Exec(t, "SELECT * FROM idx_test.test@test_a_idx") testTableIDStr := testingCluster.tenantConn(2).QueryStr(t, "SELECT 'idx_test.test'::regclass::oid")[0][0] testTableID, err := strconv.Atoi(testTableIDStr) @@ -789,7 +791,7 @@ WHERE ` actual := testingCluster.tenantConn(2).QueryStr(t, query, testTableID) expected := [][]string{ - {testTableIDStr, "1", "1", "true"}, + {testTableIDStr, "1", "2", "true"}, {testTableIDStr, "2", "1", "true"}, } diff --git a/pkg/server/index_usage_stats_test.go b/pkg/server/index_usage_stats_test.go index 7c7507d83f39..4f8b46a3768c 100644 --- a/pkg/server/index_usage_stats_test.go +++ b/pkg/server/index_usage_stats_test.go @@ -76,6 +76,11 @@ func TestStatusAPIIndexUsage(t *testing.T) { LastRead: timeutil.Now(), } + expectedStatsIndexPrimary := roachpb.IndexUsageStatistics{ + TotalReadCount: 1, + LastRead: timeutil.Now(), + } + firstPgURL, firstServerConnCleanup := sqlutils.PGUrl( t, firstServer.ServingSQLAddr(), "CreateConnections" /* prefix */, url.User(security.RootUser)) defer firstServerConnCleanup() @@ -111,6 +116,11 @@ func TestStatusAPIIndexUsage(t *testing.T) { require.NoError(t, err) require.False(t, rows.Next()) + indexKeyPrimary := roachpb.IndexUsageKey{ + TableID: roachpb.TableID(tableID), + IndexID: 1, // t@t_pkey + } + indexKeyA := roachpb.IndexUsageKey{ TableID: roachpb.TableID(tableID), IndexID: 2, // t@t_a_idx @@ -150,7 +160,7 @@ func TestStatusAPIIndexUsage(t *testing.T) { _, err = secondServerSQLConn.Exec("SELECT k FROM t WHERE a = 10 AND b = 200") require.NoError(t, err) - // Record a full scan over t_b_idx. + // Record an index join and full scan of t_b_idx. _, err = secondServerSQLConn.Exec("SELECT * FROM t@t_b_idx") require.NoError(t, err) @@ -164,13 +174,19 @@ func TestStatusAPIIndexUsage(t *testing.T) { thirdLocalStatsReader := thirdServer.SQLServer().(*sql.Server).GetLocalIndexStatistics() // First node should have nothing. - stats := firstLocalStatsReader.Get(indexKeyA.TableID, indexKeyA.IndexID) + stats := firstLocalStatsReader.Get(indexKeyPrimary.TableID, indexKeyPrimary.IndexID) + require.Equal(t, roachpb.IndexUsageStatistics{}, stats, "expecting empty stats on node 1, but found %v", stats) + + stats = firstLocalStatsReader.Get(indexKeyA.TableID, indexKeyA.IndexID) require.Equal(t, roachpb.IndexUsageStatistics{}, stats, "expecting empty stats on node 1, but found %v", stats) stats = firstLocalStatsReader.Get(indexKeyB.TableID, indexKeyB.IndexID) require.Equal(t, roachpb.IndexUsageStatistics{}, stats, "expecting empty stats on node 1, but found %v", stats) // Third node should have nothing. + stats = firstLocalStatsReader.Get(indexKeyPrimary.TableID, indexKeyPrimary.IndexID) + require.Equal(t, roachpb.IndexUsageStatistics{}, stats, "expecting empty stats on node 3, but found %v", stats) + stats = thirdLocalStatsReader.Get(indexKeyA.TableID, indexKeyA.IndexID) require.Equal(t, roachpb.IndexUsageStatistics{}, stats, "expecting empty stats on node 3, but found %v", stats) @@ -178,6 +194,9 @@ func TestStatusAPIIndexUsage(t *testing.T) { require.Equal(t, roachpb.IndexUsageStatistics{}, stats, "expecting empty stats on node 1, but found %v", stats) // Second server should have nonempty local storage. + stats = secondLocalStatsReader.Get(indexKeyPrimary.TableID, indexKeyPrimary.IndexID) + compareStatsHelper(t, expectedStatsIndexPrimary, stats, time.Minute) + stats = secondLocalStatsReader.Get(indexKeyA.TableID, indexKeyA.IndexID) compareStatsHelper(t, expectedStatsIndexA, stats, time.Minute) @@ -197,6 +216,8 @@ func TestStatusAPIIndexUsage(t *testing.T) { } statsEntries++ switch stats.Key.IndexID { + case indexKeyPrimary.IndexID: // t@t_pkey + compareStatsHelper(t, expectedStatsIndexPrimary, stats.Stats, time.Minute) case indexKeyA.IndexID: // t@t_a_idx compareStatsHelper(t, expectedStatsIndexA, stats.Stats, time.Minute) case indexKeyB.IndexID: // t@t_b_idx @@ -204,7 +225,7 @@ func TestStatusAPIIndexUsage(t *testing.T) { } } - require.True(t, statsEntries == 2, "expect to find two stats entries in RPC response, but found %d", statsEntries) + require.Equal(t, 3, statsEntries, "expect to find 3 stats entries in RPC response, but found %d", statsEntries) // Test disabling subsystem. _, err = secondServerSQLConn.Exec("SET CLUSTER SETTING sql.metrics.index_usage_stats.enabled = false") @@ -232,7 +253,7 @@ func TestStatusAPIIndexUsage(t *testing.T) { compareStatsHelper(t, expectedStatsIndexB, stats.Stats, time.Minute) } } - require.True(t, statsEntries == 2, "expect to find two stats entries in RPC response, but found %d", statsEntries) + require.Equal(t, 3, statsEntries, "expect to find 3 stats entries in RPC response, but found %d", statsEntries) } func TestGetTableID(t *testing.T) { diff --git a/pkg/sql/opt_exec_factory.go b/pkg/sql/opt_exec_factory.go index a65b2dd63f9a..77420dd3580d 100644 --- a/pkg/sql/opt_exec_factory.go +++ b/pkg/sql/opt_exec_factory.go @@ -610,9 +610,18 @@ func (ef *execFactory) ConstructIndexJoin( return nil, err } - tableScan.index = tabDesc.GetPrimaryIndex() + idx := tabDesc.GetPrimaryIndex() + tableScan.index = idx tableScan.disableBatchLimit() + if !ef.isExplain { + idxUsageKey := roachpb.IndexUsageKey{ + TableID: roachpb.TableID(tabDesc.GetID()), + IndexID: roachpb.IndexID(idx.GetID()), + } + ef.planner.extendedEvalCtx.indexUsageStats.RecordRead(idxUsageKey) + } + n := &indexJoinNode{ input: input.(planNode), table: tableScan, From f996b3002b1417ae170267ae4aca49bff73cae6d Mon Sep 17 00:00:00 2001 From: Azhng Date: Wed, 23 Mar 2022 21:24:13 +0000 Subject: [PATCH 3/4] sql: new SQL Stats cluster settings to improve write traffic Resolves #78339 Previously, SQL Stats flushed to system table as soon as the in-memory buffer is full. This means the size of the system tables that back SQL Stats could grow faster than the cleanup job. Additionally, when the SQL Stats flush is disabled, the SQL Stats is unable to collect any more new statement / transaction statistics when the in-memory store is full. This commit introduces two non-public cluster settings: * `sql.stats.flush.minimum_interval`: this setting limits minimum interval between each flush operation. If a flush operation is triggered sooner than what is allowed by the minimum interval, (e.g. when the in-memory SQL Stats store is full), the flush operation is aborted. By default this cluster setting is set to 0. * `sql.stats.flush.force_cleanup.enabled`: which allows the in-memory SQL Stats to be cleared at the interval specified by `sql.stats.flush.interval`, even if the SQL Stats flush is disabled. By default, this cluster setting is set to false. This commit also updated the stmt_grouping_in_explicit_txn data driven test to ensure the output order is deterministic. Release note: None --- .../persistedsqlstats/cluster_settings.go | 26 +++ pkg/sql/sqlstats/persistedsqlstats/flush.go | 41 +++- .../sqlstats/persistedsqlstats/flush_test.go | 183 ++++++++++++++++++ .../sqlstats/persistedsqlstats/provider.go | 5 +- .../testdata/stmt_grouping_in_explicit_txn | 5 +- 5 files changed, 250 insertions(+), 10 deletions(-) diff --git a/pkg/sql/sqlstats/persistedsqlstats/cluster_settings.go b/pkg/sql/sqlstats/persistedsqlstats/cluster_settings.go index be17f35183ab..c77119db607b 100644 --- a/pkg/sql/sqlstats/persistedsqlstats/cluster_settings.go +++ b/pkg/sql/sqlstats/persistedsqlstats/cluster_settings.go @@ -29,6 +29,32 @@ var SQLStatsFlushInterval = settings.RegisterDurationSetting( settings.NonNegativeDurationWithMaximum(time.Hour*24), ).WithPublic() +// MinimumInterval is the cluster setting that controls the minimum interval +// between each flush operation. If flush operations get triggered faster +// than what is allowed by this setting, (e.g. when too many fingerprints are +// generated in a short span of time, which in turn cause memory pressure), the +// flush operation will be aborted. +var MinimumInterval = settings.RegisterDurationSetting( + settings.TenantWritable, + "sql.stats.flush.minimum_interval", + "the minimum interval that SQL stats can be flushes to disk. If a "+ + "flush operation starts within less than the minimum interval, the flush "+ + "operation will be aborted", + 0, + settings.NonNegativeDuration, +) + +// DiscardInMemoryStatsWhenFlushDisabled is the cluster setting that allows the +// older in-memory SQL stats to be discarded when flushing to persisted tables +// is disabled. +var DiscardInMemoryStatsWhenFlushDisabled = settings.RegisterBoolSetting( + settings.TenantWritable, + "sql.stats.flush.force_cleanup.enabled", + "if set, older SQL stats are discarded periodically when flushing to "+ + "persisted tables is disabled", + false, +) + // SQLStatsFlushEnabled is the cluster setting that controls if the sqlstats // subsystem persists the statistics into system table. var SQLStatsFlushEnabled = settings.RegisterBoolSetting( diff --git a/pkg/sql/sqlstats/persistedsqlstats/flush.go b/pkg/sql/sqlstats/persistedsqlstats/flush.go index 4d290242b6a4..110f01fc6bbd 100644 --- a/pkg/sql/sqlstats/persistedsqlstats/flush.go +++ b/pkg/sql/sqlstats/persistedsqlstats/flush.go @@ -29,18 +29,49 @@ import ( // Flush flushes in-memory sql stats into system table. Any errors encountered // during the flush will be logged as warning. func (s *PersistedSQLStats) Flush(ctx context.Context) { + now := s.getTimeNow() + + allowDiscardWhenDisabled := DiscardInMemoryStatsWhenFlushDisabled.Get(&s.cfg.Settings.SV) + minimumFlushInterval := MinimumInterval.Get(&s.cfg.Settings.SV) + + enabled := SQLStatsFlushEnabled.Get(&s.cfg.Settings.SV) + flushingTooSoon := now.Before(s.lastFlushStarted.Add(minimumFlushInterval)) + + // Handle wiping in-memory stats here, we only wipe in-memory stats under 2 + // circumstances: + // 1. flush is enabled, and we are not early aborting the flush due to flushing + // too frequently. + // 2. flush is disabled, but we allow discard in-memory stats when disabled. + shouldWipeInMemoryStats := enabled && !flushingTooSoon + shouldWipeInMemoryStats = shouldWipeInMemoryStats || (!enabled && allowDiscardWhenDisabled) + + if shouldWipeInMemoryStats { + defer func() { + if err := s.SQLStats.Reset(ctx); err != nil { + log.Warningf(ctx, "fail to reset in-memory SQL Stats: %s", err) + } + }() + } + + // Handle early abortion of the flush. + if !enabled { + return + } + + if flushingTooSoon { + log.Infof(ctx, "flush aborted due to high flush frequency. "+ + "The minimum interval between flushes is %s", minimumFlushInterval.String()) + return + } + + s.lastFlushStarted = now log.Infof(ctx, "flushing %d stmt/txn fingerprints (%d bytes) after %s", s.SQLStats.GetTotalFingerprintCount(), s.SQLStats.GetTotalFingerprintBytes(), timeutil.Since(s.lastFlushStarted)) aggregatedTs := s.ComputeAggregatedTs() - s.lastFlushStarted = s.getTimeNow() s.flushStmtStats(ctx, aggregatedTs) s.flushTxnStats(ctx, aggregatedTs) - - if err := s.SQLStats.Reset(ctx); err != nil { - log.Warningf(ctx, "fail to reset in-memory SQL Stats: %s", err) - } } func (s *PersistedSQLStats) flushStmtStats(ctx context.Context, aggregatedTs time.Time) { diff --git a/pkg/sql/sqlstats/persistedsqlstats/flush_test.go b/pkg/sql/sqlstats/persistedsqlstats/flush_test.go index 0390cc355702..c2d806e2fc86 100644 --- a/pkg/sql/sqlstats/persistedsqlstats/flush_test.go +++ b/pkg/sql/sqlstats/persistedsqlstats/flush_test.go @@ -243,6 +243,189 @@ func TestSQLStatsInitialDelay(t *testing.T) { "expected latest nextFlushAt to be %s, but found %s", maxNextRunAt, initialNextFlushAt) } +func TestSQLStatsMinimumFlushInterval(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + ctx := context.Background() + fakeTime := stubTime{ + aggInterval: time.Hour, + } + fakeTime.setTime(timeutil.Now()) + + params, _ := tests.CreateTestServerParams() + params.Knobs.SQLStatsKnobs = &sqlstats.TestingKnobs{ + StubTimeNow: fakeTime.Now, + } + s, conn, _ := serverutils.StartServer(t, params) + + defer s.Stopper().Stop(context.Background()) + + sqlConn := sqlutils.MakeSQLRunner(conn) + + sqlConn.Exec(t, "SET CLUSTER SETTING sql.stats.flush.minimum_interval = '10m'") + sqlConn.Exec(t, "SET application_name = 'min_flush_test'") + sqlConn.Exec(t, "SELECT 1") + + s.SQLServer().(*sql.Server). + GetSQLStatsProvider().(*persistedsqlstats.PersistedSQLStats).Flush(ctx) + + sqlConn.CheckQueryResults(t, ` + SELECT count(*) + FROM system.statement_statistics + WHERE app_name = 'min_flush_test' + `, [][]string{{"1"}}) + + sqlConn.CheckQueryResults(t, ` + SELECT count(*) + FROM system.transaction_statistics + WHERE app_name = 'min_flush_test' + `, [][]string{{"1"}}) + + // Since by default, the minimum flush interval is 10 minutes, a subsequent + // flush should be no-op. + s.SQLServer().(*sql.Server). + GetSQLStatsProvider().(*persistedsqlstats.PersistedSQLStats).Flush(ctx) + + sqlConn.CheckQueryResults(t, ` + SELECT count(*) + FROM system.statement_statistics + WHERE app_name = 'min_flush_test' + `, [][]string{{"1"}}) + + sqlConn.CheckQueryResults(t, ` + SELECT count(*) + FROM system.transaction_statistics + WHERE app_name = 'min_flush_test' + `, [][]string{{"1"}}) + + // We manually set the time to past the minimum flush interval, now the flush + // should succeed. + fakeTime.setTime(fakeTime.Now().Add(time.Hour)) + + s.SQLServer().(*sql.Server). + GetSQLStatsProvider().(*persistedsqlstats.PersistedSQLStats).Flush(ctx) + + sqlConn.CheckQueryResults(t, ` + SELECT count(*) > 1 + FROM system.statement_statistics + WHERE app_name = 'min_flush_test' + `, [][]string{{"true"}}) + +} + +func TestInMemoryStatsDiscard(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + ctx := context.Background() + params, _ := tests.CreateTestServerParams() + s, conn, _ := serverutils.StartServer(t, params) + observer := + serverutils.OpenDBConn(t, s.ServingSQLAddr(), "", false /* insecure */, s.Stopper()) + + defer s.Stopper().Stop(context.Background()) + + sqlConn := sqlutils.MakeSQLRunner(conn) + sqlConn.Exec(t, + "SET CLUSTER SETTING sql.stats.flush.minimum_interval = '10m'") + observerConn := sqlutils.MakeSQLRunner(observer) + + t.Run("flush_disabled", func(t *testing.T) { + sqlConn.Exec(t, + "SET CLUSTER SETTING sql.stats.flush.force_cleanup.enabled = true") + sqlConn.Exec(t, + "SET CLUSTER SETTING sql.stats.flush.enabled = false") + + sqlConn.Exec(t, "SET application_name = 'flush_disabled_test'") + sqlConn.Exec(t, "SELECT 1") + + observerConn.CheckQueryResults(t, ` + SELECT count(*) + FROM crdb_internal.statement_statistics + WHERE app_name = 'flush_disabled_test' + `, [][]string{{"1"}}) + + s.SQLServer().(*sql.Server). + GetSQLStatsProvider().(*persistedsqlstats.PersistedSQLStats).Flush(ctx) + + observerConn.CheckQueryResults(t, ` + SELECT count(*) + FROM crdb_internal.statement_statistics + WHERE app_name = 'flush_disabled_test' + `, [][]string{{"0"}}) + }) + + t.Run("flush_enabled", func(t *testing.T) { + // Now turn back SQL Stats flush. If the flush is aborted due to violating + // minimum flush interval constraint, we should not be clearing in-memory + // stats. + sqlConn.Exec(t, + "SET CLUSTER SETTING sql.stats.flush.enabled = true") + sqlConn.Exec(t, "SET application_name = 'flush_enabled_test'") + sqlConn.Exec(t, "SELECT 1") + + observerConn.CheckQueryResults(t, ` + SELECT count(*) + FROM crdb_internal.statement_statistics + WHERE app_name = 'flush_enabled_test' + `, [][]string{{"1"}}) + + observerConn.CheckQueryResults(t, ` + SELECT count(*) + FROM crdb_internal.transaction_statistics + WHERE app_name = 'flush_enabled_test' + `, [][]string{{"1"}}) + + // First flush should flush everything into the system tables. + s.SQLServer().(*sql.Server). + GetSQLStatsProvider().(*persistedsqlstats.PersistedSQLStats).Flush(ctx) + + observerConn.CheckQueryResults(t, ` + SELECT count(*) + FROM system.statement_statistics + WHERE app_name = 'flush_enabled_test' + `, [][]string{{"1"}}) + + observerConn.CheckQueryResults(t, ` + SELECT count(*) + FROM system.transaction_statistics + WHERE app_name = 'flush_enabled_test' + `, [][]string{{"1"}}) + + sqlConn.Exec(t, "SELECT 1,1") + + // Second flush should be aborted due to violating the minimum flush + // interval requirement. Though the data should still remain in-memory. + s.SQLServer().(*sql.Server). + GetSQLStatsProvider().(*persistedsqlstats.PersistedSQLStats).Flush(ctx) + + observerConn.CheckQueryResults(t, ` + SELECT count(*) + FROM system.statement_statistics + WHERE app_name = 'flush_enabled_test' + `, [][]string{{"1"}}) + + observerConn.CheckQueryResults(t, ` + SELECT count(*) + FROM system.transaction_statistics + WHERE app_name = 'flush_enabled_test' + `, [][]string{{"1"}}) + + observerConn.CheckQueryResults(t, ` + SELECT count(*) + FROM crdb_internal.statement_statistics + WHERE app_name = 'flush_enabled_test' + `, [][]string{{"2"}}) + + observerConn.CheckQueryResults(t, ` + SELECT count(*) + FROM crdb_internal.transaction_statistics + WHERE app_name = 'flush_enabled_test' + `, [][]string{{"2"}}) + }) +} + type stubTime struct { syncutil.RWMutex t time.Time diff --git a/pkg/sql/sqlstats/persistedsqlstats/provider.go b/pkg/sql/sqlstats/persistedsqlstats/provider.go index 0ea6b20c8107..bc4ef152a621 100644 --- a/pkg/sql/sqlstats/persistedsqlstats/provider.go +++ b/pkg/sql/sqlstats/persistedsqlstats/provider.go @@ -141,10 +141,7 @@ func (s *PersistedSQLStats) startSQLStatsFlushLoop(ctx context.Context, stopper return } - enabled := SQLStatsFlushEnabled.Get(&s.cfg.Settings.SV) - if enabled { - s.Flush(ctx) - } + s.Flush(ctx) } }) } diff --git a/pkg/sql/sqlstats/persistedsqlstats/testdata/stmt_grouping_in_explicit_txn b/pkg/sql/sqlstats/persistedsqlstats/testdata/stmt_grouping_in_explicit_txn index 834fe91b184c..321e190378e8 100644 --- a/pkg/sql/sqlstats/persistedsqlstats/testdata/stmt_grouping_in_explicit_txn +++ b/pkg/sql/sqlstats/persistedsqlstats/testdata/stmt_grouping_in_explicit_txn @@ -170,6 +170,9 @@ WHERE WHERE jsonb_array_length(metadata -> 'stmtFingerprintIDs') >= 3 ) +ORDER BY + (statistics -> 'statistics' ->> 'cnt')::INT +ASC ---- -df3c70bf7729b433,705fcdf3f12803ec,SELECT _,2 15c74af7a18bd6da,baa4f7bb278a6105,SELECT _, _,1 +df3c70bf7729b433,705fcdf3f12803ec,SELECT _,2 From c345b54af71c7525c3eb44b41f4bd78d5063ec5f Mon Sep 17 00:00:00 2001 From: Ricky Stewart Date: Fri, 25 Mar 2022 16:00:31 -0500 Subject: [PATCH 4/4] ci: add scripts for bazel-based `ui test`, `ui lint` ci jobs These scripts just do the same logic that `dev` does for the same functions. Release note: None --- build/README.md | 2 +- build/bazelbuilder/Dockerfile | 12 ++++++++++++ build/teamcity-bazel-support.sh | 2 +- build/teamcity/cockroach/ci/tests/ui_lint.sh | 12 ++++++++++++ build/teamcity/cockroach/ci/tests/ui_lint_impl.sh | 6 ++++++ build/teamcity/cockroach/ci/tests/ui_test.sh | 12 ++++++++++++ build/teamcity/cockroach/ci/tests/ui_test_impl.sh | 6 ++++++ pkg/cmd/bazci/bazci.go | 2 ++ 8 files changed, 52 insertions(+), 2 deletions(-) create mode 100755 build/teamcity/cockroach/ci/tests/ui_lint.sh create mode 100755 build/teamcity/cockroach/ci/tests/ui_lint_impl.sh create mode 100755 build/teamcity/cockroach/ci/tests/ui_test.sh create mode 100755 build/teamcity/cockroach/ci/tests/ui_test_impl.sh diff --git a/build/README.md b/build/README.md index 12470ef5d224..32b666da8c3e 100644 --- a/build/README.md +++ b/build/README.md @@ -144,7 +144,7 @@ The `bazelbuilder` image is used exclusively for performing builds using Bazel. docker manifest push cockroachdb/bazel:$TAG ``` - Then, update `build/teamcity-bazel-support.sh` with the new tag and commit all your changes. -- Ensure the "GitHub CI (Optional)" job passes on your PR before merging. +- Ensure the "Bazel CI" job passes on your PR before merging. # Dependencies diff --git a/build/bazelbuilder/Dockerfile b/build/bazelbuilder/Dockerfile index 5fbe29f736ee..32d799042019 100644 --- a/build/bazelbuilder/Dockerfile +++ b/build/bazelbuilder/Dockerfile @@ -64,6 +64,18 @@ RUN curl -fsSL https://packages.cloud.google.com/apt/doc/apt-key.gpg | apt-key a && DEBIAN_FRONTEND=noninteractive apt-get install -y --no-install-recommends \ google-cloud-sdk +# chrome is needed for UI tests. Unfortunately it is only distributed for x86_64 +# and not for ARM. Chrome shouldn't need to be installed when we migrate away +# from Karma for UI testing. +RUN case ${TARGETPLATFORM} in \ + "linux/amd64") \ + curl -fsSL https://dl-ssl.google.com/linux/linux_signing_key.pub | apt-key add - \ + && echo "deb [arch=amd64] https://dl.google.com/linux/chrome/deb/ stable main" | tee /etc/apt/sources.list.d/google.list \ + && apt-get update \ + && DEBIAN_FRONTEND=noninteractive apt-get install -y --no-install-recommends \ + google-chrome-stable ;; \ + esac + RUN apt-get purge -y \ apt-transport-https \ flex \ diff --git a/build/teamcity-bazel-support.sh b/build/teamcity-bazel-support.sh index f423a7a47a45..6ca3d2f2e835 100644 --- a/build/teamcity-bazel-support.sh +++ b/build/teamcity-bazel-support.sh @@ -1,7 +1,7 @@ # FYI: You can run `./dev builder` to run this Docker image. :) # `dev` depends on this variable! Don't change the name or format unless you # also update `dev` accordingly. -BAZEL_IMAGE=cockroachdb/bazel:20220121-121551 +BAZEL_IMAGE=cockroachdb/bazel:20220328-163955 # Call `run_bazel $NAME_OF_SCRIPT` to start an appropriately-configured Docker # container with the `cockroachdb/bazel` image running the given script. diff --git a/build/teamcity/cockroach/ci/tests/ui_lint.sh b/build/teamcity/cockroach/ci/tests/ui_lint.sh new file mode 100755 index 000000000000..4215b7bc60f7 --- /dev/null +++ b/build/teamcity/cockroach/ci/tests/ui_lint.sh @@ -0,0 +1,12 @@ +#!/usr/bin/env bash + +set -euo pipefail + +dir="$(dirname $(dirname $(dirname $(dirname $(dirname "${0}")))))" + +source "$dir/teamcity-support.sh" # For $root +source "$dir/teamcity-bazel-support.sh" # For run_bazel + +tc_start_block "Run UI tests" +run_bazel build/teamcity/cockroach/ci/tests/ui_lint_impl.sh +tc_end_block "Run UI tests" diff --git a/build/teamcity/cockroach/ci/tests/ui_lint_impl.sh b/build/teamcity/cockroach/ci/tests/ui_lint_impl.sh new file mode 100755 index 000000000000..9948b92e75eb --- /dev/null +++ b/build/teamcity/cockroach/ci/tests/ui_lint_impl.sh @@ -0,0 +1,6 @@ +#!/usr/bin/env bash + +set -xeuo pipefail + +bazel build //pkg/cmd/bazci --config=ci +$(bazel info bazel-bin --config=ci)/pkg/cmd/bazci/bazci_/bazci test --config=ci //pkg/ui:lint diff --git a/build/teamcity/cockroach/ci/tests/ui_test.sh b/build/teamcity/cockroach/ci/tests/ui_test.sh new file mode 100755 index 000000000000..cc6b0984f1ba --- /dev/null +++ b/build/teamcity/cockroach/ci/tests/ui_test.sh @@ -0,0 +1,12 @@ +#!/usr/bin/env bash + +set -euo pipefail + +dir="$(dirname $(dirname $(dirname $(dirname $(dirname "${0}")))))" + +source "$dir/teamcity-support.sh" # For $root +source "$dir/teamcity-bazel-support.sh" # For run_bazel + +tc_start_block "Run UI tests" +run_bazel build/teamcity/cockroach/ci/tests/ui_test_impl.sh +tc_end_block "Run UI tests" diff --git a/build/teamcity/cockroach/ci/tests/ui_test_impl.sh b/build/teamcity/cockroach/ci/tests/ui_test_impl.sh new file mode 100755 index 000000000000..b59247cef58a --- /dev/null +++ b/build/teamcity/cockroach/ci/tests/ui_test_impl.sh @@ -0,0 +1,6 @@ +#!/usr/bin/env bash + +set -xeuo pipefail + +bazel build //pkg/cmd/bazci --config=ci +$(bazel info bazel-bin --config=ci)/pkg/cmd/bazci/bazci_/bazci test --config=ci //pkg/ui/workspaces/db-console:karma //pkg/ui/workspaces/cluster-ui:jest diff --git a/pkg/cmd/bazci/bazci.go b/pkg/cmd/bazci/bazci.go index fc70716f3b06..a40711b99252 100644 --- a/pkg/cmd/bazci/bazci.go +++ b/pkg/cmd/bazci/bazci.go @@ -229,6 +229,8 @@ func getBuildInfo(args parsedArgs) (buildInfo, error) { // to replace (it's the output directory for the configuration). componentsTestlogs[len(componentsTestlogs)-2] = componentsBinLocation[len(componentsTestlogs)-2] ret.transitionTests[fullTarget] = strings.Join(componentsTestlogs, "/") + case "nodejs_test": + ret.tests = append(ret.tests, fullTarget) case "test_suite": // Expand the list of tests from the test suite with another query. allTests, err := runBazelReturningStdout("query", "tests("+fullTarget+")")