Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
105857: sql: add implicit SELECT FOR SHARE locking to FK checks r=DrewKimball,nvanbenschoten,rytaft,mgartner a=michae2

**explain: add transaction information to EXPLAIN ANALYZE**

Add transaction isolation, priority, and quality-of-service to the
output of `EXPLAIN ANALYZE`.

Release note (sql change): `EXPLAIN ANALYZE` output now includes:
- the isolation level of the statement's transaction
- the priority of the statement's transaction
- the quality of service level of the statement's transaction

---

**opt: do not use LockDurabilityGuaranteed under serializable isolation**

This is a follow-up from #103734.

We do not want to use guaranteed-durable (a.k.a. replicated) locking
under serializable isolation, because it prevents pipelining and other
optimizations, and is unnecessary for correctness. This commit amends
8cbc6d1 to only set durability for
`SELECT FOR UPDATE` locking under weaker isolation levels.

This means that query plans will be slightly different under different
isolation levels, and so we must add isolation level to the optimizer
memo staleness calculation.

Furthermore, this commit changes the error message added by
e633d5e to be about guaranteed-durable
locking rather than `SELECT FOR UPDATE`, because in a later commit this
specific error will also be triggered by foreign key checks under weaker
isolation levels.

Informs: #100144, #100156, #100193, #100194

Release note: None

---

**opt: show locking durability in EXPLAIN (OPT) output**

Because the "guaranteed-durable locking not yet implemented" error
condition is checked in execbuilder, it prevents not only execution but
also `EXPLAIN` of queries using guaranteed-durable locking. Thankfully
`EXPLAIN (OPT)` bypasses execbuilder, and hence still works, so use this
for now to verify that we are enabling durable locking for `SELECT FOR
UPDATE` under read committed isolation.

(Note that we have not yet fixed the `SELECT FOR UPDATE` plans to use
more precise locking, that will come in a later PR.)

Informs: #100194

Release note: None

---

**sql: add implicit SELECT FOR SHARE locking to FK parent checks**

Add SELECT FOR SHARE locking to FK parent checks. Under serializable
isolation, this locking is only used when
`enable_implicit_fk_locking_for_serializable` is set. Under weaker
isolation levels (snapshot and read committed) this locking is always
used.

We only need to lock during the insertion-side FK checks, which verify
the existence of a parent row. Deletion-side FK checks verify the
non-existence of a child row, and these do not need to lock. Instead, to
prevent concurrent inserts or updates to the child that would violate
the FK constraint, we rely on the intent(s) created by the deletion
conflicting with the FK locking of those concurrent inserts or updates.

Fixes: #80683
Informs: #100156

Epic: CRDB-25322

Release note (sql change): Add a new session variable,
`enable_implicit_fk_locking_for_serializable`, which controls locking
during foreign key checks under serializable isolation. With this set to
true, foreign key checks of the referenced (parent) table, such as those
performed during an INSERT or UPDATE of the referencing (child) table,
will lock the referenced row using SELECT FOR SHARE locking. (This is
somewhat analogous to the existing `enable_implicit_select_for_update`
variable but applies to the foreign key checks of a mutation statement
instead of the initial row fetch.)

Under weaker isolation levels such as read committed, SELECT FOR SHARE
locking will always be used to ensure the database maintains the foreign
key constraint, regardless of the current setting of
`enable_implicit_fk_locking_for_serializable`.

107212: ui-e2e-tests: steps to enable cypress tests r=maryliag a=rickystewart

This doesn't get the job fully working yet, but it's an improvement.

Epic: none
Part of #106584
Release note: None

107517: roachtest: add read committed variants of ycsb r=michae2 a=nvanbenschoten

Closes #107112.

This PR adds the following six roachtest variants:
```
ycsb/A/nodes=3/cpu=32/isolation-level=read-committed
ycsb/B/nodes=3/cpu=32/isolation-level=read-committed
ycsb/C/nodes=3/cpu=32/isolation-level=read-committed
ycsb/D/nodes=3/cpu=32/isolation-level=read-committed
ycsb/E/nodes=3/cpu=32/isolation-level=read-committed
ycsb/F/nodes=3/cpu=32/isolation-level=read-committed
```

