diff --git a/pkg/ccl/backupccl/backup.go b/pkg/ccl/backupccl/backup.go index 6c7d0468f96b..f7acb24dc6c6 100644 --- a/pkg/ccl/backupccl/backup.go +++ b/pkg/ccl/backupccl/backup.go @@ -854,7 +854,7 @@ func backupPlanHook( endTime := p.ExecCfg().Clock.Now() if backupStmt.AsOf.Expr != nil { var err error - if endTime, err = p.EvalAsOfTimestamp(backupStmt.AsOf, endTime); err != nil { + if endTime, err = p.EvalAsOfTimestamp(backupStmt.AsOf); err != nil { return err } } diff --git a/pkg/ccl/backupccl/backup_test.go b/pkg/ccl/backupccl/backup_test.go index 62ff615cfdc0..0e06ba65681d 100644 --- a/pkg/ccl/backupccl/backup_test.go +++ b/pkg/ccl/backupccl/backup_test.go @@ -2801,7 +2801,7 @@ func TestCreateStatsAfterRestore(t *testing.T) { stats.DefaultAsOfTime = oldAsOf }(stats.DefaultRefreshInterval, stats.DefaultAsOfTime) stats.DefaultRefreshInterval = time.Millisecond - stats.DefaultAsOfTime = 0 + stats.DefaultAsOfTime = time.Microsecond const numAccounts = 1 _, _, sqlDB, _, cleanupFn := backupRestoreTestSetup(t, singleNode, numAccounts, initNone) diff --git a/pkg/ccl/backupccl/restore.go b/pkg/ccl/backupccl/restore.go index 963115768ca8..6bf7ccdf0bc5 100644 --- a/pkg/ccl/backupccl/restore.go +++ b/pkg/ccl/backupccl/restore.go @@ -1293,10 +1293,8 @@ func restorePlanHook( } var endTime hlc.Timestamp if restoreStmt.AsOf.Expr != nil { - // Use Now() for the max timestamp because Restore does its own - // (more restrictive) check. var err error - endTime, err = p.EvalAsOfTimestamp(restoreStmt.AsOf, p.ExecCfg().Clock.Now()) + endTime, err = p.EvalAsOfTimestamp(restoreStmt.AsOf) if err != nil { return err } diff --git a/pkg/ccl/changefeedccl/changefeed_stmt.go b/pkg/ccl/changefeedccl/changefeed_stmt.go index b05a33130795..a02ffa1f4388 100644 --- a/pkg/ccl/changefeedccl/changefeed_stmt.go +++ b/pkg/ccl/changefeedccl/changefeed_stmt.go @@ -146,12 +146,14 @@ func changefeedPlanHook( jobDescription := changefeedJobDescription(changefeedStmt, sinkURI, opts) - statementTime := p.ExecCfg().Clock.Now() + statementTime := hlc.Timestamp{ + WallTime: p.ExtendedEvalContext().GetStmtTimestamp().UnixNano(), + } var initialHighWater hlc.Timestamp if cursor, ok := opts[optCursor]; ok { asOf := tree.AsOfClause{Expr: tree.NewStrVal(cursor)} var err error - if initialHighWater, err = p.EvalAsOfTimestamp(asOf, statementTime); err != nil { + if initialHighWater, err = p.EvalAsOfTimestamp(asOf); err != nil { return err } statementTime = initialHighWater diff --git a/pkg/ccl/importccl/import_stmt_test.go b/pkg/ccl/importccl/import_stmt_test.go index 47b1f214115f..2c4c185ce4ca 100644 --- a/pkg/ccl/importccl/import_stmt_test.go +++ b/pkg/ccl/importccl/import_stmt_test.go @@ -2421,7 +2421,7 @@ func TestCreateStatsAfterImport(t *testing.T) { stats.DefaultAsOfTime = oldAsOf }(stats.DefaultRefreshInterval, stats.DefaultAsOfTime) stats.DefaultRefreshInterval = time.Millisecond - stats.DefaultAsOfTime = 0 + stats.DefaultAsOfTime = time.Microsecond const nodes = 1 ctx := context.Background() diff --git a/pkg/sql/conn_executor.go b/pkg/sql/conn_executor.go index 0e6f6a0a4d8e..9249782ad832 100644 --- a/pkg/sql/conn_executor.go +++ b/pkg/sql/conn_executor.go @@ -1743,8 +1743,8 @@ func (ex *connExecutor) setTransactionModes(modes tree.TransactionModes) error { } rwMode := modes.ReadWriteMode if modes.AsOf.Expr != nil { - now := ex.server.cfg.Clock.Now() - ts, err := ex.planner.EvalAsOfTimestamp(modes.AsOf, now) + + ts, err := ex.planner.EvalAsOfTimestamp(modes.AsOf) if err != nil { ex.state.mu.Unlock() return err diff --git a/pkg/sql/conn_executor_exec.go b/pkg/sql/conn_executor_exec.go index 585a23b8bfcb..32e1257e2522 100644 --- a/pkg/sql/conn_executor_exec.go +++ b/pkg/sql/conn_executor_exec.go @@ -371,7 +371,7 @@ func (ex *connExecutor) execStmtInOpenState( } if os.ImplicitTxn.Get() { - asOfTs, err := p.isAsOf(stmt.AST, ex.server.cfg.Clock.Now()) + asOfTs, err := p.isAsOf(stmt.AST) if err != nil { return makeErrEvent(err) } @@ -385,7 +385,7 @@ func (ex *connExecutor) execStmtInOpenState( // the transaction's timestamp. This is useful for running AOST statements // using the InternalExecutor inside an external transaction; one might want // to do that to force p.avoidCachedDescriptors to be set below. - ts, err := p.isAsOf(stmt.AST, ex.server.cfg.Clock.Now()) + ts, err := p.isAsOf(stmt.AST) if err != nil { return makeErrEvent(err) } @@ -1070,7 +1070,7 @@ func (ex *connExecutor) beginTransactionTimestampsAndReadMode( } p := &ex.planner ex.resetPlanner(ctx, p, nil /* txn */, now.GoTime()) - ts, err := p.EvalAsOfTimestamp(s.Modes.AsOf, now) + ts, err := p.EvalAsOfTimestamp(s.Modes.AsOf) if err != nil { return 0, time.Time{}, nil, err } diff --git a/pkg/sql/conn_executor_prepare.go b/pkg/sql/conn_executor_prepare.go index a3c2e9c4c1e7..0e3a7f4f581c 100644 --- a/pkg/sql/conn_executor_prepare.go +++ b/pkg/sql/conn_executor_prepare.go @@ -197,7 +197,7 @@ func (ex *connExecutor) populatePrepared( p.extendedEvalCtx.ActiveMemAcc = &constantMemAcc defer constantMemAcc.Close(ctx) - protoTS, err := p.isAsOf(stmt.AST, ex.server.cfg.Clock.Now() /* max */) + protoTS, err := p.isAsOf(stmt.AST) if err != nil { return 0, err } diff --git a/pkg/sql/exec_util.go b/pkg/sql/exec_util.go index 364effe677f6..26f33d73a38c 100644 --- a/pkg/sql/exec_util.go +++ b/pkg/sql/exec_util.go @@ -671,10 +671,8 @@ func checkResultType(typ types.T) error { // EvalAsOfTimestamp evaluates and returns the timestamp from an AS OF SYSTEM // TIME clause. -func (p *planner) EvalAsOfTimestamp( - asOf tree.AsOfClause, max hlc.Timestamp, -) (hlc.Timestamp, error) { - return tree.EvalAsOfTimestamp(asOf, max, &p.semaCtx, p.EvalContext()) +func (p *planner) EvalAsOfTimestamp(asOf tree.AsOfClause) (_ hlc.Timestamp, err error) { + return tree.EvalAsOfTimestamp(asOf, &p.semaCtx, p.EvalContext()) } // ParseHLC parses a string representation of an `hlc.Timestamp`. @@ -691,10 +689,7 @@ func ParseHLC(s string) (hlc.Timestamp, error) { // timestamp is not nil, it is the timestamp to which a transaction // should be set. The statements that will be checked are Select, // ShowTrace (of a Select statement), Scrub, Export, and CreateStats. -// -// max is a lower bound on what the transaction's timestamp will be. -// Used to check that the user didn't specify a timestamp in the future. -func (p *planner) isAsOf(stmt tree.Statement, max hlc.Timestamp) (*hlc.Timestamp, error) { +func (p *planner) isAsOf(stmt tree.Statement) (*hlc.Timestamp, error) { var asOf tree.AsOfClause switch s := stmt.(type) { case *tree.Select: @@ -720,7 +715,7 @@ func (p *planner) isAsOf(stmt tree.Statement, max hlc.Timestamp) (*hlc.Timestamp } asOf = s.AsOf case *tree.Export: - return p.isAsOf(s.Query, max) + return p.isAsOf(s.Query) case *tree.CreateStats: if s.AsOf.Expr == nil { return nil, nil @@ -729,7 +724,7 @@ func (p *planner) isAsOf(stmt tree.Statement, max hlc.Timestamp) (*hlc.Timestamp default: return nil, nil } - ts, err := p.EvalAsOfTimestamp(asOf, max) + ts, err := p.EvalAsOfTimestamp(asOf) return &ts, err } diff --git a/pkg/sql/logictest/logic.go b/pkg/sql/logictest/logic.go index 805b55801458..43e23f5c8e01 100644 --- a/pkg/sql/logictest/logic.go +++ b/pkg/sql/logictest/logic.go @@ -1025,7 +1025,7 @@ func (t *logicTest) setup(cfg testClusterConfig) { } // Update the defaults for automatic statistics to avoid delays in testing. - stats.DefaultAsOfTime = 0 + stats.DefaultAsOfTime = time.Microsecond stats.DefaultRefreshInterval = time.Millisecond t.cluster = serverutils.StartTestCluster(t.t, cfg.numNodes, params) diff --git a/pkg/sql/logictest/testdata/logic_test/as_of b/pkg/sql/logictest/testdata/logic_test/as_of index f58bb51d89a9..7376ddefa242 100644 --- a/pkg/sql/logictest/testdata/logic_test/as_of +++ b/pkg/sql/logictest/testdata/logic_test/as_of @@ -8,24 +8,19 @@ INSERT INTO t VALUES (2) # Verify strings can be parsed as intervals. query I -SELECT * FROM t AS OF SYSTEM TIME '-0ns' ----- -2 - -query I -SELECT * FROM t AS OF SYSTEM TIME '-1ns' +SELECT * FROM t AS OF SYSTEM TIME '-1000ns' ---- 2 # Verify a forced interval type works. query I -SELECT * FROM t AS OF SYSTEM TIME INTERVAL '-1ns' +SELECT * FROM t AS OF SYSTEM TIME INTERVAL '-1000ns' ---- 2 # Verify that we can use computed expressions. query I -SELECT * FROM t AS OF SYSTEM TIME -( ('1' || 'ns')::INTERVAL ) +SELECT * FROM t AS OF SYSTEM TIME -( ('1000' || 'ns')::INTERVAL ) ---- 2 @@ -57,3 +52,20 @@ query T SELECT * FROM (SELECT now()) AS OF SYSTEM TIME '2018-01-01' ---- 2018-01-01 00:00:00 +0000 UTC + +# Verify that zero intervals those indistinguishable from zero cause an error. + +statement error pq: AS OF SYSTEM TIME: interval value '100ns' too small, must be <= -1µs +SELECT * FROM t AS OF SYSTEM TIME '100ns' + +statement error pq: AS OF SYSTEM TIME: interval value '0,0' too small, must be <= -1µs +SELECT * FROM t AS OF SYSTEM TIME '0,0' + +statement error pq: AS OF SYSTEM TIME: interval value '0.000000000,0' too small, must be <= -1µs +SELECT * FROM t AS OF SYSTEM TIME '0.000000000,0' + +statement error pq: AS OF SYSTEM TIME: interval value '-100ns' too small, must be <= -1µs +SELECT * FROM t AS OF SYSTEM TIME '-100ns' + +statement error pq: AS OF SYSTEM TIME: zero timestamp is invalid +SELECT * FROM t AS OF SYSTEM TIME '0' diff --git a/pkg/sql/logictest/testdata/logic_test/distsql_stats b/pkg/sql/logictest/testdata/logic_test/distsql_stats index 2491fd8093e9..28c65ae001e5 100644 --- a/pkg/sql/logictest/testdata/logic_test/distsql_stats +++ b/pkg/sql/logictest/testdata/logic_test/distsql_stats @@ -126,7 +126,7 @@ statement error pgcode 42P01 relation "data" does not exist CREATE STATISTICS s2 ON a FROM data AS OF SYSTEM TIME '2017' statement ok -CREATE STATISTICS s2 ON a FROM data AS OF SYSTEM TIME '-1ns' +CREATE STATISTICS s2 ON a FROM data AS OF SYSTEM TIME '-1000ns' query TTIII colnames SELECT statistics_name, column_names, row_count, distinct_count, null_count FROM [SHOW STATISTICS FOR TABLE data] diff --git a/pkg/sql/logictest/testdata/logic_test/subquery_correlated b/pkg/sql/logictest/testdata/logic_test/subquery_correlated index 95a82359e048..73ce6bc56cac 100644 --- a/pkg/sql/logictest/testdata/logic_test/subquery_correlated +++ b/pkg/sql/logictest/testdata/logic_test/subquery_correlated @@ -936,7 +936,7 @@ INNER JOIN o ON c.c_id=o.c_id AND o.ship = (SELECT o.ship FROM o WHERE o.c_id=c.c_id); statement error AS OF SYSTEM TIME must be provided on a top-level statement -SELECT (SELECT c_id FROM o AS OF SYSTEM TIME '-0ns') +SELECT (SELECT c_id FROM o AS OF SYSTEM TIME '-1000ns') FROM c WHERE EXISTS(SELECT * FROM o WHERE o.c_id=c.c_id) diff --git a/pkg/sql/logictest/testdata/logic_test/txn_as_of b/pkg/sql/logictest/testdata/logic_test/txn_as_of index c154d851949d..1a3045684f79 100644 --- a/pkg/sql/logictest/testdata/logic_test/txn_as_of +++ b/pkg/sql/logictest/testdata/logic_test/txn_as_of @@ -22,7 +22,7 @@ statement ok COMMIT statement ok -BEGIN AS OF SYSTEM TIME '-1ns' +BEGIN AS OF SYSTEM TIME '-1000ns' query I SELECT * FROM t @@ -33,7 +33,7 @@ statement ok COMMIT statement ok -BEGIN; SET TRANSACTION AS OF SYSTEM TIME '-1ns' +BEGIN; SET TRANSACTION AS OF SYSTEM TIME '-1000ns' query I SELECT * FROM t @@ -76,7 +76,7 @@ COMMIT # Verify transactions with a historical timestamps imply READ ONLY. statement ok -BEGIN; SET TRANSACTION AS OF SYSTEM TIME '-1ns' +BEGIN; SET TRANSACTION AS OF SYSTEM TIME '-1000ns' statement error pq: cannot execute INSERT in a read-only transaction INSERT INTO t VALUES (3) @@ -85,7 +85,7 @@ statement ok COMMIT statement ok -BEGIN AS OF SYSTEM TIME '-1ns' +BEGIN AS OF SYSTEM TIME '-1000ns' statement error pq: cannot execute INSERT in a read-only transaction INSERT INTO t VALUES (3) @@ -97,7 +97,7 @@ COMMIT # the previous value. statement ok -BEGIN AS OF SYSTEM TIME '-1h'; SET TRANSACTION AS OF SYSTEM TIME '-1ns'; +BEGIN AS OF SYSTEM TIME '-1h'; SET TRANSACTION AS OF SYSTEM TIME '-1000ns'; query I SELECT * FROM t diff --git a/pkg/sql/opt/optbuilder/select.go b/pkg/sql/opt/optbuilder/select.go index 9cce5e731a84..619b0637a9ea 100644 --- a/pkg/sql/opt/optbuilder/select.go +++ b/pkg/sql/opt/optbuilder/select.go @@ -25,7 +25,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/privilege" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sem/types" - "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/pkg/errors" ) @@ -757,7 +756,7 @@ func (b *Builder) buildFromTables(tables tree.TableExprs, inScope *scope) (outSc // validateAsOf ensures that any AS OF SYSTEM TIME timestamp is consistent with // that of the root statement. func (b *Builder) validateAsOf(asOf tree.AsOfClause) { - ts, err := tree.EvalAsOfTimestamp(asOf, hlc.MaxTimestamp, b.semaCtx, b.evalCtx) + ts, err := tree.EvalAsOfTimestamp(asOf, b.semaCtx, b.evalCtx) if err != nil { panic(builderError{err}) } diff --git a/pkg/sql/opt/optbuilder/testdata/select b/pkg/sql/opt/optbuilder/testdata/select index 9b8bd9b0624b..4613768b50ea 100644 --- a/pkg/sql/opt/optbuilder/testdata/select +++ b/pkg/sql/opt/optbuilder/testdata/select @@ -1125,7 +1125,7 @@ select # This is slightly funky, because the AS OF SYSTEM TIME timestamp only gets # interpreted by the executor, which obviously is not at play in these tests. build -SELECT * FROM a AS OF SYSTEM TIME '10ns' +SELECT * FROM a AS OF SYSTEM TIME '-1000ns' ---- error: AS OF SYSTEM TIME must be provided on a top-level statement diff --git a/pkg/sql/planhook.go b/pkg/sql/planhook.go index ef0cdd972dbf..6a6adb37618c 100644 --- a/pkg/sql/planhook.go +++ b/pkg/sql/planhook.go @@ -91,7 +91,7 @@ type PlanHookState interface { GetAllUsersAndRoles(ctx context.Context) (map[string]bool, error) BumpRoleMembershipTableVersion(ctx context.Context) error Select(ctx context.Context, n *tree.Select, desiredTypes []types.T) (planNode, error) - EvalAsOfTimestamp(asOf tree.AsOfClause, max hlc.Timestamp) (hlc.Timestamp, error) + EvalAsOfTimestamp(asOf tree.AsOfClause) (hlc.Timestamp, error) ResolveUncachedDatabaseByName( ctx context.Context, dbName string, required bool) (*UncachedDatabaseDescriptor, error) } diff --git a/pkg/sql/render.go b/pkg/sql/render.go index f433fe494df8..c3165264978e 100644 --- a/pkg/sql/render.go +++ b/pkg/sql/render.go @@ -501,7 +501,7 @@ func (p *planner) getTimestamp(asOf tree.AsOfClause) (hlc.Timestamp, bool, error // level. We accept AS OF SYSTEM TIME in multiple places (e.g. in // subqueries or view queries) but they must all point to the same // timestamp. - ts, err := p.EvalAsOfTimestamp(asOf, hlc.MaxTimestamp) + ts, err := p.EvalAsOfTimestamp(asOf) if err != nil { return hlc.MaxTimestamp, false, err } diff --git a/pkg/sql/sem/tree/as_of.go b/pkg/sql/sem/tree/as_of.go index 7677f6d23a9a..c0c4fffc35c3 100644 --- a/pkg/sql/sem/tree/as_of.go +++ b/pkg/sql/sem/tree/as_of.go @@ -36,8 +36,8 @@ var errInvalidExprForAsOf = errors.Errorf("AS OF SYSTEM TIME: only constant expr // EvalAsOfTimestamp evaluates the timestamp argument to an AS OF SYSTEM TIME query. func EvalAsOfTimestamp( - asOf AsOfClause, max hlc.Timestamp, semaCtx *SemaContext, evalCtx *EvalContext, -) (hlc.Timestamp, error) { + asOf AsOfClause, semaCtx *SemaContext, evalCtx *EvalContext, +) (tsss hlc.Timestamp, err error) { // We need to save and restore the previous value of the field in // semaCtx in case we are recursively called within a subquery // context. @@ -80,7 +80,7 @@ func EvalAsOfTimestamp( var ts hlc.Timestamp var convErr error - + stmtTimestamp := evalCtx.GetStmtTimestamp() switch d := d.(type) { case *DString: s := string(*d) @@ -97,7 +97,11 @@ func EvalAsOfTimestamp( } // Attempt to parse as an interval. if iv, err := ParseDInterval(s); err == nil { - ts.WallTime = duration.Add(evalCtx, evalCtx.GetStmtTimestamp(), iv.Duration).UnixNano() + if (iv.Duration == duration.Duration{}) { + convErr = errors.Errorf("AS OF SYSTEM TIME: interval value %v too small, must be <= %v", te, -1*time.Microsecond) + } else { + ts.WallTime = duration.Add(evalCtx, stmtTimestamp, iv.Duration).UnixNano() + } break } convErr = errors.Errorf("AS OF SYSTEM TIME: value is neither timestamp, decimal, nor interval") @@ -108,7 +112,7 @@ func EvalAsOfTimestamp( case *DDecimal: ts, convErr = DecimalToHLC(&d.Decimal) case *DInterval: - ts.WallTime = duration.Add(evalCtx, evalCtx.GetStmtTimestamp(), d.Duration).UnixNano() + ts.WallTime = duration.Add(evalCtx, stmtTimestamp, d.Duration).UnixNano() default: convErr = errors.Errorf("AS OF SYSTEM TIME: expected timestamp, decimal, or interval, got %s (%T)", d.ResolvedType(), d) } @@ -121,7 +125,7 @@ func EvalAsOfTimestamp( return ts, errors.Errorf("AS OF SYSTEM TIME: zero timestamp is invalid") } else if ts.Less(zero) { return ts, errors.Errorf("AS OF SYSTEM TIME: timestamp before 1970-01-01T00:00:00Z is invalid") - } else if max.Less(ts) { + } else if stmtTimestamp.Before(ts.GoTime()) { return ts, errors.Errorf("AS OF SYSTEM TIME: cannot specify timestamp in the future") } return ts, nil diff --git a/pkg/sql/stats/automatic_stats_test.go b/pkg/sql/stats/automatic_stats_test.go index f4c5096299b6..2c904b8d02ce 100644 --- a/pkg/sql/stats/automatic_stats_test.go +++ b/pkg/sql/stats/automatic_stats_test.go @@ -54,7 +54,7 @@ func TestMaybeRefreshStats(t *testing.T) { executor := s.InternalExecutor().(sqlutil.InternalExecutor) descA := sqlbase.GetTableDescriptor(s.DB(), "t", "a") cache := NewTableStatisticsCache(10 /* cacheSize */, s.Gossip(), kvDB, executor) - refresher := MakeRefresher(executor, cache, 0 /* asOfTime */) + refresher := MakeRefresher(executor, cache, time.Microsecond /* asOfTime */) // There should not be any stats yet. if err := checkStatsCount(ctx, cache, descA.ID, 0 /* expected */); err != nil { @@ -63,14 +63,14 @@ func TestMaybeRefreshStats(t *testing.T) { // There are no stats yet, so this must refresh the statistics on table t // even though rowsAffected=0. - refresher.maybeRefreshStats(ctx, s.Stopper(), descA.ID, 0 /* rowsAffected */, 0 /* asOf */) + refresher.maybeRefreshStats(ctx, s.Stopper(), descA.ID, 0 /* rowsAffected */, time.Microsecond /* asOf */) if err := checkStatsCount(ctx, cache, descA.ID, 1 /* expected */); err != nil { t.Fatal(err) } // Try to refresh again. With rowsAffected=0, the probability of a refresh // is 0, so refreshing will not succeed. - refresher.maybeRefreshStats(ctx, s.Stopper(), descA.ID, 0 /* rowsAffected */, 0 /* asOf */) + refresher.maybeRefreshStats(ctx, s.Stopper(), descA.ID, 0 /* rowsAffected */, time.Microsecond /* asOf */) if err := checkStatsCount(ctx, cache, descA.ID, 1 /* expected */); err != nil { t.Fatal(err) } @@ -107,7 +107,7 @@ func TestAverageRefreshTime(t *testing.T) { executor := s.InternalExecutor().(sqlutil.InternalExecutor) tableID := sqlbase.GetTableDescriptor(s.DB(), "t", "a").ID cache := NewTableStatisticsCache(10 /* cacheSize */, s.Gossip(), kvDB, executor) - refresher := MakeRefresher(executor, cache, 0 /* asOfTime */) + refresher := MakeRefresher(executor, cache, time.Microsecond /* asOfTime */) checkAverageRefreshTime := func(expected time.Duration) error { cache.InvalidateTableStats(ctx, tableID) @@ -249,7 +249,7 @@ func TestAverageRefreshTime(t *testing.T) { // average time between refreshes, so this call is not required to refresh // the statistics on table t. With rowsAffected=0, the probability of refresh // is 0. - refresher.maybeRefreshStats(ctx, s.Stopper(), tableID, 0 /* rowsAffected */, 0 /* asOf */) + refresher.maybeRefreshStats(ctx, s.Stopper(), tableID, 0 /* rowsAffected */, time.Microsecond /* asOf */) if err := checkStatsCount(ctx, cache, tableID, 20 /* expected */); err != nil { t.Fatal(err) } @@ -294,7 +294,7 @@ func TestAverageRefreshTime(t *testing.T) { // on table t even though rowsAffected=0. After refresh, only 15 stats should // remain (5 from column k and 10 from column v), since the old stats on k // were deleted. - refresher.maybeRefreshStats(ctx, s.Stopper(), tableID, 0 /* rowsAffected */, 0 /* asOf */) + refresher.maybeRefreshStats(ctx, s.Stopper(), tableID, 0 /* rowsAffected */, time.Microsecond /* asOf */) if err := checkStatsCount(ctx, cache, tableID, 15 /* expected */); err != nil { t.Fatal(err) } @@ -319,7 +319,7 @@ func TestAutoStatsReadOnlyTables(t *testing.T) { executor := s.InternalExecutor().(sqlutil.InternalExecutor) cache := NewTableStatisticsCache(10 /* cacheSize */, s.Gossip(), kvDB, executor) - refresher := MakeRefresher(executor, cache, 0 /* asOfTime */) + refresher := MakeRefresher(executor, cache, time.Microsecond /* asOfTime */) AutomaticStatisticsClusterMode.Override(&st.SV, true)