Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
77063: sql: don't panic when PREPARING a PREPARE r=jordanlewis a=jordanlewis

Depends on #77059

Closes #77061

Previously, trying to run a PREPARE statement via the extended protocol
would panic the server. This is no longer the case.

Release note (bug fix): the database no longer crashes when running a
SQL PREPARE via the Postgres extended protocol.
Release justification: Bug fixes and low-risk updates to new functionality

77113: dev: add build targets to `dev build --help` r=rickystewart a=ajwerner

It was painful to figure out what you could build before this.

```
Build the specified binaries:
	bazel-remote
	buildifier
	buildozer
	cockroach
	cockroach-oss
	cockroach-short
	cockroach-sql
	crlfmt
	dev
	docgen
	execgen
	gofmt
	goimports
	label-merged-pr
	langgen
	optfmt
	optgen
	oss
	roachprod
	roachprod-stress
	roachtest
	short
	staticcheck
	stress
	workload
```

Release justification: non-production code change

Release note: None

77131: vendor: bump Pebble to 40d39da505a5 r=jbowens a=jbowens

```
40d39da5 db: produce the FlushEnd event after installing readState
cef3f146 compaction: add support for concurrent manual compactions
cb848478 manifest: add methods for extending table bounds
7e5c8ee1 sstable: move block property collector calls to the Writer client goroutine
0e0d279a  internal/keyspan: move MergingIter from internal/rangekey
09203fd9 *: Expose range key iterators through table cache
894b57aa db: correctly set point key bounds in `TestIngest_UpdateSequenceNumber`
6c7f6ed4 base: update IsExclusiveSentinel to account for multiple range key kinds
e0589417 db: add external sstable merging iterator
85162b61 sstable: unify sstable size estimation by using a single abstraction
bac6da8f internal/rangekey: adjust MergingIter interface, return fragment sets
7eb64ae7 Revert "db: add experimental DB.RegisterFlushCompletedCallback"
2c522458 db: remove unnecessary key clones during ingest load
998400e7 db: fix skipped key during Prev at synthetic range key marker
13f8f7ce *: separately track smallest/largest point/range keys in FileMetadata
7449c652 sstable: change index block size estimation to use inflight size
31899eb1 sstable: flush index blocks only from the Writer client goroutine
129bc0d4 internal/rangekey: fix invariant violation during SeekGE(upper-bound)
87ab6c71 db: write range keys to memtables
```

Release note: None

Release justification: commits merged in Pebble before stability period

Co-authored-by: Jordan Lewis <jordanthelewis@gmail.com>
Co-authored-by: Andrew Werner <awerner32@gmail.com>
Co-authored-by: Jackson Owens <jackson@cockroachlabs.com>
  • Loading branch information