It does so after adding an `--isolation-level` flag to ycsb, which controls the isolation level to run the workload transactions under. If unset, the workload will run with the default isolation level of the database.

Release note: None

107636: schemachanger: deflake TestConcurrentDeclarativeSchemaChanges r=postamar a=postamar

This commit deflakes this test by checking that the second schema change actually does block because of the first one, rather than checking that it has blocked. The bug was that the latter wasn't always guaranteed to happen because we didn't force the schema changes to run in parallel.

Fixes #106732.

Release note: None

Co-authored-by: Michael Erickson <[email protected]>
Co-authored-by: Ricky Stewart <[email protected]>
Co-authored-by: Nathan VanBenschoten <[email protected]>
Co-authored-by: Marius Posta <[email protected]>
  • Loading branch information
5 people committed Jul 26, 2023
5 parents 363b5da + bb857aa + 3892520 + 2a335a2 + abc0966 commit 284b6c0
Show file tree
Hide file tree
Showing 67 changed files with 1,583 additions and 235 deletions.
7 changes: 7 additions & 0 deletions pkg/ccl/logictestccl/tests/3node-tenant/generated_test.go

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

26 changes: 21 additions & 5 deletions pkg/cmd/roachtest/tests/ycsb.go
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,7 @@ const envYCSBFlags = "ROACHTEST_YCSB_FLAGS"
func registerYCSB(r registry.Registry) {
workloads := []string{"A", "B", "C", "D", "E", "F"}
cpusConfigs := []int{8, 32}
cpusWithReadCommitted := 32
cpusWithGlobalMVCCRangeTombstone := 32

// concurrencyConfigs contains near-optimal concurrency levels for each
Expand All @@ -45,7 +46,7 @@ func registerYCSB(r registry.Registry) {
}

runYCSB := func(
ctx context.Context, t test.Test, c cluster.Cluster, wl string, cpus int, rangeTombstone bool,
ctx context.Context, t test.Test, c cluster.Cluster, wl string, cpus int, readCommitted, rangeTombstone bool,
) {
// For now, we only want to run the zfs tests on GCE, since only GCE supports
// starting roachprod instances on zfs.
Expand Down Expand Up @@ -75,9 +76,11 @@ func registerYCSB(r registry.Registry) {
m := c.NewMonitor(ctx, c.Range(1, nodes))
m.Go(func(ctx context.Context) error {
var args string
args += fmt.Sprintf(" --select-for-update=%t", t.IsBuildVersion("v19.2.0"))
args += " --ramp=" + ifLocal(c, "0s", "2m")
args += " --duration=" + ifLocal(c, "10s", "30m")
if readCommitted {
args += " --isolation-level=read_committed"
}
if envFlags := os.Getenv(envYCSBFlags); envFlags != "" {
args += " " + envFlags
}
Expand Down Expand Up @@ -107,7 +110,7 @@ func registerYCSB(r registry.Registry) {
Benchmark: true,
Cluster: r.MakeClusterSpec(4, spec.CPU(cpus)),
Run: func(ctx context.Context, t test.Test, c cluster.Cluster) {
runYCSB(ctx, t, c, wl, cpus, false /* rangeTombstone */)
runYCSB(ctx, t, c, wl, cpus, false /* readCommitted */, false /* rangeTombstone */)
},
Tags: registry.Tags(`aws`),
})
Expand All @@ -119,19 +122,32 @@ func registerYCSB(r registry.Registry) {
Benchmark: true,
Cluster: r.MakeClusterSpec(4, spec.CPU(cpus), spec.SetFileSystem(spec.Zfs)),
Run: func(ctx context.Context, t test.Test, c cluster.Cluster) {
runYCSB(ctx, t, c, wl, cpus, false /* rangeTombstone */)
runYCSB(ctx, t, c, wl, cpus, false /* readCommitted */, false /* rangeTombstone */)
},
})
}

if cpus == cpusWithReadCommitted {
r.Add(registry.TestSpec{
Name: fmt.Sprintf("%s/isolation-level=read-committed", name),
Owner: registry.OwnerTestEng,
Benchmark: true,
Cluster: r.MakeClusterSpec(4, spec.CPU(cpus)),
Run: func(ctx context.Context, t test.Test, c cluster.Cluster) {
runYCSB(ctx, t, c, wl, cpus, true /* readCommitted */, false /* rangeTombstone */)
},
Tags: registry.Tags(`aws`),
})
}

if cpus == cpusWithGlobalMVCCRangeTombstone {
r.Add(registry.TestSpec{
Name: fmt.Sprintf("%s/mvcc-range-keys=global", name),
Owner: registry.OwnerTestEng,
Benchmark: true,
Cluster: r.MakeClusterSpec(4, spec.CPU(cpus)),
Run: func(ctx context.Context, t test.Test, c cluster.Cluster) {
runYCSB(ctx, t, c, wl, cpus, true /* rangeTombstone */)
runYCSB(ctx, t, c, wl, cpus, false /* readCommitted */, true /* rangeTombstone */)
},
Tags: registry.Tags(`aws`),
})
Expand Down
9 changes: 7 additions & 2 deletions pkg/sql/conn_executor_exec.go
Original file line number Diff line number Diff line change
Expand Up @@ -627,7 +627,8 @@ func (ex *connExecutor) execStmtInOpenState(
if !isPausablePortal() || portal.pauseInfo.execStmtInOpenState.ihWrapper == nil {
ctx, needFinish = ih.Setup(
ctx, ex.server.cfg, ex.statsCollector, p, ex.stmtDiagnosticsRecorder,
stmt.StmtNoConstants, os.ImplicitTxn.Get(), ex.extraTxnState.shouldCollectTxnExecutionStats,
stmt.StmtNoConstants, os.ImplicitTxn.Get(), ex.state.priority,
ex.extraTxnState.shouldCollectTxnExecutionStats,
)
} else {
ctx = portal.pauseInfo.execStmtInOpenState.ihWrapper.ctx
Expand Down Expand Up @@ -2926,7 +2927,11 @@ func (ex *connExecutor) recordTransactionFinish(
RowsWritten: ex.extraTxnState.rowsWritten,
BytesRead: ex.extraTxnState.bytesRead,
Priority: ex.state.priority,
SessionData: ex.sessionData(),
// TODO(107318): add isolation level
// TODO(107318): add qos
// TODO(107318): add asoftime or ishistorical
// TODO(107318): add readonly
SessionData: ex.sessionData(),
}

if ex.server.cfg.TestingKnobs.OnRecordTxnFinish != nil {
Expand Down
4 changes: 4 additions & 0 deletions pkg/sql/exec_util.go
Original file line number Diff line number Diff line change
Expand Up @@ -3612,6 +3612,10 @@ func (m *sessionDataMutator) SetOptimizerUseImprovedJoinElimination(val bool) {
m.data.OptimizerUseImprovedJoinElimination = val
}

func (m *sessionDataMutator) SetImplicitFKLockingForSerializable(val bool) {
m.data.ImplicitFKLockingForSerializable = val
}

// Utility functions related to scrubbing sensitive information on SQL Stats.

// quantizeCounts ensures that the Count field in the
Expand Down
13 changes: 13 additions & 0 deletions pkg/sql/insert_fast_path.go
Original file line number Diff line number Diff line change
Expand Up @@ -15,6 +15,7 @@ import (
"sync"

"github.com/cockroachdb/cockroach/pkg/kv/kvpb"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/lock"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/sql/catalog"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo"
Expand Down Expand Up @@ -188,10 +189,20 @@ func (r *insertFastPathRun) addFKChecks(
if r.traceKV {
log.VEventf(ctx, 2, "FKScan %s", span)
}
lockStrength := row.GetKeyLockingStrength(descpb.ToScanLockingStrength(c.Locking.Strength))
lockWaitPolicy := row.GetWaitPolicy(descpb.ToScanLockingWaitPolicy(c.Locking.WaitPolicy))
if r.fkBatch.Header.WaitPolicy != lockWaitPolicy {
return errors.AssertionFailedf(
"FK check lock wait policy %s did not match %s",
lockWaitPolicy, r.fkBatch.Header.WaitPolicy,
)
}
reqIdx := len(r.fkBatch.Requests)
r.fkBatch.Requests = append(r.fkBatch.Requests, kvpb.RequestUnion{})
r.fkBatch.Requests[reqIdx].MustSetInner(&kvpb.ScanRequest{
RequestHeader: kvpb.RequestHeaderFromSpan(span),
KeyLocking: lockStrength,
// TODO(michae2): Once #100193 is finished, also include c.Locking.Durability.
})
r.fkSpanInfo = append(r.fkSpanInfo, insertFastPathFKSpanInfo{
check: c,
Expand Down Expand Up @@ -248,6 +259,8 @@ func (n *insertFastPathNode) startExec(params runParams) error {
}
}
maxSpans := len(n.run.fkChecks) * len(n.input)
// Any FK checks using locking should have lock wait policy BLOCK.
n.run.fkBatch.Header.WaitPolicy = lock.WaitPolicy_Block
n.run.fkBatch.Requests = make([]kvpb.RequestUnion, 0, maxSpans)
n.run.fkSpanInfo = make([]insertFastPathFKSpanInfo, 0, maxSpans)
if len(n.input) > 1 {
Expand Down
19 changes: 18 additions & 1 deletion pkg/sql/instrumentation.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,9 @@ import (
"time"

"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/isolation"
"github.com/cockroachdb/cockroach/pkg/multitenant"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/server/telemetry"
"github.com/cockroachdb/cockroach/pkg/settings"
"github.com/cockroachdb/cockroach/pkg/sql/appstatspb"
Expand All @@ -35,6 +37,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/physicalplan"
"github.com/cockroachdb/cockroach/pkg/sql/sem/eval"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sessiondatapb"
"github.com/cockroachdb/cockroach/pkg/sql/sessionphase"
"github.com/cockroachdb/cockroach/pkg/sql/sqlstats"
"github.com/cockroachdb/cockroach/pkg/sql/sqltelemetry"
Expand Down Expand Up @@ -83,8 +86,12 @@ type instrumentationHelper struct {

// Query fingerprint (anonymized statement).
fingerprint string

// Transaction information.
implicitTxn bool
codec keys.SQLCodec
txnPriority roachpb.UserPriority

codec keys.SQLCodec

// -- The following fields are initialized by Setup() --

Expand Down Expand Up @@ -254,10 +261,12 @@ func (ih *instrumentationHelper) Setup(
stmtDiagnosticsRecorder *stmtdiagnostics.Registry,
fingerprint string,
implicitTxn bool,
txnPriority roachpb.UserPriority,
collectTxnExecStats bool,
) (newCtx context.Context, needFinish bool) {
ih.fingerprint = fingerprint
ih.implicitTxn = implicitTxn
ih.txnPriority = txnPriority
ih.codec = cfg.Codec
ih.origCtx = ctx
ih.evalCtx = p.EvalContext()
Expand Down Expand Up @@ -590,6 +599,14 @@ func (ih *instrumentationHelper) emitExplainAnalyzePlanToOutputBuilder(
}
}

qos := sessiondatapb.Normal
iso := isolation.Serializable
if ih.evalCtx != nil {
qos = ih.evalCtx.QualityOfService()
iso = ih.evalCtx.TxnIsoLevel
}
ob.AddTxnInfo(iso, ih.txnPriority, qos)

if err := emitExplain(ob, ih.evalCtx, ih.codec, ih.explainPlan); err != nil {
ob.AddTopLevelField("error emitting plan", fmt.Sprint(err))
}
Expand Down
46 changes: 46 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/fk_read_committed
Original file line number Diff line number Diff line change
@@ -0,0 +1,46 @@
# LogicTest: !local-mixed-22.2-23.1

# Some foreign key checks are prohibited under weaker isolation levels until we
# improve locking. See #80683, #100156, #100193.

statement ok
CREATE TABLE jars (j INT PRIMARY KEY)

statement ok
CREATE TABLE cookies (c INT PRIMARY KEY, j INT REFERENCES jars (j))

statement ok
SET SESSION CHARACTERISTICS AS TRANSACTION ISOLATION LEVEL READ COMMITTED

statement ok
INSERT INTO jars VALUES (1), (2)

# Foreign key checks of the parent require durable shared locking under weaker
# isolation levels, and are not yet supported.
query error pgcode 0A000 guaranteed-durable locking not yet implemented
INSERT INTO cookies VALUES (1, 1)

statement ok
BEGIN TRANSACTION ISOLATION LEVEL SERIALIZABLE

statement ok
INSERT INTO cookies VALUES (1, 1)

statement ok
COMMIT

query error pgcode 0A000 guaranteed-durable locking not yet implemented
UPDATE cookies SET j = 2 WHERE c = 1

# Foreign key checks of the child do not require locking.
query error violates foreign key constraint
UPDATE jars SET j = j + 4

query error violates foreign key constraint
DELETE FROM jars WHERE j = 1

statement ok
DELETE FROM cookies WHERE c = 1

statement ok
DELETE FROM jars WHERE j = 1
1 change: 1 addition & 0 deletions pkg/sql/logictest/testdata/logic_test/information_schema
Original file line number Diff line number Diff line change
Expand Up @@ -5289,6 +5289,7 @@ enable_auto_rehoming off
enable_create_stats_using_extremes off
enable_drop_enum_value on
enable_experimental_alter_column_type_general off
enable_implicit_fk_locking_for_serializable off
enable_implicit_select_for_update on
enable_implicit_transaction_for_batch_statements on
enable_insert_fast_path on
Expand Down
3 changes: 3 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/pg_catalog
Original file line number Diff line number Diff line change
Expand Up @@ -2727,6 +2727,7 @@ distsql off N
enable_auto_rehoming off NULL NULL NULL string
enable_create_stats_using_extremes off NULL NULL NULL string
enable_experimental_alter_column_type_general off NULL NULL NULL string
enable_implicit_fk_locking_for_serializable off NULL NULL NULL string
enable_implicit_select_for_update on NULL NULL NULL string
enable_implicit_transaction_for_batch_statements on NULL NULL NULL string
enable_insert_fast_path on NULL NULL NULL string
Expand Down Expand Up @@ -2887,6 +2888,7 @@ distsql off N
enable_auto_rehoming off NULL user NULL off off
enable_create_stats_using_extremes off NULL user NULL off off
enable_experimental_alter_column_type_general off NULL user NULL off off
enable_implicit_fk_locking_for_serializable off NULL user NULL off off
enable_implicit_select_for_update on NULL user NULL on on
enable_implicit_transaction_for_batch_statements on NULL user NULL on on
enable_insert_fast_path on NULL user NULL on on
Expand Down Expand Up @@ -3044,6 +3046,7 @@ distsql_workmem NULL NULL NULL
enable_auto_rehoming NULL NULL NULL NULL NULL
enable_create_stats_using_extremes NULL NULL NULL NULL NULL
enable_experimental_alter_column_type_general NULL NULL NULL NULL NULL
enable_implicit_fk_locking_for_serializable NULL NULL NULL NULL NULL
enable_implicit_select_for_update NULL NULL NULL NULL NULL
enable_implicit_transaction_for_batch_statements NULL NULL NULL NULL NULL
enable_insert_fast_path NULL NULL NULL NULL NULL
Expand Down
20 changes: 10 additions & 10 deletions pkg/sql/logictest/testdata/logic_test/read_committed
Original file line number Diff line number Diff line change
Expand Up @@ -3,7 +3,7 @@
subtest select_for_update

# SELECT FOR UPDATE is prohibited under weaker isolation levels until we improve
# locking. See #57031, #75457, #100144.
# locking. See #57031, #75457, #100144, #100193.

statement ok
CREATE TABLE supermarket (
Expand Down Expand Up @@ -38,7 +38,7 @@ COMMIT
statement ok
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED

query error pgcode 0A000 cannot execute SELECT FOR UPDATE statements under ReadCommitted isolation
query error pgcode 0A000 guaranteed-durable locking not yet implemented
SELECT aisle FROM supermarket WHERE person = 'matilda' FOR UPDATE

statement ok
Expand All @@ -51,7 +51,7 @@ SET CLUSTER SETTING sql.txn.snapshot_isolation.enabled = true
statement ok
BEGIN TRANSACTION ISOLATION LEVEL SNAPSHOT

query error pgcode 0A000 cannot execute SELECT FOR UPDATE statements under Snapshot isolation
query error pgcode 0A000 guaranteed-durable locking not yet implemented
SELECT aisle FROM supermarket WHERE person = 'matilda' FOR UPDATE

statement ok
Expand All @@ -67,7 +67,7 @@ BEGIN TRANSACTION
statement ok
SET TRANSACTION ISOLATION LEVEL READ COMMITTED;

query error pgcode 0A000 cannot execute SELECT FOR UPDATE statements under ReadCommitted isolation
query error pgcode 0A000 guaranteed-durable locking not yet implemented
UPDATE supermarket
SET aisle = (SELECT aisle FROM supermarket WHERE person = 'matilda' FOR UPDATE)
WHERE person = 'michael'
Expand All @@ -82,7 +82,7 @@ BEGIN TRANSACTION
statement ok
SET TRANSACTION ISOLATION LEVEL READ COMMITTED;

query error pgcode 0A000 cannot execute SELECT FOR UPDATE statements under ReadCommitted isolation
query error pgcode 0A000 guaranteed-durable locking not yet implemented
WITH s AS
(SELECT aisle FROM supermarket WHERE person = 'matilda' FOR UPDATE)
SELECT aisle + 1 FROM s
Expand All @@ -99,7 +99,7 @@ statement ok
SET SESSION CHARACTERISTICS AS TRANSACTION ISOLATION LEVEL READ COMMITTED

# But calling that function should fail.
query error pgcode 0A000 cannot execute SELECT FOR UPDATE statements under ReadCommitted isolation
query error pgcode 0A000 guaranteed-durable locking not yet implemented
INSERT INTO supermarket (person, aisle) VALUES ('grandma', wrangle('matilda'))

statement ok
Expand All @@ -116,14 +116,14 @@ statement ok
PREPARE psa AS SELECT aisle FROM supermarket WHERE person = $1::STRING FOR UPDATE

# But execution should fail.
query error pgcode 0A000 cannot execute SELECT FOR UPDATE statements under ReadCommitted isolation
query error pgcode 0A000 guaranteed-durable locking not yet implemented
EXECUTE psa('matilda')

statement ok
DEALLOCATE psa

# SELECT FOR UPDATE using a lookup join should also fail.
query error pgcode 0A000 cannot execute SELECT FOR UPDATE statements under ReadCommitted isolation
query error pgcode 0A000 guaranteed-durable locking not yet implemented
WITH names AS MATERIALIZED
(SELECT 'matilda' AS person)
SELECT aisle
Expand All @@ -132,14 +132,14 @@ SELECT aisle
FOR UPDATE

# SELECT FOR UPDATE using an index join should also fail.
query error pgcode 0A000 cannot execute SELECT FOR UPDATE statements under ReadCommitted isolation
query error pgcode 0A000 guaranteed-durable locking not yet implemented
SELECT aisle
FROM supermarket@supermarket_starts_with_idx
WHERE starts_with = 'm'
FOR UPDATE

# SELECT FOR UPDATE using a zigzag join should also fail.
query error pgcode 0A000 cannot execute SELECT FOR UPDATE statements under ReadCommitted isolation
query error pgcode 0A000 guaranteed-durable locking not yet implemented
SELECT aisle
FROM supermarket@{FORCE_ZIGZAG}
WHERE starts_with = 'm' AND ends_with = 'lda'
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/logictest/testdata/logic_test/show_source
Original file line number Diff line number Diff line change
Expand Up @@ -62,6 +62,7 @@ distsql off
enable_auto_rehoming off
enable_create_stats_using_extremes off
enable_experimental_alter_column_type_general off
enable_implicit_fk_locking_for_serializable off
enable_implicit_select_for_update on
enable_implicit_transaction_for_batch_statements on
enable_insert_fast_path on
Expand Down
Loading

0 comments on commit 284b6c0

Please sign in to comment.