Skip to content

Commit

Permalink
sql: disallow using cluster_logical_timestamp as column default when …
Browse files Browse the repository at this point in the history
…backfilling

Previously, `ADD COLUMN ... DEFAULT cluster_logical_timestamp()` would
crash the node and leave the table in a corrupt state. The root cause
is a nil pointer dereference. This commit fixed it by returning an
unimplemented error and hence disallow using this builtin function as
default value when backfilling.

Release note (bug fix): fixed a bug as detailed in #98269.
  • Loading branch information
Xiang-Gu committed Mar 15, 2023
1 parent adb0556 commit 4b7aa6e
Show file tree
Hide file tree
Showing 8 changed files with 39 additions and 5 deletions.
3 changes: 3 additions & 0 deletions pkg/sql/backfill/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -24,9 +24,12 @@ go_library(
"//pkg/sql/catalog/schemaexpr",
"//pkg/sql/catalog/tabledesc",
"//pkg/sql/catalog/typedesc",
"//pkg/sql/colexecerror",
"//pkg/sql/execinfra",
"//pkg/sql/execinfrapb",
"//pkg/sql/isql",
"//pkg/sql/pgwire/pgcode",
"//pkg/sql/pgwire/pgerror",
"//pkg/sql/row",
"//pkg/sql/rowenc",
"//pkg/sql/rowinfra",
Expand Down
11 changes: 11 additions & 0 deletions pkg/sql/backfill/backfill.go
Original file line number Diff line number Diff line change
Expand Up @@ -25,8 +25,11 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/catalog/fetchpb"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/schemaexpr"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/typedesc"
"github.com/cockroachdb/cockroach/pkg/sql/colexecerror"
"github.com/cockroachdb/cockroach/pkg/sql/execinfra"
"github.com/cockroachdb/cockroach/pkg/sql/isql"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
"github.com/cockroachdb/cockroach/pkg/sql/row"
"github.com/cockroachdb/cockroach/pkg/sql/rowenc"
"github.com/cockroachdb/cockroach/pkg/sql/rowinfra"
Expand Down Expand Up @@ -372,6 +375,10 @@ func (cb *ColumnBackfiller) RunColumnBackfillChunk(
for j, e := range cb.updateExprs {
val, err := eval.Expr(ctx, cb.evalCtx, e)
if err != nil {
if errors.Is(err, colexecerror.ErrNilTxnAccessedInColBackfill) {
// Issue #98269; Cannot use `cluster_logical_timestamp()` as column default value.
return roachpb.Key{}, pgerror.WithCandidateCode(err, pgcode.FeatureNotSupported)
}
return roachpb.Key{}, sqlerrors.NewInvalidSchemaDefinitionError(err)
}
if j < len(cb.added) && !cb.added[j].IsNullable() && val == tree.DNull {
Expand Down Expand Up @@ -876,6 +883,10 @@ func (ib *IndexBackfiller) BuildIndexEntriesChunk(
}
val, err := eval.Expr(ctx, ib.evalCtx, texpr)
if err != nil {
if errors.Is(err, colexecerror.ErrNilTxnAccessedInColBackfill) {
// Issue #98269; Cannot use `cluster_logical_timestamp()` as column default value.
err = pgerror.WithCandidateCode(err, pgcode.FeatureNotSupported)
}
return err
}
colIdx, ok := ib.colIdxMap.Get(colID)
Expand Down
2 changes: 2 additions & 0 deletions pkg/sql/colexecerror/error.go
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,8 @@ import (

const panicLineSubstring = "runtime/panic.go"

var ErrNilTxnAccessedInColBackfill = errors.New("attempted to access nil txn in column backfill")

// CatchVectorizedRuntimeError executes operation, catches a runtime error if
// it is coming from the vectorized engine, and returns it. If an error not
// related to the vectorized engine occurs, it is not recovered from.
Expand Down
11 changes: 11 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/alter_table
Original file line number Diff line number Diff line change
Expand Up @@ -3173,3 +3173,14 @@ SHOW CONSTRAINTS FROM t_96728
table_name constraint_name constraint_type details validated
t_96728 t_96728_j_k_key UNIQUE UNIQUE (j ASC, k ASC) WHERE (i > 0) true
t_96728 t_96728_pkey PRIMARY KEY PRIMARY KEY (i ASC) true

# This subtest disallows using builtin function `cluster_logical_timestamp()`
# as the default expression when backfilling a column.
subtest 98269

statement ok
CREATE TABLE t_98269 (i INT PRIMARY KEY);
INSERT INTO t_98269 VALUES (0);

statement error pgcode 0A000 .* cluster_logical_timestamp\(\): attempted to access nil txn in column backfill
ALTER TABLE t_98269 ADD COLUMN j DECIMAL NOT NULL DEFAULT cluster_logical_timestamp();
2 changes: 1 addition & 1 deletion pkg/sql/sem/builtins/builtins.go
Original file line number Diff line number Diff line change
Expand Up @@ -2751,7 +2751,7 @@ nearest replica.`, builtinconstants.DefaultFollowerReadDuration),
Types: tree.ParamTypes{},
ReturnType: tree.FixedReturnType(types.Decimal),
Fn: func(ctx context.Context, evalCtx *eval.Context, args tree.Datums) (tree.Datum, error) {
return evalCtx.GetClusterTimestamp(), nil
return evalCtx.GetClusterTimestamp()
},
Info: `Returns the logical time of the current transaction as
a CockroachDB HLC in decimal form.
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/sem/eval/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -53,6 +53,7 @@ go_library(
"//pkg/settings/cluster",
"//pkg/sql/catalog/catpb",
"//pkg/sql/catalog/descpb",
"//pkg/sql/colexecerror",
"//pkg/sql/lex",
"//pkg/sql/parser",
"//pkg/sql/pgwire/pgcode",
Expand Down
10 changes: 7 additions & 3 deletions pkg/sql/sem/eval/context.go
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/catpb"
"github.com/cockroachdb/cockroach/pkg/sql/colexecerror"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgwirecancel"
Expand Down Expand Up @@ -495,12 +496,15 @@ func (ec *Context) GetStmtTimestamp() time.Time {

// GetClusterTimestamp retrieves the current cluster timestamp as per
// the evaluation context. The timestamp is guaranteed to be nonzero.
func (ec *Context) GetClusterTimestamp() *tree.DDecimal {
func (ec *Context) GetClusterTimestamp() (*tree.DDecimal, error) {
if ec.Txn == nil {
return nil, colexecerror.ErrNilTxnAccessedInColBackfill
}
ts := ec.Txn.CommitTimestamp()
if ts.IsEmpty() {
panic(errors.AssertionFailedf("zero cluster timestamp in txn"))
return nil, errors.AssertionFailedf("zero cluster timestamp in txn")
}
return TimestampToDecimalDatum(ts)
return TimestampToDecimalDatum(ts), nil
}

// HasPlaceholders returns true if this EvalContext's placeholders have been
Expand Down
4 changes: 3 additions & 1 deletion pkg/sql/sem/eval/timeconv_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/stop"
"github.com/stretchr/testify/require"
)

// Test that EvalContext.GetClusterTimestamp() gets its timestamp from the
Expand Down Expand Up @@ -76,7 +77,8 @@ func TestClusterTimestampConversion(t *testing.T) {
),
}

dec := ctx.GetClusterTimestamp()
dec, err := ctx.GetClusterTimestamp()
require.NoError(t, err)
final := dec.Text('f')
if final != d.expected {
t.Errorf("expected %s, but found %s", d.expected, final)
Expand Down

0 comments on commit 4b7aa6e

Please sign in to comment.