Skip to content

Commit

Permalink
Merge #27034 #27040
Browse files Browse the repository at this point in the history
27034: sql: turn on the optimizer for PREPARE statements r=benesch a=BramGruneir

The optimizer, when enabled, was not used for prepare statements. This patch
fixes that.

It should be noted that PREPARE has a limited subset of statements it can be
run with. Postgres only allows SELECT, INSERT, UPDATE, DELETE and VALUES
statements to be prepared.
See: https://www.postgresql.org/docs/current/static/sql-prepare.html

However, we allow a large number of additional statements.
As of right now, the optimizer only works on SELECT statements and will fallback
for all others, so this change should be safe for the foreseeable future.

Closes #26958.

Release note (sql change): When the cost based optimizer is enabled, it will
also affect prepared queries.

27040: sql: Remove SNAPSHOT isolation r=bdarnell a=bdarnell

The SNAPSHOT isolation level is no longer accessible from SQL.
Requests for SNAPSHOT will be remapped to SERIALIZABLE (in the same
way that the other ANSI levels already are).

KV-level support will remain through the next release cycle to support
transactions that began during a rolling upgrade.

Fixes #26475

Release note (sql change): The SNAPSHOT isolation level has been
removed. Transactions that request to use it are now mapped to
SERIALIZABLE.

The change in `lease_test.go` is the most complicated part here - I tried to adapt the test to a serializable transaction, but it's possible I misunderstood the intent of the test.

I left tests that used `BEGIN TRANSACTION SNAPSHOT` in place to ensure this syntax still works, and just changed the tests to reflect the new behavior of `SHOW transaction_isolation`.

While making this change I found that the previous change to map all the ANSI isolation levels to SERIALIZABLE was incomplete. There were some paths (such as `SET default_transaction_isolation`) that still used the old mapping of `READ COMMITTED` to `SNAPSHOT`. 

Also fixes a bug in `SET transaction_isolation='serializable'`, which was previously setting `snapshot` instead. 

Co-authored-by: Bram Gruneir <bram@cockroachlabs.com>
Co-authored-by: Vivek Menezes <vivek@cockroachlabs.com>
Co-authored-by: Ben Darnell <ben@cockroachlabs.com>
  • Loading branch information
