Skip to content

Commit

Permalink
sem/tree: use StmtTimestamp as max for AsOf, enforce non-zero intervals
Browse files Browse the repository at this point in the history
Before this PR there was some unexpected behavior when using AOST clauses with
literals which were very small. There are two culprits. The first is that the
logic to evaluate the AOST took a maximum value that was different and always
greater than the statement time which is the timestamp from which intervals
literals are used to compute a timestamp. That means that if the interval
literal were positive but smaller than the difference between the "max" and
the statement time, they would be judged to be valid. There's no reason we need
to compute an additional maximum, instead we just use statement time as the
maximum. Another issue is that values smaller than 1 microsecond are rounded to
zero so `100ns`, `0,0` and `-100ns` are equivalent as far as the evaluation
code was concerned. This diff makes all of these intervals which are rounded to
zero illegal.

Note that in a number of testing scenarios, particularly around backup/restore
and automatic stats collection we used to use a zero as of system time value
which this PR changes to 1 microsecond which ought to be functionally
equivalent.

Fixes #34371.

Release note (sql change): Disallow `AS OF SYSTEM TIME` interval expressions
less than 1 microsecond in the past.
  • Loading branch information
ajwerner committed Feb 22, 2019
1 parent 894baed commit 8d9c588
Show file tree
Hide file tree
Showing 21 changed files with 69 additions and 59 deletions.
2 changes: 1 addition & 1 deletion pkg/ccl/backupccl/backup.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
}
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/ccl/backupccl/backup_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -2798,7 +2798,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)
Expand Down
4 changes: 1 addition & 3 deletions pkg/ccl/backupccl/restore.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
}
Expand Down
6 changes: 4 additions & 2 deletions pkg/ccl/changefeedccl/changefeed_stmt.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down
2 changes: 1 addition & 1 deletion pkg/ccl/importccl/import_stmt_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -2436,7 +2436,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()
Expand Down
4 changes: 2 additions & 2 deletions pkg/sql/conn_executor.go
Original file line number Diff line number Diff line change
Expand Up @@ -1751,8 +1751,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
Expand Down
6 changes: 3 additions & 3 deletions pkg/sql/conn_executor_exec.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)
}
Expand All @@ -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)
}
Expand Down Expand Up @@ -1071,7 +1071,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
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/conn_executor_prepare.go
Original file line number Diff line number Diff line change
Expand Up @@ -191,7 +191,7 @@ func (ex *connExecutor) populatePrepared(

p.extendedEvalCtx.PrepareOnly = true

protoTS, err := p.isAsOf(stmt.AST, ex.server.cfg.Clock.Now() /* max */)
protoTS, err := p.isAsOf(stmt.AST)
if err != nil {
return 0, err
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/create_stats.go
Original file line number Diff line number Diff line change
Expand Up @@ -152,7 +152,7 @@ func (n *createStatsNode) startJob(ctx context.Context, resultsCh chan<- tree.Da
// Evaluate the AS OF time, if any.
var asOf *hlc.Timestamp
if n.AsOf.Expr != nil {
asOfTs, err := n.p.EvalAsOfTimestamp(n.AsOf, n.p.ExecCfg().Clock.Now())
asOfTs, err := n.p.EvalAsOfTimestamp(n.AsOf)
if err != nil {
return err
}
Expand Down
15 changes: 5 additions & 10 deletions pkg/sql/exec_util.go
Original file line number Diff line number Diff line change
Expand Up @@ -666,10 +666,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`.
Expand All @@ -686,10 +684,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:
Expand All @@ -715,7 +710,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
Expand All @@ -724,7 +719,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
}

Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/logictest/logic.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)
Expand Down
28 changes: 20 additions & 8 deletions pkg/sql/logictest/testdata/logic_test/as_of
Original file line number Diff line number Diff line change
Expand Up @@ -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 '-1us'
----
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 '-1us'
----
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

Expand Down Expand Up @@ -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 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'
2 changes: 1 addition & 1 deletion pkg/sql/logictest/testdata/logic_test/distsql_stats
Original file line number Diff line number Diff line change
Expand Up @@ -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 '-1us'

query TTIII colnames
SELECT statistics_name, column_names, row_count, distinct_count, null_count FROM [SHOW STATISTICS FOR TABLE data]
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/logictest/testdata/logic_test/subquery_correlated
Original file line number Diff line number Diff line change
Expand Up @@ -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 '-1us')
FROM c
WHERE EXISTS(SELECT * FROM o WHERE o.c_id=c.c_id)

Expand Down
10 changes: 5 additions & 5 deletions pkg/sql/logictest/testdata/logic_test/txn_as_of
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,7 @@ statement ok
COMMIT

statement ok
BEGIN AS OF SYSTEM TIME '-1ns'
BEGIN AS OF SYSTEM TIME '-1us'

query I
SELECT * FROM t
Expand All @@ -33,7 +33,7 @@ statement ok
COMMIT

statement ok
BEGIN; SET TRANSACTION AS OF SYSTEM TIME '-1ns'
BEGIN; SET TRANSACTION AS OF SYSTEM TIME '-1us'

query I
SELECT * FROM t
Expand Down Expand Up @@ -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 '-1us'

statement error pq: cannot execute INSERT in a read-only transaction
INSERT INTO t VALUES (3)
Expand All @@ -85,7 +85,7 @@ statement ok
COMMIT

statement ok
BEGIN AS OF SYSTEM TIME '-1ns'
BEGIN AS OF SYSTEM TIME '-1us'

statement error pq: cannot execute INSERT in a read-only transaction
INSERT INTO t VALUES (3)
Expand All @@ -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 '-1us';

query I
SELECT * FROM t
Expand Down
3 changes: 1 addition & 2 deletions pkg/sql/opt/optbuilder/select.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
)

Expand Down Expand Up @@ -760,7 +759,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})
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/opt/optbuilder/testdata/select
Original file line number Diff line number Diff line change
Expand Up @@ -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

Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/planhook.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/render.go
Original file line number Diff line number Diff line change
Expand Up @@ -484,7 +484,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
}
Expand Down
16 changes: 10 additions & 6 deletions pkg/sql/sem/tree/as_of.go
Original file line number Diff line number Diff line change
Expand Up @@ -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.
Expand Down Expand Up @@ -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)
Expand All @@ -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")
Expand All @@ -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)
}
Expand All @@ -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
Expand Down
Loading

0 comments on commit 8d9c588

Please sign in to comment.