4 people committed Mar 1, 2022
4 parents 07057de + c9d4ff1 + e2b2e0d + 4f29ddb commit d89e743
Show file tree
Hide file tree
Showing 16 changed files with 146 additions and 28 deletions.
6 changes: 3 additions & 3 deletions DEPS.bzl
Original file line number Diff line number Diff line change
Expand Up @@ -1287,10 +1287,10 @@ def go_deps():
patches = [
"@cockroach//build/patches:com_github_cockroachdb_pebble.patch",
],
sha256 = "1d4eff199bd4952fad40c7c1c64e167ef8600f222f57058ae7a050979c7650d8",
strip_prefix = "github.com/cockroachdb/pebble@v0.0.0-20220217165617-821db50635d6",
sha256 = "71da6a69951ab9767aa51efd34b2a4040ab655f67a5b0be87578af5a85132d26",
strip_prefix = "github.com/cockroachdb/pebble@v0.0.0-20220227235451-40d39da505a5",
urls = [
"https://storage.googleapis.com/cockroach-godeps/gomod/github.com/cockroachdb/pebble/com_github_cockroachdb_pebble-v0.0.0-20220217165617-821db50635d6.zip",
"https://storage.googleapis.com/cockroach-godeps/gomod/github.com/cockroachdb/pebble/com_github_cockroachdb_pebble-v0.0.0-20220227235451-40d39da505a5.zip",
],
)
go_repository(
Expand Down
2 changes: 1 addition & 1 deletion go.mod
Original file line number Diff line number Diff line change
Expand Up @@ -46,7 +46,7 @@ require (
github.com/cockroachdb/go-test-teamcity v0.0.0-20191211140407-cff980ad0a55
github.com/cockroachdb/gostdlib v1.13.0
github.com/cockroachdb/logtags v0.0.0-20211118104740-dabe8e521a4f
github.com/cockroachdb/pebble v0.0.0-20220217165617-821db50635d6
github.com/cockroachdb/pebble v0.0.0-20220227235451-40d39da505a5
github.com/cockroachdb/redact v1.1.3
github.com/cockroachdb/returncheck v0.0.0-20200612231554-92cdbca611dd
github.com/cockroachdb/stress v0.0.0-20220217190341-94cf65c2a29f
Expand Down
4 changes: 2 additions & 2 deletions go.sum
Original file line number Diff line number Diff line change
Expand Up @@ -436,8 +436,8 @@ github.com/cockroachdb/logtags v0.0.0-20211118104740-dabe8e521a4f h1:6jduT9Hfc0n
github.com/cockroachdb/logtags v0.0.0-20211118104740-dabe8e521a4f/go.mod h1:Vz9DsVWQQhf3vs21MhPMZpMGSht7O/2vFW2xusFUVOs=
github.com/cockroachdb/panicparse/v2 v2.0.0-20211103220158-604c82a44f1e h1:FrERdkPlRj+v7fc+PGpey3GUiDGuTR5CsmLCA54YJ8I=
github.com/cockroachdb/panicparse/v2 v2.0.0-20211103220158-604c82a44f1e/go.mod h1:pMxsKyCewnV3xPaFvvT9NfwvDTcIx2Xqg0qL5Gq0SjM=
github.com/cockroachdb/pebble v0.0.0-20220217165617-821db50635d6 h1:h0QXUCqMzrxfdxAh+WTYZZOqilZBc7sOpDTMFZHzhy4=
github.com/cockroachdb/pebble v0.0.0-20220217165617-821db50635d6/go.mod h1:buxOO9GBtOcq1DiXDpIPYrmxY020K2A8lOrwno5FetU=
github.com/cockroachdb/pebble v0.0.0-20220227235451-40d39da505a5 h1:6ZsiW1sWGEsx2kDq98bdoDfdDeO2IgfI4e2FxUQwkdk=
github.com/cockroachdb/pebble v0.0.0-20220227235451-40d39da505a5/go.mod h1:buxOO9GBtOcq1DiXDpIPYrmxY020K2A8lOrwno5FetU=
github.com/cockroachdb/redact v1.0.8/go.mod h1:BVNblN9mBWFyMyqK1k3AAiSxhvhfK2oOZZ2lK+dpvRg=
github.com/cockroachdb/redact v1.1.3 h1:AKZds10rFSIj7qADf0g46UixK8NNLwWTNdCIGS5wfSQ=
github.com/cockroachdb/redact v1.1.3/go.mod h1:BVNblN9mBWFyMyqK1k3AAiSxhvhfK2oOZZ2lK+dpvRg=
Expand Down
17 changes: 16 additions & 1 deletion pkg/cmd/dev/build.go
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@ import (
"path"
"path/filepath"
"runtime"
"sort"
"strings"

"github.com/alessio/shellescape"
Expand All @@ -40,7 +41,11 @@ func makeBuildCmd(runE func(cmd *cobra.Command, args []string) error) *cobra.Com
buildCmd := &cobra.Command{
Use: "build <binary>",
Short: "Build the specified binaries",
Long: "Build the specified binaries.",
Long: fmt.Sprintf(
"Build the specified binaries either using their bazel targets or one "+
"of the following shorthands:\n\t%s",
strings.Join(allBuildTargets, "\n\t"),
),
// TODO(irfansharif): Flesh out the example usage patterns.
Example: `
dev build cockroach
Expand Down Expand Up @@ -93,6 +98,16 @@ var buildTargetMapping = map[string]string{
"workload": "//pkg/cmd/workload:workload",
}

// allBuildTargets is a sorted list of all the available build targets.
var allBuildTargets = func() []string {
ret := make([]string, 0, len(buildTargetMapping))
for t := range buildTargetMapping {
ret = append(ret, t)
}
sort.Strings(ret)
return ret
}()

func (d *dev) build(cmd *cobra.Command, commandLine []string) error {
targets, additionalBazelArgs := splitArgsAtDash(cmd, commandLine)
ctx := cmd.Context()
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/stores_server.go
Original file line number Diff line number Diff line change
Expand Up @@ -162,7 +162,7 @@ func (is Server) CompactEngineSpan(
resp := &CompactEngineSpanResponse{}
err := is.execStoreCommand(ctx, req.StoreRequestHeader,
func(ctx context.Context, s *Store) error {
return s.Engine().CompactRange(req.Span.Key, req.Span.EndKey, true /* forceBottommost */)
return s.Engine().CompactRange(req.Span.Key, req.Span.EndKey)
})
return resp, err
}
1 change: 1 addition & 0 deletions pkg/sql/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -537,6 +537,7 @@ go_test(
"sort_test.go",
"split_test.go",
"sql_cursor_test.go",
"sql_prepare_test.go",
"statement_mark_redaction_test.go",
"table_ref_test.go",
"table_test.go",
Expand Down
36 changes: 31 additions & 5 deletions pkg/sql/conn_executor_prepare.go
Original file line number Diff line number Diff line change
Expand Up @@ -160,6 +160,17 @@ func (ex *connExecutor) prepare(
return prepared, nil
}

origNumPlaceholders := stmt.NumPlaceholders
switch stmt.AST.(type) {
case *tree.Prepare:
// Special case: we're preparing a SQL-level PREPARE using the
// wire protocol. There's an ambiguity from the perspective of this code:
// any placeholders that are inside of the statement that we're preparing
// shouldn't be treated as placeholders to the PREPARE statement. So, we
// edit the NumPlaceholders field to be 0 here.
stmt.NumPlaceholders = 0
}

var flags planFlags
prepare := func(ctx context.Context, txn *kv.Txn) (err error) {
p := &ex.planner
Expand Down Expand Up @@ -205,6 +216,9 @@ func (ex *connExecutor) prepare(
},
}
prepared.Statement = stmt.Statement
// When we set our prepared statement, we need to make sure to propagate
// the original NumPlaceholders if we're preparing a PREPARE.
prepared.Statement.NumPlaceholders = origNumPlaceholders
prepared.StatementNoConstants = stmt.StmtNoConstants
prepared.StatementSummary = stmt.StmtSummary

Expand Down Expand Up @@ -258,10 +272,9 @@ func (ex *connExecutor) populatePrepared(
// 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.
// However, we must be able to handle every type of statement below because
// the Postgres extended protocol requires running statements via the prepare
// and execute paths.
flags, err := p.prepareUsingOptimizer(ctx)
if err != nil {
log.VEventf(ctx, 1, "optimizer prepare failed: %v", err)
Expand Down Expand Up @@ -539,7 +552,20 @@ func (ex *connExecutor) execDescribe(

res.SetInferredTypes(ps.InferredTypes)

if stmtHasNoData(ps.AST) {
ast := ps.AST
if execute, ok := ast.(*tree.Execute); ok {
// If we're describing an EXECUTE, we need to look up the statement type
// of the prepared statement that the EXECUTE refers to, or else we'll
// return the wrong information for describe.
innerPs, found := ex.extraTxnState.prepStmtsNamespace.prepStmts[string(execute.Name)]
if !found {
return retErr(pgerror.Newf(
pgcode.InvalidSQLStatementName,
"unknown prepared statement %q", descCmd.Name))
}
ast = innerPs.AST
}
if stmtHasNoData(ast) {
res.SetNoDataRowDescription()
} else {
res.SetPrepStmtOutput(ctx, ps.Columns)
Expand Down
20 changes: 18 additions & 2 deletions pkg/sql/plan_opt.go
Original file line number Diff line number Diff line change
Expand Up @@ -53,7 +53,7 @@ func (p *planner) prepareUsingOptimizer(ctx context.Context) (planFlags, error)
opc := &p.optPlanningCtx
opc.reset()

switch stmt.AST.(type) {
switch t := stmt.AST.(type) {
case *tree.AlterIndex, *tree.AlterTable, *tree.AlterSequence,
*tree.Analyze,
*tree.BeginTransaction,
Expand All @@ -64,7 +64,6 @@ func (p *planner) prepareUsingOptimizer(ctx context.Context) (planFlags, error)
*tree.CreateStats,
*tree.Deallocate, *tree.Discard, *tree.DropDatabase, *tree.DropIndex,
*tree.DropTable, *tree.DropView, *tree.DropSequence, *tree.DropType,
*tree.Execute,
*tree.Grant, *tree.GrantRole,
*tree.Prepare,
*tree.ReleaseSavepoint, *tree.RenameColumn, *tree.RenameDatabase,
Expand All @@ -81,6 +80,23 @@ func (p *planner) prepareUsingOptimizer(ctx context.Context) (planFlags, error)
// descriptors and such).
return opc.flags, nil

case *tree.Execute:
// This statement is going to execute a prepared statement. To prepare it,
// we need to set the expected output columns to the output columns of the
// prepared statement that the user is trying to execute.
name := string(t.Name)
prepared, ok := p.preparedStatements.Get(name)
if !ok {
// We're trying to prepare an EXECUTE of a statement that doesn't exist.
// Let's just give up at this point.
// Postgres doesn't fail here, instead it produces an EXECUTE that returns
// no columns. This seems like dubious behavior at best.
return opc.flags, pgerror.Newf(pgcode.UndefinedPreparedStatement,
"no such prepared statement %s", name)
}
stmt.Prepared.Columns = prepared.Columns
return opc.flags, nil

case *tree.ExplainAnalyze:
// This statement returns result columns but does not support placeholders,
// and we don't want to do anything during prepare.
Expand Down
62 changes: 62 additions & 0 deletions pkg/sql/sql_prepare_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,62 @@
// Copyright 2022 The Cockroach Authors.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.

package sql

import (
"context"
"testing"

"github.com/cockroachdb/cockroach/pkg/base"
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/stretchr/testify/require"
)

// Make sure that running a wire-protocol-level PREPARE of a SQL-level PREPARE
// and SQL-level EXECUTE doesn't cause any problems.
func TestPreparePrepareExecute(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)

srv, db, _ := serverutils.StartServer(t, base.TestServerArgs{Insecure: true})
defer srv.Stopper().Stop(context.Background())
defer db.Close()

// Test that preparing an invalid EXECUTE fails at prepare-time.
_, err := db.Prepare("EXECUTE x(3)")
require.Contains(t, err.Error(), "no such prepared statement")

// Test that we can prepare and execute a PREPARE.
s, err := db.Prepare("PREPARE x AS SELECT $1::int")
require.NoError(t, err)

_, err = s.Exec()
require.NoError(t, err)

// Make sure we can't send arguments to the PREPARE even though it has a
// placeholder inside (that placeholder is for the "inner" PREPARE).
_, err = s.Exec(3)
require.Contains(t, err.Error(), "expected 0 arguments, got 1")

// Test that we can prepare and execute the corresponding EXECUTE.
s, err = db.Prepare("EXECUTE x(3)")
require.NoError(t, err)

var output int
err = s.QueryRow().Scan(&output)
require.NoError(t, err)
require.Equal(t, 3, output)

// Make sure we can't send arguments to the prepared EXECUTE.
_, err = s.Exec(3)
require.Contains(t, err.Error(), "expected 0 arguments, got 1")
}
2 changes: 1 addition & 1 deletion pkg/storage/disk_map_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -238,7 +238,7 @@ func TestPebbleMapClose(t *testing.T) {
startKey := diskMap.makeKey([]byte{'a'})
startKeyCopy := make([]byte, len(startKey))
copy(startKeyCopy, startKey)
if err := e.db.Compact(startKeyCopy, diskMap.makeKey([]byte{'z'})); err != nil {
if err := e.db.Compact(startKeyCopy, diskMap.makeKey([]byte{'z'}), false /* parallel */); err != nil {
t.Fatal(err)
}

Expand Down
6 changes: 2 additions & 4 deletions pkg/storage/engine.go
Original file line number Diff line number Diff line change
Expand Up @@ -746,10 +746,8 @@ type Engine interface {
// ApproximateDiskBytes returns an approximation of the on-disk size for the given key span.
ApproximateDiskBytes(from, to roachpb.Key) (uint64, error)
// CompactRange ensures that the specified range of key value pairs is
// optimized for space efficiency. The forceBottommost parameter ensures
// that the key range is compacted all the way to the bottommost level of
// SSTables, which is necessary to pick up changes to bloom filters.
CompactRange(start, end roachpb.Key, forceBottommost bool) error
// optimized for space efficiency.
CompactRange(start, end roachpb.Key) error
// InMem returns true if the receiver is an in-memory engine and false
// otherwise.
//
Expand Down
2 changes: 1 addition & 1 deletion pkg/storage/metamorphic/operations.go
Original file line number Diff line number Diff line change
Expand Up @@ -702,7 +702,7 @@ type compactOp struct {
}

func (c compactOp) run(ctx context.Context) string {
err := c.m.engine.CompactRange(c.key, c.endKey, false)
err := c.m.engine.CompactRange(c.key, c.endKey)
if err != nil {
return fmt.Sprintf("error: %s", err.Error())
}
Expand Down
4 changes: 2 additions & 2 deletions pkg/storage/metamorphic/options.go
Original file line number Diff line number Diff line change
Expand Up @@ -49,7 +49,7 @@ func standardOptions(i int) *pebble.Options {
`,
7: `
[Options]
mem_table_size=1000
mem_table_size=2000
`,
8: `
[Options]
Expand Down Expand Up @@ -129,7 +129,7 @@ func randomOptions() *pebble.Options {
}
opts.MaxManifestFileSize = 1 << rngIntRange(rng, 1, 28)
opts.MaxOpenFiles = int(rngIntRange(rng, 20, 2000))
opts.MemTableSize = 1 << rngIntRange(rng, 10, 28)
opts.MemTableSize = 1 << rngIntRange(rng, 11, 28)
opts.MemTableStopWritesThreshold = int(rngIntRange(rng, 2, 7))
opts.MaxConcurrentCompactions = int(rngIntRange(rng, 1, 4))

Expand Down
6 changes: 3 additions & 3 deletions pkg/storage/pebble.go
Original file line number Diff line number Diff line change
Expand Up @@ -1482,14 +1482,14 @@ func (p *Pebble) ApproximateDiskBytes(from, to roachpb.Key) (uint64, error) {

// Compact implements the Engine interface.
func (p *Pebble) Compact() error {
return p.db.Compact(nil, EncodeMVCCKey(MVCCKeyMax))
return p.db.Compact(nil, EncodeMVCCKey(MVCCKeyMax), false /* parallel */)
}

// CompactRange implements the Engine interface.
func (p *Pebble) CompactRange(start, end roachpb.Key, forceBottommost bool) error {
func (p *Pebble) CompactRange(start, end roachpb.Key) error {
bufStart := EncodeMVCCKey(MVCCKey{start, hlc.Timestamp{}})
bufEnd := EncodeMVCCKey(MVCCKey{end, hlc.Timestamp{}})
return p.db.Compact(bufStart, bufEnd)
return p.db.Compact(bufStart, bufEnd, false /* parallel */)
}

// InMem returns true if the receiver is an in-memory engine and false
Expand Down
2 changes: 1 addition & 1 deletion pkg/storage/pebble_iterator.go
Original file line number Diff line number Diff line change
Expand Up @@ -172,7 +172,7 @@ func (p *pebbleIterator) init(
// We are given an inclusive [MinTimestampHint, MaxTimestampHint]. The
// MVCCWAllTimeIntervalCollector has collected the WallTimes and we need
// [min, max), i.e., exclusive on the upper bound.
p.options.BlockPropertyFilters = []pebble.BlockPropertyFilter{
p.options.PointKeyFilters = []pebble.BlockPropertyFilter{
sstable.NewBlockIntervalFilter(mvccWallTimeIntervalCollector,
uint64(opts.MinTimestampHint.WallTime),
uint64(opts.MaxTimestampHint.WallTime)+1),
Expand Down
2 changes: 1 addition & 1 deletion vendor
Submodule vendor updated 29 files
+46 −61 github.com/cockroachdb/pebble/compaction.go
+11 −6 github.com/cockroachdb/pebble/compaction_picker.go
+66 −29 github.com/cockroachdb/pebble/db.go
+171 −0 github.com/cockroachdb/pebble/external_iterator.go
+6 −2 github.com/cockroachdb/pebble/flush_external.go
+87 −29 github.com/cockroachdb/pebble/ingest.go
+18 −3 github.com/cockroachdb/pebble/internal/base/internal.go
+10 −11 github.com/cockroachdb/pebble/internal/keyspan/bound_iter.go
+122 −189 github.com/cockroachdb/pebble/internal/keyspan/merging_iter.go
+162 −17 github.com/cockroachdb/pebble/internal/manifest/version.go
+14 −2 github.com/cockroachdb/pebble/internal/manifest/version_edit.go
+9 −12 github.com/cockroachdb/pebble/internal/rangekey/defragment.go
+13 −2 github.com/cockroachdb/pebble/internal/rangekey/interleaving_iter.go
+38 −105 github.com/cockroachdb/pebble/internal/rangekey/iter.go
+19 −14 github.com/cockroachdb/pebble/internal/rangekey/rangekey.go
+32 −1 github.com/cockroachdb/pebble/iterator.go
+4 −1 github.com/cockroachdb/pebble/level_iter.go
+32 −5 github.com/cockroachdb/pebble/mem_table.go
+1 −0 github.com/cockroachdb/pebble/open.go
+19 −11 github.com/cockroachdb/pebble/options.go
+51 −35 github.com/cockroachdb/pebble/range_keys.go
+140 −31 github.com/cockroachdb/pebble/sstable/block.go
+27 −18 github.com/cockroachdb/pebble/sstable/reader.go
+7 −3 github.com/cockroachdb/pebble/sstable/suffix_rewriter.go
+21 −12 github.com/cockroachdb/pebble/sstable/write_queue.go
+416 −217 github.com/cockroachdb/pebble/sstable/writer.go
+108 −30 github.com/cockroachdb/pebble/table_cache.go
+1 −1 github.com/cockroachdb/pebble/tool/make_test_find_db.go
+1 −1 modules.txt

0 comments on commit d89e743

Please sign in to comment.