4 people committed Jun 29, 2018
3 parents 3a60101 + c1f746f + 07f583e commit fd5743e
Show file tree
Hide file tree
Showing 16 changed files with 175 additions and 370 deletions.
2 changes: 0 additions & 2 deletions pkg/sql/conn_executor.go
Expand Up @@ -1658,8 +1658,6 @@ func (ex *connExecutor) isolationToProto(mode tree.IsolationLevel) (enginepb.Iso
switch mode {
case tree.UnspecifiedIsolation:
iso = ex.sessionData.DefaultIsolationLevel
case tree.SnapshotIsolation:
iso = enginepb.SNAPSHOT
case tree.SerializableIsolation:
iso = enginepb.SERIALIZABLE
default:
Expand Down
20 changes: 4 additions & 16 deletions pkg/sql/conn_executor_exec.go
Expand Up @@ -687,22 +687,10 @@ func (ex *connExecutor) dispatchToExecutionEngine(
) error {

planner.statsCollector.PhaseTimes()[plannerStartLogicalPlan] = timeutil.Now()
var err error
optMode := ex.sessionData.OptimizerMode
// TODO(radu): for now, the experimental force lookup join flag does not work
// with the optimizer. Turn the optimizer off for the query so the flag can
// still function.
if optMode != sessiondata.OptimizerAlways && ex.sessionData.LookupJoinEnabled {
optMode = sessiondata.OptimizerOff
}
if optMode != sessiondata.OptimizerOff {
err = planner.makeOptimizerPlan(ctx, stmt)
if canFallbackFromOpt(err, optMode, stmt) {
optMode = sessiondata.OptimizerOff
}
}

if optMode == sessiondata.OptimizerOff {
optimizerPlanned, err := planner.optionallyUseOptimizer(ctx, ex.sessionData, stmt)
if !optimizerPlanned && err == nil {
// Fallback if the optimizer was not enabled or used.
err = planner.makePlan(ctx, stmt)
}

Expand All @@ -727,7 +715,7 @@ func (ex *connExecutor) dispatchToExecutionEngine(
useDistSQL := false
// If we use the optimizer and we are in "local" mode, don't try to
// distribute.
if optMode != sessiondata.OptimizerLocal {
if ex.sessionData.OptimizerMode != sessiondata.OptimizerLocal {
ok, err := planner.prepareForDistSQLSupportCheck(
ctx, ex.sessionData.DistSQLMode == sessiondata.DistSQLAlways,
)
Expand Down
16 changes: 15 additions & 1 deletion pkg/sql/conn_executor_prepare.go
Expand Up @@ -201,9 +201,23 @@ func (ex *connExecutor) prepare(
txn.SetFixedTimestamp(ctx, *protoTS)
}

if err := p.prepare(ctx, stmt.AST); err != nil {
// PREPARE has a limited subset of statements it can be run with. Postgres
// only allows SELECT, INSERT, UPDATE, DELETE and VALUES statements to be
// prepared.
// See: https://www.postgresql.org/docs/current/static/sql-prepare.html
// However, we allow a large number of additional statements.
// As of right now, the optimizer only works on SELECT statements and will
// fallback for all others, so this should be safe for the foreseeable
// future.
if optimizerPlanned, err := p.optionallyUseOptimizer(ctx, ex.sessionData, stmt); err != nil {
return err
} else if !optimizerPlanned {
// Fallback if the optimizer was not enabled or used.
if err := p.prepare(ctx, stmt.AST); err != nil {
return err
}
}

if p.curPlan.plan == nil {
// The statement cannot be prepared. Nothing to do.
return nil
Expand Down
139 changes: 106 additions & 33 deletions pkg/sql/lease_test.go
Expand Up @@ -852,31 +852,26 @@ func TestTxnObeysTableModificationTime(t *testing.T) {
if _, err := sqlDB.Exec(`
CREATE DATABASE t;
CREATE TABLE t.kv (k CHAR PRIMARY KEY, v CHAR);
CREATE TABLE t.timestamp (k CHAR PRIMARY KEY, v CHAR);
INSERT INTO t.kv VALUES ('a', 'b');
`); err != nil {
t.Fatal(err)
}

tx, err := sqlDB.Begin()
// A read-write transaction that uses the old version of the descriptor.
txReadWrite, err := sqlDB.Begin()
if err != nil {
t.Fatal(err)
}
// Set the isolation level to Snapshot. This is because the test wants to
// check that this "old" transaction will not be allowed to commit at a new
// timestamp because of the "deadline" set according to its lease. So, the
// test will make sure that the txn is pushed. If the transaction were
// Serializable, then the push would cause it to restart regardless of the
// deadline.
if _, err := tx.Exec("SET TRANSACTION ISOLATION LEVEL SNAPSHOT"); err != nil {

// A read-only transaction that uses the old version of the descriptor.
txRead, err := sqlDB.Begin()
if err != nil {
t.Fatal(err)
}

// Insert an entry so that the transaction is guaranteed to be
// assigned a timestamp.
if _, err := tx.Exec(`
INSERT INTO t.timestamp VALUES ('a', 'b');
`); err != nil {
// A write-only transaction that uses the old version of the descriptor.
txWrite, err := sqlDB.Begin()
if err != nil {
t.Fatal(err)
}

Expand All @@ -885,37 +880,115 @@ INSERT INTO t.timestamp VALUES ('a', 'b');
t.Fatal(err)
}

rows, err := tx.Query(`SELECT * FROM t.kv`)
rows, err := txReadWrite.Query(`SELECT * FROM t.kv`)
if err != nil {
t.Fatal(err)
}
defer rows.Close()
for rows.Next() {
// The transaction is unable to see column m.
var k, v, m string
if err := rows.Scan(&k, &v, &m); !testutils.IsError(
err, "expected 2 destination arguments in Scan, not 3",
) {
t.Fatalf("err = %v", err)
}
err = rows.Scan(&k, &v)
if err != nil {
t.Fatal(err)
}
if k != "a" || v != "b" {
t.Fatalf("didn't find expected row: %s %s", k, v)

checkSelectResults := func(rows *gosql.Rows) {
defer func() {
if err := rows.Close(); err != nil {
t.Fatal(err)
}
}()
for rows.Next() {
// The transaction is unable to see column m.
var k, v, m string
if err := rows.Scan(&k, &v, &m); !testutils.IsError(
err, "expected 2 destination arguments in Scan, not 3",
) {
t.Fatalf("err = %v", err)
}
err = rows.Scan(&k, &v)
if err != nil {
t.Fatal(err)
}
if k != "a" || v != "b" {
t.Fatalf("didn't find expected row: %s %s", k, v)
}
}
}

// This INSERT will cause the transaction to be pushed past its deadline,
checkSelectResults(rows)

rows, err = txRead.Query(`SELECT * FROM t.kv`)
if err != nil {
t.Fatal(err)
}

checkSelectResults(rows)

// Read-only transaction commits just fine.
if err := txRead.Commit(); err != nil {
t.Fatal(err)
}

// This INSERT will cause the transaction to be pushed,
// which will be detected when we attempt to Commit() below.
if _, err := tx.Exec(`INSERT INTO t.kv VALUES ('c', 'd');`); err != nil {
if _, err := txReadWrite.Exec(`INSERT INTO t.kv VALUES ('c', 'd');`); err != nil {
t.Fatal(err)
}

if err := tx.Commit(); !testutils.IsError(err, "transaction deadline exceeded") {
// The transaction read at one timestamp and wrote at another so it
// has to be restarted because the spans read were modified by the backfill.
if err := txReadWrite.Commit(); !testutils.IsError(err,
"TransactionRetryError: retry txn \\(RETRY_SERIALIZABLE\\)") {
t.Fatalf("err = %v", err)
}

// This INSERT will cause the transaction to be pushed transparently,
// which will be detected when we attempt to Commit() below only because
// a deadline has been set.
if _, err := txWrite.Exec(`INSERT INTO t.kv VALUES ('c', 'd');`); err != nil {
t.Fatal(err)
}

deadlineError := "TransactionStatusError: transaction deadline exceeded"
if err := txWrite.Commit(); !testutils.IsError(err, deadlineError) {
t.Fatalf("err = %v", err)
}

// Test the deadline exceeded error with a CREATE/DROP INDEX.
schemaChanges := []struct{ sql string }{
{`CREATE INDEX foo on t.kv (v)`},
{`DROP INDEX t.kv@foo`},
}

for _, change := range schemaChanges {

txWrite, err := sqlDB.Begin()
if err != nil {
t.Fatal(err)
}
txUpdate, err := sqlDB.Begin()
if err != nil {
t.Fatal(err)
}

// Modify the table descriptor.
if _, err := sqlDB.Exec(change.sql); err != nil {
t.Fatal(err)
}

// This INSERT will cause the transaction to be pushed transparently,
// which will be detected when we attempt to Commit() below only because
// a deadline has been set.
if _, err := txWrite.Exec(`INSERT INTO t.kv VALUES ('c', 'd');`); err != nil {
t.Fatal(err)
}

if err := txWrite.Commit(); !testutils.IsError(err, deadlineError) {
t.Fatalf("err = %v", err)
}

if _, err := txUpdate.Exec(`UPDATE t.kv SET v = 'c' WHERE k = 'a';`); err != nil {
t.Fatal(err)
}

if err := txUpdate.Commit(); !testutils.IsError(err, deadlineError) {
t.Fatalf("err = %v", err)
}
}
}

// Test that a lease on a table descriptor is always acquired on the latest
Expand Down
60 changes: 0 additions & 60 deletions pkg/sql/logictest/testdata/logic_test/snapshot_certain_read

This file was deleted.

60 changes: 0 additions & 60 deletions pkg/sql/logictest/testdata/logic_test/snapshot_issue2861

This file was deleted.

0 comments on commit fd5743e

Please sign in to comment.